You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@arrow.apache.org by "rtpsw (via GitHub)" <gi...@apache.org> on 2023/02/23 10:21:20 UTC

[GitHub] [arrow] rtpsw opened a new pull request, #34311: GH-32884: [C++] Add ordered aggregation

rtpsw opened a new pull request, #34311:
URL: https://github.com/apache/arrow/pull/34311

   See https://issues.apache.org/jira/browse/ARROW-17642
   
   Replaces #14352


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] icexelloss commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "icexelloss (via GitHub)" <gi...@apache.org>.
icexelloss commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1122268774


##########
cpp/src/arrow/compute/exec/aggregate_node.cc:
##########
@@ -326,46 +446,86 @@ class ScalarAggregateNode : public ExecNode, public TracedNode {
   }
 
  private:
-  Status Finish() {
-    auto scope = TraceFinish();
+  Status ReconstructAggregates() {
+    const auto& input_schema = *inputs()[0]->output_schema();
+    auto exec_ctx = plan()->query_context()->exec_context();
+    for (size_t i = 0; i < kernels_.size(); ++i) {
+      std::vector<TypeHolder> in_types;
+      for (const auto& target : target_fieldsets_[i]) {
+        in_types.emplace_back(input_schema.field(target)->type().get());
+      }
+      states_[i].resize(plan()->query_context()->max_concurrency());

Review Comment:
   Do we need to reset it here those? Those shouldn't change between segments right?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] westonpace commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "westonpace (via GitHub)" <gi...@apache.org>.
westonpace commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1119120080


##########
cpp/src/arrow/compute/kernels/hash_aggregate_test.cc:
##########
@@ -135,22 +141,84 @@ Result<Datum> NaiveGroupBy(std::vector<Datum> arguments, std::vector<Datum> keys
   return Take(struct_arr, sorted_indices);
 }
 
+Result<Datum> MakeGroupByOutput(const std::vector<ExecBatch>& output_batches,
+                                const std::shared_ptr<Schema> output_schema,
+                                size_t num_aggregates, size_t num_keys, bool naive) {
+  ArrayVector out_arrays(num_aggregates + num_keys);
+  for (size_t i = 0; i < out_arrays.size(); ++i) {
+    std::vector<std::shared_ptr<Array>> arrays(output_batches.size());
+    for (size_t j = 0; j < output_batches.size(); ++j) {
+      arrays[j] = output_batches[j].values[i].make_array();
+    }
+    if (arrays.empty()) {
+      ARROW_ASSIGN_OR_RAISE(
+          out_arrays[i],
+          MakeArrayOfNull(output_schema->field(static_cast<int>(i))->type(),
+                          /*length=*/0));
+    } else {
+      ARROW_ASSIGN_OR_RAISE(out_arrays[i], Concatenate(arrays));
+    }
+  }
+
+  ARROW_ASSIGN_OR_RAISE(
+      std::shared_ptr<Array> struct_arr,
+      StructArray::Make(std::move(out_arrays), output_schema->fields()));
+
+  bool need_sort = !naive;
+  for (size_t i = num_aggregates; need_sort && i < out_arrays.size(); i++) {
+    if (output_schema->field(i)->type()->id() == Type::DICTIONARY) {

Review Comment:
   Windows compilers will also emit a warning in this case.  We could suppress that warning but I think it is useful to be explicit and should help remind the author to at least do a basic sanity check that an overflow is not going to happen here.



##########
cpp/src/arrow/compute/exec/aggregate_node.cc:
##########
@@ -169,35 +186,117 @@ void AggregatesToString(std::stringstream* ss, const Schema& input_schema,
   *ss << ']';
 }
 
+template <typename BatchHandler>
+Status HandleSegments(std::unique_ptr<GroupingSegmenter>& segmenter,
+                      const ExecBatch& batch, const std::vector<int>& ids,
+                      const BatchHandler& handle_batch) {
+  int64_t offset = 0;
+  ARROW_ASSIGN_OR_RAISE(auto segment_exec_batch, batch.SelectValues(ids));
+  ExecSpan segment_batch(segment_exec_batch);
+  while (true) {
+    ARROW_ASSIGN_OR_RAISE(auto segment, segmenter->GetNextSegment(segment_batch, offset));
+    if (segment.offset >= segment_batch.length) break;  // condition of no-next-segment
+    ARROW_RETURN_NOT_OK(handle_batch(batch, segment));
+    offset = segment.offset + segment.length;
+  }
+  return Status::OK();
+}
+
+Status GetScalarFields(std::vector<Datum>* values_ptr, const ExecBatch& input_batch,
+                       const std::vector<int>& field_ids) {
+  DCHECK_GT(input_batch.length, 0);
+  std::vector<Datum>& values = *values_ptr;
+  int64_t row = input_batch.length - 1;
+  values.clear();
+  values.resize(field_ids.size());
+  for (size_t i = 0; i < field_ids.size(); i++) {
+    const Datum& value = input_batch.values[field_ids[i]];
+    if (value.is_scalar()) {
+      values[i] = value;
+    } else if (value.is_array()) {
+      ARROW_ASSIGN_OR_RAISE(auto scalar, value.make_array()->GetScalar(row));
+      values[i] = scalar;
+    } else {
+      DCHECK(false);
+    }
+  }
+  return Status::OK();
+}
+
+void PlaceFields(ExecBatch& batch, size_t base, std::vector<Datum>& values) {
+  DCHECK_LE(base + values.size(), batch.values.size());
+  for (size_t i = 0; i < values.size(); i++) {
+    batch.values[base + i] = values[i];
+  }
+}
+
 class ScalarAggregateNode : public ExecNode, public TracedNode {
  public:
   ScalarAggregateNode(ExecPlan* plan, std::vector<ExecNode*> inputs,
                       std::shared_ptr<Schema> output_schema,
+                      std::unique_ptr<GroupingSegmenter> segmenter,
+                      std::vector<int> segment_field_ids,
                       std::vector<std::vector<int>> target_fieldsets,
                       std::vector<Aggregate> aggs,
                       std::vector<const ScalarAggregateKernel*> kernels,
                       std::vector<std::vector<std::unique_ptr<KernelState>>> states)
       : ExecNode(plan, std::move(inputs), {"target"},
                  /*output_schema=*/std::move(output_schema)),
         TracedNode(this),
+        segmenter_(std::move(segmenter)),
+        segment_field_ids_(std::move(segment_field_ids)),
         target_fieldsets_(std::move(target_fieldsets)),
         aggs_(std::move(aggs)),
         kernels_(std::move(kernels)),
-        states_(std::move(states)) {}
+        states_(std::move(states)) {
+    const auto& input_schema = *this->inputs()[0]->output_schema();
+    for (size_t i = 0; i < kernels_.size(); ++i) {
+      std::vector<TypeHolder> in_types;
+      for (const auto& target : target_fieldsets_[i]) {
+        in_types.emplace_back(input_schema.field(target)->type().get());
+      }
+      in_typesets_.push_back(std::move(in_types));
+    }
+  }
 
   static Result<ExecNode*> Make(ExecPlan* plan, std::vector<ExecNode*> inputs,
                                 const ExecNodeOptions& options) {
     RETURN_NOT_OK(ValidateExecNodeInputs(plan, inputs, 1, "ScalarAggregateNode"));
 
     const auto& aggregate_options = checked_cast<const AggregateNodeOptions&>(options);
     auto aggregates = aggregate_options.aggregates;
+    const auto& keys = aggregate_options.keys;
+    const auto& segment_keys = aggregate_options.segment_keys;
+
+    if (keys.size() > 0) {
+      return Status::Invalid("Scalar aggregation with some key");
+    }
+    if (plan->query_context()->exec_context()->executor()->GetCapacity() > 1 &&
+        segment_keys.size() > 0) {
+      return Status::NotImplemented("Segmented aggregation in a multi-threaded plan");
+    }
 
     const auto& input_schema = *inputs[0]->output_schema();
     auto exec_ctx = plan->query_context()->exec_context();
 
+    std::vector<int> segment_field_ids(segment_keys.size());
+    std::vector<TypeHolder> segment_key_types(segment_keys.size());
+    for (size_t i = 0; i < segment_keys.size(); i++) {
+      ARROW_ASSIGN_OR_RAISE(auto match, segment_keys[i].FindOne(input_schema));
+      if (match.indices().size() > 1) {
+        // ARROW-18369: Support nested references as segment ids
+        return Status::Invalid("Nested references cannot be used as segment ids");
+      }
+      segment_field_ids[i] = match[0];
+      segment_key_types[i] = input_schema.field(match[0])->type().get();
+    }
+
+    ARROW_ASSIGN_OR_RAISE(
+        auto segmenter, GroupingSegmenter::Make(std::move(segment_key_types), exec_ctx));

Review Comment:
   We are asserting here that the segment keys will not be null since `nullable_keys` is defaulting to `false`.  At the very least, we should document this in `options.h`
   
   On the other hand, why can't the fact grouping implementation tolerate nulls? It seems we could pick some meaning for `null`. Either:
   
    1. null represents a key of it's own
    2. null means we don't know the value and we assume we are maintaining the previous key
   
   Do you know how groupby is handling null keys today (and by extension the `AnyKeysGroupingSegmenter`)?  I think it is `#1` but I could be mistaken.  If that is the case then it probably wouldn't be too much more complexity to handle nulls in the same fashion in `SimpleKeyGroupingSegmenter`.
   
   We need to document the constraint now.  However, I'd be find deferring any extended implementation for a follow-up.
   
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rtpsw commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "rtpsw (via GitHub)" <gi...@apache.org>.
rtpsw commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1124223628


##########
cpp/src/arrow/compute/exec/aggregate_node.cc:
##########
@@ -283,28 +386,36 @@ class ScalarAggregateNode : public ExecNode, public TracedNode {
     DCHECK_EQ(input, inputs_[0]);
 
     auto thread_index = plan_->query_context()->GetThreadIndex();
-
-    ARROW_RETURN_NOT_OK(DoConsume(ExecSpan(batch), thread_index));
+    auto handler = [this, thread_index](const ExecBatch& full_batch,

Review Comment:
   This factoring of the code separates the handling of a single segment from the iteration over segments. I find it clear but don't mind changing if it would help. However, if the code is to be combined, I wouldn't do it into `HandleSegments` but just inline everything here.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rtpsw commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "rtpsw (via GitHub)" <gi...@apache.org>.
rtpsw commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1116958989


##########
cpp/src/arrow/compute/row/grouper.cc:
##########
@@ -39,12 +43,521 @@
 namespace arrow {
 
 using internal::checked_cast;
+using internal::PrimitiveScalarBase;
 
 namespace compute {
 
 namespace {
 
-struct GrouperImpl : Grouper {
+constexpr uint32_t kNoGroupId = std::numeric_limits<uint32_t>::max();
+
+using group_id_t = std::remove_const<decltype(kNoGroupId)>::type;
+using GroupIdType = CTypeTraits<group_id_t>::ArrowType;
+auto group_id_type = std::make_shared<GroupIdType>();
+
+inline const uint8_t* GetValuesAsBytes(const ArrayData& data, int64_t offset = 0) {
+  DCHECK_GT(data.type->byte_width(), 0);
+  int64_t absolute_byte_offset = (data.offset + offset) * data.type->byte_width();
+  return data.GetValues<uint8_t>(1, absolute_byte_offset);
+}
+
+inline const uint8_t* GetValuesAsBytes(const ArraySpan& data, int64_t offset = 0) {
+  DCHECK_GT(data.type->byte_width(), 0);
+  int64_t absolute_byte_offset = (data.offset + offset) * data.type->byte_width();
+  return data.GetValues<uint8_t>(1, absolute_byte_offset);
+}
+
+template <typename Value>
+Status CheckForGetNextSegment(const std::vector<Value>& values, int64_t length,
+                              int64_t offset, const std::vector<TypeHolder>& key_types) {
+  if (offset < 0 || offset > length) {
+    return Status::Invalid("invalid grouping segmenter offset: ", offset);
+  }
+  if (values.size() != key_types.size()) {
+    return Status::Invalid("expected batch size ", key_types.size(), " but got ",
+                           values.size());
+  }
+  for (size_t i = 0; i < key_types.size(); i++) {
+    const auto& value = values[i];
+    const auto& key_type = key_types[i];
+    if (*value.type() != *key_type.type) {
+      return Status::Invalid("expected batch value ", i, " of type ", *key_type.type,
+                             " but got ", *value.type());
+    }
+  }
+  return Status::OK();
+}
+
+template <typename Batch>
+enable_if_t<std::is_same<Batch, ExecSpan>::value || std::is_same<Batch, ExecBatch>::value,
+            Status>
+CheckForGetNextSegment(const Batch& batch, int64_t offset,
+                       const std::vector<TypeHolder>& key_types) {
+  return CheckForGetNextSegment(batch.values, batch.length, offset, key_types);
+}
+
+struct BaseGroupingSegmenter : public GroupingSegmenter {
+  explicit BaseGroupingSegmenter(const std::vector<TypeHolder>& key_types)
+      : key_types_(key_types) {}
+
+  const std::vector<TypeHolder>& key_types() const override { return key_types_; }
+
+  std::vector<TypeHolder> key_types_;
+};
+
+GroupingSegment MakeSegment(int64_t batch_length, int64_t offset, int64_t length,
+                            bool extends) {
+  return GroupingSegment{offset, length, offset + length >= batch_length, extends};
+}
+
+int64_t GetMatchLength(const uint8_t* match_bytes, int64_t match_width,
+                       const uint8_t* array_bytes, int64_t offset, int64_t length) {
+  int64_t cursor, byte_cursor;
+  for (cursor = offset, byte_cursor = match_width * cursor; cursor < length;
+       cursor++, byte_cursor += match_width) {
+    if (memcmp(match_bytes, array_bytes + byte_cursor,
+               static_cast<size_t>(match_width)) != 0) {
+      break;
+    }
+  }
+  return std::min(cursor, length - offset);
+}
+
+using ExtendFunc = std::function<bool(const void*)>;
+constexpr bool kDefaultExtends = true;
+constexpr bool kEmptyExtends = true;
+
+Result<GroupingSegment> GetNextSegmentChunked(
+    const std::shared_ptr<ChunkedArray>& chunked_array, int64_t offset,
+    ExtendFunc extend) {
+  if (offset >= chunked_array->length()) {
+    return MakeSegment(chunked_array->length(), chunked_array->length(), 0,
+                       kEmptyExtends);
+  }
+  int64_t remaining_offset = offset;
+  const auto& arrays = chunked_array->chunks();
+  for (size_t i = 0; remaining_offset >= 0 && i < arrays.size(); i++) {
+    // look up chunk containing offset
+    int64_t array_length = arrays[i]->length();
+    if (remaining_offset < array_length) {
+      // found - switch to matching
+      int64_t match_width = arrays[i]->type()->byte_width();
+      const uint8_t* match_bytes = GetValuesAsBytes(*arrays[i]->data(), remaining_offset);
+      int64_t total_match_length = 0;
+      for (; i < arrays.size(); i++) {
+        int64_t array_length = arrays[i]->length();
+        if (array_length <= 0) continue;
+        const uint8_t* array_bytes = GetValuesAsBytes(*arrays[i]->data());
+        int64_t match_length = GetMatchLength(match_bytes, match_width, array_bytes,
+                                              remaining_offset, array_length);
+        total_match_length += match_length;
+        remaining_offset = 0;
+        if (match_length < array_length - remaining_offset) break;
+      }
+      bool extends = extend(match_bytes);
+      return MakeSegment(chunked_array->length(), offset, total_match_length, extends);
+    }
+    remaining_offset -= array_length;
+  }
+  return Status::Invalid("segmenting invalid chunked array value");
+}
+
+struct NoKeysGroupingSegmenter : public BaseGroupingSegmenter {
+  static std::unique_ptr<GroupingSegmenter> Make() {
+    return std::make_unique<NoKeysGroupingSegmenter>();
+  }
+
+  NoKeysGroupingSegmenter() : BaseGroupingSegmenter({}) {}
+
+  Status Reset() override { return Status::OK(); }
+
+  template <typename Batch>
+  Result<GroupingSegment> GetNextSegmentImpl(const Batch& batch, int64_t offset) {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {}));
+    return MakeSegment(batch.length, offset, batch.length - offset, kDefaultExtends);
+  }
+
+  Result<GroupingSegment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    return GetNextSegmentImpl(batch, offset);
+  }
+
+  Result<GroupingSegment> GetNextSegment(const ExecBatch& batch,
+                                         int64_t offset) override {
+    return GetNextSegmentImpl(batch, offset);
+  }
+};
+
+struct SimpleKeyGroupingSegmenter : public BaseGroupingSegmenter {
+  static Result<std::unique_ptr<GroupingSegmenter>> Make(TypeHolder key_type) {
+    return std::make_unique<SimpleKeyGroupingSegmenter>(key_type);
+  }
+
+  explicit SimpleKeyGroupingSegmenter(TypeHolder key_type)
+      : BaseGroupingSegmenter({key_type}), key_type_(key_types_[0]), save_key_data_() {}
+
+  Status CheckType(const DataType& type) {
+    if (!is_fixed_width(type)) {
+      return Status::Invalid("SimpleKeyGroupingSegmenter does not support type ", type);
+    }
+    return Status::OK();
+  }
+
+  Status Reset() override {
+    save_key_data_.resize(0);
+    return Status::OK();
+  }
+
+  bool Extend(const void* data) {
+    size_t byte_width = static_cast<size_t>(key_type_.type->byte_width());
+    bool extends = save_key_data_.size() != byte_width
+                       ? kDefaultExtends
+                       : 0 == memcmp(save_key_data_.data(), data, byte_width);
+    save_key_data_.resize(byte_width);
+    memcpy(save_key_data_.data(), data, byte_width);
+    return extends;
+  }
+
+  Result<GroupingSegment> GetNextSegment(const Scalar& scalar, int64_t offset,
+                                         int64_t length) {
+    ARROW_RETURN_NOT_OK(CheckType(*scalar.type));
+    if (!scalar.is_valid) {
+      return Status::Invalid("segmenting an invalid scalar");
+    }
+    auto data = checked_cast<const PrimitiveScalarBase&>(scalar).data();
+    bool extends = length > 0 ? Extend(data) : kEmptyExtends;
+    return MakeSegment(length, 0, length, extends);
+  }
+
+  Result<GroupingSegment> GetNextSegment(const DataType& array_type,
+                                         const uint8_t* array_bytes, int64_t offset,
+                                         int64_t length) {
+    RETURN_NOT_OK(CheckType(array_type));
+    int64_t byte_width = array_type.byte_width();
+    int64_t match_length = GetMatchLength(array_bytes + offset * byte_width, byte_width,
+                                          array_bytes, offset, length);
+    bool extends = length > 0 ? Extend(array_bytes + offset * byte_width) : kEmptyExtends;
+    return MakeSegment(length, offset, match_length, extends);
+  }
+
+  Result<GroupingSegment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {key_type_}));
+    if (offset == batch.length) {
+      return MakeSegment(batch.length, offset, 0, kEmptyExtends);
+    }
+    const auto& value = batch.values[0];
+    if (value.is_scalar()) {
+      return GetNextSegment(*value.scalar, offset, batch.length);
+    }
+    ARROW_DCHECK(value.is_array());
+    const auto& array = value.array;
+    if (array.GetNullCount() > 0) {
+      return Status::NotImplemented("segmenting a nullable array");
+    }
+    return GetNextSegment(*array.type, GetValuesAsBytes(array), offset, batch.length);
+  }
+
+  Result<GroupingSegment> GetNextSegment(const ExecBatch& batch,
+                                         int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {key_type_}));
+    if (offset == batch.length) {
+      return MakeSegment(batch.length, offset, 0, kEmptyExtends);
+    }
+    const auto& value = batch.values[0];
+    if (value.is_scalar()) {
+      return GetNextSegment(*value.scalar(), offset, batch.length);
+    }
+    if (value.is_array()) {
+      auto array = value.array();
+      if (array->GetNullCount() > 0) {
+        return Status::NotImplemented("segmenting a nullable array");
+      }
+      return GetNextSegment(*array->type, GetValuesAsBytes(*array), offset, batch.length);
+    }
+    if (value.is_chunked_array()) {
+      auto array = value.chunked_array();
+      if (array->null_count() > 0) {
+        return Status::NotImplemented("segmenting a nullable array");
+      }
+      return GetNextSegmentChunked(array, offset, bound_extend_);
+    }
+    return Status::Invalid("segmenting unsupported value kind ", value.kind());
+  }
+
+ private:
+  TypeHolder key_type_;
+  std::vector<uint8_t> save_key_data_;
+  ExtendFunc bound_extend_ = [this](const void* data) { return Extend(data); };
+};
+
+struct AnyKeysGroupingSegmenter : public BaseGroupingSegmenter {
+  static Result<std::unique_ptr<GroupingSegmenter>> Make(
+      const std::vector<TypeHolder>& key_types, ExecContext* ctx) {
+    ARROW_RETURN_NOT_OK(Grouper::Make(key_types, ctx));  // check types
+    return std::make_unique<AnyKeysGroupingSegmenter>(key_types, ctx);
+  }
+
+  AnyKeysGroupingSegmenter(const std::vector<TypeHolder>& key_types, ExecContext* ctx)
+      : BaseGroupingSegmenter(key_types),
+        ctx_(ctx),
+        grouper_(nullptr),
+        save_group_id_(kNoGroupId) {}
+
+  Status Reset() override {
+    grouper_ = nullptr;
+    save_group_id_ = kNoGroupId;
+    return Status::OK();
+  }
+
+  bool Extend(const void* data) {
+    auto group_id = *static_cast<const group_id_t*>(data);
+    bool extends =
+        save_group_id_ == kNoGroupId ? kDefaultExtends : save_group_id_ == group_id;
+    save_group_id_ = group_id;
+    return extends;
+  }
+
+  template <typename Batch>
+  Result<group_id_t> MapGroupIdAt(const Batch& batch, int64_t offset) {
+    if (offset < 0 || offset >= batch.length) {
+      return Status::Invalid("requesting group id out of bounds");
+    }
+    if (!grouper_) return kNoGroupId;
+    ARROW_ASSIGN_OR_RAISE(auto datum, grouper_->Consume(batch, offset,
+                                                        /*consume_length=*/1));
+    if (!(datum.is_array() || datum.is_chunked_array())) {
+      return Status::Invalid("accessing unsupported datum kind ", datum.kind());
+    }
+    const std::shared_ptr<ArrayData>& data =
+        datum.is_array() ? datum.array() : datum.chunked_array()->chunk(0)->data();
+    ARROW_DCHECK(data->GetNullCount() == 0);
+    DCHECK_EQ(data->type->id(), GroupIdType::type_id);
+    DCHECK_EQ(1, data->length);
+    const group_id_t* values = data->GetValues<group_id_t>(1);
+    return values[0];
+  }
+
+  template <typename Batch>
+  Result<GroupingSegment> GetNextSegmentImpl(const Batch& batch, int64_t offset) {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, key_types_));
+    if (offset == batch.length) {
+      return MakeSegment(batch.length, offset, 0, kEmptyExtends);
+    }
+    // ARROW-18311: make Grouper support Reset()
+    // so it can be cached instead of recreated below
+    //
+    // the group id must be computed prior to resetting the grouper, since it is compared
+    // to save_group_id_, and after resetting the grouper produces incomparable group ids
+    ARROW_ASSIGN_OR_RAISE(auto group_id, MapGroupIdAt(batch, offset));
+    ExtendFunc bound_extend = [this, group_id](const void* data) {
+      bool extends = Extend(&group_id);
+      save_group_id_ = *static_cast<const group_id_t*>(data);
+      return extends;
+    };
+    ARROW_ASSIGN_OR_RAISE(grouper_, Grouper::Make(key_types_, ctx_));  // TODO: reset it

Review Comment:
   Yes, and there might also be some members that would not need to be recomputed (e.g., the encoders in `GrouperImpl`).



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rtpsw commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "rtpsw (via GitHub)" <gi...@apache.org>.
rtpsw commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1116963027


##########
cpp/src/arrow/compute/exec/aggregate_node.cc:
##########
@@ -169,17 +206,63 @@ void AggregatesToString(std::stringstream* ss, const Schema& input_schema,
   *ss << ']';
 }
 
+template <typename BatchHandler>
+Status HandleSegments(std::unique_ptr<GroupingSegmenter>& segmenter,
+                      const ExecBatch& batch, const std::vector<int>& ids,
+                      const BatchHandler& handle_batch) {
+  int64_t offset = 0;
+  ARROW_ASSIGN_OR_RAISE(auto segment_batch, batch.SelectValues(ids));
+  while (true) {
+    ARROW_ASSIGN_OR_RAISE(auto segment, segmenter->GetNextSegment(segment_batch, offset));
+    if (segment.offset >= segment_batch.length) break;

Review Comment:
   This is how `GetNextSegment` indicates that it didn't find a next segment. I didn't think adding a flag to indicate this was worth the while.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rtpsw commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "rtpsw (via GitHub)" <gi...@apache.org>.
rtpsw commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1116877818


##########
cpp/src/arrow/compute/exec/options.h:
##########
@@ -199,21 +199,32 @@ class ARROW_EXPORT ProjectNodeOptions : public ExecNodeOptions {
   std::vector<std::string> names;
 };
 
-/// \brief Make a node which aggregates input batches, optionally grouped by keys.
+/// \brief Make a node which aggregates input batches, optionally grouped by keys and
+/// optionally segmented by segment-keys. Both keys and segment-keys determine the group.
+/// However segment-keys are also used for determining grouping segments, which should be
+/// large, and allow streaming a partial aggregation result after processing each segment.
+/// One common use-case for segment-keys is ordered aggregation, in which the segment-key
+/// attribute specifies a column with non-decreasing values or a lexigographically-ordered
+/// set of such columns.
 ///
 /// If the keys attribute is a non-empty vector, then each aggregate in `aggregates` is
 /// expected to be a HashAggregate function. If the keys attribute is an empty vector,
 /// then each aggregate is assumed to be a ScalarAggregate function.
 class ARROW_EXPORT AggregateNodeOptions : public ExecNodeOptions {
  public:
   explicit AggregateNodeOptions(std::vector<Aggregate> aggregates,
-                                std::vector<FieldRef> keys = {})
-      : aggregates(std::move(aggregates)), keys(std::move(keys)) {}
+                                std::vector<FieldRef> keys = {},
+                                std::vector<FieldRef> segment_keys = {})
+      : aggregates(std::move(aggregates)),
+        keys(std::move(keys)),
+        segment_keys(std::move(segment_keys)) {}
 
   // aggregations which will be applied to the targetted fields
   std::vector<Aggregate> aggregates;
   // keys by which aggregations will be grouped
   std::vector<FieldRef> keys;
+  // keys by which aggregations will be segmented
+  std::vector<FieldRef> segment_keys;

Review Comment:
   I think referencing the same field should just be disallowed (this is not done) in the PR currently); doing so is not necessary because the output of segmented aggregation with keys `K` and segment-keys `S` is, by definition, equal to that with keys `K set-minus S` and segment-keys `S`.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] github-actions[bot] commented on pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "github-actions[bot] (via GitHub)" <gi...@apache.org>.
github-actions[bot] commented on PR #34311:
URL: https://github.com/apache/arrow/pull/34311#issuecomment-1441511536

   :warning: GitHub issue #32884 **has been automatically assigned in GitHub** to PR creator.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] icexelloss commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "icexelloss (via GitHub)" <gi...@apache.org>.
icexelloss commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1116054419


##########
cpp/src/arrow/compute/row/grouper.h:
##########
@@ -30,6 +30,49 @@
 namespace arrow {
 namespace compute {
 
+/// \brief A segment of contiguous rows for grouping
+struct ARROW_EXPORT GroupingSegment {
+  int64_t offset;
+  int64_t length;
+  bool is_open;
+  bool extends;
+};
+
+inline bool operator==(const GroupingSegment& segment1, const GroupingSegment& segment2) {
+  return segment1.offset == segment2.offset && segment1.length == segment2.length &&
+         segment1.is_open == segment2.is_open && segment1.extends == segment2.extends;
+}
+inline bool operator!=(const GroupingSegment& segment1, const GroupingSegment& segment2) {
+  return !(segment1 == segment2);
+}
+
+/// \brief Computes grouping segments for a batch. Each segment covers rows with identical
+/// values in the batch. The values in the batch are often selected as keys from a larger
+/// batch.
+class ARROW_EXPORT GroupingSegmenter {
+ public:
+  virtual ~GroupingSegmenter() = default;
+
+  /// \brief Construct a GroupingSegmenter which receives the specified key types
+  static Result<std::unique_ptr<GroupingSegmenter>> Make(
+      const std::vector<TypeHolder>& key_types, bool nullable_keys = false,
+      ExecContext* ctx = default_exec_context());

Review Comment:
   I feel defaults like this can lead to bugs, i.e., the caller has a ctx that is not default_exec_context() and forget to pass that in.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rtpsw commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "rtpsw (via GitHub)" <gi...@apache.org>.
rtpsw commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1116168128


##########
cpp/src/arrow/compute/exec/aggregate_node.cc:
##########
@@ -43,7 +44,45 @@ namespace compute {
 
 namespace {
 
-namespace {
+/// \brief A gated shared mutex is similar to a shared mutex, in that it allows either
+/// multiple shared readers or a unique writer access to the mutex, except that a waiting
+/// writer gates future readers by preventing them from reacquiring shared access until it
+/// has acquired and released the mutex. This is useful for ensuring a writer is never
+/// starved by readers.
+struct GatedSharedMutex {
+  std::mutex gate;
+  std::shared_mutex mutex;
+};
+
+/// \brief Acquires unique access to a gatex mutex. This is useful for a unique writer.
+class GatedUniqueLock {
+ public:
+  // acquires the gate first, to ensure future readers will wait for its release
+  explicit GatedUniqueLock(GatedSharedMutex& gated_shared_mutex)
+      : lock_gate_(gated_shared_mutex.gate), lock_mutex_(gated_shared_mutex.mutex) {}
+
+ private:
+  std::unique_lock<std::mutex> lock_gate_;
+  std::unique_lock<std::shared_mutex> lock_mutex_;
+};
+
+/// \brief Acquires shared access to a gatex mutex. This is useful for a shared reader.
+class GatedSharedLock {

Review Comment:
   It's from [an earlier phase of working on this PR](https://github.com/apache/arrow/pull/14352#issuecomment-1279750410) and is only needed for multi-threaded segmented aggregation, which we decided not to support for now, so it can (and will) be removed.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] icexelloss commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "icexelloss (via GitHub)" <gi...@apache.org>.
icexelloss commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1117209238


##########
cpp/src/arrow/compute/row/grouper.cc:
##########
@@ -39,12 +43,521 @@
 namespace arrow {
 
 using internal::checked_cast;
+using internal::PrimitiveScalarBase;
 
 namespace compute {
 
 namespace {
 
-struct GrouperImpl : Grouper {
+constexpr uint32_t kNoGroupId = std::numeric_limits<uint32_t>::max();
+
+using group_id_t = std::remove_const<decltype(kNoGroupId)>::type;
+using GroupIdType = CTypeTraits<group_id_t>::ArrowType;
+auto group_id_type = std::make_shared<GroupIdType>();
+
+inline const uint8_t* GetValuesAsBytes(const ArrayData& data, int64_t offset = 0) {
+  DCHECK_GT(data.type->byte_width(), 0);
+  int64_t absolute_byte_offset = (data.offset + offset) * data.type->byte_width();
+  return data.GetValues<uint8_t>(1, absolute_byte_offset);
+}
+
+inline const uint8_t* GetValuesAsBytes(const ArraySpan& data, int64_t offset = 0) {
+  DCHECK_GT(data.type->byte_width(), 0);
+  int64_t absolute_byte_offset = (data.offset + offset) * data.type->byte_width();
+  return data.GetValues<uint8_t>(1, absolute_byte_offset);
+}
+
+template <typename Value>
+Status CheckForGetNextSegment(const std::vector<Value>& values, int64_t length,
+                              int64_t offset, const std::vector<TypeHolder>& key_types) {
+  if (offset < 0 || offset > length) {
+    return Status::Invalid("invalid grouping segmenter offset: ", offset);
+  }
+  if (values.size() != key_types.size()) {
+    return Status::Invalid("expected batch size ", key_types.size(), " but got ",
+                           values.size());
+  }
+  for (size_t i = 0; i < key_types.size(); i++) {
+    const auto& value = values[i];
+    const auto& key_type = key_types[i];
+    if (*value.type() != *key_type.type) {
+      return Status::Invalid("expected batch value ", i, " of type ", *key_type.type,
+                             " but got ", *value.type());
+    }
+  }
+  return Status::OK();
+}
+
+template <typename Batch>
+enable_if_t<std::is_same<Batch, ExecSpan>::value || std::is_same<Batch, ExecBatch>::value,
+            Status>
+CheckForGetNextSegment(const Batch& batch, int64_t offset,
+                       const std::vector<TypeHolder>& key_types) {
+  return CheckForGetNextSegment(batch.values, batch.length, offset, key_types);
+}
+
+struct BaseGroupingSegmenter : public GroupingSegmenter {
+  explicit BaseGroupingSegmenter(const std::vector<TypeHolder>& key_types)
+      : key_types_(key_types) {}
+
+  const std::vector<TypeHolder>& key_types() const override { return key_types_; }
+
+  std::vector<TypeHolder> key_types_;
+};
+
+GroupingSegment MakeSegment(int64_t batch_length, int64_t offset, int64_t length,
+                            bool extends) {
+  return GroupingSegment{offset, length, offset + length >= batch_length, extends};
+}
+
+int64_t GetMatchLength(const uint8_t* match_bytes, int64_t match_width,

Review Comment:
   > A, A, B, B, A, A, A, A, A, A, A and the match is for A
   
   Doesn't the algorithm requires segment keys to be "segmented", i.e., the above is not a valid input?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] icexelloss commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "icexelloss (via GitHub)" <gi...@apache.org>.
icexelloss commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1117235566


##########
cpp/src/arrow/compute/exec/options.h:
##########
@@ -199,21 +199,32 @@ class ARROW_EXPORT ProjectNodeOptions : public ExecNodeOptions {
   std::vector<std::string> names;
 };
 
-/// \brief Make a node which aggregates input batches, optionally grouped by keys.
+/// \brief Make a node which aggregates input batches, optionally grouped by keys and
+/// optionally segmented by segment-keys. Both keys and segment-keys determine the group.
+/// However segment-keys are also used for determining grouping segments, which should be
+/// large, and allow streaming a partial aggregation result after processing each segment.
+/// One common use-case for segment-keys is ordered aggregation, in which the segment-key
+/// attribute specifies a column with non-decreasing values or a lexigographically-ordered
+/// set of such columns.
 ///
 /// If the keys attribute is a non-empty vector, then each aggregate in `aggregates` is
 /// expected to be a HashAggregate function. If the keys attribute is an empty vector,
 /// then each aggregate is assumed to be a ScalarAggregate function.
 class ARROW_EXPORT AggregateNodeOptions : public ExecNodeOptions {
  public:
   explicit AggregateNodeOptions(std::vector<Aggregate> aggregates,
-                                std::vector<FieldRef> keys = {})
-      : aggregates(std::move(aggregates)), keys(std::move(keys)) {}
+                                std::vector<FieldRef> keys = {},
+                                std::vector<FieldRef> segment_keys = {})
+      : aggregates(std::move(aggregates)),
+        keys(std::move(keys)),
+        segment_keys(std::move(segment_keys)) {}
 
   // aggregations which will be applied to the targetted fields
   std::vector<Aggregate> aggregates;
   // keys by which aggregations will be grouped
   std::vector<FieldRef> keys;
+  // keys by which aggregations will be segmented
+  std::vector<FieldRef> segment_keys;

Review Comment:
   When I see the word "keys" I would natural think it was the grouping key, e.g., `K`, instead of "keys that is not segmented key", i.e, `K - S`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] icexelloss commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "icexelloss (via GitHub)" <gi...@apache.org>.
icexelloss commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1117296661


##########
cpp/src/arrow/compute/kernels/hash_aggregate_test.cc:
##########
@@ -174,81 +242,117 @@ Result<Datum> RunGroupBy(const BatchesWithSchema& input,
   ARROW_ASSIGN_OR_RAISE(std::vector<ExecBatch> output_batches,
                         start_and_collect.MoveResult());
 
-  ArrayVector out_arrays(aggregates.size() + key_names.size());
   const auto& output_schema = plan->nodes()[0]->output()->output_schema();
+  if (!segmented) {
+    return MakeGroupByOutput(output_batches, output_schema, aggregates.size(),
+                             key_names.size(), naive);
+  }
+
+  std::vector<ArrayVector> out_arrays(aggregates.size() + key_names.size() +
+                                      segment_key_names.size());
   for (size_t i = 0; i < out_arrays.size(); ++i) {
     std::vector<std::shared_ptr<Array>> arrays(output_batches.size());
     for (size_t j = 0; j < output_batches.size(); ++j) {
-      arrays[j] = output_batches[j].values[i].make_array();
+      auto& value = output_batches[j].values[i];
+      if (value.is_scalar()) {
+        ARROW_ASSIGN_OR_RAISE(
+            arrays[j], MakeArrayFromScalar(*value.scalar(), output_batches[j].length));
+      } else if (value.is_array()) {
+        arrays[j] = value.make_array();
+      } else {
+        return Status::Invalid("GroupByUsingExecPlan unsupported value kind ",
+                               ToString(value.kind()));
+      }
     }
     if (arrays.empty()) {
+      arrays.resize(1);
       ARROW_ASSIGN_OR_RAISE(
-          out_arrays[i],
-          MakeArrayOfNull(output_schema->field(static_cast<int>(i))->type(),
-                          /*length=*/0));
-    } else {
-      ARROW_ASSIGN_OR_RAISE(out_arrays[i], Concatenate(arrays));
+          arrays[0], MakeArrayOfNull(output_schema->field(static_cast<int>(i))->type(),
+                                     /*length=*/0));
     }
+    out_arrays[i] = {std::move(arrays)};
   }
 
-  // The exec plan may reorder the output rows.  The tests are all setup to expect ouptut
-  // in ascending order of keys.  So we need to sort the result by the key columns.  To do
-  // that we create a table using the key columns, calculate the sort indices from that
-  // table (sorting on all fields) and then use those indices to calculate our result.
-  std::vector<std::shared_ptr<Field>> key_fields;
-  std::vector<std::shared_ptr<Array>> key_columns;
-  std::vector<SortKey> sort_keys;
-  for (std::size_t i = 0; i < key_names.size(); i++) {
-    const std::shared_ptr<Array>& arr = out_arrays[i + aggregates.size()];
-    if (arr->type_id() == Type::DICTIONARY) {
-      // Can't sort dictionary columns so need to decode
-      auto dict_arr = checked_pointer_cast<DictionaryArray>(arr);
-      ARROW_ASSIGN_OR_RAISE(auto decoded_arr,
-                            Take(*dict_arr->dictionary(), *dict_arr->indices()));
-      key_columns.push_back(decoded_arr);
-      key_fields.push_back(
-          field("name_does_not_matter", dict_arr->dict_type()->value_type()));
-    } else {
-      key_columns.push_back(arr);
-      key_fields.push_back(field("name_does_not_matter", arr->type()));
+  if (segmented && segment_key_names.size() > 0) {
+    ArrayVector struct_arrays;
+    struct_arrays.reserve(output_batches.size());
+    for (size_t j = 0; j < output_batches.size(); ++j) {
+      ArrayVector struct_fields;
+      struct_fields.reserve(out_arrays.size());
+      for (auto out_array : out_arrays) {
+        struct_fields.push_back(out_array[j]);
+      }
+      ARROW_ASSIGN_OR_RAISE(auto struct_array,
+                            StructArray::Make(struct_fields, output_schema->fields()));
+      struct_arrays.push_back(struct_array);
     }
-    sort_keys.emplace_back(static_cast<int>(i));
+    return ChunkedArray::Make(struct_arrays);
+  } else {
+    ArrayVector struct_fields(out_arrays.size());
+    for (size_t i = 0; i < out_arrays.size(); ++i) {
+      ARROW_ASSIGN_OR_RAISE(struct_fields[i], Concatenate(out_arrays[i]));
+    }
+    return StructArray::Make(std::move(struct_fields), output_schema->fields());
   }
-  std::shared_ptr<Schema> key_schema = schema(std::move(key_fields));
-  std::shared_ptr<Table> key_table = Table::Make(std::move(key_schema), key_columns);
-  SortOptions sort_options(std::move(sort_keys));
-  ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Array> sort_indices,
-                        SortIndices(key_table, sort_options));
+}
 
-  ARROW_ASSIGN_OR_RAISE(
-      std::shared_ptr<Array> struct_arr,
-      StructArray::Make(std::move(out_arrays), output_schema->fields()));
+Result<Datum> RunGroupBy(const BatchesWithSchema& input,
+                         const std::vector<std::string>& key_names,
+                         const std::vector<std::string>& segment_key_names,
+                         const std::vector<Aggregate>& aggregates, bool use_threads,
+                         bool segmented = false, bool naive = false) {
+  if (segment_key_names.size() > 0) {
+    ARROW_ASSIGN_OR_RAISE(auto thread_pool, arrow::internal::ThreadPool::Make(1));
+    ExecContext seq_ctx(default_memory_pool(), thread_pool.get());
+    return RunGroupBy(input, key_names, segment_key_names, aggregates, &seq_ctx,
+                      use_threads, segmented, naive);
+  } else {
+    return RunGroupBy(input, key_names, segment_key_names, aggregates,
+                      threaded_exec_context(), use_threads, segmented, naive);
+  }
+}
 
-  return Take(struct_arr, sort_indices);
+Result<Datum> RunGroupBy(const BatchesWithSchema& input,
+                         const std::vector<std::string>& key_names,
+                         const std::vector<Aggregate>& aggregates, bool use_threads,
+                         bool segmented = false, bool naive = false) {
+  return RunGroupBy(input, key_names, {}, aggregates, use_threads, segmented);
 }
 
 /// Simpler overload where you can give the columns as datums
 Result<Datum> RunGroupBy(const std::vector<Datum>& arguments,
                          const std::vector<Datum>& keys,
-                         const std::vector<Aggregate>& aggregates,
-                         bool use_threads = false) {
+                         const std::vector<Datum>& segment_keys,
+                         const std::vector<Aggregate>& aggregates, bool use_threads,
+                         bool segmented = false, bool naive = false) {

Review Comment:
   Agree that `segmented` seems redundant 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] icexelloss commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "icexelloss (via GitHub)" <gi...@apache.org>.
icexelloss commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1117303729


##########
cpp/src/arrow/compute/row/grouper.h:
##########
@@ -30,6 +30,49 @@
 namespace arrow {
 namespace compute {
 
+/// \brief A segment of contiguous rows for grouping
+struct ARROW_EXPORT GroupingSegment {
+  int64_t offset;
+  int64_t length;
+  bool is_open;
+  bool extends;
+};
+
+inline bool operator==(const GroupingSegment& segment1, const GroupingSegment& segment2) {
+  return segment1.offset == segment2.offset && segment1.length == segment2.length &&
+         segment1.is_open == segment2.is_open && segment1.extends == segment2.extends;
+}
+inline bool operator!=(const GroupingSegment& segment1, const GroupingSegment& segment2) {
+  return !(segment1 == segment2);
+}
+
+/// \brief Computes grouping segments for a batch. Each segment covers rows with identical
+/// values in the batch. The values in the batch are often selected as keys from a larger
+/// batch.
+class ARROW_EXPORT GroupingSegmenter {
+ public:
+  virtual ~GroupingSegmenter() = default;
+
+  /// \brief Construct a GroupingSegmenter which receives the specified key types
+  static Result<std::unique_ptr<GroupingSegmenter>> Make(
+      const std::vector<TypeHolder>& key_types, bool nullable_keys = false,
+      ExecContext* ctx = default_exec_context());
+
+  /// \brief Return the key types of this segmenter
+  virtual const std::vector<TypeHolder>& key_types() const = 0;

Review Comment:
   For my knowledge, what is the difference to use `TypeHolder` here vs using `shared_ptr<DataType>`?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] icexelloss commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "icexelloss (via GitHub)" <gi...@apache.org>.
icexelloss commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1117295168


##########
cpp/src/arrow/compute/kernels/hash_aggregate_test.cc:
##########
@@ -135,22 +141,84 @@ Result<Datum> NaiveGroupBy(std::vector<Datum> arguments, std::vector<Datum> keys
   return Take(struct_arr, sorted_indices);
 }
 
+Result<Datum> MakeGroupByOutput(const std::vector<ExecBatch>& output_batches,
+                                const std::shared_ptr<Schema> output_schema,
+                                size_t num_aggregates, size_t num_keys, bool naive) {
+  ArrayVector out_arrays(num_aggregates + num_keys);
+  for (size_t i = 0; i < out_arrays.size(); ++i) {
+    std::vector<std::shared_ptr<Array>> arrays(output_batches.size());
+    for (size_t j = 0; j < output_batches.size(); ++j) {
+      arrays[j] = output_batches[j].values[i].make_array();
+    }
+    if (arrays.empty()) {
+      ARROW_ASSIGN_OR_RAISE(
+          out_arrays[i],
+          MakeArrayOfNull(output_schema->field(static_cast<int>(i))->type(),
+                          /*length=*/0));
+    } else {
+      ARROW_ASSIGN_OR_RAISE(out_arrays[i], Concatenate(arrays));
+    }
+  }
+
+  ARROW_ASSIGN_OR_RAISE(
+      std::shared_ptr<Array> struct_arr,
+      StructArray::Make(std::move(out_arrays), output_schema->fields()));
+
+  bool need_sort = !naive;
+  for (size_t i = num_aggregates; need_sort && i < out_arrays.size(); i++) {
+    if (output_schema->field(i)->type()->id() == Type::DICTIONARY) {

Review Comment:
   For my knowledge - why this?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] icexelloss commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "icexelloss (via GitHub)" <gi...@apache.org>.
icexelloss commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1117309940


##########
cpp/src/arrow/compute/exec/aggregate_node.cc:
##########
@@ -326,46 +446,86 @@ class ScalarAggregateNode : public ExecNode, public TracedNode {
   }
 
  private:
-  Status Finish() {
-    auto scope = TraceFinish();
+  Status ReconstructAggregates() {
+    const auto& input_schema = *inputs()[0]->output_schema();
+    auto exec_ctx = plan()->query_context()->exec_context();
+    for (size_t i = 0; i < kernels_.size(); ++i) {
+      std::vector<TypeHolder> in_types;
+      for (const auto& target : target_fieldsets_[i]) {
+        in_types.emplace_back(input_schema.field(target)->type().get());
+      }
+      states_[i].resize(plan()->query_context()->max_concurrency());
+      KernelContext kernel_ctx{exec_ctx};
+      RETURN_NOT_OK(Kernel::InitAll(
+          &kernel_ctx, KernelInitArgs{kernels_[i], in_types, aggs_[i].options.get()},
+          &states_[i]));
+    }
+    return Status::OK();
+  }
+
+  Status OutputResult(bool is_last = false, bool traced = false) {
+    if (is_last && !traced) {
+      auto scope = TraceFinish();
+      return OutputResult(is_last, /*traced=*/true);
+    }
+    GatedUniqueLock lock(gated_shared_mutex_);
     ExecBatch batch{{}, 1};
-    batch.values.resize(kernels_.size());
+    batch.values.resize(kernels_.size() + segment_field_ids_.size());
 
     for (size_t i = 0; i < kernels_.size(); ++i) {
       util::tracing::Span span;
       START_COMPUTE_SPAN(span, aggs_[i].function,
                          {{"function.name", aggs_[i].function},
                           {"function.options",
                            aggs_[i].options ? aggs_[i].options->ToString() : "<NULLPTR>"},
-                          {"function.kind", std::string(kind_name()) + "::Finalize"}});
+                          {"function.kind", std::string(kind_name()) + "::Output"}});
       KernelContext ctx{plan()->query_context()->exec_context()};
       ARROW_ASSIGN_OR_RAISE(auto merged, ScalarAggregateKernel::MergeAll(
                                              kernels_[i], &ctx, std::move(states_[i])));
       RETURN_NOT_OK(kernels_[i]->finalize(&ctx, &batch.values[i]));
     }
+    PlaceFields(batch, kernels_.size(), segmenter_values_);
 
-    return output_->InputReceived(this, std::move(batch));
+    ARROW_RETURN_NOT_OK(output_->InputReceived(this, std::move(batch)));
+    total_output_batches_++;

Review Comment:
   Do we output one batch per segment?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] icexelloss commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "icexelloss (via GitHub)" <gi...@apache.org>.
icexelloss commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1117325015


##########
cpp/src/arrow/compute/exec/aggregate_node.cc:
##########
@@ -326,46 +446,86 @@ class ScalarAggregateNode : public ExecNode, public TracedNode {
   }
 
  private:
-  Status Finish() {
-    auto scope = TraceFinish();
+  Status ReconstructAggregates() {
+    const auto& input_schema = *inputs()[0]->output_schema();
+    auto exec_ctx = plan()->query_context()->exec_context();
+    for (size_t i = 0; i < kernels_.size(); ++i) {
+      std::vector<TypeHolder> in_types;
+      for (const auto& target : target_fieldsets_[i]) {
+        in_types.emplace_back(input_schema.field(target)->type().get());
+      }
+      states_[i].resize(plan()->query_context()->max_concurrency());

Review Comment:
   Why is state resized to max_concurrency here?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rtpsw commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "rtpsw (via GitHub)" <gi...@apache.org>.
rtpsw commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1117546135


##########
cpp/src/arrow/compute/exec/aggregate_node.cc:
##########
@@ -326,46 +446,86 @@ class ScalarAggregateNode : public ExecNode, public TracedNode {
   }
 
  private:
-  Status Finish() {
-    auto scope = TraceFinish();
+  Status ReconstructAggregates() {
+    const auto& input_schema = *inputs()[0]->output_schema();
+    auto exec_ctx = plan()->query_context()->exec_context();
+    for (size_t i = 0; i < kernels_.size(); ++i) {
+      std::vector<TypeHolder> in_types;
+      for (const auto& target : target_fieldsets_[i]) {
+        in_types.emplace_back(input_schema.field(target)->type().get());
+      }
+      states_[i].resize(plan()->query_context()->max_concurrency());
+      KernelContext kernel_ctx{exec_ctx};
+      RETURN_NOT_OK(Kernel::InitAll(
+          &kernel_ctx, KernelInitArgs{kernels_[i], in_types, aggs_[i].options.get()},
+          &states_[i]));
+    }
+    return Status::OK();
+  }
+
+  Status OutputResult(bool is_last = false, bool traced = false) {
+    if (is_last && !traced) {
+      auto scope = TraceFinish();
+      return OutputResult(is_last, /*traced=*/true);
+    }
+    GatedUniqueLock lock(gated_shared_mutex_);
     ExecBatch batch{{}, 1};
-    batch.values.resize(kernels_.size());
+    batch.values.resize(kernels_.size() + segment_field_ids_.size());
 
     for (size_t i = 0; i < kernels_.size(); ++i) {
       util::tracing::Span span;
       START_COMPUTE_SPAN(span, aggs_[i].function,
                          {{"function.name", aggs_[i].function},
                           {"function.options",
                            aggs_[i].options ? aggs_[i].options->ToString() : "<NULLPTR>"},
-                          {"function.kind", std::string(kind_name()) + "::Finalize"}});
+                          {"function.kind", std::string(kind_name()) + "::Output"}});
       KernelContext ctx{plan()->query_context()->exec_context()};
       ARROW_ASSIGN_OR_RAISE(auto merged, ScalarAggregateKernel::MergeAll(
                                              kernels_[i], &ctx, std::move(states_[i])));
       RETURN_NOT_OK(kernels_[i]->finalize(&ctx, &batch.values[i]));
     }
+    PlaceFields(batch, kernels_.size(), segmenter_values_);
 
-    return output_->InputReceived(this, std::move(batch));
+    ARROW_RETURN_NOT_OK(output_->InputReceived(this, std::move(batch)));
+    total_output_batches_++;

Review Comment:
   Yes.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rtpsw commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "rtpsw (via GitHub)" <gi...@apache.org>.
rtpsw commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1117885560


##########
cpp/src/arrow/compute/kernels/hash_aggregate_test.cc:
##########
@@ -174,81 +242,117 @@ Result<Datum> RunGroupBy(const BatchesWithSchema& input,
   ARROW_ASSIGN_OR_RAISE(std::vector<ExecBatch> output_batches,
                         start_and_collect.MoveResult());
 
-  ArrayVector out_arrays(aggregates.size() + key_names.size());
   const auto& output_schema = plan->nodes()[0]->output()->output_schema();
+  if (!segmented) {
+    return MakeGroupByOutput(output_batches, output_schema, aggregates.size(),
+                             key_names.size(), naive);
+  }
+
+  std::vector<ArrayVector> out_arrays(aggregates.size() + key_names.size() +
+                                      segment_key_names.size());
   for (size_t i = 0; i < out_arrays.size(); ++i) {
     std::vector<std::shared_ptr<Array>> arrays(output_batches.size());
     for (size_t j = 0; j < output_batches.size(); ++j) {
-      arrays[j] = output_batches[j].values[i].make_array();
+      auto& value = output_batches[j].values[i];
+      if (value.is_scalar()) {
+        ARROW_ASSIGN_OR_RAISE(
+            arrays[j], MakeArrayFromScalar(*value.scalar(), output_batches[j].length));
+      } else if (value.is_array()) {
+        arrays[j] = value.make_array();
+      } else {
+        return Status::Invalid("GroupByUsingExecPlan unsupported value kind ",
+                               ToString(value.kind()));
+      }
     }
     if (arrays.empty()) {
+      arrays.resize(1);
       ARROW_ASSIGN_OR_RAISE(
-          out_arrays[i],
-          MakeArrayOfNull(output_schema->field(static_cast<int>(i))->type(),
-                          /*length=*/0));
-    } else {
-      ARROW_ASSIGN_OR_RAISE(out_arrays[i], Concatenate(arrays));
+          arrays[0], MakeArrayOfNull(output_schema->field(static_cast<int>(i))->type(),
+                                     /*length=*/0));
     }
+    out_arrays[i] = {std::move(arrays)};
   }
 
-  // The exec plan may reorder the output rows.  The tests are all setup to expect ouptut
-  // in ascending order of keys.  So we need to sort the result by the key columns.  To do
-  // that we create a table using the key columns, calculate the sort indices from that
-  // table (sorting on all fields) and then use those indices to calculate our result.
-  std::vector<std::shared_ptr<Field>> key_fields;
-  std::vector<std::shared_ptr<Array>> key_columns;
-  std::vector<SortKey> sort_keys;
-  for (std::size_t i = 0; i < key_names.size(); i++) {
-    const std::shared_ptr<Array>& arr = out_arrays[i + aggregates.size()];
-    if (arr->type_id() == Type::DICTIONARY) {
-      // Can't sort dictionary columns so need to decode
-      auto dict_arr = checked_pointer_cast<DictionaryArray>(arr);
-      ARROW_ASSIGN_OR_RAISE(auto decoded_arr,
-                            Take(*dict_arr->dictionary(), *dict_arr->indices()));
-      key_columns.push_back(decoded_arr);
-      key_fields.push_back(
-          field("name_does_not_matter", dict_arr->dict_type()->value_type()));
-    } else {
-      key_columns.push_back(arr);
-      key_fields.push_back(field("name_does_not_matter", arr->type()));
+  if (segmented && segment_key_names.size() > 0) {
+    ArrayVector struct_arrays;
+    struct_arrays.reserve(output_batches.size());
+    for (size_t j = 0; j < output_batches.size(); ++j) {
+      ArrayVector struct_fields;
+      struct_fields.reserve(out_arrays.size());
+      for (auto out_array : out_arrays) {
+        struct_fields.push_back(out_array[j]);
+      }
+      ARROW_ASSIGN_OR_RAISE(auto struct_array,
+                            StructArray::Make(struct_fields, output_schema->fields()));
+      struct_arrays.push_back(struct_array);
     }
-    sort_keys.emplace_back(static_cast<int>(i));
+    return ChunkedArray::Make(struct_arrays);
+  } else {
+    ArrayVector struct_fields(out_arrays.size());
+    for (size_t i = 0; i < out_arrays.size(); ++i) {
+      ARROW_ASSIGN_OR_RAISE(struct_fields[i], Concatenate(out_arrays[i]));
+    }
+    return StructArray::Make(std::move(struct_fields), output_schema->fields());
   }
-  std::shared_ptr<Schema> key_schema = schema(std::move(key_fields));
-  std::shared_ptr<Table> key_table = Table::Make(std::move(key_schema), key_columns);
-  SortOptions sort_options(std::move(sort_keys));
-  ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Array> sort_indices,
-                        SortIndices(key_table, sort_options));
+}
 
-  ARROW_ASSIGN_OR_RAISE(
-      std::shared_ptr<Array> struct_arr,
-      StructArray::Make(std::move(out_arrays), output_schema->fields()));
+Result<Datum> RunGroupBy(const BatchesWithSchema& input,
+                         const std::vector<std::string>& key_names,
+                         const std::vector<std::string>& segment_key_names,
+                         const std::vector<Aggregate>& aggregates, bool use_threads,
+                         bool segmented = false, bool naive = false) {
+  if (segment_key_names.size() > 0) {
+    ARROW_ASSIGN_OR_RAISE(auto thread_pool, arrow::internal::ThreadPool::Make(1));
+    ExecContext seq_ctx(default_memory_pool(), thread_pool.get());
+    return RunGroupBy(input, key_names, segment_key_names, aggregates, &seq_ctx,
+                      use_threads, segmented, naive);
+  } else {
+    return RunGroupBy(input, key_names, segment_key_names, aggregates,
+                      threaded_exec_context(), use_threads, segmented, naive);
+  }
+}
 
-  return Take(struct_arr, sort_indices);
+Result<Datum> RunGroupBy(const BatchesWithSchema& input,
+                         const std::vector<std::string>& key_names,
+                         const std::vector<Aggregate>& aggregates, bool use_threads,
+                         bool segmented = false, bool naive = false) {
+  return RunGroupBy(input, key_names, {}, aggregates, use_threads, segmented);
 }
 
 /// Simpler overload where you can give the columns as datums
 Result<Datum> RunGroupBy(const std::vector<Datum>& arguments,
                          const std::vector<Datum>& keys,
-                         const std::vector<Aggregate>& aggregates,
-                         bool use_threads = false) {
+                         const std::vector<Datum>& segment_keys,
+                         const std::vector<Aggregate>& aggregates, bool use_threads,
+                         bool segmented = false, bool naive = false) {

Review Comment:
   `naive` refers to `NaiveGroupBy`. When `false`, the expected group-by output is [generally sorted](https://github.com/apache/arrow/pull/34311/files#diff-ec1e8907dfc3e636abecd3307879f331421e4f28f75e504f147826cebf04d7e0R167-R172).



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] icexelloss commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "icexelloss (via GitHub)" <gi...@apache.org>.
icexelloss commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1128387312


##########
cpp/src/arrow/compute/kernels/hash_aggregate_test.cc:
##########
@@ -4202,5 +4551,260 @@ TEST(GroupBy, OnlyKeys) {
                       /*verbose=*/true);
   }
 }
+
+INSTANTIATE_TEST_SUITE_P(GroupBy, GroupBy, ::testing::Values(RunGroupByImpl));
+
+class SegmentedScalarGroupBy : public GroupBy {};
+
+class SegmentedKeyGroupBy : public GroupBy {};
+
+void TestSegment(GroupByFunction group_by, const std::shared_ptr<Table>& table,
+                 Datum output, const std::vector<Datum>& keys,
+                 const std::vector<Datum>& segment_keys, bool scalar) {

Review Comment:
   Thanks



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] icexelloss commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "icexelloss (via GitHub)" <gi...@apache.org>.
icexelloss commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1126894399


##########
cpp/src/arrow/compute/row/grouper.h:
##########
@@ -30,6 +30,49 @@
 namespace arrow {
 namespace compute {
 
+/// \brief A segment of contiguous rows for grouping
+struct ARROW_EXPORT GroupingSegment {
+  int64_t offset;
+  int64_t length;
+  bool is_open;
+  bool extends;
+};
+
+inline bool operator==(const GroupingSegment& segment1, const GroupingSegment& segment2) {
+  return segment1.offset == segment2.offset && segment1.length == segment2.length &&
+         segment1.is_open == segment2.is_open && segment1.extends == segment2.extends;
+}
+inline bool operator!=(const GroupingSegment& segment1, const GroupingSegment& segment2) {
+  return !(segment1 == segment2);
+}
+
+/// \brief Computes grouping segments for a batch. Each segment covers rows with identical
+/// values in the batch. The values in the batch are often selected as keys from a larger
+/// batch.
+class ARROW_EXPORT GroupingSegmenter {
+ public:
+  virtual ~GroupingSegmenter() = default;
+
+  /// \brief Construct a GroupingSegmenter which receives the specified key types
+  static Result<std::unique_ptr<GroupingSegmenter>> Make(
+      const std::vector<TypeHolder>& key_types, bool nullable_keys = false,
+      ExecContext* ctx = default_exec_context());

Review Comment:
   Oh I see - you are trying to create a "Make" interface for both `RowSegmenter` and `Grouper` class. IIUC, the common interface is only used in templating test since `RowSegmenter` and `Grouper` don't really share the same interface. Actually I found the templating of these two classes together quite confusing - all it is testing is the "Make" function. We might as well having them tested separately to avoid confusion.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] icexelloss commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "icexelloss (via GitHub)" <gi...@apache.org>.
icexelloss commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1127017346


##########
cpp/src/arrow/compute/row/grouper.cc:
##########
@@ -39,12 +43,330 @@
 namespace arrow {
 
 using internal::checked_cast;
+using internal::PrimitiveScalarBase;
 
 namespace compute {
 
 namespace {
 
-struct GrouperImpl : Grouper {
+constexpr uint32_t kNoGroupId = std::numeric_limits<uint32_t>::max();
+
+using group_id_t = std::remove_const<decltype(kNoGroupId)>::type;
+using GroupIdType = CTypeTraits<group_id_t>::ArrowType;
+auto g_group_id_type = std::make_shared<GroupIdType>();
+
+inline const uint8_t* GetValuesAsBytes(const ArraySpan& data, int64_t offset = 0) {
+  DCHECK_GT(data.type->byte_width(), 0);
+  int64_t absolute_byte_offset = (data.offset + offset) * data.type->byte_width();
+  return data.GetValues<uint8_t>(1, absolute_byte_offset);
+}
+
+template <typename Value>
+Status CheckForGetNextSegment(const std::vector<Value>& values, int64_t length,
+                              int64_t offset, const std::vector<TypeHolder>& key_types) {
+  if (offset < 0 || offset > length) {
+    return Status::Invalid("invalid grouping segmenter offset: ", offset);
+  }
+  if (values.size() != key_types.size()) {
+    return Status::Invalid("expected batch size ", key_types.size(), " but got ",
+                           values.size());
+  }
+  for (size_t i = 0; i < key_types.size(); i++) {
+    const auto& value = values[i];
+    const auto& key_type = key_types[i];
+    if (*value.type() != *key_type.type) {
+      return Status::Invalid("expected batch value ", i, " of type ", *key_type.type,
+                             " but got ", *value.type());
+    }
+  }
+  return Status::OK();
+}
+
+template <typename Batch>
+enable_if_t<std::is_same<Batch, ExecSpan>::value || std::is_same<Batch, ExecBatch>::value,
+            Status>
+CheckForGetNextSegment(const Batch& batch, int64_t offset,
+                       const std::vector<TypeHolder>& key_types) {
+  return CheckForGetNextSegment(batch.values, batch.length, offset, key_types);
+}
+
+struct BaseRowSegmenter : public RowSegmenter {
+  explicit BaseRowSegmenter(const std::vector<TypeHolder>& key_types)
+      : key_types_(key_types) {}
+
+  const std::vector<TypeHolder>& key_types() const override { return key_types_; }
+
+  std::vector<TypeHolder> key_types_;
+};
+
+Segment MakeSegment(int64_t batch_length, int64_t offset, int64_t length, bool extends) {
+  return Segment{offset, length, offset + length >= batch_length, extends};
+}
+
+int64_t GetMatchLength(const uint8_t* match_bytes, int64_t match_width,
+                       const uint8_t* array_bytes, int64_t offset, int64_t length) {
+  int64_t cursor, byte_cursor;
+  for (cursor = offset, byte_cursor = match_width * cursor; cursor < length;
+       cursor++, byte_cursor += match_width) {
+    if (memcmp(match_bytes, array_bytes + byte_cursor,
+               static_cast<size_t>(match_width)) != 0) {
+      break;
+    }
+  }
+  return std::min(cursor, length) - offset;
+}
+
+using ExtendFunc = std::function<bool(const void*)>;
+constexpr bool kDefaultExtends = true;
+constexpr bool kEmptyExtends = true;
+
+struct NoKeysSegmenter : public BaseRowSegmenter {
+  static std::unique_ptr<RowSegmenter> Make() {
+    return std::make_unique<NoKeysSegmenter>();
+  }
+
+  NoKeysSegmenter() : BaseRowSegmenter({}) {}
+
+  Status Reset() override { return Status::OK(); }
+
+  Result<Segment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {}));
+    return MakeSegment(batch.length, offset, batch.length - offset, kDefaultExtends);
+  }
+};
+
+struct SimpleKeySegmenter : public BaseRowSegmenter {

Review Comment:
   Can you add test to check `SimpleKeySegmenter` gives the same result as `AnyKeysSegmenter` with one key?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] icexelloss commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "icexelloss (via GitHub)" <gi...@apache.org>.
icexelloss commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1125574200


##########
cpp/src/arrow/compute/exec/aggregate_node.cc:
##########
@@ -169,35 +185,117 @@ void AggregatesToString(std::stringstream* ss, const Schema& input_schema,
   *ss << ']';
 }
 
+template <typename BatchHandler>
+Status HandleSegments(std::unique_ptr<GroupingSegmenter>& segmenter,
+                      const ExecBatch& batch, const std::vector<int>& ids,
+                      const BatchHandler& handle_batch) {
+  int64_t offset = 0;
+  ARROW_ASSIGN_OR_RAISE(auto segment_exec_batch, batch.SelectValues(ids));
+  ExecSpan segment_batch(segment_exec_batch);
+  while (true) {
+    ARROW_ASSIGN_OR_RAISE(auto segment, segmenter->GetNextSegment(segment_batch, offset));
+    if (segment.offset >= segment_batch.length) break;  // condition of no-next-segment
+    ARROW_RETURN_NOT_OK(handle_batch(batch, segment));
+    offset = segment.offset + segment.length;
+  }
+  return Status::OK();
+}
+
+Status GetScalarFields(std::vector<Datum>* values_ptr, const ExecBatch& input_batch,
+                       const std::vector<int>& field_ids) {
+  DCHECK_GT(input_batch.length, 0);
+  std::vector<Datum>& values = *values_ptr;
+  int64_t row = input_batch.length - 1;
+  values.clear();
+  values.resize(field_ids.size());
+  for (size_t i = 0; i < field_ids.size(); i++) {
+    const Datum& value = input_batch.values[field_ids[i]];
+    if (value.is_scalar()) {
+      values[i] = value;
+    } else if (value.is_array()) {
+      ARROW_ASSIGN_OR_RAISE(auto scalar, value.make_array()->GetScalar(row));
+      values[i] = scalar;
+    } else {
+      DCHECK(false);
+    }
+  }
+  return Status::OK();
+}
+
+void PlaceFields(ExecBatch& batch, size_t base, std::vector<Datum>& values) {
+  DCHECK_LE(base + values.size(), batch.values.size());
+  for (size_t i = 0; i < values.size(); i++) {
+    batch.values[base + i] = values[i];
+  }
+}
+
 class ScalarAggregateNode : public ExecNode, public TracedNode {
  public:
   ScalarAggregateNode(ExecPlan* plan, std::vector<ExecNode*> inputs,
                       std::shared_ptr<Schema> output_schema,
+                      std::unique_ptr<GroupingSegmenter> segmenter,
+                      std::vector<int> segment_field_ids,
                       std::vector<std::vector<int>> target_fieldsets,
                       std::vector<Aggregate> aggs,
                       std::vector<const ScalarAggregateKernel*> kernels,
                       std::vector<std::vector<std::unique_ptr<KernelState>>> states)
       : ExecNode(plan, std::move(inputs), {"target"},
                  /*output_schema=*/std::move(output_schema)),
         TracedNode(this),
+        segmenter_(std::move(segmenter)),
+        segment_field_ids_(std::move(segment_field_ids)),
         target_fieldsets_(std::move(target_fieldsets)),
         aggs_(std::move(aggs)),
         kernels_(std::move(kernels)),
-        states_(std::move(states)) {}
+        states_(std::move(states)) {
+    const auto& input_schema = *this->inputs()[0]->output_schema();
+    for (size_t i = 0; i < kernels_.size(); ++i) {
+      std::vector<TypeHolder> in_types;
+      for (const auto& target : target_fieldsets_[i]) {
+        in_types.emplace_back(input_schema.field(target)->type().get());
+      }
+      in_typesets_.push_back(std::move(in_types));
+    }
+  }
 
   static Result<ExecNode*> Make(ExecPlan* plan, std::vector<ExecNode*> inputs,
                                 const ExecNodeOptions& options) {
     RETURN_NOT_OK(ValidateExecNodeInputs(plan, inputs, 1, "ScalarAggregateNode"));
 
     const auto& aggregate_options = checked_cast<const AggregateNodeOptions&>(options);
     auto aggregates = aggregate_options.aggregates;
+    const auto& keys = aggregate_options.keys;
+    const auto& segment_keys = aggregate_options.segment_keys;
+
+    if (keys.size() > 0) {
+      return Status::Invalid("Scalar aggregation with some key");
+    }
+    if (plan->query_context()->exec_context()->executor()->GetCapacity() > 1 &&
+        segment_keys.size() > 0) {
+      return Status::NotImplemented("Segmented aggregation in a multi-threaded plan");
+    }
 
     const auto& input_schema = *inputs[0]->output_schema();
     auto exec_ctx = plan->query_context()->exec_context();
 
+    std::vector<int> segment_field_ids(segment_keys.size());
+    std::vector<TypeHolder> segment_key_types(segment_keys.size());
+    for (size_t i = 0; i < segment_keys.size(); i++) {
+      ARROW_ASSIGN_OR_RAISE(auto match, segment_keys[i].FindOne(input_schema));
+      if (match.indices().size() > 1) {
+        // ARROW-18369: Support nested references as segment ids
+        return Status::Invalid("Nested references cannot be used as segment ids");
+      }
+      segment_field_ids[i] = match[0];
+      segment_key_types[i] = input_schema.field(match[0])->type().get();
+    }
+
+    ARROW_ASSIGN_OR_RAISE(
+        auto segmenter, GroupingSegmenter::Make(std::move(segment_key_types), exec_ctx));
+
     std::vector<const ScalarAggregateKernel*> kernels(aggregates.size());
     std::vector<std::vector<std::unique_ptr<KernelState>>> states(kernels.size());
-    FieldVector fields(kernels.size());
+    FieldVector fields(kernels.size() + segment_keys.size());

Review Comment:
   I understand you are allocating spaces for segment key space here, but it's not obvious to me what are "fields" here and why do we need to allocate space for segement keys in the "fields"?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rtpsw commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "rtpsw (via GitHub)" <gi...@apache.org>.
rtpsw commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1126991132


##########
cpp/src/arrow/compute/row/grouper.cc:
##########
@@ -39,12 +43,330 @@
 namespace arrow {
 
 using internal::checked_cast;
+using internal::PrimitiveScalarBase;
 
 namespace compute {
 
 namespace {
 
-struct GrouperImpl : Grouper {
+constexpr uint32_t kNoGroupId = std::numeric_limits<uint32_t>::max();
+
+using group_id_t = std::remove_const<decltype(kNoGroupId)>::type;
+using GroupIdType = CTypeTraits<group_id_t>::ArrowType;
+auto g_group_id_type = std::make_shared<GroupIdType>();
+
+inline const uint8_t* GetValuesAsBytes(const ArraySpan& data, int64_t offset = 0) {
+  DCHECK_GT(data.type->byte_width(), 0);
+  int64_t absolute_byte_offset = (data.offset + offset) * data.type->byte_width();
+  return data.GetValues<uint8_t>(1, absolute_byte_offset);
+}
+
+template <typename Value>
+Status CheckForGetNextSegment(const std::vector<Value>& values, int64_t length,
+                              int64_t offset, const std::vector<TypeHolder>& key_types) {
+  if (offset < 0 || offset > length) {
+    return Status::Invalid("invalid grouping segmenter offset: ", offset);
+  }
+  if (values.size() != key_types.size()) {
+    return Status::Invalid("expected batch size ", key_types.size(), " but got ",
+                           values.size());
+  }
+  for (size_t i = 0; i < key_types.size(); i++) {
+    const auto& value = values[i];
+    const auto& key_type = key_types[i];
+    if (*value.type() != *key_type.type) {
+      return Status::Invalid("expected batch value ", i, " of type ", *key_type.type,
+                             " but got ", *value.type());
+    }
+  }
+  return Status::OK();
+}
+
+template <typename Batch>
+enable_if_t<std::is_same<Batch, ExecSpan>::value || std::is_same<Batch, ExecBatch>::value,
+            Status>
+CheckForGetNextSegment(const Batch& batch, int64_t offset,
+                       const std::vector<TypeHolder>& key_types) {
+  return CheckForGetNextSegment(batch.values, batch.length, offset, key_types);
+}
+
+struct BaseRowSegmenter : public RowSegmenter {
+  explicit BaseRowSegmenter(const std::vector<TypeHolder>& key_types)
+      : key_types_(key_types) {}
+
+  const std::vector<TypeHolder>& key_types() const override { return key_types_; }
+
+  std::vector<TypeHolder> key_types_;
+};
+
+Segment MakeSegment(int64_t batch_length, int64_t offset, int64_t length, bool extends) {
+  return Segment{offset, length, offset + length >= batch_length, extends};
+}
+
+int64_t GetMatchLength(const uint8_t* match_bytes, int64_t match_width,
+                       const uint8_t* array_bytes, int64_t offset, int64_t length) {
+  int64_t cursor, byte_cursor;
+  for (cursor = offset, byte_cursor = match_width * cursor; cursor < length;
+       cursor++, byte_cursor += match_width) {
+    if (memcmp(match_bytes, array_bytes + byte_cursor,
+               static_cast<size_t>(match_width)) != 0) {
+      break;
+    }
+  }
+  return std::min(cursor, length) - offset;
+}
+
+using ExtendFunc = std::function<bool(const void*)>;
+constexpr bool kDefaultExtends = true;
+constexpr bool kEmptyExtends = true;
+
+struct NoKeysSegmenter : public BaseRowSegmenter {
+  static std::unique_ptr<RowSegmenter> Make() {
+    return std::make_unique<NoKeysSegmenter>();
+  }
+
+  NoKeysSegmenter() : BaseRowSegmenter({}) {}
+
+  Status Reset() override { return Status::OK(); }
+
+  Result<Segment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {}));
+    return MakeSegment(batch.length, offset, batch.length - offset, kDefaultExtends);
+  }
+};
+
+struct SimpleKeySegmenter : public BaseRowSegmenter {
+  static Result<std::unique_ptr<RowSegmenter>> Make(TypeHolder key_type) {
+    return std::make_unique<SimpleKeySegmenter>(key_type);
+  }
+
+  explicit SimpleKeySegmenter(TypeHolder key_type)
+      : BaseRowSegmenter({key_type}), key_type_(key_types_[0]), save_key_data_() {}
+
+  Status CheckType(const DataType& type) {
+    if (!is_fixed_width(type)) {
+      return Status::Invalid("SimpleKeySegmenter does not support type ", type);
+    }
+    return Status::OK();
+  }
+
+  Status Reset() override {
+    save_key_data_.resize(0);
+    return Status::OK();
+  }
+
+  // Checks whether the given grouping data extends the current segment, i.e., is equal to
+  // previously seen grouping data, which is updated with each invocation.
+  bool Extend(const void* data) {
+    size_t byte_width = static_cast<size_t>(key_type_.type->byte_width());
+    bool extends = save_key_data_.size() != byte_width
+                       ? kDefaultExtends
+                       : 0 == memcmp(save_key_data_.data(), data, byte_width);
+    save_key_data_.resize(byte_width);
+    memcpy(save_key_data_.data(), data, byte_width);
+    return extends;
+  }
+
+  Result<Segment> GetNextSegment(const Scalar& scalar, int64_t offset, int64_t length) {
+    ARROW_RETURN_NOT_OK(CheckType(*scalar.type));
+    if (!scalar.is_valid) {
+      return Status::Invalid("segmenting an invalid scalar");
+    }
+    auto data = checked_cast<const PrimitiveScalarBase&>(scalar).data();
+    bool extends = length > 0 ? Extend(data) : kEmptyExtends;
+    return MakeSegment(length, offset, length, extends);
+  }
+
+  Result<Segment> GetNextSegment(const DataType& array_type, const uint8_t* array_bytes,
+                                 int64_t offset, int64_t length) {
+    RETURN_NOT_OK(CheckType(array_type));
+    int64_t byte_width = array_type.byte_width();
+    int64_t match_length = GetMatchLength(array_bytes + offset * byte_width, byte_width,
+                                          array_bytes, offset, length);
+    bool extends = length > 0 ? Extend(array_bytes + offset * byte_width) : kEmptyExtends;
+    return MakeSegment(length, offset, match_length, extends);
+  }
+
+  Result<Segment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {key_type_}));
+    if (offset == batch.length) {
+      return MakeSegment(batch.length, offset, 0, kEmptyExtends);
+    }
+    const auto& value = batch.values[0];
+    if (value.is_scalar()) {
+      return GetNextSegment(*value.scalar, offset, batch.length);
+    }
+    ARROW_DCHECK(value.is_array());
+    const auto& array = value.array;
+    if (array.GetNullCount() > 0) {
+      return Status::NotImplemented("segmenting a nullable array");
+    }
+    return GetNextSegment(*array.type, GetValuesAsBytes(array), offset, batch.length);
+  }
+
+ private:
+  TypeHolder key_type_;
+  std::vector<uint8_t> save_key_data_;
+};
+
+struct AnyKeysSegmenter : public BaseRowSegmenter {
+  static Result<std::unique_ptr<RowSegmenter>> Make(
+      const std::vector<TypeHolder>& key_types, ExecContext* ctx) {
+    ARROW_RETURN_NOT_OK(Grouper::Make(key_types, ctx));  // check types
+    return std::make_unique<AnyKeysSegmenter>(key_types, ctx);
+  }
+
+  AnyKeysSegmenter(const std::vector<TypeHolder>& key_types, ExecContext* ctx)
+      : BaseRowSegmenter(key_types),
+        ctx_(ctx),
+        grouper_(nullptr),
+        save_group_id_(kNoGroupId) {}
+
+  Status Reset() override {
+    grouper_ = nullptr;
+    save_group_id_ = kNoGroupId;
+    return Status::OK();
+  }
+
+  bool Extend(const void* data) {
+    auto group_id = *static_cast<const group_id_t*>(data);
+    bool extends =
+        save_group_id_ == kNoGroupId ? kDefaultExtends : save_group_id_ == group_id;
+    save_group_id_ = group_id;
+    return extends;
+  }
+
+  // Runs the grouper on a single row.  This is used to determine the group id of the
+  // first row of a new segment to see if it extends the previous segment.
+  template <typename Batch>
+  Result<group_id_t> MapGroupIdAt(const Batch& batch, int64_t offset) {
+    if (!grouper_) return kNoGroupId;
+    ARROW_ASSIGN_OR_RAISE(auto datum, grouper_->Consume(batch, offset,
+                                                        /*length=*/1));
+    if (!datum.is_array()) {
+      return Status::Invalid("accessing unsupported datum kind ", datum.kind());
+    }
+    const std::shared_ptr<ArrayData>& data = datum.array();
+    ARROW_DCHECK(data->GetNullCount() == 0);
+    DCHECK_EQ(data->type->id(), GroupIdType::type_id);
+    DCHECK_EQ(1, data->length);
+    const group_id_t* values = data->GetValues<group_id_t>(1);
+    return values[0];
+  }
+
+  Result<Segment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, key_types_));
+    if (offset == batch.length) {
+      return MakeSegment(batch.length, offset, 0, kEmptyExtends);
+    }
+    // ARROW-18311: make Grouper support Reset()

Review Comment:
   A bit further down the code there is a `Grouper::Make` invocation that just ends up resetting the grouper. If `Grouper::Reset` were available then it would be called instead. The doc says "so it can be cached instead of recreated below".



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] icexelloss commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "icexelloss (via GitHub)" <gi...@apache.org>.
icexelloss commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1127133753


##########
cpp/src/arrow/compute/row/grouper.cc:
##########
@@ -39,12 +43,330 @@
 namespace arrow {
 
 using internal::checked_cast;
+using internal::PrimitiveScalarBase;
 
 namespace compute {
 
 namespace {
 
-struct GrouperImpl : Grouper {
+constexpr uint32_t kNoGroupId = std::numeric_limits<uint32_t>::max();
+
+using group_id_t = std::remove_const<decltype(kNoGroupId)>::type;
+using GroupIdType = CTypeTraits<group_id_t>::ArrowType;
+auto g_group_id_type = std::make_shared<GroupIdType>();
+
+inline const uint8_t* GetValuesAsBytes(const ArraySpan& data, int64_t offset = 0) {
+  DCHECK_GT(data.type->byte_width(), 0);
+  int64_t absolute_byte_offset = (data.offset + offset) * data.type->byte_width();
+  return data.GetValues<uint8_t>(1, absolute_byte_offset);
+}
+
+template <typename Value>
+Status CheckForGetNextSegment(const std::vector<Value>& values, int64_t length,
+                              int64_t offset, const std::vector<TypeHolder>& key_types) {
+  if (offset < 0 || offset > length) {
+    return Status::Invalid("invalid grouping segmenter offset: ", offset);
+  }
+  if (values.size() != key_types.size()) {
+    return Status::Invalid("expected batch size ", key_types.size(), " but got ",
+                           values.size());
+  }
+  for (size_t i = 0; i < key_types.size(); i++) {
+    const auto& value = values[i];
+    const auto& key_type = key_types[i];
+    if (*value.type() != *key_type.type) {
+      return Status::Invalid("expected batch value ", i, " of type ", *key_type.type,
+                             " but got ", *value.type());
+    }
+  }
+  return Status::OK();
+}
+
+template <typename Batch>
+enable_if_t<std::is_same<Batch, ExecSpan>::value || std::is_same<Batch, ExecBatch>::value,
+            Status>
+CheckForGetNextSegment(const Batch& batch, int64_t offset,
+                       const std::vector<TypeHolder>& key_types) {
+  return CheckForGetNextSegment(batch.values, batch.length, offset, key_types);
+}
+
+struct BaseRowSegmenter : public RowSegmenter {
+  explicit BaseRowSegmenter(const std::vector<TypeHolder>& key_types)
+      : key_types_(key_types) {}
+
+  const std::vector<TypeHolder>& key_types() const override { return key_types_; }
+
+  std::vector<TypeHolder> key_types_;
+};
+
+Segment MakeSegment(int64_t batch_length, int64_t offset, int64_t length, bool extends) {
+  return Segment{offset, length, offset + length >= batch_length, extends};
+}
+
+int64_t GetMatchLength(const uint8_t* match_bytes, int64_t match_width,
+                       const uint8_t* array_bytes, int64_t offset, int64_t length) {
+  int64_t cursor, byte_cursor;
+  for (cursor = offset, byte_cursor = match_width * cursor; cursor < length;
+       cursor++, byte_cursor += match_width) {
+    if (memcmp(match_bytes, array_bytes + byte_cursor,
+               static_cast<size_t>(match_width)) != 0) {
+      break;
+    }
+  }
+  return std::min(cursor, length) - offset;
+}
+
+using ExtendFunc = std::function<bool(const void*)>;
+constexpr bool kDefaultExtends = true;
+constexpr bool kEmptyExtends = true;
+
+struct NoKeysSegmenter : public BaseRowSegmenter {
+  static std::unique_ptr<RowSegmenter> Make() {
+    return std::make_unique<NoKeysSegmenter>();
+  }
+
+  NoKeysSegmenter() : BaseRowSegmenter({}) {}
+
+  Status Reset() override { return Status::OK(); }
+
+  Result<Segment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {}));
+    return MakeSegment(batch.length, offset, batch.length - offset, kDefaultExtends);
+  }
+};
+
+struct SimpleKeySegmenter : public BaseRowSegmenter {
+  static Result<std::unique_ptr<RowSegmenter>> Make(TypeHolder key_type) {
+    return std::make_unique<SimpleKeySegmenter>(key_type);
+  }
+
+  explicit SimpleKeySegmenter(TypeHolder key_type)
+      : BaseRowSegmenter({key_type}), key_type_(key_types_[0]), save_key_data_() {}
+
+  Status CheckType(const DataType& type) {
+    if (!is_fixed_width(type)) {
+      return Status::Invalid("SimpleKeySegmenter does not support type ", type);
+    }
+    return Status::OK();
+  }
+
+  Status Reset() override {
+    save_key_data_.resize(0);
+    return Status::OK();
+  }
+
+  // Checks whether the given grouping data extends the current segment, i.e., is equal to
+  // previously seen grouping data, which is updated with each invocation.
+  bool Extend(const void* data) {
+    size_t byte_width = static_cast<size_t>(key_type_.type->byte_width());
+    bool extends = save_key_data_.size() != byte_width
+                       ? kDefaultExtends
+                       : 0 == memcmp(save_key_data_.data(), data, byte_width);
+    save_key_data_.resize(byte_width);

Review Comment:
   Can we not initialize it to the right size?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rtpsw commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "rtpsw (via GitHub)" <gi...@apache.org>.
rtpsw commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1127113622


##########
cpp/src/arrow/compute/row/grouper.cc:
##########
@@ -39,12 +43,330 @@
 namespace arrow {
 
 using internal::checked_cast;
+using internal::PrimitiveScalarBase;
 
 namespace compute {
 
 namespace {
 
-struct GrouperImpl : Grouper {
+constexpr uint32_t kNoGroupId = std::numeric_limits<uint32_t>::max();
+
+using group_id_t = std::remove_const<decltype(kNoGroupId)>::type;
+using GroupIdType = CTypeTraits<group_id_t>::ArrowType;
+auto g_group_id_type = std::make_shared<GroupIdType>();
+
+inline const uint8_t* GetValuesAsBytes(const ArraySpan& data, int64_t offset = 0) {
+  DCHECK_GT(data.type->byte_width(), 0);
+  int64_t absolute_byte_offset = (data.offset + offset) * data.type->byte_width();
+  return data.GetValues<uint8_t>(1, absolute_byte_offset);
+}
+
+template <typename Value>
+Status CheckForGetNextSegment(const std::vector<Value>& values, int64_t length,
+                              int64_t offset, const std::vector<TypeHolder>& key_types) {
+  if (offset < 0 || offset > length) {
+    return Status::Invalid("invalid grouping segmenter offset: ", offset);
+  }
+  if (values.size() != key_types.size()) {
+    return Status::Invalid("expected batch size ", key_types.size(), " but got ",
+                           values.size());
+  }
+  for (size_t i = 0; i < key_types.size(); i++) {
+    const auto& value = values[i];
+    const auto& key_type = key_types[i];
+    if (*value.type() != *key_type.type) {
+      return Status::Invalid("expected batch value ", i, " of type ", *key_type.type,
+                             " but got ", *value.type());
+    }
+  }
+  return Status::OK();
+}
+
+template <typename Batch>
+enable_if_t<std::is_same<Batch, ExecSpan>::value || std::is_same<Batch, ExecBatch>::value,
+            Status>
+CheckForGetNextSegment(const Batch& batch, int64_t offset,
+                       const std::vector<TypeHolder>& key_types) {
+  return CheckForGetNextSegment(batch.values, batch.length, offset, key_types);
+}
+
+struct BaseRowSegmenter : public RowSegmenter {
+  explicit BaseRowSegmenter(const std::vector<TypeHolder>& key_types)
+      : key_types_(key_types) {}
+
+  const std::vector<TypeHolder>& key_types() const override { return key_types_; }
+
+  std::vector<TypeHolder> key_types_;
+};
+
+Segment MakeSegment(int64_t batch_length, int64_t offset, int64_t length, bool extends) {
+  return Segment{offset, length, offset + length >= batch_length, extends};
+}
+
+int64_t GetMatchLength(const uint8_t* match_bytes, int64_t match_width,
+                       const uint8_t* array_bytes, int64_t offset, int64_t length) {
+  int64_t cursor, byte_cursor;
+  for (cursor = offset, byte_cursor = match_width * cursor; cursor < length;
+       cursor++, byte_cursor += match_width) {
+    if (memcmp(match_bytes, array_bytes + byte_cursor,
+               static_cast<size_t>(match_width)) != 0) {
+      break;
+    }
+  }
+  return std::min(cursor, length) - offset;
+}
+
+using ExtendFunc = std::function<bool(const void*)>;
+constexpr bool kDefaultExtends = true;
+constexpr bool kEmptyExtends = true;
+
+struct NoKeysSegmenter : public BaseRowSegmenter {
+  static std::unique_ptr<RowSegmenter> Make() {
+    return std::make_unique<NoKeysSegmenter>();
+  }
+
+  NoKeysSegmenter() : BaseRowSegmenter({}) {}
+
+  Status Reset() override { return Status::OK(); }
+
+  Result<Segment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {}));
+    return MakeSegment(batch.length, offset, batch.length - offset, kDefaultExtends);
+  }
+};
+
+struct SimpleKeySegmenter : public BaseRowSegmenter {
+  static Result<std::unique_ptr<RowSegmenter>> Make(TypeHolder key_type) {
+    return std::make_unique<SimpleKeySegmenter>(key_type);
+  }
+
+  explicit SimpleKeySegmenter(TypeHolder key_type)
+      : BaseRowSegmenter({key_type}), key_type_(key_types_[0]), save_key_data_() {}
+
+  Status CheckType(const DataType& type) {
+    if (!is_fixed_width(type)) {
+      return Status::Invalid("SimpleKeySegmenter does not support type ", type);
+    }
+    return Status::OK();
+  }
+
+  Status Reset() override {
+    save_key_data_.resize(0);
+    return Status::OK();
+  }
+
+  // Checks whether the given grouping data extends the current segment, i.e., is equal to
+  // previously seen grouping data, which is updated with each invocation.
+  bool Extend(const void* data) {
+    size_t byte_width = static_cast<size_t>(key_type_.type->byte_width());
+    bool extends = save_key_data_.size() != byte_width
+                       ? kDefaultExtends
+                       : 0 == memcmp(save_key_data_.data(), data, byte_width);
+    save_key_data_.resize(byte_width);
+    memcpy(save_key_data_.data(), data, byte_width);
+    return extends;
+  }
+
+  Result<Segment> GetNextSegment(const Scalar& scalar, int64_t offset, int64_t length) {
+    ARROW_RETURN_NOT_OK(CheckType(*scalar.type));
+    if (!scalar.is_valid) {
+      return Status::Invalid("segmenting an invalid scalar");
+    }
+    auto data = checked_cast<const PrimitiveScalarBase&>(scalar).data();
+    bool extends = length > 0 ? Extend(data) : kEmptyExtends;
+    return MakeSegment(length, offset, length, extends);
+  }
+
+  Result<Segment> GetNextSegment(const DataType& array_type, const uint8_t* array_bytes,

Review Comment:
   Will do.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] icexelloss commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "icexelloss (via GitHub)" <gi...@apache.org>.
icexelloss commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1127153660


##########
cpp/src/arrow/compute/row/grouper.cc:
##########
@@ -39,12 +43,330 @@
 namespace arrow {
 
 using internal::checked_cast;
+using internal::PrimitiveScalarBase;
 
 namespace compute {
 
 namespace {
 
-struct GrouperImpl : Grouper {
+constexpr uint32_t kNoGroupId = std::numeric_limits<uint32_t>::max();
+
+using group_id_t = std::remove_const<decltype(kNoGroupId)>::type;
+using GroupIdType = CTypeTraits<group_id_t>::ArrowType;
+auto g_group_id_type = std::make_shared<GroupIdType>();
+
+inline const uint8_t* GetValuesAsBytes(const ArraySpan& data, int64_t offset = 0) {
+  DCHECK_GT(data.type->byte_width(), 0);
+  int64_t absolute_byte_offset = (data.offset + offset) * data.type->byte_width();
+  return data.GetValues<uint8_t>(1, absolute_byte_offset);
+}
+
+template <typename Value>
+Status CheckForGetNextSegment(const std::vector<Value>& values, int64_t length,
+                              int64_t offset, const std::vector<TypeHolder>& key_types) {
+  if (offset < 0 || offset > length) {
+    return Status::Invalid("invalid grouping segmenter offset: ", offset);
+  }
+  if (values.size() != key_types.size()) {
+    return Status::Invalid("expected batch size ", key_types.size(), " but got ",
+                           values.size());
+  }
+  for (size_t i = 0; i < key_types.size(); i++) {
+    const auto& value = values[i];
+    const auto& key_type = key_types[i];
+    if (*value.type() != *key_type.type) {
+      return Status::Invalid("expected batch value ", i, " of type ", *key_type.type,
+                             " but got ", *value.type());
+    }
+  }
+  return Status::OK();
+}
+
+template <typename Batch>
+enable_if_t<std::is_same<Batch, ExecSpan>::value || std::is_same<Batch, ExecBatch>::value,
+            Status>
+CheckForGetNextSegment(const Batch& batch, int64_t offset,
+                       const std::vector<TypeHolder>& key_types) {
+  return CheckForGetNextSegment(batch.values, batch.length, offset, key_types);
+}
+
+struct BaseRowSegmenter : public RowSegmenter {
+  explicit BaseRowSegmenter(const std::vector<TypeHolder>& key_types)
+      : key_types_(key_types) {}
+
+  const std::vector<TypeHolder>& key_types() const override { return key_types_; }
+
+  std::vector<TypeHolder> key_types_;
+};
+
+Segment MakeSegment(int64_t batch_length, int64_t offset, int64_t length, bool extends) {
+  return Segment{offset, length, offset + length >= batch_length, extends};
+}
+
+int64_t GetMatchLength(const uint8_t* match_bytes, int64_t match_width,
+                       const uint8_t* array_bytes, int64_t offset, int64_t length) {
+  int64_t cursor, byte_cursor;
+  for (cursor = offset, byte_cursor = match_width * cursor; cursor < length;
+       cursor++, byte_cursor += match_width) {
+    if (memcmp(match_bytes, array_bytes + byte_cursor,
+               static_cast<size_t>(match_width)) != 0) {
+      break;
+    }
+  }
+  return std::min(cursor, length) - offset;
+}
+
+using ExtendFunc = std::function<bool(const void*)>;
+constexpr bool kDefaultExtends = true;
+constexpr bool kEmptyExtends = true;
+
+struct NoKeysSegmenter : public BaseRowSegmenter {
+  static std::unique_ptr<RowSegmenter> Make() {
+    return std::make_unique<NoKeysSegmenter>();
+  }
+
+  NoKeysSegmenter() : BaseRowSegmenter({}) {}
+
+  Status Reset() override { return Status::OK(); }
+
+  Result<Segment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {}));
+    return MakeSegment(batch.length, offset, batch.length - offset, kDefaultExtends);
+  }
+};
+
+struct SimpleKeySegmenter : public BaseRowSegmenter {
+  static Result<std::unique_ptr<RowSegmenter>> Make(TypeHolder key_type) {
+    return std::make_unique<SimpleKeySegmenter>(key_type);
+  }
+
+  explicit SimpleKeySegmenter(TypeHolder key_type)
+      : BaseRowSegmenter({key_type}), key_type_(key_types_[0]), save_key_data_() {}
+
+  Status CheckType(const DataType& type) {
+    if (!is_fixed_width(type)) {
+      return Status::Invalid("SimpleKeySegmenter does not support type ", type);
+    }
+    return Status::OK();
+  }
+
+  Status Reset() override {
+    save_key_data_.resize(0);
+    return Status::OK();
+  }
+
+  // Checks whether the given grouping data extends the current segment, i.e., is equal to
+  // previously seen grouping data, which is updated with each invocation.
+  bool Extend(const void* data) {
+    size_t byte_width = static_cast<size_t>(key_type_.type->byte_width());
+    bool extends = save_key_data_.size() != byte_width
+                       ? kDefaultExtends
+                       : 0 == memcmp(save_key_data_.data(), data, byte_width);
+    save_key_data_.resize(byte_width);
+    memcpy(save_key_data_.data(), data, byte_width);
+    return extends;
+  }
+
+  Result<Segment> GetNextSegment(const Scalar& scalar, int64_t offset, int64_t length) {
+    ARROW_RETURN_NOT_OK(CheckType(*scalar.type));
+    if (!scalar.is_valid) {
+      return Status::Invalid("segmenting an invalid scalar");
+    }
+    auto data = checked_cast<const PrimitiveScalarBase&>(scalar).data();
+    bool extends = length > 0 ? Extend(data) : kEmptyExtends;
+    return MakeSegment(length, offset, length, extends);
+  }
+
+  Result<Segment> GetNextSegment(const DataType& array_type, const uint8_t* array_bytes,
+                                 int64_t offset, int64_t length) {
+    RETURN_NOT_OK(CheckType(array_type));
+    int64_t byte_width = array_type.byte_width();
+    int64_t match_length = GetMatchLength(array_bytes + offset * byte_width, byte_width,
+                                          array_bytes, offset, length);
+    bool extends = length > 0 ? Extend(array_bytes + offset * byte_width) : kEmptyExtends;
+    return MakeSegment(length, offset, match_length, extends);
+  }
+
+  Result<Segment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {key_type_}));
+    if (offset == batch.length) {
+      return MakeSegment(batch.length, offset, 0, kEmptyExtends);
+    }
+    const auto& value = batch.values[0];
+    if (value.is_scalar()) {
+      return GetNextSegment(*value.scalar, offset, batch.length);
+    }
+    ARROW_DCHECK(value.is_array());
+    const auto& array = value.array;
+    if (array.GetNullCount() > 0) {
+      return Status::NotImplemented("segmenting a nullable array");
+    }
+    return GetNextSegment(*array.type, GetValuesAsBytes(array), offset, batch.length);
+  }
+
+ private:
+  TypeHolder key_type_;
+  std::vector<uint8_t> save_key_data_;
+};
+
+struct AnyKeysSegmenter : public BaseRowSegmenter {
+  static Result<std::unique_ptr<RowSegmenter>> Make(
+      const std::vector<TypeHolder>& key_types, ExecContext* ctx) {
+    ARROW_RETURN_NOT_OK(Grouper::Make(key_types, ctx));  // check types
+    return std::make_unique<AnyKeysSegmenter>(key_types, ctx);
+  }
+
+  AnyKeysSegmenter(const std::vector<TypeHolder>& key_types, ExecContext* ctx)
+      : BaseRowSegmenter(key_types),
+        ctx_(ctx),
+        grouper_(nullptr),
+        save_group_id_(kNoGroupId) {}
+
+  Status Reset() override {

Review Comment:
   How would this be used in the future? Also, how is this tested if currently not used?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] icexelloss commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "icexelloss (via GitHub)" <gi...@apache.org>.
icexelloss commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1127153660


##########
cpp/src/arrow/compute/row/grouper.cc:
##########
@@ -39,12 +43,330 @@
 namespace arrow {
 
 using internal::checked_cast;
+using internal::PrimitiveScalarBase;
 
 namespace compute {
 
 namespace {
 
-struct GrouperImpl : Grouper {
+constexpr uint32_t kNoGroupId = std::numeric_limits<uint32_t>::max();
+
+using group_id_t = std::remove_const<decltype(kNoGroupId)>::type;
+using GroupIdType = CTypeTraits<group_id_t>::ArrowType;
+auto g_group_id_type = std::make_shared<GroupIdType>();
+
+inline const uint8_t* GetValuesAsBytes(const ArraySpan& data, int64_t offset = 0) {
+  DCHECK_GT(data.type->byte_width(), 0);
+  int64_t absolute_byte_offset = (data.offset + offset) * data.type->byte_width();
+  return data.GetValues<uint8_t>(1, absolute_byte_offset);
+}
+
+template <typename Value>
+Status CheckForGetNextSegment(const std::vector<Value>& values, int64_t length,
+                              int64_t offset, const std::vector<TypeHolder>& key_types) {
+  if (offset < 0 || offset > length) {
+    return Status::Invalid("invalid grouping segmenter offset: ", offset);
+  }
+  if (values.size() != key_types.size()) {
+    return Status::Invalid("expected batch size ", key_types.size(), " but got ",
+                           values.size());
+  }
+  for (size_t i = 0; i < key_types.size(); i++) {
+    const auto& value = values[i];
+    const auto& key_type = key_types[i];
+    if (*value.type() != *key_type.type) {
+      return Status::Invalid("expected batch value ", i, " of type ", *key_type.type,
+                             " but got ", *value.type());
+    }
+  }
+  return Status::OK();
+}
+
+template <typename Batch>
+enable_if_t<std::is_same<Batch, ExecSpan>::value || std::is_same<Batch, ExecBatch>::value,
+            Status>
+CheckForGetNextSegment(const Batch& batch, int64_t offset,
+                       const std::vector<TypeHolder>& key_types) {
+  return CheckForGetNextSegment(batch.values, batch.length, offset, key_types);
+}
+
+struct BaseRowSegmenter : public RowSegmenter {
+  explicit BaseRowSegmenter(const std::vector<TypeHolder>& key_types)
+      : key_types_(key_types) {}
+
+  const std::vector<TypeHolder>& key_types() const override { return key_types_; }
+
+  std::vector<TypeHolder> key_types_;
+};
+
+Segment MakeSegment(int64_t batch_length, int64_t offset, int64_t length, bool extends) {
+  return Segment{offset, length, offset + length >= batch_length, extends};
+}
+
+int64_t GetMatchLength(const uint8_t* match_bytes, int64_t match_width,
+                       const uint8_t* array_bytes, int64_t offset, int64_t length) {
+  int64_t cursor, byte_cursor;
+  for (cursor = offset, byte_cursor = match_width * cursor; cursor < length;
+       cursor++, byte_cursor += match_width) {
+    if (memcmp(match_bytes, array_bytes + byte_cursor,
+               static_cast<size_t>(match_width)) != 0) {
+      break;
+    }
+  }
+  return std::min(cursor, length) - offset;
+}
+
+using ExtendFunc = std::function<bool(const void*)>;
+constexpr bool kDefaultExtends = true;
+constexpr bool kEmptyExtends = true;
+
+struct NoKeysSegmenter : public BaseRowSegmenter {
+  static std::unique_ptr<RowSegmenter> Make() {
+    return std::make_unique<NoKeysSegmenter>();
+  }
+
+  NoKeysSegmenter() : BaseRowSegmenter({}) {}
+
+  Status Reset() override { return Status::OK(); }
+
+  Result<Segment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {}));
+    return MakeSegment(batch.length, offset, batch.length - offset, kDefaultExtends);
+  }
+};
+
+struct SimpleKeySegmenter : public BaseRowSegmenter {
+  static Result<std::unique_ptr<RowSegmenter>> Make(TypeHolder key_type) {
+    return std::make_unique<SimpleKeySegmenter>(key_type);
+  }
+
+  explicit SimpleKeySegmenter(TypeHolder key_type)
+      : BaseRowSegmenter({key_type}), key_type_(key_types_[0]), save_key_data_() {}
+
+  Status CheckType(const DataType& type) {
+    if (!is_fixed_width(type)) {
+      return Status::Invalid("SimpleKeySegmenter does not support type ", type);
+    }
+    return Status::OK();
+  }
+
+  Status Reset() override {
+    save_key_data_.resize(0);
+    return Status::OK();
+  }
+
+  // Checks whether the given grouping data extends the current segment, i.e., is equal to
+  // previously seen grouping data, which is updated with each invocation.
+  bool Extend(const void* data) {
+    size_t byte_width = static_cast<size_t>(key_type_.type->byte_width());
+    bool extends = save_key_data_.size() != byte_width
+                       ? kDefaultExtends
+                       : 0 == memcmp(save_key_data_.data(), data, byte_width);
+    save_key_data_.resize(byte_width);
+    memcpy(save_key_data_.data(), data, byte_width);
+    return extends;
+  }
+
+  Result<Segment> GetNextSegment(const Scalar& scalar, int64_t offset, int64_t length) {
+    ARROW_RETURN_NOT_OK(CheckType(*scalar.type));
+    if (!scalar.is_valid) {
+      return Status::Invalid("segmenting an invalid scalar");
+    }
+    auto data = checked_cast<const PrimitiveScalarBase&>(scalar).data();
+    bool extends = length > 0 ? Extend(data) : kEmptyExtends;
+    return MakeSegment(length, offset, length, extends);
+  }
+
+  Result<Segment> GetNextSegment(const DataType& array_type, const uint8_t* array_bytes,
+                                 int64_t offset, int64_t length) {
+    RETURN_NOT_OK(CheckType(array_type));
+    int64_t byte_width = array_type.byte_width();
+    int64_t match_length = GetMatchLength(array_bytes + offset * byte_width, byte_width,
+                                          array_bytes, offset, length);
+    bool extends = length > 0 ? Extend(array_bytes + offset * byte_width) : kEmptyExtends;
+    return MakeSegment(length, offset, match_length, extends);
+  }
+
+  Result<Segment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {key_type_}));
+    if (offset == batch.length) {
+      return MakeSegment(batch.length, offset, 0, kEmptyExtends);
+    }
+    const auto& value = batch.values[0];
+    if (value.is_scalar()) {
+      return GetNextSegment(*value.scalar, offset, batch.length);
+    }
+    ARROW_DCHECK(value.is_array());
+    const auto& array = value.array;
+    if (array.GetNullCount() > 0) {
+      return Status::NotImplemented("segmenting a nullable array");
+    }
+    return GetNextSegment(*array.type, GetValuesAsBytes(array), offset, batch.length);
+  }
+
+ private:
+  TypeHolder key_type_;
+  std::vector<uint8_t> save_key_data_;
+};
+
+struct AnyKeysSegmenter : public BaseRowSegmenter {
+  static Result<std::unique_ptr<RowSegmenter>> Make(
+      const std::vector<TypeHolder>& key_types, ExecContext* ctx) {
+    ARROW_RETURN_NOT_OK(Grouper::Make(key_types, ctx));  // check types
+    return std::make_unique<AnyKeysSegmenter>(key_types, ctx);
+  }
+
+  AnyKeysSegmenter(const std::vector<TypeHolder>& key_types, ExecContext* ctx)
+      : BaseRowSegmenter(key_types),
+        ctx_(ctx),
+        grouper_(nullptr),
+        save_group_id_(kNoGroupId) {}
+
+  Status Reset() override {

Review Comment:
   In your mind, how would this be used in the future? 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] icexelloss commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "icexelloss (via GitHub)" <gi...@apache.org>.
icexelloss commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1128135909


##########
cpp/src/arrow/compute/kernels/hash_aggregate_test.cc:
##########
@@ -4202,5 +4551,260 @@ TEST(GroupBy, OnlyKeys) {
                       /*verbose=*/true);
   }
 }
+
+INSTANTIATE_TEST_SUITE_P(GroupBy, GroupBy, ::testing::Values(RunGroupByImpl));
+
+class SegmentedScalarGroupBy : public GroupBy {};
+
+class SegmentedKeyGroupBy : public GroupBy {};
+
+void TestSegment(GroupByFunction group_by, const std::shared_ptr<Table>& table,
+                 Datum output, const std::vector<Datum>& keys,
+                 const std::vector<Datum>& segment_keys, bool scalar) {
+  const char* names[] = {
+      scalar ? "count" : "hash_count",
+      scalar ? "sum" : "hash_sum",
+      scalar ? "min_max" : "hash_min_max",
+  };
+  ASSERT_OK_AND_ASSIGN(Datum aggregated_and_grouped,
+                       group_by(
+                           {
+                               table->GetColumnByName("argument"),
+                               table->GetColumnByName("argument"),
+                               table->GetColumnByName("argument"),
+                           },
+                           keys, segment_keys,
+                           {
+                               {names[0], nullptr, "agg_0", names[0]},
+                               {names[1], nullptr, "agg_1", names[1]},
+                               {names[2], nullptr, "agg_2", names[2]},
+                           },
+                           kDefaultUseThreads, /*naive=*/false));
+
+  AssertDatumsEqual(output, aggregated_and_grouped, /*verbose=*/true);
+}
+
+void TestSegmentScalar(GroupByFunction group_by, const std::shared_ptr<Table>& table,

Review Comment:
   What is the difference between `TestSegmentScalar` and `TestSegmentKey`?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rtpsw commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "rtpsw (via GitHub)" <gi...@apache.org>.
rtpsw commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1122292888


##########
cpp/src/arrow/compute/row/grouper.h:
##########
@@ -30,6 +30,69 @@
 namespace arrow {
 namespace compute {
 
+/// \brief A segment of contiguous rows for grouping
+struct ARROW_EXPORT GroupingSegment {
+  /// \brief the offset into the batch where the segment starts
+  int64_t offset;
+  /// \brief the length of the segment
+  int64_t length;
+  /// \brief whether the segment may be extended by a next one
+  bool is_open;
+  /// \brief whether the segment extends a preceeding one
+  bool extends;
+};
+
+inline bool operator==(const GroupingSegment& segment1, const GroupingSegment& segment2) {
+  return segment1.offset == segment2.offset && segment1.length == segment2.length &&
+         segment1.is_open == segment2.is_open && segment1.extends == segment2.extends;
+}
+inline bool operator!=(const GroupingSegment& segment1, const GroupingSegment& segment2) {
+  return !(segment1 == segment2);
+}

Review Comment:
   `TestSegments` in `hash_aggregate_test.cc` uses the above `operator==`. I don't think `operator!=` is used right now.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rtpsw commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "rtpsw (via GitHub)" <gi...@apache.org>.
rtpsw commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1116865809


##########
cpp/src/arrow/compute/row/grouper.h:
##########
@@ -30,6 +30,49 @@
 namespace arrow {
 namespace compute {
 
+/// \brief A segment of contiguous rows for grouping
+struct ARROW_EXPORT GroupingSegment {
+  int64_t offset;
+  int64_t length;
+  bool is_open;
+  bool extends;
+};
+
+inline bool operator==(const GroupingSegment& segment1, const GroupingSegment& segment2) {
+  return segment1.offset == segment2.offset && segment1.length == segment2.length &&
+         segment1.is_open == segment2.is_open && segment1.extends == segment2.extends;
+}
+inline bool operator!=(const GroupingSegment& segment1, const GroupingSegment& segment2) {
+  return !(segment1 == segment2);
+}
+
+/// \brief Computes grouping segments for a batch. Each segment covers rows with identical
+/// values in the batch. The values in the batch are often selected as keys from a larger
+/// batch.
+class ARROW_EXPORT GroupingSegmenter {
+ public:
+  virtual ~GroupingSegmenter() = default;
+
+  /// \brief Construct a GroupingSegmenter which receives the specified key types
+  static Result<std::unique_ptr<GroupingSegmenter>> Make(
+      const std::vector<TypeHolder>& key_types, bool nullable_keys = false,
+      ExecContext* ctx = default_exec_context());
+
+  /// \brief Return the key types of this segmenter
+  virtual const std::vector<TypeHolder>& key_types() const = 0;

Review Comment:
   It behaves like [a type pointer](https://github.com/apache/arrow/blob/e8e50a3ca96ec8eb2f2beb383cf9005ce1c93486/cpp/src/arrow/type.h#L213-L214).



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] icexelloss commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "icexelloss (via GitHub)" <gi...@apache.org>.
icexelloss commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1117313249


##########
cpp/src/arrow/compute/exec/aggregate_node.cc:
##########
@@ -326,46 +446,86 @@ class ScalarAggregateNode : public ExecNode, public TracedNode {
   }
 
  private:
-  Status Finish() {
-    auto scope = TraceFinish();
+  Status ReconstructAggregates() {
+    const auto& input_schema = *inputs()[0]->output_schema();
+    auto exec_ctx = plan()->query_context()->exec_context();
+    for (size_t i = 0; i < kernels_.size(); ++i) {
+      std::vector<TypeHolder> in_types;
+      for (const auto& target : target_fieldsets_[i]) {
+        in_types.emplace_back(input_schema.field(target)->type().get());
+      }
+      states_[i].resize(plan()->query_context()->max_concurrency());
+      KernelContext kernel_ctx{exec_ctx};
+      RETURN_NOT_OK(Kernel::InitAll(
+          &kernel_ctx, KernelInitArgs{kernels_[i], in_types, aggs_[i].options.get()},
+          &states_[i]));
+    }
+    return Status::OK();
+  }
+
+  Status OutputResult(bool is_last = false, bool traced = false) {
+    if (is_last && !traced) {
+      auto scope = TraceFinish();
+      return OutputResult(is_last, /*traced=*/true);

Review Comment:
   Is this the same as continuing in the function?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] westonpace commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "westonpace (via GitHub)" <gi...@apache.org>.
westonpace commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1117677195


##########
cpp/src/arrow/compute/row/grouper.cc:
##########
@@ -39,12 +43,521 @@
 namespace arrow {
 
 using internal::checked_cast;
+using internal::PrimitiveScalarBase;
 
 namespace compute {
 
 namespace {
 
-struct GrouperImpl : Grouper {
+constexpr uint32_t kNoGroupId = std::numeric_limits<uint32_t>::max();
+
+using group_id_t = std::remove_const<decltype(kNoGroupId)>::type;
+using GroupIdType = CTypeTraits<group_id_t>::ArrowType;
+auto group_id_type = std::make_shared<GroupIdType>();
+
+inline const uint8_t* GetValuesAsBytes(const ArrayData& data, int64_t offset = 0) {
+  DCHECK_GT(data.type->byte_width(), 0);
+  int64_t absolute_byte_offset = (data.offset + offset) * data.type->byte_width();
+  return data.GetValues<uint8_t>(1, absolute_byte_offset);
+}
+
+inline const uint8_t* GetValuesAsBytes(const ArraySpan& data, int64_t offset = 0) {
+  DCHECK_GT(data.type->byte_width(), 0);
+  int64_t absolute_byte_offset = (data.offset + offset) * data.type->byte_width();
+  return data.GetValues<uint8_t>(1, absolute_byte_offset);
+}
+
+template <typename Value>
+Status CheckForGetNextSegment(const std::vector<Value>& values, int64_t length,
+                              int64_t offset, const std::vector<TypeHolder>& key_types) {
+  if (offset < 0 || offset > length) {
+    return Status::Invalid("invalid grouping segmenter offset: ", offset);
+  }
+  if (values.size() != key_types.size()) {
+    return Status::Invalid("expected batch size ", key_types.size(), " but got ",
+                           values.size());
+  }
+  for (size_t i = 0; i < key_types.size(); i++) {
+    const auto& value = values[i];
+    const auto& key_type = key_types[i];
+    if (*value.type() != *key_type.type) {
+      return Status::Invalid("expected batch value ", i, " of type ", *key_type.type,
+                             " but got ", *value.type());
+    }
+  }
+  return Status::OK();
+}
+
+template <typename Batch>
+enable_if_t<std::is_same<Batch, ExecSpan>::value || std::is_same<Batch, ExecBatch>::value,
+            Status>
+CheckForGetNextSegment(const Batch& batch, int64_t offset,
+                       const std::vector<TypeHolder>& key_types) {
+  return CheckForGetNextSegment(batch.values, batch.length, offset, key_types);
+}
+
+struct BaseGroupingSegmenter : public GroupingSegmenter {
+  explicit BaseGroupingSegmenter(const std::vector<TypeHolder>& key_types)
+      : key_types_(key_types) {}
+
+  const std::vector<TypeHolder>& key_types() const override { return key_types_; }
+
+  std::vector<TypeHolder> key_types_;
+};
+
+GroupingSegment MakeSegment(int64_t batch_length, int64_t offset, int64_t length,
+                            bool extends) {
+  return GroupingSegment{offset, length, offset + length >= batch_length, extends};
+}
+
+int64_t GetMatchLength(const uint8_t* match_bytes, int64_t match_width,

Review Comment:
   Segmented is a weaker requirement than sorted.
   
   I agree binary search cannot be used here since, at the moment, we are only assuming the data is segmented.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] westonpace commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "westonpace (via GitHub)" <gi...@apache.org>.
westonpace commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1117684110


##########
cpp/src/arrow/compute/exec/options.h:
##########
@@ -199,21 +199,32 @@ class ARROW_EXPORT ProjectNodeOptions : public ExecNodeOptions {
   std::vector<std::string> names;
 };
 
-/// \brief Make a node which aggregates input batches, optionally grouped by keys.
+/// \brief Make a node which aggregates input batches, optionally grouped by keys and
+/// optionally segmented by segment-keys. Both keys and segment-keys determine the group.
+/// However segment-keys are also used for determining grouping segments, which should be
+/// large, and allow streaming a partial aggregation result after processing each segment.
+/// One common use-case for segment-keys is ordered aggregation, in which the segment-key
+/// attribute specifies a column with non-decreasing values or a lexigographically-ordered
+/// set of such columns.
 ///
 /// If the keys attribute is a non-empty vector, then each aggregate in `aggregates` is
 /// expected to be a HashAggregate function. If the keys attribute is an empty vector,
 /// then each aggregate is assumed to be a ScalarAggregate function.
 class ARROW_EXPORT AggregateNodeOptions : public ExecNodeOptions {
  public:
   explicit AggregateNodeOptions(std::vector<Aggregate> aggregates,
-                                std::vector<FieldRef> keys = {})
-      : aggregates(std::move(aggregates)), keys(std::move(keys)) {}
+                                std::vector<FieldRef> keys = {},
+                                std::vector<FieldRef> segment_keys = {})
+      : aggregates(std::move(aggregates)),
+        keys(std::move(keys)),
+        segment_keys(std::move(segment_keys)) {}
 
   // aggregations which will be applied to the targetted fields
   std::vector<Aggregate> aggregates;
   // keys by which aggregations will be grouped
   std::vector<FieldRef> keys;
+  // keys by which aggregations will be segmented
+  std::vector<FieldRef> segment_keys;

Review Comment:
   Ok.  So if I want to group by A, B, C and I know my data is sorted by A then I can do segments=A and keys=B C?
   
   In that case I agree we should explicitly disallow this to help future users.  Also, we might want to improve the description of these fields a bit and provide some examples, but that isn't urgent for right now.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rtpsw commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "rtpsw (via GitHub)" <gi...@apache.org>.
rtpsw commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1117884033


##########
cpp/src/arrow/compute/row/grouper.h:
##########
@@ -30,6 +30,49 @@
 namespace arrow {
 namespace compute {
 
+/// \brief A segment of contiguous rows for grouping
+struct ARROW_EXPORT GroupingSegment {
+  int64_t offset;
+  int64_t length;
+  bool is_open;
+  bool extends;
+};
+
+inline bool operator==(const GroupingSegment& segment1, const GroupingSegment& segment2) {
+  return segment1.offset == segment2.offset && segment1.length == segment2.length &&
+         segment1.is_open == segment2.is_open && segment1.extends == segment2.extends;
+}
+inline bool operator!=(const GroupingSegment& segment1, const GroupingSegment& segment2) {
+  return !(segment1 == segment2);
+}
+
+/// \brief Computes grouping segments for a batch. Each segment covers rows with identical
+/// values in the batch. The values in the batch are often selected as keys from a larger
+/// batch.
+class ARROW_EXPORT GroupingSegmenter {
+ public:
+  virtual ~GroupingSegmenter() = default;
+
+  /// \brief Construct a GroupingSegmenter which receives the specified key types
+  static Result<std::unique_ptr<GroupingSegmenter>> Make(
+      const std::vector<TypeHolder>& key_types, bool nullable_keys = false,
+      ExecContext* ctx = default_exec_context());
+
+  /// \brief Return the key types of this segmenter
+  virtual const std::vector<TypeHolder>& key_types() const = 0;
+
+  /// \brief Reset this grouping segmenter
+  virtual Status Reset() = 0;
+
+  /// \brief Get the next segment for the given batch starting from the given offset
+  virtual Result<GroupingSegment> GetNextSegment(const ExecSpan& batch,
+                                                 int64_t offset) = 0;
+
+  /// \brief Get the next segment for the given batch starting from the given offset
+  virtual Result<GroupingSegment> GetNextSegment(const ExecBatch& batch,

Review Comment:
   One needs `ExecBatch` when it contains chunked arrays and `ExecSpan` cannot capture the view. Since we removed support for chunked arrays, I think the `ExecBatch` API can be removed too.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rtpsw commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "rtpsw (via GitHub)" <gi...@apache.org>.
rtpsw commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1117891738


##########
cpp/src/arrow/compute/exec/aggregate_node.cc:
##########
@@ -326,46 +446,86 @@ class ScalarAggregateNode : public ExecNode, public TracedNode {
   }
 
  private:
-  Status Finish() {
-    auto scope = TraceFinish();
+  Status ReconstructAggregates() {
+    const auto& input_schema = *inputs()[0]->output_schema();
+    auto exec_ctx = plan()->query_context()->exec_context();
+    for (size_t i = 0; i < kernels_.size(); ++i) {
+      std::vector<TypeHolder> in_types;

Review Comment:
   Fixed.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] icexelloss commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "icexelloss (via GitHub)" <gi...@apache.org>.
icexelloss commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1117293000


##########
cpp/src/arrow/compute/exec/aggregate_node.cc:
##########
@@ -697,16 +925,22 @@ class GroupByNode : public ExecNode, public TracedNode {
   }
 
   int output_task_group_id_;
+  std::unique_ptr<GroupingSegmenter> segmenter_;
+  std::vector<Datum> segmenter_values_;
 
   const std::vector<int> key_field_ids_;
+  const std::vector<int> segment_key_field_ids_;
+  const std::vector<std::vector<TypeHolder>> agg_src_types_;
   const std::vector<std::vector<int>> agg_src_fieldsets_;
   const std::vector<Aggregate> aggs_;
-  const std::vector<const HashAggregateKernel*> agg_kernels_;
+  std::vector<const HashAggregateKernel*> agg_kernels_;

Review Comment:
   Why removing const?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rtpsw commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "rtpsw (via GitHub)" <gi...@apache.org>.
rtpsw commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1117886302


##########
cpp/src/arrow/compute/row/grouper.cc:
##########
@@ -39,12 +43,521 @@
 namespace arrow {
 
 using internal::checked_cast;
+using internal::PrimitiveScalarBase;
 
 namespace compute {
 
 namespace {
 
-struct GrouperImpl : Grouper {
+constexpr uint32_t kNoGroupId = std::numeric_limits<uint32_t>::max();
+
+using group_id_t = std::remove_const<decltype(kNoGroupId)>::type;
+using GroupIdType = CTypeTraits<group_id_t>::ArrowType;
+auto group_id_type = std::make_shared<GroupIdType>();
+
+inline const uint8_t* GetValuesAsBytes(const ArrayData& data, int64_t offset = 0) {
+  DCHECK_GT(data.type->byte_width(), 0);
+  int64_t absolute_byte_offset = (data.offset + offset) * data.type->byte_width();
+  return data.GetValues<uint8_t>(1, absolute_byte_offset);
+}
+
+inline const uint8_t* GetValuesAsBytes(const ArraySpan& data, int64_t offset = 0) {
+  DCHECK_GT(data.type->byte_width(), 0);
+  int64_t absolute_byte_offset = (data.offset + offset) * data.type->byte_width();
+  return data.GetValues<uint8_t>(1, absolute_byte_offset);
+}
+
+template <typename Value>
+Status CheckForGetNextSegment(const std::vector<Value>& values, int64_t length,
+                              int64_t offset, const std::vector<TypeHolder>& key_types) {
+  if (offset < 0 || offset > length) {
+    return Status::Invalid("invalid grouping segmenter offset: ", offset);
+  }
+  if (values.size() != key_types.size()) {
+    return Status::Invalid("expected batch size ", key_types.size(), " but got ",
+                           values.size());
+  }
+  for (size_t i = 0; i < key_types.size(); i++) {
+    const auto& value = values[i];
+    const auto& key_type = key_types[i];
+    if (*value.type() != *key_type.type) {
+      return Status::Invalid("expected batch value ", i, " of type ", *key_type.type,
+                             " but got ", *value.type());
+    }
+  }
+  return Status::OK();
+}
+
+template <typename Batch>
+enable_if_t<std::is_same<Batch, ExecSpan>::value || std::is_same<Batch, ExecBatch>::value,
+            Status>
+CheckForGetNextSegment(const Batch& batch, int64_t offset,
+                       const std::vector<TypeHolder>& key_types) {
+  return CheckForGetNextSegment(batch.values, batch.length, offset, key_types);
+}
+
+struct BaseGroupingSegmenter : public GroupingSegmenter {
+  explicit BaseGroupingSegmenter(const std::vector<TypeHolder>& key_types)
+      : key_types_(key_types) {}
+
+  const std::vector<TypeHolder>& key_types() const override { return key_types_; }
+
+  std::vector<TypeHolder> key_types_;
+};
+
+GroupingSegment MakeSegment(int64_t batch_length, int64_t offset, int64_t length,
+                            bool extends) {
+  return GroupingSegment{offset, length, offset + length >= batch_length, extends};
+}
+
+int64_t GetMatchLength(const uint8_t* match_bytes, int64_t match_width,
+                       const uint8_t* array_bytes, int64_t offset, int64_t length) {
+  int64_t cursor, byte_cursor;
+  for (cursor = offset, byte_cursor = match_width * cursor; cursor < length;
+       cursor++, byte_cursor += match_width) {
+    if (memcmp(match_bytes, array_bytes + byte_cursor,
+               static_cast<size_t>(match_width)) != 0) {
+      break;
+    }
+  }
+  return std::min(cursor, length - offset);
+}
+
+using ExtendFunc = std::function<bool(const void*)>;
+constexpr bool kDefaultExtends = true;
+constexpr bool kEmptyExtends = true;
+
+Result<GroupingSegment> GetNextSegmentChunked(
+    const std::shared_ptr<ChunkedArray>& chunked_array, int64_t offset,
+    ExtendFunc extend) {
+  if (offset >= chunked_array->length()) {
+    return MakeSegment(chunked_array->length(), chunked_array->length(), 0,
+                       kEmptyExtends);
+  }
+  int64_t remaining_offset = offset;
+  const auto& arrays = chunked_array->chunks();
+  for (size_t i = 0; remaining_offset >= 0 && i < arrays.size(); i++) {
+    // look up chunk containing offset
+    int64_t array_length = arrays[i]->length();
+    if (remaining_offset < array_length) {
+      // found - switch to matching
+      int64_t match_width = arrays[i]->type()->byte_width();
+      const uint8_t* match_bytes = GetValuesAsBytes(*arrays[i]->data(), remaining_offset);
+      int64_t total_match_length = 0;
+      for (; i < arrays.size(); i++) {
+        int64_t array_length = arrays[i]->length();
+        if (array_length <= 0) continue;
+        const uint8_t* array_bytes = GetValuesAsBytes(*arrays[i]->data());
+        int64_t match_length = GetMatchLength(match_bytes, match_width, array_bytes,
+                                              remaining_offset, array_length);
+        total_match_length += match_length;
+        remaining_offset = 0;
+        if (match_length < array_length - remaining_offset) break;
+      }
+      bool extends = extend(match_bytes);
+      return MakeSegment(chunked_array->length(), offset, total_match_length, extends);
+    }
+    remaining_offset -= array_length;
+  }
+  return Status::Invalid("segmenting invalid chunked array value");
+}
+
+struct NoKeysGroupingSegmenter : public BaseGroupingSegmenter {
+  static std::unique_ptr<GroupingSegmenter> Make() {
+    return std::make_unique<NoKeysGroupingSegmenter>();
+  }
+
+  NoKeysGroupingSegmenter() : BaseGroupingSegmenter({}) {}
+
+  Status Reset() override { return Status::OK(); }
+
+  template <typename Batch>
+  Result<GroupingSegment> GetNextSegmentImpl(const Batch& batch, int64_t offset) {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {}));
+    return MakeSegment(batch.length, offset, batch.length - offset, kDefaultExtends);
+  }
+
+  Result<GroupingSegment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    return GetNextSegmentImpl(batch, offset);
+  }
+
+  Result<GroupingSegment> GetNextSegment(const ExecBatch& batch,
+                                         int64_t offset) override {
+    return GetNextSegmentImpl(batch, offset);
+  }
+};
+
+struct SimpleKeyGroupingSegmenter : public BaseGroupingSegmenter {
+  static Result<std::unique_ptr<GroupingSegmenter>> Make(TypeHolder key_type) {
+    return std::make_unique<SimpleKeyGroupingSegmenter>(key_type);
+  }
+
+  explicit SimpleKeyGroupingSegmenter(TypeHolder key_type)
+      : BaseGroupingSegmenter({key_type}), key_type_(key_types_[0]), save_key_data_() {}
+
+  Status CheckType(const DataType& type) {
+    if (!is_fixed_width(type)) {
+      return Status::Invalid("SimpleKeyGroupingSegmenter does not support type ", type);
+    }
+    return Status::OK();
+  }
+
+  Status Reset() override {
+    save_key_data_.resize(0);
+    return Status::OK();
+  }
+
+  bool Extend(const void* data) {
+    size_t byte_width = static_cast<size_t>(key_type_.type->byte_width());
+    bool extends = save_key_data_.size() != byte_width
+                       ? kDefaultExtends
+                       : 0 == memcmp(save_key_data_.data(), data, byte_width);
+    save_key_data_.resize(byte_width);
+    memcpy(save_key_data_.data(), data, byte_width);
+    return extends;
+  }
+
+  Result<GroupingSegment> GetNextSegment(const Scalar& scalar, int64_t offset,
+                                         int64_t length) {
+    ARROW_RETURN_NOT_OK(CheckType(*scalar.type));
+    if (!scalar.is_valid) {
+      return Status::Invalid("segmenting an invalid scalar");
+    }
+    auto data = checked_cast<const PrimitiveScalarBase&>(scalar).data();
+    bool extends = length > 0 ? Extend(data) : kEmptyExtends;
+    return MakeSegment(length, 0, length, extends);
+  }
+
+  Result<GroupingSegment> GetNextSegment(const DataType& array_type,
+                                         const uint8_t* array_bytes, int64_t offset,
+                                         int64_t length) {
+    RETURN_NOT_OK(CheckType(array_type));
+    int64_t byte_width = array_type.byte_width();
+    int64_t match_length = GetMatchLength(array_bytes + offset * byte_width, byte_width,
+                                          array_bytes, offset, length);
+    bool extends = length > 0 ? Extend(array_bytes + offset * byte_width) : kEmptyExtends;
+    return MakeSegment(length, offset, match_length, extends);
+  }
+
+  Result<GroupingSegment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {key_type_}));
+    if (offset == batch.length) {
+      return MakeSegment(batch.length, offset, 0, kEmptyExtends);
+    }
+    const auto& value = batch.values[0];
+    if (value.is_scalar()) {
+      return GetNextSegment(*value.scalar, offset, batch.length);
+    }
+    ARROW_DCHECK(value.is_array());
+    const auto& array = value.array;
+    if (array.GetNullCount() > 0) {
+      return Status::NotImplemented("segmenting a nullable array");
+    }
+    return GetNextSegment(*array.type, GetValuesAsBytes(array), offset, batch.length);
+  }
+
+  Result<GroupingSegment> GetNextSegment(const ExecBatch& batch,
+                                         int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {key_type_}));
+    if (offset == batch.length) {
+      return MakeSegment(batch.length, offset, 0, kEmptyExtends);
+    }
+    const auto& value = batch.values[0];
+    if (value.is_scalar()) {
+      return GetNextSegment(*value.scalar(), offset, batch.length);
+    }
+    if (value.is_array()) {
+      auto array = value.array();
+      if (array->GetNullCount() > 0) {
+        return Status::NotImplemented("segmenting a nullable array");
+      }
+      return GetNextSegment(*array->type, GetValuesAsBytes(*array), offset, batch.length);
+    }
+    if (value.is_chunked_array()) {
+      auto array = value.chunked_array();
+      if (array->null_count() > 0) {
+        return Status::NotImplemented("segmenting a nullable array");
+      }
+      return GetNextSegmentChunked(array, offset, bound_extend_);
+    }
+    return Status::Invalid("segmenting unsupported value kind ", value.kind());
+  }
+
+ private:
+  TypeHolder key_type_;
+  std::vector<uint8_t> save_key_data_;
+  ExtendFunc bound_extend_ = [this](const void* data) { return Extend(data); };
+};
+
+struct AnyKeysGroupingSegmenter : public BaseGroupingSegmenter {
+  static Result<std::unique_ptr<GroupingSegmenter>> Make(
+      const std::vector<TypeHolder>& key_types, ExecContext* ctx) {
+    ARROW_RETURN_NOT_OK(Grouper::Make(key_types, ctx));  // check types
+    return std::make_unique<AnyKeysGroupingSegmenter>(key_types, ctx);
+  }
+
+  AnyKeysGroupingSegmenter(const std::vector<TypeHolder>& key_types, ExecContext* ctx)
+      : BaseGroupingSegmenter(key_types),
+        ctx_(ctx),
+        grouper_(nullptr),
+        save_group_id_(kNoGroupId) {}
+
+  Status Reset() override {
+    grouper_ = nullptr;
+    save_group_id_ = kNoGroupId;
+    return Status::OK();
+  }
+
+  bool Extend(const void* data) {
+    auto group_id = *static_cast<const group_id_t*>(data);
+    bool extends =
+        save_group_id_ == kNoGroupId ? kDefaultExtends : save_group_id_ == group_id;
+    save_group_id_ = group_id;
+    return extends;
+  }
+
+  template <typename Batch>
+  Result<group_id_t> MapGroupIdAt(const Batch& batch, int64_t offset) {
+    if (offset < 0 || offset >= batch.length) {
+      return Status::Invalid("requesting group id out of bounds");
+    }
+    if (!grouper_) return kNoGroupId;
+    ARROW_ASSIGN_OR_RAISE(auto datum, grouper_->Consume(batch, offset,
+                                                        /*consume_length=*/1));
+    if (!(datum.is_array() || datum.is_chunked_array())) {
+      return Status::Invalid("accessing unsupported datum kind ", datum.kind());
+    }
+    const std::shared_ptr<ArrayData>& data =
+        datum.is_array() ? datum.array() : datum.chunked_array()->chunk(0)->data();
+    ARROW_DCHECK(data->GetNullCount() == 0);
+    DCHECK_EQ(data->type->id(), GroupIdType::type_id);
+    DCHECK_EQ(1, data->length);
+    const group_id_t* values = data->GetValues<group_id_t>(1);
+    return values[0];
+  }
+
+  template <typename Batch>
+  Result<GroupingSegment> GetNextSegmentImpl(const Batch& batch, int64_t offset) {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, key_types_));
+    if (offset == batch.length) {
+      return MakeSegment(batch.length, offset, 0, kEmptyExtends);
+    }
+    // ARROW-18311: make Grouper support Reset()
+    // so it can be cached instead of recreated below
+    //
+    // the group id must be computed prior to resetting the grouper, since it is compared
+    // to save_group_id_, and after resetting the grouper produces incomparable group ids
+    ARROW_ASSIGN_OR_RAISE(auto group_id, MapGroupIdAt(batch, offset));
+    ExtendFunc bound_extend = [this, group_id](const void* data) {
+      bool extends = Extend(&group_id);
+      save_group_id_ = *static_cast<const group_id_t*>(data);
+      return extends;
+    };
+    ARROW_ASSIGN_OR_RAISE(grouper_, Grouper::Make(key_types_, ctx_));  // TODO: reset it
+    ARROW_ASSIGN_OR_RAISE(auto datum, grouper_->Consume(batch, offset));
+    if (datum.is_array()) {
+      const std::shared_ptr<ArrayData>& data = datum.array();
+      ARROW_DCHECK(data->GetNullCount() == 0);
+      DCHECK_EQ(data->type->id(), GroupIdType::type_id);
+      const group_id_t* values = data->GetValues<group_id_t>(1);
+      int64_t cursor;
+      for (cursor = 1; cursor < data->length; cursor++) {
+        if (values[0] != values[cursor]) break;
+      }
+      int64_t length = std::min(cursor, batch.length - offset);
+      bool extends = length > 0 ? bound_extend(values) : kEmptyExtends;
+      return MakeSegment(batch.length, offset, length, extends);
+    } else if (datum.is_chunked_array()) {
+      ARROW_ASSIGN_OR_RAISE(
+          auto segment, GetNextSegmentChunked(datum.chunked_array(), 0, bound_extend));
+      segment.offset += offset;
+      return segment;

Review Comment:
   Done.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] icexelloss commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "icexelloss (via GitHub)" <gi...@apache.org>.
icexelloss commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1123736880


##########
cpp/src/arrow/compute/row/grouper.cc:
##########
@@ -39,12 +43,336 @@
 namespace arrow {
 
 using internal::checked_cast;
+using internal::PrimitiveScalarBase;
 
 namespace compute {
 
 namespace {
 
-struct GrouperImpl : Grouper {
+constexpr uint32_t kNoGroupId = std::numeric_limits<uint32_t>::max();
+
+using group_id_t = std::remove_const<decltype(kNoGroupId)>::type;
+using GroupIdType = CTypeTraits<group_id_t>::ArrowType;
+auto g_group_id_type = std::make_shared<GroupIdType>();
+
+inline const uint8_t* GetValuesAsBytes(const ArraySpan& data, int64_t offset = 0) {
+  DCHECK_GT(data.type->byte_width(), 0);
+  int64_t absolute_byte_offset = (data.offset + offset) * data.type->byte_width();
+  return data.GetValues<uint8_t>(1, absolute_byte_offset);
+}
+
+template <typename Value>
+Status CheckForGetNextSegment(const std::vector<Value>& values, int64_t length,
+                              int64_t offset, const std::vector<TypeHolder>& key_types) {
+  if (offset < 0 || offset > length) {
+    return Status::Invalid("invalid grouping segmenter offset: ", offset);
+  }
+  if (values.size() != key_types.size()) {
+    return Status::Invalid("expected batch size ", key_types.size(), " but got ",
+                           values.size());
+  }
+  for (size_t i = 0; i < key_types.size(); i++) {
+    const auto& value = values[i];
+    const auto& key_type = key_types[i];
+    if (*value.type() != *key_type.type) {
+      return Status::Invalid("expected batch value ", i, " of type ", *key_type.type,
+                             " but got ", *value.type());
+    }
+  }
+  return Status::OK();
+}
+
+template <typename Batch>
+enable_if_t<std::is_same<Batch, ExecSpan>::value || std::is_same<Batch, ExecBatch>::value,
+            Status>
+CheckForGetNextSegment(const Batch& batch, int64_t offset,
+                       const std::vector<TypeHolder>& key_types) {
+  return CheckForGetNextSegment(batch.values, batch.length, offset, key_types);
+}
+
+struct BaseGroupingSegmenter : public GroupingSegmenter {
+  explicit BaseGroupingSegmenter(const std::vector<TypeHolder>& key_types)
+      : key_types_(key_types) {}
+
+  const std::vector<TypeHolder>& key_types() const override { return key_types_; }
+
+  std::vector<TypeHolder> key_types_;
+};
+
+GroupingSegment MakeSegment(int64_t batch_length, int64_t offset, int64_t length,
+                            bool extends) {
+  return GroupingSegment{offset, length, offset + length >= batch_length, extends};
+}
+
+int64_t GetMatchLength(const uint8_t* match_bytes, int64_t match_width,
+                       const uint8_t* array_bytes, int64_t offset, int64_t length) {
+  int64_t cursor, byte_cursor;
+  for (cursor = offset, byte_cursor = match_width * cursor; cursor < length;
+       cursor++, byte_cursor += match_width) {
+    if (memcmp(match_bytes, array_bytes + byte_cursor,
+               static_cast<size_t>(match_width)) != 0) {
+      break;
+    }
+  }
+  return std::min(cursor, length - offset);
+}
+
+using ExtendFunc = std::function<bool(const void*)>;
+constexpr bool kDefaultExtends = true;
+constexpr bool kEmptyExtends = true;
+
+struct NoKeysGroupingSegmenter : public BaseGroupingSegmenter {
+  static std::unique_ptr<GroupingSegmenter> Make() {
+    return std::make_unique<NoKeysGroupingSegmenter>();
+  }
+
+  NoKeysGroupingSegmenter() : BaseGroupingSegmenter({}) {}
+
+  Status Reset() override { return Status::OK(); }
+
+  Result<GroupingSegment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {}));
+    return MakeSegment(batch.length, offset, batch.length - offset, kDefaultExtends);
+  }
+};
+
+struct SimpleKeyGroupingSegmenter : public BaseGroupingSegmenter {

Review Comment:
   Why do we need NoKey/SimpleKey/AnyKeys segmenter instead of just using one? 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rtpsw commented on pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "rtpsw (via GitHub)" <gi...@apache.org>.
rtpsw commented on PR #34311:
URL: https://github.com/apache/arrow/pull/34311#issuecomment-1454898853

   > Also, it seems the testing of the new segmenter classes are lacky - can you add some unittest for the segmenter classes? (particular to cover edge cases)
   
   The current test-cases cover several edge cases, including various errors. I'll see what I can come up with to cover more. What edge cases did you have in mind?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] icexelloss commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "icexelloss (via GitHub)" <gi...@apache.org>.
icexelloss commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1127957690


##########
cpp/src/arrow/compute/row/grouper.cc:
##########
@@ -39,12 +43,330 @@
 namespace arrow {
 
 using internal::checked_cast;
+using internal::PrimitiveScalarBase;
 
 namespace compute {
 
 namespace {
 
-struct GrouperImpl : Grouper {
+constexpr uint32_t kNoGroupId = std::numeric_limits<uint32_t>::max();
+
+using group_id_t = std::remove_const<decltype(kNoGroupId)>::type;
+using GroupIdType = CTypeTraits<group_id_t>::ArrowType;
+auto g_group_id_type = std::make_shared<GroupIdType>();
+
+inline const uint8_t* GetValuesAsBytes(const ArraySpan& data, int64_t offset = 0) {
+  DCHECK_GT(data.type->byte_width(), 0);
+  int64_t absolute_byte_offset = (data.offset + offset) * data.type->byte_width();
+  return data.GetValues<uint8_t>(1, absolute_byte_offset);
+}
+
+template <typename Value>
+Status CheckForGetNextSegment(const std::vector<Value>& values, int64_t length,
+                              int64_t offset, const std::vector<TypeHolder>& key_types) {
+  if (offset < 0 || offset > length) {
+    return Status::Invalid("invalid grouping segmenter offset: ", offset);
+  }
+  if (values.size() != key_types.size()) {
+    return Status::Invalid("expected batch size ", key_types.size(), " but got ",
+                           values.size());
+  }
+  for (size_t i = 0; i < key_types.size(); i++) {
+    const auto& value = values[i];
+    const auto& key_type = key_types[i];
+    if (*value.type() != *key_type.type) {
+      return Status::Invalid("expected batch value ", i, " of type ", *key_type.type,
+                             " but got ", *value.type());
+    }
+  }
+  return Status::OK();
+}
+
+template <typename Batch>
+enable_if_t<std::is_same<Batch, ExecSpan>::value || std::is_same<Batch, ExecBatch>::value,
+            Status>
+CheckForGetNextSegment(const Batch& batch, int64_t offset,
+                       const std::vector<TypeHolder>& key_types) {
+  return CheckForGetNextSegment(batch.values, batch.length, offset, key_types);
+}
+
+struct BaseRowSegmenter : public RowSegmenter {
+  explicit BaseRowSegmenter(const std::vector<TypeHolder>& key_types)
+      : key_types_(key_types) {}
+
+  const std::vector<TypeHolder>& key_types() const override { return key_types_; }
+
+  std::vector<TypeHolder> key_types_;
+};
+
+Segment MakeSegment(int64_t batch_length, int64_t offset, int64_t length, bool extends) {
+  return Segment{offset, length, offset + length >= batch_length, extends};
+}
+
+int64_t GetMatchLength(const uint8_t* match_bytes, int64_t match_width,
+                       const uint8_t* array_bytes, int64_t offset, int64_t length) {
+  int64_t cursor, byte_cursor;
+  for (cursor = offset, byte_cursor = match_width * cursor; cursor < length;
+       cursor++, byte_cursor += match_width) {
+    if (memcmp(match_bytes, array_bytes + byte_cursor,
+               static_cast<size_t>(match_width)) != 0) {
+      break;
+    }
+  }
+  return std::min(cursor, length) - offset;
+}
+
+using ExtendFunc = std::function<bool(const void*)>;
+constexpr bool kDefaultExtends = true;
+constexpr bool kEmptyExtends = true;
+
+struct NoKeysSegmenter : public BaseRowSegmenter {
+  static std::unique_ptr<RowSegmenter> Make() {
+    return std::make_unique<NoKeysSegmenter>();
+  }
+
+  NoKeysSegmenter() : BaseRowSegmenter({}) {}
+
+  Status Reset() override { return Status::OK(); }
+
+  Result<Segment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {}));
+    return MakeSegment(batch.length, offset, batch.length - offset, kDefaultExtends);
+  }
+};
+
+struct SimpleKeySegmenter : public BaseRowSegmenter {
+  static Result<std::unique_ptr<RowSegmenter>> Make(TypeHolder key_type) {
+    return std::make_unique<SimpleKeySegmenter>(key_type);
+  }
+
+  explicit SimpleKeySegmenter(TypeHolder key_type)
+      : BaseRowSegmenter({key_type}), key_type_(key_types_[0]), save_key_data_() {}
+
+  Status CheckType(const DataType& type) {
+    if (!is_fixed_width(type)) {
+      return Status::Invalid("SimpleKeySegmenter does not support type ", type);
+    }
+    return Status::OK();
+  }
+
+  Status Reset() override {
+    save_key_data_.resize(0);
+    return Status::OK();
+  }
+
+  // Checks whether the given grouping data extends the current segment, i.e., is equal to
+  // previously seen grouping data, which is updated with each invocation.
+  bool Extend(const void* data) {
+    size_t byte_width = static_cast<size_t>(key_type_.type->byte_width());
+    bool extends = save_key_data_.size() != byte_width
+                       ? kDefaultExtends
+                       : 0 == memcmp(save_key_data_.data(), data, byte_width);
+    save_key_data_.resize(byte_width);
+    memcpy(save_key_data_.data(), data, byte_width);
+    return extends;
+  }
+
+  Result<Segment> GetNextSegment(const Scalar& scalar, int64_t offset, int64_t length) {
+    ARROW_RETURN_NOT_OK(CheckType(*scalar.type));
+    if (!scalar.is_valid) {
+      return Status::Invalid("segmenting an invalid scalar");
+    }
+    auto data = checked_cast<const PrimitiveScalarBase&>(scalar).data();
+    bool extends = length > 0 ? Extend(data) : kEmptyExtends;
+    return MakeSegment(length, offset, length, extends);
+  }
+
+  Result<Segment> GetNextSegment(const DataType& array_type, const uint8_t* array_bytes,
+                                 int64_t offset, int64_t length) {
+    RETURN_NOT_OK(CheckType(array_type));
+    int64_t byte_width = array_type.byte_width();
+    int64_t match_length = GetMatchLength(array_bytes + offset * byte_width, byte_width,
+                                          array_bytes, offset, length);
+    bool extends = length > 0 ? Extend(array_bytes + offset * byte_width) : kEmptyExtends;
+    return MakeSegment(length, offset, match_length, extends);
+  }
+
+  Result<Segment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {key_type_}));
+    if (offset == batch.length) {
+      return MakeSegment(batch.length, offset, 0, kEmptyExtends);
+    }
+    const auto& value = batch.values[0];
+    if (value.is_scalar()) {
+      return GetNextSegment(*value.scalar, offset, batch.length);
+    }
+    ARROW_DCHECK(value.is_array());
+    const auto& array = value.array;
+    if (array.GetNullCount() > 0) {
+      return Status::NotImplemented("segmenting a nullable array");
+    }
+    return GetNextSegment(*array.type, GetValuesAsBytes(array), offset, batch.length);
+  }
+
+ private:
+  TypeHolder key_type_;
+  std::vector<uint8_t> save_key_data_;
+};
+
+struct AnyKeysSegmenter : public BaseRowSegmenter {
+  static Result<std::unique_ptr<RowSegmenter>> Make(
+      const std::vector<TypeHolder>& key_types, ExecContext* ctx) {
+    ARROW_RETURN_NOT_OK(Grouper::Make(key_types, ctx));  // check types
+    return std::make_unique<AnyKeysSegmenter>(key_types, ctx);
+  }
+
+  AnyKeysSegmenter(const std::vector<TypeHolder>& key_types, ExecContext* ctx)
+      : BaseRowSegmenter(key_types),
+        ctx_(ctx),
+        grouper_(nullptr),
+        save_group_id_(kNoGroupId) {}
+
+  Status Reset() override {

Review Comment:
   Ok



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] icexelloss commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "icexelloss (via GitHub)" <gi...@apache.org>.
icexelloss commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1132489205


##########
cpp/src/arrow/compute/exec/options.h:
##########
@@ -199,21 +199,39 @@ class ARROW_EXPORT ProjectNodeOptions : public ExecNodeOptions {
   std::vector<std::string> names;
 };
 
-/// \brief Make a node which aggregates input batches, optionally grouped by keys.
+/// \brief Make a node which aggregates input batches, optionally grouped by keys and
+/// optionally segmented by segment-keys. Both keys and segment-keys determine the group.
+/// However segment-keys are also used for determining grouping segments, which should be
+/// large, and allow streaming a partial aggregation result after processing each segment.
+/// One common use-case for segment-keys is ordered aggregation, in which the segment-key
+/// attribute specifies a column with non-decreasing values or a lexicographically-ordered
+/// set of such columns.
 ///
 /// If the keys attribute is a non-empty vector, then each aggregate in `aggregates` is
 /// expected to be a HashAggregate function. If the keys attribute is an empty vector,
 /// then each aggregate is assumed to be a ScalarAggregate function.
+///
+/// If the segment_keys attribute is a non-empty vector, then segmented aggregation, as
+/// described above, applies.
+///
+/// The keys and segment_keys vectors must be disjoint.
+///
+/// See also doc in `aggregate_node.cc`

Review Comment:
   @rtpsw Looks like you missed out on this comment



##########
cpp/src/arrow/compute/exec/options.h:
##########
@@ -199,21 +199,39 @@ class ARROW_EXPORT ProjectNodeOptions : public ExecNodeOptions {
   std::vector<std::string> names;
 };
 
-/// \brief Make a node which aggregates input batches, optionally grouped by keys.
+/// \brief Make a node which aggregates input batches, optionally grouped by keys and
+/// optionally segmented by segment-keys. Both keys and segment-keys determine the group.
+/// However segment-keys are also used for determining grouping segments, which should be
+/// large, and allow streaming a partial aggregation result after processing each segment.
+/// One common use-case for segment-keys is ordered aggregation, in which the segment-key
+/// attribute specifies a column with non-decreasing values or a lexicographically-ordered
+/// set of such columns.
 ///
 /// If the keys attribute is a non-empty vector, then each aggregate in `aggregates` is
 /// expected to be a HashAggregate function. If the keys attribute is an empty vector,
 /// then each aggregate is assumed to be a ScalarAggregate function.
+///
+/// If the segment_keys attribute is a non-empty vector, then segmented aggregation, as
+/// described above, applies.
+///
+/// The keys and segment_keys vectors must be disjoint.
+///
+/// See also doc in `aggregate_node.cc`

Review Comment:
   @rtpsw Looks like you missed out on this comment (minor issue)



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rtpsw commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "rtpsw (via GitHub)" <gi...@apache.org>.
rtpsw commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1126975307


##########
cpp/src/arrow/compute/kernels/hash_aggregate_test.cc:
##########
@@ -1917,57 +2243,7 @@ TEST(GroupBy, MinMaxBinary) {
   }
 }
 
-TEST(GroupBy, MinMaxFixedSizeBinary) {

Review Comment:
   I'm not sure. There is a test case of the same name, but with different code, just above in the diff. Both these test-cases I did not author. I'll defer to @westonpace for which of these two to keep.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] icexelloss commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "icexelloss (via GitHub)" <gi...@apache.org>.
icexelloss commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1128132409


##########
cpp/src/arrow/compute/kernels/hash_aggregate_test.cc:
##########
@@ -4202,5 +4551,260 @@ TEST(GroupBy, OnlyKeys) {
                       /*verbose=*/true);
   }
 }
+
+INSTANTIATE_TEST_SUITE_P(GroupBy, GroupBy, ::testing::Values(RunGroupByImpl));
+
+class SegmentedScalarGroupBy : public GroupBy {};
+
+class SegmentedKeyGroupBy : public GroupBy {};
+
+void TestSegment(GroupByFunction group_by, const std::shared_ptr<Table>& table,
+                 Datum output, const std::vector<Datum>& keys,
+                 const std::vector<Datum>& segment_keys, bool scalar) {

Review Comment:
   What does "scalar" mean here?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rtpsw commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "rtpsw (via GitHub)" <gi...@apache.org>.
rtpsw commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1128235652


##########
cpp/src/arrow/compute/kernels/hash_aggregate_test.cc:
##########
@@ -4202,5 +4551,260 @@ TEST(GroupBy, OnlyKeys) {
                       /*verbose=*/true);
   }
 }
+
+INSTANTIATE_TEST_SUITE_P(GroupBy, GroupBy, ::testing::Values(RunGroupByImpl));
+
+class SegmentedScalarGroupBy : public GroupBy {};
+
+class SegmentedKeyGroupBy : public GroupBy {};
+
+void TestSegment(GroupByFunction group_by, const std::shared_ptr<Table>& table,
+                 Datum output, const std::vector<Datum>& keys,
+                 const std::vector<Datum>& segment_keys, bool scalar) {
+  const char* names[] = {
+      scalar ? "count" : "hash_count",
+      scalar ? "sum" : "hash_sum",
+      scalar ? "min_max" : "hash_min_max",
+  };
+  ASSERT_OK_AND_ASSIGN(Datum aggregated_and_grouped,
+                       group_by(
+                           {
+                               table->GetColumnByName("argument"),
+                               table->GetColumnByName("argument"),
+                               table->GetColumnByName("argument"),
+                           },
+                           keys, segment_keys,
+                           {
+                               {names[0], nullptr, "agg_0", names[0]},
+                               {names[1], nullptr, "agg_1", names[1]},
+                               {names[2], nullptr, "agg_2", names[2]},
+                           },
+                           kDefaultUseThreads, /*naive=*/false));
+
+  AssertDatumsEqual(output, aggregated_and_grouped, /*verbose=*/true);
+}
+
+void TestSegmentScalar(GroupByFunction group_by, const std::shared_ptr<Table>& table,
+                       Datum output, const std::vector<Datum>& segment_keys) {
+  TestSegment(group_by, table, output, {}, segment_keys, /*scalar=*/true);
+}
+
+void TestSegmentKey(GroupByFunction group_by, const std::shared_ptr<Table>& table,
+                    Datum output, const std::vector<Datum>& segment_keys) {
+  TestSegment(group_by, table, output, {table->GetColumnByName("key")}, segment_keys,
+              /*scalar=*/false);
+}
+
+Result<std::shared_ptr<Table>> GetSingleSegmentInputAsChunked() {
+  auto table = TableFromJSON(schema({field("argument", float64()), field("key", int64()),
+                                     field("segment_key", int64())}),
+                             {R"([{"argument": 1.0,   "key": 1,    "segment_key": 1},
+                         {"argument": null,  "key": 1,    "segment_key": 1}
+                        ])",
+                              R"([{"argument": 0.0,   "key": 2,    "segment_key": 1},
+                         {"argument": null,  "key": 3,    "segment_key": 1},
+                         {"argument": 4.0,   "key": null, "segment_key": 1},
+                         {"argument": 3.25,  "key": 1,    "segment_key": 1},
+                         {"argument": 0.125, "key": 2,    "segment_key": 1},
+                         {"argument": -0.25, "key": 2,    "segment_key": 1},
+                         {"argument": 0.75,  "key": null, "segment_key": 1},
+                         {"argument": null,  "key": 3,    "segment_key": 1}
+                        ])",
+                              R"([{"argument": 1.0,   "key": 1,    "segment_key": 0},
+                         {"argument": null,  "key": 1,    "segment_key": 0}
+                        ])",
+                              R"([{"argument": 0.0,   "key": 2,    "segment_key": 0},
+                         {"argument": null,  "key": 3,    "segment_key": 0},
+                         {"argument": 4.0,   "key": null, "segment_key": 0},
+                         {"argument": 3.25,  "key": 1,    "segment_key": 0},
+                         {"argument": 0.125, "key": 2,    "segment_key": 0},
+                         {"argument": -0.25, "key": 2,    "segment_key": 0},
+                         {"argument": 0.75,  "key": null, "segment_key": 0},
+                         {"argument": null,  "key": 3,    "segment_key": 0}
+                        ])"});
+  return table;
+}
+
+Result<std::shared_ptr<Table>> GetSingleSegmentInputAsCombined() {
+  ARROW_ASSIGN_OR_RAISE(auto table, GetSingleSegmentInputAsChunked());
+  return table->CombineChunks();
+}
+
+Result<std::shared_ptr<ChunkedArray>> GetSingleSegmentScalarOutput() {
+  return ChunkedArrayFromJSON(struct_({
+                                  field("count", int64()),
+                                  field("sum", float64()),
+                                  field("min_max", struct_({
+                                                       field("min", float64()),
+                                                       field("max", float64()),
+                                                   })),
+                                  field("key_0", int64()),
+                              }),
+                              {R"([
+    [7, 8.875, {"min": -0.25, "max": 4.0}, 1]
+  ])",
+                               R"([
+    [7, 8.875, {"min": -0.25, "max": 4.0}, 0]
+  ])"});
+}
+
+Result<std::shared_ptr<ChunkedArray>> GetSingleSegmentKeyOutput() {
+  return ChunkedArrayFromJSON(struct_({
+                                  field("hash_count", int64()),
+                                  field("hash_sum", float64()),
+                                  field("hash_min_max", struct_({
+                                                            field("min", float64()),
+                                                            field("max", float64()),
+                                                        })),
+                                  field("key_0", int64()),
+                                  field("key_1", int64()),
+                              }),
+                              {R"([
+    [2, 4.25,   {"min": 1.0,   "max": 3.25},  1, 1],
+    [3, -0.125, {"min": -0.25, "max": 0.125}, 2, 1],
+    [0, null,   {"min": null,  "max": null},  3, 1],
+    [2, 4.75,   {"min": 0.75,  "max": 4.0},   null, 1]
+  ])",
+                               R"([
+    [2, 4.25,   {"min": 1.0,   "max": 3.25},  1, 0],
+    [3, -0.125, {"min": -0.25, "max": 0.125}, 2, 0],
+    [0, null,   {"min": null,  "max": null},  3, 0],
+    [2, 4.75,   {"min": 0.75,  "max": 4.0},   null, 0]
+  ])"});
+}
+
+void TestSingleSegmentScalar(GroupByFunction group_by,
+                             std::function<Result<std::shared_ptr<Table>>()> get_table) {
+  ASSERT_OK_AND_ASSIGN(auto table, get_table());
+  ASSERT_OK_AND_ASSIGN(auto output, GetSingleSegmentScalarOutput());
+  TestSegmentScalar(group_by, table, output, {table->GetColumnByName("segment_key")});
+}
+
+void TestSingleSegmentKey(GroupByFunction group_by,
+                          std::function<Result<std::shared_ptr<Table>>()> get_table) {
+  ASSERT_OK_AND_ASSIGN(auto table, get_table());
+  ASSERT_OK_AND_ASSIGN(auto output, GetSingleSegmentKeyOutput());
+  TestSegmentKey(group_by, table, output, {table->GetColumnByName("segment_key")});
+}
+
+TEST_P(SegmentedScalarGroupBy, SingleSegmentScalarChunked) {
+  TestSingleSegmentScalar(GetParam(), GetSingleSegmentInputAsChunked);
+}
+
+TEST_P(SegmentedScalarGroupBy, SingleSegmentScalarCombined) {
+  TestSingleSegmentScalar(GetParam(), GetSingleSegmentInputAsCombined);
+}
+
+TEST_P(SegmentedKeyGroupBy, SingleSegmentKeyChunked) {
+  TestSingleSegmentKey(GetParam(), GetSingleSegmentInputAsChunked);
+}
+
+TEST_P(SegmentedKeyGroupBy, SingleSegmentKeyCombined) {
+  TestSingleSegmentKey(GetParam(), GetSingleSegmentInputAsCombined);
+}
+
+// extracts one segment of the obtained (single-segment-key) table
+Result<std::shared_ptr<Table>> GetEmptySegmentInput(

Review Comment:
   This means input for empty-segment-keys test cases.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rtpsw commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "rtpsw (via GitHub)" <gi...@apache.org>.
rtpsw commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1126993604


##########
cpp/src/arrow/compute/row/grouper.cc:
##########
@@ -39,12 +43,330 @@
 namespace arrow {
 
 using internal::checked_cast;
+using internal::PrimitiveScalarBase;
 
 namespace compute {
 
 namespace {
 
-struct GrouperImpl : Grouper {
+constexpr uint32_t kNoGroupId = std::numeric_limits<uint32_t>::max();
+
+using group_id_t = std::remove_const<decltype(kNoGroupId)>::type;
+using GroupIdType = CTypeTraits<group_id_t>::ArrowType;
+auto g_group_id_type = std::make_shared<GroupIdType>();
+
+inline const uint8_t* GetValuesAsBytes(const ArraySpan& data, int64_t offset = 0) {
+  DCHECK_GT(data.type->byte_width(), 0);
+  int64_t absolute_byte_offset = (data.offset + offset) * data.type->byte_width();
+  return data.GetValues<uint8_t>(1, absolute_byte_offset);
+}
+
+template <typename Value>
+Status CheckForGetNextSegment(const std::vector<Value>& values, int64_t length,
+                              int64_t offset, const std::vector<TypeHolder>& key_types) {
+  if (offset < 0 || offset > length) {
+    return Status::Invalid("invalid grouping segmenter offset: ", offset);
+  }
+  if (values.size() != key_types.size()) {
+    return Status::Invalid("expected batch size ", key_types.size(), " but got ",
+                           values.size());
+  }
+  for (size_t i = 0; i < key_types.size(); i++) {
+    const auto& value = values[i];
+    const auto& key_type = key_types[i];
+    if (*value.type() != *key_type.type) {
+      return Status::Invalid("expected batch value ", i, " of type ", *key_type.type,
+                             " but got ", *value.type());
+    }
+  }
+  return Status::OK();
+}
+
+template <typename Batch>
+enable_if_t<std::is_same<Batch, ExecSpan>::value || std::is_same<Batch, ExecBatch>::value,
+            Status>
+CheckForGetNextSegment(const Batch& batch, int64_t offset,
+                       const std::vector<TypeHolder>& key_types) {
+  return CheckForGetNextSegment(batch.values, batch.length, offset, key_types);
+}
+
+struct BaseRowSegmenter : public RowSegmenter {
+  explicit BaseRowSegmenter(const std::vector<TypeHolder>& key_types)
+      : key_types_(key_types) {}
+
+  const std::vector<TypeHolder>& key_types() const override { return key_types_; }
+
+  std::vector<TypeHolder> key_types_;
+};
+
+Segment MakeSegment(int64_t batch_length, int64_t offset, int64_t length, bool extends) {
+  return Segment{offset, length, offset + length >= batch_length, extends};
+}
+
+int64_t GetMatchLength(const uint8_t* match_bytes, int64_t match_width,
+                       const uint8_t* array_bytes, int64_t offset, int64_t length) {
+  int64_t cursor, byte_cursor;
+  for (cursor = offset, byte_cursor = match_width * cursor; cursor < length;
+       cursor++, byte_cursor += match_width) {
+    if (memcmp(match_bytes, array_bytes + byte_cursor,
+               static_cast<size_t>(match_width)) != 0) {
+      break;
+    }
+  }
+  return std::min(cursor, length) - offset;
+}
+
+using ExtendFunc = std::function<bool(const void*)>;
+constexpr bool kDefaultExtends = true;
+constexpr bool kEmptyExtends = true;
+
+struct NoKeysSegmenter : public BaseRowSegmenter {
+  static std::unique_ptr<RowSegmenter> Make() {
+    return std::make_unique<NoKeysSegmenter>();
+  }
+
+  NoKeysSegmenter() : BaseRowSegmenter({}) {}
+
+  Status Reset() override { return Status::OK(); }
+
+  Result<Segment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {}));
+    return MakeSegment(batch.length, offset, batch.length - offset, kDefaultExtends);
+  }
+};
+
+struct SimpleKeySegmenter : public BaseRowSegmenter {
+  static Result<std::unique_ptr<RowSegmenter>> Make(TypeHolder key_type) {
+    return std::make_unique<SimpleKeySegmenter>(key_type);
+  }
+
+  explicit SimpleKeySegmenter(TypeHolder key_type)
+      : BaseRowSegmenter({key_type}), key_type_(key_types_[0]), save_key_data_() {}
+
+  Status CheckType(const DataType& type) {
+    if (!is_fixed_width(type)) {
+      return Status::Invalid("SimpleKeySegmenter does not support type ", type);
+    }
+    return Status::OK();
+  }
+
+  Status Reset() override {
+    save_key_data_.resize(0);
+    return Status::OK();
+  }
+
+  // Checks whether the given grouping data extends the current segment, i.e., is equal to
+  // previously seen grouping data, which is updated with each invocation.
+  bool Extend(const void* data) {
+    size_t byte_width = static_cast<size_t>(key_type_.type->byte_width());
+    bool extends = save_key_data_.size() != byte_width
+                       ? kDefaultExtends
+                       : 0 == memcmp(save_key_data_.data(), data, byte_width);
+    save_key_data_.resize(byte_width);
+    memcpy(save_key_data_.data(), data, byte_width);
+    return extends;
+  }
+
+  Result<Segment> GetNextSegment(const Scalar& scalar, int64_t offset, int64_t length) {
+    ARROW_RETURN_NOT_OK(CheckType(*scalar.type));
+    if (!scalar.is_valid) {
+      return Status::Invalid("segmenting an invalid scalar");
+    }
+    auto data = checked_cast<const PrimitiveScalarBase&>(scalar).data();
+    bool extends = length > 0 ? Extend(data) : kEmptyExtends;
+    return MakeSegment(length, offset, length, extends);
+  }
+
+  Result<Segment> GetNextSegment(const DataType& array_type, const uint8_t* array_bytes,
+                                 int64_t offset, int64_t length) {
+    RETURN_NOT_OK(CheckType(array_type));
+    int64_t byte_width = array_type.byte_width();
+    int64_t match_length = GetMatchLength(array_bytes + offset * byte_width, byte_width,
+                                          array_bytes, offset, length);
+    bool extends = length > 0 ? Extend(array_bytes + offset * byte_width) : kEmptyExtends;
+    return MakeSegment(length, offset, match_length, extends);
+  }
+
+  Result<Segment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {key_type_}));
+    if (offset == batch.length) {
+      return MakeSegment(batch.length, offset, 0, kEmptyExtends);
+    }
+    const auto& value = batch.values[0];
+    if (value.is_scalar()) {
+      return GetNextSegment(*value.scalar, offset, batch.length);
+    }
+    ARROW_DCHECK(value.is_array());
+    const auto& array = value.array;
+    if (array.GetNullCount() > 0) {
+      return Status::NotImplemented("segmenting a nullable array");
+    }
+    return GetNextSegment(*array.type, GetValuesAsBytes(array), offset, batch.length);
+  }
+
+ private:
+  TypeHolder key_type_;
+  std::vector<uint8_t> save_key_data_;
+};
+
+struct AnyKeysSegmenter : public BaseRowSegmenter {
+  static Result<std::unique_ptr<RowSegmenter>> Make(
+      const std::vector<TypeHolder>& key_types, ExecContext* ctx) {
+    ARROW_RETURN_NOT_OK(Grouper::Make(key_types, ctx));  // check types
+    return std::make_unique<AnyKeysSegmenter>(key_types, ctx);
+  }
+
+  AnyKeysSegmenter(const std::vector<TypeHolder>& key_types, ExecContext* ctx)
+      : BaseRowSegmenter(key_types),
+        ctx_(ctx),
+        grouper_(nullptr),
+        save_group_id_(kNoGroupId) {}
+
+  Status Reset() override {
+    grouper_ = nullptr;
+    save_group_id_ = kNoGroupId;
+    return Status::OK();
+  }
+
+  bool Extend(const void* data) {
+    auto group_id = *static_cast<const group_id_t*>(data);
+    bool extends =
+        save_group_id_ == kNoGroupId ? kDefaultExtends : save_group_id_ == group_id;
+    save_group_id_ = group_id;
+    return extends;
+  }
+
+  // Runs the grouper on a single row.  This is used to determine the group id of the
+  // first row of a new segment to see if it extends the previous segment.
+  template <typename Batch>
+  Result<group_id_t> MapGroupIdAt(const Batch& batch, int64_t offset) {
+    if (!grouper_) return kNoGroupId;
+    ARROW_ASSIGN_OR_RAISE(auto datum, grouper_->Consume(batch, offset,
+                                                        /*length=*/1));
+    if (!datum.is_array()) {
+      return Status::Invalid("accessing unsupported datum kind ", datum.kind());
+    }
+    const std::shared_ptr<ArrayData>& data = datum.array();
+    ARROW_DCHECK(data->GetNullCount() == 0);
+    DCHECK_EQ(data->type->id(), GroupIdType::type_id);
+    DCHECK_EQ(1, data->length);
+    const group_id_t* values = data->GetValues<group_id_t>(1);
+    return values[0];
+  }
+
+  Result<Segment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, key_types_));
+    if (offset == batch.length) {
+      return MakeSegment(batch.length, offset, 0, kEmptyExtends);
+    }
+    // ARROW-18311: make Grouper support Reset()
+    // so it can be cached instead of recreated below
+    //
+    // the group id must be computed prior to resetting the grouper, since it is compared
+    // to save_group_id_, and after resetting the grouper produces incomparable group ids
+    ARROW_ASSIGN_OR_RAISE(auto group_id, MapGroupIdAt(batch, offset));
+    ExtendFunc bound_extend = [this, group_id](const void* data) {
+      bool extends = Extend(&group_id);
+      save_group_id_ = *static_cast<const group_id_t*>(data);
+      return extends;
+    };
+    ARROW_ASSIGN_OR_RAISE(grouper_, Grouper::Make(key_types_, ctx_));  // TODO: reset it
+    ARROW_ASSIGN_OR_RAISE(auto datum, grouper_->Consume(batch, offset));
+    if (datum.is_array()) {
+      const std::shared_ptr<ArrayData>& data = datum.array();
+      ARROW_DCHECK(data->GetNullCount() == 0);
+      DCHECK_EQ(data->type->id(), GroupIdType::type_id);
+      const group_id_t* values = data->GetValues<group_id_t>(1);
+      int64_t cursor;
+      for (cursor = 1; cursor < data->length; cursor++) {
+        if (values[0] != values[cursor]) break;
+      }
+      int64_t length = std::min(cursor, batch.length - offset);

Review Comment:
   Good catch. I believe it should be `int64_t length = std::min(cursor, batch.length) - offset`.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rtpsw commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "rtpsw (via GitHub)" <gi...@apache.org>.
rtpsw commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1126991132


##########
cpp/src/arrow/compute/row/grouper.cc:
##########
@@ -39,12 +43,330 @@
 namespace arrow {
 
 using internal::checked_cast;
+using internal::PrimitiveScalarBase;
 
 namespace compute {
 
 namespace {
 
-struct GrouperImpl : Grouper {
+constexpr uint32_t kNoGroupId = std::numeric_limits<uint32_t>::max();
+
+using group_id_t = std::remove_const<decltype(kNoGroupId)>::type;
+using GroupIdType = CTypeTraits<group_id_t>::ArrowType;
+auto g_group_id_type = std::make_shared<GroupIdType>();
+
+inline const uint8_t* GetValuesAsBytes(const ArraySpan& data, int64_t offset = 0) {
+  DCHECK_GT(data.type->byte_width(), 0);
+  int64_t absolute_byte_offset = (data.offset + offset) * data.type->byte_width();
+  return data.GetValues<uint8_t>(1, absolute_byte_offset);
+}
+
+template <typename Value>
+Status CheckForGetNextSegment(const std::vector<Value>& values, int64_t length,
+                              int64_t offset, const std::vector<TypeHolder>& key_types) {
+  if (offset < 0 || offset > length) {
+    return Status::Invalid("invalid grouping segmenter offset: ", offset);
+  }
+  if (values.size() != key_types.size()) {
+    return Status::Invalid("expected batch size ", key_types.size(), " but got ",
+                           values.size());
+  }
+  for (size_t i = 0; i < key_types.size(); i++) {
+    const auto& value = values[i];
+    const auto& key_type = key_types[i];
+    if (*value.type() != *key_type.type) {
+      return Status::Invalid("expected batch value ", i, " of type ", *key_type.type,
+                             " but got ", *value.type());
+    }
+  }
+  return Status::OK();
+}
+
+template <typename Batch>
+enable_if_t<std::is_same<Batch, ExecSpan>::value || std::is_same<Batch, ExecBatch>::value,
+            Status>
+CheckForGetNextSegment(const Batch& batch, int64_t offset,
+                       const std::vector<TypeHolder>& key_types) {
+  return CheckForGetNextSegment(batch.values, batch.length, offset, key_types);
+}
+
+struct BaseRowSegmenter : public RowSegmenter {
+  explicit BaseRowSegmenter(const std::vector<TypeHolder>& key_types)
+      : key_types_(key_types) {}
+
+  const std::vector<TypeHolder>& key_types() const override { return key_types_; }
+
+  std::vector<TypeHolder> key_types_;
+};
+
+Segment MakeSegment(int64_t batch_length, int64_t offset, int64_t length, bool extends) {
+  return Segment{offset, length, offset + length >= batch_length, extends};
+}
+
+int64_t GetMatchLength(const uint8_t* match_bytes, int64_t match_width,
+                       const uint8_t* array_bytes, int64_t offset, int64_t length) {
+  int64_t cursor, byte_cursor;
+  for (cursor = offset, byte_cursor = match_width * cursor; cursor < length;
+       cursor++, byte_cursor += match_width) {
+    if (memcmp(match_bytes, array_bytes + byte_cursor,
+               static_cast<size_t>(match_width)) != 0) {
+      break;
+    }
+  }
+  return std::min(cursor, length) - offset;
+}
+
+using ExtendFunc = std::function<bool(const void*)>;
+constexpr bool kDefaultExtends = true;
+constexpr bool kEmptyExtends = true;
+
+struct NoKeysSegmenter : public BaseRowSegmenter {
+  static std::unique_ptr<RowSegmenter> Make() {
+    return std::make_unique<NoKeysSegmenter>();
+  }
+
+  NoKeysSegmenter() : BaseRowSegmenter({}) {}
+
+  Status Reset() override { return Status::OK(); }
+
+  Result<Segment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {}));
+    return MakeSegment(batch.length, offset, batch.length - offset, kDefaultExtends);
+  }
+};
+
+struct SimpleKeySegmenter : public BaseRowSegmenter {
+  static Result<std::unique_ptr<RowSegmenter>> Make(TypeHolder key_type) {
+    return std::make_unique<SimpleKeySegmenter>(key_type);
+  }
+
+  explicit SimpleKeySegmenter(TypeHolder key_type)
+      : BaseRowSegmenter({key_type}), key_type_(key_types_[0]), save_key_data_() {}
+
+  Status CheckType(const DataType& type) {
+    if (!is_fixed_width(type)) {
+      return Status::Invalid("SimpleKeySegmenter does not support type ", type);
+    }
+    return Status::OK();
+  }
+
+  Status Reset() override {
+    save_key_data_.resize(0);
+    return Status::OK();
+  }
+
+  // Checks whether the given grouping data extends the current segment, i.e., is equal to
+  // previously seen grouping data, which is updated with each invocation.
+  bool Extend(const void* data) {
+    size_t byte_width = static_cast<size_t>(key_type_.type->byte_width());
+    bool extends = save_key_data_.size() != byte_width
+                       ? kDefaultExtends
+                       : 0 == memcmp(save_key_data_.data(), data, byte_width);
+    save_key_data_.resize(byte_width);
+    memcpy(save_key_data_.data(), data, byte_width);
+    return extends;
+  }
+
+  Result<Segment> GetNextSegment(const Scalar& scalar, int64_t offset, int64_t length) {
+    ARROW_RETURN_NOT_OK(CheckType(*scalar.type));
+    if (!scalar.is_valid) {
+      return Status::Invalid("segmenting an invalid scalar");
+    }
+    auto data = checked_cast<const PrimitiveScalarBase&>(scalar).data();
+    bool extends = length > 0 ? Extend(data) : kEmptyExtends;
+    return MakeSegment(length, offset, length, extends);
+  }
+
+  Result<Segment> GetNextSegment(const DataType& array_type, const uint8_t* array_bytes,
+                                 int64_t offset, int64_t length) {
+    RETURN_NOT_OK(CheckType(array_type));
+    int64_t byte_width = array_type.byte_width();
+    int64_t match_length = GetMatchLength(array_bytes + offset * byte_width, byte_width,
+                                          array_bytes, offset, length);
+    bool extends = length > 0 ? Extend(array_bytes + offset * byte_width) : kEmptyExtends;
+    return MakeSegment(length, offset, match_length, extends);
+  }
+
+  Result<Segment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {key_type_}));
+    if (offset == batch.length) {
+      return MakeSegment(batch.length, offset, 0, kEmptyExtends);
+    }
+    const auto& value = batch.values[0];
+    if (value.is_scalar()) {
+      return GetNextSegment(*value.scalar, offset, batch.length);
+    }
+    ARROW_DCHECK(value.is_array());
+    const auto& array = value.array;
+    if (array.GetNullCount() > 0) {
+      return Status::NotImplemented("segmenting a nullable array");
+    }
+    return GetNextSegment(*array.type, GetValuesAsBytes(array), offset, batch.length);
+  }
+
+ private:
+  TypeHolder key_type_;
+  std::vector<uint8_t> save_key_data_;
+};
+
+struct AnyKeysSegmenter : public BaseRowSegmenter {
+  static Result<std::unique_ptr<RowSegmenter>> Make(
+      const std::vector<TypeHolder>& key_types, ExecContext* ctx) {
+    ARROW_RETURN_NOT_OK(Grouper::Make(key_types, ctx));  // check types
+    return std::make_unique<AnyKeysSegmenter>(key_types, ctx);
+  }
+
+  AnyKeysSegmenter(const std::vector<TypeHolder>& key_types, ExecContext* ctx)
+      : BaseRowSegmenter(key_types),
+        ctx_(ctx),
+        grouper_(nullptr),
+        save_group_id_(kNoGroupId) {}
+
+  Status Reset() override {
+    grouper_ = nullptr;
+    save_group_id_ = kNoGroupId;
+    return Status::OK();
+  }
+
+  bool Extend(const void* data) {
+    auto group_id = *static_cast<const group_id_t*>(data);
+    bool extends =
+        save_group_id_ == kNoGroupId ? kDefaultExtends : save_group_id_ == group_id;
+    save_group_id_ = group_id;
+    return extends;
+  }
+
+  // Runs the grouper on a single row.  This is used to determine the group id of the
+  // first row of a new segment to see if it extends the previous segment.
+  template <typename Batch>
+  Result<group_id_t> MapGroupIdAt(const Batch& batch, int64_t offset) {
+    if (!grouper_) return kNoGroupId;
+    ARROW_ASSIGN_OR_RAISE(auto datum, grouper_->Consume(batch, offset,
+                                                        /*length=*/1));
+    if (!datum.is_array()) {
+      return Status::Invalid("accessing unsupported datum kind ", datum.kind());
+    }
+    const std::shared_ptr<ArrayData>& data = datum.array();
+    ARROW_DCHECK(data->GetNullCount() == 0);
+    DCHECK_EQ(data->type->id(), GroupIdType::type_id);
+    DCHECK_EQ(1, data->length);
+    const group_id_t* values = data->GetValues<group_id_t>(1);
+    return values[0];
+  }
+
+  Result<Segment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, key_types_));
+    if (offset == batch.length) {
+      return MakeSegment(batch.length, offset, 0, kEmptyExtends);
+    }
+    // ARROW-18311: make Grouper support Reset()

Review Comment:
   A bit further down the code there is a `Grouper::Make` invocation that just ends up resetting the grouper. If `Grouper::Reset` were available then it would be called instead.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] westonpace commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "westonpace (via GitHub)" <gi...@apache.org>.
westonpace commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1128841550


##########
cpp/src/arrow/compute/exec/aggregate_node.cc:
##########
@@ -35,6 +36,25 @@
 #include "arrow/util/thread_pool.h"
 #include "arrow/util/tracing_internal.h"
 
+// This file implements both regular and segmented group-by aggregation, which is a
+// generalization of ordered aggregation in which the key columns are not required to be
+// ordered.
+//
+// In (regular) group-by aggregation, the input rows are partitioned into groups using a
+// set of columns called keys, where in a given group each row has the same values for
+// these columns. In segmented group-by aggregation, a second set of columns called
+// segment-keys is used to refine the partitioning. However, segment-keys are different in
+// that they partition only consecutive rows into a single group. Such a partition of
+// consecutive rows is called a segment group. For example, consider a column X with
+// values [A, B, A] at row-indices [0, 1, 2]. A regular group-by aggregation with keys [X]
+// yields a row-index partitioning [[0, 2], [1]] whereas a segmented-group-by aggregation
+// with segment-keys [X] yields [[0], [1], [2]].

Review Comment:
   Minor nit: This example could be slightly improved I think if you used `[A, A, B, A]` so that readers could see that the segmented group by still does segment.



##########
cpp/src/arrow/compute/exec/aggregate_node.cc:
##########
@@ -584,29 +797,52 @@ class GroupByNode : public ExecNode, public TracedNode {
     ARROW_ASSIGN_OR_RAISE(out_data_, Finalize());
 
     int64_t num_output_batches = bit_util::CeilDiv(out_data_.length, output_batch_size());
-    RETURN_NOT_OK(output_->InputFinished(this, static_cast<int>(num_output_batches)));
-    return plan_->query_context()->StartTaskGroup(output_task_group_id_,
-                                                  num_output_batches);
+    total_output_batches_ += static_cast<int>(num_output_batches);
+    if (is_last) {
+      ARROW_RETURN_NOT_OK(output_->InputFinished(this, total_output_batches_));
+      RETURN_NOT_OK(plan_->query_context()->StartTaskGroup(output_task_group_id_,
+                                                           num_output_batches));
+    } else {
+      for (int64_t i = 0; i < num_output_batches; i++) {
+        ARROW_RETURN_NOT_OK(OutputNthBatch(i));
+      }
+      ARROW_RETURN_NOT_OK(ResetKernelStates());
+    }
+    return Status::OK();
   }
 
   Status InputReceived(ExecNode* input, ExecBatch batch) override {
     auto scope = TraceInputReceived(batch);
 
     DCHECK_EQ(input, inputs_[0]);
 
-    ARROW_RETURN_NOT_OK(Consume(ExecSpan(batch)));
+    auto handler = [this](const ExecBatch& full_batch, const Segment& segment) {
+      if (!segment.extends && segment.offset == 0) RETURN_NOT_OK(OutputResult(false));
+      // This is not zero copy - we should refactor the code to pass
+      // offset and length to Consume to avoid copying here

Review Comment:
   Same as above.  `full_batch.Slice` is zero-copy I believe.



##########
cpp/src/arrow/compute/exec/aggregate_node.cc:
##########
@@ -169,20 +185,79 @@ void AggregatesToString(std::stringstream* ss, const Schema& input_schema,
   *ss << ']';
 }
 
+// Handle the input batch
+// If a segment is closed by this batch, then we output the aggregation for the segment
+// If a segment is not closed by this batch, then we add the batch to the segment
+template <typename BatchHandler>
+Status HandleSegments(std::unique_ptr<RowSegmenter>& segmenter, const ExecBatch& batch,

Review Comment:
   ```suggestion
   Status HandleSegments(RowSegmenter* segmenter, const ExecBatch& batch,
   ```
   Prefer pointer over mutable reference.



##########
cpp/src/arrow/compute/row/grouper.cc:
##########
@@ -39,12 +43,521 @@
 namespace arrow {
 
 using internal::checked_cast;
+using internal::PrimitiveScalarBase;
 
 namespace compute {
 
 namespace {
 
-struct GrouperImpl : Grouper {
+constexpr uint32_t kNoGroupId = std::numeric_limits<uint32_t>::max();
+
+using group_id_t = std::remove_const<decltype(kNoGroupId)>::type;
+using GroupIdType = CTypeTraits<group_id_t>::ArrowType;
+auto group_id_type = std::make_shared<GroupIdType>();
+
+inline const uint8_t* GetValuesAsBytes(const ArrayData& data, int64_t offset = 0) {
+  DCHECK_GT(data.type->byte_width(), 0);
+  int64_t absolute_byte_offset = (data.offset + offset) * data.type->byte_width();
+  return data.GetValues<uint8_t>(1, absolute_byte_offset);
+}
+
+inline const uint8_t* GetValuesAsBytes(const ArraySpan& data, int64_t offset = 0) {
+  DCHECK_GT(data.type->byte_width(), 0);
+  int64_t absolute_byte_offset = (data.offset + offset) * data.type->byte_width();
+  return data.GetValues<uint8_t>(1, absolute_byte_offset);
+}
+
+template <typename Value>
+Status CheckForGetNextSegment(const std::vector<Value>& values, int64_t length,
+                              int64_t offset, const std::vector<TypeHolder>& key_types) {
+  if (offset < 0 || offset > length) {
+    return Status::Invalid("invalid grouping segmenter offset: ", offset);
+  }
+  if (values.size() != key_types.size()) {
+    return Status::Invalid("expected batch size ", key_types.size(), " but got ",
+                           values.size());
+  }
+  for (size_t i = 0; i < key_types.size(); i++) {
+    const auto& value = values[i];
+    const auto& key_type = key_types[i];
+    if (*value.type() != *key_type.type) {
+      return Status::Invalid("expected batch value ", i, " of type ", *key_type.type,
+                             " but got ", *value.type());
+    }
+  }
+  return Status::OK();
+}
+
+template <typename Batch>
+enable_if_t<std::is_same<Batch, ExecSpan>::value || std::is_same<Batch, ExecBatch>::value,
+            Status>
+CheckForGetNextSegment(const Batch& batch, int64_t offset,
+                       const std::vector<TypeHolder>& key_types) {
+  return CheckForGetNextSegment(batch.values, batch.length, offset, key_types);
+}
+
+struct BaseGroupingSegmenter : public GroupingSegmenter {
+  explicit BaseGroupingSegmenter(const std::vector<TypeHolder>& key_types)
+      : key_types_(key_types) {}
+
+  const std::vector<TypeHolder>& key_types() const override { return key_types_; }
+
+  std::vector<TypeHolder> key_types_;
+};
+
+GroupingSegment MakeSegment(int64_t batch_length, int64_t offset, int64_t length,
+                            bool extends) {
+  return GroupingSegment{offset, length, offset + length >= batch_length, extends};
+}
+
+int64_t GetMatchLength(const uint8_t* match_bytes, int64_t match_width,

Review Comment:
   To be fair, it's not clear yet that the performance cost here is significant, and will depend on I/O.  Once we implement "multi-threaded segmented aggregation" then it should be fairly clear.  The "find the next segment" portion will be the spot that we have to serialize (we can consume batches, compute aggregates, and output results in parallel).  So it should be clear from a trace (once we have tracing a bit better supported) if this spot is becoming a bottleneck.
   
   I am in favor of deferring the optimization to future PRs.



##########
cpp/src/arrow/compute/row/grouper.cc:
##########
@@ -39,12 +43,336 @@
 namespace arrow {
 
 using internal::checked_cast;
+using internal::PrimitiveScalarBase;
 
 namespace compute {
 
 namespace {
 
-struct GrouperImpl : Grouper {
+constexpr uint32_t kNoGroupId = std::numeric_limits<uint32_t>::max();
+
+using group_id_t = std::remove_const<decltype(kNoGroupId)>::type;
+using GroupIdType = CTypeTraits<group_id_t>::ArrowType;
+auto g_group_id_type = std::make_shared<GroupIdType>();
+
+inline const uint8_t* GetValuesAsBytes(const ArraySpan& data, int64_t offset = 0) {
+  DCHECK_GT(data.type->byte_width(), 0);
+  int64_t absolute_byte_offset = (data.offset + offset) * data.type->byte_width();
+  return data.GetValues<uint8_t>(1, absolute_byte_offset);
+}
+
+template <typename Value>
+Status CheckForGetNextSegment(const std::vector<Value>& values, int64_t length,
+                              int64_t offset, const std::vector<TypeHolder>& key_types) {
+  if (offset < 0 || offset > length) {

Review Comment:
   I agree.  My general rule of thumb is "would a user understand this error?" or "could this error be triggered by invalid user input?"  In this case, I think the answer is "no" and so these could be `DCHECK`.  I wouldn't count "custom node developers" as users.



##########
cpp/src/arrow/compute/row/grouper.cc:
##########
@@ -39,12 +43,330 @@
 namespace arrow {
 
 using internal::checked_cast;
+using internal::PrimitiveScalarBase;
 
 namespace compute {
 
 namespace {
 
-struct GrouperImpl : Grouper {
+constexpr uint32_t kNoGroupId = std::numeric_limits<uint32_t>::max();
+
+using group_id_t = std::remove_const<decltype(kNoGroupId)>::type;
+using GroupIdType = CTypeTraits<group_id_t>::ArrowType;
+auto g_group_id_type = std::make_shared<GroupIdType>();
+
+inline const uint8_t* GetValuesAsBytes(const ArraySpan& data, int64_t offset = 0) {
+  DCHECK_GT(data.type->byte_width(), 0);
+  int64_t absolute_byte_offset = (data.offset + offset) * data.type->byte_width();
+  return data.GetValues<uint8_t>(1, absolute_byte_offset);
+}
+
+template <typename Value>
+Status CheckForGetNextSegment(const std::vector<Value>& values, int64_t length,
+                              int64_t offset, const std::vector<TypeHolder>& key_types) {
+  if (offset < 0 || offset > length) {
+    return Status::Invalid("invalid grouping segmenter offset: ", offset);
+  }
+  if (values.size() != key_types.size()) {
+    return Status::Invalid("expected batch size ", key_types.size(), " but got ",
+                           values.size());
+  }
+  for (size_t i = 0; i < key_types.size(); i++) {
+    const auto& value = values[i];
+    const auto& key_type = key_types[i];
+    if (*value.type() != *key_type.type) {
+      return Status::Invalid("expected batch value ", i, " of type ", *key_type.type,
+                             " but got ", *value.type());
+    }
+  }
+  return Status::OK();
+}
+
+template <typename Batch>
+enable_if_t<std::is_same<Batch, ExecSpan>::value || std::is_same<Batch, ExecBatch>::value,
+            Status>
+CheckForGetNextSegment(const Batch& batch, int64_t offset,
+                       const std::vector<TypeHolder>& key_types) {
+  return CheckForGetNextSegment(batch.values, batch.length, offset, key_types);
+}
+
+struct BaseRowSegmenter : public RowSegmenter {
+  explicit BaseRowSegmenter(const std::vector<TypeHolder>& key_types)
+      : key_types_(key_types) {}
+
+  const std::vector<TypeHolder>& key_types() const override { return key_types_; }
+
+  std::vector<TypeHolder> key_types_;
+};
+
+Segment MakeSegment(int64_t batch_length, int64_t offset, int64_t length, bool extends) {
+  return Segment{offset, length, offset + length >= batch_length, extends};
+}
+
+int64_t GetMatchLength(const uint8_t* match_bytes, int64_t match_width,
+                       const uint8_t* array_bytes, int64_t offset, int64_t length) {
+  int64_t cursor, byte_cursor;
+  for (cursor = offset, byte_cursor = match_width * cursor; cursor < length;
+       cursor++, byte_cursor += match_width) {
+    if (memcmp(match_bytes, array_bytes + byte_cursor,
+               static_cast<size_t>(match_width)) != 0) {
+      break;
+    }
+  }
+  return std::min(cursor, length) - offset;
+}
+
+using ExtendFunc = std::function<bool(const void*)>;
+constexpr bool kDefaultExtends = true;
+constexpr bool kEmptyExtends = true;
+
+struct NoKeysSegmenter : public BaseRowSegmenter {
+  static std::unique_ptr<RowSegmenter> Make() {
+    return std::make_unique<NoKeysSegmenter>();
+  }
+
+  NoKeysSegmenter() : BaseRowSegmenter({}) {}
+
+  Status Reset() override { return Status::OK(); }
+
+  Result<Segment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {}));
+    return MakeSegment(batch.length, offset, batch.length - offset, kDefaultExtends);
+  }
+};
+
+struct SimpleKeySegmenter : public BaseRowSegmenter {
+  static Result<std::unique_ptr<RowSegmenter>> Make(TypeHolder key_type) {
+    return std::make_unique<SimpleKeySegmenter>(key_type);
+  }
+
+  explicit SimpleKeySegmenter(TypeHolder key_type)
+      : BaseRowSegmenter({key_type}), key_type_(key_types_[0]), save_key_data_() {}
+
+  Status CheckType(const DataType& type) {
+    if (!is_fixed_width(type)) {
+      return Status::Invalid("SimpleKeySegmenter does not support type ", type);
+    }
+    return Status::OK();
+  }
+
+  Status Reset() override {
+    save_key_data_.resize(0);
+    return Status::OK();
+  }
+
+  // Checks whether the given grouping data extends the current segment, i.e., is equal to
+  // previously seen grouping data, which is updated with each invocation.
+  bool Extend(const void* data) {
+    size_t byte_width = static_cast<size_t>(key_type_.type->byte_width());
+    bool extends = save_key_data_.size() != byte_width
+                       ? kDefaultExtends
+                       : 0 == memcmp(save_key_data_.data(), data, byte_width);
+    save_key_data_.resize(byte_width);
+    memcpy(save_key_data_.data(), data, byte_width);
+    return extends;
+  }
+
+  Result<Segment> GetNextSegment(const Scalar& scalar, int64_t offset, int64_t length) {
+    ARROW_RETURN_NOT_OK(CheckType(*scalar.type));
+    if (!scalar.is_valid) {
+      return Status::Invalid("segmenting an invalid scalar");
+    }
+    auto data = checked_cast<const PrimitiveScalarBase&>(scalar).data();
+    bool extends = length > 0 ? Extend(data) : kEmptyExtends;
+    return MakeSegment(length, offset, length, extends);
+  }
+
+  Result<Segment> GetNextSegment(const DataType& array_type, const uint8_t* array_bytes,
+                                 int64_t offset, int64_t length) {
+    RETURN_NOT_OK(CheckType(array_type));
+    int64_t byte_width = array_type.byte_width();
+    int64_t match_length = GetMatchLength(array_bytes + offset * byte_width, byte_width,
+                                          array_bytes, offset, length);
+    bool extends = length > 0 ? Extend(array_bytes + offset * byte_width) : kEmptyExtends;
+    return MakeSegment(length, offset, match_length, extends);
+  }
+
+  Result<Segment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {key_type_}));
+    if (offset == batch.length) {
+      return MakeSegment(batch.length, offset, 0, kEmptyExtends);
+    }
+    const auto& value = batch.values[0];
+    if (value.is_scalar()) {
+      return GetNextSegment(*value.scalar, offset, batch.length);
+    }
+    ARROW_DCHECK(value.is_array());
+    const auto& array = value.array;
+    if (array.GetNullCount() > 0) {
+      return Status::NotImplemented("segmenting a nullable array");
+    }
+    return GetNextSegment(*array.type, GetValuesAsBytes(array), offset, batch.length);
+  }
+
+ private:
+  TypeHolder key_type_;
+  std::vector<uint8_t> save_key_data_;
+};
+
+struct AnyKeysSegmenter : public BaseRowSegmenter {
+  static Result<std::unique_ptr<RowSegmenter>> Make(
+      const std::vector<TypeHolder>& key_types, ExecContext* ctx) {
+    ARROW_RETURN_NOT_OK(Grouper::Make(key_types, ctx));  // check types
+    return std::make_unique<AnyKeysSegmenter>(key_types, ctx);
+  }
+
+  AnyKeysSegmenter(const std::vector<TypeHolder>& key_types, ExecContext* ctx)
+      : BaseRowSegmenter(key_types),
+        ctx_(ctx),
+        grouper_(nullptr),
+        save_group_id_(kNoGroupId) {}
+
+  Status Reset() override {

Review Comment:
   We should also keep in mind that "custom exec node developers" are a valid persona to support.  As we build up utilities like these we could in theory even expose them to python (e.g. pyarrow users have asked for things like ExecBatchBuilder and the row table in the past).  I think it's ok to err a little bit on the side of "a complete abstraction" as long as it doesn't go overboard.
   
   To help review length in the future, we could probably have reviewed the segmenters independently and then come back and reviewed the changes to the node itself.  But that ship has sailed at this point :)



##########
cpp/src/arrow/compute/exec/aggregate_node.cc:
##########
@@ -169,20 +185,79 @@ void AggregatesToString(std::stringstream* ss, const Schema& input_schema,
   *ss << ']';
 }
 
+// Handle the input batch
+// If a segment is closed by this batch, then we output the aggregation for the segment
+// If a segment is not closed by this batch, then we add the batch to the segment

Review Comment:
   ```suggestion
   // Extract segments from a batch and run the given handler on them.  Note that the
   // handle may be called on open segments which are not yet finished.  Typically a
   // handler should accumulate those open segments until a closed segment is reached.
   ```



##########
cpp/src/arrow/compute/exec/aggregate_node.cc:
##########
@@ -283,28 +393,47 @@ class ScalarAggregateNode : public ExecNode, public TracedNode {
     DCHECK_EQ(input, inputs_[0]);
 
     auto thread_index = plan_->query_context()->GetThreadIndex();
-
-    ARROW_RETURN_NOT_OK(DoConsume(ExecSpan(batch), thread_index));
+    auto handler = [this, thread_index](const ExecBatch& full_batch,
+                                        const Segment& segment) {
+      // (1) The segment is starting of a new segment group and points to
+      // the beginning of the batch, then it means no data in the batch belongs
+      // to the current segment group. We can output and reset kernel states.
+      if (!segment.extends && segment.offset == 0) RETURN_NOT_OK(OutputResult(false));
+
+      // We add segment to the current segment group aggregation
+      // GH-34475: change to zero-copy slicing
+      auto exec_batch = full_batch.Slice(segment.offset, segment.length);

Review Comment:
   Isn't `full_batch.Slice` a zero-copy operation?  I'm not sure I understand why we need a follow-up.



##########
cpp/src/arrow/compute/exec/aggregate_node.cc:
##########
@@ -169,35 +185,117 @@ void AggregatesToString(std::stringstream* ss, const Schema& input_schema,
   *ss << ']';
 }
 
+template <typename BatchHandler>
+Status HandleSegments(std::unique_ptr<GroupingSegmenter>& segmenter,
+                      const ExecBatch& batch, const std::vector<int>& ids,
+                      const BatchHandler& handle_batch) {
+  int64_t offset = 0;
+  ARROW_ASSIGN_OR_RAISE(auto segment_exec_batch, batch.SelectValues(ids));
+  ExecSpan segment_batch(segment_exec_batch);
+  while (true) {
+    ARROW_ASSIGN_OR_RAISE(auto segment, segmenter->GetNextSegment(segment_batch, offset));
+    if (segment.offset >= segment_batch.length) break;  // condition of no-next-segment
+    ARROW_RETURN_NOT_OK(handle_batch(batch, segment));
+    offset = segment.offset + segment.length;
+  }
+  return Status::OK();
+}
+
+Status GetScalarFields(std::vector<Datum>* values_ptr, const ExecBatch& input_batch,
+                       const std::vector<int>& field_ids) {
+  DCHECK_GT(input_batch.length, 0);
+  std::vector<Datum>& values = *values_ptr;
+  int64_t row = input_batch.length - 1;
+  values.clear();
+  values.resize(field_ids.size());
+  for (size_t i = 0; i < field_ids.size(); i++) {
+    const Datum& value = input_batch.values[field_ids[i]];
+    if (value.is_scalar()) {

Review Comment:
   > Hmm.. does the input_batch not have an universal interface to at the value of row i regardless of whether it is a constant or not?
   
   It does not.  `Datum` arose early on in the compute infrastructure and can actually be quite a few things (chunked array, record batch, etc.)  So, in that context "get the value at index i" isn't universal.  I wouldn't be surprised if there is some helper function for this somewhere but I don't know what it is.  Typically data values are only accessed within compute kernels and often we have completely separate kernels for working with scalars vs. arrays.  And, for the kernels that handle both, there are a lot of helper classes for "do x for each item in scalar/array", but those are inside the compute kernels logic and probably dealing with spans, etc.
   
   Within Acero, a `Datum` will only be `Scalar` or `Array`.  I think this is probably fine.  Maybe there is a longer term refactor to use something more limited in Acero instead of `Datum` to help avoid this confusion.



##########
cpp/src/arrow/compute/row/grouper.cc:
##########
@@ -39,12 +43,330 @@
 namespace arrow {
 
 using internal::checked_cast;
+using internal::PrimitiveScalarBase;
 
 namespace compute {
 
 namespace {
 
-struct GrouperImpl : Grouper {
+constexpr uint32_t kNoGroupId = std::numeric_limits<uint32_t>::max();
+
+using group_id_t = std::remove_const<decltype(kNoGroupId)>::type;
+using GroupIdType = CTypeTraits<group_id_t>::ArrowType;
+auto g_group_id_type = std::make_shared<GroupIdType>();
+
+inline const uint8_t* GetValuesAsBytes(const ArraySpan& data, int64_t offset = 0) {
+  DCHECK_GT(data.type->byte_width(), 0);
+  int64_t absolute_byte_offset = (data.offset + offset) * data.type->byte_width();
+  return data.GetValues<uint8_t>(1, absolute_byte_offset);
+}
+
+template <typename Value>
+Status CheckForGetNextSegment(const std::vector<Value>& values, int64_t length,
+                              int64_t offset, const std::vector<TypeHolder>& key_types) {
+  if (offset < 0 || offset > length) {
+    return Status::Invalid("invalid grouping segmenter offset: ", offset);
+  }
+  if (values.size() != key_types.size()) {
+    return Status::Invalid("expected batch size ", key_types.size(), " but got ",
+                           values.size());
+  }
+  for (size_t i = 0; i < key_types.size(); i++) {
+    const auto& value = values[i];
+    const auto& key_type = key_types[i];
+    if (*value.type() != *key_type.type) {
+      return Status::Invalid("expected batch value ", i, " of type ", *key_type.type,
+                             " but got ", *value.type());
+    }
+  }
+  return Status::OK();
+}
+
+template <typename Batch>
+enable_if_t<std::is_same<Batch, ExecSpan>::value || std::is_same<Batch, ExecBatch>::value,
+            Status>
+CheckForGetNextSegment(const Batch& batch, int64_t offset,
+                       const std::vector<TypeHolder>& key_types) {
+  return CheckForGetNextSegment(batch.values, batch.length, offset, key_types);
+}
+
+struct BaseRowSegmenter : public RowSegmenter {
+  explicit BaseRowSegmenter(const std::vector<TypeHolder>& key_types)
+      : key_types_(key_types) {}
+
+  const std::vector<TypeHolder>& key_types() const override { return key_types_; }
+
+  std::vector<TypeHolder> key_types_;
+};
+
+Segment MakeSegment(int64_t batch_length, int64_t offset, int64_t length, bool extends) {
+  return Segment{offset, length, offset + length >= batch_length, extends};
+}
+
+int64_t GetMatchLength(const uint8_t* match_bytes, int64_t match_width,
+                       const uint8_t* array_bytes, int64_t offset, int64_t length) {
+  int64_t cursor, byte_cursor;
+  for (cursor = offset, byte_cursor = match_width * cursor; cursor < length;
+       cursor++, byte_cursor += match_width) {
+    if (memcmp(match_bytes, array_bytes + byte_cursor,
+               static_cast<size_t>(match_width)) != 0) {
+      break;
+    }
+  }
+  return std::min(cursor, length) - offset;
+}
+
+using ExtendFunc = std::function<bool(const void*)>;
+constexpr bool kDefaultExtends = true;
+constexpr bool kEmptyExtends = true;
+
+struct NoKeysSegmenter : public BaseRowSegmenter {
+  static std::unique_ptr<RowSegmenter> Make() {
+    return std::make_unique<NoKeysSegmenter>();
+  }
+
+  NoKeysSegmenter() : BaseRowSegmenter({}) {}
+
+  Status Reset() override { return Status::OK(); }
+
+  Result<Segment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {}));
+    return MakeSegment(batch.length, offset, batch.length - offset, kDefaultExtends);
+  }
+};
+
+struct SimpleKeySegmenter : public BaseRowSegmenter {
+  static Result<std::unique_ptr<RowSegmenter>> Make(TypeHolder key_type) {
+    return std::make_unique<SimpleKeySegmenter>(key_type);
+  }
+
+  explicit SimpleKeySegmenter(TypeHolder key_type)
+      : BaseRowSegmenter({key_type}), key_type_(key_types_[0]), save_key_data_() {}
+
+  Status CheckType(const DataType& type) {
+    if (!is_fixed_width(type)) {
+      return Status::Invalid("SimpleKeySegmenter does not support type ", type);
+    }
+    return Status::OK();
+  }
+
+  Status Reset() override {
+    save_key_data_.resize(0);
+    return Status::OK();
+  }
+
+  // Checks whether the given grouping data extends the current segment, i.e., is equal to
+  // previously seen grouping data, which is updated with each invocation.
+  bool Extend(const void* data) {
+    size_t byte_width = static_cast<size_t>(key_type_.type->byte_width());
+    bool extends = save_key_data_.size() != byte_width
+                       ? kDefaultExtends
+                       : 0 == memcmp(save_key_data_.data(), data, byte_width);
+    save_key_data_.resize(byte_width);
+    memcpy(save_key_data_.data(), data, byte_width);
+    return extends;
+  }
+
+  Result<Segment> GetNextSegment(const Scalar& scalar, int64_t offset, int64_t length) {
+    ARROW_RETURN_NOT_OK(CheckType(*scalar.type));
+    if (!scalar.is_valid) {
+      return Status::Invalid("segmenting an invalid scalar");
+    }
+    auto data = checked_cast<const PrimitiveScalarBase&>(scalar).data();
+    bool extends = length > 0 ? Extend(data) : kEmptyExtends;
+    return MakeSegment(length, offset, length, extends);
+  }
+
+  Result<Segment> GetNextSegment(const DataType& array_type, const uint8_t* array_bytes,
+                                 int64_t offset, int64_t length) {
+    RETURN_NOT_OK(CheckType(array_type));
+    int64_t byte_width = array_type.byte_width();
+    int64_t match_length = GetMatchLength(array_bytes + offset * byte_width, byte_width,
+                                          array_bytes, offset, length);
+    bool extends = length > 0 ? Extend(array_bytes + offset * byte_width) : kEmptyExtends;
+    return MakeSegment(length, offset, match_length, extends);
+  }
+
+  Result<Segment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {key_type_}));
+    if (offset == batch.length) {
+      return MakeSegment(batch.length, offset, 0, kEmptyExtends);
+    }
+    const auto& value = batch.values[0];
+    if (value.is_scalar()) {
+      return GetNextSegment(*value.scalar, offset, batch.length);
+    }
+    ARROW_DCHECK(value.is_array());
+    const auto& array = value.array;
+    if (array.GetNullCount() > 0) {
+      return Status::NotImplemented("segmenting a nullable array");
+    }
+    return GetNextSegment(*array.type, GetValuesAsBytes(array), offset, batch.length);
+  }
+
+ private:
+  TypeHolder key_type_;
+  std::vector<uint8_t> save_key_data_;
+};
+
+struct AnyKeysSegmenter : public BaseRowSegmenter {
+  static Result<std::unique_ptr<RowSegmenter>> Make(
+      const std::vector<TypeHolder>& key_types, ExecContext* ctx) {
+    ARROW_RETURN_NOT_OK(Grouper::Make(key_types, ctx));  // check types
+    return std::make_unique<AnyKeysSegmenter>(key_types, ctx);
+  }
+
+  AnyKeysSegmenter(const std::vector<TypeHolder>& key_types, ExecContext* ctx)
+      : BaseRowSegmenter(key_types),
+        ctx_(ctx),
+        grouper_(nullptr),
+        save_group_id_(kNoGroupId) {}
+
+  Status Reset() override {
+    grouper_ = nullptr;
+    save_group_id_ = kNoGroupId;
+    return Status::OK();
+  }
+
+  bool Extend(const void* data) {
+    auto group_id = *static_cast<const group_id_t*>(data);
+    bool extends =
+        save_group_id_ == kNoGroupId ? kDefaultExtends : save_group_id_ == group_id;
+    save_group_id_ = group_id;
+    return extends;
+  }
+
+  // Runs the grouper on a single row.  This is used to determine the group id of the
+  // first row of a new segment to see if it extends the previous segment.
+  template <typename Batch>
+  Result<group_id_t> MapGroupIdAt(const Batch& batch, int64_t offset) {
+    if (!grouper_) return kNoGroupId;
+    ARROW_ASSIGN_OR_RAISE(auto datum, grouper_->Consume(batch, offset,
+                                                        /*length=*/1));
+    if (!datum.is_array()) {
+      return Status::Invalid("accessing unsupported datum kind ", datum.kind());
+    }
+    const std::shared_ptr<ArrayData>& data = datum.array();
+    ARROW_DCHECK(data->GetNullCount() == 0);
+    DCHECK_EQ(data->type->id(), GroupIdType::type_id);
+    DCHECK_EQ(1, data->length);
+    const group_id_t* values = data->GetValues<group_id_t>(1);

Review Comment:
   I'd enjoy a pattern like this someday: https://github.com/apache/arrow/blob/main/cpp/src/arrow/compute/light_array.h#L118-L121
   
   but for the time being @rtpsw is probably right.  This is part of the "implicit knowledge" required for arrow-c++.



##########
cpp/src/arrow/compute/exec/options.h:
##########
@@ -199,21 +199,39 @@ class ARROW_EXPORT ProjectNodeOptions : public ExecNodeOptions {
   std::vector<std::string> names;
 };
 
-/// \brief Make a node which aggregates input batches, optionally grouped by keys.
+/// \brief Make a node which aggregates input batches, optionally grouped by keys and
+/// optionally segmented by segment-keys. Both keys and segment-keys determine the group.
+/// However segment-keys are also used for determining grouping segments, which should be
+/// large, and allow streaming a partial aggregation result after processing each segment.
+/// One common use-case for segment-keys is ordered aggregation, in which the segment-key
+/// attribute specifies a column with non-decreasing values or a lexicographically-ordered
+/// set of such columns.
 ///
 /// If the keys attribute is a non-empty vector, then each aggregate in `aggregates` is
 /// expected to be a HashAggregate function. If the keys attribute is an empty vector,
 /// then each aggregate is assumed to be a ScalarAggregate function.
+///
+/// If the segment_keys attribute is a non-empty vector, then segmented aggregation, as
+/// described above, applies.
+///
+/// The keys and segment_keys vectors must be disjoint.
+///
+/// See also doc in `aggregate_node.cc`

Review Comment:
   ```suggestion
   ```
   
   This documentation is for users.  I'm not sure we should be directing users to `aggregate_node.cc`.  Also, it's not clear what doc this is referring to.  I think this is fine as it is without the "see also".



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] icexelloss commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "icexelloss (via GitHub)" <gi...@apache.org>.
icexelloss commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1126698809


##########
cpp/src/arrow/compute/kernels/hash_aggregate_test.cc:
##########
@@ -174,81 +242,117 @@ Result<Datum> RunGroupBy(const BatchesWithSchema& input,
   ARROW_ASSIGN_OR_RAISE(std::vector<ExecBatch> output_batches,
                         start_and_collect.MoveResult());
 
-  ArrayVector out_arrays(aggregates.size() + key_names.size());
   const auto& output_schema = plan->nodes()[0]->output()->output_schema();
+  if (!segmented) {
+    return MakeGroupByOutput(output_batches, output_schema, aggregates.size(),
+                             key_names.size(), naive);
+  }
+
+  std::vector<ArrayVector> out_arrays(aggregates.size() + key_names.size() +
+                                      segment_key_names.size());
   for (size_t i = 0; i < out_arrays.size(); ++i) {
     std::vector<std::shared_ptr<Array>> arrays(output_batches.size());
     for (size_t j = 0; j < output_batches.size(); ++j) {
-      arrays[j] = output_batches[j].values[i].make_array();
+      auto& value = output_batches[j].values[i];
+      if (value.is_scalar()) {
+        ARROW_ASSIGN_OR_RAISE(
+            arrays[j], MakeArrayFromScalar(*value.scalar(), output_batches[j].length));
+      } else if (value.is_array()) {
+        arrays[j] = value.make_array();
+      } else {
+        return Status::Invalid("GroupByUsingExecPlan unsupported value kind ",
+                               ToString(value.kind()));
+      }
     }
     if (arrays.empty()) {
+      arrays.resize(1);
       ARROW_ASSIGN_OR_RAISE(
-          out_arrays[i],
-          MakeArrayOfNull(output_schema->field(static_cast<int>(i))->type(),
-                          /*length=*/0));
-    } else {
-      ARROW_ASSIGN_OR_RAISE(out_arrays[i], Concatenate(arrays));
+          arrays[0], MakeArrayOfNull(output_schema->field(static_cast<int>(i))->type(),
+                                     /*length=*/0));
     }
+    out_arrays[i] = {std::move(arrays)};
   }
 
-  // The exec plan may reorder the output rows.  The tests are all setup to expect ouptut
-  // in ascending order of keys.  So we need to sort the result by the key columns.  To do
-  // that we create a table using the key columns, calculate the sort indices from that
-  // table (sorting on all fields) and then use those indices to calculate our result.
-  std::vector<std::shared_ptr<Field>> key_fields;
-  std::vector<std::shared_ptr<Array>> key_columns;
-  std::vector<SortKey> sort_keys;
-  for (std::size_t i = 0; i < key_names.size(); i++) {
-    const std::shared_ptr<Array>& arr = out_arrays[i + aggregates.size()];
-    if (arr->type_id() == Type::DICTIONARY) {
-      // Can't sort dictionary columns so need to decode
-      auto dict_arr = checked_pointer_cast<DictionaryArray>(arr);
-      ARROW_ASSIGN_OR_RAISE(auto decoded_arr,
-                            Take(*dict_arr->dictionary(), *dict_arr->indices()));
-      key_columns.push_back(decoded_arr);
-      key_fields.push_back(
-          field("name_does_not_matter", dict_arr->dict_type()->value_type()));
-    } else {
-      key_columns.push_back(arr);
-      key_fields.push_back(field("name_does_not_matter", arr->type()));
+  if (segmented && segment_key_names.size() > 0) {
+    ArrayVector struct_arrays;
+    struct_arrays.reserve(output_batches.size());
+    for (size_t j = 0; j < output_batches.size(); ++j) {
+      ArrayVector struct_fields;
+      struct_fields.reserve(out_arrays.size());
+      for (auto out_array : out_arrays) {
+        struct_fields.push_back(out_array[j]);
+      }
+      ARROW_ASSIGN_OR_RAISE(auto struct_array,
+                            StructArray::Make(struct_fields, output_schema->fields()));
+      struct_arrays.push_back(struct_array);
     }
-    sort_keys.emplace_back(static_cast<int>(i));
+    return ChunkedArray::Make(struct_arrays);
+  } else {
+    ArrayVector struct_fields(out_arrays.size());
+    for (size_t i = 0; i < out_arrays.size(); ++i) {
+      ARROW_ASSIGN_OR_RAISE(struct_fields[i], Concatenate(out_arrays[i]));
+    }
+    return StructArray::Make(std::move(struct_fields), output_schema->fields());
   }
-  std::shared_ptr<Schema> key_schema = schema(std::move(key_fields));
-  std::shared_ptr<Table> key_table = Table::Make(std::move(key_schema), key_columns);
-  SortOptions sort_options(std::move(sort_keys));
-  ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Array> sort_indices,
-                        SortIndices(key_table, sort_options));
+}
 
-  ARROW_ASSIGN_OR_RAISE(
-      std::shared_ptr<Array> struct_arr,
-      StructArray::Make(std::move(out_arrays), output_schema->fields()));
+Result<Datum> RunGroupBy(const BatchesWithSchema& input,
+                         const std::vector<std::string>& key_names,
+                         const std::vector<std::string>& segment_key_names,
+                         const std::vector<Aggregate>& aggregates, bool use_threads,
+                         bool segmented = false, bool naive = false) {
+  if (segment_key_names.size() > 0) {
+    ARROW_ASSIGN_OR_RAISE(auto thread_pool, arrow::internal::ThreadPool::Make(1));
+    ExecContext seq_ctx(default_memory_pool(), thread_pool.get());
+    return RunGroupBy(input, key_names, segment_key_names, aggregates, &seq_ctx,
+                      use_threads, segmented, naive);
+  } else {
+    return RunGroupBy(input, key_names, segment_key_names, aggregates,
+                      threaded_exec_context(), use_threads, segmented, naive);
+  }
+}
 
-  return Take(struct_arr, sort_indices);
+Result<Datum> RunGroupBy(const BatchesWithSchema& input,
+                         const std::vector<std::string>& key_names,
+                         const std::vector<Aggregate>& aggregates, bool use_threads,
+                         bool segmented = false, bool naive = false) {
+  return RunGroupBy(input, key_names, {}, aggregates, use_threads, segmented);
 }
 
 /// Simpler overload where you can give the columns as datums
 Result<Datum> RunGroupBy(const std::vector<Datum>& arguments,
                          const std::vector<Datum>& keys,
-                         const std::vector<Aggregate>& aggregates,
-                         bool use_threads = false) {
+                         const std::vector<Datum>& segment_keys,
+                         const std::vector<Aggregate>& aggregates, bool use_threads,
+                         bool segmented = false, bool naive = false) {

Review Comment:
   > I wanted to test the case of segmented aggregation with empty segment-keys. We could decide to remove this.
   
   What do you mean by "segmented aggregation with empty segment-keys"? Isn't this what happens when segment-keys is empty? (The original hash aggregate)



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] icexelloss commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "icexelloss (via GitHub)" <gi...@apache.org>.
icexelloss commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1126695136


##########
cpp/src/arrow/compute/kernels/hash_aggregate_test.cc:
##########
@@ -174,81 +242,117 @@ Result<Datum> RunGroupBy(const BatchesWithSchema& input,
   ARROW_ASSIGN_OR_RAISE(std::vector<ExecBatch> output_batches,
                         start_and_collect.MoveResult());
 
-  ArrayVector out_arrays(aggregates.size() + key_names.size());
   const auto& output_schema = plan->nodes()[0]->output()->output_schema();
+  if (!segmented) {
+    return MakeGroupByOutput(output_batches, output_schema, aggregates.size(),
+                             key_names.size(), naive);
+  }
+
+  std::vector<ArrayVector> out_arrays(aggregates.size() + key_names.size() +
+                                      segment_key_names.size());
   for (size_t i = 0; i < out_arrays.size(); ++i) {
     std::vector<std::shared_ptr<Array>> arrays(output_batches.size());
     for (size_t j = 0; j < output_batches.size(); ++j) {
-      arrays[j] = output_batches[j].values[i].make_array();
+      auto& value = output_batches[j].values[i];
+      if (value.is_scalar()) {
+        ARROW_ASSIGN_OR_RAISE(
+            arrays[j], MakeArrayFromScalar(*value.scalar(), output_batches[j].length));
+      } else if (value.is_array()) {
+        arrays[j] = value.make_array();
+      } else {
+        return Status::Invalid("GroupByUsingExecPlan unsupported value kind ",
+                               ToString(value.kind()));
+      }
     }
     if (arrays.empty()) {
+      arrays.resize(1);
       ARROW_ASSIGN_OR_RAISE(
-          out_arrays[i],
-          MakeArrayOfNull(output_schema->field(static_cast<int>(i))->type(),
-                          /*length=*/0));
-    } else {
-      ARROW_ASSIGN_OR_RAISE(out_arrays[i], Concatenate(arrays));
+          arrays[0], MakeArrayOfNull(output_schema->field(static_cast<int>(i))->type(),
+                                     /*length=*/0));
     }
+    out_arrays[i] = {std::move(arrays)};
   }
 
-  // The exec plan may reorder the output rows.  The tests are all setup to expect ouptut
-  // in ascending order of keys.  So we need to sort the result by the key columns.  To do
-  // that we create a table using the key columns, calculate the sort indices from that
-  // table (sorting on all fields) and then use those indices to calculate our result.
-  std::vector<std::shared_ptr<Field>> key_fields;
-  std::vector<std::shared_ptr<Array>> key_columns;
-  std::vector<SortKey> sort_keys;
-  for (std::size_t i = 0; i < key_names.size(); i++) {
-    const std::shared_ptr<Array>& arr = out_arrays[i + aggregates.size()];
-    if (arr->type_id() == Type::DICTIONARY) {
-      // Can't sort dictionary columns so need to decode
-      auto dict_arr = checked_pointer_cast<DictionaryArray>(arr);
-      ARROW_ASSIGN_OR_RAISE(auto decoded_arr,
-                            Take(*dict_arr->dictionary(), *dict_arr->indices()));
-      key_columns.push_back(decoded_arr);
-      key_fields.push_back(
-          field("name_does_not_matter", dict_arr->dict_type()->value_type()));
-    } else {
-      key_columns.push_back(arr);
-      key_fields.push_back(field("name_does_not_matter", arr->type()));
+  if (segmented && segment_key_names.size() > 0) {
+    ArrayVector struct_arrays;
+    struct_arrays.reserve(output_batches.size());
+    for (size_t j = 0; j < output_batches.size(); ++j) {
+      ArrayVector struct_fields;
+      struct_fields.reserve(out_arrays.size());
+      for (auto out_array : out_arrays) {
+        struct_fields.push_back(out_array[j]);
+      }
+      ARROW_ASSIGN_OR_RAISE(auto struct_array,
+                            StructArray::Make(struct_fields, output_schema->fields()));
+      struct_arrays.push_back(struct_array);
     }
-    sort_keys.emplace_back(static_cast<int>(i));
+    return ChunkedArray::Make(struct_arrays);
+  } else {
+    ArrayVector struct_fields(out_arrays.size());
+    for (size_t i = 0; i < out_arrays.size(); ++i) {
+      ARROW_ASSIGN_OR_RAISE(struct_fields[i], Concatenate(out_arrays[i]));
+    }
+    return StructArray::Make(std::move(struct_fields), output_schema->fields());
   }
-  std::shared_ptr<Schema> key_schema = schema(std::move(key_fields));
-  std::shared_ptr<Table> key_table = Table::Make(std::move(key_schema), key_columns);
-  SortOptions sort_options(std::move(sort_keys));
-  ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Array> sort_indices,
-                        SortIndices(key_table, sort_options));
+}
 
-  ARROW_ASSIGN_OR_RAISE(
-      std::shared_ptr<Array> struct_arr,
-      StructArray::Make(std::move(out_arrays), output_schema->fields()));
+Result<Datum> RunGroupBy(const BatchesWithSchema& input,
+                         const std::vector<std::string>& key_names,
+                         const std::vector<std::string>& segment_key_names,
+                         const std::vector<Aggregate>& aggregates, bool use_threads,
+                         bool segmented = false, bool naive = false) {
+  if (segment_key_names.size() > 0) {
+    ARROW_ASSIGN_OR_RAISE(auto thread_pool, arrow::internal::ThreadPool::Make(1));
+    ExecContext seq_ctx(default_memory_pool(), thread_pool.get());
+    return RunGroupBy(input, key_names, segment_key_names, aggregates, &seq_ctx,
+                      use_threads, segmented, naive);
+  } else {
+    return RunGroupBy(input, key_names, segment_key_names, aggregates,
+                      threaded_exec_context(), use_threads, segmented, naive);
+  }
+}
 
-  return Take(struct_arr, sort_indices);
+Result<Datum> RunGroupBy(const BatchesWithSchema& input,
+                         const std::vector<std::string>& key_names,
+                         const std::vector<Aggregate>& aggregates, bool use_threads,
+                         bool segmented = false, bool naive = false) {
+  return RunGroupBy(input, key_names, {}, aggregates, use_threads, segmented);
 }
 
 /// Simpler overload where you can give the columns as datums
 Result<Datum> RunGroupBy(const std::vector<Datum>& arguments,
                          const std::vector<Datum>& keys,
-                         const std::vector<Aggregate>& aggregates,
-                         bool use_threads = false) {
+                         const std::vector<Datum>& segment_keys,
+                         const std::vector<Aggregate>& aggregates, bool use_threads,
+                         bool segmented = false, bool naive = false) {

Review Comment:
   Sorry I am still confused - what is "naive group by"?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] icexelloss commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "icexelloss (via GitHub)" <gi...@apache.org>.
icexelloss commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1127974592


##########
cpp/src/arrow/compute/row/grouper.cc:
##########
@@ -39,12 +43,336 @@
 namespace arrow {
 
 using internal::checked_cast;
+using internal::PrimitiveScalarBase;
 
 namespace compute {
 
 namespace {
 
-struct GrouperImpl : Grouper {
+constexpr uint32_t kNoGroupId = std::numeric_limits<uint32_t>::max();
+
+using group_id_t = std::remove_const<decltype(kNoGroupId)>::type;
+using GroupIdType = CTypeTraits<group_id_t>::ArrowType;
+auto g_group_id_type = std::make_shared<GroupIdType>();
+
+inline const uint8_t* GetValuesAsBytes(const ArraySpan& data, int64_t offset = 0) {
+  DCHECK_GT(data.type->byte_width(), 0);
+  int64_t absolute_byte_offset = (data.offset + offset) * data.type->byte_width();
+  return data.GetValues<uint8_t>(1, absolute_byte_offset);
+}
+
+template <typename Value>
+Status CheckForGetNextSegment(const std::vector<Value>& values, int64_t length,
+                              int64_t offset, const std::vector<TypeHolder>& key_types) {
+  if (offset < 0 || offset > length) {
+    return Status::Invalid("invalid grouping segmenter offset: ", offset);
+  }
+  if (values.size() != key_types.size()) {
+    return Status::Invalid("expected batch size ", key_types.size(), " but got ",
+                           values.size());
+  }
+  for (size_t i = 0; i < key_types.size(); i++) {
+    const auto& value = values[i];
+    const auto& key_type = key_types[i];
+    if (*value.type() != *key_type.type) {
+      return Status::Invalid("expected batch value ", i, " of type ", *key_type.type,
+                             " but got ", *value.type());
+    }
+  }
+  return Status::OK();
+}
+
+template <typename Batch>
+enable_if_t<std::is_same<Batch, ExecSpan>::value || std::is_same<Batch, ExecBatch>::value,
+            Status>
+CheckForGetNextSegment(const Batch& batch, int64_t offset,
+                       const std::vector<TypeHolder>& key_types) {
+  return CheckForGetNextSegment(batch.values, batch.length, offset, key_types);
+}
+
+struct BaseGroupingSegmenter : public GroupingSegmenter {
+  explicit BaseGroupingSegmenter(const std::vector<TypeHolder>& key_types)
+      : key_types_(key_types) {}
+
+  const std::vector<TypeHolder>& key_types() const override { return key_types_; }
+
+  std::vector<TypeHolder> key_types_;
+};
+
+GroupingSegment MakeSegment(int64_t batch_length, int64_t offset, int64_t length,
+                            bool extends) {
+  return GroupingSegment{offset, length, offset + length >= batch_length, extends};
+}
+
+int64_t GetMatchLength(const uint8_t* match_bytes, int64_t match_width,
+                       const uint8_t* array_bytes, int64_t offset, int64_t length) {
+  int64_t cursor, byte_cursor;
+  for (cursor = offset, byte_cursor = match_width * cursor; cursor < length;
+       cursor++, byte_cursor += match_width) {
+    if (memcmp(match_bytes, array_bytes + byte_cursor,
+               static_cast<size_t>(match_width)) != 0) {
+      break;
+    }
+  }
+  return std::min(cursor, length - offset);
+}
+
+using ExtendFunc = std::function<bool(const void*)>;
+constexpr bool kDefaultExtends = true;
+constexpr bool kEmptyExtends = true;
+
+struct NoKeysGroupingSegmenter : public BaseGroupingSegmenter {
+  static std::unique_ptr<GroupingSegmenter> Make() {
+    return std::make_unique<NoKeysGroupingSegmenter>();
+  }
+
+  NoKeysGroupingSegmenter() : BaseGroupingSegmenter({}) {}
+
+  Status Reset() override { return Status::OK(); }
+
+  Result<GroupingSegment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {}));
+    return MakeSegment(batch.length, offset, batch.length - offset, kDefaultExtends);
+  }
+};
+
+struct SimpleKeyGroupingSegmenter : public BaseGroupingSegmenter {
+  static Result<std::unique_ptr<GroupingSegmenter>> Make(TypeHolder key_type) {
+    return std::make_unique<SimpleKeyGroupingSegmenter>(key_type);
+  }
+
+  explicit SimpleKeyGroupingSegmenter(TypeHolder key_type)
+      : BaseGroupingSegmenter({key_type}), key_type_(key_types_[0]), save_key_data_() {}
+
+  Status CheckType(const DataType& type) {
+    if (!is_fixed_width(type)) {
+      return Status::Invalid("SimpleKeyGroupingSegmenter does not support type ", type);
+    }
+    return Status::OK();
+  }
+
+  Status Reset() override {
+    save_key_data_.resize(0);
+    return Status::OK();
+  }
+
+  // Checks whether the given grouping data extends the current segment, i.e., is equal to
+  // previously seen grouping data, which is updated with each invocation.
+  bool Extend(const void* data) {
+    size_t byte_width = static_cast<size_t>(key_type_.type->byte_width());
+    bool extends = save_key_data_.size() != byte_width
+                       ? kDefaultExtends
+                       : 0 == memcmp(save_key_data_.data(), data, byte_width);
+    save_key_data_.resize(byte_width);
+    memcpy(save_key_data_.data(), data, byte_width);
+    return extends;
+  }
+
+  Result<GroupingSegment> GetNextSegment(const Scalar& scalar, int64_t offset,
+                                         int64_t length) {
+    ARROW_RETURN_NOT_OK(CheckType(*scalar.type));
+    if (!scalar.is_valid) {
+      return Status::Invalid("segmenting an invalid scalar");
+    }
+    auto data = checked_cast<const PrimitiveScalarBase&>(scalar).data();
+    bool extends = length > 0 ? Extend(data) : kEmptyExtends;
+    return MakeSegment(length, offset, length, extends);
+  }
+
+  Result<GroupingSegment> GetNextSegment(const DataType& array_type,
+                                         const uint8_t* array_bytes, int64_t offset,
+                                         int64_t length) {
+    RETURN_NOT_OK(CheckType(array_type));
+    int64_t byte_width = array_type.byte_width();
+    int64_t match_length = GetMatchLength(array_bytes + offset * byte_width, byte_width,
+                                          array_bytes, offset, length);
+    bool extends = length > 0 ? Extend(array_bytes + offset * byte_width) : kEmptyExtends;
+    return MakeSegment(length, offset, match_length, extends);
+  }
+
+  Result<GroupingSegment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {key_type_}));
+    if (offset == batch.length) {
+      return MakeSegment(batch.length, offset, 0, kEmptyExtends);
+    }
+    const auto& value = batch.values[0];
+    if (value.is_scalar()) {
+      return GetNextSegment(*value.scalar, offset, batch.length);
+    }
+    ARROW_DCHECK(value.is_array());
+    const auto& array = value.array;
+    if (array.GetNullCount() > 0) {
+      return Status::NotImplemented("segmenting a nullable array");
+    }
+    return GetNextSegment(*array.type, GetValuesAsBytes(array), offset, batch.length);
+  }
+
+ private:
+  TypeHolder key_type_;
+  std::vector<uint8_t> save_key_data_;
+};
+
+struct AnyKeysGroupingSegmenter : public BaseGroupingSegmenter {
+  static Result<std::unique_ptr<GroupingSegmenter>> Make(
+      const std::vector<TypeHolder>& key_types, ExecContext* ctx) {
+    ARROW_RETURN_NOT_OK(Grouper::Make(key_types, ctx));  // check types
+    return std::make_unique<AnyKeysGroupingSegmenter>(key_types, ctx);
+  }
+
+  AnyKeysGroupingSegmenter(const std::vector<TypeHolder>& key_types, ExecContext* ctx)
+      : BaseGroupingSegmenter(key_types),
+        ctx_(ctx),
+        grouper_(nullptr),
+        save_group_id_(kNoGroupId) {}
+
+  Status Reset() override {
+    grouper_ = nullptr;
+    save_group_id_ = kNoGroupId;
+    return Status::OK();
+  }
+
+  bool Extend(const void* data) {
+    auto group_id = *static_cast<const group_id_t*>(data);
+    bool extends =
+        save_group_id_ == kNoGroupId ? kDefaultExtends : save_group_id_ == group_id;
+    save_group_id_ = group_id;
+    return extends;
+  }
+
+  // Runs the grouper on a single row.  This is used to determine the group id of the
+  // first row of a new segment to see if it extends the previous segment.
+  template <typename Batch>
+  Result<group_id_t> MapGroupIdAt(const Batch& batch, int64_t offset) {
+    if (!grouper_) return kNoGroupId;
+    ARROW_ASSIGN_OR_RAISE(auto datum, grouper_->Consume(batch, offset,
+                                                        /*consume_length=*/1));
+    if (!datum.is_array()) {
+      return Status::Invalid("accessing unsupported datum kind ", datum.kind());
+    }
+    const std::shared_ptr<ArrayData>& data = datum.array();
+    ARROW_DCHECK(data->GetNullCount() == 0);
+    DCHECK_EQ(data->type->id(), GroupIdType::type_id);
+    DCHECK_EQ(1, data->length);
+    const group_id_t* values = data->GetValues<group_id_t>(1);
+    return values[0];
+  }
+
+  Result<GroupingSegment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, key_types_));
+    if (offset == batch.length) {
+      return MakeSegment(batch.length, offset, 0, kEmptyExtends);
+    }
+    // ARROW-18311: make Grouper support Reset()
+    // so it can be cached instead of recreated below
+    //
+    // the group id must be computed prior to resetting the grouper, since it is compared
+    // to save_group_id_, and after resetting the grouper produces incomparable group ids
+    ARROW_ASSIGN_OR_RAISE(auto group_id, MapGroupIdAt(batch, offset));
+    ExtendFunc bound_extend = [this, group_id](const void* data) {
+      bool extends = Extend(&group_id);
+      save_group_id_ = *static_cast<const group_id_t*>(data);
+      return extends;
+    };
+    ARROW_ASSIGN_OR_RAISE(grouper_, Grouper::Make(key_types_, ctx_));  // TODO: reset it
+    ARROW_ASSIGN_OR_RAISE(auto datum, grouper_->Consume(batch, offset));
+    if (datum.is_array()) {
+      const std::shared_ptr<ArrayData>& data = datum.array();
+      ARROW_DCHECK(data->GetNullCount() == 0);
+      DCHECK_EQ(data->type->id(), GroupIdType::type_id);
+      const group_id_t* values = data->GetValues<group_id_t>(1);
+      int64_t cursor;
+      for (cursor = 1; cursor < data->length; cursor++) {
+        if (values[0] != values[cursor]) break;
+      }
+      int64_t length = std::min(cursor, batch.length - offset);
+      bool extends = length > 0 ? bound_extend(values) : kEmptyExtends;
+      return MakeSegment(batch.length, offset, length, extends);
+    } else {
+      return Status::Invalid("segmenting unsupported datum kind ", datum.kind());
+    }
+  }
+
+ private:
+  ExecContext* const ctx_;
+  std::unique_ptr<Grouper> grouper_;
+  group_id_t save_group_id_;
+};
+
+Status CheckForConsume(int64_t batch_length, int64_t& consume_offset,
+                       int64_t* consume_length) {
+  if (consume_offset < 0) {
+    return Status::Invalid("invalid grouper consume offset: ", consume_offset);
+  }
+  if (*consume_length < 0) {
+    *consume_length = batch_length - consume_offset;

Review Comment:
   Maybe `UpdateConsumeLength` is a better function name?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] icexelloss commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "icexelloss (via GitHub)" <gi...@apache.org>.
icexelloss commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1128152483


##########
cpp/src/arrow/compute/kernels/hash_aggregate_test.cc:
##########
@@ -301,53 +439,249 @@ Result<Datum> GroupByTest(const std::vector<Datum>& arguments,
         {t_agg.function, t_agg.options, "agg_" + ToChars(idx), t_agg.function});
     idx = idx + 1;
   }
-  return RunGroupBy(arguments, keys, internal_aggregates, use_threads);
+  return group_by(arguments, keys, segment_keys, internal_aggregates, use_threads,
+                  /*naive=*/false);
 }
 
-}  // namespace
+Result<Datum> GroupByTest(GroupByFunction group_by, const std::vector<Datum>& arguments,
+                          const std::vector<Datum>& keys,
+                          const std::vector<TestAggregate>& aggregates,
+                          bool use_threads) {
+  return GroupByTest(group_by, arguments, keys, {}, aggregates, use_threads);
+}
 
-TEST(Grouper, SupportedKeys) {
-  ASSERT_OK(Grouper::Make({boolean()}));
+template <typename GroupClass>
+void TestGroupClassSupportedKeys(
+    std::function<Result<std::unique_ptr<GroupClass>>(const std::vector<TypeHolder>&)>
+        make_func) {
+  ASSERT_OK(make_func({boolean()}));
 
-  ASSERT_OK(Grouper::Make({int8(), uint16(), int32(), uint64()}));
+  ASSERT_OK(make_func({int8(), uint16(), int32(), uint64()}));
 
-  ASSERT_OK(Grouper::Make({dictionary(int64(), utf8())}));
+  ASSERT_OK(make_func({dictionary(int64(), utf8())}));
 
-  ASSERT_OK(Grouper::Make({float16(), float32(), float64()}));
+  ASSERT_OK(make_func({float16(), float32(), float64()}));
 
-  ASSERT_OK(Grouper::Make({utf8(), binary(), large_utf8(), large_binary()}));
+  ASSERT_OK(make_func({utf8(), binary(), large_utf8(), large_binary()}));
 
-  ASSERT_OK(Grouper::Make({fixed_size_binary(16), fixed_size_binary(32)}));
+  ASSERT_OK(make_func({fixed_size_binary(16), fixed_size_binary(32)}));
 
-  ASSERT_OK(Grouper::Make({decimal128(32, 10), decimal256(76, 20)}));
+  ASSERT_OK(make_func({decimal128(32, 10), decimal256(76, 20)}));
 
-  ASSERT_OK(Grouper::Make({date32(), date64()}));
+  ASSERT_OK(make_func({date32(), date64()}));
 
   for (auto unit : {
            TimeUnit::SECOND,
            TimeUnit::MILLI,
            TimeUnit::MICRO,
            TimeUnit::NANO,
        }) {
-    ASSERT_OK(Grouper::Make({timestamp(unit), duration(unit)}));
+    ASSERT_OK(make_func({timestamp(unit), duration(unit)}));
   }
 
   ASSERT_OK(
-      Grouper::Make({day_time_interval(), month_interval(), month_day_nano_interval()}));
+      make_func({day_time_interval(), month_interval(), month_day_nano_interval()}));
+
+  ASSERT_OK(make_func({null()}));
 
-  ASSERT_OK(Grouper::Make({null()}));
+  ASSERT_RAISES(NotImplemented, make_func({struct_({field("", int64())})}));
 
-  ASSERT_RAISES(NotImplemented, Grouper::Make({struct_({field("", int64())})}));
+  ASSERT_RAISES(NotImplemented, make_func({struct_({})}));
 
-  ASSERT_RAISES(NotImplemented, Grouper::Make({struct_({})}));
+  ASSERT_RAISES(NotImplemented, make_func({list(int32())}));
 
-  ASSERT_RAISES(NotImplemented, Grouper::Make({list(int32())}));
+  ASSERT_RAISES(NotImplemented, make_func({fixed_size_list(int32(), 5)}));
+
+  ASSERT_RAISES(NotImplemented, make_func({dense_union({field("", int32())})}));
+}
 
-  ASSERT_RAISES(NotImplemented, Grouper::Make({fixed_size_list(int32(), 5)}));
+void TestSegments(std::unique_ptr<RowSegmenter>& segmenter, const ExecSpan& batch,
+                  std::vector<Segment> expected_segments) {
+  int64_t offset = 0, segment_num = 0;
+  for (auto expected_segment : expected_segments) {
+    SCOPED_TRACE("segment #" + ToChars(segment_num++));
+    ASSERT_OK_AND_ASSIGN(auto segment, segmenter->GetNextSegment(batch, offset));
+    ASSERT_EQ(expected_segment, segment);
+    offset = segment.offset + segment.length;
+  }
+}
+
+Result<std::unique_ptr<Grouper>> MakeGrouper(const std::vector<TypeHolder>& key_types) {
+  return Grouper::Make(key_types, default_exec_context());
+}
+
+Result<std::unique_ptr<RowSegmenter>> MakeRowSegmenter(
+    const std::vector<TypeHolder>& key_types) {
+  return RowSegmenter::Make(key_types, /*nullable_leys=*/false, default_exec_context());
+}
+
+Result<std::unique_ptr<RowSegmenter>> MakeGenericSegmenter(
+    const std::vector<TypeHolder>& key_types) {
+  return MakeAnyKeysSegmenter(key_types, default_exec_context());
+}
+
+}  // namespace
+
+TEST(RowSegmenter, SupportedKeys) {
+  TestGroupClassSupportedKeys<RowSegmenter>(MakeRowSegmenter);
+}
+
+TEST(RowSegmenter, Basics) {
+  std::vector<TypeHolder> bad_types2 = {int32(), float32()};
+  std::vector<TypeHolder> types2 = {int32(), int32()};
+  std::vector<TypeHolder> bad_types1 = {float32()};
+  std::vector<TypeHolder> types1 = {int32()};
+  std::vector<TypeHolder> types0 = {};
+  auto batch2 = ExecBatchFromJSON(types2, "[[1, 1], [1, 2], [2, 2]]");
+  auto batch1 = ExecBatchFromJSON(types1, "[[1], [1], [2]]");
+  ExecBatch batch0({}, 3);
+  {
+    SCOPED_TRACE("offset");
+    ASSERT_OK_AND_ASSIGN(auto segmenter, MakeRowSegmenter(types0));
+    ExecSpan span0(batch0);
+    for (int64_t offset : {-1, 4}) {
+      EXPECT_RAISES_WITH_MESSAGE_THAT(Invalid,
+                                      HasSubstr("invalid grouping segmenter offset"),
+                                      segmenter->GetNextSegment(span0, offset));
+    }
+  }
+  {
+    SCOPED_TRACE("types0 segmenting of batch2");
+    ASSERT_OK_AND_ASSIGN(auto segmenter, MakeRowSegmenter(types0));
+    ExecSpan span2(batch2);
+    EXPECT_RAISES_WITH_MESSAGE_THAT(Invalid, HasSubstr("expected batch size 0 "),
+                                    segmenter->GetNextSegment(span2, 0));
+    ExecSpan span0(batch0);
+    TestSegments(segmenter, span0, {{0, 3, true, true}, {3, 0, true, true}});
+  }
+  {
+    SCOPED_TRACE("bad_types1 segmenting of batch1");
+    ASSERT_OK_AND_ASSIGN(auto segmenter, MakeRowSegmenter(bad_types1));
+    ExecSpan span1(batch1);
+    EXPECT_RAISES_WITH_MESSAGE_THAT(Invalid, HasSubstr("expected batch value 0 of type "),
+                                    segmenter->GetNextSegment(span1, 0));
+  }
+  {
+    SCOPED_TRACE("types1 segmenting of batch2");
+    ASSERT_OK_AND_ASSIGN(auto segmenter, MakeRowSegmenter(types1));
+    ExecSpan span2(batch2);
+    EXPECT_RAISES_WITH_MESSAGE_THAT(Invalid, HasSubstr("expected batch size 1 "),
+                                    segmenter->GetNextSegment(span2, 0));
+    ExecSpan span1(batch1);
+    TestSegments(segmenter, span1,
+                 {{0, 2, false, true}, {2, 1, true, false}, {3, 0, true, true}});
+  }
+  {
+    SCOPED_TRACE("bad_types2 segmenting of batch2");
+    ASSERT_OK_AND_ASSIGN(auto segmenter, MakeRowSegmenter(bad_types2));
+    ExecSpan span2(batch2);
+    EXPECT_RAISES_WITH_MESSAGE_THAT(Invalid, HasSubstr("expected batch value 1 of type "),
+                                    segmenter->GetNextSegment(span2, 0));
+  }
+  {
+    SCOPED_TRACE("types2 segmenting of batch1");
+    ASSERT_OK_AND_ASSIGN(auto segmenter, MakeRowSegmenter(types2));
+    ExecSpan span1(batch1);
+    EXPECT_RAISES_WITH_MESSAGE_THAT(Invalid, HasSubstr("expected batch size 2 "),
+                                    segmenter->GetNextSegment(span1, 0));
+    ExecSpan span2(batch2);
+    TestSegments(segmenter, span2,
+                 {{0, 1, false, true},
+                  {1, 1, false, false},
+                  {2, 1, true, false},
+                  {3, 0, true, true}});
+  }
+}
+
+namespace {
+
+void test_row_segmenter_constant_batch(

Review Comment:
   Rename to `TestRowSegmenterConstantBatch`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] icexelloss commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "icexelloss (via GitHub)" <gi...@apache.org>.
icexelloss commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1129910144


##########
cpp/src/arrow/compute/kernels/hash_aggregate_test.cc:
##########
@@ -174,81 +255,117 @@ Result<Datum> RunGroupBy(const BatchesWithSchema& input,
   ARROW_ASSIGN_OR_RAISE(std::vector<ExecBatch> output_batches,
                         start_and_collect.MoveResult());
 
-  ArrayVector out_arrays(aggregates.size() + key_names.size());
   const auto& output_schema = plan->nodes()[0]->output()->output_schema();
+  if (!segmented) {
+    return MakeGroupByOutput(output_batches, output_schema, aggregates.size(),
+                             key_names.size(), naive);
+  }
+
+  std::vector<ArrayVector> out_arrays(aggregates.size() + key_names.size() +
+                                      segment_key_names.size());
   for (size_t i = 0; i < out_arrays.size(); ++i) {
     std::vector<std::shared_ptr<Array>> arrays(output_batches.size());
     for (size_t j = 0; j < output_batches.size(); ++j) {
-      arrays[j] = output_batches[j].values[i].make_array();
+      auto& value = output_batches[j].values[i];
+      if (value.is_scalar()) {
+        ARROW_ASSIGN_OR_RAISE(
+            arrays[j], MakeArrayFromScalar(*value.scalar(), output_batches[j].length));
+      } else if (value.is_array()) {
+        arrays[j] = value.make_array();
+      } else {
+        return Status::Invalid("GroupByUsingExecPlan unsupported value kind ",
+                               ToString(value.kind()));
+      }
     }
     if (arrays.empty()) {
+      arrays.resize(1);
       ARROW_ASSIGN_OR_RAISE(
-          out_arrays[i],
-          MakeArrayOfNull(output_schema->field(static_cast<int>(i))->type(),
-                          /*length=*/0));
-    } else {
-      ARROW_ASSIGN_OR_RAISE(out_arrays[i], Concatenate(arrays));
+          arrays[0], MakeArrayOfNull(output_schema->field(static_cast<int>(i))->type(),
+                                     /*length=*/0));
     }
+    out_arrays[i] = {std::move(arrays)};
   }
 
-  // The exec plan may reorder the output rows.  The tests are all setup to expect ouptut
-  // in ascending order of keys.  So we need to sort the result by the key columns.  To do
-  // that we create a table using the key columns, calculate the sort indices from that
-  // table (sorting on all fields) and then use those indices to calculate our result.
-  std::vector<std::shared_ptr<Field>> key_fields;
-  std::vector<std::shared_ptr<Array>> key_columns;
-  std::vector<SortKey> sort_keys;
-  for (std::size_t i = 0; i < key_names.size(); i++) {
-    const std::shared_ptr<Array>& arr = out_arrays[i + aggregates.size()];
-    if (arr->type_id() == Type::DICTIONARY) {
-      // Can't sort dictionary columns so need to decode
-      auto dict_arr = checked_pointer_cast<DictionaryArray>(arr);
-      ARROW_ASSIGN_OR_RAISE(auto decoded_arr,
-                            Take(*dict_arr->dictionary(), *dict_arr->indices()));
-      key_columns.push_back(decoded_arr);
-      key_fields.push_back(
-          field("name_does_not_matter", dict_arr->dict_type()->value_type()));
-    } else {
-      key_columns.push_back(arr);
-      key_fields.push_back(field("name_does_not_matter", arr->type()));
+  if (segmented && segment_key_names.size() > 0) {
+    ArrayVector struct_arrays;
+    struct_arrays.reserve(output_batches.size());
+    for (size_t j = 0; j < output_batches.size(); ++j) {
+      ArrayVector struct_fields;
+      struct_fields.reserve(out_arrays.size());
+      for (auto out_array : out_arrays) {
+        struct_fields.push_back(out_array[j]);
+      }
+      ARROW_ASSIGN_OR_RAISE(auto struct_array,
+                            StructArray::Make(struct_fields, output_schema->fields()));
+      struct_arrays.push_back(struct_array);
     }
-    sort_keys.emplace_back(static_cast<int>(i));
+    return ChunkedArray::Make(struct_arrays);
+  } else {
+    ArrayVector struct_fields(out_arrays.size());
+    for (size_t i = 0; i < out_arrays.size(); ++i) {
+      ARROW_ASSIGN_OR_RAISE(struct_fields[i], Concatenate(out_arrays[i]));
+    }
+    return StructArray::Make(std::move(struct_fields), output_schema->fields());
   }
-  std::shared_ptr<Schema> key_schema = schema(std::move(key_fields));
-  std::shared_ptr<Table> key_table = Table::Make(std::move(key_schema), key_columns);
-  SortOptions sort_options(std::move(sort_keys));
-  ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Array> sort_indices,
-                        SortIndices(key_table, sort_options));
+}
 
-  ARROW_ASSIGN_OR_RAISE(
-      std::shared_ptr<Array> struct_arr,
-      StructArray::Make(std::move(out_arrays), output_schema->fields()));
+Result<Datum> RunGroupBy(const BatchesWithSchema& input,
+                         const std::vector<std::string>& key_names,
+                         const std::vector<std::string>& segment_key_names,
+                         const std::vector<Aggregate>& aggregates, bool use_threads,
+                         bool segmented = false, bool naive = false) {
+  if (segment_key_names.size() > 0) {
+    ARROW_ASSIGN_OR_RAISE(auto thread_pool, arrow::internal::ThreadPool::Make(1));
+    ExecContext seq_ctx(default_memory_pool(), thread_pool.get());
+    return RunGroupBy(input, key_names, segment_key_names, aggregates, &seq_ctx,

Review Comment:
   > It appears that use_threads determines whether we are using threads in the mock input
   This makes more sense. @rtpsw since you are here can you add a comment to clarify this?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rtpsw commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "rtpsw (via GitHub)" <gi...@apache.org>.
rtpsw commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1117891629


##########
cpp/src/arrow/compute/row/grouper.h:
##########
@@ -30,6 +30,49 @@
 namespace arrow {
 namespace compute {
 
+/// \brief A segment of contiguous rows for grouping
+struct ARROW_EXPORT GroupingSegment {
+  int64_t offset;
+  int64_t length;
+  bool is_open;
+  bool extends;
+};
+
+inline bool operator==(const GroupingSegment& segment1, const GroupingSegment& segment2) {
+  return segment1.offset == segment2.offset && segment1.length == segment2.length &&
+         segment1.is_open == segment2.is_open && segment1.extends == segment2.extends;
+}
+inline bool operator!=(const GroupingSegment& segment1, const GroupingSegment& segment2) {
+  return !(segment1 == segment2);
+}
+
+/// \brief Computes grouping segments for a batch. Each segment covers rows with identical
+/// values in the batch. The values in the batch are often selected as keys from a larger
+/// batch.
+class ARROW_EXPORT GroupingSegmenter {
+ public:
+  virtual ~GroupingSegmenter() = default;
+
+  /// \brief Construct a GroupingSegmenter which receives the specified key types
+  static Result<std::unique_ptr<GroupingSegmenter>> Make(
+      const std::vector<TypeHolder>& key_types, bool nullable_keys = false,
+      ExecContext* ctx = default_exec_context());
+
+  /// \brief Return the key types of this segmenter
+  virtual const std::vector<TypeHolder>& key_types() const = 0;
+
+  /// \brief Reset this grouping segmenter
+  virtual Status Reset() = 0;
+
+  /// \brief Get the next segment for the given batch starting from the given offset
+  virtual Result<GroupingSegment> GetNextSegment(const ExecSpan& batch,
+                                                 int64_t offset) = 0;
+
+  /// \brief Get the next segment for the given batch starting from the given offset
+  virtual Result<GroupingSegment> GetNextSegment(const ExecBatch& batch,

Review Comment:
   Done.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rtpsw commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "rtpsw (via GitHub)" <gi...@apache.org>.
rtpsw commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1117886353


##########
cpp/src/arrow/compute/row/grouper.cc:
##########
@@ -39,12 +43,521 @@
 namespace arrow {
 
 using internal::checked_cast;
+using internal::PrimitiveScalarBase;
 
 namespace compute {
 
 namespace {
 
-struct GrouperImpl : Grouper {
+constexpr uint32_t kNoGroupId = std::numeric_limits<uint32_t>::max();
+
+using group_id_t = std::remove_const<decltype(kNoGroupId)>::type;
+using GroupIdType = CTypeTraits<group_id_t>::ArrowType;
+auto group_id_type = std::make_shared<GroupIdType>();
+
+inline const uint8_t* GetValuesAsBytes(const ArrayData& data, int64_t offset = 0) {
+  DCHECK_GT(data.type->byte_width(), 0);
+  int64_t absolute_byte_offset = (data.offset + offset) * data.type->byte_width();
+  return data.GetValues<uint8_t>(1, absolute_byte_offset);
+}
+
+inline const uint8_t* GetValuesAsBytes(const ArraySpan& data, int64_t offset = 0) {
+  DCHECK_GT(data.type->byte_width(), 0);
+  int64_t absolute_byte_offset = (data.offset + offset) * data.type->byte_width();
+  return data.GetValues<uint8_t>(1, absolute_byte_offset);
+}
+
+template <typename Value>
+Status CheckForGetNextSegment(const std::vector<Value>& values, int64_t length,
+                              int64_t offset, const std::vector<TypeHolder>& key_types) {
+  if (offset < 0 || offset > length) {
+    return Status::Invalid("invalid grouping segmenter offset: ", offset);
+  }
+  if (values.size() != key_types.size()) {
+    return Status::Invalid("expected batch size ", key_types.size(), " but got ",
+                           values.size());
+  }
+  for (size_t i = 0; i < key_types.size(); i++) {
+    const auto& value = values[i];
+    const auto& key_type = key_types[i];
+    if (*value.type() != *key_type.type) {
+      return Status::Invalid("expected batch value ", i, " of type ", *key_type.type,
+                             " but got ", *value.type());
+    }
+  }
+  return Status::OK();
+}
+
+template <typename Batch>
+enable_if_t<std::is_same<Batch, ExecSpan>::value || std::is_same<Batch, ExecBatch>::value,
+            Status>
+CheckForGetNextSegment(const Batch& batch, int64_t offset,
+                       const std::vector<TypeHolder>& key_types) {
+  return CheckForGetNextSegment(batch.values, batch.length, offset, key_types);
+}
+
+struct BaseGroupingSegmenter : public GroupingSegmenter {
+  explicit BaseGroupingSegmenter(const std::vector<TypeHolder>& key_types)
+      : key_types_(key_types) {}
+
+  const std::vector<TypeHolder>& key_types() const override { return key_types_; }
+
+  std::vector<TypeHolder> key_types_;
+};
+
+GroupingSegment MakeSegment(int64_t batch_length, int64_t offset, int64_t length,
+                            bool extends) {
+  return GroupingSegment{offset, length, offset + length >= batch_length, extends};
+}
+
+int64_t GetMatchLength(const uint8_t* match_bytes, int64_t match_width,
+                       const uint8_t* array_bytes, int64_t offset, int64_t length) {
+  int64_t cursor, byte_cursor;
+  for (cursor = offset, byte_cursor = match_width * cursor; cursor < length;
+       cursor++, byte_cursor += match_width) {
+    if (memcmp(match_bytes, array_bytes + byte_cursor,
+               static_cast<size_t>(match_width)) != 0) {
+      break;
+    }
+  }
+  return std::min(cursor, length - offset);
+}
+
+using ExtendFunc = std::function<bool(const void*)>;
+constexpr bool kDefaultExtends = true;
+constexpr bool kEmptyExtends = true;
+
+Result<GroupingSegment> GetNextSegmentChunked(
+    const std::shared_ptr<ChunkedArray>& chunked_array, int64_t offset,
+    ExtendFunc extend) {
+  if (offset >= chunked_array->length()) {
+    return MakeSegment(chunked_array->length(), chunked_array->length(), 0,
+                       kEmptyExtends);
+  }
+  int64_t remaining_offset = offset;
+  const auto& arrays = chunked_array->chunks();
+  for (size_t i = 0; remaining_offset >= 0 && i < arrays.size(); i++) {
+    // look up chunk containing offset
+    int64_t array_length = arrays[i]->length();
+    if (remaining_offset < array_length) {
+      // found - switch to matching
+      int64_t match_width = arrays[i]->type()->byte_width();
+      const uint8_t* match_bytes = GetValuesAsBytes(*arrays[i]->data(), remaining_offset);
+      int64_t total_match_length = 0;
+      for (; i < arrays.size(); i++) {
+        int64_t array_length = arrays[i]->length();
+        if (array_length <= 0) continue;
+        const uint8_t* array_bytes = GetValuesAsBytes(*arrays[i]->data());
+        int64_t match_length = GetMatchLength(match_bytes, match_width, array_bytes,
+                                              remaining_offset, array_length);
+        total_match_length += match_length;
+        remaining_offset = 0;
+        if (match_length < array_length - remaining_offset) break;
+      }
+      bool extends = extend(match_bytes);
+      return MakeSegment(chunked_array->length(), offset, total_match_length, extends);
+    }
+    remaining_offset -= array_length;
+  }
+  return Status::Invalid("segmenting invalid chunked array value");
+}
+
+struct NoKeysGroupingSegmenter : public BaseGroupingSegmenter {
+  static std::unique_ptr<GroupingSegmenter> Make() {
+    return std::make_unique<NoKeysGroupingSegmenter>();
+  }
+
+  NoKeysGroupingSegmenter() : BaseGroupingSegmenter({}) {}
+
+  Status Reset() override { return Status::OK(); }
+
+  template <typename Batch>
+  Result<GroupingSegment> GetNextSegmentImpl(const Batch& batch, int64_t offset) {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {}));
+    return MakeSegment(batch.length, offset, batch.length - offset, kDefaultExtends);
+  }
+
+  Result<GroupingSegment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    return GetNextSegmentImpl(batch, offset);
+  }
+
+  Result<GroupingSegment> GetNextSegment(const ExecBatch& batch,
+                                         int64_t offset) override {
+    return GetNextSegmentImpl(batch, offset);
+  }
+};
+
+struct SimpleKeyGroupingSegmenter : public BaseGroupingSegmenter {
+  static Result<std::unique_ptr<GroupingSegmenter>> Make(TypeHolder key_type) {
+    return std::make_unique<SimpleKeyGroupingSegmenter>(key_type);
+  }
+
+  explicit SimpleKeyGroupingSegmenter(TypeHolder key_type)
+      : BaseGroupingSegmenter({key_type}), key_type_(key_types_[0]), save_key_data_() {}
+
+  Status CheckType(const DataType& type) {
+    if (!is_fixed_width(type)) {
+      return Status::Invalid("SimpleKeyGroupingSegmenter does not support type ", type);
+    }
+    return Status::OK();
+  }
+
+  Status Reset() override {
+    save_key_data_.resize(0);
+    return Status::OK();
+  }
+
+  bool Extend(const void* data) {
+    size_t byte_width = static_cast<size_t>(key_type_.type->byte_width());
+    bool extends = save_key_data_.size() != byte_width
+                       ? kDefaultExtends
+                       : 0 == memcmp(save_key_data_.data(), data, byte_width);
+    save_key_data_.resize(byte_width);
+    memcpy(save_key_data_.data(), data, byte_width);
+    return extends;
+  }
+
+  Result<GroupingSegment> GetNextSegment(const Scalar& scalar, int64_t offset,
+                                         int64_t length) {
+    ARROW_RETURN_NOT_OK(CheckType(*scalar.type));
+    if (!scalar.is_valid) {
+      return Status::Invalid("segmenting an invalid scalar");
+    }
+    auto data = checked_cast<const PrimitiveScalarBase&>(scalar).data();
+    bool extends = length > 0 ? Extend(data) : kEmptyExtends;
+    return MakeSegment(length, 0, length, extends);
+  }
+
+  Result<GroupingSegment> GetNextSegment(const DataType& array_type,
+                                         const uint8_t* array_bytes, int64_t offset,
+                                         int64_t length) {
+    RETURN_NOT_OK(CheckType(array_type));
+    int64_t byte_width = array_type.byte_width();
+    int64_t match_length = GetMatchLength(array_bytes + offset * byte_width, byte_width,
+                                          array_bytes, offset, length);
+    bool extends = length > 0 ? Extend(array_bytes + offset * byte_width) : kEmptyExtends;
+    return MakeSegment(length, offset, match_length, extends);
+  }
+
+  Result<GroupingSegment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {key_type_}));
+    if (offset == batch.length) {
+      return MakeSegment(batch.length, offset, 0, kEmptyExtends);
+    }
+    const auto& value = batch.values[0];
+    if (value.is_scalar()) {
+      return GetNextSegment(*value.scalar, offset, batch.length);
+    }
+    ARROW_DCHECK(value.is_array());
+    const auto& array = value.array;
+    if (array.GetNullCount() > 0) {
+      return Status::NotImplemented("segmenting a nullable array");
+    }
+    return GetNextSegment(*array.type, GetValuesAsBytes(array), offset, batch.length);
+  }
+
+  Result<GroupingSegment> GetNextSegment(const ExecBatch& batch,
+                                         int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {key_type_}));
+    if (offset == batch.length) {
+      return MakeSegment(batch.length, offset, 0, kEmptyExtends);
+    }
+    const auto& value = batch.values[0];
+    if (value.is_scalar()) {
+      return GetNextSegment(*value.scalar(), offset, batch.length);
+    }
+    if (value.is_array()) {
+      auto array = value.array();
+      if (array->GetNullCount() > 0) {
+        return Status::NotImplemented("segmenting a nullable array");
+      }
+      return GetNextSegment(*array->type, GetValuesAsBytes(*array), offset, batch.length);
+    }
+    if (value.is_chunked_array()) {
+      auto array = value.chunked_array();
+      if (array->null_count() > 0) {
+        return Status::NotImplemented("segmenting a nullable array");
+      }
+      return GetNextSegmentChunked(array, offset, bound_extend_);
+    }
+    return Status::Invalid("segmenting unsupported value kind ", value.kind());
+  }
+
+ private:
+  TypeHolder key_type_;
+  std::vector<uint8_t> save_key_data_;
+  ExtendFunc bound_extend_ = [this](const void* data) { return Extend(data); };
+};
+
+struct AnyKeysGroupingSegmenter : public BaseGroupingSegmenter {
+  static Result<std::unique_ptr<GroupingSegmenter>> Make(
+      const std::vector<TypeHolder>& key_types, ExecContext* ctx) {
+    ARROW_RETURN_NOT_OK(Grouper::Make(key_types, ctx));  // check types
+    return std::make_unique<AnyKeysGroupingSegmenter>(key_types, ctx);
+  }
+
+  AnyKeysGroupingSegmenter(const std::vector<TypeHolder>& key_types, ExecContext* ctx)
+      : BaseGroupingSegmenter(key_types),
+        ctx_(ctx),
+        grouper_(nullptr),
+        save_group_id_(kNoGroupId) {}
+
+  Status Reset() override {
+    grouper_ = nullptr;
+    save_group_id_ = kNoGroupId;
+    return Status::OK();
+  }
+
+  bool Extend(const void* data) {
+    auto group_id = *static_cast<const group_id_t*>(data);
+    bool extends =
+        save_group_id_ == kNoGroupId ? kDefaultExtends : save_group_id_ == group_id;
+    save_group_id_ = group_id;
+    return extends;
+  }
+
+  template <typename Batch>
+  Result<group_id_t> MapGroupIdAt(const Batch& batch, int64_t offset) {
+    if (offset < 0 || offset >= batch.length) {
+      return Status::Invalid("requesting group id out of bounds");
+    }
+    if (!grouper_) return kNoGroupId;
+    ARROW_ASSIGN_OR_RAISE(auto datum, grouper_->Consume(batch, offset,
+                                                        /*consume_length=*/1));
+    if (!(datum.is_array() || datum.is_chunked_array())) {
+      return Status::Invalid("accessing unsupported datum kind ", datum.kind());
+    }
+    const std::shared_ptr<ArrayData>& data =
+        datum.is_array() ? datum.array() : datum.chunked_array()->chunk(0)->data();
+    ARROW_DCHECK(data->GetNullCount() == 0);
+    DCHECK_EQ(data->type->id(), GroupIdType::type_id);
+    DCHECK_EQ(1, data->length);
+    const group_id_t* values = data->GetValues<group_id_t>(1);
+    return values[0];
+  }
+
+  template <typename Batch>
+  Result<GroupingSegment> GetNextSegmentImpl(const Batch& batch, int64_t offset) {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, key_types_));
+    if (offset == batch.length) {
+      return MakeSegment(batch.length, offset, 0, kEmptyExtends);
+    }
+    // ARROW-18311: make Grouper support Reset()
+    // so it can be cached instead of recreated below
+    //
+    // the group id must be computed prior to resetting the grouper, since it is compared
+    // to save_group_id_, and after resetting the grouper produces incomparable group ids
+    ARROW_ASSIGN_OR_RAISE(auto group_id, MapGroupIdAt(batch, offset));
+    ExtendFunc bound_extend = [this, group_id](const void* data) {
+      bool extends = Extend(&group_id);
+      save_group_id_ = *static_cast<const group_id_t*>(data);
+      return extends;
+    };
+    ARROW_ASSIGN_OR_RAISE(grouper_, Grouper::Make(key_types_, ctx_));  // TODO: reset it
+    ARROW_ASSIGN_OR_RAISE(auto datum, grouper_->Consume(batch, offset));
+    if (datum.is_array()) {
+      const std::shared_ptr<ArrayData>& data = datum.array();
+      ARROW_DCHECK(data->GetNullCount() == 0);
+      DCHECK_EQ(data->type->id(), GroupIdType::type_id);
+      const group_id_t* values = data->GetValues<group_id_t>(1);
+      int64_t cursor;
+      for (cursor = 1; cursor < data->length; cursor++) {
+        if (values[0] != values[cursor]) break;
+      }
+      int64_t length = std::min(cursor, batch.length - offset);
+      bool extends = length > 0 ? bound_extend(values) : kEmptyExtends;
+      return MakeSegment(batch.length, offset, length, extends);
+    } else if (datum.is_chunked_array()) {
+      ARROW_ASSIGN_OR_RAISE(
+          auto segment, GetNextSegmentChunked(datum.chunked_array(), 0, bound_extend));
+      segment.offset += offset;
+      return segment;
+    } else {
+      return Status::Invalid("segmenting unsupported datum kind ", datum.kind());
+    }
+  }
+
+  Result<GroupingSegment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    return GetNextSegmentImpl(batch, offset);
+  }
+
+  Result<GroupingSegment> GetNextSegment(const ExecBatch& batch,
+                                         int64_t offset) override {
+    return GetNextSegmentImpl(batch, offset);
+  }
+
+ private:
+  ExecContext* const ctx_;
+  std::unique_ptr<Grouper> grouper_;
+  group_id_t save_group_id_;
+};
+
+Status CheckForConsume(int64_t batch_length, int64_t& consume_offset,
+                       int64_t* consume_length) {
+  if (consume_offset < 0) {
+    return Status::Invalid("invalid grouper consume offset: ", consume_offset);
+  }
+  if (*consume_length < 0) {
+    *consume_length = batch_length - consume_offset;
+  }
+  return Status::OK();
+}
+
+}  // namespace
+
+Result<std::unique_ptr<GroupingSegmenter>> GroupingSegmenter::Make(
+    const std::vector<TypeHolder>& key_types, bool nullable_keys, ExecContext* ctx) {
+  if (key_types.size() == 0) {
+    return NoKeysGroupingSegmenter::Make();
+  } else if (!nullable_keys && key_types.size() == 1) {
+    const DataType* type = key_types[0].type;
+    if (type != NULLPTR && is_fixed_width(*type)) {
+      return SimpleKeyGroupingSegmenter::Make(key_types[0]);
+    }
+  }
+  return AnyKeysGroupingSegmenter::Make(key_types, ctx);
+}
+
+namespace {
+
+struct BaseGrouper : public Grouper {
+  int IndexOfChunk(const ExecBatch& batch) {
+    int i = 0;
+    for (const auto& value : batch.values) {
+      if (value.is_chunked_array()) {
+        return i;
+      }
+      ++i;
+    }
+    return -1;
+  }
+
+  bool HasConsistentChunks(const ExecBatch& batch, int index_of_chunk) {
+    auto first_chunked_array = batch.values[index_of_chunk].chunked_array();
+    if (first_chunked_array < 0) {
+      // having no chunks is considered consistent
+      return true;
+    }
+    int num_chunks = first_chunked_array->num_chunks();
+    int64_t length = first_chunked_array->length();
+    for (const auto& value : batch.values) {
+      if (!value.is_chunked_array()) {
+        continue;
+      }
+      auto curr_chunk = value.chunked_array();
+      if (num_chunks != curr_chunk->num_chunks() || length != curr_chunk->length()) {
+        return false;
+      }
+    }
+    if (num_chunks > 0) {
+      for (int i = 0; i < num_chunks; i++) {
+        int64_t chunk_length = first_chunked_array->chunk(i)->length();
+        for (const auto& value : batch.values) {
+          if (!value.is_chunked_array()) {
+            continue;
+          }
+          auto curr_chunk = value.chunked_array();
+          if (chunk_length != curr_chunk->chunk(i)->length()) {
+            return false;
+          }
+        }
+      }
+    }
+    return true;
+  }

Review Comment:
   Done.



##########
cpp/src/arrow/compute/row/grouper.cc:
##########
@@ -39,12 +43,521 @@
 namespace arrow {
 
 using internal::checked_cast;
+using internal::PrimitiveScalarBase;
 
 namespace compute {
 
 namespace {
 
-struct GrouperImpl : Grouper {
+constexpr uint32_t kNoGroupId = std::numeric_limits<uint32_t>::max();
+
+using group_id_t = std::remove_const<decltype(kNoGroupId)>::type;
+using GroupIdType = CTypeTraits<group_id_t>::ArrowType;
+auto group_id_type = std::make_shared<GroupIdType>();
+
+inline const uint8_t* GetValuesAsBytes(const ArrayData& data, int64_t offset = 0) {
+  DCHECK_GT(data.type->byte_width(), 0);
+  int64_t absolute_byte_offset = (data.offset + offset) * data.type->byte_width();
+  return data.GetValues<uint8_t>(1, absolute_byte_offset);
+}
+
+inline const uint8_t* GetValuesAsBytes(const ArraySpan& data, int64_t offset = 0) {
+  DCHECK_GT(data.type->byte_width(), 0);
+  int64_t absolute_byte_offset = (data.offset + offset) * data.type->byte_width();
+  return data.GetValues<uint8_t>(1, absolute_byte_offset);
+}
+
+template <typename Value>
+Status CheckForGetNextSegment(const std::vector<Value>& values, int64_t length,
+                              int64_t offset, const std::vector<TypeHolder>& key_types) {
+  if (offset < 0 || offset > length) {
+    return Status::Invalid("invalid grouping segmenter offset: ", offset);
+  }
+  if (values.size() != key_types.size()) {
+    return Status::Invalid("expected batch size ", key_types.size(), " but got ",
+                           values.size());
+  }
+  for (size_t i = 0; i < key_types.size(); i++) {
+    const auto& value = values[i];
+    const auto& key_type = key_types[i];
+    if (*value.type() != *key_type.type) {
+      return Status::Invalid("expected batch value ", i, " of type ", *key_type.type,
+                             " but got ", *value.type());
+    }
+  }
+  return Status::OK();
+}
+
+template <typename Batch>
+enable_if_t<std::is_same<Batch, ExecSpan>::value || std::is_same<Batch, ExecBatch>::value,
+            Status>
+CheckForGetNextSegment(const Batch& batch, int64_t offset,
+                       const std::vector<TypeHolder>& key_types) {
+  return CheckForGetNextSegment(batch.values, batch.length, offset, key_types);
+}
+
+struct BaseGroupingSegmenter : public GroupingSegmenter {
+  explicit BaseGroupingSegmenter(const std::vector<TypeHolder>& key_types)
+      : key_types_(key_types) {}
+
+  const std::vector<TypeHolder>& key_types() const override { return key_types_; }
+
+  std::vector<TypeHolder> key_types_;
+};
+
+GroupingSegment MakeSegment(int64_t batch_length, int64_t offset, int64_t length,
+                            bool extends) {
+  return GroupingSegment{offset, length, offset + length >= batch_length, extends};
+}
+
+int64_t GetMatchLength(const uint8_t* match_bytes, int64_t match_width,
+                       const uint8_t* array_bytes, int64_t offset, int64_t length) {
+  int64_t cursor, byte_cursor;
+  for (cursor = offset, byte_cursor = match_width * cursor; cursor < length;
+       cursor++, byte_cursor += match_width) {
+    if (memcmp(match_bytes, array_bytes + byte_cursor,
+               static_cast<size_t>(match_width)) != 0) {
+      break;
+    }
+  }
+  return std::min(cursor, length - offset);
+}
+
+using ExtendFunc = std::function<bool(const void*)>;
+constexpr bool kDefaultExtends = true;
+constexpr bool kEmptyExtends = true;
+
+Result<GroupingSegment> GetNextSegmentChunked(
+    const std::shared_ptr<ChunkedArray>& chunked_array, int64_t offset,
+    ExtendFunc extend) {
+  if (offset >= chunked_array->length()) {
+    return MakeSegment(chunked_array->length(), chunked_array->length(), 0,
+                       kEmptyExtends);
+  }
+  int64_t remaining_offset = offset;
+  const auto& arrays = chunked_array->chunks();
+  for (size_t i = 0; remaining_offset >= 0 && i < arrays.size(); i++) {
+    // look up chunk containing offset
+    int64_t array_length = arrays[i]->length();
+    if (remaining_offset < array_length) {
+      // found - switch to matching
+      int64_t match_width = arrays[i]->type()->byte_width();
+      const uint8_t* match_bytes = GetValuesAsBytes(*arrays[i]->data(), remaining_offset);
+      int64_t total_match_length = 0;
+      for (; i < arrays.size(); i++) {
+        int64_t array_length = arrays[i]->length();
+        if (array_length <= 0) continue;
+        const uint8_t* array_bytes = GetValuesAsBytes(*arrays[i]->data());
+        int64_t match_length = GetMatchLength(match_bytes, match_width, array_bytes,
+                                              remaining_offset, array_length);
+        total_match_length += match_length;
+        remaining_offset = 0;
+        if (match_length < array_length - remaining_offset) break;
+      }
+      bool extends = extend(match_bytes);
+      return MakeSegment(chunked_array->length(), offset, total_match_length, extends);
+    }
+    remaining_offset -= array_length;
+  }
+  return Status::Invalid("segmenting invalid chunked array value");
+}
+
+struct NoKeysGroupingSegmenter : public BaseGroupingSegmenter {
+  static std::unique_ptr<GroupingSegmenter> Make() {
+    return std::make_unique<NoKeysGroupingSegmenter>();
+  }
+
+  NoKeysGroupingSegmenter() : BaseGroupingSegmenter({}) {}
+
+  Status Reset() override { return Status::OK(); }
+
+  template <typename Batch>
+  Result<GroupingSegment> GetNextSegmentImpl(const Batch& batch, int64_t offset) {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {}));
+    return MakeSegment(batch.length, offset, batch.length - offset, kDefaultExtends);
+  }
+
+  Result<GroupingSegment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    return GetNextSegmentImpl(batch, offset);
+  }
+
+  Result<GroupingSegment> GetNextSegment(const ExecBatch& batch,
+                                         int64_t offset) override {
+    return GetNextSegmentImpl(batch, offset);
+  }
+};
+
+struct SimpleKeyGroupingSegmenter : public BaseGroupingSegmenter {
+  static Result<std::unique_ptr<GroupingSegmenter>> Make(TypeHolder key_type) {
+    return std::make_unique<SimpleKeyGroupingSegmenter>(key_type);
+  }
+
+  explicit SimpleKeyGroupingSegmenter(TypeHolder key_type)
+      : BaseGroupingSegmenter({key_type}), key_type_(key_types_[0]), save_key_data_() {}
+
+  Status CheckType(const DataType& type) {
+    if (!is_fixed_width(type)) {
+      return Status::Invalid("SimpleKeyGroupingSegmenter does not support type ", type);
+    }
+    return Status::OK();
+  }
+
+  Status Reset() override {
+    save_key_data_.resize(0);
+    return Status::OK();
+  }
+
+  bool Extend(const void* data) {
+    size_t byte_width = static_cast<size_t>(key_type_.type->byte_width());
+    bool extends = save_key_data_.size() != byte_width
+                       ? kDefaultExtends
+                       : 0 == memcmp(save_key_data_.data(), data, byte_width);
+    save_key_data_.resize(byte_width);
+    memcpy(save_key_data_.data(), data, byte_width);
+    return extends;
+  }
+
+  Result<GroupingSegment> GetNextSegment(const Scalar& scalar, int64_t offset,
+                                         int64_t length) {
+    ARROW_RETURN_NOT_OK(CheckType(*scalar.type));
+    if (!scalar.is_valid) {
+      return Status::Invalid("segmenting an invalid scalar");
+    }
+    auto data = checked_cast<const PrimitiveScalarBase&>(scalar).data();
+    bool extends = length > 0 ? Extend(data) : kEmptyExtends;
+    return MakeSegment(length, 0, length, extends);
+  }
+
+  Result<GroupingSegment> GetNextSegment(const DataType& array_type,
+                                         const uint8_t* array_bytes, int64_t offset,
+                                         int64_t length) {
+    RETURN_NOT_OK(CheckType(array_type));
+    int64_t byte_width = array_type.byte_width();
+    int64_t match_length = GetMatchLength(array_bytes + offset * byte_width, byte_width,
+                                          array_bytes, offset, length);
+    bool extends = length > 0 ? Extend(array_bytes + offset * byte_width) : kEmptyExtends;
+    return MakeSegment(length, offset, match_length, extends);
+  }
+
+  Result<GroupingSegment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {key_type_}));
+    if (offset == batch.length) {
+      return MakeSegment(batch.length, offset, 0, kEmptyExtends);
+    }
+    const auto& value = batch.values[0];
+    if (value.is_scalar()) {
+      return GetNextSegment(*value.scalar, offset, batch.length);
+    }
+    ARROW_DCHECK(value.is_array());
+    const auto& array = value.array;
+    if (array.GetNullCount() > 0) {
+      return Status::NotImplemented("segmenting a nullable array");
+    }
+    return GetNextSegment(*array.type, GetValuesAsBytes(array), offset, batch.length);
+  }
+
+  Result<GroupingSegment> GetNextSegment(const ExecBatch& batch,
+                                         int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {key_type_}));
+    if (offset == batch.length) {
+      return MakeSegment(batch.length, offset, 0, kEmptyExtends);
+    }
+    const auto& value = batch.values[0];
+    if (value.is_scalar()) {
+      return GetNextSegment(*value.scalar(), offset, batch.length);
+    }
+    if (value.is_array()) {
+      auto array = value.array();
+      if (array->GetNullCount() > 0) {
+        return Status::NotImplemented("segmenting a nullable array");
+      }
+      return GetNextSegment(*array->type, GetValuesAsBytes(*array), offset, batch.length);
+    }
+    if (value.is_chunked_array()) {
+      auto array = value.chunked_array();
+      if (array->null_count() > 0) {
+        return Status::NotImplemented("segmenting a nullable array");
+      }
+      return GetNextSegmentChunked(array, offset, bound_extend_);
+    }
+    return Status::Invalid("segmenting unsupported value kind ", value.kind());
+  }
+
+ private:
+  TypeHolder key_type_;
+  std::vector<uint8_t> save_key_data_;
+  ExtendFunc bound_extend_ = [this](const void* data) { return Extend(data); };
+};
+
+struct AnyKeysGroupingSegmenter : public BaseGroupingSegmenter {
+  static Result<std::unique_ptr<GroupingSegmenter>> Make(
+      const std::vector<TypeHolder>& key_types, ExecContext* ctx) {
+    ARROW_RETURN_NOT_OK(Grouper::Make(key_types, ctx));  // check types
+    return std::make_unique<AnyKeysGroupingSegmenter>(key_types, ctx);
+  }
+
+  AnyKeysGroupingSegmenter(const std::vector<TypeHolder>& key_types, ExecContext* ctx)
+      : BaseGroupingSegmenter(key_types),
+        ctx_(ctx),
+        grouper_(nullptr),
+        save_group_id_(kNoGroupId) {}
+
+  Status Reset() override {
+    grouper_ = nullptr;
+    save_group_id_ = kNoGroupId;
+    return Status::OK();
+  }
+
+  bool Extend(const void* data) {
+    auto group_id = *static_cast<const group_id_t*>(data);
+    bool extends =
+        save_group_id_ == kNoGroupId ? kDefaultExtends : save_group_id_ == group_id;
+    save_group_id_ = group_id;
+    return extends;
+  }
+
+  template <typename Batch>
+  Result<group_id_t> MapGroupIdAt(const Batch& batch, int64_t offset) {
+    if (offset < 0 || offset >= batch.length) {
+      return Status::Invalid("requesting group id out of bounds");
+    }
+    if (!grouper_) return kNoGroupId;
+    ARROW_ASSIGN_OR_RAISE(auto datum, grouper_->Consume(batch, offset,
+                                                        /*consume_length=*/1));
+    if (!(datum.is_array() || datum.is_chunked_array())) {
+      return Status::Invalid("accessing unsupported datum kind ", datum.kind());
+    }
+    const std::shared_ptr<ArrayData>& data =
+        datum.is_array() ? datum.array() : datum.chunked_array()->chunk(0)->data();
+    ARROW_DCHECK(data->GetNullCount() == 0);
+    DCHECK_EQ(data->type->id(), GroupIdType::type_id);
+    DCHECK_EQ(1, data->length);
+    const group_id_t* values = data->GetValues<group_id_t>(1);
+    return values[0];
+  }
+
+  template <typename Batch>
+  Result<GroupingSegment> GetNextSegmentImpl(const Batch& batch, int64_t offset) {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, key_types_));
+    if (offset == batch.length) {
+      return MakeSegment(batch.length, offset, 0, kEmptyExtends);
+    }
+    // ARROW-18311: make Grouper support Reset()
+    // so it can be cached instead of recreated below
+    //
+    // the group id must be computed prior to resetting the grouper, since it is compared
+    // to save_group_id_, and after resetting the grouper produces incomparable group ids
+    ARROW_ASSIGN_OR_RAISE(auto group_id, MapGroupIdAt(batch, offset));
+    ExtendFunc bound_extend = [this, group_id](const void* data) {
+      bool extends = Extend(&group_id);
+      save_group_id_ = *static_cast<const group_id_t*>(data);
+      return extends;
+    };
+    ARROW_ASSIGN_OR_RAISE(grouper_, Grouper::Make(key_types_, ctx_));  // TODO: reset it
+    ARROW_ASSIGN_OR_RAISE(auto datum, grouper_->Consume(batch, offset));
+    if (datum.is_array()) {
+      const std::shared_ptr<ArrayData>& data = datum.array();
+      ARROW_DCHECK(data->GetNullCount() == 0);
+      DCHECK_EQ(data->type->id(), GroupIdType::type_id);
+      const group_id_t* values = data->GetValues<group_id_t>(1);
+      int64_t cursor;
+      for (cursor = 1; cursor < data->length; cursor++) {
+        if (values[0] != values[cursor]) break;
+      }
+      int64_t length = std::min(cursor, batch.length - offset);
+      bool extends = length > 0 ? bound_extend(values) : kEmptyExtends;
+      return MakeSegment(batch.length, offset, length, extends);
+    } else if (datum.is_chunked_array()) {
+      ARROW_ASSIGN_OR_RAISE(
+          auto segment, GetNextSegmentChunked(datum.chunked_array(), 0, bound_extend));
+      segment.offset += offset;
+      return segment;
+    } else {
+      return Status::Invalid("segmenting unsupported datum kind ", datum.kind());
+    }
+  }
+
+  Result<GroupingSegment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    return GetNextSegmentImpl(batch, offset);
+  }
+
+  Result<GroupingSegment> GetNextSegment(const ExecBatch& batch,
+                                         int64_t offset) override {
+    return GetNextSegmentImpl(batch, offset);
+  }
+
+ private:
+  ExecContext* const ctx_;
+  std::unique_ptr<Grouper> grouper_;
+  group_id_t save_group_id_;
+};
+
+Status CheckForConsume(int64_t batch_length, int64_t& consume_offset,
+                       int64_t* consume_length) {
+  if (consume_offset < 0) {
+    return Status::Invalid("invalid grouper consume offset: ", consume_offset);
+  }
+  if (*consume_length < 0) {
+    *consume_length = batch_length - consume_offset;
+  }
+  return Status::OK();
+}
+
+}  // namespace
+
+Result<std::unique_ptr<GroupingSegmenter>> GroupingSegmenter::Make(
+    const std::vector<TypeHolder>& key_types, bool nullable_keys, ExecContext* ctx) {
+  if (key_types.size() == 0) {
+    return NoKeysGroupingSegmenter::Make();
+  } else if (!nullable_keys && key_types.size() == 1) {
+    const DataType* type = key_types[0].type;
+    if (type != NULLPTR && is_fixed_width(*type)) {
+      return SimpleKeyGroupingSegmenter::Make(key_types[0]);
+    }
+  }
+  return AnyKeysGroupingSegmenter::Make(key_types, ctx);
+}
+
+namespace {
+
+struct BaseGrouper : public Grouper {
+  int IndexOfChunk(const ExecBatch& batch) {
+    int i = 0;
+    for (const auto& value : batch.values) {
+      if (value.is_chunked_array()) {
+        return i;
+      }
+      ++i;
+    }
+    return -1;
+  }
+
+  bool HasConsistentChunks(const ExecBatch& batch, int index_of_chunk) {
+    auto first_chunked_array = batch.values[index_of_chunk].chunked_array();
+    if (first_chunked_array < 0) {
+      // having no chunks is considered consistent
+      return true;
+    }
+    int num_chunks = first_chunked_array->num_chunks();
+    int64_t length = first_chunked_array->length();
+    for (const auto& value : batch.values) {
+      if (!value.is_chunked_array()) {
+        continue;
+      }
+      auto curr_chunk = value.chunked_array();
+      if (num_chunks != curr_chunk->num_chunks() || length != curr_chunk->length()) {
+        return false;
+      }
+    }
+    if (num_chunks > 0) {
+      for (int i = 0; i < num_chunks; i++) {
+        int64_t chunk_length = first_chunked_array->chunk(i)->length();
+        for (const auto& value : batch.values) {
+          if (!value.is_chunked_array()) {
+            continue;
+          }
+          auto curr_chunk = value.chunked_array();
+          if (chunk_length != curr_chunk->chunk(i)->length()) {
+            return false;
+          }
+        }
+      }
+    }
+    return true;
+  }
+
+  using Grouper::Consume;
+
+  Result<Datum> Consume(const ExecBatch& batch, int64_t consume_offset,

Review Comment:
   Done.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rtpsw commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "rtpsw (via GitHub)" <gi...@apache.org>.
rtpsw commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1117885933


##########
cpp/src/arrow/compute/row/grouper.cc:
##########
@@ -39,12 +43,521 @@
 namespace arrow {
 
 using internal::checked_cast;
+using internal::PrimitiveScalarBase;
 
 namespace compute {
 
 namespace {
 
-struct GrouperImpl : Grouper {
+constexpr uint32_t kNoGroupId = std::numeric_limits<uint32_t>::max();
+
+using group_id_t = std::remove_const<decltype(kNoGroupId)>::type;
+using GroupIdType = CTypeTraits<group_id_t>::ArrowType;
+auto group_id_type = std::make_shared<GroupIdType>();
+
+inline const uint8_t* GetValuesAsBytes(const ArrayData& data, int64_t offset = 0) {
+  DCHECK_GT(data.type->byte_width(), 0);
+  int64_t absolute_byte_offset = (data.offset + offset) * data.type->byte_width();
+  return data.GetValues<uint8_t>(1, absolute_byte_offset);
+}
+
+inline const uint8_t* GetValuesAsBytes(const ArraySpan& data, int64_t offset = 0) {
+  DCHECK_GT(data.type->byte_width(), 0);
+  int64_t absolute_byte_offset = (data.offset + offset) * data.type->byte_width();
+  return data.GetValues<uint8_t>(1, absolute_byte_offset);
+}
+
+template <typename Value>
+Status CheckForGetNextSegment(const std::vector<Value>& values, int64_t length,
+                              int64_t offset, const std::vector<TypeHolder>& key_types) {
+  if (offset < 0 || offset > length) {
+    return Status::Invalid("invalid grouping segmenter offset: ", offset);
+  }
+  if (values.size() != key_types.size()) {
+    return Status::Invalid("expected batch size ", key_types.size(), " but got ",
+                           values.size());
+  }
+  for (size_t i = 0; i < key_types.size(); i++) {
+    const auto& value = values[i];
+    const auto& key_type = key_types[i];
+    if (*value.type() != *key_type.type) {
+      return Status::Invalid("expected batch value ", i, " of type ", *key_type.type,
+                             " but got ", *value.type());
+    }
+  }
+  return Status::OK();
+}
+
+template <typename Batch>
+enable_if_t<std::is_same<Batch, ExecSpan>::value || std::is_same<Batch, ExecBatch>::value,
+            Status>
+CheckForGetNextSegment(const Batch& batch, int64_t offset,
+                       const std::vector<TypeHolder>& key_types) {
+  return CheckForGetNextSegment(batch.values, batch.length, offset, key_types);
+}
+
+struct BaseGroupingSegmenter : public GroupingSegmenter {
+  explicit BaseGroupingSegmenter(const std::vector<TypeHolder>& key_types)
+      : key_types_(key_types) {}
+
+  const std::vector<TypeHolder>& key_types() const override { return key_types_; }
+
+  std::vector<TypeHolder> key_types_;
+};
+
+GroupingSegment MakeSegment(int64_t batch_length, int64_t offset, int64_t length,
+                            bool extends) {
+  return GroupingSegment{offset, length, offset + length >= batch_length, extends};
+}
+
+int64_t GetMatchLength(const uint8_t* match_bytes, int64_t match_width,
+                       const uint8_t* array_bytes, int64_t offset, int64_t length) {
+  int64_t cursor, byte_cursor;
+  for (cursor = offset, byte_cursor = match_width * cursor; cursor < length;
+       cursor++, byte_cursor += match_width) {
+    if (memcmp(match_bytes, array_bytes + byte_cursor,
+               static_cast<size_t>(match_width)) != 0) {
+      break;
+    }
+  }
+  return std::min(cursor, length - offset);
+}
+
+using ExtendFunc = std::function<bool(const void*)>;
+constexpr bool kDefaultExtends = true;
+constexpr bool kEmptyExtends = true;
+
+Result<GroupingSegment> GetNextSegmentChunked(
+    const std::shared_ptr<ChunkedArray>& chunked_array, int64_t offset,
+    ExtendFunc extend) {
+  if (offset >= chunked_array->length()) {
+    return MakeSegment(chunked_array->length(), chunked_array->length(), 0,
+                       kEmptyExtends);
+  }
+  int64_t remaining_offset = offset;
+  const auto& arrays = chunked_array->chunks();
+  for (size_t i = 0; remaining_offset >= 0 && i < arrays.size(); i++) {
+    // look up chunk containing offset
+    int64_t array_length = arrays[i]->length();
+    if (remaining_offset < array_length) {
+      // found - switch to matching
+      int64_t match_width = arrays[i]->type()->byte_width();
+      const uint8_t* match_bytes = GetValuesAsBytes(*arrays[i]->data(), remaining_offset);
+      int64_t total_match_length = 0;
+      for (; i < arrays.size(); i++) {
+        int64_t array_length = arrays[i]->length();
+        if (array_length <= 0) continue;
+        const uint8_t* array_bytes = GetValuesAsBytes(*arrays[i]->data());
+        int64_t match_length = GetMatchLength(match_bytes, match_width, array_bytes,
+                                              remaining_offset, array_length);
+        total_match_length += match_length;
+        remaining_offset = 0;
+        if (match_length < array_length - remaining_offset) break;
+      }
+      bool extends = extend(match_bytes);
+      return MakeSegment(chunked_array->length(), offset, total_match_length, extends);
+    }
+    remaining_offset -= array_length;
+  }
+  return Status::Invalid("segmenting invalid chunked array value");
+}
+
+struct NoKeysGroupingSegmenter : public BaseGroupingSegmenter {
+  static std::unique_ptr<GroupingSegmenter> Make() {
+    return std::make_unique<NoKeysGroupingSegmenter>();
+  }
+
+  NoKeysGroupingSegmenter() : BaseGroupingSegmenter({}) {}
+
+  Status Reset() override { return Status::OK(); }
+
+  template <typename Batch>
+  Result<GroupingSegment> GetNextSegmentImpl(const Batch& batch, int64_t offset) {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {}));
+    return MakeSegment(batch.length, offset, batch.length - offset, kDefaultExtends);
+  }
+
+  Result<GroupingSegment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    return GetNextSegmentImpl(batch, offset);
+  }
+
+  Result<GroupingSegment> GetNextSegment(const ExecBatch& batch,
+                                         int64_t offset) override {
+    return GetNextSegmentImpl(batch, offset);
+  }
+};
+
+struct SimpleKeyGroupingSegmenter : public BaseGroupingSegmenter {
+  static Result<std::unique_ptr<GroupingSegmenter>> Make(TypeHolder key_type) {
+    return std::make_unique<SimpleKeyGroupingSegmenter>(key_type);
+  }
+
+  explicit SimpleKeyGroupingSegmenter(TypeHolder key_type)
+      : BaseGroupingSegmenter({key_type}), key_type_(key_types_[0]), save_key_data_() {}
+
+  Status CheckType(const DataType& type) {
+    if (!is_fixed_width(type)) {
+      return Status::Invalid("SimpleKeyGroupingSegmenter does not support type ", type);
+    }
+    return Status::OK();
+  }
+
+  Status Reset() override {
+    save_key_data_.resize(0);
+    return Status::OK();
+  }
+
+  bool Extend(const void* data) {
+    size_t byte_width = static_cast<size_t>(key_type_.type->byte_width());
+    bool extends = save_key_data_.size() != byte_width
+                       ? kDefaultExtends
+                       : 0 == memcmp(save_key_data_.data(), data, byte_width);
+    save_key_data_.resize(byte_width);
+    memcpy(save_key_data_.data(), data, byte_width);
+    return extends;
+  }
+
+  Result<GroupingSegment> GetNextSegment(const Scalar& scalar, int64_t offset,
+                                         int64_t length) {
+    ARROW_RETURN_NOT_OK(CheckType(*scalar.type));
+    if (!scalar.is_valid) {
+      return Status::Invalid("segmenting an invalid scalar");
+    }
+    auto data = checked_cast<const PrimitiveScalarBase&>(scalar).data();
+    bool extends = length > 0 ? Extend(data) : kEmptyExtends;
+    return MakeSegment(length, 0, length, extends);
+  }
+
+  Result<GroupingSegment> GetNextSegment(const DataType& array_type,
+                                         const uint8_t* array_bytes, int64_t offset,
+                                         int64_t length) {
+    RETURN_NOT_OK(CheckType(array_type));
+    int64_t byte_width = array_type.byte_width();
+    int64_t match_length = GetMatchLength(array_bytes + offset * byte_width, byte_width,
+                                          array_bytes, offset, length);
+    bool extends = length > 0 ? Extend(array_bytes + offset * byte_width) : kEmptyExtends;
+    return MakeSegment(length, offset, match_length, extends);
+  }
+
+  Result<GroupingSegment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {key_type_}));
+    if (offset == batch.length) {
+      return MakeSegment(batch.length, offset, 0, kEmptyExtends);
+    }
+    const auto& value = batch.values[0];
+    if (value.is_scalar()) {
+      return GetNextSegment(*value.scalar, offset, batch.length);
+    }
+    ARROW_DCHECK(value.is_array());
+    const auto& array = value.array;
+    if (array.GetNullCount() > 0) {
+      return Status::NotImplemented("segmenting a nullable array");
+    }
+    return GetNextSegment(*array.type, GetValuesAsBytes(array), offset, batch.length);
+  }
+
+  Result<GroupingSegment> GetNextSegment(const ExecBatch& batch,
+                                         int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {key_type_}));
+    if (offset == batch.length) {
+      return MakeSegment(batch.length, offset, 0, kEmptyExtends);
+    }
+    const auto& value = batch.values[0];
+    if (value.is_scalar()) {
+      return GetNextSegment(*value.scalar(), offset, batch.length);
+    }
+    if (value.is_array()) {
+      auto array = value.array();
+      if (array->GetNullCount() > 0) {
+        return Status::NotImplemented("segmenting a nullable array");
+      }
+      return GetNextSegment(*array->type, GetValuesAsBytes(*array), offset, batch.length);
+    }
+    if (value.is_chunked_array()) {
+      auto array = value.chunked_array();
+      if (array->null_count() > 0) {
+        return Status::NotImplemented("segmenting a nullable array");
+      }
+      return GetNextSegmentChunked(array, offset, bound_extend_);
+    }
+    return Status::Invalid("segmenting unsupported value kind ", value.kind());
+  }
+
+ private:
+  TypeHolder key_type_;
+  std::vector<uint8_t> save_key_data_;
+  ExtendFunc bound_extend_ = [this](const void* data) { return Extend(data); };
+};
+
+struct AnyKeysGroupingSegmenter : public BaseGroupingSegmenter {
+  static Result<std::unique_ptr<GroupingSegmenter>> Make(
+      const std::vector<TypeHolder>& key_types, ExecContext* ctx) {
+    ARROW_RETURN_NOT_OK(Grouper::Make(key_types, ctx));  // check types
+    return std::make_unique<AnyKeysGroupingSegmenter>(key_types, ctx);
+  }
+
+  AnyKeysGroupingSegmenter(const std::vector<TypeHolder>& key_types, ExecContext* ctx)
+      : BaseGroupingSegmenter(key_types),
+        ctx_(ctx),
+        grouper_(nullptr),
+        save_group_id_(kNoGroupId) {}
+
+  Status Reset() override {
+    grouper_ = nullptr;
+    save_group_id_ = kNoGroupId;
+    return Status::OK();
+  }
+
+  bool Extend(const void* data) {
+    auto group_id = *static_cast<const group_id_t*>(data);
+    bool extends =
+        save_group_id_ == kNoGroupId ? kDefaultExtends : save_group_id_ == group_id;
+    save_group_id_ = group_id;
+    return extends;
+  }
+
+  template <typename Batch>
+  Result<group_id_t> MapGroupIdAt(const Batch& batch, int64_t offset) {
+    if (offset < 0 || offset >= batch.length) {
+      return Status::Invalid("requesting group id out of bounds");
+    }
+    if (!grouper_) return kNoGroupId;
+    ARROW_ASSIGN_OR_RAISE(auto datum, grouper_->Consume(batch, offset,
+                                                        /*consume_length=*/1));
+    if (!(datum.is_array() || datum.is_chunked_array())) {

Review Comment:
   Removed the chunked array code.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rtpsw commented on pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "rtpsw (via GitHub)" <gi...@apache.org>.
rtpsw commented on PR #34311:
URL: https://github.com/apache/arrow/pull/34311#issuecomment-1444389630

   > Also, can we add at least one or two basic end-to-end tests in `plan_test.cc` (or you could create an `aggregate_node_test.cc`). Partly as examples for future readers as much as anything.
   
   My [recent commit](https://github.com/apache/arrow/pull/34311/commits/70acbe6697206ff378e40d1552c70bf77ee335f1) should have the requested fixes except the above one for adding tests (TBD).


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] icexelloss commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "icexelloss (via GitHub)" <gi...@apache.org>.
icexelloss commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1117331915


##########
cpp/src/arrow/compute/exec/aggregate_node.cc:
##########
@@ -282,12 +396,19 @@ class ScalarAggregateNode : public ExecNode, public TracedNode {
     auto scope = TraceInputReceived(batch);
     DCHECK_EQ(input, inputs_[0]);
 
-    auto thread_index = plan_->query_context()->GetThreadIndex();
-
-    ARROW_RETURN_NOT_OK(DoConsume(ExecSpan(batch), thread_index));
+    auto handler = [this](const ExecBatch& full_batch, const GroupingSegment& segment) {
+      if (!segment.extends && segment.offset == 0) RETURN_NOT_OK(OutputResult());
+      auto exec_batch = full_batch.Slice(segment.offset, segment.length);
+      auto batch = ExecSpan(exec_batch);
+      RETURN_NOT_OK(DoConsume(batch, plan_->query_context()->GetThreadIndex()));

Review Comment:
   Oh wait u didn't, you just named the ExecSpan to "batch" which confused me



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rtpsw commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "rtpsw (via GitHub)" <gi...@apache.org>.
rtpsw commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1116169422


##########
cpp/src/arrow/compute/exec/aggregate_node.cc:
##########
@@ -169,17 +206,63 @@ void AggregatesToString(std::stringstream* ss, const Schema& input_schema,
   *ss << ']';
 }
 
+template <typename BatchHandler>
+Status HandleSegments(std::unique_ptr<GroupingSegmenter>& segmenter,
+                      const ExecBatch& batch, const std::vector<int>& ids,
+                      const BatchHandler& handle_batch) {
+  int64_t offset = 0;
+  ARROW_ASSIGN_OR_RAISE(auto segment_batch, batch.SelectValues(ids));
+  while (true) {
+    ARROW_ASSIGN_OR_RAISE(auto segment, segmenter->GetNextSegment(segment_batch, offset));
+    if (segment.offset >= segment_batch.length) break;
+    ARROW_RETURN_NOT_OK(handle_batch(batch, segment));
+    offset = segment.offset + segment.length;
+  }
+  return Status::OK();
+}
+
+Status GetScalarFields(std::vector<Datum>& values, const ExecBatch& input_batch,
+                       const std::vector<int>& field_ids) {
+  DCHECK_GT(input_batch.length, 0);
+  int64_t row = input_batch.length - 1;
+  values.clear();
+  values.resize(field_ids.size());
+  for (size_t i = 0; i < field_ids.size(); i++) {
+    const Datum& value = input_batch.values[field_ids[i]];
+    if (value.is_scalar()) {
+      values[i] = value;
+    } else if (value.is_array()) {
+      ARROW_ASSIGN_OR_RAISE(auto scalar, value.make_array()->GetScalar(row));
+      values[i] = scalar;
+    } else {
+      DCHECK(false);
+    }
+  }
+  return Status::OK();
+}
+
+void PlaceFields(ExecBatch& batch, size_t base, std::vector<Datum>& values) {
+  DCHECK_LE(base + values.size(), batch.values.size());
+  for (size_t i = 0; i < values.size(); i++) {
+    batch.values[base + i] = values[i];
+  }
+}
+
 class ScalarAggregateNode : public ExecNode, public TracedNode {
  public:
   ScalarAggregateNode(ExecPlan* plan, std::vector<ExecNode*> inputs,
                       std::shared_ptr<Schema> output_schema,
+                      std::unique_ptr<GroupingSegmenter> segmenter,

Review Comment:
   In regular (non-segmented/ordered) aggregation, the segmenter implementation takes the entire stream as one segment.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] icexelloss commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "icexelloss (via GitHub)" <gi...@apache.org>.
icexelloss commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1116221078


##########
cpp/src/arrow/compute/exec/aggregate_node.cc:
##########
@@ -43,7 +44,45 @@ namespace compute {
 
 namespace {
 
-namespace {
+/// \brief A gated shared mutex is similar to a shared mutex, in that it allows either
+/// multiple shared readers or a unique writer access to the mutex, except that a waiting
+/// writer gates future readers by preventing them from reacquiring shared access until it
+/// has acquired and released the mutex. This is useful for ensuring a writer is never
+/// starved by readers.
+struct GatedSharedMutex {
+  std::mutex gate;
+  std::shared_mutex mutex;
+};
+
+/// \brief Acquires unique access to a gatex mutex. This is useful for a unique writer.
+class GatedUniqueLock {
+ public:
+  // acquires the gate first, to ensure future readers will wait for its release
+  explicit GatedUniqueLock(GatedSharedMutex& gated_shared_mutex)
+      : lock_gate_(gated_shared_mutex.gate), lock_mutex_(gated_shared_mutex.mutex) {}
+
+ private:
+  std::unique_lock<std::mutex> lock_gate_;
+  std::unique_lock<std::shared_mutex> lock_mutex_;
+};
+
+/// \brief Acquires shared access to a gatex mutex. This is useful for a shared reader.
+class GatedSharedLock {

Review Comment:
   kk sounds good - let's keep things simple for now



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rtpsw commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "rtpsw (via GitHub)" <gi...@apache.org>.
rtpsw commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1117553145


##########
cpp/src/arrow/compute/exec/aggregate_node.cc:
##########
@@ -326,46 +446,86 @@ class ScalarAggregateNode : public ExecNode, public TracedNode {
   }
 
  private:
-  Status Finish() {
-    auto scope = TraceFinish();
+  Status ReconstructAggregates() {
+    const auto& input_schema = *inputs()[0]->output_schema();
+    auto exec_ctx = plan()->query_context()->exec_context();
+    for (size_t i = 0; i < kernels_.size(); ++i) {
+      std::vector<TypeHolder> in_types;
+      for (const auto& target : target_fieldsets_[i]) {
+        in_types.emplace_back(input_schema.field(target)->type().get());
+      }
+      states_[i].resize(plan()->query_context()->max_concurrency());

Review Comment:
   This pattern exists pre-PR and is used to support multi-threading (which, per our decision, is currently not supported in segmented aggregation but is in regular aggregation). Each worker threads is associated with a unique index from 0 to `max_concurrency - 1`. The code here creates a dedicated state instance per thread.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] icexelloss commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "icexelloss (via GitHub)" <gi...@apache.org>.
icexelloss commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1126886851


##########
cpp/src/arrow/compute/row/grouper.h:
##########
@@ -30,6 +30,49 @@
 namespace arrow {
 namespace compute {
 
+/// \brief A segment of contiguous rows for grouping
+struct ARROW_EXPORT GroupingSegment {
+  int64_t offset;
+  int64_t length;
+  bool is_open;
+  bool extends;
+};
+
+inline bool operator==(const GroupingSegment& segment1, const GroupingSegment& segment2) {
+  return segment1.offset == segment2.offset && segment1.length == segment2.length &&
+         segment1.is_open == segment2.is_open && segment1.extends == segment2.extends;
+}
+inline bool operator!=(const GroupingSegment& segment1, const GroupingSegment& segment2) {
+  return !(segment1 == segment2);
+}
+
+/// \brief Computes grouping segments for a batch. Each segment covers rows with identical
+/// values in the batch. The values in the batch are often selected as keys from a larger
+/// batch.
+class ARROW_EXPORT GroupingSegmenter {
+ public:
+  virtual ~GroupingSegmenter() = default;
+
+  /// \brief Construct a GroupingSegmenter which receives the specified key types
+  static Result<std::unique_ptr<GroupingSegmenter>> Make(
+      const std::vector<TypeHolder>& key_types, bool nullable_keys = false,
+      ExecContext* ctx = default_exec_context());

Review Comment:
   In general I think having default values for internal function makes it easier to make mistakes such as forgetting to override a default value. Most of the "Make" function of ExecNode, for example, don't use default values.
   
   "I designed the default values to get a common interface "
   
   Can you elaborate this? To get a common interface for what and how is that related to having default values here?
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] icexelloss commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "icexelloss (via GitHub)" <gi...@apache.org>.
icexelloss commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1126999723


##########
cpp/src/arrow/compute/row/grouper.cc:
##########
@@ -39,12 +43,330 @@
 namespace arrow {
 
 using internal::checked_cast;
+using internal::PrimitiveScalarBase;
 
 namespace compute {
 
 namespace {
 
-struct GrouperImpl : Grouper {
+constexpr uint32_t kNoGroupId = std::numeric_limits<uint32_t>::max();
+
+using group_id_t = std::remove_const<decltype(kNoGroupId)>::type;
+using GroupIdType = CTypeTraits<group_id_t>::ArrowType;
+auto g_group_id_type = std::make_shared<GroupIdType>();
+
+inline const uint8_t* GetValuesAsBytes(const ArraySpan& data, int64_t offset = 0) {
+  DCHECK_GT(data.type->byte_width(), 0);
+  int64_t absolute_byte_offset = (data.offset + offset) * data.type->byte_width();
+  return data.GetValues<uint8_t>(1, absolute_byte_offset);
+}
+
+template <typename Value>
+Status CheckForGetNextSegment(const std::vector<Value>& values, int64_t length,
+                              int64_t offset, const std::vector<TypeHolder>& key_types) {
+  if (offset < 0 || offset > length) {
+    return Status::Invalid("invalid grouping segmenter offset: ", offset);
+  }
+  if (values.size() != key_types.size()) {
+    return Status::Invalid("expected batch size ", key_types.size(), " but got ",
+                           values.size());
+  }
+  for (size_t i = 0; i < key_types.size(); i++) {
+    const auto& value = values[i];
+    const auto& key_type = key_types[i];
+    if (*value.type() != *key_type.type) {
+      return Status::Invalid("expected batch value ", i, " of type ", *key_type.type,
+                             " but got ", *value.type());
+    }
+  }
+  return Status::OK();
+}
+
+template <typename Batch>
+enable_if_t<std::is_same<Batch, ExecSpan>::value || std::is_same<Batch, ExecBatch>::value,
+            Status>
+CheckForGetNextSegment(const Batch& batch, int64_t offset,
+                       const std::vector<TypeHolder>& key_types) {
+  return CheckForGetNextSegment(batch.values, batch.length, offset, key_types);
+}
+
+struct BaseRowSegmenter : public RowSegmenter {
+  explicit BaseRowSegmenter(const std::vector<TypeHolder>& key_types)
+      : key_types_(key_types) {}
+
+  const std::vector<TypeHolder>& key_types() const override { return key_types_; }
+
+  std::vector<TypeHolder> key_types_;
+};
+
+Segment MakeSegment(int64_t batch_length, int64_t offset, int64_t length, bool extends) {
+  return Segment{offset, length, offset + length >= batch_length, extends};
+}
+
+int64_t GetMatchLength(const uint8_t* match_bytes, int64_t match_width,
+                       const uint8_t* array_bytes, int64_t offset, int64_t length) {
+  int64_t cursor, byte_cursor;
+  for (cursor = offset, byte_cursor = match_width * cursor; cursor < length;
+       cursor++, byte_cursor += match_width) {
+    if (memcmp(match_bytes, array_bytes + byte_cursor,
+               static_cast<size_t>(match_width)) != 0) {
+      break;
+    }
+  }
+  return std::min(cursor, length) - offset;
+}
+
+using ExtendFunc = std::function<bool(const void*)>;
+constexpr bool kDefaultExtends = true;
+constexpr bool kEmptyExtends = true;
+
+struct NoKeysSegmenter : public BaseRowSegmenter {
+  static std::unique_ptr<RowSegmenter> Make() {
+    return std::make_unique<NoKeysSegmenter>();
+  }
+
+  NoKeysSegmenter() : BaseRowSegmenter({}) {}
+
+  Status Reset() override { return Status::OK(); }
+
+  Result<Segment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {}));
+    return MakeSegment(batch.length, offset, batch.length - offset, kDefaultExtends);
+  }
+};
+
+struct SimpleKeySegmenter : public BaseRowSegmenter {
+  static Result<std::unique_ptr<RowSegmenter>> Make(TypeHolder key_type) {
+    return std::make_unique<SimpleKeySegmenter>(key_type);
+  }
+
+  explicit SimpleKeySegmenter(TypeHolder key_type)
+      : BaseRowSegmenter({key_type}), key_type_(key_types_[0]), save_key_data_() {}
+
+  Status CheckType(const DataType& type) {
+    if (!is_fixed_width(type)) {
+      return Status::Invalid("SimpleKeySegmenter does not support type ", type);
+    }
+    return Status::OK();
+  }
+
+  Status Reset() override {
+    save_key_data_.resize(0);
+    return Status::OK();
+  }
+
+  // Checks whether the given grouping data extends the current segment, i.e., is equal to
+  // previously seen grouping data, which is updated with each invocation.
+  bool Extend(const void* data) {
+    size_t byte_width = static_cast<size_t>(key_type_.type->byte_width());
+    bool extends = save_key_data_.size() != byte_width
+                       ? kDefaultExtends
+                       : 0 == memcmp(save_key_data_.data(), data, byte_width);
+    save_key_data_.resize(byte_width);
+    memcpy(save_key_data_.data(), data, byte_width);
+    return extends;
+  }
+
+  Result<Segment> GetNextSegment(const Scalar& scalar, int64_t offset, int64_t length) {
+    ARROW_RETURN_NOT_OK(CheckType(*scalar.type));
+    if (!scalar.is_valid) {
+      return Status::Invalid("segmenting an invalid scalar");
+    }
+    auto data = checked_cast<const PrimitiveScalarBase&>(scalar).data();
+    bool extends = length > 0 ? Extend(data) : kEmptyExtends;
+    return MakeSegment(length, offset, length, extends);
+  }
+
+  Result<Segment> GetNextSegment(const DataType& array_type, const uint8_t* array_bytes,
+                                 int64_t offset, int64_t length) {
+    RETURN_NOT_OK(CheckType(array_type));
+    int64_t byte_width = array_type.byte_width();
+    int64_t match_length = GetMatchLength(array_bytes + offset * byte_width, byte_width,
+                                          array_bytes, offset, length);
+    bool extends = length > 0 ? Extend(array_bytes + offset * byte_width) : kEmptyExtends;
+    return MakeSegment(length, offset, match_length, extends);
+  }
+
+  Result<Segment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {key_type_}));
+    if (offset == batch.length) {
+      return MakeSegment(batch.length, offset, 0, kEmptyExtends);
+    }
+    const auto& value = batch.values[0];
+    if (value.is_scalar()) {
+      return GetNextSegment(*value.scalar, offset, batch.length);
+    }
+    ARROW_DCHECK(value.is_array());
+    const auto& array = value.array;
+    if (array.GetNullCount() > 0) {
+      return Status::NotImplemented("segmenting a nullable array");
+    }
+    return GetNextSegment(*array.type, GetValuesAsBytes(array), offset, batch.length);
+  }
+
+ private:
+  TypeHolder key_type_;
+  std::vector<uint8_t> save_key_data_;
+};
+
+struct AnyKeysSegmenter : public BaseRowSegmenter {
+  static Result<std::unique_ptr<RowSegmenter>> Make(
+      const std::vector<TypeHolder>& key_types, ExecContext* ctx) {
+    ARROW_RETURN_NOT_OK(Grouper::Make(key_types, ctx));  // check types
+    return std::make_unique<AnyKeysSegmenter>(key_types, ctx);
+  }
+
+  AnyKeysSegmenter(const std::vector<TypeHolder>& key_types, ExecContext* ctx)
+      : BaseRowSegmenter(key_types),
+        ctx_(ctx),
+        grouper_(nullptr),
+        save_group_id_(kNoGroupId) {}
+
+  Status Reset() override {
+    grouper_ = nullptr;
+    save_group_id_ = kNoGroupId;
+    return Status::OK();
+  }
+
+  bool Extend(const void* data) {
+    auto group_id = *static_cast<const group_id_t*>(data);
+    bool extends =
+        save_group_id_ == kNoGroupId ? kDefaultExtends : save_group_id_ == group_id;
+    save_group_id_ = group_id;
+    return extends;
+  }
+
+  // Runs the grouper on a single row.  This is used to determine the group id of the
+  // first row of a new segment to see if it extends the previous segment.
+  template <typename Batch>
+  Result<group_id_t> MapGroupIdAt(const Batch& batch, int64_t offset) {
+    if (!grouper_) return kNoGroupId;
+    ARROW_ASSIGN_OR_RAISE(auto datum, grouper_->Consume(batch, offset,
+                                                        /*length=*/1));
+    if (!datum.is_array()) {
+      return Status::Invalid("accessing unsupported datum kind ", datum.kind());
+    }
+    const std::shared_ptr<ArrayData>& data = datum.array();
+    ARROW_DCHECK(data->GetNullCount() == 0);
+    DCHECK_EQ(data->type->id(), GroupIdType::type_id);
+    DCHECK_EQ(1, data->length);
+    const group_id_t* values = data->GetValues<group_id_t>(1);
+    return values[0];
+  }
+
+  Result<Segment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, key_types_));
+    if (offset == batch.length) {
+      return MakeSegment(batch.length, offset, 0, kEmptyExtends);
+    }
+    // ARROW-18311: make Grouper support Reset()
+    // so it can be cached instead of recreated below
+    //
+    // the group id must be computed prior to resetting the grouper, since it is compared
+    // to save_group_id_, and after resetting the grouper produces incomparable group ids
+    ARROW_ASSIGN_OR_RAISE(auto group_id, MapGroupIdAt(batch, offset));
+    ExtendFunc bound_extend = [this, group_id](const void* data) {
+      bool extends = Extend(&group_id);
+      save_group_id_ = *static_cast<const group_id_t*>(data);
+      return extends;
+    };
+    ARROW_ASSIGN_OR_RAISE(grouper_, Grouper::Make(key_types_, ctx_));  // TODO: reset it

Review Comment:
   Why do we need make sure a new grouper_ for each next segment?



##########
cpp/src/arrow/compute/row/grouper.cc:
##########
@@ -39,12 +43,330 @@
 namespace arrow {
 
 using internal::checked_cast;
+using internal::PrimitiveScalarBase;
 
 namespace compute {
 
 namespace {
 
-struct GrouperImpl : Grouper {
+constexpr uint32_t kNoGroupId = std::numeric_limits<uint32_t>::max();
+
+using group_id_t = std::remove_const<decltype(kNoGroupId)>::type;
+using GroupIdType = CTypeTraits<group_id_t>::ArrowType;
+auto g_group_id_type = std::make_shared<GroupIdType>();
+
+inline const uint8_t* GetValuesAsBytes(const ArraySpan& data, int64_t offset = 0) {
+  DCHECK_GT(data.type->byte_width(), 0);
+  int64_t absolute_byte_offset = (data.offset + offset) * data.type->byte_width();
+  return data.GetValues<uint8_t>(1, absolute_byte_offset);
+}
+
+template <typename Value>
+Status CheckForGetNextSegment(const std::vector<Value>& values, int64_t length,
+                              int64_t offset, const std::vector<TypeHolder>& key_types) {
+  if (offset < 0 || offset > length) {
+    return Status::Invalid("invalid grouping segmenter offset: ", offset);
+  }
+  if (values.size() != key_types.size()) {
+    return Status::Invalid("expected batch size ", key_types.size(), " but got ",
+                           values.size());
+  }
+  for (size_t i = 0; i < key_types.size(); i++) {
+    const auto& value = values[i];
+    const auto& key_type = key_types[i];
+    if (*value.type() != *key_type.type) {
+      return Status::Invalid("expected batch value ", i, " of type ", *key_type.type,
+                             " but got ", *value.type());
+    }
+  }
+  return Status::OK();
+}
+
+template <typename Batch>
+enable_if_t<std::is_same<Batch, ExecSpan>::value || std::is_same<Batch, ExecBatch>::value,
+            Status>
+CheckForGetNextSegment(const Batch& batch, int64_t offset,
+                       const std::vector<TypeHolder>& key_types) {
+  return CheckForGetNextSegment(batch.values, batch.length, offset, key_types);
+}
+
+struct BaseRowSegmenter : public RowSegmenter {
+  explicit BaseRowSegmenter(const std::vector<TypeHolder>& key_types)
+      : key_types_(key_types) {}
+
+  const std::vector<TypeHolder>& key_types() const override { return key_types_; }
+
+  std::vector<TypeHolder> key_types_;
+};
+
+Segment MakeSegment(int64_t batch_length, int64_t offset, int64_t length, bool extends) {
+  return Segment{offset, length, offset + length >= batch_length, extends};
+}
+
+int64_t GetMatchLength(const uint8_t* match_bytes, int64_t match_width,
+                       const uint8_t* array_bytes, int64_t offset, int64_t length) {
+  int64_t cursor, byte_cursor;
+  for (cursor = offset, byte_cursor = match_width * cursor; cursor < length;
+       cursor++, byte_cursor += match_width) {
+    if (memcmp(match_bytes, array_bytes + byte_cursor,
+               static_cast<size_t>(match_width)) != 0) {
+      break;
+    }
+  }
+  return std::min(cursor, length) - offset;
+}
+
+using ExtendFunc = std::function<bool(const void*)>;
+constexpr bool kDefaultExtends = true;
+constexpr bool kEmptyExtends = true;
+
+struct NoKeysSegmenter : public BaseRowSegmenter {
+  static std::unique_ptr<RowSegmenter> Make() {
+    return std::make_unique<NoKeysSegmenter>();
+  }
+
+  NoKeysSegmenter() : BaseRowSegmenter({}) {}
+
+  Status Reset() override { return Status::OK(); }
+
+  Result<Segment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {}));
+    return MakeSegment(batch.length, offset, batch.length - offset, kDefaultExtends);
+  }
+};
+
+struct SimpleKeySegmenter : public BaseRowSegmenter {
+  static Result<std::unique_ptr<RowSegmenter>> Make(TypeHolder key_type) {
+    return std::make_unique<SimpleKeySegmenter>(key_type);
+  }
+
+  explicit SimpleKeySegmenter(TypeHolder key_type)
+      : BaseRowSegmenter({key_type}), key_type_(key_types_[0]), save_key_data_() {}
+
+  Status CheckType(const DataType& type) {
+    if (!is_fixed_width(type)) {
+      return Status::Invalid("SimpleKeySegmenter does not support type ", type);
+    }
+    return Status::OK();
+  }
+
+  Status Reset() override {
+    save_key_data_.resize(0);
+    return Status::OK();
+  }
+
+  // Checks whether the given grouping data extends the current segment, i.e., is equal to
+  // previously seen grouping data, which is updated with each invocation.
+  bool Extend(const void* data) {
+    size_t byte_width = static_cast<size_t>(key_type_.type->byte_width());
+    bool extends = save_key_data_.size() != byte_width
+                       ? kDefaultExtends
+                       : 0 == memcmp(save_key_data_.data(), data, byte_width);
+    save_key_data_.resize(byte_width);
+    memcpy(save_key_data_.data(), data, byte_width);
+    return extends;
+  }
+
+  Result<Segment> GetNextSegment(const Scalar& scalar, int64_t offset, int64_t length) {
+    ARROW_RETURN_NOT_OK(CheckType(*scalar.type));
+    if (!scalar.is_valid) {
+      return Status::Invalid("segmenting an invalid scalar");
+    }
+    auto data = checked_cast<const PrimitiveScalarBase&>(scalar).data();
+    bool extends = length > 0 ? Extend(data) : kEmptyExtends;
+    return MakeSegment(length, offset, length, extends);
+  }
+
+  Result<Segment> GetNextSegment(const DataType& array_type, const uint8_t* array_bytes,
+                                 int64_t offset, int64_t length) {
+    RETURN_NOT_OK(CheckType(array_type));
+    int64_t byte_width = array_type.byte_width();
+    int64_t match_length = GetMatchLength(array_bytes + offset * byte_width, byte_width,
+                                          array_bytes, offset, length);
+    bool extends = length > 0 ? Extend(array_bytes + offset * byte_width) : kEmptyExtends;
+    return MakeSegment(length, offset, match_length, extends);
+  }
+
+  Result<Segment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {key_type_}));
+    if (offset == batch.length) {
+      return MakeSegment(batch.length, offset, 0, kEmptyExtends);
+    }
+    const auto& value = batch.values[0];
+    if (value.is_scalar()) {
+      return GetNextSegment(*value.scalar, offset, batch.length);
+    }
+    ARROW_DCHECK(value.is_array());
+    const auto& array = value.array;
+    if (array.GetNullCount() > 0) {
+      return Status::NotImplemented("segmenting a nullable array");
+    }
+    return GetNextSegment(*array.type, GetValuesAsBytes(array), offset, batch.length);
+  }
+
+ private:
+  TypeHolder key_type_;
+  std::vector<uint8_t> save_key_data_;
+};
+
+struct AnyKeysSegmenter : public BaseRowSegmenter {
+  static Result<std::unique_ptr<RowSegmenter>> Make(
+      const std::vector<TypeHolder>& key_types, ExecContext* ctx) {
+    ARROW_RETURN_NOT_OK(Grouper::Make(key_types, ctx));  // check types
+    return std::make_unique<AnyKeysSegmenter>(key_types, ctx);
+  }
+
+  AnyKeysSegmenter(const std::vector<TypeHolder>& key_types, ExecContext* ctx)
+      : BaseRowSegmenter(key_types),
+        ctx_(ctx),
+        grouper_(nullptr),
+        save_group_id_(kNoGroupId) {}
+
+  Status Reset() override {
+    grouper_ = nullptr;
+    save_group_id_ = kNoGroupId;
+    return Status::OK();
+  }
+
+  bool Extend(const void* data) {
+    auto group_id = *static_cast<const group_id_t*>(data);
+    bool extends =
+        save_group_id_ == kNoGroupId ? kDefaultExtends : save_group_id_ == group_id;
+    save_group_id_ = group_id;
+    return extends;
+  }
+
+  // Runs the grouper on a single row.  This is used to determine the group id of the
+  // first row of a new segment to see if it extends the previous segment.
+  template <typename Batch>
+  Result<group_id_t> MapGroupIdAt(const Batch& batch, int64_t offset) {
+    if (!grouper_) return kNoGroupId;
+    ARROW_ASSIGN_OR_RAISE(auto datum, grouper_->Consume(batch, offset,
+                                                        /*length=*/1));
+    if (!datum.is_array()) {
+      return Status::Invalid("accessing unsupported datum kind ", datum.kind());
+    }
+    const std::shared_ptr<ArrayData>& data = datum.array();
+    ARROW_DCHECK(data->GetNullCount() == 0);
+    DCHECK_EQ(data->type->id(), GroupIdType::type_id);
+    DCHECK_EQ(1, data->length);
+    const group_id_t* values = data->GetValues<group_id_t>(1);
+    return values[0];
+  }
+
+  Result<Segment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, key_types_));
+    if (offset == batch.length) {
+      return MakeSegment(batch.length, offset, 0, kEmptyExtends);
+    }
+    // ARROW-18311: make Grouper support Reset()
+    // so it can be cached instead of recreated below
+    //
+    // the group id must be computed prior to resetting the grouper, since it is compared
+    // to save_group_id_, and after resetting the grouper produces incomparable group ids
+    ARROW_ASSIGN_OR_RAISE(auto group_id, MapGroupIdAt(batch, offset));
+    ExtendFunc bound_extend = [this, group_id](const void* data) {
+      bool extends = Extend(&group_id);
+      save_group_id_ = *static_cast<const group_id_t*>(data);
+      return extends;
+    };
+    ARROW_ASSIGN_OR_RAISE(grouper_, Grouper::Make(key_types_, ctx_));  // TODO: reset it

Review Comment:
   Why do we need make  a new grouper_ for each next segment?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rtpsw commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "rtpsw (via GitHub)" <gi...@apache.org>.
rtpsw commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1127053729


##########
cpp/src/arrow/compute/row/grouper.cc:
##########
@@ -39,12 +43,330 @@
 namespace arrow {
 
 using internal::checked_cast;
+using internal::PrimitiveScalarBase;
 
 namespace compute {
 
 namespace {
 
-struct GrouperImpl : Grouper {
+constexpr uint32_t kNoGroupId = std::numeric_limits<uint32_t>::max();
+
+using group_id_t = std::remove_const<decltype(kNoGroupId)>::type;
+using GroupIdType = CTypeTraits<group_id_t>::ArrowType;
+auto g_group_id_type = std::make_shared<GroupIdType>();
+
+inline const uint8_t* GetValuesAsBytes(const ArraySpan& data, int64_t offset = 0) {
+  DCHECK_GT(data.type->byte_width(), 0);
+  int64_t absolute_byte_offset = (data.offset + offset) * data.type->byte_width();
+  return data.GetValues<uint8_t>(1, absolute_byte_offset);
+}
+
+template <typename Value>
+Status CheckForGetNextSegment(const std::vector<Value>& values, int64_t length,
+                              int64_t offset, const std::vector<TypeHolder>& key_types) {
+  if (offset < 0 || offset > length) {
+    return Status::Invalid("invalid grouping segmenter offset: ", offset);
+  }
+  if (values.size() != key_types.size()) {
+    return Status::Invalid("expected batch size ", key_types.size(), " but got ",
+                           values.size());
+  }
+  for (size_t i = 0; i < key_types.size(); i++) {
+    const auto& value = values[i];
+    const auto& key_type = key_types[i];
+    if (*value.type() != *key_type.type) {
+      return Status::Invalid("expected batch value ", i, " of type ", *key_type.type,
+                             " but got ", *value.type());
+    }
+  }
+  return Status::OK();
+}
+
+template <typename Batch>
+enable_if_t<std::is_same<Batch, ExecSpan>::value || std::is_same<Batch, ExecBatch>::value,
+            Status>
+CheckForGetNextSegment(const Batch& batch, int64_t offset,
+                       const std::vector<TypeHolder>& key_types) {
+  return CheckForGetNextSegment(batch.values, batch.length, offset, key_types);
+}
+
+struct BaseRowSegmenter : public RowSegmenter {
+  explicit BaseRowSegmenter(const std::vector<TypeHolder>& key_types)
+      : key_types_(key_types) {}
+
+  const std::vector<TypeHolder>& key_types() const override { return key_types_; }
+
+  std::vector<TypeHolder> key_types_;
+};
+
+Segment MakeSegment(int64_t batch_length, int64_t offset, int64_t length, bool extends) {
+  return Segment{offset, length, offset + length >= batch_length, extends};
+}
+
+int64_t GetMatchLength(const uint8_t* match_bytes, int64_t match_width,
+                       const uint8_t* array_bytes, int64_t offset, int64_t length) {
+  int64_t cursor, byte_cursor;
+  for (cursor = offset, byte_cursor = match_width * cursor; cursor < length;
+       cursor++, byte_cursor += match_width) {
+    if (memcmp(match_bytes, array_bytes + byte_cursor,
+               static_cast<size_t>(match_width)) != 0) {
+      break;
+    }
+  }
+  return std::min(cursor, length) - offset;
+}
+
+using ExtendFunc = std::function<bool(const void*)>;
+constexpr bool kDefaultExtends = true;
+constexpr bool kEmptyExtends = true;
+
+struct NoKeysSegmenter : public BaseRowSegmenter {
+  static std::unique_ptr<RowSegmenter> Make() {
+    return std::make_unique<NoKeysSegmenter>();
+  }
+
+  NoKeysSegmenter() : BaseRowSegmenter({}) {}
+
+  Status Reset() override { return Status::OK(); }
+
+  Result<Segment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {}));
+    return MakeSegment(batch.length, offset, batch.length - offset, kDefaultExtends);
+  }
+};
+
+struct SimpleKeySegmenter : public BaseRowSegmenter {
+  static Result<std::unique_ptr<RowSegmenter>> Make(TypeHolder key_type) {
+    return std::make_unique<SimpleKeySegmenter>(key_type);
+  }
+
+  explicit SimpleKeySegmenter(TypeHolder key_type)
+      : BaseRowSegmenter({key_type}), key_type_(key_types_[0]), save_key_data_() {}
+
+  Status CheckType(const DataType& type) {
+    if (!is_fixed_width(type)) {
+      return Status::Invalid("SimpleKeySegmenter does not support type ", type);
+    }
+    return Status::OK();
+  }
+
+  Status Reset() override {
+    save_key_data_.resize(0);
+    return Status::OK();
+  }
+
+  // Checks whether the given grouping data extends the current segment, i.e., is equal to
+  // previously seen grouping data, which is updated with each invocation.
+  bool Extend(const void* data) {
+    size_t byte_width = static_cast<size_t>(key_type_.type->byte_width());
+    bool extends = save_key_data_.size() != byte_width
+                       ? kDefaultExtends
+                       : 0 == memcmp(save_key_data_.data(), data, byte_width);
+    save_key_data_.resize(byte_width);
+    memcpy(save_key_data_.data(), data, byte_width);
+    return extends;
+  }
+
+  Result<Segment> GetNextSegment(const Scalar& scalar, int64_t offset, int64_t length) {
+    ARROW_RETURN_NOT_OK(CheckType(*scalar.type));
+    if (!scalar.is_valid) {
+      return Status::Invalid("segmenting an invalid scalar");
+    }
+    auto data = checked_cast<const PrimitiveScalarBase&>(scalar).data();
+    bool extends = length > 0 ? Extend(data) : kEmptyExtends;
+    return MakeSegment(length, offset, length, extends);
+  }
+
+  Result<Segment> GetNextSegment(const DataType& array_type, const uint8_t* array_bytes,
+                                 int64_t offset, int64_t length) {
+    RETURN_NOT_OK(CheckType(array_type));
+    int64_t byte_width = array_type.byte_width();
+    int64_t match_length = GetMatchLength(array_bytes + offset * byte_width, byte_width,
+                                          array_bytes, offset, length);
+    bool extends = length > 0 ? Extend(array_bytes + offset * byte_width) : kEmptyExtends;
+    return MakeSegment(length, offset, match_length, extends);
+  }
+
+  Result<Segment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {key_type_}));
+    if (offset == batch.length) {
+      return MakeSegment(batch.length, offset, 0, kEmptyExtends);
+    }
+    const auto& value = batch.values[0];
+    if (value.is_scalar()) {
+      return GetNextSegment(*value.scalar, offset, batch.length);
+    }
+    ARROW_DCHECK(value.is_array());
+    const auto& array = value.array;
+    if (array.GetNullCount() > 0) {
+      return Status::NotImplemented("segmenting a nullable array");
+    }
+    return GetNextSegment(*array.type, GetValuesAsBytes(array), offset, batch.length);
+  }
+
+ private:
+  TypeHolder key_type_;
+  std::vector<uint8_t> save_key_data_;
+};
+
+struct AnyKeysSegmenter : public BaseRowSegmenter {
+  static Result<std::unique_ptr<RowSegmenter>> Make(
+      const std::vector<TypeHolder>& key_types, ExecContext* ctx) {
+    ARROW_RETURN_NOT_OK(Grouper::Make(key_types, ctx));  // check types
+    return std::make_unique<AnyKeysSegmenter>(key_types, ctx);
+  }
+
+  AnyKeysSegmenter(const std::vector<TypeHolder>& key_types, ExecContext* ctx)
+      : BaseRowSegmenter(key_types),
+        ctx_(ctx),
+        grouper_(nullptr),
+        save_group_id_(kNoGroupId) {}
+
+  Status Reset() override {
+    grouper_ = nullptr;
+    save_group_id_ = kNoGroupId;
+    return Status::OK();
+  }
+
+  bool Extend(const void* data) {
+    auto group_id = *static_cast<const group_id_t*>(data);
+    bool extends =
+        save_group_id_ == kNoGroupId ? kDefaultExtends : save_group_id_ == group_id;
+    save_group_id_ = group_id;
+    return extends;
+  }
+
+  // Runs the grouper on a single row.  This is used to determine the group id of the
+  // first row of a new segment to see if it extends the previous segment.
+  template <typename Batch>
+  Result<group_id_t> MapGroupIdAt(const Batch& batch, int64_t offset) {
+    if (!grouper_) return kNoGroupId;
+    ARROW_ASSIGN_OR_RAISE(auto datum, grouper_->Consume(batch, offset,
+                                                        /*length=*/1));
+    if (!datum.is_array()) {
+      return Status::Invalid("accessing unsupported datum kind ", datum.kind());
+    }
+    const std::shared_ptr<ArrayData>& data = datum.array();
+    ARROW_DCHECK(data->GetNullCount() == 0);
+    DCHECK_EQ(data->type->id(), GroupIdType::type_id);
+    DCHECK_EQ(1, data->length);
+    const group_id_t* values = data->GetValues<group_id_t>(1);

Review Comment:
   This is standard Arrow code for accessing the (fixed-width) data buffer within the array.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rtpsw commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "rtpsw (via GitHub)" <gi...@apache.org>.
rtpsw commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1122287140


##########
cpp/src/arrow/compute/exec/aggregate_node.cc:
##########
@@ -584,29 +778,50 @@ class GroupByNode : public ExecNode, public TracedNode {
     ARROW_ASSIGN_OR_RAISE(out_data_, Finalize());
 
     int64_t num_output_batches = bit_util::CeilDiv(out_data_.length, output_batch_size());
-    RETURN_NOT_OK(output_->InputFinished(this, static_cast<int>(num_output_batches)));
-    return plan_->query_context()->StartTaskGroup(output_task_group_id_,
-                                                  num_output_batches);
+    total_output_batches_ += static_cast<int>(num_output_batches);
+    if (is_last) {
+      ARROW_RETURN_NOT_OK(output_->InputFinished(this, total_output_batches_));
+      RETURN_NOT_OK(plan_->query_context()->StartTaskGroup(output_task_group_id_,
+                                                           num_output_batches));
+    } else {
+      for (int64_t i = 0; i < num_output_batches; i++) {
+        ARROW_RETURN_NOT_OK(OutputNthBatch(i));

Review Comment:
   #34407



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rtpsw commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "rtpsw (via GitHub)" <gi...@apache.org>.
rtpsw commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1125563673


##########
cpp/src/arrow/compute/exec/aggregate_node.cc:
##########
@@ -169,17 +206,63 @@ void AggregatesToString(std::stringstream* ss, const Schema& input_schema,
   *ss << ']';
 }
 
+template <typename BatchHandler>
+Status HandleSegments(std::unique_ptr<GroupingSegmenter>& segmenter,
+                      const ExecBatch& batch, const std::vector<int>& ids,
+                      const BatchHandler& handle_batch) {
+  int64_t offset = 0;
+  ARROW_ASSIGN_OR_RAISE(auto segment_batch, batch.SelectValues(ids));
+  while (true) {
+    ARROW_ASSIGN_OR_RAISE(auto segment, segmenter->GetNextSegment(segment_batch, offset));
+    if (segment.offset >= segment_batch.length) break;
+    ARROW_RETURN_NOT_OK(handle_batch(batch, segment));
+    offset = segment.offset + segment.length;
+  }
+  return Status::OK();
+}
+
+Status GetScalarFields(std::vector<Datum>& values, const ExecBatch& input_batch,

Review Comment:
   Renamed to `SelectConstantFields`.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] icexelloss commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "icexelloss (via GitHub)" <gi...@apache.org>.
icexelloss commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1126700233


##########
cpp/src/arrow/compute/kernels/hash_aggregate_test.cc:
##########
@@ -301,53 +426,184 @@ Result<Datum> GroupByTest(const std::vector<Datum>& arguments,
         {t_agg.function, t_agg.options, "agg_" + ToChars(idx), t_agg.function});
     idx = idx + 1;
   }
-  return RunGroupBy(arguments, keys, internal_aggregates, use_threads);
+  return group_by(arguments, keys, segment_keys, internal_aggregates, use_threads,
+                  /*naive=*/false);
 }
 
-}  // namespace
+Result<Datum> GroupByTest(GroupByFunction group_by, const std::vector<Datum>& arguments,
+                          const std::vector<Datum>& keys,
+                          const std::vector<TestAggregate>& aggregates,
+                          bool use_threads) {
+  return GroupByTest(group_by, arguments, keys, {}, aggregates, use_threads);
+}
 
-TEST(Grouper, SupportedKeys) {
-  ASSERT_OK(Grouper::Make({boolean()}));
+template <typename GroupClass>

Review Comment:
   Can we add some comments about this in the test file? Seems like both me and Weston found this confusing.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rtpsw commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "rtpsw (via GitHub)" <gi...@apache.org>.
rtpsw commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1127476475


##########
cpp/src/arrow/compute/row/grouper.cc:
##########
@@ -39,12 +43,330 @@
 namespace arrow {
 
 using internal::checked_cast;
+using internal::PrimitiveScalarBase;
 
 namespace compute {
 
 namespace {
 
-struct GrouperImpl : Grouper {
+constexpr uint32_t kNoGroupId = std::numeric_limits<uint32_t>::max();
+
+using group_id_t = std::remove_const<decltype(kNoGroupId)>::type;
+using GroupIdType = CTypeTraits<group_id_t>::ArrowType;
+auto g_group_id_type = std::make_shared<GroupIdType>();
+
+inline const uint8_t* GetValuesAsBytes(const ArraySpan& data, int64_t offset = 0) {
+  DCHECK_GT(data.type->byte_width(), 0);
+  int64_t absolute_byte_offset = (data.offset + offset) * data.type->byte_width();
+  return data.GetValues<uint8_t>(1, absolute_byte_offset);
+}
+
+template <typename Value>
+Status CheckForGetNextSegment(const std::vector<Value>& values, int64_t length,
+                              int64_t offset, const std::vector<TypeHolder>& key_types) {
+  if (offset < 0 || offset > length) {
+    return Status::Invalid("invalid grouping segmenter offset: ", offset);
+  }
+  if (values.size() != key_types.size()) {
+    return Status::Invalid("expected batch size ", key_types.size(), " but got ",
+                           values.size());
+  }
+  for (size_t i = 0; i < key_types.size(); i++) {
+    const auto& value = values[i];
+    const auto& key_type = key_types[i];
+    if (*value.type() != *key_type.type) {
+      return Status::Invalid("expected batch value ", i, " of type ", *key_type.type,
+                             " but got ", *value.type());
+    }
+  }
+  return Status::OK();
+}
+
+template <typename Batch>
+enable_if_t<std::is_same<Batch, ExecSpan>::value || std::is_same<Batch, ExecBatch>::value,
+            Status>
+CheckForGetNextSegment(const Batch& batch, int64_t offset,
+                       const std::vector<TypeHolder>& key_types) {
+  return CheckForGetNextSegment(batch.values, batch.length, offset, key_types);
+}
+
+struct BaseRowSegmenter : public RowSegmenter {
+  explicit BaseRowSegmenter(const std::vector<TypeHolder>& key_types)
+      : key_types_(key_types) {}
+
+  const std::vector<TypeHolder>& key_types() const override { return key_types_; }
+
+  std::vector<TypeHolder> key_types_;
+};
+
+Segment MakeSegment(int64_t batch_length, int64_t offset, int64_t length, bool extends) {
+  return Segment{offset, length, offset + length >= batch_length, extends};
+}
+
+int64_t GetMatchLength(const uint8_t* match_bytes, int64_t match_width,
+                       const uint8_t* array_bytes, int64_t offset, int64_t length) {
+  int64_t cursor, byte_cursor;
+  for (cursor = offset, byte_cursor = match_width * cursor; cursor < length;
+       cursor++, byte_cursor += match_width) {
+    if (memcmp(match_bytes, array_bytes + byte_cursor,
+               static_cast<size_t>(match_width)) != 0) {
+      break;
+    }
+  }
+  return std::min(cursor, length) - offset;
+}
+
+using ExtendFunc = std::function<bool(const void*)>;
+constexpr bool kDefaultExtends = true;
+constexpr bool kEmptyExtends = true;
+
+struct NoKeysSegmenter : public BaseRowSegmenter {
+  static std::unique_ptr<RowSegmenter> Make() {
+    return std::make_unique<NoKeysSegmenter>();
+  }
+
+  NoKeysSegmenter() : BaseRowSegmenter({}) {}
+
+  Status Reset() override { return Status::OK(); }
+
+  Result<Segment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {}));
+    return MakeSegment(batch.length, offset, batch.length - offset, kDefaultExtends);
+  }
+};
+
+struct SimpleKeySegmenter : public BaseRowSegmenter {
+  static Result<std::unique_ptr<RowSegmenter>> Make(TypeHolder key_type) {
+    return std::make_unique<SimpleKeySegmenter>(key_type);
+  }
+
+  explicit SimpleKeySegmenter(TypeHolder key_type)
+      : BaseRowSegmenter({key_type}), key_type_(key_types_[0]), save_key_data_() {}
+
+  Status CheckType(const DataType& type) {
+    if (!is_fixed_width(type)) {
+      return Status::Invalid("SimpleKeySegmenter does not support type ", type);
+    }
+    return Status::OK();
+  }
+
+  Status Reset() override {
+    save_key_data_.resize(0);
+    return Status::OK();
+  }
+
+  // Checks whether the given grouping data extends the current segment, i.e., is equal to
+  // previously seen grouping data, which is updated with each invocation.
+  bool Extend(const void* data) {
+    size_t byte_width = static_cast<size_t>(key_type_.type->byte_width());
+    bool extends = save_key_data_.size() != byte_width
+                       ? kDefaultExtends
+                       : 0 == memcmp(save_key_data_.data(), data, byte_width);
+    save_key_data_.resize(byte_width);
+    memcpy(save_key_data_.data(), data, byte_width);
+    return extends;
+  }
+
+  Result<Segment> GetNextSegment(const Scalar& scalar, int64_t offset, int64_t length) {
+    ARROW_RETURN_NOT_OK(CheckType(*scalar.type));
+    if (!scalar.is_valid) {
+      return Status::Invalid("segmenting an invalid scalar");
+    }
+    auto data = checked_cast<const PrimitiveScalarBase&>(scalar).data();
+    bool extends = length > 0 ? Extend(data) : kEmptyExtends;
+    return MakeSegment(length, offset, length, extends);
+  }
+
+  Result<Segment> GetNextSegment(const DataType& array_type, const uint8_t* array_bytes,
+                                 int64_t offset, int64_t length) {
+    RETURN_NOT_OK(CheckType(array_type));
+    int64_t byte_width = array_type.byte_width();
+    int64_t match_length = GetMatchLength(array_bytes + offset * byte_width, byte_width,
+                                          array_bytes, offset, length);
+    bool extends = length > 0 ? Extend(array_bytes + offset * byte_width) : kEmptyExtends;
+    return MakeSegment(length, offset, match_length, extends);
+  }
+
+  Result<Segment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {key_type_}));
+    if (offset == batch.length) {
+      return MakeSegment(batch.length, offset, 0, kEmptyExtends);
+    }
+    const auto& value = batch.values[0];
+    if (value.is_scalar()) {
+      return GetNextSegment(*value.scalar, offset, batch.length);
+    }
+    ARROW_DCHECK(value.is_array());
+    const auto& array = value.array;
+    if (array.GetNullCount() > 0) {
+      return Status::NotImplemented("segmenting a nullable array");
+    }
+    return GetNextSegment(*array.type, GetValuesAsBytes(array), offset, batch.length);
+  }
+
+ private:
+  TypeHolder key_type_;
+  std::vector<uint8_t> save_key_data_;
+};
+
+struct AnyKeysSegmenter : public BaseRowSegmenter {
+  static Result<std::unique_ptr<RowSegmenter>> Make(
+      const std::vector<TypeHolder>& key_types, ExecContext* ctx) {
+    ARROW_RETURN_NOT_OK(Grouper::Make(key_types, ctx));  // check types
+    return std::make_unique<AnyKeysSegmenter>(key_types, ctx);
+  }
+
+  AnyKeysSegmenter(const std::vector<TypeHolder>& key_types, ExecContext* ctx)
+      : BaseRowSegmenter(key_types),
+        ctx_(ctx),
+        grouper_(nullptr),
+        save_group_id_(kNoGroupId) {}
+
+  Status Reset() override {
+    grouper_ = nullptr;
+    save_group_id_ = kNoGroupId;
+    return Status::OK();
+  }
+
+  bool Extend(const void* data) {
+    auto group_id = *static_cast<const group_id_t*>(data);
+    bool extends =
+        save_group_id_ == kNoGroupId ? kDefaultExtends : save_group_id_ == group_id;
+    save_group_id_ = group_id;
+    return extends;
+  }
+
+  // Runs the grouper on a single row.  This is used to determine the group id of the
+  // first row of a new segment to see if it extends the previous segment.
+  template <typename Batch>
+  Result<group_id_t> MapGroupIdAt(const Batch& batch, int64_t offset) {
+    if (!grouper_) return kNoGroupId;
+    ARROW_ASSIGN_OR_RAISE(auto datum, grouper_->Consume(batch, offset,
+                                                        /*length=*/1));
+    if (!datum.is_array()) {
+      return Status::Invalid("accessing unsupported datum kind ", datum.kind());
+    }
+    const std::shared_ptr<ArrayData>& data = datum.array();
+    ARROW_DCHECK(data->GetNullCount() == 0);
+    DCHECK_EQ(data->type->id(), GroupIdType::type_id);
+    DCHECK_EQ(1, data->length);
+    const group_id_t* values = data->GetValues<group_id_t>(1);

Review Comment:
   I really don't see a reason to add a comment for this idiom. There are plenty of places in Arrow code where this idiom is used with no comment. The columnar format of [arrays](https://arrow.apache.org/docs/format/Columnar.html) and [fixed-width arrays](https://arrow.apache.org/docs/format/Columnar.html#fixed-size-primitive-layout) is well-documented.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rtpsw commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "rtpsw (via GitHub)" <gi...@apache.org>.
rtpsw commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1125648808


##########
cpp/src/arrow/compute/exec/aggregate_node.cc:
##########
@@ -697,13 +909,18 @@ class GroupByNode : public ExecNode, public TracedNode {
   }
 
   int output_task_group_id_;
+  std::unique_ptr<GroupingSegmenter> segmenter_;

Review Comment:
   [Here](https://github.com/apache/arrow/pull/34311/files#diff-f53040e5fc284581263737f96238c8a76ef54d13f01e34305c1d1d9de5e6e34bR492-R509) and [here](https://github.com/apache/arrow/pull/34311/files#diff-f53040e5fc284581263737f96238c8a76ef54d13f01e34305c1d1d9de5e6e34bR925-R941).



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] icexelloss commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "icexelloss (via GitHub)" <gi...@apache.org>.
icexelloss commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1126985371


##########
cpp/src/arrow/compute/row/grouper.cc:
##########
@@ -39,12 +43,330 @@
 namespace arrow {
 
 using internal::checked_cast;
+using internal::PrimitiveScalarBase;
 
 namespace compute {
 
 namespace {
 
-struct GrouperImpl : Grouper {
+constexpr uint32_t kNoGroupId = std::numeric_limits<uint32_t>::max();
+
+using group_id_t = std::remove_const<decltype(kNoGroupId)>::type;
+using GroupIdType = CTypeTraits<group_id_t>::ArrowType;
+auto g_group_id_type = std::make_shared<GroupIdType>();
+
+inline const uint8_t* GetValuesAsBytes(const ArraySpan& data, int64_t offset = 0) {
+  DCHECK_GT(data.type->byte_width(), 0);
+  int64_t absolute_byte_offset = (data.offset + offset) * data.type->byte_width();
+  return data.GetValues<uint8_t>(1, absolute_byte_offset);
+}
+
+template <typename Value>
+Status CheckForGetNextSegment(const std::vector<Value>& values, int64_t length,
+                              int64_t offset, const std::vector<TypeHolder>& key_types) {
+  if (offset < 0 || offset > length) {
+    return Status::Invalid("invalid grouping segmenter offset: ", offset);
+  }
+  if (values.size() != key_types.size()) {
+    return Status::Invalid("expected batch size ", key_types.size(), " but got ",
+                           values.size());
+  }
+  for (size_t i = 0; i < key_types.size(); i++) {
+    const auto& value = values[i];
+    const auto& key_type = key_types[i];
+    if (*value.type() != *key_type.type) {
+      return Status::Invalid("expected batch value ", i, " of type ", *key_type.type,
+                             " but got ", *value.type());
+    }
+  }
+  return Status::OK();
+}
+
+template <typename Batch>
+enable_if_t<std::is_same<Batch, ExecSpan>::value || std::is_same<Batch, ExecBatch>::value,
+            Status>
+CheckForGetNextSegment(const Batch& batch, int64_t offset,
+                       const std::vector<TypeHolder>& key_types) {
+  return CheckForGetNextSegment(batch.values, batch.length, offset, key_types);
+}
+
+struct BaseRowSegmenter : public RowSegmenter {
+  explicit BaseRowSegmenter(const std::vector<TypeHolder>& key_types)
+      : key_types_(key_types) {}
+
+  const std::vector<TypeHolder>& key_types() const override { return key_types_; }
+
+  std::vector<TypeHolder> key_types_;
+};
+
+Segment MakeSegment(int64_t batch_length, int64_t offset, int64_t length, bool extends) {
+  return Segment{offset, length, offset + length >= batch_length, extends};
+}
+
+int64_t GetMatchLength(const uint8_t* match_bytes, int64_t match_width,
+                       const uint8_t* array_bytes, int64_t offset, int64_t length) {
+  int64_t cursor, byte_cursor;
+  for (cursor = offset, byte_cursor = match_width * cursor; cursor < length;
+       cursor++, byte_cursor += match_width) {
+    if (memcmp(match_bytes, array_bytes + byte_cursor,
+               static_cast<size_t>(match_width)) != 0) {
+      break;
+    }
+  }
+  return std::min(cursor, length) - offset;
+}
+
+using ExtendFunc = std::function<bool(const void*)>;
+constexpr bool kDefaultExtends = true;
+constexpr bool kEmptyExtends = true;
+
+struct NoKeysSegmenter : public BaseRowSegmenter {
+  static std::unique_ptr<RowSegmenter> Make() {
+    return std::make_unique<NoKeysSegmenter>();
+  }
+
+  NoKeysSegmenter() : BaseRowSegmenter({}) {}
+
+  Status Reset() override { return Status::OK(); }
+
+  Result<Segment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {}));
+    return MakeSegment(batch.length, offset, batch.length - offset, kDefaultExtends);
+  }
+};
+
+struct SimpleKeySegmenter : public BaseRowSegmenter {
+  static Result<std::unique_ptr<RowSegmenter>> Make(TypeHolder key_type) {
+    return std::make_unique<SimpleKeySegmenter>(key_type);
+  }
+
+  explicit SimpleKeySegmenter(TypeHolder key_type)
+      : BaseRowSegmenter({key_type}), key_type_(key_types_[0]), save_key_data_() {}
+
+  Status CheckType(const DataType& type) {
+    if (!is_fixed_width(type)) {
+      return Status::Invalid("SimpleKeySegmenter does not support type ", type);
+    }
+    return Status::OK();
+  }
+
+  Status Reset() override {
+    save_key_data_.resize(0);
+    return Status::OK();
+  }
+
+  // Checks whether the given grouping data extends the current segment, i.e., is equal to
+  // previously seen grouping data, which is updated with each invocation.
+  bool Extend(const void* data) {
+    size_t byte_width = static_cast<size_t>(key_type_.type->byte_width());
+    bool extends = save_key_data_.size() != byte_width
+                       ? kDefaultExtends
+                       : 0 == memcmp(save_key_data_.data(), data, byte_width);
+    save_key_data_.resize(byte_width);
+    memcpy(save_key_data_.data(), data, byte_width);
+    return extends;
+  }
+
+  Result<Segment> GetNextSegment(const Scalar& scalar, int64_t offset, int64_t length) {
+    ARROW_RETURN_NOT_OK(CheckType(*scalar.type));
+    if (!scalar.is_valid) {
+      return Status::Invalid("segmenting an invalid scalar");
+    }
+    auto data = checked_cast<const PrimitiveScalarBase&>(scalar).data();
+    bool extends = length > 0 ? Extend(data) : kEmptyExtends;
+    return MakeSegment(length, offset, length, extends);
+  }
+
+  Result<Segment> GetNextSegment(const DataType& array_type, const uint8_t* array_bytes,
+                                 int64_t offset, int64_t length) {
+    RETURN_NOT_OK(CheckType(array_type));
+    int64_t byte_width = array_type.byte_width();
+    int64_t match_length = GetMatchLength(array_bytes + offset * byte_width, byte_width,
+                                          array_bytes, offset, length);
+    bool extends = length > 0 ? Extend(array_bytes + offset * byte_width) : kEmptyExtends;
+    return MakeSegment(length, offset, match_length, extends);
+  }
+
+  Result<Segment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {key_type_}));
+    if (offset == batch.length) {
+      return MakeSegment(batch.length, offset, 0, kEmptyExtends);
+    }
+    const auto& value = batch.values[0];
+    if (value.is_scalar()) {
+      return GetNextSegment(*value.scalar, offset, batch.length);
+    }
+    ARROW_DCHECK(value.is_array());
+    const auto& array = value.array;
+    if (array.GetNullCount() > 0) {
+      return Status::NotImplemented("segmenting a nullable array");
+    }
+    return GetNextSegment(*array.type, GetValuesAsBytes(array), offset, batch.length);
+  }
+
+ private:
+  TypeHolder key_type_;
+  std::vector<uint8_t> save_key_data_;
+};
+
+struct AnyKeysSegmenter : public BaseRowSegmenter {
+  static Result<std::unique_ptr<RowSegmenter>> Make(
+      const std::vector<TypeHolder>& key_types, ExecContext* ctx) {
+    ARROW_RETURN_NOT_OK(Grouper::Make(key_types, ctx));  // check types
+    return std::make_unique<AnyKeysSegmenter>(key_types, ctx);
+  }
+
+  AnyKeysSegmenter(const std::vector<TypeHolder>& key_types, ExecContext* ctx)
+      : BaseRowSegmenter(key_types),
+        ctx_(ctx),
+        grouper_(nullptr),
+        save_group_id_(kNoGroupId) {}
+
+  Status Reset() override {
+    grouper_ = nullptr;
+    save_group_id_ = kNoGroupId;
+    return Status::OK();
+  }
+
+  bool Extend(const void* data) {
+    auto group_id = *static_cast<const group_id_t*>(data);
+    bool extends =
+        save_group_id_ == kNoGroupId ? kDefaultExtends : save_group_id_ == group_id;
+    save_group_id_ = group_id;
+    return extends;
+  }
+
+  // Runs the grouper on a single row.  This is used to determine the group id of the
+  // first row of a new segment to see if it extends the previous segment.
+  template <typename Batch>
+  Result<group_id_t> MapGroupIdAt(const Batch& batch, int64_t offset) {
+    if (!grouper_) return kNoGroupId;
+    ARROW_ASSIGN_OR_RAISE(auto datum, grouper_->Consume(batch, offset,
+                                                        /*length=*/1));
+    if (!datum.is_array()) {
+      return Status::Invalid("accessing unsupported datum kind ", datum.kind());
+    }
+    const std::shared_ptr<ArrayData>& data = datum.array();
+    ARROW_DCHECK(data->GetNullCount() == 0);
+    DCHECK_EQ(data->type->id(), GroupIdType::type_id);
+    DCHECK_EQ(1, data->length);
+    const group_id_t* values = data->GetValues<group_id_t>(1);
+    return values[0];
+  }
+
+  Result<Segment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, key_types_));
+    if (offset == batch.length) {
+      return MakeSegment(batch.length, offset, 0, kEmptyExtends);
+    }
+    // ARROW-18311: make Grouper support Reset()
+    // so it can be cached instead of recreated below
+    //
+    // the group id must be computed prior to resetting the grouper, since it is compared
+    // to save_group_id_, and after resetting the grouper produces incomparable group ids
+    ARROW_ASSIGN_OR_RAISE(auto group_id, MapGroupIdAt(batch, offset));
+    ExtendFunc bound_extend = [this, group_id](const void* data) {
+      bool extends = Extend(&group_id);
+      save_group_id_ = *static_cast<const group_id_t*>(data);
+      return extends;
+    };
+    ARROW_ASSIGN_OR_RAISE(grouper_, Grouper::Make(key_types_, ctx_));  // TODO: reset it
+    ARROW_ASSIGN_OR_RAISE(auto datum, grouper_->Consume(batch, offset));
+    if (datum.is_array()) {
+      const std::shared_ptr<ArrayData>& data = datum.array();
+      ARROW_DCHECK(data->GetNullCount() == 0);
+      DCHECK_EQ(data->type->id(), GroupIdType::type_id);
+      const group_id_t* values = data->GetValues<group_id_t>(1);
+      int64_t cursor;
+      for (cursor = 1; cursor < data->length; cursor++) {
+        if (values[0] != values[cursor]) break;
+      }
+      int64_t length = std::min(cursor, batch.length - offset);

Review Comment:
   Is this correct? Shouldn't length be `cursor - offset`?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] icexelloss commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "icexelloss (via GitHub)" <gi...@apache.org>.
icexelloss commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1127036454


##########
cpp/src/arrow/compute/row/grouper.cc:
##########
@@ -39,12 +43,330 @@
 namespace arrow {
 
 using internal::checked_cast;
+using internal::PrimitiveScalarBase;
 
 namespace compute {
 
 namespace {
 
-struct GrouperImpl : Grouper {
+constexpr uint32_t kNoGroupId = std::numeric_limits<uint32_t>::max();
+
+using group_id_t = std::remove_const<decltype(kNoGroupId)>::type;
+using GroupIdType = CTypeTraits<group_id_t>::ArrowType;
+auto g_group_id_type = std::make_shared<GroupIdType>();
+
+inline const uint8_t* GetValuesAsBytes(const ArraySpan& data, int64_t offset = 0) {
+  DCHECK_GT(data.type->byte_width(), 0);
+  int64_t absolute_byte_offset = (data.offset + offset) * data.type->byte_width();
+  return data.GetValues<uint8_t>(1, absolute_byte_offset);
+}
+
+template <typename Value>
+Status CheckForGetNextSegment(const std::vector<Value>& values, int64_t length,
+                              int64_t offset, const std::vector<TypeHolder>& key_types) {
+  if (offset < 0 || offset > length) {
+    return Status::Invalid("invalid grouping segmenter offset: ", offset);
+  }
+  if (values.size() != key_types.size()) {
+    return Status::Invalid("expected batch size ", key_types.size(), " but got ",
+                           values.size());
+  }
+  for (size_t i = 0; i < key_types.size(); i++) {
+    const auto& value = values[i];
+    const auto& key_type = key_types[i];
+    if (*value.type() != *key_type.type) {
+      return Status::Invalid("expected batch value ", i, " of type ", *key_type.type,
+                             " but got ", *value.type());
+    }
+  }
+  return Status::OK();
+}
+
+template <typename Batch>
+enable_if_t<std::is_same<Batch, ExecSpan>::value || std::is_same<Batch, ExecBatch>::value,
+            Status>
+CheckForGetNextSegment(const Batch& batch, int64_t offset,
+                       const std::vector<TypeHolder>& key_types) {
+  return CheckForGetNextSegment(batch.values, batch.length, offset, key_types);
+}
+
+struct BaseRowSegmenter : public RowSegmenter {
+  explicit BaseRowSegmenter(const std::vector<TypeHolder>& key_types)
+      : key_types_(key_types) {}
+
+  const std::vector<TypeHolder>& key_types() const override { return key_types_; }
+
+  std::vector<TypeHolder> key_types_;
+};
+
+Segment MakeSegment(int64_t batch_length, int64_t offset, int64_t length, bool extends) {
+  return Segment{offset, length, offset + length >= batch_length, extends};
+}
+
+int64_t GetMatchLength(const uint8_t* match_bytes, int64_t match_width,
+                       const uint8_t* array_bytes, int64_t offset, int64_t length) {
+  int64_t cursor, byte_cursor;
+  for (cursor = offset, byte_cursor = match_width * cursor; cursor < length;
+       cursor++, byte_cursor += match_width) {
+    if (memcmp(match_bytes, array_bytes + byte_cursor,
+               static_cast<size_t>(match_width)) != 0) {
+      break;
+    }
+  }
+  return std::min(cursor, length) - offset;
+}
+
+using ExtendFunc = std::function<bool(const void*)>;
+constexpr bool kDefaultExtends = true;
+constexpr bool kEmptyExtends = true;
+
+struct NoKeysSegmenter : public BaseRowSegmenter {
+  static std::unique_ptr<RowSegmenter> Make() {
+    return std::make_unique<NoKeysSegmenter>();
+  }
+
+  NoKeysSegmenter() : BaseRowSegmenter({}) {}
+
+  Status Reset() override { return Status::OK(); }
+
+  Result<Segment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {}));
+    return MakeSegment(batch.length, offset, batch.length - offset, kDefaultExtends);
+  }
+};
+
+struct SimpleKeySegmenter : public BaseRowSegmenter {
+  static Result<std::unique_ptr<RowSegmenter>> Make(TypeHolder key_type) {
+    return std::make_unique<SimpleKeySegmenter>(key_type);
+  }
+
+  explicit SimpleKeySegmenter(TypeHolder key_type)
+      : BaseRowSegmenter({key_type}), key_type_(key_types_[0]), save_key_data_() {}
+
+  Status CheckType(const DataType& type) {
+    if (!is_fixed_width(type)) {
+      return Status::Invalid("SimpleKeySegmenter does not support type ", type);
+    }
+    return Status::OK();
+  }
+
+  Status Reset() override {
+    save_key_data_.resize(0);
+    return Status::OK();
+  }
+
+  // Checks whether the given grouping data extends the current segment, i.e., is equal to
+  // previously seen grouping data, which is updated with each invocation.
+  bool Extend(const void* data) {
+    size_t byte_width = static_cast<size_t>(key_type_.type->byte_width());
+    bool extends = save_key_data_.size() != byte_width
+                       ? kDefaultExtends
+                       : 0 == memcmp(save_key_data_.data(), data, byte_width);
+    save_key_data_.resize(byte_width);

Review Comment:
   Why resize here? Do we expect byte_width of key_type to change?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] icexelloss commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "icexelloss (via GitHub)" <gi...@apache.org>.
icexelloss commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1123611908


##########
cpp/src/arrow/compute/exec/aggregate_node.cc:
##########
@@ -283,28 +386,36 @@ class ScalarAggregateNode : public ExecNode, public TracedNode {
     DCHECK_EQ(input, inputs_[0]);
 
     auto thread_index = plan_->query_context()->GetThreadIndex();
-
-    ARROW_RETURN_NOT_OK(DoConsume(ExecSpan(batch), thread_index));
+    auto handler = [this, thread_index](const ExecBatch& full_batch,

Review Comment:
   Why do we pass the handler here instead of just implementing the logic inside `HandleSegments`? I found the indirection here a bit confusing/unnecessary. Is that a specific reason to do it this way?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rtpsw commented on pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "rtpsw (via GitHub)" <gi...@apache.org>.
rtpsw commented on PR #34311:
URL: https://github.com/apache/arrow/pull/34311#issuecomment-1454734602

   @westonpace, any idea what these failures are about?
   ```
   [ RUN      ] RecordReaderPrimitveTypeTests/RecordReaderPrimitiveTypeTest.ReadNullableRepeated/true
   D:/a/arrow/arrow/cpp/src/parquet/column_reader_test.cc:939: Failure
   Expected equality of these values:
     record_reader_->descr()->max_definition_level()
       Which is: 3
     1
   [  FAILED  ] RecordReaderPrimitveTypeTests/RecordReaderPrimitiveTypeTest.ReadNullableRepeated/true, where GetParam() = true (0 ms)
   [ RUN      ] RecordReaderPrimitveTypeTests/RecordReaderPrimitiveTypeTest.ReadNullableRepeated/false
   D:/a/arrow/arrow/cpp/src/parquet/column_reader_test.cc:939: Failure
   Expected equality of these values:
     record_reader_->descr()->max_definition_level()
       Which is: 3
     1
   [  FAILED  ] RecordReaderPrimitveTypeTests/RecordReaderPrimitiveTypeTest.ReadNullableRepeated/false, where GetParam() = false (0 ms)
   ``` 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] icexelloss commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "icexelloss (via GitHub)" <gi...@apache.org>.
icexelloss commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1123700998


##########
cpp/src/arrow/compute/exec/aggregate_node.cc:
##########
@@ -326,10 +437,22 @@ class ScalarAggregateNode : public ExecNode, public TracedNode {
   }
 
  private:
-  Status Finish() {
-    auto scope = TraceFinish();
+  Status ResetAggregates() {
+    auto exec_ctx = plan()->query_context()->exec_context();
+    for (size_t i = 0; i < kernels_.size(); ++i) {
+      const std::vector<TypeHolder>& in_types = in_typesets_[i];
+      states_[i].resize(plan()->query_context()->max_concurrency());
+      KernelContext kernel_ctx{exec_ctx};
+      RETURN_NOT_OK(Kernel::InitAll(
+          &kernel_ctx, KernelInitArgs{kernels_[i], in_types, aggs_[i].options.get()},
+          &states_[i]));
+    }
+    return Status::OK();
+  }
+
+  Status OutputResult(bool is_last = false) {
     ExecBatch batch{{}, 1};
-    batch.values.resize(kernels_.size());
+    batch.values.resize(kernels_.size() + segment_field_ids_.size());

Review Comment:
   Why do we need to include segment key sizes in  the output batch size?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] icexelloss commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "icexelloss (via GitHub)" <gi...@apache.org>.
icexelloss commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1129911003


##########
cpp/src/arrow/compute/exec/aggregate_node.cc:
##########
@@ -169,35 +185,117 @@ void AggregatesToString(std::stringstream* ss, const Schema& input_schema,
   *ss << ']';
 }
 
+template <typename BatchHandler>
+Status HandleSegments(std::unique_ptr<GroupingSegmenter>& segmenter,
+                      const ExecBatch& batch, const std::vector<int>& ids,
+                      const BatchHandler& handle_batch) {
+  int64_t offset = 0;
+  ARROW_ASSIGN_OR_RAISE(auto segment_exec_batch, batch.SelectValues(ids));
+  ExecSpan segment_batch(segment_exec_batch);
+  while (true) {
+    ARROW_ASSIGN_OR_RAISE(auto segment, segmenter->GetNextSegment(segment_batch, offset));
+    if (segment.offset >= segment_batch.length) break;  // condition of no-next-segment
+    ARROW_RETURN_NOT_OK(handle_batch(batch, segment));
+    offset = segment.offset + segment.length;
+  }
+  return Status::OK();
+}
+
+Status GetScalarFields(std::vector<Datum>* values_ptr, const ExecBatch& input_batch,
+                       const std::vector<int>& field_ids) {
+  DCHECK_GT(input_batch.length, 0);
+  std::vector<Datum>& values = *values_ptr;
+  int64_t row = input_batch.length - 1;
+  values.clear();
+  values.resize(field_ids.size());
+  for (size_t i = 0; i < field_ids.size(); i++) {
+    const Datum& value = input_batch.values[field_ids[i]];
+    if (value.is_scalar()) {

Review Comment:
   Thanks Weston



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rtpsw commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "rtpsw (via GitHub)" <gi...@apache.org>.
rtpsw commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1128215690


##########
cpp/src/arrow/compute/kernels/hash_aggregate_test.cc:
##########
@@ -174,81 +255,117 @@ Result<Datum> RunGroupBy(const BatchesWithSchema& input,
   ARROW_ASSIGN_OR_RAISE(std::vector<ExecBatch> output_batches,
                         start_and_collect.MoveResult());
 
-  ArrayVector out_arrays(aggregates.size() + key_names.size());
   const auto& output_schema = plan->nodes()[0]->output()->output_schema();
+  if (!segmented) {
+    return MakeGroupByOutput(output_batches, output_schema, aggregates.size(),
+                             key_names.size(), naive);
+  }
+
+  std::vector<ArrayVector> out_arrays(aggregates.size() + key_names.size() +
+                                      segment_key_names.size());
   for (size_t i = 0; i < out_arrays.size(); ++i) {
     std::vector<std::shared_ptr<Array>> arrays(output_batches.size());
     for (size_t j = 0; j < output_batches.size(); ++j) {
-      arrays[j] = output_batches[j].values[i].make_array();
+      auto& value = output_batches[j].values[i];
+      if (value.is_scalar()) {
+        ARROW_ASSIGN_OR_RAISE(
+            arrays[j], MakeArrayFromScalar(*value.scalar(), output_batches[j].length));
+      } else if (value.is_array()) {
+        arrays[j] = value.make_array();
+      } else {
+        return Status::Invalid("GroupByUsingExecPlan unsupported value kind ",
+                               ToString(value.kind()));
+      }
     }
     if (arrays.empty()) {
+      arrays.resize(1);
       ARROW_ASSIGN_OR_RAISE(
-          out_arrays[i],
-          MakeArrayOfNull(output_schema->field(static_cast<int>(i))->type(),
-                          /*length=*/0));
-    } else {
-      ARROW_ASSIGN_OR_RAISE(out_arrays[i], Concatenate(arrays));
+          arrays[0], MakeArrayOfNull(output_schema->field(static_cast<int>(i))->type(),
+                                     /*length=*/0));
     }
+    out_arrays[i] = {std::move(arrays)};
   }
 
-  // The exec plan may reorder the output rows.  The tests are all setup to expect ouptut
-  // in ascending order of keys.  So we need to sort the result by the key columns.  To do
-  // that we create a table using the key columns, calculate the sort indices from that
-  // table (sorting on all fields) and then use those indices to calculate our result.
-  std::vector<std::shared_ptr<Field>> key_fields;
-  std::vector<std::shared_ptr<Array>> key_columns;
-  std::vector<SortKey> sort_keys;
-  for (std::size_t i = 0; i < key_names.size(); i++) {
-    const std::shared_ptr<Array>& arr = out_arrays[i + aggregates.size()];
-    if (arr->type_id() == Type::DICTIONARY) {
-      // Can't sort dictionary columns so need to decode
-      auto dict_arr = checked_pointer_cast<DictionaryArray>(arr);
-      ARROW_ASSIGN_OR_RAISE(auto decoded_arr,
-                            Take(*dict_arr->dictionary(), *dict_arr->indices()));
-      key_columns.push_back(decoded_arr);
-      key_fields.push_back(
-          field("name_does_not_matter", dict_arr->dict_type()->value_type()));
-    } else {
-      key_columns.push_back(arr);
-      key_fields.push_back(field("name_does_not_matter", arr->type()));
+  if (segmented && segment_key_names.size() > 0) {
+    ArrayVector struct_arrays;
+    struct_arrays.reserve(output_batches.size());
+    for (size_t j = 0; j < output_batches.size(); ++j) {
+      ArrayVector struct_fields;
+      struct_fields.reserve(out_arrays.size());
+      for (auto out_array : out_arrays) {
+        struct_fields.push_back(out_array[j]);
+      }
+      ARROW_ASSIGN_OR_RAISE(auto struct_array,
+                            StructArray::Make(struct_fields, output_schema->fields()));
+      struct_arrays.push_back(struct_array);
     }
-    sort_keys.emplace_back(static_cast<int>(i));
+    return ChunkedArray::Make(struct_arrays);
+  } else {
+    ArrayVector struct_fields(out_arrays.size());
+    for (size_t i = 0; i < out_arrays.size(); ++i) {
+      ARROW_ASSIGN_OR_RAISE(struct_fields[i], Concatenate(out_arrays[i]));
+    }
+    return StructArray::Make(std::move(struct_fields), output_schema->fields());
   }
-  std::shared_ptr<Schema> key_schema = schema(std::move(key_fields));
-  std::shared_ptr<Table> key_table = Table::Make(std::move(key_schema), key_columns);
-  SortOptions sort_options(std::move(sort_keys));
-  ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Array> sort_indices,
-                        SortIndices(key_table, sort_options));
+}
 
-  ARROW_ASSIGN_OR_RAISE(
-      std::shared_ptr<Array> struct_arr,
-      StructArray::Make(std::move(out_arrays), output_schema->fields()));
+Result<Datum> RunGroupBy(const BatchesWithSchema& input,
+                         const std::vector<std::string>& key_names,
+                         const std::vector<std::string>& segment_key_names,
+                         const std::vector<Aggregate>& aggregates, bool use_threads,
+                         bool segmented = false, bool naive = false) {
+  if (segment_key_names.size() > 0) {
+    ARROW_ASSIGN_OR_RAISE(auto thread_pool, arrow::internal::ThreadPool::Make(1));
+    ExecContext seq_ctx(default_memory_pool(), thread_pool.get());
+    return RunGroupBy(input, key_names, segment_key_names, aggregates, &seq_ctx,

Review Comment:
   I'll defer on this to @westonpace because the pre-PR code, which was refactored into the else-branch, also uses both `ExecContext` and `use_threads` as arguments to `RunGroupBy`.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rtpsw commented on pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "rtpsw (via GitHub)" <gi...@apache.org>.
rtpsw commented on PR #34311:
URL: https://github.com/apache/arrow/pull/34311#issuecomment-1460545751

   @westonpace, is this good to go?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] icexelloss commented on pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "icexelloss (via GitHub)" <gi...@apache.org>.
icexelloss commented on PR #34311:
URL: https://github.com/apache/arrow/pull/34311#issuecomment-1462272967

   Thanks @rtpsw I will take a look later Today.
   
   Can you gather all the follow up issues and put them as a list in the PR description and the origin GH issue as well?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] icexelloss commented on pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "icexelloss (via GitHub)" <gi...@apache.org>.
icexelloss commented on PR #34311:
URL: https://github.com/apache/arrow/pull/34311#issuecomment-1463923522

   I also edited the PR description to make the purpose of this PR a bit more clear.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] icexelloss commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "icexelloss (via GitHub)" <gi...@apache.org>.
icexelloss commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1132447798


##########
cpp/src/arrow/compute/row/grouper.cc:
##########
@@ -39,12 +43,336 @@
 namespace arrow {
 
 using internal::checked_cast;
+using internal::PrimitiveScalarBase;
 
 namespace compute {
 
 namespace {
 
-struct GrouperImpl : Grouper {
+constexpr uint32_t kNoGroupId = std::numeric_limits<uint32_t>::max();
+
+using group_id_t = std::remove_const<decltype(kNoGroupId)>::type;
+using GroupIdType = CTypeTraits<group_id_t>::ArrowType;
+auto g_group_id_type = std::make_shared<GroupIdType>();
+
+inline const uint8_t* GetValuesAsBytes(const ArraySpan& data, int64_t offset = 0) {
+  DCHECK_GT(data.type->byte_width(), 0);
+  int64_t absolute_byte_offset = (data.offset + offset) * data.type->byte_width();
+  return data.GetValues<uint8_t>(1, absolute_byte_offset);
+}
+
+template <typename Value>
+Status CheckForGetNextSegment(const std::vector<Value>& values, int64_t length,
+                              int64_t offset, const std::vector<TypeHolder>& key_types) {
+  if (offset < 0 || offset > length) {

Review Comment:
   Tracked in https://github.com/apache/arrow/issues/34529



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] icexelloss commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "icexelloss (via GitHub)" <gi...@apache.org>.
icexelloss commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1132440778


##########
cpp/src/arrow/compute/kernels/hash_aggregate_test.cc:
##########
@@ -1917,57 +2243,7 @@ TEST(GroupBy, MinMaxBinary) {
   }
 }
 
-TEST(GroupBy, MinMaxFixedSizeBinary) {

Review Comment:
   LGTM now



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rtpsw commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "rtpsw (via GitHub)" <gi...@apache.org>.
rtpsw commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1129049031


##########
cpp/src/arrow/compute/exec/aggregate_node.cc:
##########
@@ -283,28 +393,47 @@ class ScalarAggregateNode : public ExecNode, public TracedNode {
     DCHECK_EQ(input, inputs_[0]);
 
     auto thread_index = plan_->query_context()->GetThreadIndex();
-
-    ARROW_RETURN_NOT_OK(DoConsume(ExecSpan(batch), thread_index));
+    auto handler = [this, thread_index](const ExecBatch& full_batch,
+                                        const Segment& segment) {
+      // (1) The segment is starting of a new segment group and points to
+      // the beginning of the batch, then it means no data in the batch belongs
+      // to the current segment group. We can output and reset kernel states.
+      if (!segment.extends && segment.offset == 0) RETURN_NOT_OK(OutputResult(false));
+
+      // We add segment to the current segment group aggregation
+      // GH-34475: change to zero-copy slicing
+      auto exec_batch = full_batch.Slice(segment.offset, segment.length);

Review Comment:
   Looks like you're right. It's not documented on `ExecBatch::Slice` but the implementation uses `Array::Slice` and `ChunkedArray::Slice` which are documented as zero-copy.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] github-actions[bot] commented on pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "github-actions[bot] (via GitHub)" <gi...@apache.org>.
github-actions[bot] commented on PR #34311:
URL: https://github.com/apache/arrow/pull/34311#issuecomment-1462651920

   :warning: GitHub issue #32884 **has been automatically assigned in GitHub** to PR creator.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] icexelloss commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "icexelloss (via GitHub)" <gi...@apache.org>.
icexelloss commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1126947659


##########
cpp/src/arrow/compute/kernels/hash_aggregate_test.cc:
##########
@@ -174,81 +242,117 @@ Result<Datum> RunGroupBy(const BatchesWithSchema& input,
   ARROW_ASSIGN_OR_RAISE(std::vector<ExecBatch> output_batches,
                         start_and_collect.MoveResult());
 
-  ArrayVector out_arrays(aggregates.size() + key_names.size());
   const auto& output_schema = plan->nodes()[0]->output()->output_schema();
+  if (!segmented) {
+    return MakeGroupByOutput(output_batches, output_schema, aggregates.size(),
+                             key_names.size(), naive);
+  }
+
+  std::vector<ArrayVector> out_arrays(aggregates.size() + key_names.size() +
+                                      segment_key_names.size());
   for (size_t i = 0; i < out_arrays.size(); ++i) {
     std::vector<std::shared_ptr<Array>> arrays(output_batches.size());
     for (size_t j = 0; j < output_batches.size(); ++j) {
-      arrays[j] = output_batches[j].values[i].make_array();
+      auto& value = output_batches[j].values[i];
+      if (value.is_scalar()) {
+        ARROW_ASSIGN_OR_RAISE(
+            arrays[j], MakeArrayFromScalar(*value.scalar(), output_batches[j].length));
+      } else if (value.is_array()) {
+        arrays[j] = value.make_array();
+      } else {
+        return Status::Invalid("GroupByUsingExecPlan unsupported value kind ",
+                               ToString(value.kind()));
+      }
     }
     if (arrays.empty()) {
+      arrays.resize(1);
       ARROW_ASSIGN_OR_RAISE(
-          out_arrays[i],
-          MakeArrayOfNull(output_schema->field(static_cast<int>(i))->type(),
-                          /*length=*/0));
-    } else {
-      ARROW_ASSIGN_OR_RAISE(out_arrays[i], Concatenate(arrays));
+          arrays[0], MakeArrayOfNull(output_schema->field(static_cast<int>(i))->type(),
+                                     /*length=*/0));
     }
+    out_arrays[i] = {std::move(arrays)};
   }
 
-  // The exec plan may reorder the output rows.  The tests are all setup to expect ouptut
-  // in ascending order of keys.  So we need to sort the result by the key columns.  To do
-  // that we create a table using the key columns, calculate the sort indices from that
-  // table (sorting on all fields) and then use those indices to calculate our result.
-  std::vector<std::shared_ptr<Field>> key_fields;
-  std::vector<std::shared_ptr<Array>> key_columns;
-  std::vector<SortKey> sort_keys;
-  for (std::size_t i = 0; i < key_names.size(); i++) {
-    const std::shared_ptr<Array>& arr = out_arrays[i + aggregates.size()];
-    if (arr->type_id() == Type::DICTIONARY) {
-      // Can't sort dictionary columns so need to decode
-      auto dict_arr = checked_pointer_cast<DictionaryArray>(arr);
-      ARROW_ASSIGN_OR_RAISE(auto decoded_arr,
-                            Take(*dict_arr->dictionary(), *dict_arr->indices()));
-      key_columns.push_back(decoded_arr);
-      key_fields.push_back(
-          field("name_does_not_matter", dict_arr->dict_type()->value_type()));
-    } else {
-      key_columns.push_back(arr);
-      key_fields.push_back(field("name_does_not_matter", arr->type()));
+  if (segmented && segment_key_names.size() > 0) {
+    ArrayVector struct_arrays;
+    struct_arrays.reserve(output_batches.size());
+    for (size_t j = 0; j < output_batches.size(); ++j) {
+      ArrayVector struct_fields;
+      struct_fields.reserve(out_arrays.size());
+      for (auto out_array : out_arrays) {
+        struct_fields.push_back(out_array[j]);
+      }
+      ARROW_ASSIGN_OR_RAISE(auto struct_array,
+                            StructArray::Make(struct_fields, output_schema->fields()));
+      struct_arrays.push_back(struct_array);
     }
-    sort_keys.emplace_back(static_cast<int>(i));
+    return ChunkedArray::Make(struct_arrays);
+  } else {
+    ArrayVector struct_fields(out_arrays.size());
+    for (size_t i = 0; i < out_arrays.size(); ++i) {
+      ARROW_ASSIGN_OR_RAISE(struct_fields[i], Concatenate(out_arrays[i]));
+    }
+    return StructArray::Make(std::move(struct_fields), output_schema->fields());
   }
-  std::shared_ptr<Schema> key_schema = schema(std::move(key_fields));
-  std::shared_ptr<Table> key_table = Table::Make(std::move(key_schema), key_columns);
-  SortOptions sort_options(std::move(sort_keys));
-  ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Array> sort_indices,
-                        SortIndices(key_table, sort_options));
+}
 
-  ARROW_ASSIGN_OR_RAISE(
-      std::shared_ptr<Array> struct_arr,
-      StructArray::Make(std::move(out_arrays), output_schema->fields()));
+Result<Datum> RunGroupBy(const BatchesWithSchema& input,
+                         const std::vector<std::string>& key_names,
+                         const std::vector<std::string>& segment_key_names,
+                         const std::vector<Aggregate>& aggregates, bool use_threads,
+                         bool segmented = false, bool naive = false) {
+  if (segment_key_names.size() > 0) {
+    ARROW_ASSIGN_OR_RAISE(auto thread_pool, arrow::internal::ThreadPool::Make(1));
+    ExecContext seq_ctx(default_memory_pool(), thread_pool.get());
+    return RunGroupBy(input, key_names, segment_key_names, aggregates, &seq_ctx,
+                      use_threads, segmented, naive);
+  } else {
+    return RunGroupBy(input, key_names, segment_key_names, aggregates,
+                      threaded_exec_context(), use_threads, segmented, naive);
+  }
+}
 
-  return Take(struct_arr, sort_indices);
+Result<Datum> RunGroupBy(const BatchesWithSchema& input,
+                         const std::vector<std::string>& key_names,
+                         const std::vector<Aggregate>& aggregates, bool use_threads,
+                         bool segmented = false, bool naive = false) {
+  return RunGroupBy(input, key_names, {}, aggregates, use_threads, segmented);
 }
 
 /// Simpler overload where you can give the columns as datums
 Result<Datum> RunGroupBy(const std::vector<Datum>& arguments,
                          const std::vector<Datum>& keys,
-                         const std::vector<Aggregate>& aggregates,
-                         bool use_threads = false) {
+                         const std::vector<Datum>& segment_keys,
+                         const std::vector<Aggregate>& aggregates, bool use_threads,
+                         bool segmented = false, bool naive = false) {

Review Comment:
   > NaiveGroupBy is [a tester function](https://github.com/apache/arrow/blob/d5b3b4737838774db658d3c488fcd3e72bc13f7e/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc#L75-L136) that computes the expected aggregation result (in a naive/simple/non-optimized way).
   
   I am still quite confused - what is this testing with `naive=true` vs `naive=false`? 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] icexelloss commented on pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "icexelloss (via GitHub)" <gi...@apache.org>.
icexelloss commented on PR #34311:
URL: https://github.com/apache/arrow/pull/34311#issuecomment-1462677016

   > Linked in both PR and original GH issue to #34475 which has the list.
   
   Can you put the follow up GH issue link in the list?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] icexelloss commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "icexelloss (via GitHub)" <gi...@apache.org>.
icexelloss commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1129915631


##########
cpp/src/arrow/compute/row/grouper.cc:
##########
@@ -39,12 +43,330 @@
 namespace arrow {
 
 using internal::checked_cast;
+using internal::PrimitiveScalarBase;
 
 namespace compute {
 
 namespace {
 
-struct GrouperImpl : Grouper {
+constexpr uint32_t kNoGroupId = std::numeric_limits<uint32_t>::max();
+
+using group_id_t = std::remove_const<decltype(kNoGroupId)>::type;
+using GroupIdType = CTypeTraits<group_id_t>::ArrowType;
+auto g_group_id_type = std::make_shared<GroupIdType>();
+
+inline const uint8_t* GetValuesAsBytes(const ArraySpan& data, int64_t offset = 0) {
+  DCHECK_GT(data.type->byte_width(), 0);
+  int64_t absolute_byte_offset = (data.offset + offset) * data.type->byte_width();
+  return data.GetValues<uint8_t>(1, absolute_byte_offset);
+}
+
+template <typename Value>
+Status CheckForGetNextSegment(const std::vector<Value>& values, int64_t length,
+                              int64_t offset, const std::vector<TypeHolder>& key_types) {
+  if (offset < 0 || offset > length) {
+    return Status::Invalid("invalid grouping segmenter offset: ", offset);
+  }
+  if (values.size() != key_types.size()) {
+    return Status::Invalid("expected batch size ", key_types.size(), " but got ",
+                           values.size());
+  }
+  for (size_t i = 0; i < key_types.size(); i++) {
+    const auto& value = values[i];
+    const auto& key_type = key_types[i];
+    if (*value.type() != *key_type.type) {
+      return Status::Invalid("expected batch value ", i, " of type ", *key_type.type,
+                             " but got ", *value.type());
+    }
+  }
+  return Status::OK();
+}
+
+template <typename Batch>
+enable_if_t<std::is_same<Batch, ExecSpan>::value || std::is_same<Batch, ExecBatch>::value,
+            Status>
+CheckForGetNextSegment(const Batch& batch, int64_t offset,
+                       const std::vector<TypeHolder>& key_types) {
+  return CheckForGetNextSegment(batch.values, batch.length, offset, key_types);
+}
+
+struct BaseRowSegmenter : public RowSegmenter {
+  explicit BaseRowSegmenter(const std::vector<TypeHolder>& key_types)
+      : key_types_(key_types) {}
+
+  const std::vector<TypeHolder>& key_types() const override { return key_types_; }
+
+  std::vector<TypeHolder> key_types_;
+};
+
+Segment MakeSegment(int64_t batch_length, int64_t offset, int64_t length, bool extends) {
+  return Segment{offset, length, offset + length >= batch_length, extends};
+}
+
+int64_t GetMatchLength(const uint8_t* match_bytes, int64_t match_width,
+                       const uint8_t* array_bytes, int64_t offset, int64_t length) {
+  int64_t cursor, byte_cursor;
+  for (cursor = offset, byte_cursor = match_width * cursor; cursor < length;
+       cursor++, byte_cursor += match_width) {
+    if (memcmp(match_bytes, array_bytes + byte_cursor,
+               static_cast<size_t>(match_width)) != 0) {
+      break;
+    }
+  }
+  return std::min(cursor, length) - offset;
+}
+
+using ExtendFunc = std::function<bool(const void*)>;
+constexpr bool kDefaultExtends = true;
+constexpr bool kEmptyExtends = true;
+
+struct NoKeysSegmenter : public BaseRowSegmenter {
+  static std::unique_ptr<RowSegmenter> Make() {
+    return std::make_unique<NoKeysSegmenter>();
+  }
+
+  NoKeysSegmenter() : BaseRowSegmenter({}) {}
+
+  Status Reset() override { return Status::OK(); }
+
+  Result<Segment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {}));
+    return MakeSegment(batch.length, offset, batch.length - offset, kDefaultExtends);
+  }
+};
+
+struct SimpleKeySegmenter : public BaseRowSegmenter {
+  static Result<std::unique_ptr<RowSegmenter>> Make(TypeHolder key_type) {
+    return std::make_unique<SimpleKeySegmenter>(key_type);
+  }
+
+  explicit SimpleKeySegmenter(TypeHolder key_type)
+      : BaseRowSegmenter({key_type}), key_type_(key_types_[0]), save_key_data_() {}
+
+  Status CheckType(const DataType& type) {
+    if (!is_fixed_width(type)) {
+      return Status::Invalid("SimpleKeySegmenter does not support type ", type);
+    }
+    return Status::OK();
+  }
+
+  Status Reset() override {
+    save_key_data_.resize(0);
+    return Status::OK();
+  }
+
+  // Checks whether the given grouping data extends the current segment, i.e., is equal to
+  // previously seen grouping data, which is updated with each invocation.
+  bool Extend(const void* data) {
+    size_t byte_width = static_cast<size_t>(key_type_.type->byte_width());
+    bool extends = save_key_data_.size() != byte_width
+                       ? kDefaultExtends
+                       : 0 == memcmp(save_key_data_.data(), data, byte_width);
+    save_key_data_.resize(byte_width);
+    memcpy(save_key_data_.data(), data, byte_width);
+    return extends;
+  }
+
+  Result<Segment> GetNextSegment(const Scalar& scalar, int64_t offset, int64_t length) {
+    ARROW_RETURN_NOT_OK(CheckType(*scalar.type));
+    if (!scalar.is_valid) {
+      return Status::Invalid("segmenting an invalid scalar");
+    }
+    auto data = checked_cast<const PrimitiveScalarBase&>(scalar).data();
+    bool extends = length > 0 ? Extend(data) : kEmptyExtends;
+    return MakeSegment(length, offset, length, extends);
+  }
+
+  Result<Segment> GetNextSegment(const DataType& array_type, const uint8_t* array_bytes,
+                                 int64_t offset, int64_t length) {
+    RETURN_NOT_OK(CheckType(array_type));
+    int64_t byte_width = array_type.byte_width();
+    int64_t match_length = GetMatchLength(array_bytes + offset * byte_width, byte_width,
+                                          array_bytes, offset, length);
+    bool extends = length > 0 ? Extend(array_bytes + offset * byte_width) : kEmptyExtends;
+    return MakeSegment(length, offset, match_length, extends);
+  }
+
+  Result<Segment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {key_type_}));
+    if (offset == batch.length) {
+      return MakeSegment(batch.length, offset, 0, kEmptyExtends);
+    }
+    const auto& value = batch.values[0];
+    if (value.is_scalar()) {
+      return GetNextSegment(*value.scalar, offset, batch.length);
+    }
+    ARROW_DCHECK(value.is_array());
+    const auto& array = value.array;
+    if (array.GetNullCount() > 0) {
+      return Status::NotImplemented("segmenting a nullable array");
+    }
+    return GetNextSegment(*array.type, GetValuesAsBytes(array), offset, batch.length);
+  }
+
+ private:
+  TypeHolder key_type_;
+  std::vector<uint8_t> save_key_data_;
+};
+
+struct AnyKeysSegmenter : public BaseRowSegmenter {
+  static Result<std::unique_ptr<RowSegmenter>> Make(
+      const std::vector<TypeHolder>& key_types, ExecContext* ctx) {
+    ARROW_RETURN_NOT_OK(Grouper::Make(key_types, ctx));  // check types
+    return std::make_unique<AnyKeysSegmenter>(key_types, ctx);
+  }
+
+  AnyKeysSegmenter(const std::vector<TypeHolder>& key_types, ExecContext* ctx)
+      : BaseRowSegmenter(key_types),
+        ctx_(ctx),
+        grouper_(nullptr),
+        save_group_id_(kNoGroupId) {}
+
+  Status Reset() override {

Review Comment:
   I actually agree with Weston that smaller PR helps in this case. Reviewing a 1k+ line PR is quite painful and very easy to miss things. Let's try breaking up into smaller PR's next time.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] icexelloss commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "icexelloss (via GitHub)" <gi...@apache.org>.
icexelloss commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1123681376


##########
cpp/src/arrow/compute/exec/aggregate_node.cc:
##########
@@ -326,46 +446,86 @@ class ScalarAggregateNode : public ExecNode, public TracedNode {
   }
 
  private:
-  Status Finish() {
-    auto scope = TraceFinish();
+  Status ReconstructAggregates() {
+    const auto& input_schema = *inputs()[0]->output_schema();
+    auto exec_ctx = plan()->query_context()->exec_context();
+    for (size_t i = 0; i < kernels_.size(); ++i) {
+      std::vector<TypeHolder> in_types;
+      for (const auto& target : target_fieldsets_[i]) {
+        in_types.emplace_back(input_schema.field(target)->type().get());
+      }
+      states_[i].resize(plan()->query_context()->max_concurrency());
+      KernelContext kernel_ctx{exec_ctx};
+      RETURN_NOT_OK(Kernel::InitAll(
+          &kernel_ctx, KernelInitArgs{kernels_[i], in_types, aggs_[i].options.get()},
+          &states_[i]));
+    }
+    return Status::OK();
+  }
+
+  Status OutputResult(bool is_last = false, bool traced = false) {
+    if (is_last && !traced) {
+      auto scope = TraceFinish();
+      return OutputResult(is_last, /*traced=*/true);
+    }
+    GatedUniqueLock lock(gated_shared_mutex_);
     ExecBatch batch{{}, 1};
-    batch.values.resize(kernels_.size());
+    batch.values.resize(kernels_.size() + segment_field_ids_.size());
 
     for (size_t i = 0; i < kernels_.size(); ++i) {
       util::tracing::Span span;
       START_COMPUTE_SPAN(span, aggs_[i].function,
                          {{"function.name", aggs_[i].function},
                           {"function.options",
                            aggs_[i].options ? aggs_[i].options->ToString() : "<NULLPTR>"},
-                          {"function.kind", std::string(kind_name()) + "::Finalize"}});
+                          {"function.kind", std::string(kind_name()) + "::Output"}});
       KernelContext ctx{plan()->query_context()->exec_context()};
       ARROW_ASSIGN_OR_RAISE(auto merged, ScalarAggregateKernel::MergeAll(
                                              kernels_[i], &ctx, std::move(states_[i])));
       RETURN_NOT_OK(kernels_[i]->finalize(&ctx, &batch.values[i]));
     }
+    PlaceFields(batch, kernels_.size(), segmenter_values_);
 
-    return output_->InputReceived(this, std::move(batch));
+    ARROW_RETURN_NOT_OK(output_->InputReceived(this, std::move(batch)));
+    total_output_batches_++;

Review Comment:
   Can you document this somewhere? It wasn't obvious until I read this line.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rtpsw commented on pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "rtpsw (via GitHub)" <gi...@apache.org>.
rtpsw commented on PR #34311:
URL: https://github.com/apache/arrow/pull/34311#issuecomment-1458534944

   > @rtpsw Do you have tests for the RowSegmenter classes to validate offset, length, open, extend in various cases? If so can you point me to it?
   
   Yes, any test-case that invokes `TestSegments` checks segments for all these members, e.g., `TEST(RowSegmenter, Basics)`.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rtpsw commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "rtpsw (via GitHub)" <gi...@apache.org>.
rtpsw commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1128244532


##########
cpp/src/arrow/compute/kernels/hash_aggregate_test.cc:
##########
@@ -1917,57 +2243,7 @@ TEST(GroupBy, MinMaxBinary) {
   }
 }
 
-TEST(GroupBy, MinMaxFixedSizeBinary) {

Review Comment:
   Will do. As noted elsewhere, I'd like @westonpace to review this conflict resolution.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rtpsw commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "rtpsw (via GitHub)" <gi...@apache.org>.
rtpsw commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1127113767


##########
cpp/src/arrow/compute/row/grouper.cc:
##########
@@ -39,12 +43,330 @@
 namespace arrow {
 
 using internal::checked_cast;
+using internal::PrimitiveScalarBase;
 
 namespace compute {
 
 namespace {
 
-struct GrouperImpl : Grouper {
+constexpr uint32_t kNoGroupId = std::numeric_limits<uint32_t>::max();
+
+using group_id_t = std::remove_const<decltype(kNoGroupId)>::type;
+using GroupIdType = CTypeTraits<group_id_t>::ArrowType;
+auto g_group_id_type = std::make_shared<GroupIdType>();
+
+inline const uint8_t* GetValuesAsBytes(const ArraySpan& data, int64_t offset = 0) {
+  DCHECK_GT(data.type->byte_width(), 0);
+  int64_t absolute_byte_offset = (data.offset + offset) * data.type->byte_width();
+  return data.GetValues<uint8_t>(1, absolute_byte_offset);
+}
+
+template <typename Value>
+Status CheckForGetNextSegment(const std::vector<Value>& values, int64_t length,
+                              int64_t offset, const std::vector<TypeHolder>& key_types) {
+  if (offset < 0 || offset > length) {
+    return Status::Invalid("invalid grouping segmenter offset: ", offset);
+  }
+  if (values.size() != key_types.size()) {
+    return Status::Invalid("expected batch size ", key_types.size(), " but got ",
+                           values.size());
+  }
+  for (size_t i = 0; i < key_types.size(); i++) {
+    const auto& value = values[i];
+    const auto& key_type = key_types[i];
+    if (*value.type() != *key_type.type) {
+      return Status::Invalid("expected batch value ", i, " of type ", *key_type.type,
+                             " but got ", *value.type());
+    }
+  }
+  return Status::OK();
+}
+
+template <typename Batch>
+enable_if_t<std::is_same<Batch, ExecSpan>::value || std::is_same<Batch, ExecBatch>::value,
+            Status>
+CheckForGetNextSegment(const Batch& batch, int64_t offset,
+                       const std::vector<TypeHolder>& key_types) {
+  return CheckForGetNextSegment(batch.values, batch.length, offset, key_types);
+}
+
+struct BaseRowSegmenter : public RowSegmenter {
+  explicit BaseRowSegmenter(const std::vector<TypeHolder>& key_types)
+      : key_types_(key_types) {}
+
+  const std::vector<TypeHolder>& key_types() const override { return key_types_; }
+
+  std::vector<TypeHolder> key_types_;
+};
+
+Segment MakeSegment(int64_t batch_length, int64_t offset, int64_t length, bool extends) {
+  return Segment{offset, length, offset + length >= batch_length, extends};
+}
+
+int64_t GetMatchLength(const uint8_t* match_bytes, int64_t match_width,
+                       const uint8_t* array_bytes, int64_t offset, int64_t length) {
+  int64_t cursor, byte_cursor;
+  for (cursor = offset, byte_cursor = match_width * cursor; cursor < length;
+       cursor++, byte_cursor += match_width) {
+    if (memcmp(match_bytes, array_bytes + byte_cursor,
+               static_cast<size_t>(match_width)) != 0) {
+      break;
+    }
+  }
+  return std::min(cursor, length) - offset;
+}
+
+using ExtendFunc = std::function<bool(const void*)>;
+constexpr bool kDefaultExtends = true;
+constexpr bool kEmptyExtends = true;
+
+struct NoKeysSegmenter : public BaseRowSegmenter {
+  static std::unique_ptr<RowSegmenter> Make() {
+    return std::make_unique<NoKeysSegmenter>();
+  }
+
+  NoKeysSegmenter() : BaseRowSegmenter({}) {}
+
+  Status Reset() override { return Status::OK(); }
+
+  Result<Segment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {}));
+    return MakeSegment(batch.length, offset, batch.length - offset, kDefaultExtends);
+  }
+};
+
+struct SimpleKeySegmenter : public BaseRowSegmenter {
+  static Result<std::unique_ptr<RowSegmenter>> Make(TypeHolder key_type) {
+    return std::make_unique<SimpleKeySegmenter>(key_type);
+  }
+
+  explicit SimpleKeySegmenter(TypeHolder key_type)
+      : BaseRowSegmenter({key_type}), key_type_(key_types_[0]), save_key_data_() {}
+
+  Status CheckType(const DataType& type) {
+    if (!is_fixed_width(type)) {
+      return Status::Invalid("SimpleKeySegmenter does not support type ", type);
+    }
+    return Status::OK();
+  }
+
+  Status Reset() override {
+    save_key_data_.resize(0);
+    return Status::OK();
+  }
+
+  // Checks whether the given grouping data extends the current segment, i.e., is equal to
+  // previously seen grouping data, which is updated with each invocation.
+  bool Extend(const void* data) {
+    size_t byte_width = static_cast<size_t>(key_type_.type->byte_width());
+    bool extends = save_key_data_.size() != byte_width
+                       ? kDefaultExtends
+                       : 0 == memcmp(save_key_data_.data(), data, byte_width);
+    save_key_data_.resize(byte_width);
+    memcpy(save_key_data_.data(), data, byte_width);
+    return extends;
+  }
+
+  Result<Segment> GetNextSegment(const Scalar& scalar, int64_t offset, int64_t length) {
+    ARROW_RETURN_NOT_OK(CheckType(*scalar.type));
+    if (!scalar.is_valid) {
+      return Status::Invalid("segmenting an invalid scalar");
+    }
+    auto data = checked_cast<const PrimitiveScalarBase&>(scalar).data();
+    bool extends = length > 0 ? Extend(data) : kEmptyExtends;
+    return MakeSegment(length, offset, length, extends);
+  }
+
+  Result<Segment> GetNextSegment(const DataType& array_type, const uint8_t* array_bytes,
+                                 int64_t offset, int64_t length) {
+    RETURN_NOT_OK(CheckType(array_type));
+    int64_t byte_width = array_type.byte_width();
+    int64_t match_length = GetMatchLength(array_bytes + offset * byte_width, byte_width,
+                                          array_bytes, offset, length);
+    bool extends = length > 0 ? Extend(array_bytes + offset * byte_width) : kEmptyExtends;
+    return MakeSegment(length, offset, match_length, extends);
+  }
+
+  Result<Segment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {key_type_}));
+    if (offset == batch.length) {
+      return MakeSegment(batch.length, offset, 0, kEmptyExtends);
+    }
+    const auto& value = batch.values[0];
+    if (value.is_scalar()) {
+      return GetNextSegment(*value.scalar, offset, batch.length);
+    }
+    ARROW_DCHECK(value.is_array());
+    const auto& array = value.array;
+    if (array.GetNullCount() > 0) {
+      return Status::NotImplemented("segmenting a nullable array");
+    }
+    return GetNextSegment(*array.type, GetValuesAsBytes(array), offset, batch.length);
+  }
+
+ private:
+  TypeHolder key_type_;
+  std::vector<uint8_t> save_key_data_;

Review Comment:
   Will do.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rtpsw commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "rtpsw (via GitHub)" <gi...@apache.org>.
rtpsw commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1127111119


##########
cpp/src/arrow/compute/kernels/hash_aggregate_test.cc:
##########
@@ -2087,7 +2363,7 @@ TEST(GroupBy, AnyAndAll) {
     auto keep_nulls_min_count =
         std::make_shared<ScalarAggregateOptions>(/*skip_nulls=*/false, /*min_count=*/3);
     ASSERT_OK_AND_ASSIGN(Datum aggregated_and_grouped,
-                         RunGroupBy(
+                         AltGroupBy(

Review Comment:
   The [`AltGroupBy` function](https://github.com/apache/arrow/pull/34311/files#diff-ec1e8907dfc3e636abecd3307879f331421e4f28f75e504f147826cebf04d7e0R1116-R1123) chooses an alternative for the group-by invocation (via `GetParam()`). There are 2 alternatives: `RunGroupByImpl` and `RunSegmentedGroupByImpl`.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] icexelloss commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "icexelloss (via GitHub)" <gi...@apache.org>.
icexelloss commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1127159801


##########
cpp/src/arrow/compute/row/grouper.cc:
##########
@@ -39,12 +43,330 @@
 namespace arrow {
 
 using internal::checked_cast;
+using internal::PrimitiveScalarBase;
 
 namespace compute {
 
 namespace {
 
-struct GrouperImpl : Grouper {
+constexpr uint32_t kNoGroupId = std::numeric_limits<uint32_t>::max();
+
+using group_id_t = std::remove_const<decltype(kNoGroupId)>::type;
+using GroupIdType = CTypeTraits<group_id_t>::ArrowType;
+auto g_group_id_type = std::make_shared<GroupIdType>();
+
+inline const uint8_t* GetValuesAsBytes(const ArraySpan& data, int64_t offset = 0) {
+  DCHECK_GT(data.type->byte_width(), 0);
+  int64_t absolute_byte_offset = (data.offset + offset) * data.type->byte_width();
+  return data.GetValues<uint8_t>(1, absolute_byte_offset);
+}
+
+template <typename Value>
+Status CheckForGetNextSegment(const std::vector<Value>& values, int64_t length,
+                              int64_t offset, const std::vector<TypeHolder>& key_types) {
+  if (offset < 0 || offset > length) {
+    return Status::Invalid("invalid grouping segmenter offset: ", offset);
+  }
+  if (values.size() != key_types.size()) {
+    return Status::Invalid("expected batch size ", key_types.size(), " but got ",
+                           values.size());
+  }
+  for (size_t i = 0; i < key_types.size(); i++) {
+    const auto& value = values[i];
+    const auto& key_type = key_types[i];
+    if (*value.type() != *key_type.type) {
+      return Status::Invalid("expected batch value ", i, " of type ", *key_type.type,
+                             " but got ", *value.type());
+    }
+  }
+  return Status::OK();
+}
+
+template <typename Batch>
+enable_if_t<std::is_same<Batch, ExecSpan>::value || std::is_same<Batch, ExecBatch>::value,
+            Status>
+CheckForGetNextSegment(const Batch& batch, int64_t offset,
+                       const std::vector<TypeHolder>& key_types) {
+  return CheckForGetNextSegment(batch.values, batch.length, offset, key_types);
+}
+
+struct BaseRowSegmenter : public RowSegmenter {
+  explicit BaseRowSegmenter(const std::vector<TypeHolder>& key_types)
+      : key_types_(key_types) {}
+
+  const std::vector<TypeHolder>& key_types() const override { return key_types_; }
+
+  std::vector<TypeHolder> key_types_;
+};
+
+Segment MakeSegment(int64_t batch_length, int64_t offset, int64_t length, bool extends) {
+  return Segment{offset, length, offset + length >= batch_length, extends};
+}
+
+int64_t GetMatchLength(const uint8_t* match_bytes, int64_t match_width,
+                       const uint8_t* array_bytes, int64_t offset, int64_t length) {
+  int64_t cursor, byte_cursor;
+  for (cursor = offset, byte_cursor = match_width * cursor; cursor < length;
+       cursor++, byte_cursor += match_width) {
+    if (memcmp(match_bytes, array_bytes + byte_cursor,
+               static_cast<size_t>(match_width)) != 0) {
+      break;
+    }
+  }
+  return std::min(cursor, length) - offset;
+}
+
+using ExtendFunc = std::function<bool(const void*)>;
+constexpr bool kDefaultExtends = true;
+constexpr bool kEmptyExtends = true;
+
+struct NoKeysSegmenter : public BaseRowSegmenter {
+  static std::unique_ptr<RowSegmenter> Make() {
+    return std::make_unique<NoKeysSegmenter>();
+  }
+
+  NoKeysSegmenter() : BaseRowSegmenter({}) {}
+
+  Status Reset() override { return Status::OK(); }
+
+  Result<Segment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {}));
+    return MakeSegment(batch.length, offset, batch.length - offset, kDefaultExtends);
+  }
+};
+
+struct SimpleKeySegmenter : public BaseRowSegmenter {
+  static Result<std::unique_ptr<RowSegmenter>> Make(TypeHolder key_type) {
+    return std::make_unique<SimpleKeySegmenter>(key_type);
+  }
+
+  explicit SimpleKeySegmenter(TypeHolder key_type)
+      : BaseRowSegmenter({key_type}), key_type_(key_types_[0]), save_key_data_() {}
+
+  Status CheckType(const DataType& type) {
+    if (!is_fixed_width(type)) {
+      return Status::Invalid("SimpleKeySegmenter does not support type ", type);
+    }
+    return Status::OK();
+  }
+
+  Status Reset() override {
+    save_key_data_.resize(0);
+    return Status::OK();
+  }
+
+  // Checks whether the given grouping data extends the current segment, i.e., is equal to
+  // previously seen grouping data, which is updated with each invocation.
+  bool Extend(const void* data) {
+    size_t byte_width = static_cast<size_t>(key_type_.type->byte_width());
+    bool extends = save_key_data_.size() != byte_width
+                       ? kDefaultExtends
+                       : 0 == memcmp(save_key_data_.data(), data, byte_width);
+    save_key_data_.resize(byte_width);
+    memcpy(save_key_data_.data(), data, byte_width);
+    return extends;
+  }
+
+  Result<Segment> GetNextSegment(const Scalar& scalar, int64_t offset, int64_t length) {
+    ARROW_RETURN_NOT_OK(CheckType(*scalar.type));
+    if (!scalar.is_valid) {
+      return Status::Invalid("segmenting an invalid scalar");
+    }
+    auto data = checked_cast<const PrimitiveScalarBase&>(scalar).data();
+    bool extends = length > 0 ? Extend(data) : kEmptyExtends;
+    return MakeSegment(length, offset, length, extends);
+  }
+
+  Result<Segment> GetNextSegment(const DataType& array_type, const uint8_t* array_bytes,
+                                 int64_t offset, int64_t length) {
+    RETURN_NOT_OK(CheckType(array_type));
+    int64_t byte_width = array_type.byte_width();
+    int64_t match_length = GetMatchLength(array_bytes + offset * byte_width, byte_width,
+                                          array_bytes, offset, length);
+    bool extends = length > 0 ? Extend(array_bytes + offset * byte_width) : kEmptyExtends;
+    return MakeSegment(length, offset, match_length, extends);
+  }
+
+  Result<Segment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {key_type_}));
+    if (offset == batch.length) {
+      return MakeSegment(batch.length, offset, 0, kEmptyExtends);
+    }
+    const auto& value = batch.values[0];
+    if (value.is_scalar()) {
+      return GetNextSegment(*value.scalar, offset, batch.length);
+    }
+    ARROW_DCHECK(value.is_array());
+    const auto& array = value.array;
+    if (array.GetNullCount() > 0) {
+      return Status::NotImplemented("segmenting a nullable array");
+    }
+    return GetNextSegment(*array.type, GetValuesAsBytes(array), offset, batch.length);
+  }
+
+ private:
+  TypeHolder key_type_;
+  std::vector<uint8_t> save_key_data_;
+};
+
+struct AnyKeysSegmenter : public BaseRowSegmenter {
+  static Result<std::unique_ptr<RowSegmenter>> Make(
+      const std::vector<TypeHolder>& key_types, ExecContext* ctx) {
+    ARROW_RETURN_NOT_OK(Grouper::Make(key_types, ctx));  // check types
+    return std::make_unique<AnyKeysSegmenter>(key_types, ctx);
+  }
+
+  AnyKeysSegmenter(const std::vector<TypeHolder>& key_types, ExecContext* ctx)
+      : BaseRowSegmenter(key_types),
+        ctx_(ctx),
+        grouper_(nullptr),
+        save_group_id_(kNoGroupId) {}
+
+  Status Reset() override {

Review Comment:
   I won't hold it if you feel strongly. But when I often prefer to keep code changes to minimal (especially considering this is a already a 1000+ line change set) plus as a reviewer, the intent of the author of keeping unused method for future is unclear to me. So if you feel strongly about keeping those I'd suggesting explaining how u think this can be used in the future. (It's not clear to me when we would want to reset a segmenter)



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] icexelloss commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "icexelloss (via GitHub)" <gi...@apache.org>.
icexelloss commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1123590253


##########
cpp/src/arrow/compute/exec/aggregate_node.cc:
##########
@@ -169,35 +185,117 @@ void AggregatesToString(std::stringstream* ss, const Schema& input_schema,
   *ss << ']';
 }
 
+template <typename BatchHandler>
+Status HandleSegments(std::unique_ptr<GroupingSegmenter>& segmenter,
+                      const ExecBatch& batch, const std::vector<int>& ids,
+                      const BatchHandler& handle_batch) {
+  int64_t offset = 0;
+  ARROW_ASSIGN_OR_RAISE(auto segment_exec_batch, batch.SelectValues(ids));
+  ExecSpan segment_batch(segment_exec_batch);
+  while (true) {
+    ARROW_ASSIGN_OR_RAISE(auto segment, segmenter->GetNextSegment(segment_batch, offset));
+    if (segment.offset >= segment_batch.length) break;  // condition of no-next-segment
+    ARROW_RETURN_NOT_OK(handle_batch(batch, segment));
+    offset = segment.offset + segment.length;
+  }
+  return Status::OK();
+}
+
+Status GetScalarFields(std::vector<Datum>* values_ptr, const ExecBatch& input_batch,
+                       const std::vector<int>& field_ids) {
+  DCHECK_GT(input_batch.length, 0);
+  std::vector<Datum>& values = *values_ptr;
+  int64_t row = input_batch.length - 1;
+  values.clear();
+  values.resize(field_ids.size());
+  for (size_t i = 0; i < field_ids.size(); i++) {
+    const Datum& value = input_batch.values[field_ids[i]];
+    if (value.is_scalar()) {
+      values[i] = value;
+    } else if (value.is_array()) {
+      ARROW_ASSIGN_OR_RAISE(auto scalar, value.make_array()->GetScalar(row));
+      values[i] = scalar;
+    } else {
+      DCHECK(false);
+    }
+  }
+  return Status::OK();
+}
+
+void PlaceFields(ExecBatch& batch, size_t base, std::vector<Datum>& values) {
+  DCHECK_LE(base + values.size(), batch.values.size());
+  for (size_t i = 0; i < values.size(); i++) {
+    batch.values[base + i] = values[i];
+  }
+}
+
 class ScalarAggregateNode : public ExecNode, public TracedNode {
  public:
   ScalarAggregateNode(ExecPlan* plan, std::vector<ExecNode*> inputs,
                       std::shared_ptr<Schema> output_schema,
+                      std::unique_ptr<GroupingSegmenter> segmenter,
+                      std::vector<int> segment_field_ids,
                       std::vector<std::vector<int>> target_fieldsets,
                       std::vector<Aggregate> aggs,
                       std::vector<const ScalarAggregateKernel*> kernels,
                       std::vector<std::vector<std::unique_ptr<KernelState>>> states)
       : ExecNode(plan, std::move(inputs), {"target"},
                  /*output_schema=*/std::move(output_schema)),
         TracedNode(this),
+        segmenter_(std::move(segmenter)),
+        segment_field_ids_(std::move(segment_field_ids)),
         target_fieldsets_(std::move(target_fieldsets)),
         aggs_(std::move(aggs)),
         kernels_(std::move(kernels)),
-        states_(std::move(states)) {}
+        states_(std::move(states)) {
+    const auto& input_schema = *this->inputs()[0]->output_schema();
+    for (size_t i = 0; i < kernels_.size(); ++i) {
+      std::vector<TypeHolder> in_types;
+      for (const auto& target : target_fieldsets_[i]) {
+        in_types.emplace_back(input_schema.field(target)->type().get());
+      }
+      in_typesets_.push_back(std::move(in_types));
+    }
+  }
 
   static Result<ExecNode*> Make(ExecPlan* plan, std::vector<ExecNode*> inputs,
                                 const ExecNodeOptions& options) {
     RETURN_NOT_OK(ValidateExecNodeInputs(plan, inputs, 1, "ScalarAggregateNode"));
 
     const auto& aggregate_options = checked_cast<const AggregateNodeOptions&>(options);
     auto aggregates = aggregate_options.aggregates;
+    const auto& keys = aggregate_options.keys;
+    const auto& segment_keys = aggregate_options.segment_keys;
+
+    if (keys.size() > 0) {
+      return Status::Invalid("Scalar aggregation with some key");
+    }
+    if (plan->query_context()->exec_context()->executor()->GetCapacity() > 1 &&
+        segment_keys.size() > 0) {
+      return Status::NotImplemented("Segmented aggregation in a multi-threaded plan");
+    }
 
     const auto& input_schema = *inputs[0]->output_schema();
     auto exec_ctx = plan->query_context()->exec_context();
 
+    std::vector<int> segment_field_ids(segment_keys.size());
+    std::vector<TypeHolder> segment_key_types(segment_keys.size());
+    for (size_t i = 0; i < segment_keys.size(); i++) {
+      ARROW_ASSIGN_OR_RAISE(auto match, segment_keys[i].FindOne(input_schema));
+      if (match.indices().size() > 1) {
+        // ARROW-18369: Support nested references as segment ids
+        return Status::Invalid("Nested references cannot be used as segment ids");
+      }
+      segment_field_ids[i] = match[0];
+      segment_key_types[i] = input_schema.field(match[0])->type().get();
+    }
+
+    ARROW_ASSIGN_OR_RAISE(
+        auto segmenter, GroupingSegmenter::Make(std::move(segment_key_types), exec_ctx));
+
     std::vector<const ScalarAggregateKernel*> kernels(aggregates.size());
     std::vector<std::vector<std::unique_ptr<KernelState>>> states(kernels.size());
-    FieldVector fields(kernels.size());
+    FieldVector fields(kernels.size() + segment_keys.size());

Review Comment:
   Why do we need to do this here?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] icexelloss commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "icexelloss (via GitHub)" <gi...@apache.org>.
icexelloss commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1123599518


##########
cpp/src/arrow/compute/exec/aggregate_node.cc:
##########
@@ -169,35 +185,117 @@ void AggregatesToString(std::stringstream* ss, const Schema& input_schema,
   *ss << ']';
 }
 
+template <typename BatchHandler>
+Status HandleSegments(std::unique_ptr<GroupingSegmenter>& segmenter,
+                      const ExecBatch& batch, const std::vector<int>& ids,
+                      const BatchHandler& handle_batch) {
+  int64_t offset = 0;
+  ARROW_ASSIGN_OR_RAISE(auto segment_exec_batch, batch.SelectValues(ids));
+  ExecSpan segment_batch(segment_exec_batch);
+  while (true) {
+    ARROW_ASSIGN_OR_RAISE(auto segment, segmenter->GetNextSegment(segment_batch, offset));
+    if (segment.offset >= segment_batch.length) break;  // condition of no-next-segment
+    ARROW_RETURN_NOT_OK(handle_batch(batch, segment));
+    offset = segment.offset + segment.length;
+  }
+  return Status::OK();
+}
+
+Status GetScalarFields(std::vector<Datum>* values_ptr, const ExecBatch& input_batch,
+                       const std::vector<int>& field_ids) {
+  DCHECK_GT(input_batch.length, 0);
+  std::vector<Datum>& values = *values_ptr;
+  int64_t row = input_batch.length - 1;
+  values.clear();
+  values.resize(field_ids.size());
+  for (size_t i = 0; i < field_ids.size(); i++) {
+    const Datum& value = input_batch.values[field_ids[i]];
+    if (value.is_scalar()) {

Review Comment:
   When  can this be scalar and when can this be array?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rtpsw commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "rtpsw (via GitHub)" <gi...@apache.org>.
rtpsw commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1127066497


##########
cpp/src/arrow/compute/row/grouper.cc:
##########
@@ -39,12 +43,330 @@
 namespace arrow {
 
 using internal::checked_cast;
+using internal::PrimitiveScalarBase;
 
 namespace compute {
 
 namespace {
 
-struct GrouperImpl : Grouper {
+constexpr uint32_t kNoGroupId = std::numeric_limits<uint32_t>::max();
+
+using group_id_t = std::remove_const<decltype(kNoGroupId)>::type;
+using GroupIdType = CTypeTraits<group_id_t>::ArrowType;
+auto g_group_id_type = std::make_shared<GroupIdType>();
+
+inline const uint8_t* GetValuesAsBytes(const ArraySpan& data, int64_t offset = 0) {
+  DCHECK_GT(data.type->byte_width(), 0);
+  int64_t absolute_byte_offset = (data.offset + offset) * data.type->byte_width();
+  return data.GetValues<uint8_t>(1, absolute_byte_offset);
+}
+
+template <typename Value>
+Status CheckForGetNextSegment(const std::vector<Value>& values, int64_t length,
+                              int64_t offset, const std::vector<TypeHolder>& key_types) {
+  if (offset < 0 || offset > length) {
+    return Status::Invalid("invalid grouping segmenter offset: ", offset);
+  }
+  if (values.size() != key_types.size()) {
+    return Status::Invalid("expected batch size ", key_types.size(), " but got ",
+                           values.size());
+  }
+  for (size_t i = 0; i < key_types.size(); i++) {
+    const auto& value = values[i];
+    const auto& key_type = key_types[i];
+    if (*value.type() != *key_type.type) {
+      return Status::Invalid("expected batch value ", i, " of type ", *key_type.type,
+                             " but got ", *value.type());
+    }
+  }
+  return Status::OK();
+}
+
+template <typename Batch>
+enable_if_t<std::is_same<Batch, ExecSpan>::value || std::is_same<Batch, ExecBatch>::value,
+            Status>
+CheckForGetNextSegment(const Batch& batch, int64_t offset,
+                       const std::vector<TypeHolder>& key_types) {
+  return CheckForGetNextSegment(batch.values, batch.length, offset, key_types);
+}
+
+struct BaseRowSegmenter : public RowSegmenter {
+  explicit BaseRowSegmenter(const std::vector<TypeHolder>& key_types)
+      : key_types_(key_types) {}
+
+  const std::vector<TypeHolder>& key_types() const override { return key_types_; }
+
+  std::vector<TypeHolder> key_types_;
+};
+
+Segment MakeSegment(int64_t batch_length, int64_t offset, int64_t length, bool extends) {
+  return Segment{offset, length, offset + length >= batch_length, extends};
+}
+
+int64_t GetMatchLength(const uint8_t* match_bytes, int64_t match_width,
+                       const uint8_t* array_bytes, int64_t offset, int64_t length) {
+  int64_t cursor, byte_cursor;
+  for (cursor = offset, byte_cursor = match_width * cursor; cursor < length;
+       cursor++, byte_cursor += match_width) {
+    if (memcmp(match_bytes, array_bytes + byte_cursor,
+               static_cast<size_t>(match_width)) != 0) {
+      break;
+    }
+  }
+  return std::min(cursor, length) - offset;
+}
+
+using ExtendFunc = std::function<bool(const void*)>;
+constexpr bool kDefaultExtends = true;
+constexpr bool kEmptyExtends = true;
+
+struct NoKeysSegmenter : public BaseRowSegmenter {
+  static std::unique_ptr<RowSegmenter> Make() {
+    return std::make_unique<NoKeysSegmenter>();
+  }
+
+  NoKeysSegmenter() : BaseRowSegmenter({}) {}
+
+  Status Reset() override { return Status::OK(); }
+
+  Result<Segment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {}));
+    return MakeSegment(batch.length, offset, batch.length - offset, kDefaultExtends);
+  }
+};
+
+struct SimpleKeySegmenter : public BaseRowSegmenter {
+  static Result<std::unique_ptr<RowSegmenter>> Make(TypeHolder key_type) {
+    return std::make_unique<SimpleKeySegmenter>(key_type);
+  }
+
+  explicit SimpleKeySegmenter(TypeHolder key_type)
+      : BaseRowSegmenter({key_type}), key_type_(key_types_[0]), save_key_data_() {}
+
+  Status CheckType(const DataType& type) {
+    if (!is_fixed_width(type)) {
+      return Status::Invalid("SimpleKeySegmenter does not support type ", type);
+    }
+    return Status::OK();
+  }
+
+  Status Reset() override {
+    save_key_data_.resize(0);
+    return Status::OK();
+  }
+
+  // Checks whether the given grouping data extends the current segment, i.e., is equal to
+  // previously seen grouping data, which is updated with each invocation.
+  bool Extend(const void* data) {
+    size_t byte_width = static_cast<size_t>(key_type_.type->byte_width());
+    bool extends = save_key_data_.size() != byte_width
+                       ? kDefaultExtends
+                       : 0 == memcmp(save_key_data_.data(), data, byte_width);
+    save_key_data_.resize(byte_width);
+    memcpy(save_key_data_.data(), data, byte_width);
+    return extends;
+  }
+
+  Result<Segment> GetNextSegment(const Scalar& scalar, int64_t offset, int64_t length) {
+    ARROW_RETURN_NOT_OK(CheckType(*scalar.type));
+    if (!scalar.is_valid) {
+      return Status::Invalid("segmenting an invalid scalar");
+    }
+    auto data = checked_cast<const PrimitiveScalarBase&>(scalar).data();
+    bool extends = length > 0 ? Extend(data) : kEmptyExtends;
+    return MakeSegment(length, offset, length, extends);
+  }
+
+  Result<Segment> GetNextSegment(const DataType& array_type, const uint8_t* array_bytes,
+                                 int64_t offset, int64_t length) {
+    RETURN_NOT_OK(CheckType(array_type));
+    int64_t byte_width = array_type.byte_width();
+    int64_t match_length = GetMatchLength(array_bytes + offset * byte_width, byte_width,
+                                          array_bytes, offset, length);
+    bool extends = length > 0 ? Extend(array_bytes + offset * byte_width) : kEmptyExtends;
+    return MakeSegment(length, offset, match_length, extends);
+  }
+
+  Result<Segment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {key_type_}));
+    if (offset == batch.length) {
+      return MakeSegment(batch.length, offset, 0, kEmptyExtends);
+    }
+    const auto& value = batch.values[0];
+    if (value.is_scalar()) {
+      return GetNextSegment(*value.scalar, offset, batch.length);
+    }
+    ARROW_DCHECK(value.is_array());
+    const auto& array = value.array;
+    if (array.GetNullCount() > 0) {
+      return Status::NotImplemented("segmenting a nullable array");
+    }
+    return GetNextSegment(*array.type, GetValuesAsBytes(array), offset, batch.length);
+  }
+
+ private:
+  TypeHolder key_type_;
+  std::vector<uint8_t> save_key_data_;
+};
+
+struct AnyKeysSegmenter : public BaseRowSegmenter {
+  static Result<std::unique_ptr<RowSegmenter>> Make(
+      const std::vector<TypeHolder>& key_types, ExecContext* ctx) {
+    ARROW_RETURN_NOT_OK(Grouper::Make(key_types, ctx));  // check types
+    return std::make_unique<AnyKeysSegmenter>(key_types, ctx);
+  }
+
+  AnyKeysSegmenter(const std::vector<TypeHolder>& key_types, ExecContext* ctx)
+      : BaseRowSegmenter(key_types),
+        ctx_(ctx),
+        grouper_(nullptr),
+        save_group_id_(kNoGroupId) {}
+
+  Status Reset() override {

Review Comment:
   No, it is not used, and still it is useful to be able to reset a segmenter. It looks like we disagree on what justifies adding code. I feel your approach is to remove all code that is not currently used, considering a particular application in mind. My approach is different, accounting for library usage, and I'll opt to add such code when I find other justifications for doing so, like that it is simple/safe to implement or useful for future authoring, especially in the more generic parts of the code. I'd keep the code here.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] icexelloss commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "icexelloss (via GitHub)" <gi...@apache.org>.
icexelloss commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1128177714


##########
cpp/src/arrow/compute/kernels/hash_aggregate_test.cc:
##########
@@ -174,81 +255,117 @@ Result<Datum> RunGroupBy(const BatchesWithSchema& input,
   ARROW_ASSIGN_OR_RAISE(std::vector<ExecBatch> output_batches,
                         start_and_collect.MoveResult());
 
-  ArrayVector out_arrays(aggregates.size() + key_names.size());
   const auto& output_schema = plan->nodes()[0]->output()->output_schema();
+  if (!segmented) {
+    return MakeGroupByOutput(output_batches, output_schema, aggregates.size(),
+                             key_names.size(), naive);
+  }
+
+  std::vector<ArrayVector> out_arrays(aggregates.size() + key_names.size() +
+                                      segment_key_names.size());
   for (size_t i = 0; i < out_arrays.size(); ++i) {
     std::vector<std::shared_ptr<Array>> arrays(output_batches.size());
     for (size_t j = 0; j < output_batches.size(); ++j) {
-      arrays[j] = output_batches[j].values[i].make_array();
+      auto& value = output_batches[j].values[i];
+      if (value.is_scalar()) {
+        ARROW_ASSIGN_OR_RAISE(
+            arrays[j], MakeArrayFromScalar(*value.scalar(), output_batches[j].length));
+      } else if (value.is_array()) {
+        arrays[j] = value.make_array();
+      } else {
+        return Status::Invalid("GroupByUsingExecPlan unsupported value kind ",
+                               ToString(value.kind()));
+      }
     }
     if (arrays.empty()) {
+      arrays.resize(1);
       ARROW_ASSIGN_OR_RAISE(
-          out_arrays[i],
-          MakeArrayOfNull(output_schema->field(static_cast<int>(i))->type(),
-                          /*length=*/0));
-    } else {
-      ARROW_ASSIGN_OR_RAISE(out_arrays[i], Concatenate(arrays));
+          arrays[0], MakeArrayOfNull(output_schema->field(static_cast<int>(i))->type(),
+                                     /*length=*/0));
     }
+    out_arrays[i] = {std::move(arrays)};
   }
 
-  // The exec plan may reorder the output rows.  The tests are all setup to expect ouptut
-  // in ascending order of keys.  So we need to sort the result by the key columns.  To do
-  // that we create a table using the key columns, calculate the sort indices from that
-  // table (sorting on all fields) and then use those indices to calculate our result.
-  std::vector<std::shared_ptr<Field>> key_fields;
-  std::vector<std::shared_ptr<Array>> key_columns;
-  std::vector<SortKey> sort_keys;
-  for (std::size_t i = 0; i < key_names.size(); i++) {
-    const std::shared_ptr<Array>& arr = out_arrays[i + aggregates.size()];
-    if (arr->type_id() == Type::DICTIONARY) {
-      // Can't sort dictionary columns so need to decode
-      auto dict_arr = checked_pointer_cast<DictionaryArray>(arr);
-      ARROW_ASSIGN_OR_RAISE(auto decoded_arr,
-                            Take(*dict_arr->dictionary(), *dict_arr->indices()));
-      key_columns.push_back(decoded_arr);
-      key_fields.push_back(
-          field("name_does_not_matter", dict_arr->dict_type()->value_type()));
-    } else {
-      key_columns.push_back(arr);
-      key_fields.push_back(field("name_does_not_matter", arr->type()));
+  if (segmented && segment_key_names.size() > 0) {
+    ArrayVector struct_arrays;
+    struct_arrays.reserve(output_batches.size());
+    for (size_t j = 0; j < output_batches.size(); ++j) {
+      ArrayVector struct_fields;
+      struct_fields.reserve(out_arrays.size());
+      for (auto out_array : out_arrays) {
+        struct_fields.push_back(out_array[j]);
+      }
+      ARROW_ASSIGN_OR_RAISE(auto struct_array,
+                            StructArray::Make(struct_fields, output_schema->fields()));
+      struct_arrays.push_back(struct_array);
     }
-    sort_keys.emplace_back(static_cast<int>(i));
+    return ChunkedArray::Make(struct_arrays);

Review Comment:
   Nvm - not important



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rtpsw commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "rtpsw (via GitHub)" <gi...@apache.org>.
rtpsw commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1124234482


##########
cpp/src/arrow/compute/row/grouper.cc:
##########
@@ -39,12 +43,336 @@
 namespace arrow {
 
 using internal::checked_cast;
+using internal::PrimitiveScalarBase;
 
 namespace compute {
 
 namespace {
 
-struct GrouperImpl : Grouper {
+constexpr uint32_t kNoGroupId = std::numeric_limits<uint32_t>::max();
+
+using group_id_t = std::remove_const<decltype(kNoGroupId)>::type;
+using GroupIdType = CTypeTraits<group_id_t>::ArrowType;
+auto g_group_id_type = std::make_shared<GroupIdType>();
+
+inline const uint8_t* GetValuesAsBytes(const ArraySpan& data, int64_t offset = 0) {
+  DCHECK_GT(data.type->byte_width(), 0);
+  int64_t absolute_byte_offset = (data.offset + offset) * data.type->byte_width();
+  return data.GetValues<uint8_t>(1, absolute_byte_offset);
+}
+
+template <typename Value>
+Status CheckForGetNextSegment(const std::vector<Value>& values, int64_t length,
+                              int64_t offset, const std::vector<TypeHolder>& key_types) {
+  if (offset < 0 || offset > length) {
+    return Status::Invalid("invalid grouping segmenter offset: ", offset);
+  }
+  if (values.size() != key_types.size()) {
+    return Status::Invalid("expected batch size ", key_types.size(), " but got ",
+                           values.size());
+  }
+  for (size_t i = 0; i < key_types.size(); i++) {
+    const auto& value = values[i];
+    const auto& key_type = key_types[i];
+    if (*value.type() != *key_type.type) {
+      return Status::Invalid("expected batch value ", i, " of type ", *key_type.type,
+                             " but got ", *value.type());
+    }
+  }
+  return Status::OK();
+}
+
+template <typename Batch>
+enable_if_t<std::is_same<Batch, ExecSpan>::value || std::is_same<Batch, ExecBatch>::value,
+            Status>
+CheckForGetNextSegment(const Batch& batch, int64_t offset,
+                       const std::vector<TypeHolder>& key_types) {
+  return CheckForGetNextSegment(batch.values, batch.length, offset, key_types);
+}
+
+struct BaseGroupingSegmenter : public GroupingSegmenter {
+  explicit BaseGroupingSegmenter(const std::vector<TypeHolder>& key_types)
+      : key_types_(key_types) {}
+
+  const std::vector<TypeHolder>& key_types() const override { return key_types_; }
+
+  std::vector<TypeHolder> key_types_;
+};
+
+GroupingSegment MakeSegment(int64_t batch_length, int64_t offset, int64_t length,
+                            bool extends) {
+  return GroupingSegment{offset, length, offset + length >= batch_length, extends};
+}
+
+int64_t GetMatchLength(const uint8_t* match_bytes, int64_t match_width,
+                       const uint8_t* array_bytes, int64_t offset, int64_t length) {
+  int64_t cursor, byte_cursor;
+  for (cursor = offset, byte_cursor = match_width * cursor; cursor < length;
+       cursor++, byte_cursor += match_width) {
+    if (memcmp(match_bytes, array_bytes + byte_cursor,
+               static_cast<size_t>(match_width)) != 0) {
+      break;
+    }
+  }
+  return std::min(cursor, length - offset);
+}
+
+using ExtendFunc = std::function<bool(const void*)>;
+constexpr bool kDefaultExtends = true;
+constexpr bool kEmptyExtends = true;
+
+struct NoKeysGroupingSegmenter : public BaseGroupingSegmenter {
+  static std::unique_ptr<GroupingSegmenter> Make() {
+    return std::make_unique<NoKeysGroupingSegmenter>();
+  }
+
+  NoKeysGroupingSegmenter() : BaseGroupingSegmenter({}) {}
+
+  Status Reset() override { return Status::OK(); }
+
+  Result<GroupingSegment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {}));
+    return MakeSegment(batch.length, offset, batch.length - offset, kDefaultExtends);
+  }
+};
+
+struct SimpleKeyGroupingSegmenter : public BaseGroupingSegmenter {
+  static Result<std::unique_ptr<GroupingSegmenter>> Make(TypeHolder key_type) {
+    return std::make_unique<SimpleKeyGroupingSegmenter>(key_type);
+  }
+
+  explicit SimpleKeyGroupingSegmenter(TypeHolder key_type)
+      : BaseGroupingSegmenter({key_type}), key_type_(key_types_[0]), save_key_data_() {}
+
+  Status CheckType(const DataType& type) {
+    if (!is_fixed_width(type)) {
+      return Status::Invalid("SimpleKeyGroupingSegmenter does not support type ", type);
+    }
+    return Status::OK();
+  }
+
+  Status Reset() override {
+    save_key_data_.resize(0);
+    return Status::OK();
+  }
+
+  // Checks whether the given grouping data extends the current segment, i.e., is equal to
+  // previously seen grouping data, which is updated with each invocation.
+  bool Extend(const void* data) {
+    size_t byte_width = static_cast<size_t>(key_type_.type->byte_width());
+    bool extends = save_key_data_.size() != byte_width
+                       ? kDefaultExtends
+                       : 0 == memcmp(save_key_data_.data(), data, byte_width);
+    save_key_data_.resize(byte_width);
+    memcpy(save_key_data_.data(), data, byte_width);
+    return extends;
+  }
+
+  Result<GroupingSegment> GetNextSegment(const Scalar& scalar, int64_t offset,
+                                         int64_t length) {
+    ARROW_RETURN_NOT_OK(CheckType(*scalar.type));
+    if (!scalar.is_valid) {
+      return Status::Invalid("segmenting an invalid scalar");
+    }
+    auto data = checked_cast<const PrimitiveScalarBase&>(scalar).data();
+    bool extends = length > 0 ? Extend(data) : kEmptyExtends;
+    return MakeSegment(length, offset, length, extends);
+  }
+
+  Result<GroupingSegment> GetNextSegment(const DataType& array_type,
+                                         const uint8_t* array_bytes, int64_t offset,
+                                         int64_t length) {
+    RETURN_NOT_OK(CheckType(array_type));
+    int64_t byte_width = array_type.byte_width();
+    int64_t match_length = GetMatchLength(array_bytes + offset * byte_width, byte_width,
+                                          array_bytes, offset, length);
+    bool extends = length > 0 ? Extend(array_bytes + offset * byte_width) : kEmptyExtends;
+    return MakeSegment(length, offset, match_length, extends);
+  }
+
+  Result<GroupingSegment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {key_type_}));
+    if (offset == batch.length) {
+      return MakeSegment(batch.length, offset, 0, kEmptyExtends);
+    }
+    const auto& value = batch.values[0];
+    if (value.is_scalar()) {
+      return GetNextSegment(*value.scalar, offset, batch.length);
+    }
+    ARROW_DCHECK(value.is_array());
+    const auto& array = value.array;
+    if (array.GetNullCount() > 0) {
+      return Status::NotImplemented("segmenting a nullable array");
+    }
+    return GetNextSegment(*array.type, GetValuesAsBytes(array), offset, batch.length);
+  }
+
+ private:
+  TypeHolder key_type_;
+  std::vector<uint8_t> save_key_data_;
+};
+
+struct AnyKeysGroupingSegmenter : public BaseGroupingSegmenter {
+  static Result<std::unique_ptr<GroupingSegmenter>> Make(
+      const std::vector<TypeHolder>& key_types, ExecContext* ctx) {
+    ARROW_RETURN_NOT_OK(Grouper::Make(key_types, ctx));  // check types
+    return std::make_unique<AnyKeysGroupingSegmenter>(key_types, ctx);
+  }
+
+  AnyKeysGroupingSegmenter(const std::vector<TypeHolder>& key_types, ExecContext* ctx)
+      : BaseGroupingSegmenter(key_types),
+        ctx_(ctx),
+        grouper_(nullptr),
+        save_group_id_(kNoGroupId) {}
+
+  Status Reset() override {
+    grouper_ = nullptr;
+    save_group_id_ = kNoGroupId;
+    return Status::OK();
+  }
+
+  bool Extend(const void* data) {
+    auto group_id = *static_cast<const group_id_t*>(data);
+    bool extends =
+        save_group_id_ == kNoGroupId ? kDefaultExtends : save_group_id_ == group_id;
+    save_group_id_ = group_id;
+    return extends;
+  }
+
+  // Runs the grouper on a single row.  This is used to determine the group id of the
+  // first row of a new segment to see if it extends the previous segment.
+  template <typename Batch>
+  Result<group_id_t> MapGroupIdAt(const Batch& batch, int64_t offset) {
+    if (!grouper_) return kNoGroupId;
+    ARROW_ASSIGN_OR_RAISE(auto datum, grouper_->Consume(batch, offset,
+                                                        /*consume_length=*/1));
+    if (!datum.is_array()) {
+      return Status::Invalid("accessing unsupported datum kind ", datum.kind());
+    }
+    const std::shared_ptr<ArrayData>& data = datum.array();
+    ARROW_DCHECK(data->GetNullCount() == 0);
+    DCHECK_EQ(data->type->id(), GroupIdType::type_id);
+    DCHECK_EQ(1, data->length);
+    const group_id_t* values = data->GetValues<group_id_t>(1);
+    return values[0];
+  }
+
+  Result<GroupingSegment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, key_types_));
+    if (offset == batch.length) {
+      return MakeSegment(batch.length, offset, 0, kEmptyExtends);
+    }
+    // ARROW-18311: make Grouper support Reset()
+    // so it can be cached instead of recreated below
+    //
+    // the group id must be computed prior to resetting the grouper, since it is compared
+    // to save_group_id_, and after resetting the grouper produces incomparable group ids
+    ARROW_ASSIGN_OR_RAISE(auto group_id, MapGroupIdAt(batch, offset));
+    ExtendFunc bound_extend = [this, group_id](const void* data) {
+      bool extends = Extend(&group_id);
+      save_group_id_ = *static_cast<const group_id_t*>(data);
+      return extends;
+    };
+    ARROW_ASSIGN_OR_RAISE(grouper_, Grouper::Make(key_types_, ctx_));  // TODO: reset it
+    ARROW_ASSIGN_OR_RAISE(auto datum, grouper_->Consume(batch, offset));
+    if (datum.is_array()) {
+      const std::shared_ptr<ArrayData>& data = datum.array();
+      ARROW_DCHECK(data->GetNullCount() == 0);
+      DCHECK_EQ(data->type->id(), GroupIdType::type_id);
+      const group_id_t* values = data->GetValues<group_id_t>(1);
+      int64_t cursor;
+      for (cursor = 1; cursor < data->length; cursor++) {
+        if (values[0] != values[cursor]) break;
+      }
+      int64_t length = std::min(cursor, batch.length - offset);
+      bool extends = length > 0 ? bound_extend(values) : kEmptyExtends;
+      return MakeSegment(batch.length, offset, length, extends);
+    } else {
+      return Status::Invalid("segmenting unsupported datum kind ", datum.kind());
+    }
+  }
+
+ private:
+  ExecContext* const ctx_;
+  std::unique_ptr<Grouper> grouper_;
+  group_id_t save_group_id_;
+};
+
+Status CheckForConsume(int64_t batch_length, int64_t& consume_offset,
+                       int64_t* consume_length) {
+  if (consume_offset < 0) {
+    return Status::Invalid("invalid grouper consume offset: ", consume_offset);
+  }
+  if (*consume_length < 0) {
+    *consume_length = batch_length - consume_offset;

Review Comment:
   The API has `int64_t* consume_length` - it is an in/out parameter. However, this modification is internal and doesn't affect anything outside this class. It just caps the consume-length to the maximum remaining in the batch.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] icexelloss commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "icexelloss (via GitHub)" <gi...@apache.org>.
icexelloss commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1127037212


##########
cpp/src/arrow/compute/row/grouper.cc:
##########
@@ -39,12 +43,330 @@
 namespace arrow {
 
 using internal::checked_cast;
+using internal::PrimitiveScalarBase;
 
 namespace compute {
 
 namespace {
 
-struct GrouperImpl : Grouper {
+constexpr uint32_t kNoGroupId = std::numeric_limits<uint32_t>::max();
+
+using group_id_t = std::remove_const<decltype(kNoGroupId)>::type;
+using GroupIdType = CTypeTraits<group_id_t>::ArrowType;
+auto g_group_id_type = std::make_shared<GroupIdType>();
+
+inline const uint8_t* GetValuesAsBytes(const ArraySpan& data, int64_t offset = 0) {
+  DCHECK_GT(data.type->byte_width(), 0);
+  int64_t absolute_byte_offset = (data.offset + offset) * data.type->byte_width();
+  return data.GetValues<uint8_t>(1, absolute_byte_offset);
+}
+
+template <typename Value>
+Status CheckForGetNextSegment(const std::vector<Value>& values, int64_t length,
+                              int64_t offset, const std::vector<TypeHolder>& key_types) {
+  if (offset < 0 || offset > length) {
+    return Status::Invalid("invalid grouping segmenter offset: ", offset);
+  }
+  if (values.size() != key_types.size()) {
+    return Status::Invalid("expected batch size ", key_types.size(), " but got ",
+                           values.size());
+  }
+  for (size_t i = 0; i < key_types.size(); i++) {
+    const auto& value = values[i];
+    const auto& key_type = key_types[i];
+    if (*value.type() != *key_type.type) {
+      return Status::Invalid("expected batch value ", i, " of type ", *key_type.type,
+                             " but got ", *value.type());
+    }
+  }
+  return Status::OK();
+}
+
+template <typename Batch>
+enable_if_t<std::is_same<Batch, ExecSpan>::value || std::is_same<Batch, ExecBatch>::value,
+            Status>
+CheckForGetNextSegment(const Batch& batch, int64_t offset,
+                       const std::vector<TypeHolder>& key_types) {
+  return CheckForGetNextSegment(batch.values, batch.length, offset, key_types);
+}
+
+struct BaseRowSegmenter : public RowSegmenter {
+  explicit BaseRowSegmenter(const std::vector<TypeHolder>& key_types)
+      : key_types_(key_types) {}
+
+  const std::vector<TypeHolder>& key_types() const override { return key_types_; }
+
+  std::vector<TypeHolder> key_types_;
+};
+
+Segment MakeSegment(int64_t batch_length, int64_t offset, int64_t length, bool extends) {
+  return Segment{offset, length, offset + length >= batch_length, extends};
+}
+
+int64_t GetMatchLength(const uint8_t* match_bytes, int64_t match_width,
+                       const uint8_t* array_bytes, int64_t offset, int64_t length) {
+  int64_t cursor, byte_cursor;
+  for (cursor = offset, byte_cursor = match_width * cursor; cursor < length;
+       cursor++, byte_cursor += match_width) {
+    if (memcmp(match_bytes, array_bytes + byte_cursor,
+               static_cast<size_t>(match_width)) != 0) {
+      break;
+    }
+  }
+  return std::min(cursor, length) - offset;
+}
+
+using ExtendFunc = std::function<bool(const void*)>;
+constexpr bool kDefaultExtends = true;
+constexpr bool kEmptyExtends = true;
+
+struct NoKeysSegmenter : public BaseRowSegmenter {
+  static std::unique_ptr<RowSegmenter> Make() {
+    return std::make_unique<NoKeysSegmenter>();
+  }
+
+  NoKeysSegmenter() : BaseRowSegmenter({}) {}
+
+  Status Reset() override { return Status::OK(); }
+
+  Result<Segment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {}));
+    return MakeSegment(batch.length, offset, batch.length - offset, kDefaultExtends);
+  }
+};
+
+struct SimpleKeySegmenter : public BaseRowSegmenter {
+  static Result<std::unique_ptr<RowSegmenter>> Make(TypeHolder key_type) {
+    return std::make_unique<SimpleKeySegmenter>(key_type);
+  }
+
+  explicit SimpleKeySegmenter(TypeHolder key_type)
+      : BaseRowSegmenter({key_type}), key_type_(key_types_[0]), save_key_data_() {}
+
+  Status CheckType(const DataType& type) {
+    if (!is_fixed_width(type)) {
+      return Status::Invalid("SimpleKeySegmenter does not support type ", type);
+    }
+    return Status::OK();
+  }
+
+  Status Reset() override {
+    save_key_data_.resize(0);
+    return Status::OK();
+  }
+
+  // Checks whether the given grouping data extends the current segment, i.e., is equal to
+  // previously seen grouping data, which is updated with each invocation.
+  bool Extend(const void* data) {
+    size_t byte_width = static_cast<size_t>(key_type_.type->byte_width());
+    bool extends = save_key_data_.size() != byte_width
+                       ? kDefaultExtends
+                       : 0 == memcmp(save_key_data_.data(), data, byte_width);
+    save_key_data_.resize(byte_width);
+    memcpy(save_key_data_.data(), data, byte_width);
+    return extends;
+  }
+
+  Result<Segment> GetNextSegment(const Scalar& scalar, int64_t offset, int64_t length) {
+    ARROW_RETURN_NOT_OK(CheckType(*scalar.type));
+    if (!scalar.is_valid) {
+      return Status::Invalid("segmenting an invalid scalar");
+    }
+    auto data = checked_cast<const PrimitiveScalarBase&>(scalar).data();
+    bool extends = length > 0 ? Extend(data) : kEmptyExtends;
+    return MakeSegment(length, offset, length, extends);
+  }
+
+  Result<Segment> GetNextSegment(const DataType& array_type, const uint8_t* array_bytes,
+                                 int64_t offset, int64_t length) {
+    RETURN_NOT_OK(CheckType(array_type));
+    int64_t byte_width = array_type.byte_width();
+    int64_t match_length = GetMatchLength(array_bytes + offset * byte_width, byte_width,
+                                          array_bytes, offset, length);
+    bool extends = length > 0 ? Extend(array_bytes + offset * byte_width) : kEmptyExtends;
+    return MakeSegment(length, offset, match_length, extends);
+  }
+
+  Result<Segment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {key_type_}));
+    if (offset == batch.length) {
+      return MakeSegment(batch.length, offset, 0, kEmptyExtends);
+    }
+    const auto& value = batch.values[0];
+    if (value.is_scalar()) {
+      return GetNextSegment(*value.scalar, offset, batch.length);
+    }
+    ARROW_DCHECK(value.is_array());
+    const auto& array = value.array;
+    if (array.GetNullCount() > 0) {
+      return Status::NotImplemented("segmenting a nullable array");
+    }
+    return GetNextSegment(*array.type, GetValuesAsBytes(array), offset, batch.length);
+  }
+
+ private:
+  TypeHolder key_type_;
+  std::vector<uint8_t> save_key_data_;

Review Comment:
   Nvm - this is just a byte array



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rtpsw commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "rtpsw (via GitHub)" <gi...@apache.org>.
rtpsw commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1127461157


##########
cpp/src/arrow/compute/row/grouper.cc:
##########
@@ -39,12 +43,330 @@
 namespace arrow {
 
 using internal::checked_cast;
+using internal::PrimitiveScalarBase;
 
 namespace compute {
 
 namespace {
 
-struct GrouperImpl : Grouper {
+constexpr uint32_t kNoGroupId = std::numeric_limits<uint32_t>::max();
+
+using group_id_t = std::remove_const<decltype(kNoGroupId)>::type;
+using GroupIdType = CTypeTraits<group_id_t>::ArrowType;
+auto g_group_id_type = std::make_shared<GroupIdType>();
+
+inline const uint8_t* GetValuesAsBytes(const ArraySpan& data, int64_t offset = 0) {
+  DCHECK_GT(data.type->byte_width(), 0);
+  int64_t absolute_byte_offset = (data.offset + offset) * data.type->byte_width();
+  return data.GetValues<uint8_t>(1, absolute_byte_offset);
+}
+
+template <typename Value>
+Status CheckForGetNextSegment(const std::vector<Value>& values, int64_t length,
+                              int64_t offset, const std::vector<TypeHolder>& key_types) {
+  if (offset < 0 || offset > length) {
+    return Status::Invalid("invalid grouping segmenter offset: ", offset);
+  }
+  if (values.size() != key_types.size()) {
+    return Status::Invalid("expected batch size ", key_types.size(), " but got ",
+                           values.size());
+  }
+  for (size_t i = 0; i < key_types.size(); i++) {
+    const auto& value = values[i];
+    const auto& key_type = key_types[i];
+    if (*value.type() != *key_type.type) {
+      return Status::Invalid("expected batch value ", i, " of type ", *key_type.type,
+                             " but got ", *value.type());
+    }
+  }
+  return Status::OK();
+}
+
+template <typename Batch>
+enable_if_t<std::is_same<Batch, ExecSpan>::value || std::is_same<Batch, ExecBatch>::value,
+            Status>
+CheckForGetNextSegment(const Batch& batch, int64_t offset,
+                       const std::vector<TypeHolder>& key_types) {
+  return CheckForGetNextSegment(batch.values, batch.length, offset, key_types);
+}
+
+struct BaseRowSegmenter : public RowSegmenter {
+  explicit BaseRowSegmenter(const std::vector<TypeHolder>& key_types)
+      : key_types_(key_types) {}
+
+  const std::vector<TypeHolder>& key_types() const override { return key_types_; }
+
+  std::vector<TypeHolder> key_types_;
+};
+
+Segment MakeSegment(int64_t batch_length, int64_t offset, int64_t length, bool extends) {
+  return Segment{offset, length, offset + length >= batch_length, extends};
+}
+
+int64_t GetMatchLength(const uint8_t* match_bytes, int64_t match_width,
+                       const uint8_t* array_bytes, int64_t offset, int64_t length) {
+  int64_t cursor, byte_cursor;
+  for (cursor = offset, byte_cursor = match_width * cursor; cursor < length;
+       cursor++, byte_cursor += match_width) {
+    if (memcmp(match_bytes, array_bytes + byte_cursor,
+               static_cast<size_t>(match_width)) != 0) {
+      break;
+    }
+  }
+  return std::min(cursor, length) - offset;
+}
+
+using ExtendFunc = std::function<bool(const void*)>;
+constexpr bool kDefaultExtends = true;
+constexpr bool kEmptyExtends = true;
+
+struct NoKeysSegmenter : public BaseRowSegmenter {
+  static std::unique_ptr<RowSegmenter> Make() {
+    return std::make_unique<NoKeysSegmenter>();
+  }
+
+  NoKeysSegmenter() : BaseRowSegmenter({}) {}
+
+  Status Reset() override { return Status::OK(); }
+
+  Result<Segment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {}));
+    return MakeSegment(batch.length, offset, batch.length - offset, kDefaultExtends);
+  }
+};
+
+struct SimpleKeySegmenter : public BaseRowSegmenter {
+  static Result<std::unique_ptr<RowSegmenter>> Make(TypeHolder key_type) {
+    return std::make_unique<SimpleKeySegmenter>(key_type);
+  }
+
+  explicit SimpleKeySegmenter(TypeHolder key_type)
+      : BaseRowSegmenter({key_type}), key_type_(key_types_[0]), save_key_data_() {}
+
+  Status CheckType(const DataType& type) {
+    if (!is_fixed_width(type)) {
+      return Status::Invalid("SimpleKeySegmenter does not support type ", type);
+    }
+    return Status::OK();
+  }
+
+  Status Reset() override {
+    save_key_data_.resize(0);
+    return Status::OK();
+  }
+
+  // Checks whether the given grouping data extends the current segment, i.e., is equal to
+  // previously seen grouping data, which is updated with each invocation.
+  bool Extend(const void* data) {
+    size_t byte_width = static_cast<size_t>(key_type_.type->byte_width());
+    bool extends = save_key_data_.size() != byte_width
+                       ? kDefaultExtends
+                       : 0 == memcmp(save_key_data_.data(), data, byte_width);
+    save_key_data_.resize(byte_width);
+    memcpy(save_key_data_.data(), data, byte_width);
+    return extends;
+  }
+
+  Result<Segment> GetNextSegment(const Scalar& scalar, int64_t offset, int64_t length) {
+    ARROW_RETURN_NOT_OK(CheckType(*scalar.type));
+    if (!scalar.is_valid) {
+      return Status::Invalid("segmenting an invalid scalar");
+    }
+    auto data = checked_cast<const PrimitiveScalarBase&>(scalar).data();
+    bool extends = length > 0 ? Extend(data) : kEmptyExtends;
+    return MakeSegment(length, offset, length, extends);
+  }
+
+  Result<Segment> GetNextSegment(const DataType& array_type, const uint8_t* array_bytes,
+                                 int64_t offset, int64_t length) {
+    RETURN_NOT_OK(CheckType(array_type));
+    int64_t byte_width = array_type.byte_width();
+    int64_t match_length = GetMatchLength(array_bytes + offset * byte_width, byte_width,
+                                          array_bytes, offset, length);
+    bool extends = length > 0 ? Extend(array_bytes + offset * byte_width) : kEmptyExtends;
+    return MakeSegment(length, offset, match_length, extends);
+  }
+
+  Result<Segment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {key_type_}));
+    if (offset == batch.length) {
+      return MakeSegment(batch.length, offset, 0, kEmptyExtends);
+    }
+    const auto& value = batch.values[0];
+    if (value.is_scalar()) {
+      return GetNextSegment(*value.scalar, offset, batch.length);
+    }
+    ARROW_DCHECK(value.is_array());
+    const auto& array = value.array;
+    if (array.GetNullCount() > 0) {
+      return Status::NotImplemented("segmenting a nullable array");
+    }
+    return GetNextSegment(*array.type, GetValuesAsBytes(array), offset, batch.length);
+  }
+
+ private:
+  TypeHolder key_type_;
+  std::vector<uint8_t> save_key_data_;
+};
+
+struct AnyKeysSegmenter : public BaseRowSegmenter {
+  static Result<std::unique_ptr<RowSegmenter>> Make(
+      const std::vector<TypeHolder>& key_types, ExecContext* ctx) {
+    ARROW_RETURN_NOT_OK(Grouper::Make(key_types, ctx));  // check types
+    return std::make_unique<AnyKeysSegmenter>(key_types, ctx);
+  }
+
+  AnyKeysSegmenter(const std::vector<TypeHolder>& key_types, ExecContext* ctx)
+      : BaseRowSegmenter(key_types),
+        ctx_(ctx),
+        grouper_(nullptr),
+        save_group_id_(kNoGroupId) {}
+
+  Status Reset() override {

Review Comment:
   I'll add test coverage for segmenter resetting.
   
   > In your mind, how would this be used in the future?
   
   For example, outside the context of this PR, it makes sense to reset a segmenter when each batch is processed independently, rather than within a stream of batches, so that a segment must not cross into the next batch.
   
   > I'd suggesting explaining how u think this can be used in the future.
   
   I'll add a doc for this.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] icexelloss commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "icexelloss (via GitHub)" <gi...@apache.org>.
icexelloss commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1123635688


##########
cpp/src/arrow/compute/row/grouper.cc:
##########
@@ -39,12 +43,336 @@
 namespace arrow {
 
 using internal::checked_cast;
+using internal::PrimitiveScalarBase;
 
 namespace compute {
 
 namespace {
 
-struct GrouperImpl : Grouper {
+constexpr uint32_t kNoGroupId = std::numeric_limits<uint32_t>::max();
+
+using group_id_t = std::remove_const<decltype(kNoGroupId)>::type;
+using GroupIdType = CTypeTraits<group_id_t>::ArrowType;
+auto g_group_id_type = std::make_shared<GroupIdType>();
+
+inline const uint8_t* GetValuesAsBytes(const ArraySpan& data, int64_t offset = 0) {
+  DCHECK_GT(data.type->byte_width(), 0);
+  int64_t absolute_byte_offset = (data.offset + offset) * data.type->byte_width();
+  return data.GetValues<uint8_t>(1, absolute_byte_offset);
+}
+
+template <typename Value>
+Status CheckForGetNextSegment(const std::vector<Value>& values, int64_t length,
+                              int64_t offset, const std::vector<TypeHolder>& key_types) {
+  if (offset < 0 || offset > length) {

Review Comment:
   I think the entire function can be probably DCHECK - these checks are more like "invariants" to me. @westonpace WDYT?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rtpsw commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "rtpsw (via GitHub)" <gi...@apache.org>.
rtpsw commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1126793417


##########
cpp/src/arrow/compute/kernels/hash_aggregate_test.cc:
##########
@@ -174,81 +242,117 @@ Result<Datum> RunGroupBy(const BatchesWithSchema& input,
   ARROW_ASSIGN_OR_RAISE(std::vector<ExecBatch> output_batches,
                         start_and_collect.MoveResult());
 
-  ArrayVector out_arrays(aggregates.size() + key_names.size());
   const auto& output_schema = plan->nodes()[0]->output()->output_schema();
+  if (!segmented) {
+    return MakeGroupByOutput(output_batches, output_schema, aggregates.size(),
+                             key_names.size(), naive);
+  }
+
+  std::vector<ArrayVector> out_arrays(aggregates.size() + key_names.size() +
+                                      segment_key_names.size());
   for (size_t i = 0; i < out_arrays.size(); ++i) {
     std::vector<std::shared_ptr<Array>> arrays(output_batches.size());
     for (size_t j = 0; j < output_batches.size(); ++j) {
-      arrays[j] = output_batches[j].values[i].make_array();
+      auto& value = output_batches[j].values[i];
+      if (value.is_scalar()) {
+        ARROW_ASSIGN_OR_RAISE(
+            arrays[j], MakeArrayFromScalar(*value.scalar(), output_batches[j].length));
+      } else if (value.is_array()) {
+        arrays[j] = value.make_array();
+      } else {
+        return Status::Invalid("GroupByUsingExecPlan unsupported value kind ",
+                               ToString(value.kind()));
+      }
     }
     if (arrays.empty()) {
+      arrays.resize(1);
       ARROW_ASSIGN_OR_RAISE(
-          out_arrays[i],
-          MakeArrayOfNull(output_schema->field(static_cast<int>(i))->type(),
-                          /*length=*/0));
-    } else {
-      ARROW_ASSIGN_OR_RAISE(out_arrays[i], Concatenate(arrays));
+          arrays[0], MakeArrayOfNull(output_schema->field(static_cast<int>(i))->type(),
+                                     /*length=*/0));
     }
+    out_arrays[i] = {std::move(arrays)};
   }
 
-  // The exec plan may reorder the output rows.  The tests are all setup to expect ouptut
-  // in ascending order of keys.  So we need to sort the result by the key columns.  To do
-  // that we create a table using the key columns, calculate the sort indices from that
-  // table (sorting on all fields) and then use those indices to calculate our result.
-  std::vector<std::shared_ptr<Field>> key_fields;
-  std::vector<std::shared_ptr<Array>> key_columns;
-  std::vector<SortKey> sort_keys;
-  for (std::size_t i = 0; i < key_names.size(); i++) {
-    const std::shared_ptr<Array>& arr = out_arrays[i + aggregates.size()];
-    if (arr->type_id() == Type::DICTIONARY) {
-      // Can't sort dictionary columns so need to decode
-      auto dict_arr = checked_pointer_cast<DictionaryArray>(arr);
-      ARROW_ASSIGN_OR_RAISE(auto decoded_arr,
-                            Take(*dict_arr->dictionary(), *dict_arr->indices()));
-      key_columns.push_back(decoded_arr);
-      key_fields.push_back(
-          field("name_does_not_matter", dict_arr->dict_type()->value_type()));
-    } else {
-      key_columns.push_back(arr);
-      key_fields.push_back(field("name_does_not_matter", arr->type()));
+  if (segmented && segment_key_names.size() > 0) {
+    ArrayVector struct_arrays;
+    struct_arrays.reserve(output_batches.size());
+    for (size_t j = 0; j < output_batches.size(); ++j) {
+      ArrayVector struct_fields;
+      struct_fields.reserve(out_arrays.size());
+      for (auto out_array : out_arrays) {
+        struct_fields.push_back(out_array[j]);
+      }
+      ARROW_ASSIGN_OR_RAISE(auto struct_array,
+                            StructArray::Make(struct_fields, output_schema->fields()));
+      struct_arrays.push_back(struct_array);
     }
-    sort_keys.emplace_back(static_cast<int>(i));
+    return ChunkedArray::Make(struct_arrays);
+  } else {
+    ArrayVector struct_fields(out_arrays.size());
+    for (size_t i = 0; i < out_arrays.size(); ++i) {
+      ARROW_ASSIGN_OR_RAISE(struct_fields[i], Concatenate(out_arrays[i]));
+    }
+    return StructArray::Make(std::move(struct_fields), output_schema->fields());
   }
-  std::shared_ptr<Schema> key_schema = schema(std::move(key_fields));
-  std::shared_ptr<Table> key_table = Table::Make(std::move(key_schema), key_columns);
-  SortOptions sort_options(std::move(sort_keys));
-  ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Array> sort_indices,
-                        SortIndices(key_table, sort_options));
+}
 
-  ARROW_ASSIGN_OR_RAISE(
-      std::shared_ptr<Array> struct_arr,
-      StructArray::Make(std::move(out_arrays), output_schema->fields()));
+Result<Datum> RunGroupBy(const BatchesWithSchema& input,
+                         const std::vector<std::string>& key_names,
+                         const std::vector<std::string>& segment_key_names,
+                         const std::vector<Aggregate>& aggregates, bool use_threads,
+                         bool segmented = false, bool naive = false) {
+  if (segment_key_names.size() > 0) {
+    ARROW_ASSIGN_OR_RAISE(auto thread_pool, arrow::internal::ThreadPool::Make(1));
+    ExecContext seq_ctx(default_memory_pool(), thread_pool.get());
+    return RunGroupBy(input, key_names, segment_key_names, aggregates, &seq_ctx,
+                      use_threads, segmented, naive);
+  } else {
+    return RunGroupBy(input, key_names, segment_key_names, aggregates,
+                      threaded_exec_context(), use_threads, segmented, naive);
+  }
+}
 
-  return Take(struct_arr, sort_indices);
+Result<Datum> RunGroupBy(const BatchesWithSchema& input,
+                         const std::vector<std::string>& key_names,
+                         const std::vector<Aggregate>& aggregates, bool use_threads,
+                         bool segmented = false, bool naive = false) {
+  return RunGroupBy(input, key_names, {}, aggregates, use_threads, segmented);
 }
 
 /// Simpler overload where you can give the columns as datums
 Result<Datum> RunGroupBy(const std::vector<Datum>& arguments,
                          const std::vector<Datum>& keys,
-                         const std::vector<Aggregate>& aggregates,
-                         bool use_threads = false) {
+                         const std::vector<Datum>& segment_keys,
+                         const std::vector<Aggregate>& aggregates, bool use_threads,
+                         bool segmented = false, bool naive = false) {

Review Comment:
   > Sorry I am still confused - what is "naive group by"?
   
   `NaiveGroupBy` is [a tester function](https://github.com/apache/arrow/blob/d5b3b4737838774db658d3c488fcd3e72bc13f7e/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc#L75-L136) that computes the expected aggregation result (in a naive/simple/non-optimized way).
   
   > What do you mean by "segmented aggregation with empty segment-keys"? Isn't this what happens when segment-keys is empty? (The original hash aggregate)
   
   Let's take a step back. When `segment_keys` is non-empty the `segmented` flag is always `true`; otherwise (when empty), it may still be set to `true`, and this is the case we're discussing here. In this case, the tester restructures (without changing the data of) the result of `RunGroupBy` from `std::vector<ExecBatch> output_batches` to `std::vector<ArrayVector> out_arrays`, which have the structure typical of the case of a non-empty `segment_keys` (with multiple arrays per column, one array per segment) but only one array per column (because, technically, there is only one segment in this case). What all this boils down to is a test focusing on the structure, rather than the data, of the result.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rtpsw commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "rtpsw (via GitHub)" <gi...@apache.org>.
rtpsw commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1126795096


##########
cpp/src/arrow/compute/kernels/hash_aggregate_test.cc:
##########
@@ -301,53 +426,184 @@ Result<Datum> GroupByTest(const std::vector<Datum>& arguments,
         {t_agg.function, t_agg.options, "agg_" + ToChars(idx), t_agg.function});
     idx = idx + 1;
   }
-  return RunGroupBy(arguments, keys, internal_aggregates, use_threads);
+  return group_by(arguments, keys, segment_keys, internal_aggregates, use_threads,
+                  /*naive=*/false);
 }
 
-}  // namespace
+Result<Datum> GroupByTest(GroupByFunction group_by, const std::vector<Datum>& arguments,
+                          const std::vector<Datum>& keys,
+                          const std::vector<TestAggregate>& aggregates,
+                          bool use_threads) {
+  return GroupByTest(group_by, arguments, keys, {}, aggregates, use_threads);
+}
 
-TEST(Grouper, SupportedKeys) {
-  ASSERT_OK(Grouper::Make({boolean()}));
+template <typename GroupClass>

Review Comment:
   > Can we add some comments about this in the test file? Seems like both me and Weston found this confusing.
   
   Please see if you prefer [my suggested alternative](https://github.com/apache/arrow/pull/34311#discussion_r1126768877), which should make this unnecessary.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rtpsw commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "rtpsw (via GitHub)" <gi...@apache.org>.
rtpsw commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1127077710


##########
cpp/src/arrow/compute/row/grouper.h:
##########
@@ -39,10 +108,19 @@ class ARROW_EXPORT Grouper {
   static Result<std::unique_ptr<Grouper>> Make(const std::vector<TypeHolder>& key_types,
                                                ExecContext* ctx = default_exec_context());
 
-  /// Consume a batch of keys, producing the corresponding group ids as an integer array.
+  /// Consume a batch of keys, producing the corresponding group ids as an integer array,
+  /// over a slice defined by an offset and length, which defaults to the batch length.
+  /// Currently only uint32 indices will be produced, eventually the bit width will only
+  /// be as wide as necessary.
+  virtual Result<Datum> Consume(const ExecSpan& batch, int64_t offset = 0,
+                                int64_t length = -1) = 0;
+
+  /// Consume a batch of keys, producing the corresponding group ids as an integer array,
+  /// over a slice defined by an offset and length, which defaults to the batch length.
   /// Currently only uint32 indices will be produced, eventually the bit width will only
   /// be as wide as necessary.
-  virtual Result<Datum> Consume(const ExecSpan& batch) = 0;
+  virtual Result<Datum> Consume(const ExecBatch& batch, int64_t offset = 0,

Review Comment:
   Right, I'll remove this code along with some more that would become unnecessary.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] icexelloss commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "icexelloss (via GitHub)" <gi...@apache.org>.
icexelloss commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1127955354


##########
cpp/src/arrow/compute/row/grouper.cc:
##########
@@ -39,12 +43,330 @@
 namespace arrow {
 
 using internal::checked_cast;
+using internal::PrimitiveScalarBase;
 
 namespace compute {
 
 namespace {
 
-struct GrouperImpl : Grouper {
+constexpr uint32_t kNoGroupId = std::numeric_limits<uint32_t>::max();
+
+using group_id_t = std::remove_const<decltype(kNoGroupId)>::type;
+using GroupIdType = CTypeTraits<group_id_t>::ArrowType;
+auto g_group_id_type = std::make_shared<GroupIdType>();
+
+inline const uint8_t* GetValuesAsBytes(const ArraySpan& data, int64_t offset = 0) {
+  DCHECK_GT(data.type->byte_width(), 0);
+  int64_t absolute_byte_offset = (data.offset + offset) * data.type->byte_width();
+  return data.GetValues<uint8_t>(1, absolute_byte_offset);
+}
+
+template <typename Value>
+Status CheckForGetNextSegment(const std::vector<Value>& values, int64_t length,
+                              int64_t offset, const std::vector<TypeHolder>& key_types) {
+  if (offset < 0 || offset > length) {
+    return Status::Invalid("invalid grouping segmenter offset: ", offset);
+  }
+  if (values.size() != key_types.size()) {
+    return Status::Invalid("expected batch size ", key_types.size(), " but got ",
+                           values.size());
+  }
+  for (size_t i = 0; i < key_types.size(); i++) {
+    const auto& value = values[i];
+    const auto& key_type = key_types[i];
+    if (*value.type() != *key_type.type) {
+      return Status::Invalid("expected batch value ", i, " of type ", *key_type.type,
+                             " but got ", *value.type());
+    }
+  }
+  return Status::OK();
+}
+
+template <typename Batch>
+enable_if_t<std::is_same<Batch, ExecSpan>::value || std::is_same<Batch, ExecBatch>::value,
+            Status>
+CheckForGetNextSegment(const Batch& batch, int64_t offset,
+                       const std::vector<TypeHolder>& key_types) {
+  return CheckForGetNextSegment(batch.values, batch.length, offset, key_types);
+}
+
+struct BaseRowSegmenter : public RowSegmenter {
+  explicit BaseRowSegmenter(const std::vector<TypeHolder>& key_types)
+      : key_types_(key_types) {}
+
+  const std::vector<TypeHolder>& key_types() const override { return key_types_; }
+
+  std::vector<TypeHolder> key_types_;
+};
+
+Segment MakeSegment(int64_t batch_length, int64_t offset, int64_t length, bool extends) {
+  return Segment{offset, length, offset + length >= batch_length, extends};
+}
+
+int64_t GetMatchLength(const uint8_t* match_bytes, int64_t match_width,
+                       const uint8_t* array_bytes, int64_t offset, int64_t length) {
+  int64_t cursor, byte_cursor;
+  for (cursor = offset, byte_cursor = match_width * cursor; cursor < length;
+       cursor++, byte_cursor += match_width) {
+    if (memcmp(match_bytes, array_bytes + byte_cursor,
+               static_cast<size_t>(match_width)) != 0) {
+      break;
+    }
+  }
+  return std::min(cursor, length) - offset;
+}
+
+using ExtendFunc = std::function<bool(const void*)>;
+constexpr bool kDefaultExtends = true;
+constexpr bool kEmptyExtends = true;
+
+struct NoKeysSegmenter : public BaseRowSegmenter {
+  static std::unique_ptr<RowSegmenter> Make() {
+    return std::make_unique<NoKeysSegmenter>();
+  }
+
+  NoKeysSegmenter() : BaseRowSegmenter({}) {}
+
+  Status Reset() override { return Status::OK(); }
+
+  Result<Segment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {}));
+    return MakeSegment(batch.length, offset, batch.length - offset, kDefaultExtends);
+  }
+};
+
+struct SimpleKeySegmenter : public BaseRowSegmenter {
+  static Result<std::unique_ptr<RowSegmenter>> Make(TypeHolder key_type) {
+    return std::make_unique<SimpleKeySegmenter>(key_type);
+  }
+
+  explicit SimpleKeySegmenter(TypeHolder key_type)
+      : BaseRowSegmenter({key_type}), key_type_(key_types_[0]), save_key_data_() {}
+
+  Status CheckType(const DataType& type) {
+    if (!is_fixed_width(type)) {
+      return Status::Invalid("SimpleKeySegmenter does not support type ", type);
+    }
+    return Status::OK();
+  }
+
+  Status Reset() override {
+    save_key_data_.resize(0);
+    return Status::OK();
+  }
+
+  // Checks whether the given grouping data extends the current segment, i.e., is equal to
+  // previously seen grouping data, which is updated with each invocation.
+  bool Extend(const void* data) {
+    size_t byte_width = static_cast<size_t>(key_type_.type->byte_width());
+    bool extends = save_key_data_.size() != byte_width
+                       ? kDefaultExtends
+                       : 0 == memcmp(save_key_data_.data(), data, byte_width);
+    save_key_data_.resize(byte_width);
+    memcpy(save_key_data_.data(), data, byte_width);
+    return extends;
+  }
+
+  Result<Segment> GetNextSegment(const Scalar& scalar, int64_t offset, int64_t length) {
+    ARROW_RETURN_NOT_OK(CheckType(*scalar.type));
+    if (!scalar.is_valid) {
+      return Status::Invalid("segmenting an invalid scalar");
+    }
+    auto data = checked_cast<const PrimitiveScalarBase&>(scalar).data();
+    bool extends = length > 0 ? Extend(data) : kEmptyExtends;
+    return MakeSegment(length, offset, length, extends);
+  }
+
+  Result<Segment> GetNextSegment(const DataType& array_type, const uint8_t* array_bytes,
+                                 int64_t offset, int64_t length) {
+    RETURN_NOT_OK(CheckType(array_type));
+    int64_t byte_width = array_type.byte_width();
+    int64_t match_length = GetMatchLength(array_bytes + offset * byte_width, byte_width,
+                                          array_bytes, offset, length);
+    bool extends = length > 0 ? Extend(array_bytes + offset * byte_width) : kEmptyExtends;
+    return MakeSegment(length, offset, match_length, extends);
+  }
+
+  Result<Segment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {key_type_}));
+    if (offset == batch.length) {
+      return MakeSegment(batch.length, offset, 0, kEmptyExtends);
+    }
+    const auto& value = batch.values[0];
+    if (value.is_scalar()) {
+      return GetNextSegment(*value.scalar, offset, batch.length);
+    }
+    ARROW_DCHECK(value.is_array());
+    const auto& array = value.array;
+    if (array.GetNullCount() > 0) {
+      return Status::NotImplemented("segmenting a nullable array");
+    }
+    return GetNextSegment(*array.type, GetValuesAsBytes(array), offset, batch.length);
+  }
+
+ private:
+  TypeHolder key_type_;
+  std::vector<uint8_t> save_key_data_;
+};
+
+struct AnyKeysSegmenter : public BaseRowSegmenter {
+  static Result<std::unique_ptr<RowSegmenter>> Make(
+      const std::vector<TypeHolder>& key_types, ExecContext* ctx) {
+    ARROW_RETURN_NOT_OK(Grouper::Make(key_types, ctx));  // check types
+    return std::make_unique<AnyKeysSegmenter>(key_types, ctx);
+  }
+
+  AnyKeysSegmenter(const std::vector<TypeHolder>& key_types, ExecContext* ctx)
+      : BaseRowSegmenter(key_types),
+        ctx_(ctx),
+        grouper_(nullptr),
+        save_group_id_(kNoGroupId) {}
+
+  Status Reset() override {
+    grouper_ = nullptr;
+    save_group_id_ = kNoGroupId;
+    return Status::OK();
+  }
+
+  bool Extend(const void* data) {
+    auto group_id = *static_cast<const group_id_t*>(data);
+    bool extends =
+        save_group_id_ == kNoGroupId ? kDefaultExtends : save_group_id_ == group_id;
+    save_group_id_ = group_id;
+    return extends;
+  }
+
+  // Runs the grouper on a single row.  This is used to determine the group id of the
+  // first row of a new segment to see if it extends the previous segment.
+  template <typename Batch>
+  Result<group_id_t> MapGroupIdAt(const Batch& batch, int64_t offset) {
+    if (!grouper_) return kNoGroupId;
+    ARROW_ASSIGN_OR_RAISE(auto datum, grouper_->Consume(batch, offset,
+                                                        /*length=*/1));
+    if (!datum.is_array()) {
+      return Status::Invalid("accessing unsupported datum kind ", datum.kind());
+    }
+    const std::shared_ptr<ArrayData>& data = datum.array();
+    ARROW_DCHECK(data->GetNullCount() == 0);
+    DCHECK_EQ(data->type->id(), GroupIdType::type_id);
+    DCHECK_EQ(1, data->length);
+    const group_id_t* values = data->GetValues<group_id_t>(1);
+    return values[0];
+  }
+
+  Result<Segment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, key_types_));
+    if (offset == batch.length) {
+      return MakeSegment(batch.length, offset, 0, kEmptyExtends);
+    }
+    // ARROW-18311: make Grouper support Reset()
+    // so it can be cached instead of recreated below
+    //
+    // the group id must be computed prior to resetting the grouper, since it is compared
+    // to save_group_id_, and after resetting the grouper produces incomparable group ids
+    ARROW_ASSIGN_OR_RAISE(auto group_id, MapGroupIdAt(batch, offset));
+    ExtendFunc bound_extend = [this, group_id](const void* data) {
+      bool extends = Extend(&group_id);
+      save_group_id_ = *static_cast<const group_id_t*>(data);
+      return extends;
+    };
+    ARROW_ASSIGN_OR_RAISE(grouper_, Grouper::Make(key_types_, ctx_));  // TODO: reset it

Review Comment:
   I see - please put this explanation in the comment to help future reader understand



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] icexelloss commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "icexelloss (via GitHub)" <gi...@apache.org>.
icexelloss commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1127956115


##########
cpp/src/arrow/compute/row/grouper.cc:
##########
@@ -39,12 +43,330 @@
 namespace arrow {
 
 using internal::checked_cast;
+using internal::PrimitiveScalarBase;
 
 namespace compute {
 
 namespace {
 
-struct GrouperImpl : Grouper {
+constexpr uint32_t kNoGroupId = std::numeric_limits<uint32_t>::max();
+
+using group_id_t = std::remove_const<decltype(kNoGroupId)>::type;
+using GroupIdType = CTypeTraits<group_id_t>::ArrowType;
+auto g_group_id_type = std::make_shared<GroupIdType>();
+
+inline const uint8_t* GetValuesAsBytes(const ArraySpan& data, int64_t offset = 0) {
+  DCHECK_GT(data.type->byte_width(), 0);
+  int64_t absolute_byte_offset = (data.offset + offset) * data.type->byte_width();
+  return data.GetValues<uint8_t>(1, absolute_byte_offset);
+}
+
+template <typename Value>
+Status CheckForGetNextSegment(const std::vector<Value>& values, int64_t length,
+                              int64_t offset, const std::vector<TypeHolder>& key_types) {
+  if (offset < 0 || offset > length) {
+    return Status::Invalid("invalid grouping segmenter offset: ", offset);
+  }
+  if (values.size() != key_types.size()) {
+    return Status::Invalid("expected batch size ", key_types.size(), " but got ",
+                           values.size());
+  }
+  for (size_t i = 0; i < key_types.size(); i++) {
+    const auto& value = values[i];
+    const auto& key_type = key_types[i];
+    if (*value.type() != *key_type.type) {
+      return Status::Invalid("expected batch value ", i, " of type ", *key_type.type,
+                             " but got ", *value.type());
+    }
+  }
+  return Status::OK();
+}
+
+template <typename Batch>
+enable_if_t<std::is_same<Batch, ExecSpan>::value || std::is_same<Batch, ExecBatch>::value,
+            Status>
+CheckForGetNextSegment(const Batch& batch, int64_t offset,
+                       const std::vector<TypeHolder>& key_types) {
+  return CheckForGetNextSegment(batch.values, batch.length, offset, key_types);
+}
+
+struct BaseRowSegmenter : public RowSegmenter {
+  explicit BaseRowSegmenter(const std::vector<TypeHolder>& key_types)
+      : key_types_(key_types) {}
+
+  const std::vector<TypeHolder>& key_types() const override { return key_types_; }
+
+  std::vector<TypeHolder> key_types_;
+};
+
+Segment MakeSegment(int64_t batch_length, int64_t offset, int64_t length, bool extends) {
+  return Segment{offset, length, offset + length >= batch_length, extends};
+}
+
+int64_t GetMatchLength(const uint8_t* match_bytes, int64_t match_width,
+                       const uint8_t* array_bytes, int64_t offset, int64_t length) {
+  int64_t cursor, byte_cursor;
+  for (cursor = offset, byte_cursor = match_width * cursor; cursor < length;
+       cursor++, byte_cursor += match_width) {
+    if (memcmp(match_bytes, array_bytes + byte_cursor,
+               static_cast<size_t>(match_width)) != 0) {
+      break;
+    }
+  }
+  return std::min(cursor, length) - offset;
+}
+
+using ExtendFunc = std::function<bool(const void*)>;
+constexpr bool kDefaultExtends = true;
+constexpr bool kEmptyExtends = true;
+
+struct NoKeysSegmenter : public BaseRowSegmenter {
+  static std::unique_ptr<RowSegmenter> Make() {
+    return std::make_unique<NoKeysSegmenter>();
+  }
+
+  NoKeysSegmenter() : BaseRowSegmenter({}) {}
+
+  Status Reset() override { return Status::OK(); }
+
+  Result<Segment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {}));
+    return MakeSegment(batch.length, offset, batch.length - offset, kDefaultExtends);
+  }
+};
+
+struct SimpleKeySegmenter : public BaseRowSegmenter {
+  static Result<std::unique_ptr<RowSegmenter>> Make(TypeHolder key_type) {
+    return std::make_unique<SimpleKeySegmenter>(key_type);
+  }
+
+  explicit SimpleKeySegmenter(TypeHolder key_type)
+      : BaseRowSegmenter({key_type}), key_type_(key_types_[0]), save_key_data_() {}
+
+  Status CheckType(const DataType& type) {
+    if (!is_fixed_width(type)) {
+      return Status::Invalid("SimpleKeySegmenter does not support type ", type);
+    }
+    return Status::OK();
+  }
+
+  Status Reset() override {
+    save_key_data_.resize(0);
+    return Status::OK();
+  }
+
+  // Checks whether the given grouping data extends the current segment, i.e., is equal to
+  // previously seen grouping data, which is updated with each invocation.
+  bool Extend(const void* data) {
+    size_t byte_width = static_cast<size_t>(key_type_.type->byte_width());
+    bool extends = save_key_data_.size() != byte_width
+                       ? kDefaultExtends
+                       : 0 == memcmp(save_key_data_.data(), data, byte_width);
+    save_key_data_.resize(byte_width);
+    memcpy(save_key_data_.data(), data, byte_width);
+    return extends;
+  }
+
+  Result<Segment> GetNextSegment(const Scalar& scalar, int64_t offset, int64_t length) {
+    ARROW_RETURN_NOT_OK(CheckType(*scalar.type));
+    if (!scalar.is_valid) {
+      return Status::Invalid("segmenting an invalid scalar");
+    }
+    auto data = checked_cast<const PrimitiveScalarBase&>(scalar).data();
+    bool extends = length > 0 ? Extend(data) : kEmptyExtends;
+    return MakeSegment(length, offset, length, extends);
+  }
+
+  Result<Segment> GetNextSegment(const DataType& array_type, const uint8_t* array_bytes,
+                                 int64_t offset, int64_t length) {
+    RETURN_NOT_OK(CheckType(array_type));
+    int64_t byte_width = array_type.byte_width();
+    int64_t match_length = GetMatchLength(array_bytes + offset * byte_width, byte_width,
+                                          array_bytes, offset, length);
+    bool extends = length > 0 ? Extend(array_bytes + offset * byte_width) : kEmptyExtends;
+    return MakeSegment(length, offset, match_length, extends);
+  }
+
+  Result<Segment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {key_type_}));
+    if (offset == batch.length) {
+      return MakeSegment(batch.length, offset, 0, kEmptyExtends);
+    }
+    const auto& value = batch.values[0];
+    if (value.is_scalar()) {
+      return GetNextSegment(*value.scalar, offset, batch.length);
+    }
+    ARROW_DCHECK(value.is_array());
+    const auto& array = value.array;
+    if (array.GetNullCount() > 0) {
+      return Status::NotImplemented("segmenting a nullable array");
+    }
+    return GetNextSegment(*array.type, GetValuesAsBytes(array), offset, batch.length);
+  }
+
+ private:
+  TypeHolder key_type_;
+  std::vector<uint8_t> save_key_data_;
+};
+
+struct AnyKeysSegmenter : public BaseRowSegmenter {
+  static Result<std::unique_ptr<RowSegmenter>> Make(
+      const std::vector<TypeHolder>& key_types, ExecContext* ctx) {
+    ARROW_RETURN_NOT_OK(Grouper::Make(key_types, ctx));  // check types
+    return std::make_unique<AnyKeysSegmenter>(key_types, ctx);
+  }
+
+  AnyKeysSegmenter(const std::vector<TypeHolder>& key_types, ExecContext* ctx)
+      : BaseRowSegmenter(key_types),
+        ctx_(ctx),
+        grouper_(nullptr),
+        save_group_id_(kNoGroupId) {}
+
+  Status Reset() override {
+    grouper_ = nullptr;
+    save_group_id_ = kNoGroupId;
+    return Status::OK();
+  }
+
+  bool Extend(const void* data) {
+    auto group_id = *static_cast<const group_id_t*>(data);
+    bool extends =
+        save_group_id_ == kNoGroupId ? kDefaultExtends : save_group_id_ == group_id;
+    save_group_id_ = group_id;
+    return extends;
+  }
+
+  // Runs the grouper on a single row.  This is used to determine the group id of the
+  // first row of a new segment to see if it extends the previous segment.
+  template <typename Batch>
+  Result<group_id_t> MapGroupIdAt(const Batch& batch, int64_t offset) {
+    if (!grouper_) return kNoGroupId;
+    ARROW_ASSIGN_OR_RAISE(auto datum, grouper_->Consume(batch, offset,
+                                                        /*length=*/1));
+    if (!datum.is_array()) {
+      return Status::Invalid("accessing unsupported datum kind ", datum.kind());
+    }
+    const std::shared_ptr<ArrayData>& data = datum.array();
+    ARROW_DCHECK(data->GetNullCount() == 0);
+    DCHECK_EQ(data->type->id(), GroupIdType::type_id);
+    DCHECK_EQ(1, data->length);
+    const group_id_t* values = data->GetValues<group_id_t>(1);

Review Comment:
   That's fine.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rtpsw commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "rtpsw (via GitHub)" <gi...@apache.org>.
rtpsw commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1116163443


##########
cpp/src/arrow/compute/exec.h:
##########
@@ -240,6 +246,8 @@ struct ARROW_EXPORT ExecBatch {
 
   ExecBatch Slice(int64_t offset, int64_t length) const;
 
+  Result<ExecBatch> SelectValues(const std::vector<int>& ids) const;

Review Comment:
   It is [used](https://github.com/apache/arrow/pull/34311/files#diff-f53040e5fc284581263737f96238c8a76ef54d13f01e34305c1d1d9de5e6e34bR214) for selecting the values (i.e., columns) of the batch that correspond to the segment-keys. The resulting sub-batch, having just these values, is [processed by the segmenter](https://github.com/apache/arrow/pull/34311/files#diff-f53040e5fc284581263737f96238c8a76ef54d13f01e34305c1d1d9de5e6e34bR216).



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rtpsw commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "rtpsw (via GitHub)" <gi...@apache.org>.
rtpsw commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1117018148


##########
cpp/src/arrow/compute/row/grouper.h:
##########
@@ -30,6 +30,49 @@
 namespace arrow {
 namespace compute {
 
+/// \brief A segment of contiguous rows for grouping
+struct ARROW_EXPORT GroupingSegment {
+  int64_t offset;
+  int64_t length;
+  bool is_open;
+  bool extends;
+};
+
+inline bool operator==(const GroupingSegment& segment1, const GroupingSegment& segment2) {
+  return segment1.offset == segment2.offset && segment1.length == segment2.length &&
+         segment1.is_open == segment2.is_open && segment1.extends == segment2.extends;
+}
+inline bool operator!=(const GroupingSegment& segment1, const GroupingSegment& segment2) {
+  return !(segment1 == segment2);
+}
+
+/// \brief Computes grouping segments for a batch. Each segment covers rows with identical
+/// values in the batch. The values in the batch are often selected as keys from a larger
+/// batch.
+class ARROW_EXPORT GroupingSegmenter {
+ public:
+  virtual ~GroupingSegmenter() = default;
+
+  /// \brief Construct a GroupingSegmenter which receives the specified key types
+  static Result<std::unique_ptr<GroupingSegmenter>> Make(
+      const std::vector<TypeHolder>& key_types, bool nullable_keys = false,
+      ExecContext* ctx = default_exec_context());

Review Comment:
   This removal would break the [template-invocation discussed here](https://github.com/apache/arrow/pull/34311#discussion_r1116357888). I could refactor this invocation, if you feel strongly about the API here.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rtpsw commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "rtpsw (via GitHub)" <gi...@apache.org>.
rtpsw commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1117530261


##########
cpp/src/arrow/compute/row/grouper.cc:
##########
@@ -39,12 +43,521 @@
 namespace arrow {
 
 using internal::checked_cast;
+using internal::PrimitiveScalarBase;
 
 namespace compute {
 
 namespace {
 
-struct GrouperImpl : Grouper {
+constexpr uint32_t kNoGroupId = std::numeric_limits<uint32_t>::max();
+
+using group_id_t = std::remove_const<decltype(kNoGroupId)>::type;
+using GroupIdType = CTypeTraits<group_id_t>::ArrowType;
+auto group_id_type = std::make_shared<GroupIdType>();
+
+inline const uint8_t* GetValuesAsBytes(const ArrayData& data, int64_t offset = 0) {
+  DCHECK_GT(data.type->byte_width(), 0);
+  int64_t absolute_byte_offset = (data.offset + offset) * data.type->byte_width();
+  return data.GetValues<uint8_t>(1, absolute_byte_offset);
+}
+
+inline const uint8_t* GetValuesAsBytes(const ArraySpan& data, int64_t offset = 0) {
+  DCHECK_GT(data.type->byte_width(), 0);
+  int64_t absolute_byte_offset = (data.offset + offset) * data.type->byte_width();
+  return data.GetValues<uint8_t>(1, absolute_byte_offset);
+}
+
+template <typename Value>
+Status CheckForGetNextSegment(const std::vector<Value>& values, int64_t length,
+                              int64_t offset, const std::vector<TypeHolder>& key_types) {
+  if (offset < 0 || offset > length) {
+    return Status::Invalid("invalid grouping segmenter offset: ", offset);
+  }
+  if (values.size() != key_types.size()) {
+    return Status::Invalid("expected batch size ", key_types.size(), " but got ",
+                           values.size());
+  }
+  for (size_t i = 0; i < key_types.size(); i++) {
+    const auto& value = values[i];
+    const auto& key_type = key_types[i];
+    if (*value.type() != *key_type.type) {
+      return Status::Invalid("expected batch value ", i, " of type ", *key_type.type,
+                             " but got ", *value.type());
+    }
+  }
+  return Status::OK();
+}
+
+template <typename Batch>
+enable_if_t<std::is_same<Batch, ExecSpan>::value || std::is_same<Batch, ExecBatch>::value,
+            Status>
+CheckForGetNextSegment(const Batch& batch, int64_t offset,
+                       const std::vector<TypeHolder>& key_types) {
+  return CheckForGetNextSegment(batch.values, batch.length, offset, key_types);
+}
+
+struct BaseGroupingSegmenter : public GroupingSegmenter {
+  explicit BaseGroupingSegmenter(const std::vector<TypeHolder>& key_types)
+      : key_types_(key_types) {}
+
+  const std::vector<TypeHolder>& key_types() const override { return key_types_; }
+
+  std::vector<TypeHolder> key_types_;
+};
+
+GroupingSegment MakeSegment(int64_t batch_length, int64_t offset, int64_t length,
+                            bool extends) {
+  return GroupingSegment{offset, length, offset + length >= batch_length, extends};
+}
+
+int64_t GetMatchLength(const uint8_t* match_bytes, int64_t match_width,
+                       const uint8_t* array_bytes, int64_t offset, int64_t length) {
+  int64_t cursor, byte_cursor;
+  for (cursor = offset, byte_cursor = match_width * cursor; cursor < length;
+       cursor++, byte_cursor += match_width) {
+    if (memcmp(match_bytes, array_bytes + byte_cursor,
+               static_cast<size_t>(match_width)) != 0) {
+      break;
+    }
+  }
+  return std::min(cursor, length - offset);
+}
+
+using ExtendFunc = std::function<bool(const void*)>;
+constexpr bool kDefaultExtends = true;
+constexpr bool kEmptyExtends = true;
+
+Result<GroupingSegment> GetNextSegmentChunked(
+    const std::shared_ptr<ChunkedArray>& chunked_array, int64_t offset,
+    ExtendFunc extend) {
+  if (offset >= chunked_array->length()) {
+    return MakeSegment(chunked_array->length(), chunked_array->length(), 0,
+                       kEmptyExtends);
+  }
+  int64_t remaining_offset = offset;
+  const auto& arrays = chunked_array->chunks();
+  for (size_t i = 0; remaining_offset >= 0 && i < arrays.size(); i++) {
+    // look up chunk containing offset
+    int64_t array_length = arrays[i]->length();
+    if (remaining_offset < array_length) {
+      // found - switch to matching
+      int64_t match_width = arrays[i]->type()->byte_width();
+      const uint8_t* match_bytes = GetValuesAsBytes(*arrays[i]->data(), remaining_offset);
+      int64_t total_match_length = 0;
+      for (; i < arrays.size(); i++) {
+        int64_t array_length = arrays[i]->length();
+        if (array_length <= 0) continue;
+        const uint8_t* array_bytes = GetValuesAsBytes(*arrays[i]->data());
+        int64_t match_length = GetMatchLength(match_bytes, match_width, array_bytes,
+                                              remaining_offset, array_length);
+        total_match_length += match_length;
+        remaining_offset = 0;
+        if (match_length < array_length - remaining_offset) break;
+      }
+      bool extends = extend(match_bytes);
+      return MakeSegment(chunked_array->length(), offset, total_match_length, extends);
+    }
+    remaining_offset -= array_length;
+  }
+  return Status::Invalid("segmenting invalid chunked array value");
+}
+
+struct NoKeysGroupingSegmenter : public BaseGroupingSegmenter {
+  static std::unique_ptr<GroupingSegmenter> Make() {
+    return std::make_unique<NoKeysGroupingSegmenter>();
+  }
+
+  NoKeysGroupingSegmenter() : BaseGroupingSegmenter({}) {}
+
+  Status Reset() override { return Status::OK(); }
+
+  template <typename Batch>
+  Result<GroupingSegment> GetNextSegmentImpl(const Batch& batch, int64_t offset) {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {}));
+    return MakeSegment(batch.length, offset, batch.length - offset, kDefaultExtends);
+  }
+
+  Result<GroupingSegment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    return GetNextSegmentImpl(batch, offset);
+  }
+
+  Result<GroupingSegment> GetNextSegment(const ExecBatch& batch,
+                                         int64_t offset) override {
+    return GetNextSegmentImpl(batch, offset);
+  }
+};
+
+struct SimpleKeyGroupingSegmenter : public BaseGroupingSegmenter {
+  static Result<std::unique_ptr<GroupingSegmenter>> Make(TypeHolder key_type) {
+    return std::make_unique<SimpleKeyGroupingSegmenter>(key_type);
+  }
+
+  explicit SimpleKeyGroupingSegmenter(TypeHolder key_type)
+      : BaseGroupingSegmenter({key_type}), key_type_(key_types_[0]), save_key_data_() {}
+
+  Status CheckType(const DataType& type) {
+    if (!is_fixed_width(type)) {
+      return Status::Invalid("SimpleKeyGroupingSegmenter does not support type ", type);
+    }
+    return Status::OK();
+  }
+
+  Status Reset() override {
+    save_key_data_.resize(0);
+    return Status::OK();
+  }
+
+  bool Extend(const void* data) {
+    size_t byte_width = static_cast<size_t>(key_type_.type->byte_width());
+    bool extends = save_key_data_.size() != byte_width
+                       ? kDefaultExtends
+                       : 0 == memcmp(save_key_data_.data(), data, byte_width);
+    save_key_data_.resize(byte_width);
+    memcpy(save_key_data_.data(), data, byte_width);
+    return extends;
+  }
+
+  Result<GroupingSegment> GetNextSegment(const Scalar& scalar, int64_t offset,
+                                         int64_t length) {
+    ARROW_RETURN_NOT_OK(CheckType(*scalar.type));
+    if (!scalar.is_valid) {
+      return Status::Invalid("segmenting an invalid scalar");
+    }
+    auto data = checked_cast<const PrimitiveScalarBase&>(scalar).data();
+    bool extends = length > 0 ? Extend(data) : kEmptyExtends;
+    return MakeSegment(length, 0, length, extends);
+  }
+
+  Result<GroupingSegment> GetNextSegment(const DataType& array_type,
+                                         const uint8_t* array_bytes, int64_t offset,
+                                         int64_t length) {
+    RETURN_NOT_OK(CheckType(array_type));
+    int64_t byte_width = array_type.byte_width();
+    int64_t match_length = GetMatchLength(array_bytes + offset * byte_width, byte_width,
+                                          array_bytes, offset, length);
+    bool extends = length > 0 ? Extend(array_bytes + offset * byte_width) : kEmptyExtends;
+    return MakeSegment(length, offset, match_length, extends);
+  }
+
+  Result<GroupingSegment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {key_type_}));
+    if (offset == batch.length) {
+      return MakeSegment(batch.length, offset, 0, kEmptyExtends);
+    }
+    const auto& value = batch.values[0];
+    if (value.is_scalar()) {
+      return GetNextSegment(*value.scalar, offset, batch.length);
+    }
+    ARROW_DCHECK(value.is_array());
+    const auto& array = value.array;
+    if (array.GetNullCount() > 0) {
+      return Status::NotImplemented("segmenting a nullable array");
+    }
+    return GetNextSegment(*array.type, GetValuesAsBytes(array), offset, batch.length);
+  }
+
+  Result<GroupingSegment> GetNextSegment(const ExecBatch& batch,
+                                         int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {key_type_}));
+    if (offset == batch.length) {
+      return MakeSegment(batch.length, offset, 0, kEmptyExtends);
+    }
+    const auto& value = batch.values[0];
+    if (value.is_scalar()) {
+      return GetNextSegment(*value.scalar(), offset, batch.length);
+    }
+    if (value.is_array()) {
+      auto array = value.array();
+      if (array->GetNullCount() > 0) {
+        return Status::NotImplemented("segmenting a nullable array");
+      }
+      return GetNextSegment(*array->type, GetValuesAsBytes(*array), offset, batch.length);
+    }
+    if (value.is_chunked_array()) {
+      auto array = value.chunked_array();
+      if (array->null_count() > 0) {
+        return Status::NotImplemented("segmenting a nullable array");
+      }
+      return GetNextSegmentChunked(array, offset, bound_extend_);
+    }
+    return Status::Invalid("segmenting unsupported value kind ", value.kind());
+  }
+
+ private:
+  TypeHolder key_type_;
+  std::vector<uint8_t> save_key_data_;
+  ExtendFunc bound_extend_ = [this](const void* data) { return Extend(data); };
+};
+
+struct AnyKeysGroupingSegmenter : public BaseGroupingSegmenter {
+  static Result<std::unique_ptr<GroupingSegmenter>> Make(
+      const std::vector<TypeHolder>& key_types, ExecContext* ctx) {
+    ARROW_RETURN_NOT_OK(Grouper::Make(key_types, ctx));  // check types
+    return std::make_unique<AnyKeysGroupingSegmenter>(key_types, ctx);
+  }
+
+  AnyKeysGroupingSegmenter(const std::vector<TypeHolder>& key_types, ExecContext* ctx)
+      : BaseGroupingSegmenter(key_types),
+        ctx_(ctx),
+        grouper_(nullptr),
+        save_group_id_(kNoGroupId) {}
+
+  Status Reset() override {
+    grouper_ = nullptr;
+    save_group_id_ = kNoGroupId;
+    return Status::OK();
+  }
+
+  bool Extend(const void* data) {
+    auto group_id = *static_cast<const group_id_t*>(data);
+    bool extends =
+        save_group_id_ == kNoGroupId ? kDefaultExtends : save_group_id_ == group_id;
+    save_group_id_ = group_id;
+    return extends;
+  }
+
+  template <typename Batch>
+  Result<group_id_t> MapGroupIdAt(const Batch& batch, int64_t offset) {
+    if (offset < 0 || offset >= batch.length) {
+      return Status::Invalid("requesting group id out of bounds");
+    }
+    if (!grouper_) return kNoGroupId;
+    ARROW_ASSIGN_OR_RAISE(auto datum, grouper_->Consume(batch, offset,
+                                                        /*consume_length=*/1));
+    if (!(datum.is_array() || datum.is_chunked_array())) {
+      return Status::Invalid("accessing unsupported datum kind ", datum.kind());
+    }
+    const std::shared_ptr<ArrayData>& data =
+        datum.is_array() ? datum.array() : datum.chunked_array()->chunk(0)->data();
+    ARROW_DCHECK(data->GetNullCount() == 0);
+    DCHECK_EQ(data->type->id(), GroupIdType::type_id);
+    DCHECK_EQ(1, data->length);
+    const group_id_t* values = data->GetValues<group_id_t>(1);
+    return values[0];
+  }
+
+  template <typename Batch>
+  Result<GroupingSegment> GetNextSegmentImpl(const Batch& batch, int64_t offset) {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, key_types_));
+    if (offset == batch.length) {
+      return MakeSegment(batch.length, offset, 0, kEmptyExtends);
+    }
+    // ARROW-18311: make Grouper support Reset()
+    // so it can be cached instead of recreated below
+    //
+    // the group id must be computed prior to resetting the grouper, since it is compared
+    // to save_group_id_, and after resetting the grouper produces incomparable group ids
+    ARROW_ASSIGN_OR_RAISE(auto group_id, MapGroupIdAt(batch, offset));
+    ExtendFunc bound_extend = [this, group_id](const void* data) {
+      bool extends = Extend(&group_id);
+      save_group_id_ = *static_cast<const group_id_t*>(data);
+      return extends;
+    };
+    ARROW_ASSIGN_OR_RAISE(grouper_, Grouper::Make(key_types_, ctx_));  // TODO: reset it
+    ARROW_ASSIGN_OR_RAISE(auto datum, grouper_->Consume(batch, offset));
+    if (datum.is_array()) {
+      const std::shared_ptr<ArrayData>& data = datum.array();
+      ARROW_DCHECK(data->GetNullCount() == 0);
+      DCHECK_EQ(data->type->id(), GroupIdType::type_id);
+      const group_id_t* values = data->GetValues<group_id_t>(1);
+      int64_t cursor;
+      for (cursor = 1; cursor < data->length; cursor++) {
+        if (values[0] != values[cursor]) break;

Review Comment:
   See [this post](https://github.com/apache/arrow/pull/34311#discussion_r1117202937).



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rtpsw commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "rtpsw (via GitHub)" <gi...@apache.org>.
rtpsw commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1117883531


##########
cpp/src/arrow/compute/exec.h:
##########
@@ -240,6 +246,8 @@ struct ARROW_EXPORT ExecBatch {
 
   ExecBatch Slice(int64_t offset, int64_t length) const;
 
+  Result<ExecBatch> SelectValues(const std::vector<int>& ids) const;

Review Comment:
   Regular aggregation works differently - it uses encoders that compute a hash for the selected keys. The segmenter doesn't use encoders.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] icexelloss commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "icexelloss (via GitHub)" <gi...@apache.org>.
icexelloss commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1123686937


##########
cpp/src/arrow/compute/exec/options.h:
##########
@@ -199,21 +199,32 @@ class ARROW_EXPORT ProjectNodeOptions : public ExecNodeOptions {
   std::vector<std::string> names;
 };
 
-/// \brief Make a node which aggregates input batches, optionally grouped by keys.
+/// \brief Make a node which aggregates input batches, optionally grouped by keys and
+/// optionally segmented by segment-keys. Both keys and segment-keys determine the group.
+/// However segment-keys are also used for determining grouping segments, which should be
+/// large, and allow streaming a partial aggregation result after processing each segment.
+/// One common use-case for segment-keys is ordered aggregation, in which the segment-key
+/// attribute specifies a column with non-decreasing values or a lexigographically-ordered
+/// set of such columns.
 ///
 /// If the keys attribute is a non-empty vector, then each aggregate in `aggregates` is
 /// expected to be a HashAggregate function. If the keys attribute is an empty vector,
 /// then each aggregate is assumed to be a ScalarAggregate function.
 class ARROW_EXPORT AggregateNodeOptions : public ExecNodeOptions {
  public:
   explicit AggregateNodeOptions(std::vector<Aggregate> aggregates,
-                                std::vector<FieldRef> keys = {})
-      : aggregates(std::move(aggregates)), keys(std::move(keys)) {}
+                                std::vector<FieldRef> keys = {},
+                                std::vector<FieldRef> segment_keys = {})
+      : aggregates(std::move(aggregates)),
+        keys(std::move(keys)),
+        segment_keys(std::move(segment_keys)) {}
 
   // aggregations which will be applied to the targetted fields
   std::vector<Aggregate> aggregates;
   // keys by which aggregations will be grouped
   std::vector<FieldRef> keys;
+  // keys by which aggregations will be segmented
+  std::vector<FieldRef> segment_keys;

Review Comment:
   Can we document this in AggregateNodeOptions and make it super clear?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rtpsw commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "rtpsw (via GitHub)" <gi...@apache.org>.
rtpsw commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1125561702


##########
cpp/src/arrow/compute/exec/aggregate_node.cc:
##########
@@ -169,35 +186,117 @@ void AggregatesToString(std::stringstream* ss, const Schema& input_schema,
   *ss << ']';
 }
 
+template <typename BatchHandler>
+Status HandleSegments(std::unique_ptr<GroupingSegmenter>& segmenter,
+                      const ExecBatch& batch, const std::vector<int>& ids,
+                      const BatchHandler& handle_batch) {
+  int64_t offset = 0;
+  ARROW_ASSIGN_OR_RAISE(auto segment_exec_batch, batch.SelectValues(ids));
+  ExecSpan segment_batch(segment_exec_batch);
+  while (true) {
+    ARROW_ASSIGN_OR_RAISE(auto segment, segmenter->GetNextSegment(segment_batch, offset));
+    if (segment.offset >= segment_batch.length) break;  // condition of no-next-segment
+    ARROW_RETURN_NOT_OK(handle_batch(batch, segment));
+    offset = segment.offset + segment.length;
+  }
+  return Status::OK();
+}
+
+Status GetScalarFields(std::vector<Datum>* values_ptr, const ExecBatch& input_batch,
+                       const std::vector<int>& field_ids) {
+  DCHECK_GT(input_batch.length, 0);
+  std::vector<Datum>& values = *values_ptr;
+  int64_t row = input_batch.length - 1;
+  values.clear();
+  values.resize(field_ids.size());
+  for (size_t i = 0; i < field_ids.size(); i++) {
+    const Datum& value = input_batch.values[field_ids[i]];
+    if (value.is_scalar()) {
+      values[i] = value;
+    } else if (value.is_array()) {
+      ARROW_ASSIGN_OR_RAISE(auto scalar, value.make_array()->GetScalar(row));
+      values[i] = scalar;
+    } else {
+      DCHECK(false);
+    }
+  }
+  return Status::OK();
+}
+
+void PlaceFields(ExecBatch& batch, size_t base, std::vector<Datum>& values) {
+  DCHECK_LE(base + values.size(), batch.values.size());
+  for (size_t i = 0; i < values.size(); i++) {
+    batch.values[base + i] = values[i];
+  }
+}
+
 class ScalarAggregateNode : public ExecNode, public TracedNode {
  public:
   ScalarAggregateNode(ExecPlan* plan, std::vector<ExecNode*> inputs,
                       std::shared_ptr<Schema> output_schema,
+                      std::unique_ptr<GroupingSegmenter> segmenter,
+                      std::vector<int> segment_field_ids,
                       std::vector<std::vector<int>> target_fieldsets,
                       std::vector<Aggregate> aggs,
                       std::vector<const ScalarAggregateKernel*> kernels,
                       std::vector<std::vector<std::unique_ptr<KernelState>>> states)
       : ExecNode(plan, std::move(inputs), {"target"},
                  /*output_schema=*/std::move(output_schema)),
         TracedNode(this),
+        segmenter_(std::move(segmenter)),
+        segment_field_ids_(std::move(segment_field_ids)),
         target_fieldsets_(std::move(target_fieldsets)),
         aggs_(std::move(aggs)),
         kernels_(std::move(kernels)),
-        states_(std::move(states)) {}
+        states_(std::move(states)) {
+    const auto& input_schema = *this->inputs()[0]->output_schema();
+    for (size_t i = 0; i < kernels_.size(); ++i) {
+      std::vector<TypeHolder> in_types;
+      for (const auto& target : target_fieldsets_[i]) {
+        in_types.emplace_back(input_schema.field(target)->type().get());
+      }
+      in_typesets_.push_back(std::move(in_types));
+    }
+  }
 
   static Result<ExecNode*> Make(ExecPlan* plan, std::vector<ExecNode*> inputs,
                                 const ExecNodeOptions& options) {
     RETURN_NOT_OK(ValidateExecNodeInputs(plan, inputs, 1, "ScalarAggregateNode"));
 
     const auto& aggregate_options = checked_cast<const AggregateNodeOptions&>(options);
     auto aggregates = aggregate_options.aggregates;
+    const auto& keys = aggregate_options.keys;
+    const auto& segment_keys = aggregate_options.segment_keys;
+
+    if (keys.size() > 0) {
+      return Status::Invalid("Scalar aggregation with some key");
+    }
+    if (plan->query_context()->exec_context()->executor()->GetCapacity() > 1 &&
+        segment_keys.size() > 0) {
+      return Status::NotImplemented("Segmented aggregation in a multi-threaded plan");
+    }
 
     const auto& input_schema = *inputs[0]->output_schema();
     auto exec_ctx = plan->query_context()->exec_context();
 
+    std::vector<int> segment_field_ids(segment_keys.size());
+    std::vector<TypeHolder> segment_key_types(segment_keys.size());
+    for (size_t i = 0; i < segment_keys.size(); i++) {
+      ARROW_ASSIGN_OR_RAISE(auto match, segment_keys[i].FindOne(input_schema));
+      if (match.indices().size() > 1) {
+        // ARROW-18369: Support nested references as segment ids
+        return Status::Invalid("Nested references cannot be used as segment ids");
+      }
+      segment_field_ids[i] = match[0];
+      segment_key_types[i] = input_schema.field(match[0])->type().get();
+    }
+
+    ARROW_ASSIGN_OR_RAISE(
+        auto segmenter, GroupingSegmenter::Make(std::move(segment_key_types), exec_ctx));

Review Comment:
   The code is outdated now, so I may be missing the context, but note that the segmenter uses a grouper that validates using `GetNullCount`.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rtpsw commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "rtpsw (via GitHub)" <gi...@apache.org>.
rtpsw commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1125563915


##########
cpp/src/arrow/compute/exec/aggregate_node.cc:
##########
@@ -697,13 +909,18 @@ class GroupByNode : public ExecNode, public TracedNode {
   }
 
   int output_task_group_id_;
+  std::unique_ptr<GroupingSegmenter> segmenter_;

Review Comment:
   Done.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] icexelloss commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "icexelloss (via GitHub)" <gi...@apache.org>.
icexelloss commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1123712492


##########
cpp/src/arrow/compute/exec/aggregate_node.cc:
##########
@@ -584,29 +775,50 @@ class GroupByNode : public ExecNode, public TracedNode {
     ARROW_ASSIGN_OR_RAISE(out_data_, Finalize());
 
     int64_t num_output_batches = bit_util::CeilDiv(out_data_.length, output_batch_size());
-    RETURN_NOT_OK(output_->InputFinished(this, static_cast<int>(num_output_batches)));
-    return plan_->query_context()->StartTaskGroup(output_task_group_id_,
-                                                  num_output_batches);
+    total_output_batches_ += static_cast<int>(num_output_batches);
+    if (is_last) {
+      ARROW_RETURN_NOT_OK(output_->InputFinished(this, total_output_batches_));
+      RETURN_NOT_OK(plan_->query_context()->StartTaskGroup(output_task_group_id_,
+                                                           num_output_batches));
+    } else {
+      for (int64_t i = 0; i < num_output_batches; i++) {
+        ARROW_RETURN_NOT_OK(OutputNthBatch(i));
+      }
+      ARROW_RETURN_NOT_OK(ResetAggregates());
+    }
+    return Status::OK();
   }
 
   Status InputReceived(ExecNode* input, ExecBatch batch) override {
     auto scope = TraceInputReceived(batch);
 
     DCHECK_EQ(input, inputs_[0]);
 
-    ARROW_RETURN_NOT_OK(Consume(ExecSpan(batch)));
+    auto handler = [this](const ExecBatch& full_batch, const GroupingSegment& segment) {
+      if (!segment.extends && segment.offset == 0) RETURN_NOT_OK(OutputResult());
+      auto exec_batch = full_batch.Slice(segment.offset, segment.length);
+      auto batch = ExecSpan(exec_batch);
+      RETURN_NOT_OK(Consume(batch));
+      RETURN_NOT_OK(
+          GetScalarFields(&segmenter_values_, exec_batch, segment_key_field_ids_));

Review Comment:
   The method name is misnomer. Looks like this method "updates the value of segmenter_values_ to the values of next segment", if that is correct, this method should probably be called sth like 
   ```
   UpdateSegmentValues(exec_batch)
   ```
   Also I don't think this method need to take `segmenter_values_` and `segment_key_field_ids_` because those are member variables that you can already access from the "this". (Similar to the "Consume" method)



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] icexelloss commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "icexelloss (via GitHub)" <gi...@apache.org>.
icexelloss commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1124624680


##########
cpp/src/arrow/compute/exec/aggregate_node.cc:
##########
@@ -440,12 +613,29 @@ class GroupByNode : public ExecNode, public TracedNode {
       int key_field_id = key_field_ids[i];
       output_fields[base + i] = input_schema->field(key_field_id);
     }
+    base += keys.size();
+    for (size_t i = 0; i < segment_keys.size(); ++i) {
+      int segment_key_field_id = segment_key_field_ids[i];
+      output_fields[base + i] = input_schema->field(segment_key_field_id);
+    }
 
     return input->plan()->EmplaceNode<GroupByNode>(
         input, schema(std::move(output_fields)), std::move(key_field_ids),
+        std::move(segment_key_field_ids), std::move(segmenter), std::move(agg_src_types),
         std::move(agg_src_fieldsets), std::move(aggs), std::move(agg_kernels));
   }
 
+  Status ResetAggregates() {

Review Comment:
   This is very confusing - why are there two "ResetAggregates" method defined and do different things?



##########
cpp/src/arrow/compute/exec/aggregate_node.cc:
##########
@@ -440,12 +613,29 @@ class GroupByNode : public ExecNode, public TracedNode {
       int key_field_id = key_field_ids[i];
       output_fields[base + i] = input_schema->field(key_field_id);
     }
+    base += keys.size();
+    for (size_t i = 0; i < segment_keys.size(); ++i) {
+      int segment_key_field_id = segment_key_field_ids[i];
+      output_fields[base + i] = input_schema->field(segment_key_field_id);
+    }
 
     return input->plan()->EmplaceNode<GroupByNode>(
         input, schema(std::move(output_fields)), std::move(key_field_ids),
+        std::move(segment_key_field_ids), std::move(segmenter), std::move(agg_src_types),
         std::move(agg_src_fieldsets), std::move(aggs), std::move(agg_kernels));
   }
 
+  Status ResetAggregates() {

Review Comment:
   This is very confusing - why are there two "ResetAggregates" method defined?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] icexelloss commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "icexelloss (via GitHub)" <gi...@apache.org>.
icexelloss commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1125574584


##########
cpp/src/arrow/compute/exec/aggregate_node.cc:
##########
@@ -584,29 +775,50 @@ class GroupByNode : public ExecNode, public TracedNode {
     ARROW_ASSIGN_OR_RAISE(out_data_, Finalize());
 
     int64_t num_output_batches = bit_util::CeilDiv(out_data_.length, output_batch_size());
-    RETURN_NOT_OK(output_->InputFinished(this, static_cast<int>(num_output_batches)));
-    return plan_->query_context()->StartTaskGroup(output_task_group_id_,
-                                                  num_output_batches);
+    total_output_batches_ += static_cast<int>(num_output_batches);
+    if (is_last) {
+      ARROW_RETURN_NOT_OK(output_->InputFinished(this, total_output_batches_));
+      RETURN_NOT_OK(plan_->query_context()->StartTaskGroup(output_task_group_id_,
+                                                           num_output_batches));
+    } else {
+      for (int64_t i = 0; i < num_output_batches; i++) {
+        ARROW_RETURN_NOT_OK(OutputNthBatch(i));
+      }
+      ARROW_RETURN_NOT_OK(ResetAggregates());
+    }
+    return Status::OK();
   }
 
   Status InputReceived(ExecNode* input, ExecBatch batch) override {
     auto scope = TraceInputReceived(batch);
 
     DCHECK_EQ(input, inputs_[0]);
 
-    ARROW_RETURN_NOT_OK(Consume(ExecSpan(batch)));
+    auto handler = [this](const ExecBatch& full_batch, const GroupingSegment& segment) {
+      if (!segment.extends && segment.offset == 0) RETURN_NOT_OK(OutputResult());
+      auto exec_batch = full_batch.Slice(segment.offset, segment.length);
+      auto batch = ExecSpan(exec_batch);
+      RETURN_NOT_OK(Consume(batch));
+      RETURN_NOT_OK(
+          GetScalarFields(&segmenter_values_, exec_batch, segment_key_field_ids_));

Review Comment:
   This methods is only used by segment aggregation, no?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] icexelloss commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "icexelloss (via GitHub)" <gi...@apache.org>.
icexelloss commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1125574091


##########
cpp/src/arrow/compute/exec/aggregate_node.cc:
##########
@@ -169,35 +185,117 @@ void AggregatesToString(std::stringstream* ss, const Schema& input_schema,
   *ss << ']';
 }
 
+template <typename BatchHandler>
+Status HandleSegments(std::unique_ptr<GroupingSegmenter>& segmenter,
+                      const ExecBatch& batch, const std::vector<int>& ids,
+                      const BatchHandler& handle_batch) {
+  int64_t offset = 0;
+  ARROW_ASSIGN_OR_RAISE(auto segment_exec_batch, batch.SelectValues(ids));
+  ExecSpan segment_batch(segment_exec_batch);
+  while (true) {
+    ARROW_ASSIGN_OR_RAISE(auto segment, segmenter->GetNextSegment(segment_batch, offset));
+    if (segment.offset >= segment_batch.length) break;  // condition of no-next-segment
+    ARROW_RETURN_NOT_OK(handle_batch(batch, segment));
+    offset = segment.offset + segment.length;
+  }
+  return Status::OK();
+}
+
+Status GetScalarFields(std::vector<Datum>* values_ptr, const ExecBatch& input_batch,
+                       const std::vector<int>& field_ids) {
+  DCHECK_GT(input_batch.length, 0);
+  std::vector<Datum>& values = *values_ptr;
+  int64_t row = input_batch.length - 1;
+  values.clear();
+  values.resize(field_ids.size());
+  for (size_t i = 0; i < field_ids.size(); i++) {
+    const Datum& value = input_batch.values[field_ids[i]];
+    if (value.is_scalar()) {

Review Comment:
   Hmm.. does the input_batch not have an universal interface to at the value of row i regardless of whether it is a constant or not?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rtpsw commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "rtpsw (via GitHub)" <gi...@apache.org>.
rtpsw commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1125605663


##########
cpp/src/arrow/compute/kernels/hash_aggregate_test.cc:
##########
@@ -301,53 +426,143 @@ Result<Datum> GroupByTest(const std::vector<Datum>& arguments,
         {t_agg.function, t_agg.options, "agg_" + ToChars(idx), t_agg.function});
     idx = idx + 1;
   }
-  return RunGroupBy(arguments, keys, internal_aggregates, use_threads);
+  return group_by(arguments, keys, segment_keys, internal_aggregates, use_threads,
+                  /*naive=*/false);
 }
 
-}  // namespace
+Result<Datum> GroupByTest(GroupByFunction group_by, const std::vector<Datum>& arguments,
+                          const std::vector<Datum>& keys,
+                          const std::vector<TestAggregate>& aggregates,
+                          bool use_threads) {
+  return GroupByTest(group_by, arguments, keys, {}, aggregates, use_threads);
+}
 
-TEST(Grouper, SupportedKeys) {
-  ASSERT_OK(Grouper::Make({boolean()}));
+template <typename GroupClass>
+void TestGroupClassSupportedKeys() {
+  ASSERT_OK(GroupClass::Make({boolean()}));
 
-  ASSERT_OK(Grouper::Make({int8(), uint16(), int32(), uint64()}));
+  ASSERT_OK(GroupClass::Make({int8(), uint16(), int32(), uint64()}));
 
-  ASSERT_OK(Grouper::Make({dictionary(int64(), utf8())}));
+  ASSERT_OK(GroupClass::Make({dictionary(int64(), utf8())}));
 
-  ASSERT_OK(Grouper::Make({float16(), float32(), float64()}));
+  ASSERT_OK(GroupClass::Make({float16(), float32(), float64()}));
 
-  ASSERT_OK(Grouper::Make({utf8(), binary(), large_utf8(), large_binary()}));
+  ASSERT_OK(GroupClass::Make({utf8(), binary(), large_utf8(), large_binary()}));
 
-  ASSERT_OK(Grouper::Make({fixed_size_binary(16), fixed_size_binary(32)}));
+  ASSERT_OK(GroupClass::Make({fixed_size_binary(16), fixed_size_binary(32)}));
 
-  ASSERT_OK(Grouper::Make({decimal128(32, 10), decimal256(76, 20)}));
+  ASSERT_OK(GroupClass::Make({decimal128(32, 10), decimal256(76, 20)}));
 
-  ASSERT_OK(Grouper::Make({date32(), date64()}));
+  ASSERT_OK(GroupClass::Make({date32(), date64()}));
 
   for (auto unit : {
            TimeUnit::SECOND,
            TimeUnit::MILLI,
            TimeUnit::MICRO,
            TimeUnit::NANO,
        }) {
-    ASSERT_OK(Grouper::Make({timestamp(unit), duration(unit)}));
+    ASSERT_OK(GroupClass::Make({timestamp(unit), duration(unit)}));
   }
 
-  ASSERT_OK(
-      Grouper::Make({day_time_interval(), month_interval(), month_day_nano_interval()}));
+  ASSERT_OK(GroupClass::Make(
+      {day_time_interval(), month_interval(), month_day_nano_interval()}));
+
+  ASSERT_OK(GroupClass::Make({null()}));
+
+  ASSERT_RAISES(NotImplemented, GroupClass::Make({struct_({field("", int64())})}));
+
+  ASSERT_RAISES(NotImplemented, GroupClass::Make({struct_({})}));
 
-  ASSERT_OK(Grouper::Make({null()}));
+  ASSERT_RAISES(NotImplemented, GroupClass::Make({list(int32())}));
 
-  ASSERT_RAISES(NotImplemented, Grouper::Make({struct_({field("", int64())})}));
+  ASSERT_RAISES(NotImplemented, GroupClass::Make({fixed_size_list(int32(), 5)}));
 
-  ASSERT_RAISES(NotImplemented, Grouper::Make({struct_({})}));
+  ASSERT_RAISES(NotImplemented, GroupClass::Make({dense_union({field("", int32())})}));
+}
 
-  ASSERT_RAISES(NotImplemented, Grouper::Make({list(int32())}));
+void TestSegments(std::unique_ptr<RowSegmenter>& segmenter, const ExecSpan& batch,

Review Comment:
   There are [plenty of (pre-PR) existing `Grouper` tests](https://github.com/apache/arrow/blob/237705bf17486cfc35ab7d1ddfe59dd60f042ab8/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc#L309-L784) in `hash_aggregate_test.cc`. I think such refactoring, if needed, should not be part of this PR.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rtpsw commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "rtpsw (via GitHub)" <gi...@apache.org>.
rtpsw commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1125649916


##########
cpp/src/arrow/compute/exec/aggregate_node.cc:
##########
@@ -440,12 +613,29 @@ class GroupByNode : public ExecNode, public TracedNode {
       int key_field_id = key_field_ids[i];
       output_fields[base + i] = input_schema->field(key_field_id);
     }
+    base += keys.size();
+    for (size_t i = 0; i < segment_keys.size(); ++i) {
+      int segment_key_field_id = segment_key_field_ids[i];
+      output_fields[base + i] = input_schema->field(segment_key_field_id);
+    }
 
     return input->plan()->EmplaceNode<GroupByNode>(
         input, schema(std::move(output_fields)), std::move(key_field_ids),
+        std::move(segment_key_field_ids), std::move(segmenter), std::move(agg_src_types),
         std::move(agg_src_fieldsets), std::move(aggs), std::move(agg_kernels));
   }
 
+  Status ResetAggregates() {

Review Comment:
   `ScalarAggregateNode` can be seen as a no-keys optimized version of `GroupByNode`. This PR adds support for segment-keys to both these nodes. Note that pre-PR both nodes outputted a single batch, where `ScalarAggregateNode`'s had a single row and `GroupByNode`'s had one row per group, whereas post-PR both output multiple batches of the same structure.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rtpsw commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "rtpsw (via GitHub)" <gi...@apache.org>.
rtpsw commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1125556833


##########
cpp/src/arrow/compute/exec/aggregate_node.cc:
##########
@@ -584,29 +775,50 @@ class GroupByNode : public ExecNode, public TracedNode {
     ARROW_ASSIGN_OR_RAISE(out_data_, Finalize());
 
     int64_t num_output_batches = bit_util::CeilDiv(out_data_.length, output_batch_size());
-    RETURN_NOT_OK(output_->InputFinished(this, static_cast<int>(num_output_batches)));
-    return plan_->query_context()->StartTaskGroup(output_task_group_id_,
-                                                  num_output_batches);
+    total_output_batches_ += static_cast<int>(num_output_batches);
+    if (is_last) {
+      ARROW_RETURN_NOT_OK(output_->InputFinished(this, total_output_batches_));
+      RETURN_NOT_OK(plan_->query_context()->StartTaskGroup(output_task_group_id_,
+                                                           num_output_batches));
+    } else {
+      for (int64_t i = 0; i < num_output_batches; i++) {
+        ARROW_RETURN_NOT_OK(OutputNthBatch(i));
+      }
+      ARROW_RETURN_NOT_OK(ResetAggregates());
+    }
+    return Status::OK();
   }
 
   Status InputReceived(ExecNode* input, ExecBatch batch) override {
     auto scope = TraceInputReceived(batch);
 
     DCHECK_EQ(input, inputs_[0]);
 
-    ARROW_RETURN_NOT_OK(Consume(ExecSpan(batch)));
+    auto handler = [this](const ExecBatch& full_batch, const GroupingSegment& segment) {
+      if (!segment.extends && segment.offset == 0) RETURN_NOT_OK(OutputResult());
+      auto exec_batch = full_batch.Slice(segment.offset, segment.length);
+      auto batch = ExecSpan(exec_batch);
+      RETURN_NOT_OK(Consume(batch));
+      RETURN_NOT_OK(
+          GetScalarFields(&segmenter_values_, exec_batch, segment_key_field_ids_));

Review Comment:
   This is a global function, now a method, so it doesn't have access to `this`. Reconsidering, I think a good name for it is `SelectConstantFields` because it output a subset of fields, each being constant-valued. I don't think the name should include `Segment` in it, because the function knows nothing about segmenting.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] icexelloss commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "icexelloss (via GitHub)" <gi...@apache.org>.
icexelloss commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1127033937


##########
cpp/src/arrow/compute/row/grouper.cc:
##########
@@ -39,12 +43,521 @@
 namespace arrow {
 
 using internal::checked_cast;
+using internal::PrimitiveScalarBase;
 
 namespace compute {
 
 namespace {
 
-struct GrouperImpl : Grouper {
+constexpr uint32_t kNoGroupId = std::numeric_limits<uint32_t>::max();
+
+using group_id_t = std::remove_const<decltype(kNoGroupId)>::type;
+using GroupIdType = CTypeTraits<group_id_t>::ArrowType;
+auto group_id_type = std::make_shared<GroupIdType>();
+
+inline const uint8_t* GetValuesAsBytes(const ArrayData& data, int64_t offset = 0) {
+  DCHECK_GT(data.type->byte_width(), 0);
+  int64_t absolute_byte_offset = (data.offset + offset) * data.type->byte_width();
+  return data.GetValues<uint8_t>(1, absolute_byte_offset);
+}
+
+inline const uint8_t* GetValuesAsBytes(const ArraySpan& data, int64_t offset = 0) {
+  DCHECK_GT(data.type->byte_width(), 0);
+  int64_t absolute_byte_offset = (data.offset + offset) * data.type->byte_width();
+  return data.GetValues<uint8_t>(1, absolute_byte_offset);
+}
+
+template <typename Value>
+Status CheckForGetNextSegment(const std::vector<Value>& values, int64_t length,
+                              int64_t offset, const std::vector<TypeHolder>& key_types) {
+  if (offset < 0 || offset > length) {
+    return Status::Invalid("invalid grouping segmenter offset: ", offset);
+  }
+  if (values.size() != key_types.size()) {
+    return Status::Invalid("expected batch size ", key_types.size(), " but got ",
+                           values.size());
+  }
+  for (size_t i = 0; i < key_types.size(); i++) {
+    const auto& value = values[i];
+    const auto& key_type = key_types[i];
+    if (*value.type() != *key_type.type) {
+      return Status::Invalid("expected batch value ", i, " of type ", *key_type.type,
+                             " but got ", *value.type());
+    }
+  }
+  return Status::OK();
+}
+
+template <typename Batch>
+enable_if_t<std::is_same<Batch, ExecSpan>::value || std::is_same<Batch, ExecBatch>::value,
+            Status>
+CheckForGetNextSegment(const Batch& batch, int64_t offset,
+                       const std::vector<TypeHolder>& key_types) {
+  return CheckForGetNextSegment(batch.values, batch.length, offset, key_types);
+}
+
+struct BaseGroupingSegmenter : public GroupingSegmenter {
+  explicit BaseGroupingSegmenter(const std::vector<TypeHolder>& key_types)
+      : key_types_(key_types) {}
+
+  const std::vector<TypeHolder>& key_types() const override { return key_types_; }
+
+  std::vector<TypeHolder> key_types_;
+};
+
+GroupingSegment MakeSegment(int64_t batch_length, int64_t offset, int64_t length,
+                            bool extends) {
+  return GroupingSegment{offset, length, offset + length >= batch_length, extends};
+}
+
+int64_t GetMatchLength(const uint8_t* match_bytes, int64_t match_width,

Review Comment:
   TBH I don't care about `A, A, B, B, A, A` use case at all so not real reason to support it 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rtpsw commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "rtpsw (via GitHub)" <gi...@apache.org>.
rtpsw commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1128180120


##########
cpp/src/arrow/compute/exec.h:
##########
@@ -240,6 +246,8 @@ struct ARROW_EXPORT ExecBatch {
 
   ExecBatch Slice(int64_t offset, int64_t length) const;
 
+  Result<ExecBatch> SelectValues(const std::vector<int>& ids) const;

Review Comment:
   #34475



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] icexelloss commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "icexelloss (via GitHub)" <gi...@apache.org>.
icexelloss commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1126966232


##########
cpp/src/arrow/compute/row/grouper.cc:
##########
@@ -39,12 +43,330 @@
 namespace arrow {
 
 using internal::checked_cast;
+using internal::PrimitiveScalarBase;
 
 namespace compute {
 
 namespace {
 
-struct GrouperImpl : Grouper {
+constexpr uint32_t kNoGroupId = std::numeric_limits<uint32_t>::max();
+
+using group_id_t = std::remove_const<decltype(kNoGroupId)>::type;
+using GroupIdType = CTypeTraits<group_id_t>::ArrowType;
+auto g_group_id_type = std::make_shared<GroupIdType>();
+
+inline const uint8_t* GetValuesAsBytes(const ArraySpan& data, int64_t offset = 0) {
+  DCHECK_GT(data.type->byte_width(), 0);
+  int64_t absolute_byte_offset = (data.offset + offset) * data.type->byte_width();
+  return data.GetValues<uint8_t>(1, absolute_byte_offset);
+}
+
+template <typename Value>
+Status CheckForGetNextSegment(const std::vector<Value>& values, int64_t length,
+                              int64_t offset, const std::vector<TypeHolder>& key_types) {
+  if (offset < 0 || offset > length) {
+    return Status::Invalid("invalid grouping segmenter offset: ", offset);
+  }
+  if (values.size() != key_types.size()) {
+    return Status::Invalid("expected batch size ", key_types.size(), " but got ",
+                           values.size());
+  }
+  for (size_t i = 0; i < key_types.size(); i++) {
+    const auto& value = values[i];
+    const auto& key_type = key_types[i];
+    if (*value.type() != *key_type.type) {
+      return Status::Invalid("expected batch value ", i, " of type ", *key_type.type,
+                             " but got ", *value.type());
+    }
+  }
+  return Status::OK();
+}
+
+template <typename Batch>
+enable_if_t<std::is_same<Batch, ExecSpan>::value || std::is_same<Batch, ExecBatch>::value,
+            Status>
+CheckForGetNextSegment(const Batch& batch, int64_t offset,
+                       const std::vector<TypeHolder>& key_types) {
+  return CheckForGetNextSegment(batch.values, batch.length, offset, key_types);
+}
+
+struct BaseRowSegmenter : public RowSegmenter {
+  explicit BaseRowSegmenter(const std::vector<TypeHolder>& key_types)
+      : key_types_(key_types) {}
+
+  const std::vector<TypeHolder>& key_types() const override { return key_types_; }
+
+  std::vector<TypeHolder> key_types_;
+};
+
+Segment MakeSegment(int64_t batch_length, int64_t offset, int64_t length, bool extends) {
+  return Segment{offset, length, offset + length >= batch_length, extends};
+}
+
+int64_t GetMatchLength(const uint8_t* match_bytes, int64_t match_width,
+                       const uint8_t* array_bytes, int64_t offset, int64_t length) {
+  int64_t cursor, byte_cursor;
+  for (cursor = offset, byte_cursor = match_width * cursor; cursor < length;
+       cursor++, byte_cursor += match_width) {
+    if (memcmp(match_bytes, array_bytes + byte_cursor,
+               static_cast<size_t>(match_width)) != 0) {
+      break;
+    }
+  }
+  return std::min(cursor, length) - offset;
+}
+
+using ExtendFunc = std::function<bool(const void*)>;
+constexpr bool kDefaultExtends = true;
+constexpr bool kEmptyExtends = true;
+
+struct NoKeysSegmenter : public BaseRowSegmenter {
+  static std::unique_ptr<RowSegmenter> Make() {
+    return std::make_unique<NoKeysSegmenter>();
+  }
+
+  NoKeysSegmenter() : BaseRowSegmenter({}) {}
+
+  Status Reset() override { return Status::OK(); }
+
+  Result<Segment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {}));
+    return MakeSegment(batch.length, offset, batch.length - offset, kDefaultExtends);
+  }
+};
+
+struct SimpleKeySegmenter : public BaseRowSegmenter {
+  static Result<std::unique_ptr<RowSegmenter>> Make(TypeHolder key_type) {
+    return std::make_unique<SimpleKeySegmenter>(key_type);
+  }
+
+  explicit SimpleKeySegmenter(TypeHolder key_type)
+      : BaseRowSegmenter({key_type}), key_type_(key_types_[0]), save_key_data_() {}
+
+  Status CheckType(const DataType& type) {
+    if (!is_fixed_width(type)) {
+      return Status::Invalid("SimpleKeySegmenter does not support type ", type);
+    }
+    return Status::OK();
+  }
+
+  Status Reset() override {
+    save_key_data_.resize(0);
+    return Status::OK();
+  }
+
+  // Checks whether the given grouping data extends the current segment, i.e., is equal to
+  // previously seen grouping data, which is updated with each invocation.
+  bool Extend(const void* data) {
+    size_t byte_width = static_cast<size_t>(key_type_.type->byte_width());
+    bool extends = save_key_data_.size() != byte_width
+                       ? kDefaultExtends
+                       : 0 == memcmp(save_key_data_.data(), data, byte_width);
+    save_key_data_.resize(byte_width);
+    memcpy(save_key_data_.data(), data, byte_width);
+    return extends;
+  }
+
+  Result<Segment> GetNextSegment(const Scalar& scalar, int64_t offset, int64_t length) {
+    ARROW_RETURN_NOT_OK(CheckType(*scalar.type));
+    if (!scalar.is_valid) {
+      return Status::Invalid("segmenting an invalid scalar");
+    }
+    auto data = checked_cast<const PrimitiveScalarBase&>(scalar).data();
+    bool extends = length > 0 ? Extend(data) : kEmptyExtends;
+    return MakeSegment(length, offset, length, extends);
+  }
+
+  Result<Segment> GetNextSegment(const DataType& array_type, const uint8_t* array_bytes,
+                                 int64_t offset, int64_t length) {
+    RETURN_NOT_OK(CheckType(array_type));
+    int64_t byte_width = array_type.byte_width();
+    int64_t match_length = GetMatchLength(array_bytes + offset * byte_width, byte_width,
+                                          array_bytes, offset, length);
+    bool extends = length > 0 ? Extend(array_bytes + offset * byte_width) : kEmptyExtends;
+    return MakeSegment(length, offset, match_length, extends);
+  }
+
+  Result<Segment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {key_type_}));
+    if (offset == batch.length) {
+      return MakeSegment(batch.length, offset, 0, kEmptyExtends);
+    }
+    const auto& value = batch.values[0];
+    if (value.is_scalar()) {
+      return GetNextSegment(*value.scalar, offset, batch.length);
+    }
+    ARROW_DCHECK(value.is_array());
+    const auto& array = value.array;
+    if (array.GetNullCount() > 0) {
+      return Status::NotImplemented("segmenting a nullable array");
+    }
+    return GetNextSegment(*array.type, GetValuesAsBytes(array), offset, batch.length);
+  }
+
+ private:
+  TypeHolder key_type_;
+  std::vector<uint8_t> save_key_data_;
+};
+
+struct AnyKeysSegmenter : public BaseRowSegmenter {
+  static Result<std::unique_ptr<RowSegmenter>> Make(
+      const std::vector<TypeHolder>& key_types, ExecContext* ctx) {
+    ARROW_RETURN_NOT_OK(Grouper::Make(key_types, ctx));  // check types
+    return std::make_unique<AnyKeysSegmenter>(key_types, ctx);
+  }
+
+  AnyKeysSegmenter(const std::vector<TypeHolder>& key_types, ExecContext* ctx)
+      : BaseRowSegmenter(key_types),
+        ctx_(ctx),
+        grouper_(nullptr),
+        save_group_id_(kNoGroupId) {}
+
+  Status Reset() override {
+    grouper_ = nullptr;
+    save_group_id_ = kNoGroupId;
+    return Status::OK();
+  }
+
+  bool Extend(const void* data) {
+    auto group_id = *static_cast<const group_id_t*>(data);
+    bool extends =
+        save_group_id_ == kNoGroupId ? kDefaultExtends : save_group_id_ == group_id;
+    save_group_id_ = group_id;
+    return extends;
+  }
+
+  // Runs the grouper on a single row.  This is used to determine the group id of the
+  // first row of a new segment to see if it extends the previous segment.
+  template <typename Batch>
+  Result<group_id_t> MapGroupIdAt(const Batch& batch, int64_t offset) {
+    if (!grouper_) return kNoGroupId;
+    ARROW_ASSIGN_OR_RAISE(auto datum, grouper_->Consume(batch, offset,
+                                                        /*length=*/1));
+    if (!datum.is_array()) {
+      return Status::Invalid("accessing unsupported datum kind ", datum.kind());
+    }
+    const std::shared_ptr<ArrayData>& data = datum.array();
+    ARROW_DCHECK(data->GetNullCount() == 0);
+    DCHECK_EQ(data->type->id(), GroupIdType::type_id);
+    DCHECK_EQ(1, data->length);
+    const group_id_t* values = data->GetValues<group_id_t>(1);
+    return values[0];
+  }
+
+  Result<Segment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, key_types_));
+    if (offset == batch.length) {
+      return MakeSegment(batch.length, offset, 0, kEmptyExtends);
+    }
+    // ARROW-18311: make Grouper support Reset()

Review Comment:
   Is this copy / paste error? This is "Grouper" class



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] icexelloss commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "icexelloss (via GitHub)" <gi...@apache.org>.
icexelloss commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1127020653


##########
cpp/src/arrow/compute/row/grouper.cc:
##########
@@ -39,12 +43,330 @@
 namespace arrow {
 
 using internal::checked_cast;
+using internal::PrimitiveScalarBase;
 
 namespace compute {
 
 namespace {
 
-struct GrouperImpl : Grouper {
+constexpr uint32_t kNoGroupId = std::numeric_limits<uint32_t>::max();
+
+using group_id_t = std::remove_const<decltype(kNoGroupId)>::type;
+using GroupIdType = CTypeTraits<group_id_t>::ArrowType;
+auto g_group_id_type = std::make_shared<GroupIdType>();
+
+inline const uint8_t* GetValuesAsBytes(const ArraySpan& data, int64_t offset = 0) {
+  DCHECK_GT(data.type->byte_width(), 0);
+  int64_t absolute_byte_offset = (data.offset + offset) * data.type->byte_width();
+  return data.GetValues<uint8_t>(1, absolute_byte_offset);
+}
+
+template <typename Value>
+Status CheckForGetNextSegment(const std::vector<Value>& values, int64_t length,
+                              int64_t offset, const std::vector<TypeHolder>& key_types) {
+  if (offset < 0 || offset > length) {
+    return Status::Invalid("invalid grouping segmenter offset: ", offset);
+  }
+  if (values.size() != key_types.size()) {
+    return Status::Invalid("expected batch size ", key_types.size(), " but got ",
+                           values.size());
+  }
+  for (size_t i = 0; i < key_types.size(); i++) {
+    const auto& value = values[i];
+    const auto& key_type = key_types[i];
+    if (*value.type() != *key_type.type) {
+      return Status::Invalid("expected batch value ", i, " of type ", *key_type.type,
+                             " but got ", *value.type());
+    }
+  }
+  return Status::OK();
+}
+
+template <typename Batch>
+enable_if_t<std::is_same<Batch, ExecSpan>::value || std::is_same<Batch, ExecBatch>::value,
+            Status>
+CheckForGetNextSegment(const Batch& batch, int64_t offset,
+                       const std::vector<TypeHolder>& key_types) {
+  return CheckForGetNextSegment(batch.values, batch.length, offset, key_types);
+}
+
+struct BaseRowSegmenter : public RowSegmenter {
+  explicit BaseRowSegmenter(const std::vector<TypeHolder>& key_types)
+      : key_types_(key_types) {}
+
+  const std::vector<TypeHolder>& key_types() const override { return key_types_; }
+
+  std::vector<TypeHolder> key_types_;
+};
+
+Segment MakeSegment(int64_t batch_length, int64_t offset, int64_t length, bool extends) {
+  return Segment{offset, length, offset + length >= batch_length, extends};
+}
+
+int64_t GetMatchLength(const uint8_t* match_bytes, int64_t match_width,
+                       const uint8_t* array_bytes, int64_t offset, int64_t length) {
+  int64_t cursor, byte_cursor;
+  for (cursor = offset, byte_cursor = match_width * cursor; cursor < length;
+       cursor++, byte_cursor += match_width) {
+    if (memcmp(match_bytes, array_bytes + byte_cursor,
+               static_cast<size_t>(match_width)) != 0) {
+      break;
+    }
+  }
+  return std::min(cursor, length) - offset;
+}
+
+using ExtendFunc = std::function<bool(const void*)>;
+constexpr bool kDefaultExtends = true;
+constexpr bool kEmptyExtends = true;
+
+struct NoKeysSegmenter : public BaseRowSegmenter {
+  static std::unique_ptr<RowSegmenter> Make() {
+    return std::make_unique<NoKeysSegmenter>();
+  }
+
+  NoKeysSegmenter() : BaseRowSegmenter({}) {}
+
+  Status Reset() override { return Status::OK(); }
+
+  Result<Segment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {}));
+    return MakeSegment(batch.length, offset, batch.length - offset, kDefaultExtends);
+  }
+};
+
+struct SimpleKeySegmenter : public BaseRowSegmenter {
+  static Result<std::unique_ptr<RowSegmenter>> Make(TypeHolder key_type) {
+    return std::make_unique<SimpleKeySegmenter>(key_type);
+  }
+
+  explicit SimpleKeySegmenter(TypeHolder key_type)
+      : BaseRowSegmenter({key_type}), key_type_(key_types_[0]), save_key_data_() {}
+
+  Status CheckType(const DataType& type) {
+    if (!is_fixed_width(type)) {
+      return Status::Invalid("SimpleKeySegmenter does not support type ", type);
+    }
+    return Status::OK();
+  }
+
+  Status Reset() override {
+    save_key_data_.resize(0);
+    return Status::OK();
+  }
+
+  // Checks whether the given grouping data extends the current segment, i.e., is equal to
+  // previously seen grouping data, which is updated with each invocation.
+  bool Extend(const void* data) {
+    size_t byte_width = static_cast<size_t>(key_type_.type->byte_width());
+    bool extends = save_key_data_.size() != byte_width
+                       ? kDefaultExtends
+                       : 0 == memcmp(save_key_data_.data(), data, byte_width);
+    save_key_data_.resize(byte_width);
+    memcpy(save_key_data_.data(), data, byte_width);
+    return extends;
+  }
+
+  Result<Segment> GetNextSegment(const Scalar& scalar, int64_t offset, int64_t length) {
+    ARROW_RETURN_NOT_OK(CheckType(*scalar.type));
+    if (!scalar.is_valid) {
+      return Status::Invalid("segmenting an invalid scalar");
+    }
+    auto data = checked_cast<const PrimitiveScalarBase&>(scalar).data();
+    bool extends = length > 0 ? Extend(data) : kEmptyExtends;
+    return MakeSegment(length, offset, length, extends);
+  }
+
+  Result<Segment> GetNextSegment(const DataType& array_type, const uint8_t* array_bytes,
+                                 int64_t offset, int64_t length) {
+    RETURN_NOT_OK(CheckType(array_type));
+    int64_t byte_width = array_type.byte_width();
+    int64_t match_length = GetMatchLength(array_bytes + offset * byte_width, byte_width,
+                                          array_bytes, offset, length);
+    bool extends = length > 0 ? Extend(array_bytes + offset * byte_width) : kEmptyExtends;
+    return MakeSegment(length, offset, match_length, extends);
+  }
+
+  Result<Segment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {key_type_}));
+    if (offset == batch.length) {
+      return MakeSegment(batch.length, offset, 0, kEmptyExtends);
+    }
+    const auto& value = batch.values[0];
+    if (value.is_scalar()) {
+      return GetNextSegment(*value.scalar, offset, batch.length);
+    }
+    ARROW_DCHECK(value.is_array());
+    const auto& array = value.array;
+    if (array.GetNullCount() > 0) {
+      return Status::NotImplemented("segmenting a nullable array");
+    }
+    return GetNextSegment(*array.type, GetValuesAsBytes(array), offset, batch.length);
+  }
+
+ private:
+  TypeHolder key_type_;
+  std::vector<uint8_t> save_key_data_;

Review Comment:
   What is this variable and why is this a vector?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] icexelloss commented on pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "icexelloss (via GitHub)" <gi...@apache.org>.
icexelloss commented on PR #34311:
URL: https://github.com/apache/arrow/pull/34311#issuecomment-1458503240

   @rtpsw Do you have tests for the `RowSegmenter` classes to validate offset, length, open, extend in various cases? If so can you point me to it?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rtpsw commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "rtpsw (via GitHub)" <gi...@apache.org>.
rtpsw commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1129268410


##########
cpp/src/arrow/compute/row/grouper.cc:
##########
@@ -39,12 +43,330 @@
 namespace arrow {
 
 using internal::checked_cast;
+using internal::PrimitiveScalarBase;
 
 namespace compute {
 
 namespace {
 
-struct GrouperImpl : Grouper {
+constexpr uint32_t kNoGroupId = std::numeric_limits<uint32_t>::max();
+
+using group_id_t = std::remove_const<decltype(kNoGroupId)>::type;
+using GroupIdType = CTypeTraits<group_id_t>::ArrowType;
+auto g_group_id_type = std::make_shared<GroupIdType>();
+
+inline const uint8_t* GetValuesAsBytes(const ArraySpan& data, int64_t offset = 0) {
+  DCHECK_GT(data.type->byte_width(), 0);
+  int64_t absolute_byte_offset = (data.offset + offset) * data.type->byte_width();
+  return data.GetValues<uint8_t>(1, absolute_byte_offset);
+}
+
+template <typename Value>
+Status CheckForGetNextSegment(const std::vector<Value>& values, int64_t length,
+                              int64_t offset, const std::vector<TypeHolder>& key_types) {
+  if (offset < 0 || offset > length) {
+    return Status::Invalid("invalid grouping segmenter offset: ", offset);
+  }
+  if (values.size() != key_types.size()) {
+    return Status::Invalid("expected batch size ", key_types.size(), " but got ",
+                           values.size());
+  }
+  for (size_t i = 0; i < key_types.size(); i++) {
+    const auto& value = values[i];
+    const auto& key_type = key_types[i];
+    if (*value.type() != *key_type.type) {
+      return Status::Invalid("expected batch value ", i, " of type ", *key_type.type,
+                             " but got ", *value.type());
+    }
+  }
+  return Status::OK();
+}
+
+template <typename Batch>
+enable_if_t<std::is_same<Batch, ExecSpan>::value || std::is_same<Batch, ExecBatch>::value,
+            Status>
+CheckForGetNextSegment(const Batch& batch, int64_t offset,
+                       const std::vector<TypeHolder>& key_types) {
+  return CheckForGetNextSegment(batch.values, batch.length, offset, key_types);
+}
+
+struct BaseRowSegmenter : public RowSegmenter {
+  explicit BaseRowSegmenter(const std::vector<TypeHolder>& key_types)
+      : key_types_(key_types) {}
+
+  const std::vector<TypeHolder>& key_types() const override { return key_types_; }
+
+  std::vector<TypeHolder> key_types_;
+};
+
+Segment MakeSegment(int64_t batch_length, int64_t offset, int64_t length, bool extends) {
+  return Segment{offset, length, offset + length >= batch_length, extends};
+}
+
+int64_t GetMatchLength(const uint8_t* match_bytes, int64_t match_width,
+                       const uint8_t* array_bytes, int64_t offset, int64_t length) {
+  int64_t cursor, byte_cursor;
+  for (cursor = offset, byte_cursor = match_width * cursor; cursor < length;
+       cursor++, byte_cursor += match_width) {
+    if (memcmp(match_bytes, array_bytes + byte_cursor,
+               static_cast<size_t>(match_width)) != 0) {
+      break;
+    }
+  }
+  return std::min(cursor, length) - offset;
+}
+
+using ExtendFunc = std::function<bool(const void*)>;
+constexpr bool kDefaultExtends = true;
+constexpr bool kEmptyExtends = true;
+
+struct NoKeysSegmenter : public BaseRowSegmenter {
+  static std::unique_ptr<RowSegmenter> Make() {
+    return std::make_unique<NoKeysSegmenter>();
+  }
+
+  NoKeysSegmenter() : BaseRowSegmenter({}) {}
+
+  Status Reset() override { return Status::OK(); }
+
+  Result<Segment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {}));
+    return MakeSegment(batch.length, offset, batch.length - offset, kDefaultExtends);
+  }
+};
+
+struct SimpleKeySegmenter : public BaseRowSegmenter {
+  static Result<std::unique_ptr<RowSegmenter>> Make(TypeHolder key_type) {
+    return std::make_unique<SimpleKeySegmenter>(key_type);
+  }
+
+  explicit SimpleKeySegmenter(TypeHolder key_type)
+      : BaseRowSegmenter({key_type}), key_type_(key_types_[0]), save_key_data_() {}
+
+  Status CheckType(const DataType& type) {
+    if (!is_fixed_width(type)) {
+      return Status::Invalid("SimpleKeySegmenter does not support type ", type);
+    }
+    return Status::OK();
+  }
+
+  Status Reset() override {
+    save_key_data_.resize(0);
+    return Status::OK();
+  }
+
+  // Checks whether the given grouping data extends the current segment, i.e., is equal to
+  // previously seen grouping data, which is updated with each invocation.
+  bool Extend(const void* data) {
+    size_t byte_width = static_cast<size_t>(key_type_.type->byte_width());
+    bool extends = save_key_data_.size() != byte_width
+                       ? kDefaultExtends
+                       : 0 == memcmp(save_key_data_.data(), data, byte_width);
+    save_key_data_.resize(byte_width);
+    memcpy(save_key_data_.data(), data, byte_width);
+    return extends;
+  }
+
+  Result<Segment> GetNextSegment(const Scalar& scalar, int64_t offset, int64_t length) {
+    ARROW_RETURN_NOT_OK(CheckType(*scalar.type));
+    if (!scalar.is_valid) {
+      return Status::Invalid("segmenting an invalid scalar");
+    }
+    auto data = checked_cast<const PrimitiveScalarBase&>(scalar).data();
+    bool extends = length > 0 ? Extend(data) : kEmptyExtends;
+    return MakeSegment(length, offset, length, extends);
+  }
+
+  Result<Segment> GetNextSegment(const DataType& array_type, const uint8_t* array_bytes,
+                                 int64_t offset, int64_t length) {
+    RETURN_NOT_OK(CheckType(array_type));
+    int64_t byte_width = array_type.byte_width();
+    int64_t match_length = GetMatchLength(array_bytes + offset * byte_width, byte_width,
+                                          array_bytes, offset, length);
+    bool extends = length > 0 ? Extend(array_bytes + offset * byte_width) : kEmptyExtends;
+    return MakeSegment(length, offset, match_length, extends);
+  }
+
+  Result<Segment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {key_type_}));
+    if (offset == batch.length) {
+      return MakeSegment(batch.length, offset, 0, kEmptyExtends);
+    }
+    const auto& value = batch.values[0];
+    if (value.is_scalar()) {
+      return GetNextSegment(*value.scalar, offset, batch.length);
+    }
+    ARROW_DCHECK(value.is_array());
+    const auto& array = value.array;
+    if (array.GetNullCount() > 0) {
+      return Status::NotImplemented("segmenting a nullable array");
+    }
+    return GetNextSegment(*array.type, GetValuesAsBytes(array), offset, batch.length);
+  }
+
+ private:
+  TypeHolder key_type_;
+  std::vector<uint8_t> save_key_data_;
+};
+
+struct AnyKeysSegmenter : public BaseRowSegmenter {
+  static Result<std::unique_ptr<RowSegmenter>> Make(
+      const std::vector<TypeHolder>& key_types, ExecContext* ctx) {
+    ARROW_RETURN_NOT_OK(Grouper::Make(key_types, ctx));  // check types
+    return std::make_unique<AnyKeysSegmenter>(key_types, ctx);
+  }
+
+  AnyKeysSegmenter(const std::vector<TypeHolder>& key_types, ExecContext* ctx)
+      : BaseRowSegmenter(key_types),
+        ctx_(ctx),
+        grouper_(nullptr),
+        save_group_id_(kNoGroupId) {}
+
+  Status Reset() override {

Review Comment:
   > we could probably have reviewed the segmenters independently
   
   Perhaps, but that may have replaced one problem with another. In practice, I would have locally developed the same code as in this PR so I'd know the segmenter API is in fact what is needed for implementing segmented aggregation. Then I would have created a PR for just the segmenter code, and questions around why this or that particular code was needed would have popped up. It would be much harder to justify the segmenter code when segmented aggregation code is not shown. Moreover, even when the segmenter code were accepted, the next PR for segmented aggregation would likely have led to requested changes that affect the segmenter API too. In short, it's possible we would not have gained much from phasing this way.
   
   OTOH, I think it can be useful to start with the larger PR and after some discussions decide to split it up, because then it is clear how the full code is going to look like and how the phases are justified.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] icexelloss commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "icexelloss (via GitHub)" <gi...@apache.org>.
icexelloss commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1129563494


##########
cpp/src/arrow/compute/row/grouper.cc:
##########
@@ -39,12 +43,330 @@
 namespace arrow {
 
 using internal::checked_cast;
+using internal::PrimitiveScalarBase;
 
 namespace compute {
 
 namespace {
 
-struct GrouperImpl : Grouper {
+constexpr uint32_t kNoGroupId = std::numeric_limits<uint32_t>::max();
+
+using group_id_t = std::remove_const<decltype(kNoGroupId)>::type;
+using GroupIdType = CTypeTraits<group_id_t>::ArrowType;
+auto g_group_id_type = std::make_shared<GroupIdType>();
+
+inline const uint8_t* GetValuesAsBytes(const ArraySpan& data, int64_t offset = 0) {
+  DCHECK_GT(data.type->byte_width(), 0);
+  int64_t absolute_byte_offset = (data.offset + offset) * data.type->byte_width();
+  return data.GetValues<uint8_t>(1, absolute_byte_offset);
+}
+
+template <typename Value>
+Status CheckForGetNextSegment(const std::vector<Value>& values, int64_t length,
+                              int64_t offset, const std::vector<TypeHolder>& key_types) {
+  if (offset < 0 || offset > length) {
+    return Status::Invalid("invalid grouping segmenter offset: ", offset);
+  }
+  if (values.size() != key_types.size()) {
+    return Status::Invalid("expected batch size ", key_types.size(), " but got ",
+                           values.size());
+  }
+  for (size_t i = 0; i < key_types.size(); i++) {
+    const auto& value = values[i];
+    const auto& key_type = key_types[i];
+    if (*value.type() != *key_type.type) {
+      return Status::Invalid("expected batch value ", i, " of type ", *key_type.type,
+                             " but got ", *value.type());
+    }
+  }
+  return Status::OK();
+}
+
+template <typename Batch>
+enable_if_t<std::is_same<Batch, ExecSpan>::value || std::is_same<Batch, ExecBatch>::value,
+            Status>
+CheckForGetNextSegment(const Batch& batch, int64_t offset,
+                       const std::vector<TypeHolder>& key_types) {
+  return CheckForGetNextSegment(batch.values, batch.length, offset, key_types);
+}
+
+struct BaseRowSegmenter : public RowSegmenter {
+  explicit BaseRowSegmenter(const std::vector<TypeHolder>& key_types)
+      : key_types_(key_types) {}
+
+  const std::vector<TypeHolder>& key_types() const override { return key_types_; }
+
+  std::vector<TypeHolder> key_types_;
+};
+
+Segment MakeSegment(int64_t batch_length, int64_t offset, int64_t length, bool extends) {
+  return Segment{offset, length, offset + length >= batch_length, extends};
+}
+
+int64_t GetMatchLength(const uint8_t* match_bytes, int64_t match_width,
+                       const uint8_t* array_bytes, int64_t offset, int64_t length) {
+  int64_t cursor, byte_cursor;
+  for (cursor = offset, byte_cursor = match_width * cursor; cursor < length;
+       cursor++, byte_cursor += match_width) {
+    if (memcmp(match_bytes, array_bytes + byte_cursor,
+               static_cast<size_t>(match_width)) != 0) {
+      break;
+    }
+  }
+  return std::min(cursor, length) - offset;
+}
+
+using ExtendFunc = std::function<bool(const void*)>;
+constexpr bool kDefaultExtends = true;
+constexpr bool kEmptyExtends = true;
+
+struct NoKeysSegmenter : public BaseRowSegmenter {
+  static std::unique_ptr<RowSegmenter> Make() {
+    return std::make_unique<NoKeysSegmenter>();
+  }
+
+  NoKeysSegmenter() : BaseRowSegmenter({}) {}
+
+  Status Reset() override { return Status::OK(); }
+
+  Result<Segment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {}));
+    return MakeSegment(batch.length, offset, batch.length - offset, kDefaultExtends);
+  }
+};
+
+struct SimpleKeySegmenter : public BaseRowSegmenter {
+  static Result<std::unique_ptr<RowSegmenter>> Make(TypeHolder key_type) {
+    return std::make_unique<SimpleKeySegmenter>(key_type);
+  }
+
+  explicit SimpleKeySegmenter(TypeHolder key_type)
+      : BaseRowSegmenter({key_type}), key_type_(key_types_[0]), save_key_data_() {}
+
+  Status CheckType(const DataType& type) {
+    if (!is_fixed_width(type)) {
+      return Status::Invalid("SimpleKeySegmenter does not support type ", type);
+    }
+    return Status::OK();
+  }
+
+  Status Reset() override {
+    save_key_data_.resize(0);
+    return Status::OK();
+  }
+
+  // Checks whether the given grouping data extends the current segment, i.e., is equal to
+  // previously seen grouping data, which is updated with each invocation.
+  bool Extend(const void* data) {
+    size_t byte_width = static_cast<size_t>(key_type_.type->byte_width());
+    bool extends = save_key_data_.size() != byte_width
+                       ? kDefaultExtends
+                       : 0 == memcmp(save_key_data_.data(), data, byte_width);
+    save_key_data_.resize(byte_width);
+    memcpy(save_key_data_.data(), data, byte_width);
+    return extends;
+  }
+
+  Result<Segment> GetNextSegment(const Scalar& scalar, int64_t offset, int64_t length) {
+    ARROW_RETURN_NOT_OK(CheckType(*scalar.type));
+    if (!scalar.is_valid) {
+      return Status::Invalid("segmenting an invalid scalar");
+    }
+    auto data = checked_cast<const PrimitiveScalarBase&>(scalar).data();
+    bool extends = length > 0 ? Extend(data) : kEmptyExtends;
+    return MakeSegment(length, offset, length, extends);
+  }
+
+  Result<Segment> GetNextSegment(const DataType& array_type, const uint8_t* array_bytes,
+                                 int64_t offset, int64_t length) {
+    RETURN_NOT_OK(CheckType(array_type));
+    int64_t byte_width = array_type.byte_width();
+    int64_t match_length = GetMatchLength(array_bytes + offset * byte_width, byte_width,
+                                          array_bytes, offset, length);
+    bool extends = length > 0 ? Extend(array_bytes + offset * byte_width) : kEmptyExtends;
+    return MakeSegment(length, offset, match_length, extends);
+  }
+
+  Result<Segment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {key_type_}));
+    if (offset == batch.length) {
+      return MakeSegment(batch.length, offset, 0, kEmptyExtends);
+    }
+    const auto& value = batch.values[0];
+    if (value.is_scalar()) {
+      return GetNextSegment(*value.scalar, offset, batch.length);
+    }
+    ARROW_DCHECK(value.is_array());
+    const auto& array = value.array;
+    if (array.GetNullCount() > 0) {
+      return Status::NotImplemented("segmenting a nullable array");
+    }
+    return GetNextSegment(*array.type, GetValuesAsBytes(array), offset, batch.length);
+  }
+
+ private:
+  TypeHolder key_type_;
+  std::vector<uint8_t> save_key_data_;
+};
+
+struct AnyKeysSegmenter : public BaseRowSegmenter {
+  static Result<std::unique_ptr<RowSegmenter>> Make(
+      const std::vector<TypeHolder>& key_types, ExecContext* ctx) {
+    ARROW_RETURN_NOT_OK(Grouper::Make(key_types, ctx));  // check types
+    return std::make_unique<AnyKeysSegmenter>(key_types, ctx);
+  }
+
+  AnyKeysSegmenter(const std::vector<TypeHolder>& key_types, ExecContext* ctx)
+      : BaseRowSegmenter(key_types),
+        ctx_(ctx),
+        grouper_(nullptr),
+        save_group_id_(kNoGroupId) {}
+
+  Status Reset() override {
+    grouper_ = nullptr;
+    save_group_id_ = kNoGroupId;
+    return Status::OK();
+  }
+
+  bool Extend(const void* data) {
+    auto group_id = *static_cast<const group_id_t*>(data);
+    bool extends =
+        save_group_id_ == kNoGroupId ? kDefaultExtends : save_group_id_ == group_id;
+    save_group_id_ = group_id;
+    return extends;
+  }
+
+  // Runs the grouper on a single row.  This is used to determine the group id of the
+  // first row of a new segment to see if it extends the previous segment.
+  template <typename Batch>
+  Result<group_id_t> MapGroupIdAt(const Batch& batch, int64_t offset) {
+    if (!grouper_) return kNoGroupId;
+    ARROW_ASSIGN_OR_RAISE(auto datum, grouper_->Consume(batch, offset,
+                                                        /*length=*/1));
+    if (!datum.is_array()) {
+      return Status::Invalid("accessing unsupported datum kind ", datum.kind());
+    }
+    const std::shared_ptr<ArrayData>& data = datum.array();
+    ARROW_DCHECK(data->GetNullCount() == 0);
+    DCHECK_EQ(data->type->id(), GroupIdType::type_id);
+    DCHECK_EQ(1, data->length);
+    const group_id_t* values = data->GetValues<group_id_t>(1);
+    return values[0];
+  }
+
+  Result<Segment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, key_types_));
+    if (offset == batch.length) {
+      return MakeSegment(batch.length, offset, 0, kEmptyExtends);
+    }
+    // ARROW-18311: make Grouper support Reset()
+    // so it can be cached instead of recreated below
+    //
+    // the group id must be computed prior to resetting the grouper, since it is compared
+    // to save_group_id_, and after resetting the grouper produces incomparable group ids
+    ARROW_ASSIGN_OR_RAISE(auto group_id, MapGroupIdAt(batch, offset));
+    ExtendFunc bound_extend = [this, group_id](const void* data) {
+      bool extends = Extend(&group_id);
+      save_group_id_ = *static_cast<const group_id_t*>(data);
+      return extends;
+    };
+    ARROW_ASSIGN_OR_RAISE(grouper_, Grouper::Make(key_types_, ctx_));  // TODO: reset it
+    ARROW_ASSIGN_OR_RAISE(auto datum, grouper_->Consume(batch, offset));
+    if (datum.is_array()) {
+      const std::shared_ptr<ArrayData>& data = datum.array();
+      ARROW_DCHECK(data->GetNullCount() == 0);
+      DCHECK_EQ(data->type->id(), GroupIdType::type_id);
+      const group_id_t* values = data->GetValues<group_id_t>(1);
+      int64_t cursor;
+      for (cursor = 1; cursor < data->length; cursor++) {
+        if (values[0] != values[cursor]) break;
+      }
+      int64_t length = std::min(cursor, batch.length - offset);

Review Comment:
   Please add add comments about the index/size relationship between `data` and `batch`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rtpsw commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "rtpsw (via GitHub)" <gi...@apache.org>.
rtpsw commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1126979527


##########
cpp/src/arrow/compute/row/grouper.cc:
##########
@@ -39,12 +43,330 @@
 namespace arrow {
 
 using internal::checked_cast;
+using internal::PrimitiveScalarBase;
 
 namespace compute {
 
 namespace {
 
-struct GrouperImpl : Grouper {
+constexpr uint32_t kNoGroupId = std::numeric_limits<uint32_t>::max();
+
+using group_id_t = std::remove_const<decltype(kNoGroupId)>::type;
+using GroupIdType = CTypeTraits<group_id_t>::ArrowType;
+auto g_group_id_type = std::make_shared<GroupIdType>();
+
+inline const uint8_t* GetValuesAsBytes(const ArraySpan& data, int64_t offset = 0) {
+  DCHECK_GT(data.type->byte_width(), 0);
+  int64_t absolute_byte_offset = (data.offset + offset) * data.type->byte_width();
+  return data.GetValues<uint8_t>(1, absolute_byte_offset);
+}
+
+template <typename Value>
+Status CheckForGetNextSegment(const std::vector<Value>& values, int64_t length,
+                              int64_t offset, const std::vector<TypeHolder>& key_types) {
+  if (offset < 0 || offset > length) {
+    return Status::Invalid("invalid grouping segmenter offset: ", offset);
+  }
+  if (values.size() != key_types.size()) {
+    return Status::Invalid("expected batch size ", key_types.size(), " but got ",
+                           values.size());
+  }
+  for (size_t i = 0; i < key_types.size(); i++) {
+    const auto& value = values[i];
+    const auto& key_type = key_types[i];
+    if (*value.type() != *key_type.type) {
+      return Status::Invalid("expected batch value ", i, " of type ", *key_type.type,
+                             " but got ", *value.type());
+    }
+  }
+  return Status::OK();
+}
+
+template <typename Batch>
+enable_if_t<std::is_same<Batch, ExecSpan>::value || std::is_same<Batch, ExecBatch>::value,
+            Status>
+CheckForGetNextSegment(const Batch& batch, int64_t offset,
+                       const std::vector<TypeHolder>& key_types) {
+  return CheckForGetNextSegment(batch.values, batch.length, offset, key_types);
+}
+
+struct BaseRowSegmenter : public RowSegmenter {
+  explicit BaseRowSegmenter(const std::vector<TypeHolder>& key_types)
+      : key_types_(key_types) {}
+
+  const std::vector<TypeHolder>& key_types() const override { return key_types_; }
+
+  std::vector<TypeHolder> key_types_;
+};
+
+Segment MakeSegment(int64_t batch_length, int64_t offset, int64_t length, bool extends) {
+  return Segment{offset, length, offset + length >= batch_length, extends};
+}
+
+int64_t GetMatchLength(const uint8_t* match_bytes, int64_t match_width,
+                       const uint8_t* array_bytes, int64_t offset, int64_t length) {
+  int64_t cursor, byte_cursor;
+  for (cursor = offset, byte_cursor = match_width * cursor; cursor < length;
+       cursor++, byte_cursor += match_width) {
+    if (memcmp(match_bytes, array_bytes + byte_cursor,
+               static_cast<size_t>(match_width)) != 0) {
+      break;
+    }
+  }
+  return std::min(cursor, length) - offset;
+}
+
+using ExtendFunc = std::function<bool(const void*)>;
+constexpr bool kDefaultExtends = true;
+constexpr bool kEmptyExtends = true;
+
+struct NoKeysSegmenter : public BaseRowSegmenter {
+  static std::unique_ptr<RowSegmenter> Make() {
+    return std::make_unique<NoKeysSegmenter>();
+  }
+
+  NoKeysSegmenter() : BaseRowSegmenter({}) {}
+
+  Status Reset() override { return Status::OK(); }
+
+  Result<Segment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {}));
+    return MakeSegment(batch.length, offset, batch.length - offset, kDefaultExtends);
+  }
+};
+
+struct SimpleKeySegmenter : public BaseRowSegmenter {
+  static Result<std::unique_ptr<RowSegmenter>> Make(TypeHolder key_type) {
+    return std::make_unique<SimpleKeySegmenter>(key_type);
+  }
+
+  explicit SimpleKeySegmenter(TypeHolder key_type)
+      : BaseRowSegmenter({key_type}), key_type_(key_types_[0]), save_key_data_() {}
+
+  Status CheckType(const DataType& type) {
+    if (!is_fixed_width(type)) {
+      return Status::Invalid("SimpleKeySegmenter does not support type ", type);
+    }
+    return Status::OK();
+  }
+
+  Status Reset() override {
+    save_key_data_.resize(0);
+    return Status::OK();
+  }
+
+  // Checks whether the given grouping data extends the current segment, i.e., is equal to
+  // previously seen grouping data, which is updated with each invocation.
+  bool Extend(const void* data) {
+    size_t byte_width = static_cast<size_t>(key_type_.type->byte_width());
+    bool extends = save_key_data_.size() != byte_width
+                       ? kDefaultExtends
+                       : 0 == memcmp(save_key_data_.data(), data, byte_width);
+    save_key_data_.resize(byte_width);
+    memcpy(save_key_data_.data(), data, byte_width);
+    return extends;
+  }
+
+  Result<Segment> GetNextSegment(const Scalar& scalar, int64_t offset, int64_t length) {
+    ARROW_RETURN_NOT_OK(CheckType(*scalar.type));
+    if (!scalar.is_valid) {
+      return Status::Invalid("segmenting an invalid scalar");
+    }
+    auto data = checked_cast<const PrimitiveScalarBase&>(scalar).data();
+    bool extends = length > 0 ? Extend(data) : kEmptyExtends;
+    return MakeSegment(length, offset, length, extends);
+  }
+
+  Result<Segment> GetNextSegment(const DataType& array_type, const uint8_t* array_bytes,
+                                 int64_t offset, int64_t length) {
+    RETURN_NOT_OK(CheckType(array_type));
+    int64_t byte_width = array_type.byte_width();
+    int64_t match_length = GetMatchLength(array_bytes + offset * byte_width, byte_width,
+                                          array_bytes, offset, length);
+    bool extends = length > 0 ? Extend(array_bytes + offset * byte_width) : kEmptyExtends;
+    return MakeSegment(length, offset, match_length, extends);
+  }
+
+  Result<Segment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {key_type_}));
+    if (offset == batch.length) {
+      return MakeSegment(batch.length, offset, 0, kEmptyExtends);
+    }
+    const auto& value = batch.values[0];
+    if (value.is_scalar()) {
+      return GetNextSegment(*value.scalar, offset, batch.length);
+    }
+    ARROW_DCHECK(value.is_array());
+    const auto& array = value.array;
+    if (array.GetNullCount() > 0) {
+      return Status::NotImplemented("segmenting a nullable array");
+    }
+    return GetNextSegment(*array.type, GetValuesAsBytes(array), offset, batch.length);
+  }
+
+ private:
+  TypeHolder key_type_;
+  std::vector<uint8_t> save_key_data_;
+};
+
+struct AnyKeysSegmenter : public BaseRowSegmenter {
+  static Result<std::unique_ptr<RowSegmenter>> Make(
+      const std::vector<TypeHolder>& key_types, ExecContext* ctx) {
+    ARROW_RETURN_NOT_OK(Grouper::Make(key_types, ctx));  // check types
+    return std::make_unique<AnyKeysSegmenter>(key_types, ctx);
+  }
+
+  AnyKeysSegmenter(const std::vector<TypeHolder>& key_types, ExecContext* ctx)
+      : BaseRowSegmenter(key_types),
+        ctx_(ctx),
+        grouper_(nullptr),
+        save_group_id_(kNoGroupId) {}
+
+  Status Reset() override {
+    grouper_ = nullptr;
+    save_group_id_ = kNoGroupId;
+    return Status::OK();
+  }
+
+  bool Extend(const void* data) {
+    auto group_id = *static_cast<const group_id_t*>(data);
+    bool extends =
+        save_group_id_ == kNoGroupId ? kDefaultExtends : save_group_id_ == group_id;
+    save_group_id_ = group_id;
+    return extends;
+  }
+
+  // Runs the grouper on a single row.  This is used to determine the group id of the
+  // first row of a new segment to see if it extends the previous segment.
+  template <typename Batch>
+  Result<group_id_t> MapGroupIdAt(const Batch& batch, int64_t offset) {
+    if (!grouper_) return kNoGroupId;
+    ARROW_ASSIGN_OR_RAISE(auto datum, grouper_->Consume(batch, offset,
+                                                        /*length=*/1));
+    if (!datum.is_array()) {
+      return Status::Invalid("accessing unsupported datum kind ", datum.kind());
+    }
+    const std::shared_ptr<ArrayData>& data = datum.array();
+    ARROW_DCHECK(data->GetNullCount() == 0);
+    DCHECK_EQ(data->type->id(), GroupIdType::type_id);
+    DCHECK_EQ(1, data->length);
+    const group_id_t* values = data->GetValues<group_id_t>(1);
+    return values[0];
+  }
+
+  Result<Segment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, key_types_));
+    if (offset == batch.length) {
+      return MakeSegment(batch.length, offset, 0, kEmptyExtends);
+    }
+    // ARROW-18311: make Grouper support Reset()

Review Comment:
   IIUC what you mean, this TODO is here because this is the block of code which would benefit from `Grouper` supporting `Reset`, so I think the comment is well-placed.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] icexelloss commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "icexelloss (via GitHub)" <gi...@apache.org>.
icexelloss commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1126984626


##########
cpp/src/arrow/compute/row/grouper.cc:
##########
@@ -39,12 +43,330 @@
 namespace arrow {
 
 using internal::checked_cast;
+using internal::PrimitiveScalarBase;
 
 namespace compute {
 
 namespace {
 
-struct GrouperImpl : Grouper {
+constexpr uint32_t kNoGroupId = std::numeric_limits<uint32_t>::max();
+
+using group_id_t = std::remove_const<decltype(kNoGroupId)>::type;
+using GroupIdType = CTypeTraits<group_id_t>::ArrowType;
+auto g_group_id_type = std::make_shared<GroupIdType>();
+
+inline const uint8_t* GetValuesAsBytes(const ArraySpan& data, int64_t offset = 0) {
+  DCHECK_GT(data.type->byte_width(), 0);
+  int64_t absolute_byte_offset = (data.offset + offset) * data.type->byte_width();
+  return data.GetValues<uint8_t>(1, absolute_byte_offset);
+}
+
+template <typename Value>
+Status CheckForGetNextSegment(const std::vector<Value>& values, int64_t length,
+                              int64_t offset, const std::vector<TypeHolder>& key_types) {
+  if (offset < 0 || offset > length) {
+    return Status::Invalid("invalid grouping segmenter offset: ", offset);
+  }
+  if (values.size() != key_types.size()) {
+    return Status::Invalid("expected batch size ", key_types.size(), " but got ",
+                           values.size());
+  }
+  for (size_t i = 0; i < key_types.size(); i++) {
+    const auto& value = values[i];
+    const auto& key_type = key_types[i];
+    if (*value.type() != *key_type.type) {
+      return Status::Invalid("expected batch value ", i, " of type ", *key_type.type,
+                             " but got ", *value.type());
+    }
+  }
+  return Status::OK();
+}
+
+template <typename Batch>
+enable_if_t<std::is_same<Batch, ExecSpan>::value || std::is_same<Batch, ExecBatch>::value,
+            Status>
+CheckForGetNextSegment(const Batch& batch, int64_t offset,
+                       const std::vector<TypeHolder>& key_types) {
+  return CheckForGetNextSegment(batch.values, batch.length, offset, key_types);
+}
+
+struct BaseRowSegmenter : public RowSegmenter {
+  explicit BaseRowSegmenter(const std::vector<TypeHolder>& key_types)
+      : key_types_(key_types) {}
+
+  const std::vector<TypeHolder>& key_types() const override { return key_types_; }
+
+  std::vector<TypeHolder> key_types_;
+};
+
+Segment MakeSegment(int64_t batch_length, int64_t offset, int64_t length, bool extends) {
+  return Segment{offset, length, offset + length >= batch_length, extends};
+}
+
+int64_t GetMatchLength(const uint8_t* match_bytes, int64_t match_width,
+                       const uint8_t* array_bytes, int64_t offset, int64_t length) {
+  int64_t cursor, byte_cursor;
+  for (cursor = offset, byte_cursor = match_width * cursor; cursor < length;
+       cursor++, byte_cursor += match_width) {
+    if (memcmp(match_bytes, array_bytes + byte_cursor,
+               static_cast<size_t>(match_width)) != 0) {
+      break;
+    }
+  }
+  return std::min(cursor, length) - offset;
+}
+
+using ExtendFunc = std::function<bool(const void*)>;
+constexpr bool kDefaultExtends = true;
+constexpr bool kEmptyExtends = true;
+
+struct NoKeysSegmenter : public BaseRowSegmenter {
+  static std::unique_ptr<RowSegmenter> Make() {
+    return std::make_unique<NoKeysSegmenter>();
+  }
+
+  NoKeysSegmenter() : BaseRowSegmenter({}) {}
+
+  Status Reset() override { return Status::OK(); }
+
+  Result<Segment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {}));
+    return MakeSegment(batch.length, offset, batch.length - offset, kDefaultExtends);
+  }
+};
+
+struct SimpleKeySegmenter : public BaseRowSegmenter {
+  static Result<std::unique_ptr<RowSegmenter>> Make(TypeHolder key_type) {
+    return std::make_unique<SimpleKeySegmenter>(key_type);
+  }
+
+  explicit SimpleKeySegmenter(TypeHolder key_type)
+      : BaseRowSegmenter({key_type}), key_type_(key_types_[0]), save_key_data_() {}
+
+  Status CheckType(const DataType& type) {
+    if (!is_fixed_width(type)) {
+      return Status::Invalid("SimpleKeySegmenter does not support type ", type);
+    }
+    return Status::OK();
+  }
+
+  Status Reset() override {
+    save_key_data_.resize(0);
+    return Status::OK();
+  }
+
+  // Checks whether the given grouping data extends the current segment, i.e., is equal to
+  // previously seen grouping data, which is updated with each invocation.
+  bool Extend(const void* data) {
+    size_t byte_width = static_cast<size_t>(key_type_.type->byte_width());
+    bool extends = save_key_data_.size() != byte_width
+                       ? kDefaultExtends
+                       : 0 == memcmp(save_key_data_.data(), data, byte_width);
+    save_key_data_.resize(byte_width);
+    memcpy(save_key_data_.data(), data, byte_width);
+    return extends;
+  }
+
+  Result<Segment> GetNextSegment(const Scalar& scalar, int64_t offset, int64_t length) {
+    ARROW_RETURN_NOT_OK(CheckType(*scalar.type));
+    if (!scalar.is_valid) {
+      return Status::Invalid("segmenting an invalid scalar");
+    }
+    auto data = checked_cast<const PrimitiveScalarBase&>(scalar).data();
+    bool extends = length > 0 ? Extend(data) : kEmptyExtends;
+    return MakeSegment(length, offset, length, extends);
+  }
+
+  Result<Segment> GetNextSegment(const DataType& array_type, const uint8_t* array_bytes,
+                                 int64_t offset, int64_t length) {
+    RETURN_NOT_OK(CheckType(array_type));
+    int64_t byte_width = array_type.byte_width();
+    int64_t match_length = GetMatchLength(array_bytes + offset * byte_width, byte_width,
+                                          array_bytes, offset, length);
+    bool extends = length > 0 ? Extend(array_bytes + offset * byte_width) : kEmptyExtends;
+    return MakeSegment(length, offset, match_length, extends);
+  }
+
+  Result<Segment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {key_type_}));
+    if (offset == batch.length) {
+      return MakeSegment(batch.length, offset, 0, kEmptyExtends);
+    }
+    const auto& value = batch.values[0];
+    if (value.is_scalar()) {
+      return GetNextSegment(*value.scalar, offset, batch.length);
+    }
+    ARROW_DCHECK(value.is_array());
+    const auto& array = value.array;
+    if (array.GetNullCount() > 0) {
+      return Status::NotImplemented("segmenting a nullable array");
+    }
+    return GetNextSegment(*array.type, GetValuesAsBytes(array), offset, batch.length);
+  }
+
+ private:
+  TypeHolder key_type_;
+  std::vector<uint8_t> save_key_data_;
+};
+
+struct AnyKeysSegmenter : public BaseRowSegmenter {
+  static Result<std::unique_ptr<RowSegmenter>> Make(
+      const std::vector<TypeHolder>& key_types, ExecContext* ctx) {
+    ARROW_RETURN_NOT_OK(Grouper::Make(key_types, ctx));  // check types
+    return std::make_unique<AnyKeysSegmenter>(key_types, ctx);
+  }
+
+  AnyKeysSegmenter(const std::vector<TypeHolder>& key_types, ExecContext* ctx)
+      : BaseRowSegmenter(key_types),
+        ctx_(ctx),
+        grouper_(nullptr),
+        save_group_id_(kNoGroupId) {}
+
+  Status Reset() override {
+    grouper_ = nullptr;
+    save_group_id_ = kNoGroupId;
+    return Status::OK();
+  }
+
+  bool Extend(const void* data) {
+    auto group_id = *static_cast<const group_id_t*>(data);
+    bool extends =
+        save_group_id_ == kNoGroupId ? kDefaultExtends : save_group_id_ == group_id;
+    save_group_id_ = group_id;
+    return extends;
+  }
+
+  // Runs the grouper on a single row.  This is used to determine the group id of the
+  // first row of a new segment to see if it extends the previous segment.
+  template <typename Batch>
+  Result<group_id_t> MapGroupIdAt(const Batch& batch, int64_t offset) {
+    if (!grouper_) return kNoGroupId;
+    ARROW_ASSIGN_OR_RAISE(auto datum, grouper_->Consume(batch, offset,
+                                                        /*length=*/1));
+    if (!datum.is_array()) {
+      return Status::Invalid("accessing unsupported datum kind ", datum.kind());
+    }
+    const std::shared_ptr<ArrayData>& data = datum.array();
+    ARROW_DCHECK(data->GetNullCount() == 0);
+    DCHECK_EQ(data->type->id(), GroupIdType::type_id);
+    DCHECK_EQ(1, data->length);
+    const group_id_t* values = data->GetValues<group_id_t>(1);
+    return values[0];
+  }
+
+  Result<Segment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, key_types_));
+    if (offset == batch.length) {
+      return MakeSegment(batch.length, offset, 0, kEmptyExtends);
+    }
+    // ARROW-18311: make Grouper support Reset()

Review Comment:
   What is Grouper's `Reset` supposed to do and how would this code benefit from it?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] icexelloss commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "icexelloss (via GitHub)" <gi...@apache.org>.
icexelloss commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1128114556


##########
cpp/src/arrow/compute/kernels/hash_aggregate_test.cc:
##########
@@ -135,22 +142,96 @@ Result<Datum> NaiveGroupBy(std::vector<Datum> arguments, std::vector<Datum> keys
   return Take(struct_arr, sorted_indices);
 }
 
+Result<Datum> MakeGroupByOutput(const std::vector<ExecBatch>& output_batches,
+                                const std::shared_ptr<Schema> output_schema,
+                                size_t num_aggregates, size_t num_keys, bool naive) {
+  ArrayVector out_arrays(num_aggregates + num_keys);
+  for (size_t i = 0; i < out_arrays.size(); ++i) {
+    std::vector<std::shared_ptr<Array>> arrays(output_batches.size());
+    for (size_t j = 0; j < output_batches.size(); ++j) {
+      arrays[j] = output_batches[j].values[i].make_array();
+    }
+    if (arrays.empty()) {
+      ARROW_ASSIGN_OR_RAISE(
+          out_arrays[i],
+          MakeArrayOfNull(output_schema->field(static_cast<int>(i))->type(),
+                          /*length=*/0));
+    } else {
+      ARROW_ASSIGN_OR_RAISE(out_arrays[i], Concatenate(arrays));
+    }
+  }
+
+  ARROW_ASSIGN_OR_RAISE(
+      std::shared_ptr<Array> struct_arr,
+      StructArray::Make(std::move(out_arrays), output_schema->fields()));
+
+  bool need_sort = !naive;
+  for (size_t i = num_aggregates; need_sort && i < out_arrays.size(); i++) {
+    if (output_schema->field(static_cast<int>(i))->type()->id() == Type::DICTIONARY) {
+      need_sort = false;
+    }
+  }
+  if (!need_sort) {
+    return struct_arr;
+  }
+
+  // The exec plan may reorder the output rows.  The tests are all setup to expect ouptut
+  // in ascending order of keys.  So we need to sort the result by the key columns.  To do
+  // that we create a table using the key columns, calculate the sort indices from that
+  // table (sorting on all fields) and then use those indices to calculate our result.
+  std::vector<std::shared_ptr<Field>> key_fields;
+  std::vector<std::shared_ptr<Array>> key_columns;
+  std::vector<SortKey> sort_keys;
+  for (std::size_t i = 0; i < num_keys; i++) {
+    const std::shared_ptr<Array>& arr = out_arrays[i + num_aggregates];
+    key_columns.push_back(arr);
+    key_fields.push_back(field("name_does_not_matter", arr->type()));
+    sort_keys.emplace_back(static_cast<int>(i));
+  }
+  std::shared_ptr<Schema> key_schema = schema(std::move(key_fields));
+  std::shared_ptr<Table> key_table = Table::Make(std::move(key_schema), key_columns);
+  SortOptions sort_options(std::move(sort_keys));
+  ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Array> sort_indices,
+                        SortIndices(key_table, sort_options));
+
+  return Take(struct_arr, sort_indices);
+}
+
 Result<Datum> RunGroupBy(const BatchesWithSchema& input,
                          const std::vector<std::string>& key_names,
-                         const std::vector<Aggregate>& aggregates, bool use_threads) {
+                         const std::vector<std::string>& segment_key_names,
+                         const std::vector<Aggregate>& aggregates, ExecContext* ctx,
+                         bool use_threads, bool segmented = false, bool naive = false) {
+  // When segment_keys is non-empty the `segmented` flag is always true; otherwise (when
+  // empty), it may still be set to true. In this case, the tester restructures (without
+  // changing the data of) the result of RunGroupBy from `std::vector<ExecBatch>`
+  // (output_batches) to `std::vector<ArrayVector>` (out_arrays), which have the structure
+  // typical of the case of a non-empty segment_keys (with multiple arrays per column, one
+  // array per segment) but only one array per column (because, technically, there is only
+  // one segment in this case). Thus, this case focuses on the structure of the result.
+  //
+  // The `naive` flag means that the output is expected to be like that of `NaiveGroupBy`,
+  // which in particular doesn't require sorting. The reason for the naive flag is that

Review Comment:
   "doesn't require sorting" means that the output of `NaiveGroupBy` is unsorted or does this mean sth else?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] icexelloss commented on pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "icexelloss (via GitHub)" <gi...@apache.org>.
icexelloss commented on PR #34311:
URL: https://github.com/apache/arrow/pull/34311#issuecomment-1460721299

   @rtpsw I think this is getting close but there are still a number of unresolved thread. Please check those are resolved and ping me when it is ready for me to take another look.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] icexelloss commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "icexelloss (via GitHub)" <gi...@apache.org>.
icexelloss commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1132491963


##########
cpp/src/arrow/compute/exec/aggregate_node.cc:
##########
@@ -169,20 +185,79 @@ void AggregatesToString(std::stringstream* ss, const Schema& input_schema,
   *ss << ']';
 }
 
+// Handle the input batch
+// If a segment is closed by this batch, then we output the aggregation for the segment
+// If a segment is not closed by this batch, then we add the batch to the segment
+template <typename BatchHandler>
+Status HandleSegments(std::unique_ptr<RowSegmenter>& segmenter, const ExecBatch& batch,

Review Comment:
   Looks like some of the comments in here are not addressed. I will create a follow up to track.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] icexelloss commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "icexelloss (via GitHub)" <gi...@apache.org>.
icexelloss commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1126690433


##########
cpp/src/arrow/compute/row/grouper.h:
##########
@@ -30,6 +30,49 @@
 namespace arrow {
 namespace compute {
 
+/// \brief A segment of contiguous rows for grouping
+struct ARROW_EXPORT GroupingSegment {
+  int64_t offset;
+  int64_t length;
+  bool is_open;
+  bool extends;
+};
+
+inline bool operator==(const GroupingSegment& segment1, const GroupingSegment& segment2) {
+  return segment1.offset == segment2.offset && segment1.length == segment2.length &&
+         segment1.is_open == segment2.is_open && segment1.extends == segment2.extends;
+}
+inline bool operator!=(const GroupingSegment& segment1, const GroupingSegment& segment2) {
+  return !(segment1 == segment2);
+}
+
+/// \brief Computes grouping segments for a batch. Each segment covers rows with identical
+/// values in the batch. The values in the batch are often selected as keys from a larger
+/// batch.
+class ARROW_EXPORT GroupingSegmenter {
+ public:
+  virtual ~GroupingSegmenter() = default;
+
+  /// \brief Construct a GroupingSegmenter which receives the specified key types
+  static Result<std::unique_ptr<GroupingSegmenter>> Make(
+      const std::vector<TypeHolder>& key_types, bool nullable_keys = false,
+      ExecContext* ctx = default_exec_context());

Review Comment:
   So to clarify - the default values in the constructor here are because we want to template GroupClass and Grouper in test?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rtpsw commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "rtpsw (via GitHub)" <gi...@apache.org>.
rtpsw commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1127476475


##########
cpp/src/arrow/compute/row/grouper.cc:
##########
@@ -39,12 +43,330 @@
 namespace arrow {
 
 using internal::checked_cast;
+using internal::PrimitiveScalarBase;
 
 namespace compute {
 
 namespace {
 
-struct GrouperImpl : Grouper {
+constexpr uint32_t kNoGroupId = std::numeric_limits<uint32_t>::max();
+
+using group_id_t = std::remove_const<decltype(kNoGroupId)>::type;
+using GroupIdType = CTypeTraits<group_id_t>::ArrowType;
+auto g_group_id_type = std::make_shared<GroupIdType>();
+
+inline const uint8_t* GetValuesAsBytes(const ArraySpan& data, int64_t offset = 0) {
+  DCHECK_GT(data.type->byte_width(), 0);
+  int64_t absolute_byte_offset = (data.offset + offset) * data.type->byte_width();
+  return data.GetValues<uint8_t>(1, absolute_byte_offset);
+}
+
+template <typename Value>
+Status CheckForGetNextSegment(const std::vector<Value>& values, int64_t length,
+                              int64_t offset, const std::vector<TypeHolder>& key_types) {
+  if (offset < 0 || offset > length) {
+    return Status::Invalid("invalid grouping segmenter offset: ", offset);
+  }
+  if (values.size() != key_types.size()) {
+    return Status::Invalid("expected batch size ", key_types.size(), " but got ",
+                           values.size());
+  }
+  for (size_t i = 0; i < key_types.size(); i++) {
+    const auto& value = values[i];
+    const auto& key_type = key_types[i];
+    if (*value.type() != *key_type.type) {
+      return Status::Invalid("expected batch value ", i, " of type ", *key_type.type,
+                             " but got ", *value.type());
+    }
+  }
+  return Status::OK();
+}
+
+template <typename Batch>
+enable_if_t<std::is_same<Batch, ExecSpan>::value || std::is_same<Batch, ExecBatch>::value,
+            Status>
+CheckForGetNextSegment(const Batch& batch, int64_t offset,
+                       const std::vector<TypeHolder>& key_types) {
+  return CheckForGetNextSegment(batch.values, batch.length, offset, key_types);
+}
+
+struct BaseRowSegmenter : public RowSegmenter {
+  explicit BaseRowSegmenter(const std::vector<TypeHolder>& key_types)
+      : key_types_(key_types) {}
+
+  const std::vector<TypeHolder>& key_types() const override { return key_types_; }
+
+  std::vector<TypeHolder> key_types_;
+};
+
+Segment MakeSegment(int64_t batch_length, int64_t offset, int64_t length, bool extends) {
+  return Segment{offset, length, offset + length >= batch_length, extends};
+}
+
+int64_t GetMatchLength(const uint8_t* match_bytes, int64_t match_width,
+                       const uint8_t* array_bytes, int64_t offset, int64_t length) {
+  int64_t cursor, byte_cursor;
+  for (cursor = offset, byte_cursor = match_width * cursor; cursor < length;
+       cursor++, byte_cursor += match_width) {
+    if (memcmp(match_bytes, array_bytes + byte_cursor,
+               static_cast<size_t>(match_width)) != 0) {
+      break;
+    }
+  }
+  return std::min(cursor, length) - offset;
+}
+
+using ExtendFunc = std::function<bool(const void*)>;
+constexpr bool kDefaultExtends = true;
+constexpr bool kEmptyExtends = true;
+
+struct NoKeysSegmenter : public BaseRowSegmenter {
+  static std::unique_ptr<RowSegmenter> Make() {
+    return std::make_unique<NoKeysSegmenter>();
+  }
+
+  NoKeysSegmenter() : BaseRowSegmenter({}) {}
+
+  Status Reset() override { return Status::OK(); }
+
+  Result<Segment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {}));
+    return MakeSegment(batch.length, offset, batch.length - offset, kDefaultExtends);
+  }
+};
+
+struct SimpleKeySegmenter : public BaseRowSegmenter {
+  static Result<std::unique_ptr<RowSegmenter>> Make(TypeHolder key_type) {
+    return std::make_unique<SimpleKeySegmenter>(key_type);
+  }
+
+  explicit SimpleKeySegmenter(TypeHolder key_type)
+      : BaseRowSegmenter({key_type}), key_type_(key_types_[0]), save_key_data_() {}
+
+  Status CheckType(const DataType& type) {
+    if (!is_fixed_width(type)) {
+      return Status::Invalid("SimpleKeySegmenter does not support type ", type);
+    }
+    return Status::OK();
+  }
+
+  Status Reset() override {
+    save_key_data_.resize(0);
+    return Status::OK();
+  }
+
+  // Checks whether the given grouping data extends the current segment, i.e., is equal to
+  // previously seen grouping data, which is updated with each invocation.
+  bool Extend(const void* data) {
+    size_t byte_width = static_cast<size_t>(key_type_.type->byte_width());
+    bool extends = save_key_data_.size() != byte_width
+                       ? kDefaultExtends
+                       : 0 == memcmp(save_key_data_.data(), data, byte_width);
+    save_key_data_.resize(byte_width);
+    memcpy(save_key_data_.data(), data, byte_width);
+    return extends;
+  }
+
+  Result<Segment> GetNextSegment(const Scalar& scalar, int64_t offset, int64_t length) {
+    ARROW_RETURN_NOT_OK(CheckType(*scalar.type));
+    if (!scalar.is_valid) {
+      return Status::Invalid("segmenting an invalid scalar");
+    }
+    auto data = checked_cast<const PrimitiveScalarBase&>(scalar).data();
+    bool extends = length > 0 ? Extend(data) : kEmptyExtends;
+    return MakeSegment(length, offset, length, extends);
+  }
+
+  Result<Segment> GetNextSegment(const DataType& array_type, const uint8_t* array_bytes,
+                                 int64_t offset, int64_t length) {
+    RETURN_NOT_OK(CheckType(array_type));
+    int64_t byte_width = array_type.byte_width();
+    int64_t match_length = GetMatchLength(array_bytes + offset * byte_width, byte_width,
+                                          array_bytes, offset, length);
+    bool extends = length > 0 ? Extend(array_bytes + offset * byte_width) : kEmptyExtends;
+    return MakeSegment(length, offset, match_length, extends);
+  }
+
+  Result<Segment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {key_type_}));
+    if (offset == batch.length) {
+      return MakeSegment(batch.length, offset, 0, kEmptyExtends);
+    }
+    const auto& value = batch.values[0];
+    if (value.is_scalar()) {
+      return GetNextSegment(*value.scalar, offset, batch.length);
+    }
+    ARROW_DCHECK(value.is_array());
+    const auto& array = value.array;
+    if (array.GetNullCount() > 0) {
+      return Status::NotImplemented("segmenting a nullable array");
+    }
+    return GetNextSegment(*array.type, GetValuesAsBytes(array), offset, batch.length);
+  }
+
+ private:
+  TypeHolder key_type_;
+  std::vector<uint8_t> save_key_data_;
+};
+
+struct AnyKeysSegmenter : public BaseRowSegmenter {
+  static Result<std::unique_ptr<RowSegmenter>> Make(
+      const std::vector<TypeHolder>& key_types, ExecContext* ctx) {
+    ARROW_RETURN_NOT_OK(Grouper::Make(key_types, ctx));  // check types
+    return std::make_unique<AnyKeysSegmenter>(key_types, ctx);
+  }
+
+  AnyKeysSegmenter(const std::vector<TypeHolder>& key_types, ExecContext* ctx)
+      : BaseRowSegmenter(key_types),
+        ctx_(ctx),
+        grouper_(nullptr),
+        save_group_id_(kNoGroupId) {}
+
+  Status Reset() override {
+    grouper_ = nullptr;
+    save_group_id_ = kNoGroupId;
+    return Status::OK();
+  }
+
+  bool Extend(const void* data) {
+    auto group_id = *static_cast<const group_id_t*>(data);
+    bool extends =
+        save_group_id_ == kNoGroupId ? kDefaultExtends : save_group_id_ == group_id;
+    save_group_id_ = group_id;
+    return extends;
+  }
+
+  // Runs the grouper on a single row.  This is used to determine the group id of the
+  // first row of a new segment to see if it extends the previous segment.
+  template <typename Batch>
+  Result<group_id_t> MapGroupIdAt(const Batch& batch, int64_t offset) {
+    if (!grouper_) return kNoGroupId;
+    ARROW_ASSIGN_OR_RAISE(auto datum, grouper_->Consume(batch, offset,
+                                                        /*length=*/1));
+    if (!datum.is_array()) {
+      return Status::Invalid("accessing unsupported datum kind ", datum.kind());
+    }
+    const std::shared_ptr<ArrayData>& data = datum.array();
+    ARROW_DCHECK(data->GetNullCount() == 0);
+    DCHECK_EQ(data->type->id(), GroupIdType::type_id);
+    DCHECK_EQ(1, data->length);
+    const group_id_t* values = data->GetValues<group_id_t>(1);

Review Comment:
   I really don't see a reason to add a comment for this idiom. There are plenty of places in Arrow code where this idiom is used with no comment. The columnar format of [arrays](https://arrow.apache.org/docs/format/Columnar.html and [fixed-width arrays](https://arrow.apache.org/docs/format/Columnar.html#fixed-size-primitive-layout) is well-documented.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rtpsw commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "rtpsw (via GitHub)" <gi...@apache.org>.
rtpsw commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1116932261


##########
cpp/src/arrow/compute/kernels/hash_aggregate_test.cc:
##########
@@ -174,81 +242,117 @@ Result<Datum> RunGroupBy(const BatchesWithSchema& input,
   ARROW_ASSIGN_OR_RAISE(std::vector<ExecBatch> output_batches,
                         start_and_collect.MoveResult());
 
-  ArrayVector out_arrays(aggregates.size() + key_names.size());
   const auto& output_schema = plan->nodes()[0]->output()->output_schema();
+  if (!segmented) {
+    return MakeGroupByOutput(output_batches, output_schema, aggregates.size(),
+                             key_names.size(), naive);
+  }
+
+  std::vector<ArrayVector> out_arrays(aggregates.size() + key_names.size() +
+                                      segment_key_names.size());
   for (size_t i = 0; i < out_arrays.size(); ++i) {
     std::vector<std::shared_ptr<Array>> arrays(output_batches.size());
     for (size_t j = 0; j < output_batches.size(); ++j) {
-      arrays[j] = output_batches[j].values[i].make_array();
+      auto& value = output_batches[j].values[i];
+      if (value.is_scalar()) {
+        ARROW_ASSIGN_OR_RAISE(
+            arrays[j], MakeArrayFromScalar(*value.scalar(), output_batches[j].length));
+      } else if (value.is_array()) {
+        arrays[j] = value.make_array();
+      } else {
+        return Status::Invalid("GroupByUsingExecPlan unsupported value kind ",
+                               ToString(value.kind()));
+      }
     }
     if (arrays.empty()) {
+      arrays.resize(1);
       ARROW_ASSIGN_OR_RAISE(
-          out_arrays[i],
-          MakeArrayOfNull(output_schema->field(static_cast<int>(i))->type(),
-                          /*length=*/0));
-    } else {
-      ARROW_ASSIGN_OR_RAISE(out_arrays[i], Concatenate(arrays));
+          arrays[0], MakeArrayOfNull(output_schema->field(static_cast<int>(i))->type(),
+                                     /*length=*/0));
     }
+    out_arrays[i] = {std::move(arrays)};
   }
 
-  // The exec plan may reorder the output rows.  The tests are all setup to expect ouptut
-  // in ascending order of keys.  So we need to sort the result by the key columns.  To do
-  // that we create a table using the key columns, calculate the sort indices from that
-  // table (sorting on all fields) and then use those indices to calculate our result.
-  std::vector<std::shared_ptr<Field>> key_fields;
-  std::vector<std::shared_ptr<Array>> key_columns;
-  std::vector<SortKey> sort_keys;
-  for (std::size_t i = 0; i < key_names.size(); i++) {
-    const std::shared_ptr<Array>& arr = out_arrays[i + aggregates.size()];
-    if (arr->type_id() == Type::DICTIONARY) {
-      // Can't sort dictionary columns so need to decode
-      auto dict_arr = checked_pointer_cast<DictionaryArray>(arr);
-      ARROW_ASSIGN_OR_RAISE(auto decoded_arr,
-                            Take(*dict_arr->dictionary(), *dict_arr->indices()));
-      key_columns.push_back(decoded_arr);
-      key_fields.push_back(
-          field("name_does_not_matter", dict_arr->dict_type()->value_type()));
-    } else {
-      key_columns.push_back(arr);
-      key_fields.push_back(field("name_does_not_matter", arr->type()));
+  if (segmented && segment_key_names.size() > 0) {
+    ArrayVector struct_arrays;
+    struct_arrays.reserve(output_batches.size());
+    for (size_t j = 0; j < output_batches.size(); ++j) {
+      ArrayVector struct_fields;
+      struct_fields.reserve(out_arrays.size());
+      for (auto out_array : out_arrays) {
+        struct_fields.push_back(out_array[j]);
+      }
+      ARROW_ASSIGN_OR_RAISE(auto struct_array,
+                            StructArray::Make(struct_fields, output_schema->fields()));
+      struct_arrays.push_back(struct_array);
     }
-    sort_keys.emplace_back(static_cast<int>(i));
+    return ChunkedArray::Make(struct_arrays);
+  } else {
+    ArrayVector struct_fields(out_arrays.size());
+    for (size_t i = 0; i < out_arrays.size(); ++i) {
+      ARROW_ASSIGN_OR_RAISE(struct_fields[i], Concatenate(out_arrays[i]));
+    }
+    return StructArray::Make(std::move(struct_fields), output_schema->fields());
   }
-  std::shared_ptr<Schema> key_schema = schema(std::move(key_fields));
-  std::shared_ptr<Table> key_table = Table::Make(std::move(key_schema), key_columns);
-  SortOptions sort_options(std::move(sort_keys));
-  ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Array> sort_indices,
-                        SortIndices(key_table, sort_options));
+}
 
-  ARROW_ASSIGN_OR_RAISE(
-      std::shared_ptr<Array> struct_arr,
-      StructArray::Make(std::move(out_arrays), output_schema->fields()));
+Result<Datum> RunGroupBy(const BatchesWithSchema& input,
+                         const std::vector<std::string>& key_names,
+                         const std::vector<std::string>& segment_key_names,
+                         const std::vector<Aggregate>& aggregates, bool use_threads,
+                         bool segmented = false, bool naive = false) {
+  if (segment_key_names.size() > 0) {
+    ARROW_ASSIGN_OR_RAISE(auto thread_pool, arrow::internal::ThreadPool::Make(1));
+    ExecContext seq_ctx(default_memory_pool(), thread_pool.get());
+    return RunGroupBy(input, key_names, segment_key_names, aggregates, &seq_ctx,
+                      use_threads, segmented, naive);
+  } else {
+    return RunGroupBy(input, key_names, segment_key_names, aggregates,
+                      threaded_exec_context(), use_threads, segmented, naive);
+  }
+}
 
-  return Take(struct_arr, sort_indices);
+Result<Datum> RunGroupBy(const BatchesWithSchema& input,
+                         const std::vector<std::string>& key_names,
+                         const std::vector<Aggregate>& aggregates, bool use_threads,
+                         bool segmented = false, bool naive = false) {
+  return RunGroupBy(input, key_names, {}, aggregates, use_threads, segmented);
 }
 
 /// Simpler overload where you can give the columns as datums
 Result<Datum> RunGroupBy(const std::vector<Datum>& arguments,
                          const std::vector<Datum>& keys,
-                         const std::vector<Aggregate>& aggregates,
-                         bool use_threads = false) {
+                         const std::vector<Datum>& segment_keys,
+                         const std::vector<Aggregate>& aggregates, bool use_threads,
+                         bool segmented = false, bool naive = false) {
   using arrow::compute::detail::ExecSpanIterator;
 
-  FieldVector scan_fields(arguments.size() + keys.size());
+  FieldVector scan_fields(arguments.size() + keys.size() + segment_keys.size());
   std::vector<std::string> key_names(keys.size());
+  std::vector<std::string> segment_key_names(segment_keys.size());
   for (size_t i = 0; i < arguments.size(); ++i) {
     auto name = std::string("agg_") + ToChars(i);
     scan_fields[i] = field(name, arguments[i].type());
   }
+  size_t base = arguments.size();
   for (size_t i = 0; i < keys.size(); ++i) {
     auto name = std::string("key_") + ToChars(i);
-    scan_fields[arguments.size() + i] = field(name, keys[i].type());
+    scan_fields[base + i] = field(name, keys[i].type());
     key_names[i] = std::move(name);
   }
+  base += keys.size();
+  size_t j = segmented ? keys.size() : keys.size();

Review Comment:
   Probably a text-replace silliness. I'll fix.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] icexelloss commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "icexelloss (via GitHub)" <gi...@apache.org>.
icexelloss commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1117290296


##########
cpp/src/arrow/compute/exec/aggregate_node.cc:
##########
@@ -169,17 +206,63 @@ void AggregatesToString(std::stringstream* ss, const Schema& input_schema,
   *ss << ']';
 }
 
+template <typename BatchHandler>
+Status HandleSegments(std::unique_ptr<GroupingSegmenter>& segmenter,
+                      const ExecBatch& batch, const std::vector<int>& ids,
+                      const BatchHandler& handle_batch) {
+  int64_t offset = 0;
+  ARROW_ASSIGN_OR_RAISE(auto segment_batch, batch.SelectValues(ids));
+  while (true) {
+    ARROW_ASSIGN_OR_RAISE(auto segment, segmenter->GetNextSegment(segment_batch, offset));
+    if (segment.offset >= segment_batch.length) break;
+    ARROW_RETURN_NOT_OK(handle_batch(batch, segment));
+    offset = segment.offset + segment.length;
+  }
+  return Status::OK();
+}
+
+Status GetScalarFields(std::vector<Datum>& values, const ExecBatch& input_batch,

Review Comment:
   Do we expect this to modifying "values"? The function is called "GetScalarFields" which seems like a non-modifying method



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] icexelloss commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "icexelloss (via GitHub)" <gi...@apache.org>.
icexelloss commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1117335444


##########
cpp/src/arrow/compute/exec/aggregate_node.cc:
##########
@@ -282,12 +396,19 @@ class ScalarAggregateNode : public ExecNode, public TracedNode {
     auto scope = TraceInputReceived(batch);
     DCHECK_EQ(input, inputs_[0]);
 
-    auto thread_index = plan_->query_context()->GetThreadIndex();
-
-    ARROW_RETURN_NOT_OK(DoConsume(ExecSpan(batch), thread_index));
+    auto handler = [this](const ExecBatch& full_batch, const GroupingSegment& segment) {
+      if (!segment.extends && segment.offset == 0) RETURN_NOT_OK(OutputResult());
+      auto exec_batch = full_batch.Slice(segment.offset, segment.length);
+      auto batch = ExecSpan(exec_batch);
+      RETURN_NOT_OK(DoConsume(batch, plan_->query_context()->GetThreadIndex()));

Review Comment:
   This seems unnecessary changes, before the code was
   
   ```
       auto thread_index = plan_->query_context()->GetThreadIndex();
       ARROW_RETURN_NOT_OK(DoConsume(ExecSpan(batch), thread_index));
   ```
   
   ```
         auto batch = ExecSpan(exec_batch);
         RETURN_NOT_OK(DoConsume(batch, plan_->query_context()->GetThreadIndex()));
   ```
   
   Seems exactly the same thing?
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] icexelloss commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "icexelloss (via GitHub)" <gi...@apache.org>.
icexelloss commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1117329423


##########
cpp/src/arrow/compute/exec/aggregate_node.cc:
##########
@@ -326,46 +446,86 @@ class ScalarAggregateNode : public ExecNode, public TracedNode {
   }
 
  private:
-  Status Finish() {
-    auto scope = TraceFinish();
+  Status ReconstructAggregates() {
+    const auto& input_schema = *inputs()[0]->output_schema();
+    auto exec_ctx = plan()->query_context()->exec_context();
+    for (size_t i = 0; i < kernels_.size(); ++i) {
+      std::vector<TypeHolder> in_types;

Review Comment:
   Do we not have this variable saved somewhere? Since this wouldn't change between segments it feels weird to recompute this every time we reset the state



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] icexelloss commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "icexelloss (via GitHub)" <gi...@apache.org>.
icexelloss commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1116042562


##########
cpp/src/arrow/compute/row/grouper.h:
##########
@@ -39,10 +82,19 @@ class ARROW_EXPORT Grouper {
   static Result<std::unique_ptr<Grouper>> Make(const std::vector<TypeHolder>& key_types,
                                                ExecContext* ctx = default_exec_context());
 
-  /// Consume a batch of keys, producing the corresponding group ids as an integer array.
+  /// Consume a batch of keys, producing the corresponding group ids as an integer array,
+  /// over a slice defined by an offset and length, which defaults to the batch length.
+  /// Currently only uint32 indices will be produced, eventually the bit width will only
+  /// be as wide as necessary.
+  virtual Result<Datum> Consume(const ExecSpan& batch, int64_t consume_offset = 0,

Review Comment:
   What are these changes for?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] icexelloss commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "icexelloss (via GitHub)" <gi...@apache.org>.
icexelloss commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1116049996


##########
cpp/src/arrow/compute/exec/aggregate_node.cc:
##########
@@ -570,8 +773,12 @@ class GroupByNode : public ExecNode, public TracedNode {
     return output_->InputReceived(this, out_data_.Slice(batch_size * n, batch_size));
   }
 
-  Status OutputResult() {
-    auto scope = TraceFinish();
+  Status OutputResult(bool is_last = false, bool traced = false) {

Review Comment:
   What does is_last and traced do here?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rtpsw commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "rtpsw (via GitHub)" <gi...@apache.org>.
rtpsw commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1116160373


##########
cpp/src/arrow/compute/row/grouper.h:
##########
@@ -39,10 +82,19 @@ class ARROW_EXPORT Grouper {
   static Result<std::unique_ptr<Grouper>> Make(const std::vector<TypeHolder>& key_types,
                                                ExecContext* ctx = default_exec_context());
 
-  /// Consume a batch of keys, producing the corresponding group ids as an integer array.
+  /// Consume a batch of keys, producing the corresponding group ids as an integer array,
+  /// over a slice defined by an offset and length, which defaults to the batch length.
+  /// Currently only uint32 indices will be produced, eventually the bit width will only
+  /// be as wide as necessary.
+  virtual Result<Datum> Consume(const ExecSpan& batch, int64_t consume_offset = 0,

Review Comment:
   This supports consuming part of a batch at a time. The boundaries for consuming are determined by [segmentation](https://github.com/apache/arrow/pull/34311#issuecomment-1442318634). When a segment boundary is reached, the segmented aggregation can output the aggregation state accumulated thus far.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] westonpace commented on pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "westonpace (via GitHub)" <gi...@apache.org>.
westonpace commented on PR #34311:
URL: https://github.com/apache/arrow/pull/34311#issuecomment-1442642678

   Also, can we add at least one or two basic end-to-end tests in `plan_test.cc` (or you could create an `aggregate_node_test.cc`).  Partly as examples for future readers as much as anything.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] icexelloss commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "icexelloss (via GitHub)" <gi...@apache.org>.
icexelloss commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1128375752


##########
cpp/src/arrow/compute/kernels/hash_aggregate_test.cc:
##########
@@ -301,53 +437,249 @@ Result<Datum> GroupByTest(const std::vector<Datum>& arguments,
         {t_agg.function, t_agg.options, "agg_" + ToChars(idx), t_agg.function});
     idx = idx + 1;
   }
-  return RunGroupBy(arguments, keys, internal_aggregates, use_threads);
+  return group_by(arguments, keys, segment_keys, internal_aggregates, use_threads,
+                  /*naive=*/false);
 }
 
-}  // namespace
+Result<Datum> GroupByTest(GroupByFunction group_by, const std::vector<Datum>& arguments,
+                          const std::vector<Datum>& keys,
+                          const std::vector<TestAggregate>& aggregates,
+                          bool use_threads) {
+  return GroupByTest(group_by, arguments, keys, {}, aggregates, use_threads);
+}
 
-TEST(Grouper, SupportedKeys) {
-  ASSERT_OK(Grouper::Make({boolean()}));
+template <typename GroupClass>
+void TestGroupClassSupportedKeys(
+    std::function<Result<std::unique_ptr<GroupClass>>(const std::vector<TypeHolder>&)>
+        make_func) {
+  ASSERT_OK(make_func({boolean()}));
 
-  ASSERT_OK(Grouper::Make({int8(), uint16(), int32(), uint64()}));
+  ASSERT_OK(make_func({int8(), uint16(), int32(), uint64()}));
 
-  ASSERT_OK(Grouper::Make({dictionary(int64(), utf8())}));
+  ASSERT_OK(make_func({dictionary(int64(), utf8())}));
 
-  ASSERT_OK(Grouper::Make({float16(), float32(), float64()}));
+  ASSERT_OK(make_func({float16(), float32(), float64()}));
 
-  ASSERT_OK(Grouper::Make({utf8(), binary(), large_utf8(), large_binary()}));
+  ASSERT_OK(make_func({utf8(), binary(), large_utf8(), large_binary()}));
 
-  ASSERT_OK(Grouper::Make({fixed_size_binary(16), fixed_size_binary(32)}));
+  ASSERT_OK(make_func({fixed_size_binary(16), fixed_size_binary(32)}));
 
-  ASSERT_OK(Grouper::Make({decimal128(32, 10), decimal256(76, 20)}));
+  ASSERT_OK(make_func({decimal128(32, 10), decimal256(76, 20)}));
 
-  ASSERT_OK(Grouper::Make({date32(), date64()}));
+  ASSERT_OK(make_func({date32(), date64()}));
 
   for (auto unit : {
            TimeUnit::SECOND,
            TimeUnit::MILLI,
            TimeUnit::MICRO,
            TimeUnit::NANO,
        }) {
-    ASSERT_OK(Grouper::Make({timestamp(unit), duration(unit)}));
+    ASSERT_OK(make_func({timestamp(unit), duration(unit)}));
   }
 
   ASSERT_OK(
-      Grouper::Make({day_time_interval(), month_interval(), month_day_nano_interval()}));
+      make_func({day_time_interval(), month_interval(), month_day_nano_interval()}));
+
+  ASSERT_OK(make_func({null()}));
+
+  ASSERT_RAISES(NotImplemented, make_func({struct_({field("", int64())})}));
+
+  ASSERT_RAISES(NotImplemented, make_func({struct_({})}));
+
+  ASSERT_RAISES(NotImplemented, make_func({list(int32())}));
+
+  ASSERT_RAISES(NotImplemented, make_func({fixed_size_list(int32(), 5)}));
+
+  ASSERT_RAISES(NotImplemented, make_func({dense_union({field("", int32())})}));
+}
+
+void TestSegments(std::unique_ptr<RowSegmenter>& segmenter, const ExecSpan& batch,
+                  std::vector<Segment> expected_segments) {
+  int64_t offset = 0, segment_num = 0;
+  for (auto expected_segment : expected_segments) {
+    SCOPED_TRACE("segment #" + ToChars(segment_num++));
+    ASSERT_OK_AND_ASSIGN(auto segment, segmenter->GetNextSegment(batch, offset));
+    ASSERT_EQ(expected_segment, segment);
+    offset = segment.offset + segment.length;
+  }
+}
+
+Result<std::unique_ptr<Grouper>> MakeGrouper(const std::vector<TypeHolder>& key_types) {
+  return Grouper::Make(key_types, default_exec_context());
+}
+
+Result<std::unique_ptr<RowSegmenter>> MakeRowSegmenter(
+    const std::vector<TypeHolder>& key_types) {
+  return RowSegmenter::Make(key_types, /*nullable_leys=*/false, default_exec_context());
+}
+
+Result<std::unique_ptr<RowSegmenter>> MakeGenericSegmenter(
+    const std::vector<TypeHolder>& key_types) {
+  return MakeAnyKeysSegmenter(key_types, default_exec_context());
+}
+
+}  // namespace
+
+TEST(RowSegmenter, SupportedKeys) {
+  TestGroupClassSupportedKeys<RowSegmenter>(MakeRowSegmenter);
+}
+
+TEST(RowSegmenter, Basics) {
+  std::vector<TypeHolder> bad_types2 = {int32(), float32()};
+  std::vector<TypeHolder> types2 = {int32(), int32()};
+  std::vector<TypeHolder> bad_types1 = {float32()};
+  std::vector<TypeHolder> types1 = {int32()};
+  std::vector<TypeHolder> types0 = {};
+  auto batch2 = ExecBatchFromJSON(types2, "[[1, 1], [1, 2], [2, 2]]");

Review Comment:
   Can you add a few more test cases here? 
   A few ideas:
   (1) Non-ordered segment case, e.g., `1, 1, 2, 2, 1, 1`
   (2) Empty batches (This should included in the end-to-end test)
   (3) More than 2 segments inside one partition
   
   I suggest try to cover as much edge case you can here



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] icexelloss merged pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "icexelloss (via GitHub)" <gi...@apache.org>.
icexelloss merged PR #34311:
URL: https://github.com/apache/arrow/pull/34311


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] westonpace commented on pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "westonpace (via GitHub)" <gi...@apache.org>.
westonpace commented on PR #34311:
URL: https://github.com/apache/arrow/pull/34311#issuecomment-1460633135

   From my perspective, yes.  We can merge once @icexelloss approves.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] github-actions[bot] commented on pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "github-actions[bot] (via GitHub)" <gi...@apache.org>.
github-actions[bot] commented on PR #34311:
URL: https://github.com/apache/arrow/pull/34311#issuecomment-1463883390

   :warning: GitHub issue #32884 **has been automatically assigned in GitHub** to PR creator.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] icexelloss commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "icexelloss (via GitHub)" <gi...@apache.org>.
icexelloss commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1128383202


##########
cpp/src/arrow/compute/kernels/hash_aggregate_test.cc:
##########
@@ -4202,5 +4551,260 @@ TEST(GroupBy, OnlyKeys) {
                       /*verbose=*/true);
   }
 }
+
+INSTANTIATE_TEST_SUITE_P(GroupBy, GroupBy, ::testing::Values(RunGroupByImpl));
+
+class SegmentedScalarGroupBy : public GroupBy {};
+
+class SegmentedKeyGroupBy : public GroupBy {};
+
+void TestSegment(GroupByFunction group_by, const std::shared_ptr<Table>& table,
+                 Datum output, const std::vector<Datum>& keys,
+                 const std::vector<Datum>& segment_keys, bool scalar) {
+  const char* names[] = {
+      scalar ? "count" : "hash_count",
+      scalar ? "sum" : "hash_sum",
+      scalar ? "min_max" : "hash_min_max",
+  };
+  ASSERT_OK_AND_ASSIGN(Datum aggregated_and_grouped,
+                       group_by(
+                           {
+                               table->GetColumnByName("argument"),
+                               table->GetColumnByName("argument"),
+                               table->GetColumnByName("argument"),
+                           },
+                           keys, segment_keys,
+                           {
+                               {names[0], nullptr, "agg_0", names[0]},
+                               {names[1], nullptr, "agg_1", names[1]},
+                               {names[2], nullptr, "agg_2", names[2]},
+                           },
+                           kDefaultUseThreads, /*naive=*/false));
+
+  AssertDatumsEqual(output, aggregated_and_grouped, /*verbose=*/true);
+}
+
+void TestSegmentScalar(GroupByFunction group_by, const std::shared_ptr<Table>& table,

Review Comment:
   I see - can you put a comment to clarify please?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] icexelloss commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "icexelloss (via GitHub)" <gi...@apache.org>.
icexelloss commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1116052416


##########
cpp/src/arrow/compute/row/grouper.h:
##########
@@ -30,6 +30,49 @@
 namespace arrow {
 namespace compute {
 
+/// \brief A segment of contiguous rows for grouping
+struct ARROW_EXPORT GroupingSegment {
+  int64_t offset;
+  int64_t length;
+  bool is_open;
+  bool extends;
+};
+
+inline bool operator==(const GroupingSegment& segment1, const GroupingSegment& segment2) {
+  return segment1.offset == segment2.offset && segment1.length == segment2.length &&
+         segment1.is_open == segment2.is_open && segment1.extends == segment2.extends;
+}
+inline bool operator!=(const GroupingSegment& segment1, const GroupingSegment& segment2) {
+  return !(segment1 == segment2);
+}
+
+/// \brief Computes grouping segments for a batch. Each segment covers rows with identical
+/// values in the batch. The values in the batch are often selected as keys from a larger
+/// batch.
+class ARROW_EXPORT GroupingSegmenter {

Review Comment:
   Can you add in the doc what is a "segment"? I am not following what it means here:
   
   > Each segment covers rows with identical values in the batch



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rtpsw commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "rtpsw (via GitHub)" <gi...@apache.org>.
rtpsw commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1116869071


##########
cpp/src/arrow/compute/row/grouper.h:
##########
@@ -30,6 +30,49 @@
 namespace arrow {
 namespace compute {
 
+/// \brief A segment of contiguous rows for grouping
+struct ARROW_EXPORT GroupingSegment {
+  int64_t offset;
+  int64_t length;
+  bool is_open;
+  bool extends;
+};
+
+inline bool operator==(const GroupingSegment& segment1, const GroupingSegment& segment2) {
+  return segment1.offset == segment2.offset && segment1.length == segment2.length &&
+         segment1.is_open == segment2.is_open && segment1.extends == segment2.extends;
+}
+inline bool operator!=(const GroupingSegment& segment1, const GroupingSegment& segment2) {
+  return !(segment1 == segment2);
+}
+
+/// \brief Computes grouping segments for a batch. Each segment covers rows with identical
+/// values in the batch. The values in the batch are often selected as keys from a larger
+/// batch.
+class ARROW_EXPORT GroupingSegmenter {
+ public:
+  virtual ~GroupingSegmenter() = default;
+
+  /// \brief Construct a GroupingSegmenter which receives the specified key types
+  static Result<std::unique_ptr<GroupingSegmenter>> Make(
+      const std::vector<TypeHolder>& key_types, bool nullable_keys = false,
+      ExecContext* ctx = default_exec_context());
+
+  /// \brief Return the key types of this segmenter
+  virtual const std::vector<TypeHolder>& key_types() const = 0;
+
+  /// \brief Reset this grouping segmenter
+  virtual Status Reset() = 0;
+
+  /// \brief Get the next segment for the given batch starting from the given offset
+  virtual Result<GroupingSegment> GetNextSegment(const ExecSpan& batch,
+                                                 int64_t offset) = 0;
+
+  /// \brief Get the next segment for the given batch starting from the given offset
+  virtual Result<GroupingSegment> GetNextSegment(const ExecBatch& batch,

Review Comment:
   An `ExecSpan` is [a view](https://github.com/apache/arrow/blob/e8e50a3ca96ec8eb2f2beb383cf9005ce1c93486/cpp/src/arrow/compute/exec.h#L348-L350) into an `ExecBatch`. The latter owns the memory of its members while the latter doesn't. Another difference is that an `ExecSpan` cannot be chunked, as its values (i.e., columns) can only be a view into an array, but `ExecBatch` can have chunked array values.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] westonpace commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "westonpace (via GitHub)" <gi...@apache.org>.
westonpace commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1116321964


##########
cpp/src/arrow/compute/row/grouper.h:
##########
@@ -30,6 +30,49 @@
 namespace arrow {
 namespace compute {
 
+/// \brief A segment of contiguous rows for grouping
+struct ARROW_EXPORT GroupingSegment {
+  int64_t offset;
+  int64_t length;
+  bool is_open;
+  bool extends;
+};
+
+inline bool operator==(const GroupingSegment& segment1, const GroupingSegment& segment2) {
+  return segment1.offset == segment2.offset && segment1.length == segment2.length &&
+         segment1.is_open == segment2.is_open && segment1.extends == segment2.extends;
+}
+inline bool operator!=(const GroupingSegment& segment1, const GroupingSegment& segment2) {
+  return !(segment1 == segment2);
+}
+
+/// \brief Computes grouping segments for a batch. Each segment covers rows with identical
+/// values in the batch. The values in the batch are often selected as keys from a larger
+/// batch.
+class ARROW_EXPORT GroupingSegmenter {
+ public:
+  virtual ~GroupingSegmenter() = default;
+
+  /// \brief Construct a GroupingSegmenter which receives the specified key types

Review Comment:
   ```suggestion
     /// \brief Construct a GroupingSegmenter which segments on the specified key types
   ```



##########
cpp/src/arrow/compute/exec/options.h:
##########
@@ -199,21 +199,32 @@ class ARROW_EXPORT ProjectNodeOptions : public ExecNodeOptions {
   std::vector<std::string> names;
 };
 
-/// \brief Make a node which aggregates input batches, optionally grouped by keys.
+/// \brief Make a node which aggregates input batches, optionally grouped by keys and
+/// optionally segmented by segment-keys. Both keys and segment-keys determine the group.
+/// However segment-keys are also used for determining grouping segments, which should be
+/// large, and allow streaming a partial aggregation result after processing each segment.
+/// One common use-case for segment-keys is ordered aggregation, in which the segment-key
+/// attribute specifies a column with non-decreasing values or a lexigographically-ordered
+/// set of such columns.
 ///
 /// If the keys attribute is a non-empty vector, then each aggregate in `aggregates` is
 /// expected to be a HashAggregate function. If the keys attribute is an empty vector,
 /// then each aggregate is assumed to be a ScalarAggregate function.
 class ARROW_EXPORT AggregateNodeOptions : public ExecNodeOptions {
  public:
   explicit AggregateNodeOptions(std::vector<Aggregate> aggregates,
-                                std::vector<FieldRef> keys = {})
-      : aggregates(std::move(aggregates)), keys(std::move(keys)) {}
+                                std::vector<FieldRef> keys = {},
+                                std::vector<FieldRef> segment_keys = {})
+      : aggregates(std::move(aggregates)),
+        keys(std::move(keys)),
+        segment_keys(std::move(segment_keys)) {}
 
   // aggregations which will be applied to the targetted fields
   std::vector<Aggregate> aggregates;
   // keys by which aggregations will be grouped
   std::vector<FieldRef> keys;
+  // keys by which aggregations will be segmented
+  std::vector<FieldRef> segment_keys;

Review Comment:
   Is it ok if an entry in `segment_keys` and `keys` reference the same field (I would expect this to be a pretty common case)?  Can you test this?
   
   I think all of the following would be valid:
   
   group by x,y and segment by x,y,z // Maybe this isn't valid?
   group by x,y,z and segment by x,y
   group by x,y and segment by x,y
   group by x,y and segment by a,b
   group by x,y and segment by a,x // ?



##########
cpp/src/arrow/compute/kernels/hash_aggregate_test.cc:
##########
@@ -301,53 +426,184 @@ Result<Datum> GroupByTest(const std::vector<Datum>& arguments,
         {t_agg.function, t_agg.options, "agg_" + ToChars(idx), t_agg.function});
     idx = idx + 1;
   }
-  return RunGroupBy(arguments, keys, internal_aggregates, use_threads);
+  return group_by(arguments, keys, segment_keys, internal_aggregates, use_threads,
+                  /*naive=*/false);
 }
 
-}  // namespace
+Result<Datum> GroupByTest(GroupByFunction group_by, const std::vector<Datum>& arguments,
+                          const std::vector<Datum>& keys,
+                          const std::vector<TestAggregate>& aggregates,
+                          bool use_threads) {
+  return GroupByTest(group_by, arguments, keys, {}, aggregates, use_threads);
+}
 
-TEST(Grouper, SupportedKeys) {
-  ASSERT_OK(Grouper::Make({boolean()}));
+template <typename GroupClass>

Review Comment:
   It's a little confusing to see `GroupingSegmenter` and `Grouper` used interchangeably here since they have completely different interfaces.  Though I suppose it works because they both have the same `Make` method?



##########
cpp/src/arrow/compute/row/grouper.cc:
##########
@@ -39,12 +43,521 @@
 namespace arrow {
 
 using internal::checked_cast;
+using internal::PrimitiveScalarBase;
 
 namespace compute {
 
 namespace {
 
-struct GrouperImpl : Grouper {
+constexpr uint32_t kNoGroupId = std::numeric_limits<uint32_t>::max();
+
+using group_id_t = std::remove_const<decltype(kNoGroupId)>::type;
+using GroupIdType = CTypeTraits<group_id_t>::ArrowType;
+auto group_id_type = std::make_shared<GroupIdType>();
+
+inline const uint8_t* GetValuesAsBytes(const ArrayData& data, int64_t offset = 0) {
+  DCHECK_GT(data.type->byte_width(), 0);
+  int64_t absolute_byte_offset = (data.offset + offset) * data.type->byte_width();
+  return data.GetValues<uint8_t>(1, absolute_byte_offset);
+}
+
+inline const uint8_t* GetValuesAsBytes(const ArraySpan& data, int64_t offset = 0) {
+  DCHECK_GT(data.type->byte_width(), 0);
+  int64_t absolute_byte_offset = (data.offset + offset) * data.type->byte_width();
+  return data.GetValues<uint8_t>(1, absolute_byte_offset);
+}
+
+template <typename Value>
+Status CheckForGetNextSegment(const std::vector<Value>& values, int64_t length,
+                              int64_t offset, const std::vector<TypeHolder>& key_types) {
+  if (offset < 0 || offset > length) {
+    return Status::Invalid("invalid grouping segmenter offset: ", offset);
+  }
+  if (values.size() != key_types.size()) {
+    return Status::Invalid("expected batch size ", key_types.size(), " but got ",
+                           values.size());
+  }
+  for (size_t i = 0; i < key_types.size(); i++) {
+    const auto& value = values[i];
+    const auto& key_type = key_types[i];
+    if (*value.type() != *key_type.type) {
+      return Status::Invalid("expected batch value ", i, " of type ", *key_type.type,
+                             " but got ", *value.type());
+    }
+  }
+  return Status::OK();
+}
+
+template <typename Batch>
+enable_if_t<std::is_same<Batch, ExecSpan>::value || std::is_same<Batch, ExecBatch>::value,
+            Status>
+CheckForGetNextSegment(const Batch& batch, int64_t offset,
+                       const std::vector<TypeHolder>& key_types) {
+  return CheckForGetNextSegment(batch.values, batch.length, offset, key_types);
+}
+
+struct BaseGroupingSegmenter : public GroupingSegmenter {
+  explicit BaseGroupingSegmenter(const std::vector<TypeHolder>& key_types)
+      : key_types_(key_types) {}
+
+  const std::vector<TypeHolder>& key_types() const override { return key_types_; }
+
+  std::vector<TypeHolder> key_types_;
+};
+
+GroupingSegment MakeSegment(int64_t batch_length, int64_t offset, int64_t length,
+                            bool extends) {
+  return GroupingSegment{offset, length, offset + length >= batch_length, extends};
+}
+
+int64_t GetMatchLength(const uint8_t* match_bytes, int64_t match_width,

Review Comment:
   Could you use a binary search here?



##########
cpp/src/arrow/compute/exec/options.h:
##########
@@ -199,21 +199,32 @@ class ARROW_EXPORT ProjectNodeOptions : public ExecNodeOptions {
   std::vector<std::string> names;
 };
 
-/// \brief Make a node which aggregates input batches, optionally grouped by keys.
+/// \brief Make a node which aggregates input batches, optionally grouped by keys and
+/// optionally segmented by segment-keys. Both keys and segment-keys determine the group.
+/// However segment-keys are also used for determining grouping segments, which should be
+/// large, and allow streaming a partial aggregation result after processing each segment.
+/// One common use-case for segment-keys is ordered aggregation, in which the segment-key
+/// attribute specifies a column with non-decreasing values or a lexigographically-ordered
+/// set of such columns.
 ///
 /// If the keys attribute is a non-empty vector, then each aggregate in `aggregates` is
 /// expected to be a HashAggregate function. If the keys attribute is an empty vector,
 /// then each aggregate is assumed to be a ScalarAggregate function.
 class ARROW_EXPORT AggregateNodeOptions : public ExecNodeOptions {
  public:
   explicit AggregateNodeOptions(std::vector<Aggregate> aggregates,
-                                std::vector<FieldRef> keys = {})
-      : aggregates(std::move(aggregates)), keys(std::move(keys)) {}
+                                std::vector<FieldRef> keys = {},
+                                std::vector<FieldRef> segment_keys = {})
+      : aggregates(std::move(aggregates)),
+        keys(std::move(keys)),
+        segment_keys(std::move(segment_keys)) {}
 
   // aggregations which will be applied to the targetted fields
   std::vector<Aggregate> aggregates;
   // keys by which aggregations will be grouped
   std::vector<FieldRef> keys;
+  // keys by which aggregations will be segmented

Review Comment:
   Can we clarify that this is optional?



##########
cpp/src/arrow/compute/row/grouper.cc:
##########
@@ -39,12 +43,521 @@
 namespace arrow {
 
 using internal::checked_cast;
+using internal::PrimitiveScalarBase;
 
 namespace compute {
 
 namespace {
 
-struct GrouperImpl : Grouper {
+constexpr uint32_t kNoGroupId = std::numeric_limits<uint32_t>::max();
+
+using group_id_t = std::remove_const<decltype(kNoGroupId)>::type;
+using GroupIdType = CTypeTraits<group_id_t>::ArrowType;
+auto group_id_type = std::make_shared<GroupIdType>();
+
+inline const uint8_t* GetValuesAsBytes(const ArrayData& data, int64_t offset = 0) {
+  DCHECK_GT(data.type->byte_width(), 0);
+  int64_t absolute_byte_offset = (data.offset + offset) * data.type->byte_width();
+  return data.GetValues<uint8_t>(1, absolute_byte_offset);
+}
+
+inline const uint8_t* GetValuesAsBytes(const ArraySpan& data, int64_t offset = 0) {
+  DCHECK_GT(data.type->byte_width(), 0);
+  int64_t absolute_byte_offset = (data.offset + offset) * data.type->byte_width();
+  return data.GetValues<uint8_t>(1, absolute_byte_offset);
+}
+
+template <typename Value>
+Status CheckForGetNextSegment(const std::vector<Value>& values, int64_t length,
+                              int64_t offset, const std::vector<TypeHolder>& key_types) {
+  if (offset < 0 || offset > length) {
+    return Status::Invalid("invalid grouping segmenter offset: ", offset);
+  }
+  if (values.size() != key_types.size()) {
+    return Status::Invalid("expected batch size ", key_types.size(), " but got ",
+                           values.size());
+  }
+  for (size_t i = 0; i < key_types.size(); i++) {
+    const auto& value = values[i];
+    const auto& key_type = key_types[i];
+    if (*value.type() != *key_type.type) {
+      return Status::Invalid("expected batch value ", i, " of type ", *key_type.type,
+                             " but got ", *value.type());
+    }
+  }
+  return Status::OK();
+}
+
+template <typename Batch>
+enable_if_t<std::is_same<Batch, ExecSpan>::value || std::is_same<Batch, ExecBatch>::value,
+            Status>
+CheckForGetNextSegment(const Batch& batch, int64_t offset,
+                       const std::vector<TypeHolder>& key_types) {
+  return CheckForGetNextSegment(batch.values, batch.length, offset, key_types);
+}
+
+struct BaseGroupingSegmenter : public GroupingSegmenter {
+  explicit BaseGroupingSegmenter(const std::vector<TypeHolder>& key_types)
+      : key_types_(key_types) {}
+
+  const std::vector<TypeHolder>& key_types() const override { return key_types_; }
+
+  std::vector<TypeHolder> key_types_;
+};
+
+GroupingSegment MakeSegment(int64_t batch_length, int64_t offset, int64_t length,
+                            bool extends) {
+  return GroupingSegment{offset, length, offset + length >= batch_length, extends};
+}
+
+int64_t GetMatchLength(const uint8_t* match_bytes, int64_t match_width,
+                       const uint8_t* array_bytes, int64_t offset, int64_t length) {
+  int64_t cursor, byte_cursor;
+  for (cursor = offset, byte_cursor = match_width * cursor; cursor < length;
+       cursor++, byte_cursor += match_width) {
+    if (memcmp(match_bytes, array_bytes + byte_cursor,
+               static_cast<size_t>(match_width)) != 0) {
+      break;
+    }
+  }
+  return std::min(cursor, length - offset);
+}
+
+using ExtendFunc = std::function<bool(const void*)>;
+constexpr bool kDefaultExtends = true;
+constexpr bool kEmptyExtends = true;
+
+Result<GroupingSegment> GetNextSegmentChunked(
+    const std::shared_ptr<ChunkedArray>& chunked_array, int64_t offset,
+    ExtendFunc extend) {
+  if (offset >= chunked_array->length()) {
+    return MakeSegment(chunked_array->length(), chunked_array->length(), 0,
+                       kEmptyExtends);
+  }
+  int64_t remaining_offset = offset;
+  const auto& arrays = chunked_array->chunks();
+  for (size_t i = 0; remaining_offset >= 0 && i < arrays.size(); i++) {
+    // look up chunk containing offset
+    int64_t array_length = arrays[i]->length();
+    if (remaining_offset < array_length) {
+      // found - switch to matching
+      int64_t match_width = arrays[i]->type()->byte_width();
+      const uint8_t* match_bytes = GetValuesAsBytes(*arrays[i]->data(), remaining_offset);
+      int64_t total_match_length = 0;
+      for (; i < arrays.size(); i++) {
+        int64_t array_length = arrays[i]->length();
+        if (array_length <= 0) continue;
+        const uint8_t* array_bytes = GetValuesAsBytes(*arrays[i]->data());
+        int64_t match_length = GetMatchLength(match_bytes, match_width, array_bytes,
+                                              remaining_offset, array_length);
+        total_match_length += match_length;
+        remaining_offset = 0;
+        if (match_length < array_length - remaining_offset) break;
+      }
+      bool extends = extend(match_bytes);
+      return MakeSegment(chunked_array->length(), offset, total_match_length, extends);
+    }
+    remaining_offset -= array_length;
+  }
+  return Status::Invalid("segmenting invalid chunked array value");
+}
+
+struct NoKeysGroupingSegmenter : public BaseGroupingSegmenter {
+  static std::unique_ptr<GroupingSegmenter> Make() {
+    return std::make_unique<NoKeysGroupingSegmenter>();
+  }
+
+  NoKeysGroupingSegmenter() : BaseGroupingSegmenter({}) {}
+
+  Status Reset() override { return Status::OK(); }
+
+  template <typename Batch>
+  Result<GroupingSegment> GetNextSegmentImpl(const Batch& batch, int64_t offset) {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {}));
+    return MakeSegment(batch.length, offset, batch.length - offset, kDefaultExtends);
+  }
+
+  Result<GroupingSegment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    return GetNextSegmentImpl(batch, offset);
+  }
+
+  Result<GroupingSegment> GetNextSegment(const ExecBatch& batch,
+                                         int64_t offset) override {
+    return GetNextSegmentImpl(batch, offset);
+  }
+};
+
+struct SimpleKeyGroupingSegmenter : public BaseGroupingSegmenter {
+  static Result<std::unique_ptr<GroupingSegmenter>> Make(TypeHolder key_type) {
+    return std::make_unique<SimpleKeyGroupingSegmenter>(key_type);
+  }
+
+  explicit SimpleKeyGroupingSegmenter(TypeHolder key_type)
+      : BaseGroupingSegmenter({key_type}), key_type_(key_types_[0]), save_key_data_() {}
+
+  Status CheckType(const DataType& type) {
+    if (!is_fixed_width(type)) {
+      return Status::Invalid("SimpleKeyGroupingSegmenter does not support type ", type);
+    }
+    return Status::OK();
+  }
+
+  Status Reset() override {
+    save_key_data_.resize(0);
+    return Status::OK();
+  }
+
+  bool Extend(const void* data) {
+    size_t byte_width = static_cast<size_t>(key_type_.type->byte_width());
+    bool extends = save_key_data_.size() != byte_width
+                       ? kDefaultExtends
+                       : 0 == memcmp(save_key_data_.data(), data, byte_width);
+    save_key_data_.resize(byte_width);
+    memcpy(save_key_data_.data(), data, byte_width);
+    return extends;
+  }
+
+  Result<GroupingSegment> GetNextSegment(const Scalar& scalar, int64_t offset,
+                                         int64_t length) {
+    ARROW_RETURN_NOT_OK(CheckType(*scalar.type));
+    if (!scalar.is_valid) {
+      return Status::Invalid("segmenting an invalid scalar");
+    }
+    auto data = checked_cast<const PrimitiveScalarBase&>(scalar).data();
+    bool extends = length > 0 ? Extend(data) : kEmptyExtends;
+    return MakeSegment(length, 0, length, extends);
+  }
+
+  Result<GroupingSegment> GetNextSegment(const DataType& array_type,
+                                         const uint8_t* array_bytes, int64_t offset,
+                                         int64_t length) {
+    RETURN_NOT_OK(CheckType(array_type));
+    int64_t byte_width = array_type.byte_width();
+    int64_t match_length = GetMatchLength(array_bytes + offset * byte_width, byte_width,
+                                          array_bytes, offset, length);
+    bool extends = length > 0 ? Extend(array_bytes + offset * byte_width) : kEmptyExtends;
+    return MakeSegment(length, offset, match_length, extends);
+  }
+
+  Result<GroupingSegment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {key_type_}));
+    if (offset == batch.length) {
+      return MakeSegment(batch.length, offset, 0, kEmptyExtends);
+    }
+    const auto& value = batch.values[0];
+    if (value.is_scalar()) {
+      return GetNextSegment(*value.scalar, offset, batch.length);
+    }
+    ARROW_DCHECK(value.is_array());
+    const auto& array = value.array;
+    if (array.GetNullCount() > 0) {
+      return Status::NotImplemented("segmenting a nullable array");
+    }
+    return GetNextSegment(*array.type, GetValuesAsBytes(array), offset, batch.length);
+  }
+
+  Result<GroupingSegment> GetNextSegment(const ExecBatch& batch,
+                                         int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {key_type_}));
+    if (offset == batch.length) {
+      return MakeSegment(batch.length, offset, 0, kEmptyExtends);
+    }
+    const auto& value = batch.values[0];
+    if (value.is_scalar()) {
+      return GetNextSegment(*value.scalar(), offset, batch.length);
+    }
+    if (value.is_array()) {
+      auto array = value.array();
+      if (array->GetNullCount() > 0) {
+        return Status::NotImplemented("segmenting a nullable array");
+      }
+      return GetNextSegment(*array->type, GetValuesAsBytes(*array), offset, batch.length);
+    }
+    if (value.is_chunked_array()) {
+      auto array = value.chunked_array();
+      if (array->null_count() > 0) {
+        return Status::NotImplemented("segmenting a nullable array");
+      }
+      return GetNextSegmentChunked(array, offset, bound_extend_);
+    }
+    return Status::Invalid("segmenting unsupported value kind ", value.kind());
+  }
+
+ private:
+  TypeHolder key_type_;
+  std::vector<uint8_t> save_key_data_;
+  ExtendFunc bound_extend_ = [this](const void* data) { return Extend(data); };

Review Comment:
   If you get rid of `GetNextSegmentChunked` you can get rid of this too.



##########
cpp/src/arrow/compute/exec/aggregate_node.cc:
##########
@@ -326,46 +446,86 @@ class ScalarAggregateNode : public ExecNode, public TracedNode {
   }
 
  private:
-  Status Finish() {
-    auto scope = TraceFinish();
+  Status ReconstructAggregates() {
+    const auto& input_schema = *inputs()[0]->output_schema();
+    auto exec_ctx = plan()->query_context()->exec_context();
+    for (size_t i = 0; i < kernels_.size(); ++i) {
+      std::vector<TypeHolder> in_types;
+      for (const auto& target : target_fieldsets_[i]) {
+        in_types.emplace_back(input_schema.field(target)->type().get());
+      }
+      states_[i].resize(plan()->query_context()->max_concurrency());
+      KernelContext kernel_ctx{exec_ctx};
+      RETURN_NOT_OK(Kernel::InitAll(
+          &kernel_ctx, KernelInitArgs{kernels_[i], in_types, aggs_[i].options.get()},
+          &states_[i]));
+    }
+    return Status::OK();
+  }
+
+  Status OutputResult(bool is_last = false, bool traced = false) {
+    if (is_last && !traced) {
+      auto scope = TraceFinish();
+      return OutputResult(is_last, /*traced=*/true);
+    }
+    GatedUniqueLock lock(gated_shared_mutex_);
     ExecBatch batch{{}, 1};
-    batch.values.resize(kernels_.size());
+    batch.values.resize(kernels_.size() + segment_field_ids_.size());
 
     for (size_t i = 0; i < kernels_.size(); ++i) {
       util::tracing::Span span;
       START_COMPUTE_SPAN(span, aggs_[i].function,
                          {{"function.name", aggs_[i].function},
                           {"function.options",
                            aggs_[i].options ? aggs_[i].options->ToString() : "<NULLPTR>"},
-                          {"function.kind", std::string(kind_name()) + "::Finalize"}});
+                          {"function.kind", std::string(kind_name()) + "::Output"}});
       KernelContext ctx{plan()->query_context()->exec_context()};
       ARROW_ASSIGN_OR_RAISE(auto merged, ScalarAggregateKernel::MergeAll(
                                              kernels_[i], &ctx, std::move(states_[i])));
       RETURN_NOT_OK(kernels_[i]->finalize(&ctx, &batch.values[i]));
     }
+    PlaceFields(batch, kernels_.size(), segmenter_values_);
 
-    return output_->InputReceived(this, std::move(batch));
+    ARROW_RETURN_NOT_OK(output_->InputReceived(this, std::move(batch)));
+    total_output_batches_++;
+    if (is_last) {
+      ARROW_RETURN_NOT_OK(output_->InputFinished(this, total_output_batches_));
+    } else {
+      ARROW_RETURN_NOT_OK(ReconstructAggregates());
+    }
+    return Status::OK();
   }
 
+  std::unique_ptr<GroupingSegmenter> segmenter_;
+  const std::vector<int> segment_field_ids_;
+  std::vector<Datum> segmenter_values_;
+
   const std::vector<std::vector<int>> target_fieldsets_;
   const std::vector<Aggregate> aggs_;
   const std::vector<const ScalarAggregateKernel*> kernels_;
 
   std::vector<std::vector<std::unique_ptr<KernelState>>> states_;
 
   AtomicCounter input_counter_;
+  int64_t total_output_batches_ = 0;

Review Comment:
   This should probably be `int` as require batch counts to be `int` everywhere else.



##########
cpp/src/arrow/compute/row/grouper.h:
##########
@@ -30,6 +30,49 @@
 namespace arrow {
 namespace compute {
 
+/// \brief A segment of contiguous rows for grouping
+struct ARROW_EXPORT GroupingSegment {
+  int64_t offset;
+  int64_t length;
+  bool is_open;
+  bool extends;
+};
+
+inline bool operator==(const GroupingSegment& segment1, const GroupingSegment& segment2) {
+  return segment1.offset == segment2.offset && segment1.length == segment2.length &&
+         segment1.is_open == segment2.is_open && segment1.extends == segment2.extends;
+}
+inline bool operator!=(const GroupingSegment& segment1, const GroupingSegment& segment2) {
+  return !(segment1 == segment2);
+}
+
+/// \brief Computes grouping segments for a batch. Each segment covers rows with identical
+/// values in the batch. The values in the batch are often selected as keys from a larger
+/// batch.
+class ARROW_EXPORT GroupingSegmenter {
+ public:
+  virtual ~GroupingSegmenter() = default;
+
+  /// \brief Construct a GroupingSegmenter which receives the specified key types
+  static Result<std::unique_ptr<GroupingSegmenter>> Make(
+      const std::vector<TypeHolder>& key_types, bool nullable_keys = false,

Review Comment:
   Can we clarify what `nullable_keys` means?



##########
cpp/src/arrow/compute/kernels/hash_aggregate_test.cc:
##########
@@ -135,22 +141,84 @@ Result<Datum> NaiveGroupBy(std::vector<Datum> arguments, std::vector<Datum> keys
   return Take(struct_arr, sorted_indices);
 }
 
+Result<Datum> MakeGroupByOutput(const std::vector<ExecBatch>& output_batches,
+                                const std::shared_ptr<Schema> output_schema,
+                                size_t num_aggregates, size_t num_keys, bool naive) {
+  ArrayVector out_arrays(num_aggregates + num_keys);
+  for (size_t i = 0; i < out_arrays.size(); ++i) {
+    std::vector<std::shared_ptr<Array>> arrays(output_batches.size());
+    for (size_t j = 0; j < output_batches.size(); ++j) {
+      arrays[j] = output_batches[j].values[i].make_array();
+    }
+    if (arrays.empty()) {
+      ARROW_ASSIGN_OR_RAISE(
+          out_arrays[i],
+          MakeArrayOfNull(output_schema->field(static_cast<int>(i))->type(),
+                          /*length=*/0));
+    } else {
+      ARROW_ASSIGN_OR_RAISE(out_arrays[i], Concatenate(arrays));
+    }
+  }
+
+  ARROW_ASSIGN_OR_RAISE(
+      std::shared_ptr<Array> struct_arr,
+      StructArray::Make(std::move(out_arrays), output_schema->fields()));
+
+  bool need_sort = !naive;
+  for (size_t i = num_aggregates; need_sort && i < out_arrays.size(); i++) {
+    if (output_schema->field(i)->type()->id() == Type::DICTIONARY) {

Review Comment:
   ```suggestion
       if (output_schema->field(static_cast<int>(i))->type()->id() == Type::DICTIONARY) {
   ```



##########
cpp/src/arrow/compute/row/grouper.cc:
##########
@@ -39,12 +43,521 @@
 namespace arrow {
 
 using internal::checked_cast;
+using internal::PrimitiveScalarBase;
 
 namespace compute {
 
 namespace {
 
-struct GrouperImpl : Grouper {
+constexpr uint32_t kNoGroupId = std::numeric_limits<uint32_t>::max();
+
+using group_id_t = std::remove_const<decltype(kNoGroupId)>::type;
+using GroupIdType = CTypeTraits<group_id_t>::ArrowType;
+auto group_id_type = std::make_shared<GroupIdType>();
+
+inline const uint8_t* GetValuesAsBytes(const ArrayData& data, int64_t offset = 0) {
+  DCHECK_GT(data.type->byte_width(), 0);
+  int64_t absolute_byte_offset = (data.offset + offset) * data.type->byte_width();
+  return data.GetValues<uint8_t>(1, absolute_byte_offset);
+}
+
+inline const uint8_t* GetValuesAsBytes(const ArraySpan& data, int64_t offset = 0) {
+  DCHECK_GT(data.type->byte_width(), 0);
+  int64_t absolute_byte_offset = (data.offset + offset) * data.type->byte_width();
+  return data.GetValues<uint8_t>(1, absolute_byte_offset);
+}
+
+template <typename Value>
+Status CheckForGetNextSegment(const std::vector<Value>& values, int64_t length,
+                              int64_t offset, const std::vector<TypeHolder>& key_types) {
+  if (offset < 0 || offset > length) {
+    return Status::Invalid("invalid grouping segmenter offset: ", offset);
+  }
+  if (values.size() != key_types.size()) {
+    return Status::Invalid("expected batch size ", key_types.size(), " but got ",
+                           values.size());
+  }
+  for (size_t i = 0; i < key_types.size(); i++) {
+    const auto& value = values[i];
+    const auto& key_type = key_types[i];
+    if (*value.type() != *key_type.type) {
+      return Status::Invalid("expected batch value ", i, " of type ", *key_type.type,
+                             " but got ", *value.type());
+    }
+  }
+  return Status::OK();
+}
+
+template <typename Batch>
+enable_if_t<std::is_same<Batch, ExecSpan>::value || std::is_same<Batch, ExecBatch>::value,
+            Status>
+CheckForGetNextSegment(const Batch& batch, int64_t offset,
+                       const std::vector<TypeHolder>& key_types) {
+  return CheckForGetNextSegment(batch.values, batch.length, offset, key_types);
+}
+
+struct BaseGroupingSegmenter : public GroupingSegmenter {
+  explicit BaseGroupingSegmenter(const std::vector<TypeHolder>& key_types)
+      : key_types_(key_types) {}
+
+  const std::vector<TypeHolder>& key_types() const override { return key_types_; }
+
+  std::vector<TypeHolder> key_types_;
+};
+
+GroupingSegment MakeSegment(int64_t batch_length, int64_t offset, int64_t length,
+                            bool extends) {
+  return GroupingSegment{offset, length, offset + length >= batch_length, extends};
+}
+
+int64_t GetMatchLength(const uint8_t* match_bytes, int64_t match_width,
+                       const uint8_t* array_bytes, int64_t offset, int64_t length) {
+  int64_t cursor, byte_cursor;
+  for (cursor = offset, byte_cursor = match_width * cursor; cursor < length;
+       cursor++, byte_cursor += match_width) {
+    if (memcmp(match_bytes, array_bytes + byte_cursor,
+               static_cast<size_t>(match_width)) != 0) {
+      break;
+    }
+  }
+  return std::min(cursor, length - offset);
+}
+
+using ExtendFunc = std::function<bool(const void*)>;
+constexpr bool kDefaultExtends = true;
+constexpr bool kEmptyExtends = true;
+
+Result<GroupingSegment> GetNextSegmentChunked(
+    const std::shared_ptr<ChunkedArray>& chunked_array, int64_t offset,
+    ExtendFunc extend) {
+  if (offset >= chunked_array->length()) {
+    return MakeSegment(chunked_array->length(), chunked_array->length(), 0,
+                       kEmptyExtends);
+  }
+  int64_t remaining_offset = offset;
+  const auto& arrays = chunked_array->chunks();
+  for (size_t i = 0; remaining_offset >= 0 && i < arrays.size(); i++) {
+    // look up chunk containing offset
+    int64_t array_length = arrays[i]->length();
+    if (remaining_offset < array_length) {
+      // found - switch to matching
+      int64_t match_width = arrays[i]->type()->byte_width();
+      const uint8_t* match_bytes = GetValuesAsBytes(*arrays[i]->data(), remaining_offset);
+      int64_t total_match_length = 0;
+      for (; i < arrays.size(); i++) {
+        int64_t array_length = arrays[i]->length();
+        if (array_length <= 0) continue;
+        const uint8_t* array_bytes = GetValuesAsBytes(*arrays[i]->data());
+        int64_t match_length = GetMatchLength(match_bytes, match_width, array_bytes,
+                                              remaining_offset, array_length);
+        total_match_length += match_length;
+        remaining_offset = 0;
+        if (match_length < array_length - remaining_offset) break;
+      }
+      bool extends = extend(match_bytes);
+      return MakeSegment(chunked_array->length(), offset, total_match_length, extends);
+    }
+    remaining_offset -= array_length;
+  }
+  return Status::Invalid("segmenting invalid chunked array value");
+}
+
+struct NoKeysGroupingSegmenter : public BaseGroupingSegmenter {
+  static std::unique_ptr<GroupingSegmenter> Make() {
+    return std::make_unique<NoKeysGroupingSegmenter>();
+  }
+
+  NoKeysGroupingSegmenter() : BaseGroupingSegmenter({}) {}
+
+  Status Reset() override { return Status::OK(); }
+
+  template <typename Batch>
+  Result<GroupingSegment> GetNextSegmentImpl(const Batch& batch, int64_t offset) {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {}));
+    return MakeSegment(batch.length, offset, batch.length - offset, kDefaultExtends);
+  }
+
+  Result<GroupingSegment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    return GetNextSegmentImpl(batch, offset);
+  }
+
+  Result<GroupingSegment> GetNextSegment(const ExecBatch& batch,
+                                         int64_t offset) override {
+    return GetNextSegmentImpl(batch, offset);
+  }
+};
+
+struct SimpleKeyGroupingSegmenter : public BaseGroupingSegmenter {
+  static Result<std::unique_ptr<GroupingSegmenter>> Make(TypeHolder key_type) {
+    return std::make_unique<SimpleKeyGroupingSegmenter>(key_type);
+  }
+
+  explicit SimpleKeyGroupingSegmenter(TypeHolder key_type)
+      : BaseGroupingSegmenter({key_type}), key_type_(key_types_[0]), save_key_data_() {}
+
+  Status CheckType(const DataType& type) {
+    if (!is_fixed_width(type)) {
+      return Status::Invalid("SimpleKeyGroupingSegmenter does not support type ", type);
+    }
+    return Status::OK();
+  }
+
+  Status Reset() override {
+    save_key_data_.resize(0);
+    return Status::OK();
+  }
+
+  bool Extend(const void* data) {
+    size_t byte_width = static_cast<size_t>(key_type_.type->byte_width());
+    bool extends = save_key_data_.size() != byte_width
+                       ? kDefaultExtends
+                       : 0 == memcmp(save_key_data_.data(), data, byte_width);
+    save_key_data_.resize(byte_width);
+    memcpy(save_key_data_.data(), data, byte_width);
+    return extends;
+  }
+
+  Result<GroupingSegment> GetNextSegment(const Scalar& scalar, int64_t offset,
+                                         int64_t length) {
+    ARROW_RETURN_NOT_OK(CheckType(*scalar.type));
+    if (!scalar.is_valid) {
+      return Status::Invalid("segmenting an invalid scalar");
+    }
+    auto data = checked_cast<const PrimitiveScalarBase&>(scalar).data();
+    bool extends = length > 0 ? Extend(data) : kEmptyExtends;
+    return MakeSegment(length, 0, length, extends);
+  }
+
+  Result<GroupingSegment> GetNextSegment(const DataType& array_type,
+                                         const uint8_t* array_bytes, int64_t offset,
+                                         int64_t length) {
+    RETURN_NOT_OK(CheckType(array_type));
+    int64_t byte_width = array_type.byte_width();
+    int64_t match_length = GetMatchLength(array_bytes + offset * byte_width, byte_width,
+                                          array_bytes, offset, length);
+    bool extends = length > 0 ? Extend(array_bytes + offset * byte_width) : kEmptyExtends;
+    return MakeSegment(length, offset, match_length, extends);
+  }
+
+  Result<GroupingSegment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {key_type_}));
+    if (offset == batch.length) {
+      return MakeSegment(batch.length, offset, 0, kEmptyExtends);
+    }
+    const auto& value = batch.values[0];
+    if (value.is_scalar()) {
+      return GetNextSegment(*value.scalar, offset, batch.length);
+    }
+    ARROW_DCHECK(value.is_array());
+    const auto& array = value.array;
+    if (array.GetNullCount() > 0) {
+      return Status::NotImplemented("segmenting a nullable array");
+    }
+    return GetNextSegment(*array.type, GetValuesAsBytes(array), offset, batch.length);
+  }
+
+  Result<GroupingSegment> GetNextSegment(const ExecBatch& batch,
+                                         int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {key_type_}));
+    if (offset == batch.length) {
+      return MakeSegment(batch.length, offset, 0, kEmptyExtends);
+    }
+    const auto& value = batch.values[0];
+    if (value.is_scalar()) {
+      return GetNextSegment(*value.scalar(), offset, batch.length);
+    }
+    if (value.is_array()) {
+      auto array = value.array();
+      if (array->GetNullCount() > 0) {
+        return Status::NotImplemented("segmenting a nullable array");
+      }
+      return GetNextSegment(*array->type, GetValuesAsBytes(*array), offset, batch.length);
+    }
+    if (value.is_chunked_array()) {
+      auto array = value.chunked_array();
+      if (array->null_count() > 0) {
+        return Status::NotImplemented("segmenting a nullable array");
+      }
+      return GetNextSegmentChunked(array, offset, bound_extend_);
+    }
+    return Status::Invalid("segmenting unsupported value kind ", value.kind());
+  }
+
+ private:
+  TypeHolder key_type_;
+  std::vector<uint8_t> save_key_data_;
+  ExtendFunc bound_extend_ = [this](const void* data) { return Extend(data); };
+};
+
+struct AnyKeysGroupingSegmenter : public BaseGroupingSegmenter {
+  static Result<std::unique_ptr<GroupingSegmenter>> Make(
+      const std::vector<TypeHolder>& key_types, ExecContext* ctx) {
+    ARROW_RETURN_NOT_OK(Grouper::Make(key_types, ctx));  // check types
+    return std::make_unique<AnyKeysGroupingSegmenter>(key_types, ctx);
+  }
+
+  AnyKeysGroupingSegmenter(const std::vector<TypeHolder>& key_types, ExecContext* ctx)
+      : BaseGroupingSegmenter(key_types),
+        ctx_(ctx),
+        grouper_(nullptr),
+        save_group_id_(kNoGroupId) {}
+
+  Status Reset() override {
+    grouper_ = nullptr;
+    save_group_id_ = kNoGroupId;
+    return Status::OK();
+  }
+
+  bool Extend(const void* data) {
+    auto group_id = *static_cast<const group_id_t*>(data);
+    bool extends =
+        save_group_id_ == kNoGroupId ? kDefaultExtends : save_group_id_ == group_id;
+    save_group_id_ = group_id;
+    return extends;
+  }
+
+  template <typename Batch>
+  Result<group_id_t> MapGroupIdAt(const Batch& batch, int64_t offset) {
+    if (offset < 0 || offset >= batch.length) {
+      return Status::Invalid("requesting group id out of bounds");
+    }
+    if (!grouper_) return kNoGroupId;
+    ARROW_ASSIGN_OR_RAISE(auto datum, grouper_->Consume(batch, offset,
+                                                        /*consume_length=*/1));
+    if (!(datum.is_array() || datum.is_chunked_array())) {
+      return Status::Invalid("accessing unsupported datum kind ", datum.kind());
+    }
+    const std::shared_ptr<ArrayData>& data =
+        datum.is_array() ? datum.array() : datum.chunked_array()->chunk(0)->data();
+    ARROW_DCHECK(data->GetNullCount() == 0);
+    DCHECK_EQ(data->type->id(), GroupIdType::type_id);
+    DCHECK_EQ(1, data->length);
+    const group_id_t* values = data->GetValues<group_id_t>(1);
+    return values[0];
+  }
+
+  template <typename Batch>
+  Result<GroupingSegment> GetNextSegmentImpl(const Batch& batch, int64_t offset) {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, key_types_));
+    if (offset == batch.length) {
+      return MakeSegment(batch.length, offset, 0, kEmptyExtends);
+    }
+    // ARROW-18311: make Grouper support Reset()
+    // so it can be cached instead of recreated below
+    //
+    // the group id must be computed prior to resetting the grouper, since it is compared
+    // to save_group_id_, and after resetting the grouper produces incomparable group ids
+    ARROW_ASSIGN_OR_RAISE(auto group_id, MapGroupIdAt(batch, offset));
+    ExtendFunc bound_extend = [this, group_id](const void* data) {
+      bool extends = Extend(&group_id);
+      save_group_id_ = *static_cast<const group_id_t*>(data);
+      return extends;
+    };
+    ARROW_ASSIGN_OR_RAISE(grouper_, Grouper::Make(key_types_, ctx_));  // TODO: reset it
+    ARROW_ASSIGN_OR_RAISE(auto datum, grouper_->Consume(batch, offset));
+    if (datum.is_array()) {
+      const std::shared_ptr<ArrayData>& data = datum.array();
+      ARROW_DCHECK(data->GetNullCount() == 0);
+      DCHECK_EQ(data->type->id(), GroupIdType::type_id);
+      const group_id_t* values = data->GetValues<group_id_t>(1);
+      int64_t cursor;
+      for (cursor = 1; cursor < data->length; cursor++) {
+        if (values[0] != values[cursor]) break;

Review Comment:
   binary search?



##########
cpp/src/arrow/compute/row/grouper.cc:
##########
@@ -39,12 +43,521 @@
 namespace arrow {
 
 using internal::checked_cast;
+using internal::PrimitiveScalarBase;
 
 namespace compute {
 
 namespace {
 
-struct GrouperImpl : Grouper {
+constexpr uint32_t kNoGroupId = std::numeric_limits<uint32_t>::max();
+
+using group_id_t = std::remove_const<decltype(kNoGroupId)>::type;
+using GroupIdType = CTypeTraits<group_id_t>::ArrowType;
+auto group_id_type = std::make_shared<GroupIdType>();
+
+inline const uint8_t* GetValuesAsBytes(const ArrayData& data, int64_t offset = 0) {
+  DCHECK_GT(data.type->byte_width(), 0);
+  int64_t absolute_byte_offset = (data.offset + offset) * data.type->byte_width();
+  return data.GetValues<uint8_t>(1, absolute_byte_offset);
+}
+
+inline const uint8_t* GetValuesAsBytes(const ArraySpan& data, int64_t offset = 0) {
+  DCHECK_GT(data.type->byte_width(), 0);
+  int64_t absolute_byte_offset = (data.offset + offset) * data.type->byte_width();
+  return data.GetValues<uint8_t>(1, absolute_byte_offset);
+}
+
+template <typename Value>
+Status CheckForGetNextSegment(const std::vector<Value>& values, int64_t length,
+                              int64_t offset, const std::vector<TypeHolder>& key_types) {
+  if (offset < 0 || offset > length) {
+    return Status::Invalid("invalid grouping segmenter offset: ", offset);
+  }
+  if (values.size() != key_types.size()) {
+    return Status::Invalid("expected batch size ", key_types.size(), " but got ",
+                           values.size());
+  }
+  for (size_t i = 0; i < key_types.size(); i++) {
+    const auto& value = values[i];
+    const auto& key_type = key_types[i];
+    if (*value.type() != *key_type.type) {
+      return Status::Invalid("expected batch value ", i, " of type ", *key_type.type,
+                             " but got ", *value.type());
+    }
+  }
+  return Status::OK();
+}
+
+template <typename Batch>
+enable_if_t<std::is_same<Batch, ExecSpan>::value || std::is_same<Batch, ExecBatch>::value,
+            Status>
+CheckForGetNextSegment(const Batch& batch, int64_t offset,
+                       const std::vector<TypeHolder>& key_types) {
+  return CheckForGetNextSegment(batch.values, batch.length, offset, key_types);
+}
+
+struct BaseGroupingSegmenter : public GroupingSegmenter {
+  explicit BaseGroupingSegmenter(const std::vector<TypeHolder>& key_types)
+      : key_types_(key_types) {}
+
+  const std::vector<TypeHolder>& key_types() const override { return key_types_; }
+
+  std::vector<TypeHolder> key_types_;
+};
+
+GroupingSegment MakeSegment(int64_t batch_length, int64_t offset, int64_t length,
+                            bool extends) {
+  return GroupingSegment{offset, length, offset + length >= batch_length, extends};
+}
+
+int64_t GetMatchLength(const uint8_t* match_bytes, int64_t match_width,
+                       const uint8_t* array_bytes, int64_t offset, int64_t length) {
+  int64_t cursor, byte_cursor;
+  for (cursor = offset, byte_cursor = match_width * cursor; cursor < length;
+       cursor++, byte_cursor += match_width) {
+    if (memcmp(match_bytes, array_bytes + byte_cursor,
+               static_cast<size_t>(match_width)) != 0) {
+      break;
+    }
+  }
+  return std::min(cursor, length - offset);
+}
+
+using ExtendFunc = std::function<bool(const void*)>;
+constexpr bool kDefaultExtends = true;
+constexpr bool kEmptyExtends = true;
+
+Result<GroupingSegment> GetNextSegmentChunked(
+    const std::shared_ptr<ChunkedArray>& chunked_array, int64_t offset,
+    ExtendFunc extend) {
+  if (offset >= chunked_array->length()) {
+    return MakeSegment(chunked_array->length(), chunked_array->length(), 0,
+                       kEmptyExtends);
+  }
+  int64_t remaining_offset = offset;
+  const auto& arrays = chunked_array->chunks();
+  for (size_t i = 0; remaining_offset >= 0 && i < arrays.size(); i++) {
+    // look up chunk containing offset
+    int64_t array_length = arrays[i]->length();
+    if (remaining_offset < array_length) {
+      // found - switch to matching
+      int64_t match_width = arrays[i]->type()->byte_width();
+      const uint8_t* match_bytes = GetValuesAsBytes(*arrays[i]->data(), remaining_offset);
+      int64_t total_match_length = 0;
+      for (; i < arrays.size(); i++) {
+        int64_t array_length = arrays[i]->length();
+        if (array_length <= 0) continue;
+        const uint8_t* array_bytes = GetValuesAsBytes(*arrays[i]->data());
+        int64_t match_length = GetMatchLength(match_bytes, match_width, array_bytes,
+                                              remaining_offset, array_length);
+        total_match_length += match_length;
+        remaining_offset = 0;
+        if (match_length < array_length - remaining_offset) break;
+      }
+      bool extends = extend(match_bytes);
+      return MakeSegment(chunked_array->length(), offset, total_match_length, extends);
+    }
+    remaining_offset -= array_length;
+  }
+  return Status::Invalid("segmenting invalid chunked array value");
+}
+
+struct NoKeysGroupingSegmenter : public BaseGroupingSegmenter {
+  static std::unique_ptr<GroupingSegmenter> Make() {
+    return std::make_unique<NoKeysGroupingSegmenter>();
+  }
+
+  NoKeysGroupingSegmenter() : BaseGroupingSegmenter({}) {}
+
+  Status Reset() override { return Status::OK(); }
+
+  template <typename Batch>
+  Result<GroupingSegment> GetNextSegmentImpl(const Batch& batch, int64_t offset) {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {}));
+    return MakeSegment(batch.length, offset, batch.length - offset, kDefaultExtends);
+  }
+
+  Result<GroupingSegment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    return GetNextSegmentImpl(batch, offset);
+  }
+
+  Result<GroupingSegment> GetNextSegment(const ExecBatch& batch,
+                                         int64_t offset) override {
+    return GetNextSegmentImpl(batch, offset);
+  }
+};
+
+struct SimpleKeyGroupingSegmenter : public BaseGroupingSegmenter {
+  static Result<std::unique_ptr<GroupingSegmenter>> Make(TypeHolder key_type) {
+    return std::make_unique<SimpleKeyGroupingSegmenter>(key_type);
+  }
+
+  explicit SimpleKeyGroupingSegmenter(TypeHolder key_type)
+      : BaseGroupingSegmenter({key_type}), key_type_(key_types_[0]), save_key_data_() {}
+
+  Status CheckType(const DataType& type) {
+    if (!is_fixed_width(type)) {
+      return Status::Invalid("SimpleKeyGroupingSegmenter does not support type ", type);
+    }
+    return Status::OK();
+  }
+
+  Status Reset() override {
+    save_key_data_.resize(0);
+    return Status::OK();
+  }
+
+  bool Extend(const void* data) {
+    size_t byte_width = static_cast<size_t>(key_type_.type->byte_width());
+    bool extends = save_key_data_.size() != byte_width
+                       ? kDefaultExtends
+                       : 0 == memcmp(save_key_data_.data(), data, byte_width);
+    save_key_data_.resize(byte_width);
+    memcpy(save_key_data_.data(), data, byte_width);
+    return extends;
+  }
+
+  Result<GroupingSegment> GetNextSegment(const Scalar& scalar, int64_t offset,
+                                         int64_t length) {
+    ARROW_RETURN_NOT_OK(CheckType(*scalar.type));
+    if (!scalar.is_valid) {
+      return Status::Invalid("segmenting an invalid scalar");
+    }
+    auto data = checked_cast<const PrimitiveScalarBase&>(scalar).data();
+    bool extends = length > 0 ? Extend(data) : kEmptyExtends;
+    return MakeSegment(length, 0, length, extends);
+  }
+
+  Result<GroupingSegment> GetNextSegment(const DataType& array_type,
+                                         const uint8_t* array_bytes, int64_t offset,
+                                         int64_t length) {
+    RETURN_NOT_OK(CheckType(array_type));
+    int64_t byte_width = array_type.byte_width();
+    int64_t match_length = GetMatchLength(array_bytes + offset * byte_width, byte_width,
+                                          array_bytes, offset, length);
+    bool extends = length > 0 ? Extend(array_bytes + offset * byte_width) : kEmptyExtends;
+    return MakeSegment(length, offset, match_length, extends);
+  }
+
+  Result<GroupingSegment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {key_type_}));
+    if (offset == batch.length) {
+      return MakeSegment(batch.length, offset, 0, kEmptyExtends);
+    }
+    const auto& value = batch.values[0];
+    if (value.is_scalar()) {
+      return GetNextSegment(*value.scalar, offset, batch.length);
+    }
+    ARROW_DCHECK(value.is_array());
+    const auto& array = value.array;
+    if (array.GetNullCount() > 0) {
+      return Status::NotImplemented("segmenting a nullable array");
+    }
+    return GetNextSegment(*array.type, GetValuesAsBytes(array), offset, batch.length);
+  }
+
+  Result<GroupingSegment> GetNextSegment(const ExecBatch& batch,
+                                         int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {key_type_}));
+    if (offset == batch.length) {
+      return MakeSegment(batch.length, offset, 0, kEmptyExtends);
+    }
+    const auto& value = batch.values[0];
+    if (value.is_scalar()) {
+      return GetNextSegment(*value.scalar(), offset, batch.length);
+    }
+    if (value.is_array()) {
+      auto array = value.array();
+      if (array->GetNullCount() > 0) {
+        return Status::NotImplemented("segmenting a nullable array");
+      }
+      return GetNextSegment(*array->type, GetValuesAsBytes(*array), offset, batch.length);
+    }
+    if (value.is_chunked_array()) {
+      auto array = value.chunked_array();
+      if (array->null_count() > 0) {
+        return Status::NotImplemented("segmenting a nullable array");
+      }
+      return GetNextSegmentChunked(array, offset, bound_extend_);
+    }
+    return Status::Invalid("segmenting unsupported value kind ", value.kind());
+  }
+
+ private:
+  TypeHolder key_type_;
+  std::vector<uint8_t> save_key_data_;
+  ExtendFunc bound_extend_ = [this](const void* data) { return Extend(data); };
+};
+
+struct AnyKeysGroupingSegmenter : public BaseGroupingSegmenter {
+  static Result<std::unique_ptr<GroupingSegmenter>> Make(
+      const std::vector<TypeHolder>& key_types, ExecContext* ctx) {
+    ARROW_RETURN_NOT_OK(Grouper::Make(key_types, ctx));  // check types
+    return std::make_unique<AnyKeysGroupingSegmenter>(key_types, ctx);
+  }
+
+  AnyKeysGroupingSegmenter(const std::vector<TypeHolder>& key_types, ExecContext* ctx)
+      : BaseGroupingSegmenter(key_types),
+        ctx_(ctx),
+        grouper_(nullptr),
+        save_group_id_(kNoGroupId) {}
+
+  Status Reset() override {
+    grouper_ = nullptr;
+    save_group_id_ = kNoGroupId;
+    return Status::OK();
+  }
+
+  bool Extend(const void* data) {
+    auto group_id = *static_cast<const group_id_t*>(data);
+    bool extends =
+        save_group_id_ == kNoGroupId ? kDefaultExtends : save_group_id_ == group_id;
+    save_group_id_ = group_id;
+    return extends;
+  }
+
+  template <typename Batch>
+  Result<group_id_t> MapGroupIdAt(const Batch& batch, int64_t offset) {
+    if (offset < 0 || offset >= batch.length) {
+      return Status::Invalid("requesting group id out of bounds");
+    }
+    if (!grouper_) return kNoGroupId;
+    ARROW_ASSIGN_OR_RAISE(auto datum, grouper_->Consume(batch, offset,
+                                                        /*consume_length=*/1));
+    if (!(datum.is_array() || datum.is_chunked_array())) {
+      return Status::Invalid("accessing unsupported datum kind ", datum.kind());
+    }
+    const std::shared_ptr<ArrayData>& data =
+        datum.is_array() ? datum.array() : datum.chunked_array()->chunk(0)->data();
+    ARROW_DCHECK(data->GetNullCount() == 0);
+    DCHECK_EQ(data->type->id(), GroupIdType::type_id);
+    DCHECK_EQ(1, data->length);
+    const group_id_t* values = data->GetValues<group_id_t>(1);
+    return values[0];
+  }
+
+  template <typename Batch>
+  Result<GroupingSegment> GetNextSegmentImpl(const Batch& batch, int64_t offset) {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, key_types_));
+    if (offset == batch.length) {
+      return MakeSegment(batch.length, offset, 0, kEmptyExtends);
+    }
+    // ARROW-18311: make Grouper support Reset()
+    // so it can be cached instead of recreated below
+    //
+    // the group id must be computed prior to resetting the grouper, since it is compared
+    // to save_group_id_, and after resetting the grouper produces incomparable group ids
+    ARROW_ASSIGN_OR_RAISE(auto group_id, MapGroupIdAt(batch, offset));
+    ExtendFunc bound_extend = [this, group_id](const void* data) {
+      bool extends = Extend(&group_id);
+      save_group_id_ = *static_cast<const group_id_t*>(data);
+      return extends;
+    };
+    ARROW_ASSIGN_OR_RAISE(grouper_, Grouper::Make(key_types_, ctx_));  // TODO: reset it
+    ARROW_ASSIGN_OR_RAISE(auto datum, grouper_->Consume(batch, offset));
+    if (datum.is_array()) {

Review Comment:
   What if datum is a scalar?



##########
cpp/src/arrow/compute/exec/aggregate_node.cc:
##########
@@ -169,17 +206,63 @@ void AggregatesToString(std::stringstream* ss, const Schema& input_schema,
   *ss << ']';
 }
 
+template <typename BatchHandler>
+Status HandleSegments(std::unique_ptr<GroupingSegmenter>& segmenter,
+                      const ExecBatch& batch, const std::vector<int>& ids,
+                      const BatchHandler& handle_batch) {
+  int64_t offset = 0;
+  ARROW_ASSIGN_OR_RAISE(auto segment_batch, batch.SelectValues(ids));
+  while (true) {
+    ARROW_ASSIGN_OR_RAISE(auto segment, segmenter->GetNextSegment(segment_batch, offset));
+    if (segment.offset >= segment_batch.length) break;

Review Comment:
   How can this happen?



##########
cpp/src/arrow/compute/kernels/hash_aggregate_test.cc:
##########
@@ -174,81 +242,117 @@ Result<Datum> RunGroupBy(const BatchesWithSchema& input,
   ARROW_ASSIGN_OR_RAISE(std::vector<ExecBatch> output_batches,
                         start_and_collect.MoveResult());
 
-  ArrayVector out_arrays(aggregates.size() + key_names.size());
   const auto& output_schema = plan->nodes()[0]->output()->output_schema();
+  if (!segmented) {
+    return MakeGroupByOutput(output_batches, output_schema, aggregates.size(),
+                             key_names.size(), naive);
+  }
+
+  std::vector<ArrayVector> out_arrays(aggregates.size() + key_names.size() +
+                                      segment_key_names.size());
   for (size_t i = 0; i < out_arrays.size(); ++i) {
     std::vector<std::shared_ptr<Array>> arrays(output_batches.size());
     for (size_t j = 0; j < output_batches.size(); ++j) {
-      arrays[j] = output_batches[j].values[i].make_array();
+      auto& value = output_batches[j].values[i];
+      if (value.is_scalar()) {
+        ARROW_ASSIGN_OR_RAISE(
+            arrays[j], MakeArrayFromScalar(*value.scalar(), output_batches[j].length));
+      } else if (value.is_array()) {
+        arrays[j] = value.make_array();
+      } else {
+        return Status::Invalid("GroupByUsingExecPlan unsupported value kind ",
+                               ToString(value.kind()));
+      }
     }
     if (arrays.empty()) {
+      arrays.resize(1);
       ARROW_ASSIGN_OR_RAISE(
-          out_arrays[i],
-          MakeArrayOfNull(output_schema->field(static_cast<int>(i))->type(),
-                          /*length=*/0));
-    } else {
-      ARROW_ASSIGN_OR_RAISE(out_arrays[i], Concatenate(arrays));
+          arrays[0], MakeArrayOfNull(output_schema->field(static_cast<int>(i))->type(),
+                                     /*length=*/0));
     }
+    out_arrays[i] = {std::move(arrays)};
   }
 
-  // The exec plan may reorder the output rows.  The tests are all setup to expect ouptut
-  // in ascending order of keys.  So we need to sort the result by the key columns.  To do
-  // that we create a table using the key columns, calculate the sort indices from that
-  // table (sorting on all fields) and then use those indices to calculate our result.
-  std::vector<std::shared_ptr<Field>> key_fields;
-  std::vector<std::shared_ptr<Array>> key_columns;
-  std::vector<SortKey> sort_keys;
-  for (std::size_t i = 0; i < key_names.size(); i++) {
-    const std::shared_ptr<Array>& arr = out_arrays[i + aggregates.size()];
-    if (arr->type_id() == Type::DICTIONARY) {
-      // Can't sort dictionary columns so need to decode
-      auto dict_arr = checked_pointer_cast<DictionaryArray>(arr);
-      ARROW_ASSIGN_OR_RAISE(auto decoded_arr,
-                            Take(*dict_arr->dictionary(), *dict_arr->indices()));
-      key_columns.push_back(decoded_arr);
-      key_fields.push_back(
-          field("name_does_not_matter", dict_arr->dict_type()->value_type()));
-    } else {
-      key_columns.push_back(arr);
-      key_fields.push_back(field("name_does_not_matter", arr->type()));
+  if (segmented && segment_key_names.size() > 0) {
+    ArrayVector struct_arrays;
+    struct_arrays.reserve(output_batches.size());
+    for (size_t j = 0; j < output_batches.size(); ++j) {
+      ArrayVector struct_fields;
+      struct_fields.reserve(out_arrays.size());
+      for (auto out_array : out_arrays) {
+        struct_fields.push_back(out_array[j]);
+      }
+      ARROW_ASSIGN_OR_RAISE(auto struct_array,
+                            StructArray::Make(struct_fields, output_schema->fields()));
+      struct_arrays.push_back(struct_array);
     }
-    sort_keys.emplace_back(static_cast<int>(i));
+    return ChunkedArray::Make(struct_arrays);
+  } else {
+    ArrayVector struct_fields(out_arrays.size());
+    for (size_t i = 0; i < out_arrays.size(); ++i) {
+      ARROW_ASSIGN_OR_RAISE(struct_fields[i], Concatenate(out_arrays[i]));
+    }
+    return StructArray::Make(std::move(struct_fields), output_schema->fields());
   }
-  std::shared_ptr<Schema> key_schema = schema(std::move(key_fields));
-  std::shared_ptr<Table> key_table = Table::Make(std::move(key_schema), key_columns);
-  SortOptions sort_options(std::move(sort_keys));
-  ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Array> sort_indices,
-                        SortIndices(key_table, sort_options));
+}
 
-  ARROW_ASSIGN_OR_RAISE(
-      std::shared_ptr<Array> struct_arr,
-      StructArray::Make(std::move(out_arrays), output_schema->fields()));
+Result<Datum> RunGroupBy(const BatchesWithSchema& input,
+                         const std::vector<std::string>& key_names,
+                         const std::vector<std::string>& segment_key_names,
+                         const std::vector<Aggregate>& aggregates, bool use_threads,
+                         bool segmented = false, bool naive = false) {
+  if (segment_key_names.size() > 0) {
+    ARROW_ASSIGN_OR_RAISE(auto thread_pool, arrow::internal::ThreadPool::Make(1));
+    ExecContext seq_ctx(default_memory_pool(), thread_pool.get());
+    return RunGroupBy(input, key_names, segment_key_names, aggregates, &seq_ctx,
+                      use_threads, segmented, naive);
+  } else {
+    return RunGroupBy(input, key_names, segment_key_names, aggregates,
+                      threaded_exec_context(), use_threads, segmented, naive);
+  }
+}
 
-  return Take(struct_arr, sort_indices);
+Result<Datum> RunGroupBy(const BatchesWithSchema& input,
+                         const std::vector<std::string>& key_names,
+                         const std::vector<Aggregate>& aggregates, bool use_threads,
+                         bool segmented = false, bool naive = false) {
+  return RunGroupBy(input, key_names, {}, aggregates, use_threads, segmented);
 }
 
 /// Simpler overload where you can give the columns as datums
 Result<Datum> RunGroupBy(const std::vector<Datum>& arguments,
                          const std::vector<Datum>& keys,
-                         const std::vector<Aggregate>& aggregates,
-                         bool use_threads = false) {
+                         const std::vector<Datum>& segment_keys,
+                         const std::vector<Aggregate>& aggregates, bool use_threads,
+                         bool segmented = false, bool naive = false) {
   using arrow::compute::detail::ExecSpanIterator;
 
-  FieldVector scan_fields(arguments.size() + keys.size());
+  FieldVector scan_fields(arguments.size() + keys.size() + segment_keys.size());
   std::vector<std::string> key_names(keys.size());
+  std::vector<std::string> segment_key_names(segment_keys.size());
   for (size_t i = 0; i < arguments.size(); ++i) {
     auto name = std::string("agg_") + ToChars(i);
     scan_fields[i] = field(name, arguments[i].type());
   }
+  size_t base = arguments.size();
   for (size_t i = 0; i < keys.size(); ++i) {
     auto name = std::string("key_") + ToChars(i);
-    scan_fields[arguments.size() + i] = field(name, keys[i].type());
+    scan_fields[base + i] = field(name, keys[i].type());
     key_names[i] = std::move(name);
   }
+  base += keys.size();
+  size_t j = segmented ? keys.size() : keys.size();

Review Comment:
   ?



##########
cpp/src/arrow/compute/row/grouper.cc:
##########
@@ -39,12 +43,521 @@
 namespace arrow {
 
 using internal::checked_cast;
+using internal::PrimitiveScalarBase;
 
 namespace compute {
 
 namespace {
 
-struct GrouperImpl : Grouper {
+constexpr uint32_t kNoGroupId = std::numeric_limits<uint32_t>::max();
+
+using group_id_t = std::remove_const<decltype(kNoGroupId)>::type;
+using GroupIdType = CTypeTraits<group_id_t>::ArrowType;
+auto group_id_type = std::make_shared<GroupIdType>();
+
+inline const uint8_t* GetValuesAsBytes(const ArrayData& data, int64_t offset = 0) {
+  DCHECK_GT(data.type->byte_width(), 0);
+  int64_t absolute_byte_offset = (data.offset + offset) * data.type->byte_width();
+  return data.GetValues<uint8_t>(1, absolute_byte_offset);
+}
+
+inline const uint8_t* GetValuesAsBytes(const ArraySpan& data, int64_t offset = 0) {
+  DCHECK_GT(data.type->byte_width(), 0);
+  int64_t absolute_byte_offset = (data.offset + offset) * data.type->byte_width();
+  return data.GetValues<uint8_t>(1, absolute_byte_offset);
+}
+
+template <typename Value>
+Status CheckForGetNextSegment(const std::vector<Value>& values, int64_t length,
+                              int64_t offset, const std::vector<TypeHolder>& key_types) {
+  if (offset < 0 || offset > length) {
+    return Status::Invalid("invalid grouping segmenter offset: ", offset);
+  }
+  if (values.size() != key_types.size()) {
+    return Status::Invalid("expected batch size ", key_types.size(), " but got ",
+                           values.size());
+  }
+  for (size_t i = 0; i < key_types.size(); i++) {
+    const auto& value = values[i];
+    const auto& key_type = key_types[i];
+    if (*value.type() != *key_type.type) {
+      return Status::Invalid("expected batch value ", i, " of type ", *key_type.type,
+                             " but got ", *value.type());
+    }
+  }
+  return Status::OK();
+}
+
+template <typename Batch>
+enable_if_t<std::is_same<Batch, ExecSpan>::value || std::is_same<Batch, ExecBatch>::value,
+            Status>
+CheckForGetNextSegment(const Batch& batch, int64_t offset,
+                       const std::vector<TypeHolder>& key_types) {
+  return CheckForGetNextSegment(batch.values, batch.length, offset, key_types);
+}
+
+struct BaseGroupingSegmenter : public GroupingSegmenter {
+  explicit BaseGroupingSegmenter(const std::vector<TypeHolder>& key_types)
+      : key_types_(key_types) {}
+
+  const std::vector<TypeHolder>& key_types() const override { return key_types_; }
+
+  std::vector<TypeHolder> key_types_;

Review Comment:
   Is this kept around purely for validation purposes?  Otherwise it seems you could just get the types from the batch values themselves?



##########
cpp/src/arrow/compute/exec/aggregate_node.cc:
##########
@@ -191,13 +274,38 @@ class ScalarAggregateNode : public ExecNode, public TracedNode {
 
     const auto& aggregate_options = checked_cast<const AggregateNodeOptions&>(options);
     auto aggregates = aggregate_options.aggregates;
+    const auto& keys = aggregate_options.keys;
+    const auto& segment_keys = aggregate_options.segment_keys;
+
+    if (keys.size() > 0) {

Review Comment:
   This could probably be a DCHECK.  I don't think it should be possible to get here given that the factory in RegisterAggregateNode should check for this.



##########
cpp/src/arrow/compute/row/grouper.cc:
##########
@@ -39,12 +43,521 @@
 namespace arrow {
 
 using internal::checked_cast;
+using internal::PrimitiveScalarBase;
 
 namespace compute {
 
 namespace {
 
-struct GrouperImpl : Grouper {
+constexpr uint32_t kNoGroupId = std::numeric_limits<uint32_t>::max();
+
+using group_id_t = std::remove_const<decltype(kNoGroupId)>::type;
+using GroupIdType = CTypeTraits<group_id_t>::ArrowType;
+auto group_id_type = std::make_shared<GroupIdType>();
+
+inline const uint8_t* GetValuesAsBytes(const ArrayData& data, int64_t offset = 0) {
+  DCHECK_GT(data.type->byte_width(), 0);
+  int64_t absolute_byte_offset = (data.offset + offset) * data.type->byte_width();
+  return data.GetValues<uint8_t>(1, absolute_byte_offset);
+}
+
+inline const uint8_t* GetValuesAsBytes(const ArraySpan& data, int64_t offset = 0) {
+  DCHECK_GT(data.type->byte_width(), 0);
+  int64_t absolute_byte_offset = (data.offset + offset) * data.type->byte_width();
+  return data.GetValues<uint8_t>(1, absolute_byte_offset);
+}
+
+template <typename Value>
+Status CheckForGetNextSegment(const std::vector<Value>& values, int64_t length,
+                              int64_t offset, const std::vector<TypeHolder>& key_types) {
+  if (offset < 0 || offset > length) {
+    return Status::Invalid("invalid grouping segmenter offset: ", offset);
+  }
+  if (values.size() != key_types.size()) {
+    return Status::Invalid("expected batch size ", key_types.size(), " but got ",
+                           values.size());
+  }
+  for (size_t i = 0; i < key_types.size(); i++) {
+    const auto& value = values[i];
+    const auto& key_type = key_types[i];
+    if (*value.type() != *key_type.type) {
+      return Status::Invalid("expected batch value ", i, " of type ", *key_type.type,
+                             " but got ", *value.type());
+    }
+  }
+  return Status::OK();
+}
+
+template <typename Batch>
+enable_if_t<std::is_same<Batch, ExecSpan>::value || std::is_same<Batch, ExecBatch>::value,
+            Status>
+CheckForGetNextSegment(const Batch& batch, int64_t offset,
+                       const std::vector<TypeHolder>& key_types) {
+  return CheckForGetNextSegment(batch.values, batch.length, offset, key_types);
+}
+
+struct BaseGroupingSegmenter : public GroupingSegmenter {
+  explicit BaseGroupingSegmenter(const std::vector<TypeHolder>& key_types)
+      : key_types_(key_types) {}
+
+  const std::vector<TypeHolder>& key_types() const override { return key_types_; }
+
+  std::vector<TypeHolder> key_types_;
+};
+
+GroupingSegment MakeSegment(int64_t batch_length, int64_t offset, int64_t length,
+                            bool extends) {
+  return GroupingSegment{offset, length, offset + length >= batch_length, extends};
+}
+
+int64_t GetMatchLength(const uint8_t* match_bytes, int64_t match_width,
+                       const uint8_t* array_bytes, int64_t offset, int64_t length) {
+  int64_t cursor, byte_cursor;
+  for (cursor = offset, byte_cursor = match_width * cursor; cursor < length;
+       cursor++, byte_cursor += match_width) {
+    if (memcmp(match_bytes, array_bytes + byte_cursor,
+               static_cast<size_t>(match_width)) != 0) {
+      break;
+    }
+  }
+  return std::min(cursor, length - offset);
+}
+
+using ExtendFunc = std::function<bool(const void*)>;
+constexpr bool kDefaultExtends = true;
+constexpr bool kEmptyExtends = true;
+
+Result<GroupingSegment> GetNextSegmentChunked(
+    const std::shared_ptr<ChunkedArray>& chunked_array, int64_t offset,
+    ExtendFunc extend) {
+  if (offset >= chunked_array->length()) {
+    return MakeSegment(chunked_array->length(), chunked_array->length(), 0,
+                       kEmptyExtends);
+  }
+  int64_t remaining_offset = offset;
+  const auto& arrays = chunked_array->chunks();
+  for (size_t i = 0; remaining_offset >= 0 && i < arrays.size(); i++) {
+    // look up chunk containing offset
+    int64_t array_length = arrays[i]->length();
+    if (remaining_offset < array_length) {
+      // found - switch to matching
+      int64_t match_width = arrays[i]->type()->byte_width();
+      const uint8_t* match_bytes = GetValuesAsBytes(*arrays[i]->data(), remaining_offset);
+      int64_t total_match_length = 0;
+      for (; i < arrays.size(); i++) {
+        int64_t array_length = arrays[i]->length();
+        if (array_length <= 0) continue;
+        const uint8_t* array_bytes = GetValuesAsBytes(*arrays[i]->data());
+        int64_t match_length = GetMatchLength(match_bytes, match_width, array_bytes,
+                                              remaining_offset, array_length);
+        total_match_length += match_length;
+        remaining_offset = 0;
+        if (match_length < array_length - remaining_offset) break;
+      }
+      bool extends = extend(match_bytes);
+      return MakeSegment(chunked_array->length(), offset, total_match_length, extends);
+    }
+    remaining_offset -= array_length;
+  }
+  return Status::Invalid("segmenting invalid chunked array value");
+}
+
+struct NoKeysGroupingSegmenter : public BaseGroupingSegmenter {
+  static std::unique_ptr<GroupingSegmenter> Make() {
+    return std::make_unique<NoKeysGroupingSegmenter>();
+  }
+
+  NoKeysGroupingSegmenter() : BaseGroupingSegmenter({}) {}
+
+  Status Reset() override { return Status::OK(); }
+
+  template <typename Batch>
+  Result<GroupingSegment> GetNextSegmentImpl(const Batch& batch, int64_t offset) {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {}));
+    return MakeSegment(batch.length, offset, batch.length - offset, kDefaultExtends);
+  }
+
+  Result<GroupingSegment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    return GetNextSegmentImpl(batch, offset);
+  }
+
+  Result<GroupingSegment> GetNextSegment(const ExecBatch& batch,
+                                         int64_t offset) override {
+    return GetNextSegmentImpl(batch, offset);
+  }
+};
+
+struct SimpleKeyGroupingSegmenter : public BaseGroupingSegmenter {
+  static Result<std::unique_ptr<GroupingSegmenter>> Make(TypeHolder key_type) {
+    return std::make_unique<SimpleKeyGroupingSegmenter>(key_type);
+  }
+
+  explicit SimpleKeyGroupingSegmenter(TypeHolder key_type)
+      : BaseGroupingSegmenter({key_type}), key_type_(key_types_[0]), save_key_data_() {}
+
+  Status CheckType(const DataType& type) {
+    if (!is_fixed_width(type)) {
+      return Status::Invalid("SimpleKeyGroupingSegmenter does not support type ", type);
+    }
+    return Status::OK();
+  }
+
+  Status Reset() override {
+    save_key_data_.resize(0);
+    return Status::OK();
+  }
+
+  bool Extend(const void* data) {
+    size_t byte_width = static_cast<size_t>(key_type_.type->byte_width());
+    bool extends = save_key_data_.size() != byte_width
+                       ? kDefaultExtends
+                       : 0 == memcmp(save_key_data_.data(), data, byte_width);
+    save_key_data_.resize(byte_width);
+    memcpy(save_key_data_.data(), data, byte_width);
+    return extends;
+  }
+
+  Result<GroupingSegment> GetNextSegment(const Scalar& scalar, int64_t offset,
+                                         int64_t length) {
+    ARROW_RETURN_NOT_OK(CheckType(*scalar.type));
+    if (!scalar.is_valid) {
+      return Status::Invalid("segmenting an invalid scalar");
+    }
+    auto data = checked_cast<const PrimitiveScalarBase&>(scalar).data();
+    bool extends = length > 0 ? Extend(data) : kEmptyExtends;
+    return MakeSegment(length, 0, length, extends);
+  }
+
+  Result<GroupingSegment> GetNextSegment(const DataType& array_type,
+                                         const uint8_t* array_bytes, int64_t offset,
+                                         int64_t length) {
+    RETURN_NOT_OK(CheckType(array_type));
+    int64_t byte_width = array_type.byte_width();
+    int64_t match_length = GetMatchLength(array_bytes + offset * byte_width, byte_width,
+                                          array_bytes, offset, length);
+    bool extends = length > 0 ? Extend(array_bytes + offset * byte_width) : kEmptyExtends;
+    return MakeSegment(length, offset, match_length, extends);
+  }
+
+  Result<GroupingSegment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {key_type_}));
+    if (offset == batch.length) {
+      return MakeSegment(batch.length, offset, 0, kEmptyExtends);
+    }
+    const auto& value = batch.values[0];
+    if (value.is_scalar()) {
+      return GetNextSegment(*value.scalar, offset, batch.length);
+    }
+    ARROW_DCHECK(value.is_array());
+    const auto& array = value.array;
+    if (array.GetNullCount() > 0) {
+      return Status::NotImplemented("segmenting a nullable array");
+    }
+    return GetNextSegment(*array.type, GetValuesAsBytes(array), offset, batch.length);
+  }
+
+  Result<GroupingSegment> GetNextSegment(const ExecBatch& batch,
+                                         int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {key_type_}));
+    if (offset == batch.length) {
+      return MakeSegment(batch.length, offset, 0, kEmptyExtends);
+    }
+    const auto& value = batch.values[0];
+    if (value.is_scalar()) {
+      return GetNextSegment(*value.scalar(), offset, batch.length);
+    }
+    if (value.is_array()) {
+      auto array = value.array();
+      if (array->GetNullCount() > 0) {
+        return Status::NotImplemented("segmenting a nullable array");
+      }
+      return GetNextSegment(*array->type, GetValuesAsBytes(*array), offset, batch.length);
+    }
+    if (value.is_chunked_array()) {
+      auto array = value.chunked_array();
+      if (array->null_count() > 0) {
+        return Status::NotImplemented("segmenting a nullable array");
+      }
+      return GetNextSegmentChunked(array, offset, bound_extend_);
+    }
+    return Status::Invalid("segmenting unsupported value kind ", value.kind());
+  }
+
+ private:
+  TypeHolder key_type_;
+  std::vector<uint8_t> save_key_data_;
+  ExtendFunc bound_extend_ = [this](const void* data) { return Extend(data); };
+};
+
+struct AnyKeysGroupingSegmenter : public BaseGroupingSegmenter {
+  static Result<std::unique_ptr<GroupingSegmenter>> Make(
+      const std::vector<TypeHolder>& key_types, ExecContext* ctx) {
+    ARROW_RETURN_NOT_OK(Grouper::Make(key_types, ctx));  // check types
+    return std::make_unique<AnyKeysGroupingSegmenter>(key_types, ctx);
+  }
+
+  AnyKeysGroupingSegmenter(const std::vector<TypeHolder>& key_types, ExecContext* ctx)
+      : BaseGroupingSegmenter(key_types),
+        ctx_(ctx),
+        grouper_(nullptr),
+        save_group_id_(kNoGroupId) {}
+
+  Status Reset() override {
+    grouper_ = nullptr;
+    save_group_id_ = kNoGroupId;
+    return Status::OK();
+  }
+
+  bool Extend(const void* data) {
+    auto group_id = *static_cast<const group_id_t*>(data);
+    bool extends =
+        save_group_id_ == kNoGroupId ? kDefaultExtends : save_group_id_ == group_id;
+    save_group_id_ = group_id;
+    return extends;
+  }
+
+  template <typename Batch>
+  Result<group_id_t> MapGroupIdAt(const Batch& batch, int64_t offset) {
+    if (offset < 0 || offset >= batch.length) {
+      return Status::Invalid("requesting group id out of bounds");
+    }
+    if (!grouper_) return kNoGroupId;
+    ARROW_ASSIGN_OR_RAISE(auto datum, grouper_->Consume(batch, offset,
+                                                        /*consume_length=*/1));
+    if (!(datum.is_array() || datum.is_chunked_array())) {
+      return Status::Invalid("accessing unsupported datum kind ", datum.kind());
+    }
+    const std::shared_ptr<ArrayData>& data =
+        datum.is_array() ? datum.array() : datum.chunked_array()->chunk(0)->data();
+    ARROW_DCHECK(data->GetNullCount() == 0);
+    DCHECK_EQ(data->type->id(), GroupIdType::type_id);
+    DCHECK_EQ(1, data->length);
+    const group_id_t* values = data->GetValues<group_id_t>(1);
+    return values[0];
+  }
+
+  template <typename Batch>
+  Result<GroupingSegment> GetNextSegmentImpl(const Batch& batch, int64_t offset) {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, key_types_));
+    if (offset == batch.length) {
+      return MakeSegment(batch.length, offset, 0, kEmptyExtends);
+    }
+    // ARROW-18311: make Grouper support Reset()
+    // so it can be cached instead of recreated below
+    //
+    // the group id must be computed prior to resetting the grouper, since it is compared
+    // to save_group_id_, and after resetting the grouper produces incomparable group ids
+    ARROW_ASSIGN_OR_RAISE(auto group_id, MapGroupIdAt(batch, offset));
+    ExtendFunc bound_extend = [this, group_id](const void* data) {
+      bool extends = Extend(&group_id);
+      save_group_id_ = *static_cast<const group_id_t*>(data);
+      return extends;
+    };
+    ARROW_ASSIGN_OR_RAISE(grouper_, Grouper::Make(key_types_, ctx_));  // TODO: reset it
+    ARROW_ASSIGN_OR_RAISE(auto datum, grouper_->Consume(batch, offset));
+    if (datum.is_array()) {
+      const std::shared_ptr<ArrayData>& data = datum.array();
+      ARROW_DCHECK(data->GetNullCount() == 0);
+      DCHECK_EQ(data->type->id(), GroupIdType::type_id);
+      const group_id_t* values = data->GetValues<group_id_t>(1);
+      int64_t cursor;
+      for (cursor = 1; cursor < data->length; cursor++) {
+        if (values[0] != values[cursor]) break;
+      }
+      int64_t length = std::min(cursor, batch.length - offset);
+      bool extends = length > 0 ? bound_extend(values) : kEmptyExtends;
+      return MakeSegment(batch.length, offset, length, extends);
+    } else if (datum.is_chunked_array()) {
+      ARROW_ASSIGN_OR_RAISE(
+          auto segment, GetNextSegmentChunked(datum.chunked_array(), 0, bound_extend));
+      segment.offset += offset;
+      return segment;
+    } else {
+      return Status::Invalid("segmenting unsupported datum kind ", datum.kind());
+    }
+  }
+
+  Result<GroupingSegment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    return GetNextSegmentImpl(batch, offset);
+  }
+
+  Result<GroupingSegment> GetNextSegment(const ExecBatch& batch,
+                                         int64_t offset) override {
+    return GetNextSegmentImpl(batch, offset);
+  }
+
+ private:
+  ExecContext* const ctx_;
+  std::unique_ptr<Grouper> grouper_;
+  group_id_t save_group_id_;
+};
+
+Status CheckForConsume(int64_t batch_length, int64_t& consume_offset,
+                       int64_t* consume_length) {
+  if (consume_offset < 0) {
+    return Status::Invalid("invalid grouper consume offset: ", consume_offset);
+  }
+  if (*consume_length < 0) {
+    *consume_length = batch_length - consume_offset;
+  }
+  return Status::OK();
+}
+
+}  // namespace
+
+Result<std::unique_ptr<GroupingSegmenter>> GroupingSegmenter::Make(
+    const std::vector<TypeHolder>& key_types, bool nullable_keys, ExecContext* ctx) {
+  if (key_types.size() == 0) {
+    return NoKeysGroupingSegmenter::Make();
+  } else if (!nullable_keys && key_types.size() == 1) {
+    const DataType* type = key_types[0].type;
+    if (type != NULLPTR && is_fixed_width(*type)) {
+      return SimpleKeyGroupingSegmenter::Make(key_types[0]);
+    }
+  }
+  return AnyKeysGroupingSegmenter::Make(key_types, ctx);
+}
+
+namespace {
+
+struct BaseGrouper : public Grouper {
+  int IndexOfChunk(const ExecBatch& batch) {
+    int i = 0;
+    for (const auto& value : batch.values) {
+      if (value.is_chunked_array()) {
+        return i;
+      }
+      ++i;
+    }
+    return -1;
+  }
+
+  bool HasConsistentChunks(const ExecBatch& batch, int index_of_chunk) {
+    auto first_chunked_array = batch.values[index_of_chunk].chunked_array();
+    if (first_chunked_array < 0) {
+      // having no chunks is considered consistent
+      return true;
+    }
+    int num_chunks = first_chunked_array->num_chunks();
+    int64_t length = first_chunked_array->length();
+    for (const auto& value : batch.values) {
+      if (!value.is_chunked_array()) {
+        continue;
+      }
+      auto curr_chunk = value.chunked_array();
+      if (num_chunks != curr_chunk->num_chunks() || length != curr_chunk->length()) {
+        return false;
+      }
+    }
+    if (num_chunks > 0) {
+      for (int i = 0; i < num_chunks; i++) {
+        int64_t chunk_length = first_chunked_array->chunk(i)->length();
+        for (const auto& value : batch.values) {
+          if (!value.is_chunked_array()) {
+            continue;
+          }
+          auto curr_chunk = value.chunked_array();
+          if (chunk_length != curr_chunk->chunk(i)->length()) {
+            return false;
+          }
+        }
+      }
+    }
+    return true;
+  }

Review Comment:
   Can probably remove this logic since we don't have to worry about chunked arrays.



##########
cpp/src/arrow/compute/exec/aggregate_node.cc:
##########
@@ -326,46 +446,86 @@ class ScalarAggregateNode : public ExecNode, public TracedNode {
   }
 
  private:
-  Status Finish() {
-    auto scope = TraceFinish();
+  Status ReconstructAggregates() {

Review Comment:
   Maybe "reset" instead of "reconstruct"?



##########
cpp/src/arrow/compute/kernels/hash_aggregate_test.cc:
##########
@@ -174,81 +242,117 @@ Result<Datum> RunGroupBy(const BatchesWithSchema& input,
   ARROW_ASSIGN_OR_RAISE(std::vector<ExecBatch> output_batches,
                         start_and_collect.MoveResult());
 
-  ArrayVector out_arrays(aggregates.size() + key_names.size());
   const auto& output_schema = plan->nodes()[0]->output()->output_schema();
+  if (!segmented) {
+    return MakeGroupByOutput(output_batches, output_schema, aggregates.size(),
+                             key_names.size(), naive);
+  }
+
+  std::vector<ArrayVector> out_arrays(aggregates.size() + key_names.size() +
+                                      segment_key_names.size());
   for (size_t i = 0; i < out_arrays.size(); ++i) {
     std::vector<std::shared_ptr<Array>> arrays(output_batches.size());
     for (size_t j = 0; j < output_batches.size(); ++j) {
-      arrays[j] = output_batches[j].values[i].make_array();
+      auto& value = output_batches[j].values[i];
+      if (value.is_scalar()) {
+        ARROW_ASSIGN_OR_RAISE(
+            arrays[j], MakeArrayFromScalar(*value.scalar(), output_batches[j].length));
+      } else if (value.is_array()) {
+        arrays[j] = value.make_array();
+      } else {
+        return Status::Invalid("GroupByUsingExecPlan unsupported value kind ",
+                               ToString(value.kind()));
+      }
     }
     if (arrays.empty()) {
+      arrays.resize(1);
       ARROW_ASSIGN_OR_RAISE(
-          out_arrays[i],
-          MakeArrayOfNull(output_schema->field(static_cast<int>(i))->type(),
-                          /*length=*/0));
-    } else {
-      ARROW_ASSIGN_OR_RAISE(out_arrays[i], Concatenate(arrays));
+          arrays[0], MakeArrayOfNull(output_schema->field(static_cast<int>(i))->type(),
+                                     /*length=*/0));
     }
+    out_arrays[i] = {std::move(arrays)};
   }
 
-  // The exec plan may reorder the output rows.  The tests are all setup to expect ouptut
-  // in ascending order of keys.  So we need to sort the result by the key columns.  To do
-  // that we create a table using the key columns, calculate the sort indices from that
-  // table (sorting on all fields) and then use those indices to calculate our result.
-  std::vector<std::shared_ptr<Field>> key_fields;
-  std::vector<std::shared_ptr<Array>> key_columns;
-  std::vector<SortKey> sort_keys;
-  for (std::size_t i = 0; i < key_names.size(); i++) {
-    const std::shared_ptr<Array>& arr = out_arrays[i + aggregates.size()];
-    if (arr->type_id() == Type::DICTIONARY) {
-      // Can't sort dictionary columns so need to decode
-      auto dict_arr = checked_pointer_cast<DictionaryArray>(arr);
-      ARROW_ASSIGN_OR_RAISE(auto decoded_arr,
-                            Take(*dict_arr->dictionary(), *dict_arr->indices()));
-      key_columns.push_back(decoded_arr);
-      key_fields.push_back(
-          field("name_does_not_matter", dict_arr->dict_type()->value_type()));
-    } else {
-      key_columns.push_back(arr);
-      key_fields.push_back(field("name_does_not_matter", arr->type()));
+  if (segmented && segment_key_names.size() > 0) {
+    ArrayVector struct_arrays;
+    struct_arrays.reserve(output_batches.size());
+    for (size_t j = 0; j < output_batches.size(); ++j) {
+      ArrayVector struct_fields;
+      struct_fields.reserve(out_arrays.size());
+      for (auto out_array : out_arrays) {
+        struct_fields.push_back(out_array[j]);
+      }
+      ARROW_ASSIGN_OR_RAISE(auto struct_array,
+                            StructArray::Make(struct_fields, output_schema->fields()));
+      struct_arrays.push_back(struct_array);
     }
-    sort_keys.emplace_back(static_cast<int>(i));
+    return ChunkedArray::Make(struct_arrays);
+  } else {
+    ArrayVector struct_fields(out_arrays.size());
+    for (size_t i = 0; i < out_arrays.size(); ++i) {
+      ARROW_ASSIGN_OR_RAISE(struct_fields[i], Concatenate(out_arrays[i]));
+    }
+    return StructArray::Make(std::move(struct_fields), output_schema->fields());
   }
-  std::shared_ptr<Schema> key_schema = schema(std::move(key_fields));
-  std::shared_ptr<Table> key_table = Table::Make(std::move(key_schema), key_columns);
-  SortOptions sort_options(std::move(sort_keys));
-  ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Array> sort_indices,
-                        SortIndices(key_table, sort_options));
+}
 
-  ARROW_ASSIGN_OR_RAISE(
-      std::shared_ptr<Array> struct_arr,
-      StructArray::Make(std::move(out_arrays), output_schema->fields()));
+Result<Datum> RunGroupBy(const BatchesWithSchema& input,
+                         const std::vector<std::string>& key_names,
+                         const std::vector<std::string>& segment_key_names,
+                         const std::vector<Aggregate>& aggregates, bool use_threads,
+                         bool segmented = false, bool naive = false) {
+  if (segment_key_names.size() > 0) {
+    ARROW_ASSIGN_OR_RAISE(auto thread_pool, arrow::internal::ThreadPool::Make(1));
+    ExecContext seq_ctx(default_memory_pool(), thread_pool.get());
+    return RunGroupBy(input, key_names, segment_key_names, aggregates, &seq_ctx,
+                      use_threads, segmented, naive);
+  } else {
+    return RunGroupBy(input, key_names, segment_key_names, aggregates,
+                      threaded_exec_context(), use_threads, segmented, naive);
+  }
+}
 
-  return Take(struct_arr, sort_indices);
+Result<Datum> RunGroupBy(const BatchesWithSchema& input,
+                         const std::vector<std::string>& key_names,
+                         const std::vector<Aggregate>& aggregates, bool use_threads,
+                         bool segmented = false, bool naive = false) {
+  return RunGroupBy(input, key_names, {}, aggregates, use_threads, segmented);
 }
 
 /// Simpler overload where you can give the columns as datums
 Result<Datum> RunGroupBy(const std::vector<Datum>& arguments,
                          const std::vector<Datum>& keys,
-                         const std::vector<Aggregate>& aggregates,
-                         bool use_threads = false) {
+                         const std::vector<Datum>& segment_keys,
+                         const std::vector<Aggregate>& aggregates, bool use_threads,
+                         bool segmented = false, bool naive = false) {

Review Comment:
   Why is there both `segmented` and `segment_keys`?  Wouldn't `segmented == !segment_keys.empty()`?



##########
cpp/src/arrow/compute/row/grouper.cc:
##########
@@ -39,12 +43,521 @@
 namespace arrow {
 
 using internal::checked_cast;
+using internal::PrimitiveScalarBase;
 
 namespace compute {
 
 namespace {
 
-struct GrouperImpl : Grouper {
+constexpr uint32_t kNoGroupId = std::numeric_limits<uint32_t>::max();
+
+using group_id_t = std::remove_const<decltype(kNoGroupId)>::type;
+using GroupIdType = CTypeTraits<group_id_t>::ArrowType;
+auto group_id_type = std::make_shared<GroupIdType>();
+
+inline const uint8_t* GetValuesAsBytes(const ArrayData& data, int64_t offset = 0) {
+  DCHECK_GT(data.type->byte_width(), 0);
+  int64_t absolute_byte_offset = (data.offset + offset) * data.type->byte_width();
+  return data.GetValues<uint8_t>(1, absolute_byte_offset);
+}
+
+inline const uint8_t* GetValuesAsBytes(const ArraySpan& data, int64_t offset = 0) {
+  DCHECK_GT(data.type->byte_width(), 0);
+  int64_t absolute_byte_offset = (data.offset + offset) * data.type->byte_width();
+  return data.GetValues<uint8_t>(1, absolute_byte_offset);
+}
+
+template <typename Value>
+Status CheckForGetNextSegment(const std::vector<Value>& values, int64_t length,
+                              int64_t offset, const std::vector<TypeHolder>& key_types) {
+  if (offset < 0 || offset > length) {
+    return Status::Invalid("invalid grouping segmenter offset: ", offset);
+  }
+  if (values.size() != key_types.size()) {
+    return Status::Invalid("expected batch size ", key_types.size(), " but got ",
+                           values.size());
+  }
+  for (size_t i = 0; i < key_types.size(); i++) {
+    const auto& value = values[i];
+    const auto& key_type = key_types[i];
+    if (*value.type() != *key_type.type) {
+      return Status::Invalid("expected batch value ", i, " of type ", *key_type.type,
+                             " but got ", *value.type());
+    }
+  }
+  return Status::OK();
+}
+
+template <typename Batch>
+enable_if_t<std::is_same<Batch, ExecSpan>::value || std::is_same<Batch, ExecBatch>::value,
+            Status>
+CheckForGetNextSegment(const Batch& batch, int64_t offset,
+                       const std::vector<TypeHolder>& key_types) {
+  return CheckForGetNextSegment(batch.values, batch.length, offset, key_types);
+}
+
+struct BaseGroupingSegmenter : public GroupingSegmenter {
+  explicit BaseGroupingSegmenter(const std::vector<TypeHolder>& key_types)
+      : key_types_(key_types) {}
+
+  const std::vector<TypeHolder>& key_types() const override { return key_types_; }
+
+  std::vector<TypeHolder> key_types_;
+};
+
+GroupingSegment MakeSegment(int64_t batch_length, int64_t offset, int64_t length,
+                            bool extends) {
+  return GroupingSegment{offset, length, offset + length >= batch_length, extends};
+}
+
+int64_t GetMatchLength(const uint8_t* match_bytes, int64_t match_width,
+                       const uint8_t* array_bytes, int64_t offset, int64_t length) {
+  int64_t cursor, byte_cursor;
+  for (cursor = offset, byte_cursor = match_width * cursor; cursor < length;
+       cursor++, byte_cursor += match_width) {
+    if (memcmp(match_bytes, array_bytes + byte_cursor,
+               static_cast<size_t>(match_width)) != 0) {
+      break;
+    }
+  }
+  return std::min(cursor, length - offset);
+}
+
+using ExtendFunc = std::function<bool(const void*)>;
+constexpr bool kDefaultExtends = true;
+constexpr bool kEmptyExtends = true;
+
+Result<GroupingSegment> GetNextSegmentChunked(
+    const std::shared_ptr<ChunkedArray>& chunked_array, int64_t offset,
+    ExtendFunc extend) {
+  if (offset >= chunked_array->length()) {
+    return MakeSegment(chunked_array->length(), chunked_array->length(), 0,
+                       kEmptyExtends);
+  }
+  int64_t remaining_offset = offset;
+  const auto& arrays = chunked_array->chunks();
+  for (size_t i = 0; remaining_offset >= 0 && i < arrays.size(); i++) {
+    // look up chunk containing offset
+    int64_t array_length = arrays[i]->length();
+    if (remaining_offset < array_length) {
+      // found - switch to matching
+      int64_t match_width = arrays[i]->type()->byte_width();
+      const uint8_t* match_bytes = GetValuesAsBytes(*arrays[i]->data(), remaining_offset);
+      int64_t total_match_length = 0;
+      for (; i < arrays.size(); i++) {
+        int64_t array_length = arrays[i]->length();
+        if (array_length <= 0) continue;
+        const uint8_t* array_bytes = GetValuesAsBytes(*arrays[i]->data());
+        int64_t match_length = GetMatchLength(match_bytes, match_width, array_bytes,
+                                              remaining_offset, array_length);
+        total_match_length += match_length;
+        remaining_offset = 0;
+        if (match_length < array_length - remaining_offset) break;
+      }
+      bool extends = extend(match_bytes);
+      return MakeSegment(chunked_array->length(), offset, total_match_length, extends);
+    }
+    remaining_offset -= array_length;
+  }
+  return Status::Invalid("segmenting invalid chunked array value");
+}
+
+struct NoKeysGroupingSegmenter : public BaseGroupingSegmenter {
+  static std::unique_ptr<GroupingSegmenter> Make() {
+    return std::make_unique<NoKeysGroupingSegmenter>();
+  }
+
+  NoKeysGroupingSegmenter() : BaseGroupingSegmenter({}) {}
+
+  Status Reset() override { return Status::OK(); }
+
+  template <typename Batch>
+  Result<GroupingSegment> GetNextSegmentImpl(const Batch& batch, int64_t offset) {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {}));
+    return MakeSegment(batch.length, offset, batch.length - offset, kDefaultExtends);
+  }
+
+  Result<GroupingSegment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    return GetNextSegmentImpl(batch, offset);
+  }
+
+  Result<GroupingSegment> GetNextSegment(const ExecBatch& batch,
+                                         int64_t offset) override {
+    return GetNextSegmentImpl(batch, offset);
+  }
+};
+
+struct SimpleKeyGroupingSegmenter : public BaseGroupingSegmenter {
+  static Result<std::unique_ptr<GroupingSegmenter>> Make(TypeHolder key_type) {
+    return std::make_unique<SimpleKeyGroupingSegmenter>(key_type);
+  }
+
+  explicit SimpleKeyGroupingSegmenter(TypeHolder key_type)
+      : BaseGroupingSegmenter({key_type}), key_type_(key_types_[0]), save_key_data_() {}
+
+  Status CheckType(const DataType& type) {
+    if (!is_fixed_width(type)) {
+      return Status::Invalid("SimpleKeyGroupingSegmenter does not support type ", type);
+    }
+    return Status::OK();
+  }
+
+  Status Reset() override {
+    save_key_data_.resize(0);
+    return Status::OK();
+  }
+
+  bool Extend(const void* data) {
+    size_t byte_width = static_cast<size_t>(key_type_.type->byte_width());
+    bool extends = save_key_data_.size() != byte_width
+                       ? kDefaultExtends
+                       : 0 == memcmp(save_key_data_.data(), data, byte_width);
+    save_key_data_.resize(byte_width);
+    memcpy(save_key_data_.data(), data, byte_width);
+    return extends;
+  }
+
+  Result<GroupingSegment> GetNextSegment(const Scalar& scalar, int64_t offset,
+                                         int64_t length) {
+    ARROW_RETURN_NOT_OK(CheckType(*scalar.type));
+    if (!scalar.is_valid) {
+      return Status::Invalid("segmenting an invalid scalar");
+    }
+    auto data = checked_cast<const PrimitiveScalarBase&>(scalar).data();
+    bool extends = length > 0 ? Extend(data) : kEmptyExtends;
+    return MakeSegment(length, 0, length, extends);

Review Comment:
   ```suggestion
       return MakeSegment(length, offset, length - offset, extends);
   ```
   Although I suppose this doesn't really matter.



##########
cpp/src/arrow/compute/exec/aggregate_node.cc:
##########
@@ -326,46 +446,86 @@ class ScalarAggregateNode : public ExecNode, public TracedNode {
   }
 
  private:
-  Status Finish() {
-    auto scope = TraceFinish();
+  Status ReconstructAggregates() {
+    const auto& input_schema = *inputs()[0]->output_schema();
+    auto exec_ctx = plan()->query_context()->exec_context();
+    for (size_t i = 0; i < kernels_.size(); ++i) {
+      std::vector<TypeHolder> in_types;
+      for (const auto& target : target_fieldsets_[i]) {
+        in_types.emplace_back(input_schema.field(target)->type().get());
+      }
+      states_[i].resize(plan()->query_context()->max_concurrency());
+      KernelContext kernel_ctx{exec_ctx};
+      RETURN_NOT_OK(Kernel::InitAll(
+          &kernel_ctx, KernelInitArgs{kernels_[i], in_types, aggs_[i].options.get()},
+          &states_[i]));
+    }
+    return Status::OK();
+  }
+
+  Status OutputResult(bool is_last = false, bool traced = false) {
+    if (is_last && !traced) {
+      auto scope = TraceFinish();

Review Comment:
   Could you just call `TraceFinish` in `InputFinished`?



##########
cpp/src/arrow/compute/row/grouper.cc:
##########
@@ -39,12 +43,521 @@
 namespace arrow {
 
 using internal::checked_cast;
+using internal::PrimitiveScalarBase;
 
 namespace compute {
 
 namespace {
 
-struct GrouperImpl : Grouper {
+constexpr uint32_t kNoGroupId = std::numeric_limits<uint32_t>::max();
+
+using group_id_t = std::remove_const<decltype(kNoGroupId)>::type;
+using GroupIdType = CTypeTraits<group_id_t>::ArrowType;
+auto group_id_type = std::make_shared<GroupIdType>();
+
+inline const uint8_t* GetValuesAsBytes(const ArrayData& data, int64_t offset = 0) {
+  DCHECK_GT(data.type->byte_width(), 0);
+  int64_t absolute_byte_offset = (data.offset + offset) * data.type->byte_width();
+  return data.GetValues<uint8_t>(1, absolute_byte_offset);
+}
+
+inline const uint8_t* GetValuesAsBytes(const ArraySpan& data, int64_t offset = 0) {
+  DCHECK_GT(data.type->byte_width(), 0);
+  int64_t absolute_byte_offset = (data.offset + offset) * data.type->byte_width();
+  return data.GetValues<uint8_t>(1, absolute_byte_offset);
+}
+
+template <typename Value>
+Status CheckForGetNextSegment(const std::vector<Value>& values, int64_t length,
+                              int64_t offset, const std::vector<TypeHolder>& key_types) {
+  if (offset < 0 || offset > length) {
+    return Status::Invalid("invalid grouping segmenter offset: ", offset);
+  }
+  if (values.size() != key_types.size()) {
+    return Status::Invalid("expected batch size ", key_types.size(), " but got ",
+                           values.size());
+  }
+  for (size_t i = 0; i < key_types.size(); i++) {
+    const auto& value = values[i];
+    const auto& key_type = key_types[i];
+    if (*value.type() != *key_type.type) {
+      return Status::Invalid("expected batch value ", i, " of type ", *key_type.type,
+                             " but got ", *value.type());
+    }
+  }
+  return Status::OK();
+}
+
+template <typename Batch>
+enable_if_t<std::is_same<Batch, ExecSpan>::value || std::is_same<Batch, ExecBatch>::value,
+            Status>
+CheckForGetNextSegment(const Batch& batch, int64_t offset,
+                       const std::vector<TypeHolder>& key_types) {
+  return CheckForGetNextSegment(batch.values, batch.length, offset, key_types);
+}
+
+struct BaseGroupingSegmenter : public GroupingSegmenter {
+  explicit BaseGroupingSegmenter(const std::vector<TypeHolder>& key_types)
+      : key_types_(key_types) {}
+
+  const std::vector<TypeHolder>& key_types() const override { return key_types_; }
+
+  std::vector<TypeHolder> key_types_;
+};
+
+GroupingSegment MakeSegment(int64_t batch_length, int64_t offset, int64_t length,
+                            bool extends) {
+  return GroupingSegment{offset, length, offset + length >= batch_length, extends};
+}
+
+int64_t GetMatchLength(const uint8_t* match_bytes, int64_t match_width,
+                       const uint8_t* array_bytes, int64_t offset, int64_t length) {
+  int64_t cursor, byte_cursor;
+  for (cursor = offset, byte_cursor = match_width * cursor; cursor < length;
+       cursor++, byte_cursor += match_width) {
+    if (memcmp(match_bytes, array_bytes + byte_cursor,
+               static_cast<size_t>(match_width)) != 0) {
+      break;
+    }
+  }
+  return std::min(cursor, length - offset);
+}
+
+using ExtendFunc = std::function<bool(const void*)>;
+constexpr bool kDefaultExtends = true;
+constexpr bool kEmptyExtends = true;
+
+Result<GroupingSegment> GetNextSegmentChunked(
+    const std::shared_ptr<ChunkedArray>& chunked_array, int64_t offset,
+    ExtendFunc extend) {
+  if (offset >= chunked_array->length()) {
+    return MakeSegment(chunked_array->length(), chunked_array->length(), 0,
+                       kEmptyExtends);
+  }
+  int64_t remaining_offset = offset;
+  const auto& arrays = chunked_array->chunks();
+  for (size_t i = 0; remaining_offset >= 0 && i < arrays.size(); i++) {
+    // look up chunk containing offset
+    int64_t array_length = arrays[i]->length();
+    if (remaining_offset < array_length) {
+      // found - switch to matching
+      int64_t match_width = arrays[i]->type()->byte_width();
+      const uint8_t* match_bytes = GetValuesAsBytes(*arrays[i]->data(), remaining_offset);
+      int64_t total_match_length = 0;
+      for (; i < arrays.size(); i++) {
+        int64_t array_length = arrays[i]->length();
+        if (array_length <= 0) continue;
+        const uint8_t* array_bytes = GetValuesAsBytes(*arrays[i]->data());
+        int64_t match_length = GetMatchLength(match_bytes, match_width, array_bytes,
+                                              remaining_offset, array_length);
+        total_match_length += match_length;
+        remaining_offset = 0;
+        if (match_length < array_length - remaining_offset) break;
+      }
+      bool extends = extend(match_bytes);
+      return MakeSegment(chunked_array->length(), offset, total_match_length, extends);
+    }
+    remaining_offset -= array_length;
+  }
+  return Status::Invalid("segmenting invalid chunked array value");
+}
+
+struct NoKeysGroupingSegmenter : public BaseGroupingSegmenter {
+  static std::unique_ptr<GroupingSegmenter> Make() {
+    return std::make_unique<NoKeysGroupingSegmenter>();
+  }
+
+  NoKeysGroupingSegmenter() : BaseGroupingSegmenter({}) {}
+
+  Status Reset() override { return Status::OK(); }
+
+  template <typename Batch>
+  Result<GroupingSegment> GetNextSegmentImpl(const Batch& batch, int64_t offset) {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {}));
+    return MakeSegment(batch.length, offset, batch.length - offset, kDefaultExtends);
+  }
+
+  Result<GroupingSegment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    return GetNextSegmentImpl(batch, offset);
+  }
+
+  Result<GroupingSegment> GetNextSegment(const ExecBatch& batch,
+                                         int64_t offset) override {
+    return GetNextSegmentImpl(batch, offset);
+  }
+};
+
+struct SimpleKeyGroupingSegmenter : public BaseGroupingSegmenter {
+  static Result<std::unique_ptr<GroupingSegmenter>> Make(TypeHolder key_type) {
+    return std::make_unique<SimpleKeyGroupingSegmenter>(key_type);
+  }
+
+  explicit SimpleKeyGroupingSegmenter(TypeHolder key_type)
+      : BaseGroupingSegmenter({key_type}), key_type_(key_types_[0]), save_key_data_() {}
+
+  Status CheckType(const DataType& type) {
+    if (!is_fixed_width(type)) {
+      return Status::Invalid("SimpleKeyGroupingSegmenter does not support type ", type);
+    }
+    return Status::OK();
+  }
+
+  Status Reset() override {
+    save_key_data_.resize(0);
+    return Status::OK();
+  }
+
+  bool Extend(const void* data) {

Review Comment:
   ```suggestion
     // Checks to see if this data extends the last seen value and
     // if it doesn't then updates the last seen value
     bool Extend(const void* data) {
   ```



##########
cpp/src/arrow/compute/kernels/hash_aggregate_test.cc:
##########
@@ -174,81 +242,117 @@ Result<Datum> RunGroupBy(const BatchesWithSchema& input,
   ARROW_ASSIGN_OR_RAISE(std::vector<ExecBatch> output_batches,
                         start_and_collect.MoveResult());
 
-  ArrayVector out_arrays(aggregates.size() + key_names.size());
   const auto& output_schema = plan->nodes()[0]->output()->output_schema();
+  if (!segmented) {
+    return MakeGroupByOutput(output_batches, output_schema, aggregates.size(),
+                             key_names.size(), naive);
+  }
+
+  std::vector<ArrayVector> out_arrays(aggregates.size() + key_names.size() +
+                                      segment_key_names.size());
   for (size_t i = 0; i < out_arrays.size(); ++i) {
     std::vector<std::shared_ptr<Array>> arrays(output_batches.size());
     for (size_t j = 0; j < output_batches.size(); ++j) {
-      arrays[j] = output_batches[j].values[i].make_array();
+      auto& value = output_batches[j].values[i];
+      if (value.is_scalar()) {
+        ARROW_ASSIGN_OR_RAISE(
+            arrays[j], MakeArrayFromScalar(*value.scalar(), output_batches[j].length));
+      } else if (value.is_array()) {
+        arrays[j] = value.make_array();
+      } else {
+        return Status::Invalid("GroupByUsingExecPlan unsupported value kind ",
+                               ToString(value.kind()));
+      }
     }
     if (arrays.empty()) {
+      arrays.resize(1);
       ARROW_ASSIGN_OR_RAISE(
-          out_arrays[i],
-          MakeArrayOfNull(output_schema->field(static_cast<int>(i))->type(),
-                          /*length=*/0));
-    } else {
-      ARROW_ASSIGN_OR_RAISE(out_arrays[i], Concatenate(arrays));
+          arrays[0], MakeArrayOfNull(output_schema->field(static_cast<int>(i))->type(),
+                                     /*length=*/0));
     }
+    out_arrays[i] = {std::move(arrays)};
   }
 
-  // The exec plan may reorder the output rows.  The tests are all setup to expect ouptut
-  // in ascending order of keys.  So we need to sort the result by the key columns.  To do
-  // that we create a table using the key columns, calculate the sort indices from that
-  // table (sorting on all fields) and then use those indices to calculate our result.
-  std::vector<std::shared_ptr<Field>> key_fields;
-  std::vector<std::shared_ptr<Array>> key_columns;
-  std::vector<SortKey> sort_keys;
-  for (std::size_t i = 0; i < key_names.size(); i++) {
-    const std::shared_ptr<Array>& arr = out_arrays[i + aggregates.size()];
-    if (arr->type_id() == Type::DICTIONARY) {
-      // Can't sort dictionary columns so need to decode
-      auto dict_arr = checked_pointer_cast<DictionaryArray>(arr);
-      ARROW_ASSIGN_OR_RAISE(auto decoded_arr,
-                            Take(*dict_arr->dictionary(), *dict_arr->indices()));
-      key_columns.push_back(decoded_arr);
-      key_fields.push_back(
-          field("name_does_not_matter", dict_arr->dict_type()->value_type()));
-    } else {
-      key_columns.push_back(arr);
-      key_fields.push_back(field("name_does_not_matter", arr->type()));
+  if (segmented && segment_key_names.size() > 0) {
+    ArrayVector struct_arrays;
+    struct_arrays.reserve(output_batches.size());
+    for (size_t j = 0; j < output_batches.size(); ++j) {
+      ArrayVector struct_fields;
+      struct_fields.reserve(out_arrays.size());
+      for (auto out_array : out_arrays) {
+        struct_fields.push_back(out_array[j]);
+      }
+      ARROW_ASSIGN_OR_RAISE(auto struct_array,
+                            StructArray::Make(struct_fields, output_schema->fields()));
+      struct_arrays.push_back(struct_array);
     }
-    sort_keys.emplace_back(static_cast<int>(i));
+    return ChunkedArray::Make(struct_arrays);
+  } else {
+    ArrayVector struct_fields(out_arrays.size());
+    for (size_t i = 0; i < out_arrays.size(); ++i) {
+      ARROW_ASSIGN_OR_RAISE(struct_fields[i], Concatenate(out_arrays[i]));
+    }
+    return StructArray::Make(std::move(struct_fields), output_schema->fields());
   }
-  std::shared_ptr<Schema> key_schema = schema(std::move(key_fields));
-  std::shared_ptr<Table> key_table = Table::Make(std::move(key_schema), key_columns);
-  SortOptions sort_options(std::move(sort_keys));
-  ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Array> sort_indices,
-                        SortIndices(key_table, sort_options));
+}
 
-  ARROW_ASSIGN_OR_RAISE(
-      std::shared_ptr<Array> struct_arr,
-      StructArray::Make(std::move(out_arrays), output_schema->fields()));
+Result<Datum> RunGroupBy(const BatchesWithSchema& input,
+                         const std::vector<std::string>& key_names,
+                         const std::vector<std::string>& segment_key_names,
+                         const std::vector<Aggregate>& aggregates, bool use_threads,
+                         bool segmented = false, bool naive = false) {
+  if (segment_key_names.size() > 0) {
+    ARROW_ASSIGN_OR_RAISE(auto thread_pool, arrow::internal::ThreadPool::Make(1));
+    ExecContext seq_ctx(default_memory_pool(), thread_pool.get());
+    return RunGroupBy(input, key_names, segment_key_names, aggregates, &seq_ctx,
+                      use_threads, segmented, naive);
+  } else {
+    return RunGroupBy(input, key_names, segment_key_names, aggregates,
+                      threaded_exec_context(), use_threads, segmented, naive);
+  }
+}
 
-  return Take(struct_arr, sort_indices);
+Result<Datum> RunGroupBy(const BatchesWithSchema& input,
+                         const std::vector<std::string>& key_names,
+                         const std::vector<Aggregate>& aggregates, bool use_threads,
+                         bool segmented = false, bool naive = false) {
+  return RunGroupBy(input, key_names, {}, aggregates, use_threads, segmented);
 }
 
 /// Simpler overload where you can give the columns as datums
 Result<Datum> RunGroupBy(const std::vector<Datum>& arguments,
                          const std::vector<Datum>& keys,
-                         const std::vector<Aggregate>& aggregates,
-                         bool use_threads = false) {
+                         const std::vector<Datum>& segment_keys,
+                         const std::vector<Aggregate>& aggregates, bool use_threads,
+                         bool segmented = false, bool naive = false) {
   using arrow::compute::detail::ExecSpanIterator;
 
-  FieldVector scan_fields(arguments.size() + keys.size());
+  FieldVector scan_fields(arguments.size() + keys.size() + segment_keys.size());
   std::vector<std::string> key_names(keys.size());
+  std::vector<std::string> segment_key_names(segment_keys.size());
   for (size_t i = 0; i < arguments.size(); ++i) {
     auto name = std::string("agg_") + ToChars(i);
     scan_fields[i] = field(name, arguments[i].type());
   }
+  size_t base = arguments.size();
   for (size_t i = 0; i < keys.size(); ++i) {
     auto name = std::string("key_") + ToChars(i);
-    scan_fields[arguments.size() + i] = field(name, keys[i].type());
+    scan_fields[base + i] = field(name, keys[i].type());
     key_names[i] = std::move(name);
   }
+  base += keys.size();
+  size_t j = segmented ? keys.size() : keys.size();
+  std::string prefix(segmented ? "key_" : "key_");

Review Comment:
   ?



##########
cpp/src/arrow/compute/row/grouper.h:
##########
@@ -30,6 +30,49 @@
 namespace arrow {
 namespace compute {
 
+/// \brief A segment of contiguous rows for grouping
+struct ARROW_EXPORT GroupingSegment {
+  int64_t offset;
+  int64_t length;
+  bool is_open;
+  bool extends;
+};
+
+inline bool operator==(const GroupingSegment& segment1, const GroupingSegment& segment2) {
+  return segment1.offset == segment2.offset && segment1.length == segment2.length &&
+         segment1.is_open == segment2.is_open && segment1.extends == segment2.extends;
+}
+inline bool operator!=(const GroupingSegment& segment1, const GroupingSegment& segment2) {
+  return !(segment1 == segment2);
+}
+
+/// \brief Computes grouping segments for a batch. Each segment covers rows with identical
+/// values in the batch. The values in the batch are often selected as keys from a larger
+/// batch.

Review Comment:
   ```suggestion
   /// \brief a helper class to divide a batch into segments of equal values
   ///
   /// For example, given a batch with two rows:
   ///
   /// A A
   /// A A
   /// A B
   /// A B
   /// A B
   ///
   /// Then the batch could be divided into two segments.  The first would be rows 0 & 1
   /// and the second would be rows 2, 3, & 4.
   ///
   /// In addition, a segmenter keeps track of the last value seen.  This allows it to calculate
   /// segments which span batches.  In our above example the last batch we emit would be
   /// "open" which indicates that the segment may or may not extend into the next batch.
   ///
   /// If the next call to the segmenter starts with `A B` then that segment would be marked with
   /// `extends = true` so that the caller knows this is a continuation of the last open batch.
   ```



##########
cpp/src/arrow/compute/row/grouper.cc:
##########
@@ -39,12 +43,521 @@
 namespace arrow {
 
 using internal::checked_cast;
+using internal::PrimitiveScalarBase;
 
 namespace compute {
 
 namespace {
 
-struct GrouperImpl : Grouper {
+constexpr uint32_t kNoGroupId = std::numeric_limits<uint32_t>::max();
+
+using group_id_t = std::remove_const<decltype(kNoGroupId)>::type;
+using GroupIdType = CTypeTraits<group_id_t>::ArrowType;
+auto group_id_type = std::make_shared<GroupIdType>();
+
+inline const uint8_t* GetValuesAsBytes(const ArrayData& data, int64_t offset = 0) {
+  DCHECK_GT(data.type->byte_width(), 0);
+  int64_t absolute_byte_offset = (data.offset + offset) * data.type->byte_width();
+  return data.GetValues<uint8_t>(1, absolute_byte_offset);
+}
+
+inline const uint8_t* GetValuesAsBytes(const ArraySpan& data, int64_t offset = 0) {
+  DCHECK_GT(data.type->byte_width(), 0);
+  int64_t absolute_byte_offset = (data.offset + offset) * data.type->byte_width();
+  return data.GetValues<uint8_t>(1, absolute_byte_offset);
+}
+
+template <typename Value>
+Status CheckForGetNextSegment(const std::vector<Value>& values, int64_t length,
+                              int64_t offset, const std::vector<TypeHolder>& key_types) {
+  if (offset < 0 || offset > length) {
+    return Status::Invalid("invalid grouping segmenter offset: ", offset);
+  }
+  if (values.size() != key_types.size()) {
+    return Status::Invalid("expected batch size ", key_types.size(), " but got ",
+                           values.size());
+  }
+  for (size_t i = 0; i < key_types.size(); i++) {
+    const auto& value = values[i];
+    const auto& key_type = key_types[i];
+    if (*value.type() != *key_type.type) {
+      return Status::Invalid("expected batch value ", i, " of type ", *key_type.type,
+                             " but got ", *value.type());
+    }
+  }
+  return Status::OK();
+}
+
+template <typename Batch>
+enable_if_t<std::is_same<Batch, ExecSpan>::value || std::is_same<Batch, ExecBatch>::value,
+            Status>
+CheckForGetNextSegment(const Batch& batch, int64_t offset,
+                       const std::vector<TypeHolder>& key_types) {
+  return CheckForGetNextSegment(batch.values, batch.length, offset, key_types);
+}
+
+struct BaseGroupingSegmenter : public GroupingSegmenter {
+  explicit BaseGroupingSegmenter(const std::vector<TypeHolder>& key_types)
+      : key_types_(key_types) {}
+
+  const std::vector<TypeHolder>& key_types() const override { return key_types_; }
+
+  std::vector<TypeHolder> key_types_;
+};
+
+GroupingSegment MakeSegment(int64_t batch_length, int64_t offset, int64_t length,
+                            bool extends) {
+  return GroupingSegment{offset, length, offset + length >= batch_length, extends};
+}
+
+int64_t GetMatchLength(const uint8_t* match_bytes, int64_t match_width,
+                       const uint8_t* array_bytes, int64_t offset, int64_t length) {
+  int64_t cursor, byte_cursor;
+  for (cursor = offset, byte_cursor = match_width * cursor; cursor < length;
+       cursor++, byte_cursor += match_width) {
+    if (memcmp(match_bytes, array_bytes + byte_cursor,
+               static_cast<size_t>(match_width)) != 0) {
+      break;
+    }
+  }
+  return std::min(cursor, length - offset);
+}
+
+using ExtendFunc = std::function<bool(const void*)>;
+constexpr bool kDefaultExtends = true;
+constexpr bool kEmptyExtends = true;
+
+Result<GroupingSegment> GetNextSegmentChunked(
+    const std::shared_ptr<ChunkedArray>& chunked_array, int64_t offset,
+    ExtendFunc extend) {
+  if (offset >= chunked_array->length()) {
+    return MakeSegment(chunked_array->length(), chunked_array->length(), 0,
+                       kEmptyExtends);
+  }
+  int64_t remaining_offset = offset;
+  const auto& arrays = chunked_array->chunks();
+  for (size_t i = 0; remaining_offset >= 0 && i < arrays.size(); i++) {
+    // look up chunk containing offset
+    int64_t array_length = arrays[i]->length();
+    if (remaining_offset < array_length) {
+      // found - switch to matching
+      int64_t match_width = arrays[i]->type()->byte_width();
+      const uint8_t* match_bytes = GetValuesAsBytes(*arrays[i]->data(), remaining_offset);
+      int64_t total_match_length = 0;
+      for (; i < arrays.size(); i++) {
+        int64_t array_length = arrays[i]->length();
+        if (array_length <= 0) continue;
+        const uint8_t* array_bytes = GetValuesAsBytes(*arrays[i]->data());
+        int64_t match_length = GetMatchLength(match_bytes, match_width, array_bytes,
+                                              remaining_offset, array_length);
+        total_match_length += match_length;
+        remaining_offset = 0;
+        if (match_length < array_length - remaining_offset) break;
+      }
+      bool extends = extend(match_bytes);
+      return MakeSegment(chunked_array->length(), offset, total_match_length, extends);
+    }
+    remaining_offset -= array_length;
+  }
+  return Status::Invalid("segmenting invalid chunked array value");
+}

Review Comment:
   ```suggestion
   ```
   
   Now that #14867 has merged I think we can safely get rid of this.



##########
cpp/src/arrow/compute/row/grouper.cc:
##########
@@ -39,12 +43,521 @@
 namespace arrow {
 
 using internal::checked_cast;
+using internal::PrimitiveScalarBase;
 
 namespace compute {
 
 namespace {
 
-struct GrouperImpl : Grouper {
+constexpr uint32_t kNoGroupId = std::numeric_limits<uint32_t>::max();
+
+using group_id_t = std::remove_const<decltype(kNoGroupId)>::type;
+using GroupIdType = CTypeTraits<group_id_t>::ArrowType;
+auto group_id_type = std::make_shared<GroupIdType>();
+
+inline const uint8_t* GetValuesAsBytes(const ArrayData& data, int64_t offset = 0) {
+  DCHECK_GT(data.type->byte_width(), 0);
+  int64_t absolute_byte_offset = (data.offset + offset) * data.type->byte_width();
+  return data.GetValues<uint8_t>(1, absolute_byte_offset);
+}
+
+inline const uint8_t* GetValuesAsBytes(const ArraySpan& data, int64_t offset = 0) {
+  DCHECK_GT(data.type->byte_width(), 0);
+  int64_t absolute_byte_offset = (data.offset + offset) * data.type->byte_width();
+  return data.GetValues<uint8_t>(1, absolute_byte_offset);
+}
+
+template <typename Value>
+Status CheckForGetNextSegment(const std::vector<Value>& values, int64_t length,
+                              int64_t offset, const std::vector<TypeHolder>& key_types) {
+  if (offset < 0 || offset > length) {
+    return Status::Invalid("invalid grouping segmenter offset: ", offset);
+  }
+  if (values.size() != key_types.size()) {
+    return Status::Invalid("expected batch size ", key_types.size(), " but got ",
+                           values.size());
+  }
+  for (size_t i = 0; i < key_types.size(); i++) {
+    const auto& value = values[i];
+    const auto& key_type = key_types[i];
+    if (*value.type() != *key_type.type) {
+      return Status::Invalid("expected batch value ", i, " of type ", *key_type.type,
+                             " but got ", *value.type());
+    }
+  }
+  return Status::OK();
+}
+
+template <typename Batch>
+enable_if_t<std::is_same<Batch, ExecSpan>::value || std::is_same<Batch, ExecBatch>::value,
+            Status>
+CheckForGetNextSegment(const Batch& batch, int64_t offset,
+                       const std::vector<TypeHolder>& key_types) {
+  return CheckForGetNextSegment(batch.values, batch.length, offset, key_types);
+}
+
+struct BaseGroupingSegmenter : public GroupingSegmenter {
+  explicit BaseGroupingSegmenter(const std::vector<TypeHolder>& key_types)
+      : key_types_(key_types) {}
+
+  const std::vector<TypeHolder>& key_types() const override { return key_types_; }
+
+  std::vector<TypeHolder> key_types_;
+};
+
+GroupingSegment MakeSegment(int64_t batch_length, int64_t offset, int64_t length,
+                            bool extends) {
+  return GroupingSegment{offset, length, offset + length >= batch_length, extends};
+}
+
+int64_t GetMatchLength(const uint8_t* match_bytes, int64_t match_width,
+                       const uint8_t* array_bytes, int64_t offset, int64_t length) {
+  int64_t cursor, byte_cursor;
+  for (cursor = offset, byte_cursor = match_width * cursor; cursor < length;
+       cursor++, byte_cursor += match_width) {
+    if (memcmp(match_bytes, array_bytes + byte_cursor,
+               static_cast<size_t>(match_width)) != 0) {
+      break;
+    }
+  }
+  return std::min(cursor, length - offset);
+}
+
+using ExtendFunc = std::function<bool(const void*)>;
+constexpr bool kDefaultExtends = true;
+constexpr bool kEmptyExtends = true;
+
+Result<GroupingSegment> GetNextSegmentChunked(
+    const std::shared_ptr<ChunkedArray>& chunked_array, int64_t offset,
+    ExtendFunc extend) {
+  if (offset >= chunked_array->length()) {
+    return MakeSegment(chunked_array->length(), chunked_array->length(), 0,
+                       kEmptyExtends);
+  }
+  int64_t remaining_offset = offset;
+  const auto& arrays = chunked_array->chunks();
+  for (size_t i = 0; remaining_offset >= 0 && i < arrays.size(); i++) {
+    // look up chunk containing offset
+    int64_t array_length = arrays[i]->length();
+    if (remaining_offset < array_length) {
+      // found - switch to matching
+      int64_t match_width = arrays[i]->type()->byte_width();
+      const uint8_t* match_bytes = GetValuesAsBytes(*arrays[i]->data(), remaining_offset);
+      int64_t total_match_length = 0;
+      for (; i < arrays.size(); i++) {
+        int64_t array_length = arrays[i]->length();
+        if (array_length <= 0) continue;
+        const uint8_t* array_bytes = GetValuesAsBytes(*arrays[i]->data());
+        int64_t match_length = GetMatchLength(match_bytes, match_width, array_bytes,
+                                              remaining_offset, array_length);
+        total_match_length += match_length;
+        remaining_offset = 0;
+        if (match_length < array_length - remaining_offset) break;
+      }
+      bool extends = extend(match_bytes);
+      return MakeSegment(chunked_array->length(), offset, total_match_length, extends);
+    }
+    remaining_offset -= array_length;
+  }
+  return Status::Invalid("segmenting invalid chunked array value");
+}
+
+struct NoKeysGroupingSegmenter : public BaseGroupingSegmenter {
+  static std::unique_ptr<GroupingSegmenter> Make() {
+    return std::make_unique<NoKeysGroupingSegmenter>();
+  }
+
+  NoKeysGroupingSegmenter() : BaseGroupingSegmenter({}) {}
+
+  Status Reset() override { return Status::OK(); }
+
+  template <typename Batch>
+  Result<GroupingSegment> GetNextSegmentImpl(const Batch& batch, int64_t offset) {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {}));
+    return MakeSegment(batch.length, offset, batch.length - offset, kDefaultExtends);
+  }
+
+  Result<GroupingSegment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    return GetNextSegmentImpl(batch, offset);
+  }
+
+  Result<GroupingSegment> GetNextSegment(const ExecBatch& batch,
+                                         int64_t offset) override {
+    return GetNextSegmentImpl(batch, offset);
+  }
+};
+
+struct SimpleKeyGroupingSegmenter : public BaseGroupingSegmenter {
+  static Result<std::unique_ptr<GroupingSegmenter>> Make(TypeHolder key_type) {
+    return std::make_unique<SimpleKeyGroupingSegmenter>(key_type);
+  }
+
+  explicit SimpleKeyGroupingSegmenter(TypeHolder key_type)
+      : BaseGroupingSegmenter({key_type}), key_type_(key_types_[0]), save_key_data_() {}
+
+  Status CheckType(const DataType& type) {
+    if (!is_fixed_width(type)) {
+      return Status::Invalid("SimpleKeyGroupingSegmenter does not support type ", type);
+    }
+    return Status::OK();
+  }
+
+  Status Reset() override {
+    save_key_data_.resize(0);
+    return Status::OK();
+  }
+
+  bool Extend(const void* data) {
+    size_t byte_width = static_cast<size_t>(key_type_.type->byte_width());
+    bool extends = save_key_data_.size() != byte_width
+                       ? kDefaultExtends
+                       : 0 == memcmp(save_key_data_.data(), data, byte_width);
+    save_key_data_.resize(byte_width);
+    memcpy(save_key_data_.data(), data, byte_width);
+    return extends;
+  }
+
+  Result<GroupingSegment> GetNextSegment(const Scalar& scalar, int64_t offset,
+                                         int64_t length) {
+    ARROW_RETURN_NOT_OK(CheckType(*scalar.type));
+    if (!scalar.is_valid) {
+      return Status::Invalid("segmenting an invalid scalar");
+    }
+    auto data = checked_cast<const PrimitiveScalarBase&>(scalar).data();
+    bool extends = length > 0 ? Extend(data) : kEmptyExtends;
+    return MakeSegment(length, 0, length, extends);
+  }
+
+  Result<GroupingSegment> GetNextSegment(const DataType& array_type,
+                                         const uint8_t* array_bytes, int64_t offset,
+                                         int64_t length) {
+    RETURN_NOT_OK(CheckType(array_type));
+    int64_t byte_width = array_type.byte_width();
+    int64_t match_length = GetMatchLength(array_bytes + offset * byte_width, byte_width,
+                                          array_bytes, offset, length);
+    bool extends = length > 0 ? Extend(array_bytes + offset * byte_width) : kEmptyExtends;
+    return MakeSegment(length, offset, match_length, extends);
+  }
+
+  Result<GroupingSegment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {key_type_}));
+    if (offset == batch.length) {
+      return MakeSegment(batch.length, offset, 0, kEmptyExtends);
+    }
+    const auto& value = batch.values[0];
+    if (value.is_scalar()) {
+      return GetNextSegment(*value.scalar, offset, batch.length);
+    }
+    ARROW_DCHECK(value.is_array());
+    const auto& array = value.array;
+    if (array.GetNullCount() > 0) {
+      return Status::NotImplemented("segmenting a nullable array");
+    }
+    return GetNextSegment(*array.type, GetValuesAsBytes(array), offset, batch.length);
+  }
+
+  Result<GroupingSegment> GetNextSegment(const ExecBatch& batch,
+                                         int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {key_type_}));
+    if (offset == batch.length) {
+      return MakeSegment(batch.length, offset, 0, kEmptyExtends);
+    }
+    const auto& value = batch.values[0];
+    if (value.is_scalar()) {
+      return GetNextSegment(*value.scalar(), offset, batch.length);
+    }
+    if (value.is_array()) {
+      auto array = value.array();
+      if (array->GetNullCount() > 0) {
+        return Status::NotImplemented("segmenting a nullable array");
+      }
+      return GetNextSegment(*array->type, GetValuesAsBytes(*array), offset, batch.length);
+    }
+    if (value.is_chunked_array()) {
+      auto array = value.chunked_array();
+      if (array->null_count() > 0) {
+        return Status::NotImplemented("segmenting a nullable array");
+      }
+      return GetNextSegmentChunked(array, offset, bound_extend_);
+    }
+    return Status::Invalid("segmenting unsupported value kind ", value.kind());
+  }
+
+ private:
+  TypeHolder key_type_;
+  std::vector<uint8_t> save_key_data_;
+  ExtendFunc bound_extend_ = [this](const void* data) { return Extend(data); };
+};
+
+struct AnyKeysGroupingSegmenter : public BaseGroupingSegmenter {
+  static Result<std::unique_ptr<GroupingSegmenter>> Make(
+      const std::vector<TypeHolder>& key_types, ExecContext* ctx) {
+    ARROW_RETURN_NOT_OK(Grouper::Make(key_types, ctx));  // check types
+    return std::make_unique<AnyKeysGroupingSegmenter>(key_types, ctx);
+  }
+
+  AnyKeysGroupingSegmenter(const std::vector<TypeHolder>& key_types, ExecContext* ctx)
+      : BaseGroupingSegmenter(key_types),
+        ctx_(ctx),
+        grouper_(nullptr),
+        save_group_id_(kNoGroupId) {}
+
+  Status Reset() override {
+    grouper_ = nullptr;
+    save_group_id_ = kNoGroupId;
+    return Status::OK();
+  }
+
+  bool Extend(const void* data) {
+    auto group_id = *static_cast<const group_id_t*>(data);
+    bool extends =
+        save_group_id_ == kNoGroupId ? kDefaultExtends : save_group_id_ == group_id;
+    save_group_id_ = group_id;
+    return extends;
+  }
+
+  template <typename Batch>
+  Result<group_id_t> MapGroupIdAt(const Batch& batch, int64_t offset) {
+    if (offset < 0 || offset >= batch.length) {
+      return Status::Invalid("requesting group id out of bounds");
+    }

Review Comment:
   Isn't this already checked by the call to `CheckForGetNextSegment` in `GetNextSegmentImpl`?



##########
cpp/src/arrow/compute/row/grouper.cc:
##########
@@ -39,12 +43,521 @@
 namespace arrow {
 
 using internal::checked_cast;
+using internal::PrimitiveScalarBase;
 
 namespace compute {
 
 namespace {
 
-struct GrouperImpl : Grouper {
+constexpr uint32_t kNoGroupId = std::numeric_limits<uint32_t>::max();
+
+using group_id_t = std::remove_const<decltype(kNoGroupId)>::type;
+using GroupIdType = CTypeTraits<group_id_t>::ArrowType;
+auto group_id_type = std::make_shared<GroupIdType>();
+
+inline const uint8_t* GetValuesAsBytes(const ArrayData& data, int64_t offset = 0) {
+  DCHECK_GT(data.type->byte_width(), 0);
+  int64_t absolute_byte_offset = (data.offset + offset) * data.type->byte_width();
+  return data.GetValues<uint8_t>(1, absolute_byte_offset);
+}
+
+inline const uint8_t* GetValuesAsBytes(const ArraySpan& data, int64_t offset = 0) {
+  DCHECK_GT(data.type->byte_width(), 0);
+  int64_t absolute_byte_offset = (data.offset + offset) * data.type->byte_width();
+  return data.GetValues<uint8_t>(1, absolute_byte_offset);
+}
+
+template <typename Value>
+Status CheckForGetNextSegment(const std::vector<Value>& values, int64_t length,
+                              int64_t offset, const std::vector<TypeHolder>& key_types) {
+  if (offset < 0 || offset > length) {
+    return Status::Invalid("invalid grouping segmenter offset: ", offset);
+  }
+  if (values.size() != key_types.size()) {
+    return Status::Invalid("expected batch size ", key_types.size(), " but got ",
+                           values.size());
+  }
+  for (size_t i = 0; i < key_types.size(); i++) {
+    const auto& value = values[i];
+    const auto& key_type = key_types[i];
+    if (*value.type() != *key_type.type) {
+      return Status::Invalid("expected batch value ", i, " of type ", *key_type.type,
+                             " but got ", *value.type());
+    }
+  }
+  return Status::OK();
+}
+
+template <typename Batch>
+enable_if_t<std::is_same<Batch, ExecSpan>::value || std::is_same<Batch, ExecBatch>::value,
+            Status>
+CheckForGetNextSegment(const Batch& batch, int64_t offset,
+                       const std::vector<TypeHolder>& key_types) {
+  return CheckForGetNextSegment(batch.values, batch.length, offset, key_types);
+}
+
+struct BaseGroupingSegmenter : public GroupingSegmenter {
+  explicit BaseGroupingSegmenter(const std::vector<TypeHolder>& key_types)
+      : key_types_(key_types) {}
+
+  const std::vector<TypeHolder>& key_types() const override { return key_types_; }
+
+  std::vector<TypeHolder> key_types_;
+};
+
+GroupingSegment MakeSegment(int64_t batch_length, int64_t offset, int64_t length,
+                            bool extends) {
+  return GroupingSegment{offset, length, offset + length >= batch_length, extends};
+}
+
+int64_t GetMatchLength(const uint8_t* match_bytes, int64_t match_width,
+                       const uint8_t* array_bytes, int64_t offset, int64_t length) {
+  int64_t cursor, byte_cursor;
+  for (cursor = offset, byte_cursor = match_width * cursor; cursor < length;
+       cursor++, byte_cursor += match_width) {
+    if (memcmp(match_bytes, array_bytes + byte_cursor,
+               static_cast<size_t>(match_width)) != 0) {
+      break;
+    }
+  }
+  return std::min(cursor, length - offset);
+}
+
+using ExtendFunc = std::function<bool(const void*)>;
+constexpr bool kDefaultExtends = true;
+constexpr bool kEmptyExtends = true;
+
+Result<GroupingSegment> GetNextSegmentChunked(
+    const std::shared_ptr<ChunkedArray>& chunked_array, int64_t offset,
+    ExtendFunc extend) {
+  if (offset >= chunked_array->length()) {
+    return MakeSegment(chunked_array->length(), chunked_array->length(), 0,
+                       kEmptyExtends);
+  }
+  int64_t remaining_offset = offset;
+  const auto& arrays = chunked_array->chunks();
+  for (size_t i = 0; remaining_offset >= 0 && i < arrays.size(); i++) {
+    // look up chunk containing offset
+    int64_t array_length = arrays[i]->length();
+    if (remaining_offset < array_length) {
+      // found - switch to matching
+      int64_t match_width = arrays[i]->type()->byte_width();
+      const uint8_t* match_bytes = GetValuesAsBytes(*arrays[i]->data(), remaining_offset);
+      int64_t total_match_length = 0;
+      for (; i < arrays.size(); i++) {
+        int64_t array_length = arrays[i]->length();
+        if (array_length <= 0) continue;
+        const uint8_t* array_bytes = GetValuesAsBytes(*arrays[i]->data());
+        int64_t match_length = GetMatchLength(match_bytes, match_width, array_bytes,
+                                              remaining_offset, array_length);
+        total_match_length += match_length;
+        remaining_offset = 0;
+        if (match_length < array_length - remaining_offset) break;
+      }
+      bool extends = extend(match_bytes);
+      return MakeSegment(chunked_array->length(), offset, total_match_length, extends);
+    }
+    remaining_offset -= array_length;
+  }
+  return Status::Invalid("segmenting invalid chunked array value");
+}
+
+struct NoKeysGroupingSegmenter : public BaseGroupingSegmenter {
+  static std::unique_ptr<GroupingSegmenter> Make() {
+    return std::make_unique<NoKeysGroupingSegmenter>();
+  }
+
+  NoKeysGroupingSegmenter() : BaseGroupingSegmenter({}) {}
+
+  Status Reset() override { return Status::OK(); }
+
+  template <typename Batch>
+  Result<GroupingSegment> GetNextSegmentImpl(const Batch& batch, int64_t offset) {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {}));
+    return MakeSegment(batch.length, offset, batch.length - offset, kDefaultExtends);
+  }
+
+  Result<GroupingSegment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    return GetNextSegmentImpl(batch, offset);
+  }
+
+  Result<GroupingSegment> GetNextSegment(const ExecBatch& batch,
+                                         int64_t offset) override {
+    return GetNextSegmentImpl(batch, offset);
+  }
+};
+
+struct SimpleKeyGroupingSegmenter : public BaseGroupingSegmenter {
+  static Result<std::unique_ptr<GroupingSegmenter>> Make(TypeHolder key_type) {
+    return std::make_unique<SimpleKeyGroupingSegmenter>(key_type);
+  }
+
+  explicit SimpleKeyGroupingSegmenter(TypeHolder key_type)
+      : BaseGroupingSegmenter({key_type}), key_type_(key_types_[0]), save_key_data_() {}
+
+  Status CheckType(const DataType& type) {
+    if (!is_fixed_width(type)) {
+      return Status::Invalid("SimpleKeyGroupingSegmenter does not support type ", type);
+    }
+    return Status::OK();
+  }
+
+  Status Reset() override {
+    save_key_data_.resize(0);
+    return Status::OK();
+  }
+
+  bool Extend(const void* data) {
+    size_t byte_width = static_cast<size_t>(key_type_.type->byte_width());
+    bool extends = save_key_data_.size() != byte_width
+                       ? kDefaultExtends
+                       : 0 == memcmp(save_key_data_.data(), data, byte_width);
+    save_key_data_.resize(byte_width);
+    memcpy(save_key_data_.data(), data, byte_width);
+    return extends;
+  }
+
+  Result<GroupingSegment> GetNextSegment(const Scalar& scalar, int64_t offset,
+                                         int64_t length) {
+    ARROW_RETURN_NOT_OK(CheckType(*scalar.type));
+    if (!scalar.is_valid) {
+      return Status::Invalid("segmenting an invalid scalar");
+    }
+    auto data = checked_cast<const PrimitiveScalarBase&>(scalar).data();
+    bool extends = length > 0 ? Extend(data) : kEmptyExtends;
+    return MakeSegment(length, 0, length, extends);
+  }
+
+  Result<GroupingSegment> GetNextSegment(const DataType& array_type,
+                                         const uint8_t* array_bytes, int64_t offset,
+                                         int64_t length) {
+    RETURN_NOT_OK(CheckType(array_type));
+    int64_t byte_width = array_type.byte_width();
+    int64_t match_length = GetMatchLength(array_bytes + offset * byte_width, byte_width,
+                                          array_bytes, offset, length);
+    bool extends = length > 0 ? Extend(array_bytes + offset * byte_width) : kEmptyExtends;
+    return MakeSegment(length, offset, match_length, extends);
+  }
+
+  Result<GroupingSegment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {key_type_}));
+    if (offset == batch.length) {
+      return MakeSegment(batch.length, offset, 0, kEmptyExtends);
+    }
+    const auto& value = batch.values[0];
+    if (value.is_scalar()) {
+      return GetNextSegment(*value.scalar, offset, batch.length);
+    }
+    ARROW_DCHECK(value.is_array());
+    const auto& array = value.array;
+    if (array.GetNullCount() > 0) {
+      return Status::NotImplemented("segmenting a nullable array");
+    }
+    return GetNextSegment(*array.type, GetValuesAsBytes(array), offset, batch.length);
+  }
+
+  Result<GroupingSegment> GetNextSegment(const ExecBatch& batch,
+                                         int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {key_type_}));
+    if (offset == batch.length) {
+      return MakeSegment(batch.length, offset, 0, kEmptyExtends);
+    }
+    const auto& value = batch.values[0];
+    if (value.is_scalar()) {
+      return GetNextSegment(*value.scalar(), offset, batch.length);
+    }
+    if (value.is_array()) {
+      auto array = value.array();
+      if (array->GetNullCount() > 0) {
+        return Status::NotImplemented("segmenting a nullable array");
+      }
+      return GetNextSegment(*array->type, GetValuesAsBytes(*array), offset, batch.length);
+    }
+    if (value.is_chunked_array()) {
+      auto array = value.chunked_array();
+      if (array->null_count() > 0) {
+        return Status::NotImplemented("segmenting a nullable array");
+      }
+      return GetNextSegmentChunked(array, offset, bound_extend_);
+    }
+    return Status::Invalid("segmenting unsupported value kind ", value.kind());
+  }
+
+ private:
+  TypeHolder key_type_;
+  std::vector<uint8_t> save_key_data_;
+  ExtendFunc bound_extend_ = [this](const void* data) { return Extend(data); };
+};
+
+struct AnyKeysGroupingSegmenter : public BaseGroupingSegmenter {
+  static Result<std::unique_ptr<GroupingSegmenter>> Make(
+      const std::vector<TypeHolder>& key_types, ExecContext* ctx) {
+    ARROW_RETURN_NOT_OK(Grouper::Make(key_types, ctx));  // check types
+    return std::make_unique<AnyKeysGroupingSegmenter>(key_types, ctx);
+  }
+
+  AnyKeysGroupingSegmenter(const std::vector<TypeHolder>& key_types, ExecContext* ctx)
+      : BaseGroupingSegmenter(key_types),
+        ctx_(ctx),
+        grouper_(nullptr),
+        save_group_id_(kNoGroupId) {}
+
+  Status Reset() override {
+    grouper_ = nullptr;
+    save_group_id_ = kNoGroupId;
+    return Status::OK();
+  }
+
+  bool Extend(const void* data) {
+    auto group_id = *static_cast<const group_id_t*>(data);
+    bool extends =
+        save_group_id_ == kNoGroupId ? kDefaultExtends : save_group_id_ == group_id;
+    save_group_id_ = group_id;
+    return extends;
+  }
+
+  template <typename Batch>
+  Result<group_id_t> MapGroupIdAt(const Batch& batch, int64_t offset) {
+    if (offset < 0 || offset >= batch.length) {
+      return Status::Invalid("requesting group id out of bounds");
+    }
+    if (!grouper_) return kNoGroupId;
+    ARROW_ASSIGN_OR_RAISE(auto datum, grouper_->Consume(batch, offset,
+                                                        /*consume_length=*/1));
+    if (!(datum.is_array() || datum.is_chunked_array())) {
+      return Status::Invalid("accessing unsupported datum kind ", datum.kind());
+    }
+    const std::shared_ptr<ArrayData>& data =
+        datum.is_array() ? datum.array() : datum.chunked_array()->chunk(0)->data();
+    ARROW_DCHECK(data->GetNullCount() == 0);
+    DCHECK_EQ(data->type->id(), GroupIdType::type_id);
+    DCHECK_EQ(1, data->length);
+    const group_id_t* values = data->GetValues<group_id_t>(1);
+    return values[0];
+  }
+
+  template <typename Batch>
+  Result<GroupingSegment> GetNextSegmentImpl(const Batch& batch, int64_t offset) {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, key_types_));
+    if (offset == batch.length) {
+      return MakeSegment(batch.length, offset, 0, kEmptyExtends);
+    }
+    // ARROW-18311: make Grouper support Reset()
+    // so it can be cached instead of recreated below
+    //
+    // the group id must be computed prior to resetting the grouper, since it is compared
+    // to save_group_id_, and after resetting the grouper produces incomparable group ids
+    ARROW_ASSIGN_OR_RAISE(auto group_id, MapGroupIdAt(batch, offset));
+    ExtendFunc bound_extend = [this, group_id](const void* data) {
+      bool extends = Extend(&group_id);
+      save_group_id_ = *static_cast<const group_id_t*>(data);
+      return extends;
+    };
+    ARROW_ASSIGN_OR_RAISE(grouper_, Grouper::Make(key_types_, ctx_));  // TODO: reset it

Review Comment:
   Do you expect a reset to be cheaper than just creating a new grouper?  I suppose it might be able to reuse some allocations.



##########
cpp/src/arrow/compute/row/grouper.cc:
##########
@@ -39,12 +43,521 @@
 namespace arrow {
 
 using internal::checked_cast;
+using internal::PrimitiveScalarBase;
 
 namespace compute {
 
 namespace {
 
-struct GrouperImpl : Grouper {
+constexpr uint32_t kNoGroupId = std::numeric_limits<uint32_t>::max();
+
+using group_id_t = std::remove_const<decltype(kNoGroupId)>::type;
+using GroupIdType = CTypeTraits<group_id_t>::ArrowType;
+auto group_id_type = std::make_shared<GroupIdType>();
+
+inline const uint8_t* GetValuesAsBytes(const ArrayData& data, int64_t offset = 0) {
+  DCHECK_GT(data.type->byte_width(), 0);
+  int64_t absolute_byte_offset = (data.offset + offset) * data.type->byte_width();
+  return data.GetValues<uint8_t>(1, absolute_byte_offset);
+}
+
+inline const uint8_t* GetValuesAsBytes(const ArraySpan& data, int64_t offset = 0) {
+  DCHECK_GT(data.type->byte_width(), 0);
+  int64_t absolute_byte_offset = (data.offset + offset) * data.type->byte_width();
+  return data.GetValues<uint8_t>(1, absolute_byte_offset);
+}
+
+template <typename Value>
+Status CheckForGetNextSegment(const std::vector<Value>& values, int64_t length,
+                              int64_t offset, const std::vector<TypeHolder>& key_types) {
+  if (offset < 0 || offset > length) {
+    return Status::Invalid("invalid grouping segmenter offset: ", offset);
+  }
+  if (values.size() != key_types.size()) {
+    return Status::Invalid("expected batch size ", key_types.size(), " but got ",
+                           values.size());
+  }
+  for (size_t i = 0; i < key_types.size(); i++) {
+    const auto& value = values[i];
+    const auto& key_type = key_types[i];
+    if (*value.type() != *key_type.type) {
+      return Status::Invalid("expected batch value ", i, " of type ", *key_type.type,
+                             " but got ", *value.type());
+    }
+  }
+  return Status::OK();
+}
+
+template <typename Batch>
+enable_if_t<std::is_same<Batch, ExecSpan>::value || std::is_same<Batch, ExecBatch>::value,
+            Status>
+CheckForGetNextSegment(const Batch& batch, int64_t offset,
+                       const std::vector<TypeHolder>& key_types) {
+  return CheckForGetNextSegment(batch.values, batch.length, offset, key_types);
+}
+
+struct BaseGroupingSegmenter : public GroupingSegmenter {
+  explicit BaseGroupingSegmenter(const std::vector<TypeHolder>& key_types)
+      : key_types_(key_types) {}
+
+  const std::vector<TypeHolder>& key_types() const override { return key_types_; }
+
+  std::vector<TypeHolder> key_types_;
+};
+
+GroupingSegment MakeSegment(int64_t batch_length, int64_t offset, int64_t length,
+                            bool extends) {
+  return GroupingSegment{offset, length, offset + length >= batch_length, extends};
+}
+
+int64_t GetMatchLength(const uint8_t* match_bytes, int64_t match_width,
+                       const uint8_t* array_bytes, int64_t offset, int64_t length) {
+  int64_t cursor, byte_cursor;
+  for (cursor = offset, byte_cursor = match_width * cursor; cursor < length;
+       cursor++, byte_cursor += match_width) {
+    if (memcmp(match_bytes, array_bytes + byte_cursor,
+               static_cast<size_t>(match_width)) != 0) {
+      break;
+    }
+  }
+  return std::min(cursor, length - offset);
+}
+
+using ExtendFunc = std::function<bool(const void*)>;
+constexpr bool kDefaultExtends = true;
+constexpr bool kEmptyExtends = true;
+
+Result<GroupingSegment> GetNextSegmentChunked(
+    const std::shared_ptr<ChunkedArray>& chunked_array, int64_t offset,
+    ExtendFunc extend) {
+  if (offset >= chunked_array->length()) {
+    return MakeSegment(chunked_array->length(), chunked_array->length(), 0,
+                       kEmptyExtends);
+  }
+  int64_t remaining_offset = offset;
+  const auto& arrays = chunked_array->chunks();
+  for (size_t i = 0; remaining_offset >= 0 && i < arrays.size(); i++) {
+    // look up chunk containing offset
+    int64_t array_length = arrays[i]->length();
+    if (remaining_offset < array_length) {
+      // found - switch to matching
+      int64_t match_width = arrays[i]->type()->byte_width();
+      const uint8_t* match_bytes = GetValuesAsBytes(*arrays[i]->data(), remaining_offset);
+      int64_t total_match_length = 0;
+      for (; i < arrays.size(); i++) {
+        int64_t array_length = arrays[i]->length();
+        if (array_length <= 0) continue;
+        const uint8_t* array_bytes = GetValuesAsBytes(*arrays[i]->data());
+        int64_t match_length = GetMatchLength(match_bytes, match_width, array_bytes,
+                                              remaining_offset, array_length);
+        total_match_length += match_length;
+        remaining_offset = 0;
+        if (match_length < array_length - remaining_offset) break;
+      }
+      bool extends = extend(match_bytes);
+      return MakeSegment(chunked_array->length(), offset, total_match_length, extends);
+    }
+    remaining_offset -= array_length;
+  }
+  return Status::Invalid("segmenting invalid chunked array value");
+}
+
+struct NoKeysGroupingSegmenter : public BaseGroupingSegmenter {
+  static std::unique_ptr<GroupingSegmenter> Make() {
+    return std::make_unique<NoKeysGroupingSegmenter>();
+  }
+
+  NoKeysGroupingSegmenter() : BaseGroupingSegmenter({}) {}
+
+  Status Reset() override { return Status::OK(); }
+
+  template <typename Batch>
+  Result<GroupingSegment> GetNextSegmentImpl(const Batch& batch, int64_t offset) {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {}));
+    return MakeSegment(batch.length, offset, batch.length - offset, kDefaultExtends);
+  }
+
+  Result<GroupingSegment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    return GetNextSegmentImpl(batch, offset);
+  }
+
+  Result<GroupingSegment> GetNextSegment(const ExecBatch& batch,
+                                         int64_t offset) override {
+    return GetNextSegmentImpl(batch, offset);
+  }
+};
+
+struct SimpleKeyGroupingSegmenter : public BaseGroupingSegmenter {
+  static Result<std::unique_ptr<GroupingSegmenter>> Make(TypeHolder key_type) {
+    return std::make_unique<SimpleKeyGroupingSegmenter>(key_type);
+  }
+
+  explicit SimpleKeyGroupingSegmenter(TypeHolder key_type)
+      : BaseGroupingSegmenter({key_type}), key_type_(key_types_[0]), save_key_data_() {}
+
+  Status CheckType(const DataType& type) {
+    if (!is_fixed_width(type)) {
+      return Status::Invalid("SimpleKeyGroupingSegmenter does not support type ", type);
+    }
+    return Status::OK();
+  }
+
+  Status Reset() override {
+    save_key_data_.resize(0);
+    return Status::OK();
+  }
+
+  bool Extend(const void* data) {
+    size_t byte_width = static_cast<size_t>(key_type_.type->byte_width());
+    bool extends = save_key_data_.size() != byte_width
+                       ? kDefaultExtends
+                       : 0 == memcmp(save_key_data_.data(), data, byte_width);
+    save_key_data_.resize(byte_width);
+    memcpy(save_key_data_.data(), data, byte_width);
+    return extends;
+  }
+
+  Result<GroupingSegment> GetNextSegment(const Scalar& scalar, int64_t offset,
+                                         int64_t length) {
+    ARROW_RETURN_NOT_OK(CheckType(*scalar.type));
+    if (!scalar.is_valid) {
+      return Status::Invalid("segmenting an invalid scalar");
+    }
+    auto data = checked_cast<const PrimitiveScalarBase&>(scalar).data();
+    bool extends = length > 0 ? Extend(data) : kEmptyExtends;
+    return MakeSegment(length, 0, length, extends);
+  }
+
+  Result<GroupingSegment> GetNextSegment(const DataType& array_type,
+                                         const uint8_t* array_bytes, int64_t offset,
+                                         int64_t length) {
+    RETURN_NOT_OK(CheckType(array_type));
+    int64_t byte_width = array_type.byte_width();
+    int64_t match_length = GetMatchLength(array_bytes + offset * byte_width, byte_width,
+                                          array_bytes, offset, length);
+    bool extends = length > 0 ? Extend(array_bytes + offset * byte_width) : kEmptyExtends;
+    return MakeSegment(length, offset, match_length, extends);
+  }
+
+  Result<GroupingSegment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {key_type_}));
+    if (offset == batch.length) {
+      return MakeSegment(batch.length, offset, 0, kEmptyExtends);
+    }
+    const auto& value = batch.values[0];
+    if (value.is_scalar()) {
+      return GetNextSegment(*value.scalar, offset, batch.length);
+    }
+    ARROW_DCHECK(value.is_array());
+    const auto& array = value.array;
+    if (array.GetNullCount() > 0) {
+      return Status::NotImplemented("segmenting a nullable array");
+    }
+    return GetNextSegment(*array.type, GetValuesAsBytes(array), offset, batch.length);
+  }
+
+  Result<GroupingSegment> GetNextSegment(const ExecBatch& batch,
+                                         int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {key_type_}));
+    if (offset == batch.length) {
+      return MakeSegment(batch.length, offset, 0, kEmptyExtends);
+    }
+    const auto& value = batch.values[0];
+    if (value.is_scalar()) {
+      return GetNextSegment(*value.scalar(), offset, batch.length);
+    }
+    if (value.is_array()) {
+      auto array = value.array();
+      if (array->GetNullCount() > 0) {
+        return Status::NotImplemented("segmenting a nullable array");
+      }
+      return GetNextSegment(*array->type, GetValuesAsBytes(*array), offset, batch.length);
+    }
+    if (value.is_chunked_array()) {
+      auto array = value.chunked_array();
+      if (array->null_count() > 0) {
+        return Status::NotImplemented("segmenting a nullable array");
+      }
+      return GetNextSegmentChunked(array, offset, bound_extend_);
+    }
+    return Status::Invalid("segmenting unsupported value kind ", value.kind());
+  }
+
+ private:
+  TypeHolder key_type_;
+  std::vector<uint8_t> save_key_data_;
+  ExtendFunc bound_extend_ = [this](const void* data) { return Extend(data); };
+};
+
+struct AnyKeysGroupingSegmenter : public BaseGroupingSegmenter {
+  static Result<std::unique_ptr<GroupingSegmenter>> Make(
+      const std::vector<TypeHolder>& key_types, ExecContext* ctx) {
+    ARROW_RETURN_NOT_OK(Grouper::Make(key_types, ctx));  // check types
+    return std::make_unique<AnyKeysGroupingSegmenter>(key_types, ctx);
+  }
+
+  AnyKeysGroupingSegmenter(const std::vector<TypeHolder>& key_types, ExecContext* ctx)
+      : BaseGroupingSegmenter(key_types),
+        ctx_(ctx),
+        grouper_(nullptr),
+        save_group_id_(kNoGroupId) {}
+
+  Status Reset() override {
+    grouper_ = nullptr;
+    save_group_id_ = kNoGroupId;
+    return Status::OK();
+  }
+
+  bool Extend(const void* data) {
+    auto group_id = *static_cast<const group_id_t*>(data);
+    bool extends =
+        save_group_id_ == kNoGroupId ? kDefaultExtends : save_group_id_ == group_id;
+    save_group_id_ = group_id;
+    return extends;
+  }
+
+  template <typename Batch>

Review Comment:
   ```suggestion
     // Runs the grouper on a single row.  This is used to determine
     // the group id of the first row of a new segment to see if it extends
     // the previous segment.
     template <typename Batch>
   ```



##########
cpp/src/arrow/compute/row/grouper.cc:
##########
@@ -39,12 +43,521 @@
 namespace arrow {
 
 using internal::checked_cast;
+using internal::PrimitiveScalarBase;
 
 namespace compute {
 
 namespace {
 
-struct GrouperImpl : Grouper {
+constexpr uint32_t kNoGroupId = std::numeric_limits<uint32_t>::max();
+
+using group_id_t = std::remove_const<decltype(kNoGroupId)>::type;
+using GroupIdType = CTypeTraits<group_id_t>::ArrowType;
+auto group_id_type = std::make_shared<GroupIdType>();
+
+inline const uint8_t* GetValuesAsBytes(const ArrayData& data, int64_t offset = 0) {
+  DCHECK_GT(data.type->byte_width(), 0);
+  int64_t absolute_byte_offset = (data.offset + offset) * data.type->byte_width();
+  return data.GetValues<uint8_t>(1, absolute_byte_offset);
+}
+
+inline const uint8_t* GetValuesAsBytes(const ArraySpan& data, int64_t offset = 0) {
+  DCHECK_GT(data.type->byte_width(), 0);
+  int64_t absolute_byte_offset = (data.offset + offset) * data.type->byte_width();
+  return data.GetValues<uint8_t>(1, absolute_byte_offset);
+}
+
+template <typename Value>
+Status CheckForGetNextSegment(const std::vector<Value>& values, int64_t length,
+                              int64_t offset, const std::vector<TypeHolder>& key_types) {
+  if (offset < 0 || offset > length) {
+    return Status::Invalid("invalid grouping segmenter offset: ", offset);
+  }
+  if (values.size() != key_types.size()) {
+    return Status::Invalid("expected batch size ", key_types.size(), " but got ",
+                           values.size());
+  }
+  for (size_t i = 0; i < key_types.size(); i++) {
+    const auto& value = values[i];
+    const auto& key_type = key_types[i];
+    if (*value.type() != *key_type.type) {
+      return Status::Invalid("expected batch value ", i, " of type ", *key_type.type,
+                             " but got ", *value.type());
+    }
+  }
+  return Status::OK();
+}
+
+template <typename Batch>
+enable_if_t<std::is_same<Batch, ExecSpan>::value || std::is_same<Batch, ExecBatch>::value,
+            Status>
+CheckForGetNextSegment(const Batch& batch, int64_t offset,
+                       const std::vector<TypeHolder>& key_types) {
+  return CheckForGetNextSegment(batch.values, batch.length, offset, key_types);
+}
+
+struct BaseGroupingSegmenter : public GroupingSegmenter {
+  explicit BaseGroupingSegmenter(const std::vector<TypeHolder>& key_types)
+      : key_types_(key_types) {}
+
+  const std::vector<TypeHolder>& key_types() const override { return key_types_; }
+
+  std::vector<TypeHolder> key_types_;
+};
+
+GroupingSegment MakeSegment(int64_t batch_length, int64_t offset, int64_t length,
+                            bool extends) {
+  return GroupingSegment{offset, length, offset + length >= batch_length, extends};
+}
+
+int64_t GetMatchLength(const uint8_t* match_bytes, int64_t match_width,
+                       const uint8_t* array_bytes, int64_t offset, int64_t length) {
+  int64_t cursor, byte_cursor;
+  for (cursor = offset, byte_cursor = match_width * cursor; cursor < length;
+       cursor++, byte_cursor += match_width) {
+    if (memcmp(match_bytes, array_bytes + byte_cursor,
+               static_cast<size_t>(match_width)) != 0) {
+      break;
+    }
+  }
+  return std::min(cursor, length - offset);
+}
+
+using ExtendFunc = std::function<bool(const void*)>;
+constexpr bool kDefaultExtends = true;
+constexpr bool kEmptyExtends = true;
+
+Result<GroupingSegment> GetNextSegmentChunked(
+    const std::shared_ptr<ChunkedArray>& chunked_array, int64_t offset,
+    ExtendFunc extend) {
+  if (offset >= chunked_array->length()) {
+    return MakeSegment(chunked_array->length(), chunked_array->length(), 0,
+                       kEmptyExtends);
+  }
+  int64_t remaining_offset = offset;
+  const auto& arrays = chunked_array->chunks();
+  for (size_t i = 0; remaining_offset >= 0 && i < arrays.size(); i++) {
+    // look up chunk containing offset
+    int64_t array_length = arrays[i]->length();
+    if (remaining_offset < array_length) {
+      // found - switch to matching
+      int64_t match_width = arrays[i]->type()->byte_width();
+      const uint8_t* match_bytes = GetValuesAsBytes(*arrays[i]->data(), remaining_offset);
+      int64_t total_match_length = 0;
+      for (; i < arrays.size(); i++) {
+        int64_t array_length = arrays[i]->length();
+        if (array_length <= 0) continue;
+        const uint8_t* array_bytes = GetValuesAsBytes(*arrays[i]->data());
+        int64_t match_length = GetMatchLength(match_bytes, match_width, array_bytes,
+                                              remaining_offset, array_length);
+        total_match_length += match_length;
+        remaining_offset = 0;
+        if (match_length < array_length - remaining_offset) break;
+      }
+      bool extends = extend(match_bytes);
+      return MakeSegment(chunked_array->length(), offset, total_match_length, extends);
+    }
+    remaining_offset -= array_length;
+  }
+  return Status::Invalid("segmenting invalid chunked array value");
+}
+
+struct NoKeysGroupingSegmenter : public BaseGroupingSegmenter {
+  static std::unique_ptr<GroupingSegmenter> Make() {
+    return std::make_unique<NoKeysGroupingSegmenter>();
+  }
+
+  NoKeysGroupingSegmenter() : BaseGroupingSegmenter({}) {}
+
+  Status Reset() override { return Status::OK(); }
+
+  template <typename Batch>
+  Result<GroupingSegment> GetNextSegmentImpl(const Batch& batch, int64_t offset) {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {}));
+    return MakeSegment(batch.length, offset, batch.length - offset, kDefaultExtends);
+  }
+
+  Result<GroupingSegment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    return GetNextSegmentImpl(batch, offset);
+  }
+
+  Result<GroupingSegment> GetNextSegment(const ExecBatch& batch,
+                                         int64_t offset) override {
+    return GetNextSegmentImpl(batch, offset);
+  }
+};
+
+struct SimpleKeyGroupingSegmenter : public BaseGroupingSegmenter {
+  static Result<std::unique_ptr<GroupingSegmenter>> Make(TypeHolder key_type) {
+    return std::make_unique<SimpleKeyGroupingSegmenter>(key_type);
+  }
+
+  explicit SimpleKeyGroupingSegmenter(TypeHolder key_type)
+      : BaseGroupingSegmenter({key_type}), key_type_(key_types_[0]), save_key_data_() {}
+
+  Status CheckType(const DataType& type) {
+    if (!is_fixed_width(type)) {
+      return Status::Invalid("SimpleKeyGroupingSegmenter does not support type ", type);
+    }
+    return Status::OK();
+  }
+
+  Status Reset() override {
+    save_key_data_.resize(0);
+    return Status::OK();
+  }
+
+  bool Extend(const void* data) {
+    size_t byte_width = static_cast<size_t>(key_type_.type->byte_width());
+    bool extends = save_key_data_.size() != byte_width
+                       ? kDefaultExtends
+                       : 0 == memcmp(save_key_data_.data(), data, byte_width);
+    save_key_data_.resize(byte_width);
+    memcpy(save_key_data_.data(), data, byte_width);
+    return extends;
+  }
+
+  Result<GroupingSegment> GetNextSegment(const Scalar& scalar, int64_t offset,
+                                         int64_t length) {
+    ARROW_RETURN_NOT_OK(CheckType(*scalar.type));
+    if (!scalar.is_valid) {
+      return Status::Invalid("segmenting an invalid scalar");
+    }
+    auto data = checked_cast<const PrimitiveScalarBase&>(scalar).data();
+    bool extends = length > 0 ? Extend(data) : kEmptyExtends;
+    return MakeSegment(length, 0, length, extends);
+  }
+
+  Result<GroupingSegment> GetNextSegment(const DataType& array_type,
+                                         const uint8_t* array_bytes, int64_t offset,
+                                         int64_t length) {
+    RETURN_NOT_OK(CheckType(array_type));
+    int64_t byte_width = array_type.byte_width();
+    int64_t match_length = GetMatchLength(array_bytes + offset * byte_width, byte_width,
+                                          array_bytes, offset, length);
+    bool extends = length > 0 ? Extend(array_bytes + offset * byte_width) : kEmptyExtends;
+    return MakeSegment(length, offset, match_length, extends);
+  }
+
+  Result<GroupingSegment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {key_type_}));
+    if (offset == batch.length) {
+      return MakeSegment(batch.length, offset, 0, kEmptyExtends);
+    }
+    const auto& value = batch.values[0];
+    if (value.is_scalar()) {
+      return GetNextSegment(*value.scalar, offset, batch.length);
+    }
+    ARROW_DCHECK(value.is_array());
+    const auto& array = value.array;
+    if (array.GetNullCount() > 0) {
+      return Status::NotImplemented("segmenting a nullable array");
+    }
+    return GetNextSegment(*array.type, GetValuesAsBytes(array), offset, batch.length);
+  }
+
+  Result<GroupingSegment> GetNextSegment(const ExecBatch& batch,
+                                         int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {key_type_}));
+    if (offset == batch.length) {
+      return MakeSegment(batch.length, offset, 0, kEmptyExtends);
+    }
+    const auto& value = batch.values[0];
+    if (value.is_scalar()) {
+      return GetNextSegment(*value.scalar(), offset, batch.length);
+    }
+    if (value.is_array()) {
+      auto array = value.array();
+      if (array->GetNullCount() > 0) {
+        return Status::NotImplemented("segmenting a nullable array");
+      }
+      return GetNextSegment(*array->type, GetValuesAsBytes(*array), offset, batch.length);
+    }
+    if (value.is_chunked_array()) {
+      auto array = value.chunked_array();
+      if (array->null_count() > 0) {
+        return Status::NotImplemented("segmenting a nullable array");
+      }
+      return GetNextSegmentChunked(array, offset, bound_extend_);
+    }
+    return Status::Invalid("segmenting unsupported value kind ", value.kind());
+  }
+
+ private:
+  TypeHolder key_type_;
+  std::vector<uint8_t> save_key_data_;
+  ExtendFunc bound_extend_ = [this](const void* data) { return Extend(data); };
+};
+
+struct AnyKeysGroupingSegmenter : public BaseGroupingSegmenter {
+  static Result<std::unique_ptr<GroupingSegmenter>> Make(
+      const std::vector<TypeHolder>& key_types, ExecContext* ctx) {
+    ARROW_RETURN_NOT_OK(Grouper::Make(key_types, ctx));  // check types
+    return std::make_unique<AnyKeysGroupingSegmenter>(key_types, ctx);
+  }
+
+  AnyKeysGroupingSegmenter(const std::vector<TypeHolder>& key_types, ExecContext* ctx)
+      : BaseGroupingSegmenter(key_types),
+        ctx_(ctx),
+        grouper_(nullptr),
+        save_group_id_(kNoGroupId) {}
+
+  Status Reset() override {
+    grouper_ = nullptr;
+    save_group_id_ = kNoGroupId;
+    return Status::OK();
+  }
+
+  bool Extend(const void* data) {
+    auto group_id = *static_cast<const group_id_t*>(data);
+    bool extends =
+        save_group_id_ == kNoGroupId ? kDefaultExtends : save_group_id_ == group_id;
+    save_group_id_ = group_id;
+    return extends;
+  }
+
+  template <typename Batch>
+  Result<group_id_t> MapGroupIdAt(const Batch& batch, int64_t offset) {
+    if (offset < 0 || offset >= batch.length) {
+      return Status::Invalid("requesting group id out of bounds");
+    }
+    if (!grouper_) return kNoGroupId;
+    ARROW_ASSIGN_OR_RAISE(auto datum, grouper_->Consume(batch, offset,
+                                                        /*consume_length=*/1));
+    if (!(datum.is_array() || datum.is_chunked_array())) {

Review Comment:
   Can datum be a chunked array here?



##########
cpp/src/arrow/compute/row/grouper.cc:
##########
@@ -39,12 +43,521 @@
 namespace arrow {
 
 using internal::checked_cast;
+using internal::PrimitiveScalarBase;
 
 namespace compute {
 
 namespace {
 
-struct GrouperImpl : Grouper {
+constexpr uint32_t kNoGroupId = std::numeric_limits<uint32_t>::max();
+
+using group_id_t = std::remove_const<decltype(kNoGroupId)>::type;
+using GroupIdType = CTypeTraits<group_id_t>::ArrowType;
+auto group_id_type = std::make_shared<GroupIdType>();
+
+inline const uint8_t* GetValuesAsBytes(const ArrayData& data, int64_t offset = 0) {
+  DCHECK_GT(data.type->byte_width(), 0);
+  int64_t absolute_byte_offset = (data.offset + offset) * data.type->byte_width();
+  return data.GetValues<uint8_t>(1, absolute_byte_offset);
+}
+
+inline const uint8_t* GetValuesAsBytes(const ArraySpan& data, int64_t offset = 0) {
+  DCHECK_GT(data.type->byte_width(), 0);
+  int64_t absolute_byte_offset = (data.offset + offset) * data.type->byte_width();
+  return data.GetValues<uint8_t>(1, absolute_byte_offset);
+}
+
+template <typename Value>
+Status CheckForGetNextSegment(const std::vector<Value>& values, int64_t length,
+                              int64_t offset, const std::vector<TypeHolder>& key_types) {
+  if (offset < 0 || offset > length) {
+    return Status::Invalid("invalid grouping segmenter offset: ", offset);
+  }
+  if (values.size() != key_types.size()) {
+    return Status::Invalid("expected batch size ", key_types.size(), " but got ",
+                           values.size());
+  }
+  for (size_t i = 0; i < key_types.size(); i++) {
+    const auto& value = values[i];
+    const auto& key_type = key_types[i];
+    if (*value.type() != *key_type.type) {
+      return Status::Invalid("expected batch value ", i, " of type ", *key_type.type,
+                             " but got ", *value.type());
+    }
+  }
+  return Status::OK();
+}
+
+template <typename Batch>
+enable_if_t<std::is_same<Batch, ExecSpan>::value || std::is_same<Batch, ExecBatch>::value,
+            Status>
+CheckForGetNextSegment(const Batch& batch, int64_t offset,
+                       const std::vector<TypeHolder>& key_types) {
+  return CheckForGetNextSegment(batch.values, batch.length, offset, key_types);
+}
+
+struct BaseGroupingSegmenter : public GroupingSegmenter {
+  explicit BaseGroupingSegmenter(const std::vector<TypeHolder>& key_types)
+      : key_types_(key_types) {}
+
+  const std::vector<TypeHolder>& key_types() const override { return key_types_; }
+
+  std::vector<TypeHolder> key_types_;
+};
+
+GroupingSegment MakeSegment(int64_t batch_length, int64_t offset, int64_t length,
+                            bool extends) {
+  return GroupingSegment{offset, length, offset + length >= batch_length, extends};
+}
+
+int64_t GetMatchLength(const uint8_t* match_bytes, int64_t match_width,
+                       const uint8_t* array_bytes, int64_t offset, int64_t length) {
+  int64_t cursor, byte_cursor;
+  for (cursor = offset, byte_cursor = match_width * cursor; cursor < length;
+       cursor++, byte_cursor += match_width) {
+    if (memcmp(match_bytes, array_bytes + byte_cursor,
+               static_cast<size_t>(match_width)) != 0) {
+      break;
+    }
+  }
+  return std::min(cursor, length - offset);
+}
+
+using ExtendFunc = std::function<bool(const void*)>;
+constexpr bool kDefaultExtends = true;
+constexpr bool kEmptyExtends = true;
+
+Result<GroupingSegment> GetNextSegmentChunked(
+    const std::shared_ptr<ChunkedArray>& chunked_array, int64_t offset,
+    ExtendFunc extend) {
+  if (offset >= chunked_array->length()) {
+    return MakeSegment(chunked_array->length(), chunked_array->length(), 0,
+                       kEmptyExtends);
+  }
+  int64_t remaining_offset = offset;
+  const auto& arrays = chunked_array->chunks();
+  for (size_t i = 0; remaining_offset >= 0 && i < arrays.size(); i++) {
+    // look up chunk containing offset
+    int64_t array_length = arrays[i]->length();
+    if (remaining_offset < array_length) {
+      // found - switch to matching
+      int64_t match_width = arrays[i]->type()->byte_width();
+      const uint8_t* match_bytes = GetValuesAsBytes(*arrays[i]->data(), remaining_offset);
+      int64_t total_match_length = 0;
+      for (; i < arrays.size(); i++) {
+        int64_t array_length = arrays[i]->length();
+        if (array_length <= 0) continue;
+        const uint8_t* array_bytes = GetValuesAsBytes(*arrays[i]->data());
+        int64_t match_length = GetMatchLength(match_bytes, match_width, array_bytes,
+                                              remaining_offset, array_length);
+        total_match_length += match_length;
+        remaining_offset = 0;
+        if (match_length < array_length - remaining_offset) break;
+      }
+      bool extends = extend(match_bytes);
+      return MakeSegment(chunked_array->length(), offset, total_match_length, extends);
+    }
+    remaining_offset -= array_length;
+  }
+  return Status::Invalid("segmenting invalid chunked array value");
+}
+
+struct NoKeysGroupingSegmenter : public BaseGroupingSegmenter {
+  static std::unique_ptr<GroupingSegmenter> Make() {
+    return std::make_unique<NoKeysGroupingSegmenter>();
+  }
+
+  NoKeysGroupingSegmenter() : BaseGroupingSegmenter({}) {}
+
+  Status Reset() override { return Status::OK(); }
+
+  template <typename Batch>
+  Result<GroupingSegment> GetNextSegmentImpl(const Batch& batch, int64_t offset) {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {}));
+    return MakeSegment(batch.length, offset, batch.length - offset, kDefaultExtends);
+  }
+
+  Result<GroupingSegment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    return GetNextSegmentImpl(batch, offset);
+  }
+
+  Result<GroupingSegment> GetNextSegment(const ExecBatch& batch,
+                                         int64_t offset) override {
+    return GetNextSegmentImpl(batch, offset);
+  }
+};
+
+struct SimpleKeyGroupingSegmenter : public BaseGroupingSegmenter {
+  static Result<std::unique_ptr<GroupingSegmenter>> Make(TypeHolder key_type) {
+    return std::make_unique<SimpleKeyGroupingSegmenter>(key_type);
+  }
+
+  explicit SimpleKeyGroupingSegmenter(TypeHolder key_type)
+      : BaseGroupingSegmenter({key_type}), key_type_(key_types_[0]), save_key_data_() {}
+
+  Status CheckType(const DataType& type) {
+    if (!is_fixed_width(type)) {
+      return Status::Invalid("SimpleKeyGroupingSegmenter does not support type ", type);
+    }
+    return Status::OK();
+  }
+
+  Status Reset() override {
+    save_key_data_.resize(0);
+    return Status::OK();
+  }
+
+  bool Extend(const void* data) {
+    size_t byte_width = static_cast<size_t>(key_type_.type->byte_width());
+    bool extends = save_key_data_.size() != byte_width
+                       ? kDefaultExtends
+                       : 0 == memcmp(save_key_data_.data(), data, byte_width);
+    save_key_data_.resize(byte_width);
+    memcpy(save_key_data_.data(), data, byte_width);
+    return extends;
+  }
+
+  Result<GroupingSegment> GetNextSegment(const Scalar& scalar, int64_t offset,
+                                         int64_t length) {
+    ARROW_RETURN_NOT_OK(CheckType(*scalar.type));
+    if (!scalar.is_valid) {
+      return Status::Invalid("segmenting an invalid scalar");
+    }
+    auto data = checked_cast<const PrimitiveScalarBase&>(scalar).data();
+    bool extends = length > 0 ? Extend(data) : kEmptyExtends;
+    return MakeSegment(length, 0, length, extends);
+  }
+
+  Result<GroupingSegment> GetNextSegment(const DataType& array_type,
+                                         const uint8_t* array_bytes, int64_t offset,
+                                         int64_t length) {
+    RETURN_NOT_OK(CheckType(array_type));
+    int64_t byte_width = array_type.byte_width();
+    int64_t match_length = GetMatchLength(array_bytes + offset * byte_width, byte_width,
+                                          array_bytes, offset, length);
+    bool extends = length > 0 ? Extend(array_bytes + offset * byte_width) : kEmptyExtends;
+    return MakeSegment(length, offset, match_length, extends);
+  }
+
+  Result<GroupingSegment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {key_type_}));
+    if (offset == batch.length) {
+      return MakeSegment(batch.length, offset, 0, kEmptyExtends);
+    }
+    const auto& value = batch.values[0];
+    if (value.is_scalar()) {
+      return GetNextSegment(*value.scalar, offset, batch.length);
+    }
+    ARROW_DCHECK(value.is_array());
+    const auto& array = value.array;
+    if (array.GetNullCount() > 0) {
+      return Status::NotImplemented("segmenting a nullable array");
+    }
+    return GetNextSegment(*array.type, GetValuesAsBytes(array), offset, batch.length);
+  }
+
+  Result<GroupingSegment> GetNextSegment(const ExecBatch& batch,
+                                         int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {key_type_}));
+    if (offset == batch.length) {
+      return MakeSegment(batch.length, offset, 0, kEmptyExtends);
+    }
+    const auto& value = batch.values[0];
+    if (value.is_scalar()) {
+      return GetNextSegment(*value.scalar(), offset, batch.length);
+    }
+    if (value.is_array()) {
+      auto array = value.array();
+      if (array->GetNullCount() > 0) {
+        return Status::NotImplemented("segmenting a nullable array");
+      }
+      return GetNextSegment(*array->type, GetValuesAsBytes(*array), offset, batch.length);
+    }
+    if (value.is_chunked_array()) {
+      auto array = value.chunked_array();
+      if (array->null_count() > 0) {
+        return Status::NotImplemented("segmenting a nullable array");
+      }
+      return GetNextSegmentChunked(array, offset, bound_extend_);
+    }
+    return Status::Invalid("segmenting unsupported value kind ", value.kind());
+  }
+
+ private:
+  TypeHolder key_type_;
+  std::vector<uint8_t> save_key_data_;
+  ExtendFunc bound_extend_ = [this](const void* data) { return Extend(data); };
+};
+
+struct AnyKeysGroupingSegmenter : public BaseGroupingSegmenter {
+  static Result<std::unique_ptr<GroupingSegmenter>> Make(
+      const std::vector<TypeHolder>& key_types, ExecContext* ctx) {
+    ARROW_RETURN_NOT_OK(Grouper::Make(key_types, ctx));  // check types
+    return std::make_unique<AnyKeysGroupingSegmenter>(key_types, ctx);
+  }
+
+  AnyKeysGroupingSegmenter(const std::vector<TypeHolder>& key_types, ExecContext* ctx)
+      : BaseGroupingSegmenter(key_types),
+        ctx_(ctx),
+        grouper_(nullptr),
+        save_group_id_(kNoGroupId) {}
+
+  Status Reset() override {
+    grouper_ = nullptr;
+    save_group_id_ = kNoGroupId;
+    return Status::OK();
+  }
+
+  bool Extend(const void* data) {
+    auto group_id = *static_cast<const group_id_t*>(data);
+    bool extends =
+        save_group_id_ == kNoGroupId ? kDefaultExtends : save_group_id_ == group_id;
+    save_group_id_ = group_id;
+    return extends;
+  }
+
+  template <typename Batch>
+  Result<group_id_t> MapGroupIdAt(const Batch& batch, int64_t offset) {
+    if (offset < 0 || offset >= batch.length) {
+      return Status::Invalid("requesting group id out of bounds");
+    }
+    if (!grouper_) return kNoGroupId;
+    ARROW_ASSIGN_OR_RAISE(auto datum, grouper_->Consume(batch, offset,
+                                                        /*consume_length=*/1));
+    if (!(datum.is_array() || datum.is_chunked_array())) {
+      return Status::Invalid("accessing unsupported datum kind ", datum.kind());
+    }
+    const std::shared_ptr<ArrayData>& data =
+        datum.is_array() ? datum.array() : datum.chunked_array()->chunk(0)->data();
+    ARROW_DCHECK(data->GetNullCount() == 0);
+    DCHECK_EQ(data->type->id(), GroupIdType::type_id);
+    DCHECK_EQ(1, data->length);
+    const group_id_t* values = data->GetValues<group_id_t>(1);
+    return values[0];
+  }
+
+  template <typename Batch>
+  Result<GroupingSegment> GetNextSegmentImpl(const Batch& batch, int64_t offset) {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, key_types_));
+    if (offset == batch.length) {
+      return MakeSegment(batch.length, offset, 0, kEmptyExtends);
+    }
+    // ARROW-18311: make Grouper support Reset()
+    // so it can be cached instead of recreated below
+    //
+    // the group id must be computed prior to resetting the grouper, since it is compared
+    // to save_group_id_, and after resetting the grouper produces incomparable group ids
+    ARROW_ASSIGN_OR_RAISE(auto group_id, MapGroupIdAt(batch, offset));
+    ExtendFunc bound_extend = [this, group_id](const void* data) {
+      bool extends = Extend(&group_id);
+      save_group_id_ = *static_cast<const group_id_t*>(data);
+      return extends;
+    };
+    ARROW_ASSIGN_OR_RAISE(grouper_, Grouper::Make(key_types_, ctx_));  // TODO: reset it
+    ARROW_ASSIGN_OR_RAISE(auto datum, grouper_->Consume(batch, offset));
+    if (datum.is_array()) {
+      const std::shared_ptr<ArrayData>& data = datum.array();
+      ARROW_DCHECK(data->GetNullCount() == 0);
+      DCHECK_EQ(data->type->id(), GroupIdType::type_id);
+      const group_id_t* values = data->GetValues<group_id_t>(1);
+      int64_t cursor;
+      for (cursor = 1; cursor < data->length; cursor++) {
+        if (values[0] != values[cursor]) break;
+      }
+      int64_t length = std::min(cursor, batch.length - offset);
+      bool extends = length > 0 ? bound_extend(values) : kEmptyExtends;
+      return MakeSegment(batch.length, offset, length, extends);
+    } else if (datum.is_chunked_array()) {
+      ARROW_ASSIGN_OR_RAISE(
+          auto segment, GetNextSegmentChunked(datum.chunked_array(), 0, bound_extend));
+      segment.offset += offset;
+      return segment;

Review Comment:
   I think we can remove this case



##########
cpp/src/arrow/compute/row/grouper.cc:
##########
@@ -39,12 +43,521 @@
 namespace arrow {
 
 using internal::checked_cast;
+using internal::PrimitiveScalarBase;
 
 namespace compute {
 
 namespace {
 
-struct GrouperImpl : Grouper {
+constexpr uint32_t kNoGroupId = std::numeric_limits<uint32_t>::max();
+
+using group_id_t = std::remove_const<decltype(kNoGroupId)>::type;
+using GroupIdType = CTypeTraits<group_id_t>::ArrowType;
+auto group_id_type = std::make_shared<GroupIdType>();
+
+inline const uint8_t* GetValuesAsBytes(const ArrayData& data, int64_t offset = 0) {
+  DCHECK_GT(data.type->byte_width(), 0);
+  int64_t absolute_byte_offset = (data.offset + offset) * data.type->byte_width();
+  return data.GetValues<uint8_t>(1, absolute_byte_offset);
+}
+
+inline const uint8_t* GetValuesAsBytes(const ArraySpan& data, int64_t offset = 0) {
+  DCHECK_GT(data.type->byte_width(), 0);
+  int64_t absolute_byte_offset = (data.offset + offset) * data.type->byte_width();
+  return data.GetValues<uint8_t>(1, absolute_byte_offset);
+}
+
+template <typename Value>
+Status CheckForGetNextSegment(const std::vector<Value>& values, int64_t length,
+                              int64_t offset, const std::vector<TypeHolder>& key_types) {
+  if (offset < 0 || offset > length) {
+    return Status::Invalid("invalid grouping segmenter offset: ", offset);
+  }
+  if (values.size() != key_types.size()) {
+    return Status::Invalid("expected batch size ", key_types.size(), " but got ",
+                           values.size());
+  }
+  for (size_t i = 0; i < key_types.size(); i++) {
+    const auto& value = values[i];
+    const auto& key_type = key_types[i];
+    if (*value.type() != *key_type.type) {
+      return Status::Invalid("expected batch value ", i, " of type ", *key_type.type,
+                             " but got ", *value.type());
+    }
+  }
+  return Status::OK();
+}
+
+template <typename Batch>
+enable_if_t<std::is_same<Batch, ExecSpan>::value || std::is_same<Batch, ExecBatch>::value,
+            Status>
+CheckForGetNextSegment(const Batch& batch, int64_t offset,
+                       const std::vector<TypeHolder>& key_types) {
+  return CheckForGetNextSegment(batch.values, batch.length, offset, key_types);
+}
+
+struct BaseGroupingSegmenter : public GroupingSegmenter {
+  explicit BaseGroupingSegmenter(const std::vector<TypeHolder>& key_types)
+      : key_types_(key_types) {}
+
+  const std::vector<TypeHolder>& key_types() const override { return key_types_; }
+
+  std::vector<TypeHolder> key_types_;
+};
+
+GroupingSegment MakeSegment(int64_t batch_length, int64_t offset, int64_t length,
+                            bool extends) {
+  return GroupingSegment{offset, length, offset + length >= batch_length, extends};
+}
+
+int64_t GetMatchLength(const uint8_t* match_bytes, int64_t match_width,
+                       const uint8_t* array_bytes, int64_t offset, int64_t length) {
+  int64_t cursor, byte_cursor;
+  for (cursor = offset, byte_cursor = match_width * cursor; cursor < length;
+       cursor++, byte_cursor += match_width) {
+    if (memcmp(match_bytes, array_bytes + byte_cursor,
+               static_cast<size_t>(match_width)) != 0) {
+      break;
+    }
+  }
+  return std::min(cursor, length - offset);
+}
+
+using ExtendFunc = std::function<bool(const void*)>;
+constexpr bool kDefaultExtends = true;
+constexpr bool kEmptyExtends = true;
+
+Result<GroupingSegment> GetNextSegmentChunked(
+    const std::shared_ptr<ChunkedArray>& chunked_array, int64_t offset,
+    ExtendFunc extend) {
+  if (offset >= chunked_array->length()) {
+    return MakeSegment(chunked_array->length(), chunked_array->length(), 0,
+                       kEmptyExtends);
+  }
+  int64_t remaining_offset = offset;
+  const auto& arrays = chunked_array->chunks();
+  for (size_t i = 0; remaining_offset >= 0 && i < arrays.size(); i++) {
+    // look up chunk containing offset
+    int64_t array_length = arrays[i]->length();
+    if (remaining_offset < array_length) {
+      // found - switch to matching
+      int64_t match_width = arrays[i]->type()->byte_width();
+      const uint8_t* match_bytes = GetValuesAsBytes(*arrays[i]->data(), remaining_offset);
+      int64_t total_match_length = 0;
+      for (; i < arrays.size(); i++) {
+        int64_t array_length = arrays[i]->length();
+        if (array_length <= 0) continue;
+        const uint8_t* array_bytes = GetValuesAsBytes(*arrays[i]->data());
+        int64_t match_length = GetMatchLength(match_bytes, match_width, array_bytes,
+                                              remaining_offset, array_length);
+        total_match_length += match_length;
+        remaining_offset = 0;
+        if (match_length < array_length - remaining_offset) break;
+      }
+      bool extends = extend(match_bytes);
+      return MakeSegment(chunked_array->length(), offset, total_match_length, extends);
+    }
+    remaining_offset -= array_length;
+  }
+  return Status::Invalid("segmenting invalid chunked array value");
+}
+
+struct NoKeysGroupingSegmenter : public BaseGroupingSegmenter {
+  static std::unique_ptr<GroupingSegmenter> Make() {
+    return std::make_unique<NoKeysGroupingSegmenter>();
+  }
+
+  NoKeysGroupingSegmenter() : BaseGroupingSegmenter({}) {}
+
+  Status Reset() override { return Status::OK(); }
+
+  template <typename Batch>
+  Result<GroupingSegment> GetNextSegmentImpl(const Batch& batch, int64_t offset) {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {}));
+    return MakeSegment(batch.length, offset, batch.length - offset, kDefaultExtends);
+  }
+
+  Result<GroupingSegment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    return GetNextSegmentImpl(batch, offset);
+  }
+
+  Result<GroupingSegment> GetNextSegment(const ExecBatch& batch,
+                                         int64_t offset) override {
+    return GetNextSegmentImpl(batch, offset);
+  }
+};
+
+struct SimpleKeyGroupingSegmenter : public BaseGroupingSegmenter {
+  static Result<std::unique_ptr<GroupingSegmenter>> Make(TypeHolder key_type) {
+    return std::make_unique<SimpleKeyGroupingSegmenter>(key_type);
+  }
+
+  explicit SimpleKeyGroupingSegmenter(TypeHolder key_type)
+      : BaseGroupingSegmenter({key_type}), key_type_(key_types_[0]), save_key_data_() {}
+
+  Status CheckType(const DataType& type) {
+    if (!is_fixed_width(type)) {
+      return Status::Invalid("SimpleKeyGroupingSegmenter does not support type ", type);
+    }
+    return Status::OK();
+  }
+
+  Status Reset() override {
+    save_key_data_.resize(0);
+    return Status::OK();
+  }
+
+  bool Extend(const void* data) {
+    size_t byte_width = static_cast<size_t>(key_type_.type->byte_width());
+    bool extends = save_key_data_.size() != byte_width
+                       ? kDefaultExtends
+                       : 0 == memcmp(save_key_data_.data(), data, byte_width);
+    save_key_data_.resize(byte_width);
+    memcpy(save_key_data_.data(), data, byte_width);
+    return extends;
+  }
+
+  Result<GroupingSegment> GetNextSegment(const Scalar& scalar, int64_t offset,
+                                         int64_t length) {
+    ARROW_RETURN_NOT_OK(CheckType(*scalar.type));
+    if (!scalar.is_valid) {
+      return Status::Invalid("segmenting an invalid scalar");
+    }
+    auto data = checked_cast<const PrimitiveScalarBase&>(scalar).data();
+    bool extends = length > 0 ? Extend(data) : kEmptyExtends;
+    return MakeSegment(length, 0, length, extends);
+  }
+
+  Result<GroupingSegment> GetNextSegment(const DataType& array_type,
+                                         const uint8_t* array_bytes, int64_t offset,
+                                         int64_t length) {
+    RETURN_NOT_OK(CheckType(array_type));
+    int64_t byte_width = array_type.byte_width();
+    int64_t match_length = GetMatchLength(array_bytes + offset * byte_width, byte_width,
+                                          array_bytes, offset, length);
+    bool extends = length > 0 ? Extend(array_bytes + offset * byte_width) : kEmptyExtends;
+    return MakeSegment(length, offset, match_length, extends);
+  }
+
+  Result<GroupingSegment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {key_type_}));
+    if (offset == batch.length) {
+      return MakeSegment(batch.length, offset, 0, kEmptyExtends);
+    }
+    const auto& value = batch.values[0];
+    if (value.is_scalar()) {
+      return GetNextSegment(*value.scalar, offset, batch.length);
+    }
+    ARROW_DCHECK(value.is_array());
+    const auto& array = value.array;
+    if (array.GetNullCount() > 0) {
+      return Status::NotImplemented("segmenting a nullable array");
+    }
+    return GetNextSegment(*array.type, GetValuesAsBytes(array), offset, batch.length);
+  }
+
+  Result<GroupingSegment> GetNextSegment(const ExecBatch& batch,
+                                         int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {key_type_}));
+    if (offset == batch.length) {
+      return MakeSegment(batch.length, offset, 0, kEmptyExtends);
+    }
+    const auto& value = batch.values[0];
+    if (value.is_scalar()) {
+      return GetNextSegment(*value.scalar(), offset, batch.length);
+    }
+    if (value.is_array()) {
+      auto array = value.array();
+      if (array->GetNullCount() > 0) {
+        return Status::NotImplemented("segmenting a nullable array");
+      }
+      return GetNextSegment(*array->type, GetValuesAsBytes(*array), offset, batch.length);
+    }
+    if (value.is_chunked_array()) {
+      auto array = value.chunked_array();
+      if (array->null_count() > 0) {
+        return Status::NotImplemented("segmenting a nullable array");
+      }
+      return GetNextSegmentChunked(array, offset, bound_extend_);
+    }
+    return Status::Invalid("segmenting unsupported value kind ", value.kind());
+  }
+
+ private:
+  TypeHolder key_type_;
+  std::vector<uint8_t> save_key_data_;
+  ExtendFunc bound_extend_ = [this](const void* data) { return Extend(data); };
+};
+
+struct AnyKeysGroupingSegmenter : public BaseGroupingSegmenter {
+  static Result<std::unique_ptr<GroupingSegmenter>> Make(
+      const std::vector<TypeHolder>& key_types, ExecContext* ctx) {
+    ARROW_RETURN_NOT_OK(Grouper::Make(key_types, ctx));  // check types
+    return std::make_unique<AnyKeysGroupingSegmenter>(key_types, ctx);
+  }
+
+  AnyKeysGroupingSegmenter(const std::vector<TypeHolder>& key_types, ExecContext* ctx)
+      : BaseGroupingSegmenter(key_types),
+        ctx_(ctx),
+        grouper_(nullptr),
+        save_group_id_(kNoGroupId) {}
+
+  Status Reset() override {
+    grouper_ = nullptr;
+    save_group_id_ = kNoGroupId;
+    return Status::OK();
+  }
+
+  bool Extend(const void* data) {
+    auto group_id = *static_cast<const group_id_t*>(data);
+    bool extends =
+        save_group_id_ == kNoGroupId ? kDefaultExtends : save_group_id_ == group_id;
+    save_group_id_ = group_id;
+    return extends;
+  }
+
+  template <typename Batch>
+  Result<group_id_t> MapGroupIdAt(const Batch& batch, int64_t offset) {
+    if (offset < 0 || offset >= batch.length) {
+      return Status::Invalid("requesting group id out of bounds");
+    }
+    if (!grouper_) return kNoGroupId;
+    ARROW_ASSIGN_OR_RAISE(auto datum, grouper_->Consume(batch, offset,
+                                                        /*consume_length=*/1));
+    if (!(datum.is_array() || datum.is_chunked_array())) {
+      return Status::Invalid("accessing unsupported datum kind ", datum.kind());
+    }
+    const std::shared_ptr<ArrayData>& data =
+        datum.is_array() ? datum.array() : datum.chunked_array()->chunk(0)->data();
+    ARROW_DCHECK(data->GetNullCount() == 0);
+    DCHECK_EQ(data->type->id(), GroupIdType::type_id);
+    DCHECK_EQ(1, data->length);
+    const group_id_t* values = data->GetValues<group_id_t>(1);
+    return values[0];
+  }
+
+  template <typename Batch>
+  Result<GroupingSegment> GetNextSegmentImpl(const Batch& batch, int64_t offset) {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, key_types_));
+    if (offset == batch.length) {
+      return MakeSegment(batch.length, offset, 0, kEmptyExtends);
+    }
+    // ARROW-18311: make Grouper support Reset()
+    // so it can be cached instead of recreated below
+    //
+    // the group id must be computed prior to resetting the grouper, since it is compared
+    // to save_group_id_, and after resetting the grouper produces incomparable group ids
+    ARROW_ASSIGN_OR_RAISE(auto group_id, MapGroupIdAt(batch, offset));
+    ExtendFunc bound_extend = [this, group_id](const void* data) {
+      bool extends = Extend(&group_id);
+      save_group_id_ = *static_cast<const group_id_t*>(data);
+      return extends;
+    };
+    ARROW_ASSIGN_OR_RAISE(grouper_, Grouper::Make(key_types_, ctx_));  // TODO: reset it
+    ARROW_ASSIGN_OR_RAISE(auto datum, grouper_->Consume(batch, offset));
+    if (datum.is_array()) {
+      const std::shared_ptr<ArrayData>& data = datum.array();
+      ARROW_DCHECK(data->GetNullCount() == 0);

Review Comment:
   Why?



##########
cpp/src/arrow/compute/exec/aggregate_node.cc:
##########
@@ -169,17 +206,63 @@ void AggregatesToString(std::stringstream* ss, const Schema& input_schema,
   *ss << ']';
 }
 
+template <typename BatchHandler>
+Status HandleSegments(std::unique_ptr<GroupingSegmenter>& segmenter,
+                      const ExecBatch& batch, const std::vector<int>& ids,
+                      const BatchHandler& handle_batch) {
+  int64_t offset = 0;
+  ARROW_ASSIGN_OR_RAISE(auto segment_batch, batch.SelectValues(ids));
+  while (true) {
+    ARROW_ASSIGN_OR_RAISE(auto segment, segmenter->GetNextSegment(segment_batch, offset));
+    if (segment.offset >= segment_batch.length) break;
+    ARROW_RETURN_NOT_OK(handle_batch(batch, segment));
+    offset = segment.offset + segment.length;
+  }
+  return Status::OK();
+}
+
+Status GetScalarFields(std::vector<Datum>& values, const ExecBatch& input_batch,

Review Comment:
   ```suggestion
   Status GetScalarFields(std::vector<Datum>* values, const ExecBatch& input_batch,
   ```
   Prefer pointer over mutable reference since `GetScalarFields(&values)` is more clearly modifying `values` than `GetScalarFields(values)`.



##########
cpp/src/arrow/compute/exec/aggregate_node.cc:
##########
@@ -440,15 +623,33 @@ class GroupByNode : public ExecNode, public TracedNode {
       int key_field_id = key_field_ids[i];
       output_fields[base + i] = input_schema->field(key_field_id);
     }
+    base += keys.size();
+    for (size_t i = 0; i < segment_keys.size(); ++i) {
+      int segment_key_field_id = segment_key_field_ids[i];
+      output_fields[base + i] = input_schema->field(segment_key_field_id);

Review Comment:
   If a column was used both as a key field and a segment field does this mean we would output that column twice?



##########
cpp/src/arrow/compute/row/grouper.cc:
##########
@@ -39,12 +43,521 @@
 namespace arrow {
 
 using internal::checked_cast;
+using internal::PrimitiveScalarBase;
 
 namespace compute {
 
 namespace {
 
-struct GrouperImpl : Grouper {
+constexpr uint32_t kNoGroupId = std::numeric_limits<uint32_t>::max();
+
+using group_id_t = std::remove_const<decltype(kNoGroupId)>::type;
+using GroupIdType = CTypeTraits<group_id_t>::ArrowType;
+auto group_id_type = std::make_shared<GroupIdType>();
+
+inline const uint8_t* GetValuesAsBytes(const ArrayData& data, int64_t offset = 0) {
+  DCHECK_GT(data.type->byte_width(), 0);
+  int64_t absolute_byte_offset = (data.offset + offset) * data.type->byte_width();
+  return data.GetValues<uint8_t>(1, absolute_byte_offset);
+}
+
+inline const uint8_t* GetValuesAsBytes(const ArraySpan& data, int64_t offset = 0) {
+  DCHECK_GT(data.type->byte_width(), 0);
+  int64_t absolute_byte_offset = (data.offset + offset) * data.type->byte_width();
+  return data.GetValues<uint8_t>(1, absolute_byte_offset);
+}
+
+template <typename Value>
+Status CheckForGetNextSegment(const std::vector<Value>& values, int64_t length,
+                              int64_t offset, const std::vector<TypeHolder>& key_types) {
+  if (offset < 0 || offset > length) {
+    return Status::Invalid("invalid grouping segmenter offset: ", offset);
+  }
+  if (values.size() != key_types.size()) {
+    return Status::Invalid("expected batch size ", key_types.size(), " but got ",
+                           values.size());
+  }
+  for (size_t i = 0; i < key_types.size(); i++) {
+    const auto& value = values[i];
+    const auto& key_type = key_types[i];
+    if (*value.type() != *key_type.type) {
+      return Status::Invalid("expected batch value ", i, " of type ", *key_type.type,
+                             " but got ", *value.type());
+    }
+  }
+  return Status::OK();
+}
+
+template <typename Batch>
+enable_if_t<std::is_same<Batch, ExecSpan>::value || std::is_same<Batch, ExecBatch>::value,
+            Status>
+CheckForGetNextSegment(const Batch& batch, int64_t offset,
+                       const std::vector<TypeHolder>& key_types) {
+  return CheckForGetNextSegment(batch.values, batch.length, offset, key_types);
+}
+
+struct BaseGroupingSegmenter : public GroupingSegmenter {
+  explicit BaseGroupingSegmenter(const std::vector<TypeHolder>& key_types)
+      : key_types_(key_types) {}
+
+  const std::vector<TypeHolder>& key_types() const override { return key_types_; }
+
+  std::vector<TypeHolder> key_types_;
+};
+
+GroupingSegment MakeSegment(int64_t batch_length, int64_t offset, int64_t length,
+                            bool extends) {
+  return GroupingSegment{offset, length, offset + length >= batch_length, extends};
+}
+
+int64_t GetMatchLength(const uint8_t* match_bytes, int64_t match_width,
+                       const uint8_t* array_bytes, int64_t offset, int64_t length) {
+  int64_t cursor, byte_cursor;
+  for (cursor = offset, byte_cursor = match_width * cursor; cursor < length;
+       cursor++, byte_cursor += match_width) {
+    if (memcmp(match_bytes, array_bytes + byte_cursor,
+               static_cast<size_t>(match_width)) != 0) {
+      break;
+    }
+  }
+  return std::min(cursor, length - offset);
+}
+
+using ExtendFunc = std::function<bool(const void*)>;
+constexpr bool kDefaultExtends = true;
+constexpr bool kEmptyExtends = true;
+
+Result<GroupingSegment> GetNextSegmentChunked(
+    const std::shared_ptr<ChunkedArray>& chunked_array, int64_t offset,
+    ExtendFunc extend) {
+  if (offset >= chunked_array->length()) {
+    return MakeSegment(chunked_array->length(), chunked_array->length(), 0,
+                       kEmptyExtends);
+  }
+  int64_t remaining_offset = offset;
+  const auto& arrays = chunked_array->chunks();
+  for (size_t i = 0; remaining_offset >= 0 && i < arrays.size(); i++) {
+    // look up chunk containing offset
+    int64_t array_length = arrays[i]->length();
+    if (remaining_offset < array_length) {
+      // found - switch to matching
+      int64_t match_width = arrays[i]->type()->byte_width();
+      const uint8_t* match_bytes = GetValuesAsBytes(*arrays[i]->data(), remaining_offset);
+      int64_t total_match_length = 0;
+      for (; i < arrays.size(); i++) {
+        int64_t array_length = arrays[i]->length();
+        if (array_length <= 0) continue;
+        const uint8_t* array_bytes = GetValuesAsBytes(*arrays[i]->data());
+        int64_t match_length = GetMatchLength(match_bytes, match_width, array_bytes,
+                                              remaining_offset, array_length);
+        total_match_length += match_length;
+        remaining_offset = 0;
+        if (match_length < array_length - remaining_offset) break;
+      }
+      bool extends = extend(match_bytes);
+      return MakeSegment(chunked_array->length(), offset, total_match_length, extends);
+    }
+    remaining_offset -= array_length;
+  }
+  return Status::Invalid("segmenting invalid chunked array value");
+}
+
+struct NoKeysGroupingSegmenter : public BaseGroupingSegmenter {
+  static std::unique_ptr<GroupingSegmenter> Make() {
+    return std::make_unique<NoKeysGroupingSegmenter>();
+  }
+
+  NoKeysGroupingSegmenter() : BaseGroupingSegmenter({}) {}
+
+  Status Reset() override { return Status::OK(); }
+
+  template <typename Batch>
+  Result<GroupingSegment> GetNextSegmentImpl(const Batch& batch, int64_t offset) {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {}));
+    return MakeSegment(batch.length, offset, batch.length - offset, kDefaultExtends);
+  }
+
+  Result<GroupingSegment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    return GetNextSegmentImpl(batch, offset);
+  }
+
+  Result<GroupingSegment> GetNextSegment(const ExecBatch& batch,
+                                         int64_t offset) override {
+    return GetNextSegmentImpl(batch, offset);
+  }
+};
+
+struct SimpleKeyGroupingSegmenter : public BaseGroupingSegmenter {
+  static Result<std::unique_ptr<GroupingSegmenter>> Make(TypeHolder key_type) {
+    return std::make_unique<SimpleKeyGroupingSegmenter>(key_type);
+  }
+
+  explicit SimpleKeyGroupingSegmenter(TypeHolder key_type)
+      : BaseGroupingSegmenter({key_type}), key_type_(key_types_[0]), save_key_data_() {}
+
+  Status CheckType(const DataType& type) {
+    if (!is_fixed_width(type)) {
+      return Status::Invalid("SimpleKeyGroupingSegmenter does not support type ", type);
+    }
+    return Status::OK();
+  }
+
+  Status Reset() override {
+    save_key_data_.resize(0);
+    return Status::OK();
+  }
+
+  bool Extend(const void* data) {
+    size_t byte_width = static_cast<size_t>(key_type_.type->byte_width());
+    bool extends = save_key_data_.size() != byte_width
+                       ? kDefaultExtends
+                       : 0 == memcmp(save_key_data_.data(), data, byte_width);
+    save_key_data_.resize(byte_width);
+    memcpy(save_key_data_.data(), data, byte_width);
+    return extends;
+  }
+
+  Result<GroupingSegment> GetNextSegment(const Scalar& scalar, int64_t offset,
+                                         int64_t length) {
+    ARROW_RETURN_NOT_OK(CheckType(*scalar.type));
+    if (!scalar.is_valid) {
+      return Status::Invalid("segmenting an invalid scalar");
+    }
+    auto data = checked_cast<const PrimitiveScalarBase&>(scalar).data();
+    bool extends = length > 0 ? Extend(data) : kEmptyExtends;
+    return MakeSegment(length, 0, length, extends);
+  }
+
+  Result<GroupingSegment> GetNextSegment(const DataType& array_type,
+                                         const uint8_t* array_bytes, int64_t offset,
+                                         int64_t length) {
+    RETURN_NOT_OK(CheckType(array_type));
+    int64_t byte_width = array_type.byte_width();
+    int64_t match_length = GetMatchLength(array_bytes + offset * byte_width, byte_width,
+                                          array_bytes, offset, length);
+    bool extends = length > 0 ? Extend(array_bytes + offset * byte_width) : kEmptyExtends;
+    return MakeSegment(length, offset, match_length, extends);
+  }
+
+  Result<GroupingSegment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {key_type_}));
+    if (offset == batch.length) {
+      return MakeSegment(batch.length, offset, 0, kEmptyExtends);
+    }
+    const auto& value = batch.values[0];
+    if (value.is_scalar()) {
+      return GetNextSegment(*value.scalar, offset, batch.length);
+    }
+    ARROW_DCHECK(value.is_array());
+    const auto& array = value.array;
+    if (array.GetNullCount() > 0) {
+      return Status::NotImplemented("segmenting a nullable array");
+    }
+    return GetNextSegment(*array.type, GetValuesAsBytes(array), offset, batch.length);
+  }
+
+  Result<GroupingSegment> GetNextSegment(const ExecBatch& batch,
+                                         int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {key_type_}));
+    if (offset == batch.length) {
+      return MakeSegment(batch.length, offset, 0, kEmptyExtends);
+    }
+    const auto& value = batch.values[0];
+    if (value.is_scalar()) {
+      return GetNextSegment(*value.scalar(), offset, batch.length);
+    }
+    if (value.is_array()) {
+      auto array = value.array();
+      if (array->GetNullCount() > 0) {
+        return Status::NotImplemented("segmenting a nullable array");
+      }
+      return GetNextSegment(*array->type, GetValuesAsBytes(*array), offset, batch.length);
+    }
+    if (value.is_chunked_array()) {
+      auto array = value.chunked_array();
+      if (array->null_count() > 0) {
+        return Status::NotImplemented("segmenting a nullable array");
+      }
+      return GetNextSegmentChunked(array, offset, bound_extend_);
+    }
+    return Status::Invalid("segmenting unsupported value kind ", value.kind());
+  }
+
+ private:
+  TypeHolder key_type_;
+  std::vector<uint8_t> save_key_data_;
+  ExtendFunc bound_extend_ = [this](const void* data) { return Extend(data); };
+};
+
+struct AnyKeysGroupingSegmenter : public BaseGroupingSegmenter {
+  static Result<std::unique_ptr<GroupingSegmenter>> Make(
+      const std::vector<TypeHolder>& key_types, ExecContext* ctx) {
+    ARROW_RETURN_NOT_OK(Grouper::Make(key_types, ctx));  // check types
+    return std::make_unique<AnyKeysGroupingSegmenter>(key_types, ctx);
+  }
+
+  AnyKeysGroupingSegmenter(const std::vector<TypeHolder>& key_types, ExecContext* ctx)
+      : BaseGroupingSegmenter(key_types),
+        ctx_(ctx),
+        grouper_(nullptr),
+        save_group_id_(kNoGroupId) {}
+
+  Status Reset() override {
+    grouper_ = nullptr;
+    save_group_id_ = kNoGroupId;
+    return Status::OK();
+  }
+
+  bool Extend(const void* data) {
+    auto group_id = *static_cast<const group_id_t*>(data);
+    bool extends =
+        save_group_id_ == kNoGroupId ? kDefaultExtends : save_group_id_ == group_id;
+    save_group_id_ = group_id;
+    return extends;
+  }
+
+  template <typename Batch>
+  Result<group_id_t> MapGroupIdAt(const Batch& batch, int64_t offset) {
+    if (offset < 0 || offset >= batch.length) {
+      return Status::Invalid("requesting group id out of bounds");
+    }
+    if (!grouper_) return kNoGroupId;
+    ARROW_ASSIGN_OR_RAISE(auto datum, grouper_->Consume(batch, offset,
+                                                        /*consume_length=*/1));
+    if (!(datum.is_array() || datum.is_chunked_array())) {
+      return Status::Invalid("accessing unsupported datum kind ", datum.kind());
+    }
+    const std::shared_ptr<ArrayData>& data =
+        datum.is_array() ? datum.array() : datum.chunked_array()->chunk(0)->data();
+    ARROW_DCHECK(data->GetNullCount() == 0);
+    DCHECK_EQ(data->type->id(), GroupIdType::type_id);
+    DCHECK_EQ(1, data->length);
+    const group_id_t* values = data->GetValues<group_id_t>(1);
+    return values[0];
+  }
+
+  template <typename Batch>
+  Result<GroupingSegment> GetNextSegmentImpl(const Batch& batch, int64_t offset) {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, key_types_));
+    if (offset == batch.length) {
+      return MakeSegment(batch.length, offset, 0, kEmptyExtends);
+    }
+    // ARROW-18311: make Grouper support Reset()
+    // so it can be cached instead of recreated below
+    //
+    // the group id must be computed prior to resetting the grouper, since it is compared
+    // to save_group_id_, and after resetting the grouper produces incomparable group ids
+    ARROW_ASSIGN_OR_RAISE(auto group_id, MapGroupIdAt(batch, offset));
+    ExtendFunc bound_extend = [this, group_id](const void* data) {
+      bool extends = Extend(&group_id);
+      save_group_id_ = *static_cast<const group_id_t*>(data);
+      return extends;
+    };
+    ARROW_ASSIGN_OR_RAISE(grouper_, Grouper::Make(key_types_, ctx_));  // TODO: reset it
+    ARROW_ASSIGN_OR_RAISE(auto datum, grouper_->Consume(batch, offset));
+    if (datum.is_array()) {
+      const std::shared_ptr<ArrayData>& data = datum.array();
+      ARROW_DCHECK(data->GetNullCount() == 0);
+      DCHECK_EQ(data->type->id(), GroupIdType::type_id);
+      const group_id_t* values = data->GetValues<group_id_t>(1);
+      int64_t cursor;
+      for (cursor = 1; cursor < data->length; cursor++) {
+        if (values[0] != values[cursor]) break;
+      }
+      int64_t length = std::min(cursor, batch.length - offset);
+      bool extends = length > 0 ? bound_extend(values) : kEmptyExtends;
+      return MakeSegment(batch.length, offset, length, extends);
+    } else if (datum.is_chunked_array()) {
+      ARROW_ASSIGN_OR_RAISE(
+          auto segment, GetNextSegmentChunked(datum.chunked_array(), 0, bound_extend));
+      segment.offset += offset;
+      return segment;
+    } else {
+      return Status::Invalid("segmenting unsupported datum kind ", datum.kind());
+    }
+  }
+
+  Result<GroupingSegment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    return GetNextSegmentImpl(batch, offset);
+  }
+
+  Result<GroupingSegment> GetNextSegment(const ExecBatch& batch,
+                                         int64_t offset) override {
+    return GetNextSegmentImpl(batch, offset);
+  }
+
+ private:
+  ExecContext* const ctx_;
+  std::unique_ptr<Grouper> grouper_;
+  group_id_t save_group_id_;
+};
+
+Status CheckForConsume(int64_t batch_length, int64_t& consume_offset,
+                       int64_t* consume_length) {
+  if (consume_offset < 0) {
+    return Status::Invalid("invalid grouper consume offset: ", consume_offset);
+  }
+  if (*consume_length < 0) {
+    *consume_length = batch_length - consume_offset;
+  }
+  return Status::OK();
+}
+
+}  // namespace
+
+Result<std::unique_ptr<GroupingSegmenter>> GroupingSegmenter::Make(
+    const std::vector<TypeHolder>& key_types, bool nullable_keys, ExecContext* ctx) {
+  if (key_types.size() == 0) {
+    return NoKeysGroupingSegmenter::Make();
+  } else if (!nullable_keys && key_types.size() == 1) {
+    const DataType* type = key_types[0].type;
+    if (type != NULLPTR && is_fixed_width(*type)) {
+      return SimpleKeyGroupingSegmenter::Make(key_types[0]);
+    }
+  }
+  return AnyKeysGroupingSegmenter::Make(key_types, ctx);
+}
+
+namespace {
+
+struct BaseGrouper : public Grouper {
+  int IndexOfChunk(const ExecBatch& batch) {
+    int i = 0;
+    for (const auto& value : batch.values) {
+      if (value.is_chunked_array()) {
+        return i;
+      }
+      ++i;
+    }
+    return -1;
+  }
+
+  bool HasConsistentChunks(const ExecBatch& batch, int index_of_chunk) {
+    auto first_chunked_array = batch.values[index_of_chunk].chunked_array();
+    if (first_chunked_array < 0) {
+      // having no chunks is considered consistent
+      return true;
+    }
+    int num_chunks = first_chunked_array->num_chunks();
+    int64_t length = first_chunked_array->length();
+    for (const auto& value : batch.values) {
+      if (!value.is_chunked_array()) {
+        continue;
+      }
+      auto curr_chunk = value.chunked_array();
+      if (num_chunks != curr_chunk->num_chunks() || length != curr_chunk->length()) {
+        return false;
+      }
+    }
+    if (num_chunks > 0) {
+      for (int i = 0; i < num_chunks; i++) {
+        int64_t chunk_length = first_chunked_array->chunk(i)->length();
+        for (const auto& value : batch.values) {
+          if (!value.is_chunked_array()) {
+            continue;
+          }
+          auto curr_chunk = value.chunked_array();
+          if (chunk_length != curr_chunk->chunk(i)->length()) {
+            return false;
+          }
+        }
+      }
+    }
+    return true;
+  }
+
+  using Grouper::Consume;
+
+  Result<Datum> Consume(const ExecBatch& batch, int64_t consume_offset,

Review Comment:
   This method can be considerably simplified since we don't have to worry about chunked arrays.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rtpsw commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "rtpsw (via GitHub)" <gi...@apache.org>.
rtpsw commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1117885846


##########
cpp/src/arrow/compute/row/grouper.cc:
##########
@@ -39,12 +43,521 @@
 namespace arrow {
 
 using internal::checked_cast;
+using internal::PrimitiveScalarBase;
 
 namespace compute {
 
 namespace {
 
-struct GrouperImpl : Grouper {
+constexpr uint32_t kNoGroupId = std::numeric_limits<uint32_t>::max();
+
+using group_id_t = std::remove_const<decltype(kNoGroupId)>::type;
+using GroupIdType = CTypeTraits<group_id_t>::ArrowType;
+auto group_id_type = std::make_shared<GroupIdType>();
+
+inline const uint8_t* GetValuesAsBytes(const ArrayData& data, int64_t offset = 0) {
+  DCHECK_GT(data.type->byte_width(), 0);
+  int64_t absolute_byte_offset = (data.offset + offset) * data.type->byte_width();
+  return data.GetValues<uint8_t>(1, absolute_byte_offset);
+}
+
+inline const uint8_t* GetValuesAsBytes(const ArraySpan& data, int64_t offset = 0) {
+  DCHECK_GT(data.type->byte_width(), 0);
+  int64_t absolute_byte_offset = (data.offset + offset) * data.type->byte_width();
+  return data.GetValues<uint8_t>(1, absolute_byte_offset);
+}
+
+template <typename Value>
+Status CheckForGetNextSegment(const std::vector<Value>& values, int64_t length,
+                              int64_t offset, const std::vector<TypeHolder>& key_types) {
+  if (offset < 0 || offset > length) {
+    return Status::Invalid("invalid grouping segmenter offset: ", offset);
+  }
+  if (values.size() != key_types.size()) {
+    return Status::Invalid("expected batch size ", key_types.size(), " but got ",
+                           values.size());
+  }
+  for (size_t i = 0; i < key_types.size(); i++) {
+    const auto& value = values[i];
+    const auto& key_type = key_types[i];
+    if (*value.type() != *key_type.type) {
+      return Status::Invalid("expected batch value ", i, " of type ", *key_type.type,
+                             " but got ", *value.type());
+    }
+  }
+  return Status::OK();
+}
+
+template <typename Batch>
+enable_if_t<std::is_same<Batch, ExecSpan>::value || std::is_same<Batch, ExecBatch>::value,
+            Status>
+CheckForGetNextSegment(const Batch& batch, int64_t offset,
+                       const std::vector<TypeHolder>& key_types) {
+  return CheckForGetNextSegment(batch.values, batch.length, offset, key_types);
+}
+
+struct BaseGroupingSegmenter : public GroupingSegmenter {
+  explicit BaseGroupingSegmenter(const std::vector<TypeHolder>& key_types)
+      : key_types_(key_types) {}
+
+  const std::vector<TypeHolder>& key_types() const override { return key_types_; }
+
+  std::vector<TypeHolder> key_types_;
+};
+
+GroupingSegment MakeSegment(int64_t batch_length, int64_t offset, int64_t length,
+                            bool extends) {
+  return GroupingSegment{offset, length, offset + length >= batch_length, extends};
+}
+
+int64_t GetMatchLength(const uint8_t* match_bytes, int64_t match_width,

Review Comment:
   We could create an issue to add support for declaring ordered aggregation in `AggregationNodeOptions`, in which case the input segment-keys ordering would be enforced and binary-search optimizations applied.



##########
cpp/src/arrow/compute/row/grouper.cc:
##########
@@ -39,12 +43,521 @@
 namespace arrow {
 
 using internal::checked_cast;
+using internal::PrimitiveScalarBase;
 
 namespace compute {
 
 namespace {
 
-struct GrouperImpl : Grouper {
+constexpr uint32_t kNoGroupId = std::numeric_limits<uint32_t>::max();
+
+using group_id_t = std::remove_const<decltype(kNoGroupId)>::type;
+using GroupIdType = CTypeTraits<group_id_t>::ArrowType;
+auto group_id_type = std::make_shared<GroupIdType>();
+
+inline const uint8_t* GetValuesAsBytes(const ArrayData& data, int64_t offset = 0) {
+  DCHECK_GT(data.type->byte_width(), 0);
+  int64_t absolute_byte_offset = (data.offset + offset) * data.type->byte_width();
+  return data.GetValues<uint8_t>(1, absolute_byte_offset);
+}
+
+inline const uint8_t* GetValuesAsBytes(const ArraySpan& data, int64_t offset = 0) {
+  DCHECK_GT(data.type->byte_width(), 0);
+  int64_t absolute_byte_offset = (data.offset + offset) * data.type->byte_width();
+  return data.GetValues<uint8_t>(1, absolute_byte_offset);
+}
+
+template <typename Value>
+Status CheckForGetNextSegment(const std::vector<Value>& values, int64_t length,
+                              int64_t offset, const std::vector<TypeHolder>& key_types) {
+  if (offset < 0 || offset > length) {
+    return Status::Invalid("invalid grouping segmenter offset: ", offset);
+  }
+  if (values.size() != key_types.size()) {
+    return Status::Invalid("expected batch size ", key_types.size(), " but got ",
+                           values.size());
+  }
+  for (size_t i = 0; i < key_types.size(); i++) {
+    const auto& value = values[i];
+    const auto& key_type = key_types[i];
+    if (*value.type() != *key_type.type) {
+      return Status::Invalid("expected batch value ", i, " of type ", *key_type.type,
+                             " but got ", *value.type());
+    }
+  }
+  return Status::OK();
+}
+
+template <typename Batch>
+enable_if_t<std::is_same<Batch, ExecSpan>::value || std::is_same<Batch, ExecBatch>::value,
+            Status>
+CheckForGetNextSegment(const Batch& batch, int64_t offset,
+                       const std::vector<TypeHolder>& key_types) {
+  return CheckForGetNextSegment(batch.values, batch.length, offset, key_types);
+}
+
+struct BaseGroupingSegmenter : public GroupingSegmenter {
+  explicit BaseGroupingSegmenter(const std::vector<TypeHolder>& key_types)
+      : key_types_(key_types) {}
+
+  const std::vector<TypeHolder>& key_types() const override { return key_types_; }
+
+  std::vector<TypeHolder> key_types_;
+};
+
+GroupingSegment MakeSegment(int64_t batch_length, int64_t offset, int64_t length,
+                            bool extends) {
+  return GroupingSegment{offset, length, offset + length >= batch_length, extends};
+}
+
+int64_t GetMatchLength(const uint8_t* match_bytes, int64_t match_width,
+                       const uint8_t* array_bytes, int64_t offset, int64_t length) {
+  int64_t cursor, byte_cursor;
+  for (cursor = offset, byte_cursor = match_width * cursor; cursor < length;
+       cursor++, byte_cursor += match_width) {
+    if (memcmp(match_bytes, array_bytes + byte_cursor,
+               static_cast<size_t>(match_width)) != 0) {
+      break;
+    }
+  }
+  return std::min(cursor, length - offset);
+}
+
+using ExtendFunc = std::function<bool(const void*)>;
+constexpr bool kDefaultExtends = true;
+constexpr bool kEmptyExtends = true;
+
+Result<GroupingSegment> GetNextSegmentChunked(
+    const std::shared_ptr<ChunkedArray>& chunked_array, int64_t offset,
+    ExtendFunc extend) {
+  if (offset >= chunked_array->length()) {
+    return MakeSegment(chunked_array->length(), chunked_array->length(), 0,
+                       kEmptyExtends);
+  }
+  int64_t remaining_offset = offset;
+  const auto& arrays = chunked_array->chunks();
+  for (size_t i = 0; remaining_offset >= 0 && i < arrays.size(); i++) {
+    // look up chunk containing offset
+    int64_t array_length = arrays[i]->length();
+    if (remaining_offset < array_length) {
+      // found - switch to matching
+      int64_t match_width = arrays[i]->type()->byte_width();
+      const uint8_t* match_bytes = GetValuesAsBytes(*arrays[i]->data(), remaining_offset);
+      int64_t total_match_length = 0;
+      for (; i < arrays.size(); i++) {
+        int64_t array_length = arrays[i]->length();
+        if (array_length <= 0) continue;
+        const uint8_t* array_bytes = GetValuesAsBytes(*arrays[i]->data());
+        int64_t match_length = GetMatchLength(match_bytes, match_width, array_bytes,
+                                              remaining_offset, array_length);
+        total_match_length += match_length;
+        remaining_offset = 0;
+        if (match_length < array_length - remaining_offset) break;
+      }
+      bool extends = extend(match_bytes);
+      return MakeSegment(chunked_array->length(), offset, total_match_length, extends);
+    }
+    remaining_offset -= array_length;
+  }
+  return Status::Invalid("segmenting invalid chunked array value");
+}

Review Comment:
   Done.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rtpsw commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "rtpsw (via GitHub)" <gi...@apache.org>.
rtpsw commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1117884067


##########
cpp/src/arrow/compute/row/grouper.h:
##########
@@ -30,6 +30,49 @@
 namespace arrow {
 namespace compute {
 
+/// \brief A segment of contiguous rows for grouping
+struct ARROW_EXPORT GroupingSegment {
+  int64_t offset;
+  int64_t length;
+  bool is_open;
+  bool extends;
+};
+
+inline bool operator==(const GroupingSegment& segment1, const GroupingSegment& segment2) {
+  return segment1.offset == segment2.offset && segment1.length == segment2.length &&
+         segment1.is_open == segment2.is_open && segment1.extends == segment2.extends;
+}
+inline bool operator!=(const GroupingSegment& segment1, const GroupingSegment& segment2) {
+  return !(segment1 == segment2);
+}
+
+/// \brief Computes grouping segments for a batch. Each segment covers rows with identical
+/// values in the batch. The values in the batch are often selected as keys from a larger
+/// batch.
+class ARROW_EXPORT GroupingSegmenter {
+ public:
+  virtual ~GroupingSegmenter() = default;
+
+  /// \brief Construct a GroupingSegmenter which receives the specified key types
+  static Result<std::unique_ptr<GroupingSegmenter>> Make(
+      const std::vector<TypeHolder>& key_types, bool nullable_keys = false,

Review Comment:
   Done.



##########
cpp/src/arrow/compute/exec/aggregate_node.cc:
##########
@@ -326,46 +446,86 @@ class ScalarAggregateNode : public ExecNode, public TracedNode {
   }
 
  private:
-  Status Finish() {
-    auto scope = TraceFinish();
+  Status ReconstructAggregates() {
+    const auto& input_schema = *inputs()[0]->output_schema();
+    auto exec_ctx = plan()->query_context()->exec_context();
+    for (size_t i = 0; i < kernels_.size(); ++i) {
+      std::vector<TypeHolder> in_types;
+      for (const auto& target : target_fieldsets_[i]) {
+        in_types.emplace_back(input_schema.field(target)->type().get());
+      }
+      states_[i].resize(plan()->query_context()->max_concurrency());
+      KernelContext kernel_ctx{exec_ctx};
+      RETURN_NOT_OK(Kernel::InitAll(
+          &kernel_ctx, KernelInitArgs{kernels_[i], in_types, aggs_[i].options.get()},
+          &states_[i]));
+    }
+    return Status::OK();
+  }
+
+  Status OutputResult(bool is_last = false, bool traced = false) {
+    if (is_last && !traced) {
+      auto scope = TraceFinish();
+      return OutputResult(is_last, /*traced=*/true);
+    }
+    GatedUniqueLock lock(gated_shared_mutex_);
     ExecBatch batch{{}, 1};
-    batch.values.resize(kernels_.size());
+    batch.values.resize(kernels_.size() + segment_field_ids_.size());
 
     for (size_t i = 0; i < kernels_.size(); ++i) {
       util::tracing::Span span;
       START_COMPUTE_SPAN(span, aggs_[i].function,
                          {{"function.name", aggs_[i].function},
                           {"function.options",
                            aggs_[i].options ? aggs_[i].options->ToString() : "<NULLPTR>"},
-                          {"function.kind", std::string(kind_name()) + "::Finalize"}});
+                          {"function.kind", std::string(kind_name()) + "::Output"}});
       KernelContext ctx{plan()->query_context()->exec_context()};
       ARROW_ASSIGN_OR_RAISE(auto merged, ScalarAggregateKernel::MergeAll(
                                              kernels_[i], &ctx, std::move(states_[i])));
       RETURN_NOT_OK(kernels_[i]->finalize(&ctx, &batch.values[i]));
     }
+    PlaceFields(batch, kernels_.size(), segmenter_values_);
 
-    return output_->InputReceived(this, std::move(batch));
+    ARROW_RETURN_NOT_OK(output_->InputReceived(this, std::move(batch)));
+    total_output_batches_++;
+    if (is_last) {
+      ARROW_RETURN_NOT_OK(output_->InputFinished(this, total_output_batches_));
+    } else {
+      ARROW_RETURN_NOT_OK(ReconstructAggregates());
+    }
+    return Status::OK();
   }
 
+  std::unique_ptr<GroupingSegmenter> segmenter_;
+  const std::vector<int> segment_field_ids_;
+  std::vector<Datum> segmenter_values_;
+
   const std::vector<std::vector<int>> target_fieldsets_;
   const std::vector<Aggregate> aggs_;
   const std::vector<const ScalarAggregateKernel*> kernels_;
 
   std::vector<std::vector<std::unique_ptr<KernelState>>> states_;
 
   AtomicCounter input_counter_;
+  int64_t total_output_batches_ = 0;

Review Comment:
   Done.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rtpsw commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "rtpsw (via GitHub)" <gi...@apache.org>.
rtpsw commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1117884056


##########
cpp/src/arrow/compute/exec/options.h:
##########
@@ -199,21 +199,32 @@ class ARROW_EXPORT ProjectNodeOptions : public ExecNodeOptions {
   std::vector<std::string> names;
 };
 
-/// \brief Make a node which aggregates input batches, optionally grouped by keys.
+/// \brief Make a node which aggregates input batches, optionally grouped by keys and
+/// optionally segmented by segment-keys. Both keys and segment-keys determine the group.
+/// However segment-keys are also used for determining grouping segments, which should be
+/// large, and allow streaming a partial aggregation result after processing each segment.
+/// One common use-case for segment-keys is ordered aggregation, in which the segment-key
+/// attribute specifies a column with non-decreasing values or a lexigographically-ordered
+/// set of such columns.
 ///
 /// If the keys attribute is a non-empty vector, then each aggregate in `aggregates` is
 /// expected to be a HashAggregate function. If the keys attribute is an empty vector,
 /// then each aggregate is assumed to be a ScalarAggregate function.
 class ARROW_EXPORT AggregateNodeOptions : public ExecNodeOptions {
  public:
   explicit AggregateNodeOptions(std::vector<Aggregate> aggregates,
-                                std::vector<FieldRef> keys = {})
-      : aggregates(std::move(aggregates)), keys(std::move(keys)) {}
+                                std::vector<FieldRef> keys = {},
+                                std::vector<FieldRef> segment_keys = {})
+      : aggregates(std::move(aggregates)),
+        keys(std::move(keys)),
+        segment_keys(std::move(segment_keys)) {}
 
   // aggregations which will be applied to the targetted fields
   std::vector<Aggregate> aggregates;
   // keys by which aggregations will be grouped
   std::vector<FieldRef> keys;
+  // keys by which aggregations will be segmented

Review Comment:
   Done.



##########
cpp/src/arrow/compute/row/grouper.h:
##########
@@ -30,6 +30,49 @@
 namespace arrow {
 namespace compute {
 
+/// \brief A segment of contiguous rows for grouping
+struct ARROW_EXPORT GroupingSegment {
+  int64_t offset;
+  int64_t length;
+  bool is_open;
+  bool extends;
+};
+
+inline bool operator==(const GroupingSegment& segment1, const GroupingSegment& segment2) {
+  return segment1.offset == segment2.offset && segment1.length == segment2.length &&
+         segment1.is_open == segment2.is_open && segment1.extends == segment2.extends;
+}
+inline bool operator!=(const GroupingSegment& segment1, const GroupingSegment& segment2) {
+  return !(segment1 == segment2);
+}
+
+/// \brief Computes grouping segments for a batch. Each segment covers rows with identical
+/// values in the batch. The values in the batch are often selected as keys from a larger
+/// batch.
+class ARROW_EXPORT GroupingSegmenter {
+ public:
+  virtual ~GroupingSegmenter() = default;
+
+  /// \brief Construct a GroupingSegmenter which receives the specified key types

Review Comment:
   Done.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] icexelloss commented on pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "icexelloss (via GitHub)" <gi...@apache.org>.
icexelloss commented on PR #34311:
URL: https://github.com/apache/arrow/pull/34311#issuecomment-1442394934

   > See [segmented aggregation as a generalization of ordered aggregation](https://github.com/apache/arrow/pull/14352#issuecomment-1272400060) (in the PR replaced by this one).
   
   @rtpsw Can you add some code comment explain the concept? This way the code is more documentation and the reader doesn't need to jump through review/pr links to understand the code.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rtpsw commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "rtpsw (via GitHub)" <gi...@apache.org>.
rtpsw commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1116934501


##########
cpp/src/arrow/compute/kernels/hash_aggregate_test.cc:
##########
@@ -174,81 +242,117 @@ Result<Datum> RunGroupBy(const BatchesWithSchema& input,
   ARROW_ASSIGN_OR_RAISE(std::vector<ExecBatch> output_batches,
                         start_and_collect.MoveResult());
 
-  ArrayVector out_arrays(aggregates.size() + key_names.size());
   const auto& output_schema = plan->nodes()[0]->output()->output_schema();
+  if (!segmented) {
+    return MakeGroupByOutput(output_batches, output_schema, aggregates.size(),
+                             key_names.size(), naive);
+  }
+
+  std::vector<ArrayVector> out_arrays(aggregates.size() + key_names.size() +
+                                      segment_key_names.size());
   for (size_t i = 0; i < out_arrays.size(); ++i) {
     std::vector<std::shared_ptr<Array>> arrays(output_batches.size());
     for (size_t j = 0; j < output_batches.size(); ++j) {
-      arrays[j] = output_batches[j].values[i].make_array();
+      auto& value = output_batches[j].values[i];
+      if (value.is_scalar()) {
+        ARROW_ASSIGN_OR_RAISE(
+            arrays[j], MakeArrayFromScalar(*value.scalar(), output_batches[j].length));
+      } else if (value.is_array()) {
+        arrays[j] = value.make_array();
+      } else {
+        return Status::Invalid("GroupByUsingExecPlan unsupported value kind ",
+                               ToString(value.kind()));
+      }
     }
     if (arrays.empty()) {
+      arrays.resize(1);
       ARROW_ASSIGN_OR_RAISE(
-          out_arrays[i],
-          MakeArrayOfNull(output_schema->field(static_cast<int>(i))->type(),
-                          /*length=*/0));
-    } else {
-      ARROW_ASSIGN_OR_RAISE(out_arrays[i], Concatenate(arrays));
+          arrays[0], MakeArrayOfNull(output_schema->field(static_cast<int>(i))->type(),
+                                     /*length=*/0));
     }
+    out_arrays[i] = {std::move(arrays)};
   }
 
-  // The exec plan may reorder the output rows.  The tests are all setup to expect ouptut
-  // in ascending order of keys.  So we need to sort the result by the key columns.  To do
-  // that we create a table using the key columns, calculate the sort indices from that
-  // table (sorting on all fields) and then use those indices to calculate our result.
-  std::vector<std::shared_ptr<Field>> key_fields;
-  std::vector<std::shared_ptr<Array>> key_columns;
-  std::vector<SortKey> sort_keys;
-  for (std::size_t i = 0; i < key_names.size(); i++) {
-    const std::shared_ptr<Array>& arr = out_arrays[i + aggregates.size()];
-    if (arr->type_id() == Type::DICTIONARY) {
-      // Can't sort dictionary columns so need to decode
-      auto dict_arr = checked_pointer_cast<DictionaryArray>(arr);
-      ARROW_ASSIGN_OR_RAISE(auto decoded_arr,
-                            Take(*dict_arr->dictionary(), *dict_arr->indices()));
-      key_columns.push_back(decoded_arr);
-      key_fields.push_back(
-          field("name_does_not_matter", dict_arr->dict_type()->value_type()));
-    } else {
-      key_columns.push_back(arr);
-      key_fields.push_back(field("name_does_not_matter", arr->type()));
+  if (segmented && segment_key_names.size() > 0) {
+    ArrayVector struct_arrays;
+    struct_arrays.reserve(output_batches.size());
+    for (size_t j = 0; j < output_batches.size(); ++j) {
+      ArrayVector struct_fields;
+      struct_fields.reserve(out_arrays.size());
+      for (auto out_array : out_arrays) {
+        struct_fields.push_back(out_array[j]);
+      }
+      ARROW_ASSIGN_OR_RAISE(auto struct_array,
+                            StructArray::Make(struct_fields, output_schema->fields()));
+      struct_arrays.push_back(struct_array);
     }
-    sort_keys.emplace_back(static_cast<int>(i));
+    return ChunkedArray::Make(struct_arrays);
+  } else {
+    ArrayVector struct_fields(out_arrays.size());
+    for (size_t i = 0; i < out_arrays.size(); ++i) {
+      ARROW_ASSIGN_OR_RAISE(struct_fields[i], Concatenate(out_arrays[i]));
+    }
+    return StructArray::Make(std::move(struct_fields), output_schema->fields());
   }
-  std::shared_ptr<Schema> key_schema = schema(std::move(key_fields));
-  std::shared_ptr<Table> key_table = Table::Make(std::move(key_schema), key_columns);
-  SortOptions sort_options(std::move(sort_keys));
-  ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Array> sort_indices,
-                        SortIndices(key_table, sort_options));
+}
 
-  ARROW_ASSIGN_OR_RAISE(
-      std::shared_ptr<Array> struct_arr,
-      StructArray::Make(std::move(out_arrays), output_schema->fields()));
+Result<Datum> RunGroupBy(const BatchesWithSchema& input,
+                         const std::vector<std::string>& key_names,
+                         const std::vector<std::string>& segment_key_names,
+                         const std::vector<Aggregate>& aggregates, bool use_threads,
+                         bool segmented = false, bool naive = false) {
+  if (segment_key_names.size() > 0) {
+    ARROW_ASSIGN_OR_RAISE(auto thread_pool, arrow::internal::ThreadPool::Make(1));
+    ExecContext seq_ctx(default_memory_pool(), thread_pool.get());
+    return RunGroupBy(input, key_names, segment_key_names, aggregates, &seq_ctx,
+                      use_threads, segmented, naive);
+  } else {
+    return RunGroupBy(input, key_names, segment_key_names, aggregates,
+                      threaded_exec_context(), use_threads, segmented, naive);
+  }
+}
 
-  return Take(struct_arr, sort_indices);
+Result<Datum> RunGroupBy(const BatchesWithSchema& input,
+                         const std::vector<std::string>& key_names,
+                         const std::vector<Aggregate>& aggregates, bool use_threads,
+                         bool segmented = false, bool naive = false) {
+  return RunGroupBy(input, key_names, {}, aggregates, use_threads, segmented);
 }
 
 /// Simpler overload where you can give the columns as datums
 Result<Datum> RunGroupBy(const std::vector<Datum>& arguments,
                          const std::vector<Datum>& keys,
-                         const std::vector<Aggregate>& aggregates,
-                         bool use_threads = false) {
+                         const std::vector<Datum>& segment_keys,
+                         const std::vector<Aggregate>& aggregates, bool use_threads,
+                         bool segmented = false, bool naive = false) {

Review Comment:
   I wanted to test the case of segmented aggregation with empty segment-keys. We could decide to remove this.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] icexelloss commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "icexelloss (via GitHub)" <gi...@apache.org>.
icexelloss commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1117313825


##########
cpp/src/arrow/compute/exec/aggregate_node.cc:
##########
@@ -326,46 +446,86 @@ class ScalarAggregateNode : public ExecNode, public TracedNode {
   }
 
  private:
-  Status Finish() {
-    auto scope = TraceFinish();
+  Status ReconstructAggregates() {
+    const auto& input_schema = *inputs()[0]->output_schema();
+    auto exec_ctx = plan()->query_context()->exec_context();
+    for (size_t i = 0; i < kernels_.size(); ++i) {
+      std::vector<TypeHolder> in_types;
+      for (const auto& target : target_fieldsets_[i]) {
+        in_types.emplace_back(input_schema.field(target)->type().get());
+      }
+      states_[i].resize(plan()->query_context()->max_concurrency());
+      KernelContext kernel_ctx{exec_ctx};
+      RETURN_NOT_OK(Kernel::InitAll(
+          &kernel_ctx, KernelInitArgs{kernels_[i], in_types, aggs_[i].options.get()},
+          &states_[i]));
+    }
+    return Status::OK();
+  }
+
+  Status OutputResult(bool is_last = false, bool traced = false) {
+    if (is_last && !traced) {
+      auto scope = TraceFinish();
+      return OutputResult(is_last, /*traced=*/true);

Review Comment:
   (i.e., does it change anything if we simply remove the return statement here)



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rtpsw commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "rtpsw (via GitHub)" <gi...@apache.org>.
rtpsw commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1127113622


##########
cpp/src/arrow/compute/row/grouper.cc:
##########
@@ -39,12 +43,330 @@
 namespace arrow {
 
 using internal::checked_cast;
+using internal::PrimitiveScalarBase;
 
 namespace compute {
 
 namespace {
 
-struct GrouperImpl : Grouper {
+constexpr uint32_t kNoGroupId = std::numeric_limits<uint32_t>::max();
+
+using group_id_t = std::remove_const<decltype(kNoGroupId)>::type;
+using GroupIdType = CTypeTraits<group_id_t>::ArrowType;
+auto g_group_id_type = std::make_shared<GroupIdType>();
+
+inline const uint8_t* GetValuesAsBytes(const ArraySpan& data, int64_t offset = 0) {
+  DCHECK_GT(data.type->byte_width(), 0);
+  int64_t absolute_byte_offset = (data.offset + offset) * data.type->byte_width();
+  return data.GetValues<uint8_t>(1, absolute_byte_offset);
+}
+
+template <typename Value>
+Status CheckForGetNextSegment(const std::vector<Value>& values, int64_t length,
+                              int64_t offset, const std::vector<TypeHolder>& key_types) {
+  if (offset < 0 || offset > length) {
+    return Status::Invalid("invalid grouping segmenter offset: ", offset);
+  }
+  if (values.size() != key_types.size()) {
+    return Status::Invalid("expected batch size ", key_types.size(), " but got ",
+                           values.size());
+  }
+  for (size_t i = 0; i < key_types.size(); i++) {
+    const auto& value = values[i];
+    const auto& key_type = key_types[i];
+    if (*value.type() != *key_type.type) {
+      return Status::Invalid("expected batch value ", i, " of type ", *key_type.type,
+                             " but got ", *value.type());
+    }
+  }
+  return Status::OK();
+}
+
+template <typename Batch>
+enable_if_t<std::is_same<Batch, ExecSpan>::value || std::is_same<Batch, ExecBatch>::value,
+            Status>
+CheckForGetNextSegment(const Batch& batch, int64_t offset,
+                       const std::vector<TypeHolder>& key_types) {
+  return CheckForGetNextSegment(batch.values, batch.length, offset, key_types);
+}
+
+struct BaseRowSegmenter : public RowSegmenter {
+  explicit BaseRowSegmenter(const std::vector<TypeHolder>& key_types)
+      : key_types_(key_types) {}
+
+  const std::vector<TypeHolder>& key_types() const override { return key_types_; }
+
+  std::vector<TypeHolder> key_types_;
+};
+
+Segment MakeSegment(int64_t batch_length, int64_t offset, int64_t length, bool extends) {
+  return Segment{offset, length, offset + length >= batch_length, extends};
+}
+
+int64_t GetMatchLength(const uint8_t* match_bytes, int64_t match_width,
+                       const uint8_t* array_bytes, int64_t offset, int64_t length) {
+  int64_t cursor, byte_cursor;
+  for (cursor = offset, byte_cursor = match_width * cursor; cursor < length;
+       cursor++, byte_cursor += match_width) {
+    if (memcmp(match_bytes, array_bytes + byte_cursor,
+               static_cast<size_t>(match_width)) != 0) {
+      break;
+    }
+  }
+  return std::min(cursor, length) - offset;
+}
+
+using ExtendFunc = std::function<bool(const void*)>;
+constexpr bool kDefaultExtends = true;
+constexpr bool kEmptyExtends = true;
+
+struct NoKeysSegmenter : public BaseRowSegmenter {
+  static std::unique_ptr<RowSegmenter> Make() {
+    return std::make_unique<NoKeysSegmenter>();
+  }
+
+  NoKeysSegmenter() : BaseRowSegmenter({}) {}
+
+  Status Reset() override { return Status::OK(); }
+
+  Result<Segment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {}));
+    return MakeSegment(batch.length, offset, batch.length - offset, kDefaultExtends);
+  }
+};
+
+struct SimpleKeySegmenter : public BaseRowSegmenter {
+  static Result<std::unique_ptr<RowSegmenter>> Make(TypeHolder key_type) {
+    return std::make_unique<SimpleKeySegmenter>(key_type);
+  }
+
+  explicit SimpleKeySegmenter(TypeHolder key_type)
+      : BaseRowSegmenter({key_type}), key_type_(key_types_[0]), save_key_data_() {}
+
+  Status CheckType(const DataType& type) {
+    if (!is_fixed_width(type)) {
+      return Status::Invalid("SimpleKeySegmenter does not support type ", type);
+    }
+    return Status::OK();
+  }
+
+  Status Reset() override {
+    save_key_data_.resize(0);
+    return Status::OK();
+  }
+
+  // Checks whether the given grouping data extends the current segment, i.e., is equal to
+  // previously seen grouping data, which is updated with each invocation.
+  bool Extend(const void* data) {
+    size_t byte_width = static_cast<size_t>(key_type_.type->byte_width());
+    bool extends = save_key_data_.size() != byte_width
+                       ? kDefaultExtends
+                       : 0 == memcmp(save_key_data_.data(), data, byte_width);
+    save_key_data_.resize(byte_width);
+    memcpy(save_key_data_.data(), data, byte_width);
+    return extends;
+  }
+
+  Result<Segment> GetNextSegment(const Scalar& scalar, int64_t offset, int64_t length) {
+    ARROW_RETURN_NOT_OK(CheckType(*scalar.type));
+    if (!scalar.is_valid) {
+      return Status::Invalid("segmenting an invalid scalar");
+    }
+    auto data = checked_cast<const PrimitiveScalarBase&>(scalar).data();
+    bool extends = length > 0 ? Extend(data) : kEmptyExtends;
+    return MakeSegment(length, offset, length, extends);
+  }
+
+  Result<Segment> GetNextSegment(const DataType& array_type, const uint8_t* array_bytes,

Review Comment:
   Done.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rtpsw commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "rtpsw (via GitHub)" <gi...@apache.org>.
rtpsw commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1128200101


##########
cpp/src/arrow/compute/kernels/hash_aggregate_test.cc:
##########
@@ -174,81 +255,117 @@ Result<Datum> RunGroupBy(const BatchesWithSchema& input,
   ARROW_ASSIGN_OR_RAISE(std::vector<ExecBatch> output_batches,
                         start_and_collect.MoveResult());
 
-  ArrayVector out_arrays(aggregates.size() + key_names.size());
   const auto& output_schema = plan->nodes()[0]->output()->output_schema();
+  if (!segmented) {
+    return MakeGroupByOutput(output_batches, output_schema, aggregates.size(),
+                             key_names.size(), naive);
+  }
+
+  std::vector<ArrayVector> out_arrays(aggregates.size() + key_names.size() +
+                                      segment_key_names.size());
   for (size_t i = 0; i < out_arrays.size(); ++i) {
     std::vector<std::shared_ptr<Array>> arrays(output_batches.size());
     for (size_t j = 0; j < output_batches.size(); ++j) {
-      arrays[j] = output_batches[j].values[i].make_array();
+      auto& value = output_batches[j].values[i];
+      if (value.is_scalar()) {
+        ARROW_ASSIGN_OR_RAISE(
+            arrays[j], MakeArrayFromScalar(*value.scalar(), output_batches[j].length));
+      } else if (value.is_array()) {
+        arrays[j] = value.make_array();
+      } else {
+        return Status::Invalid("GroupByUsingExecPlan unsupported value kind ",
+                               ToString(value.kind()));
+      }
     }
     if (arrays.empty()) {
+      arrays.resize(1);
       ARROW_ASSIGN_OR_RAISE(
-          out_arrays[i],
-          MakeArrayOfNull(output_schema->field(static_cast<int>(i))->type(),
-                          /*length=*/0));
-    } else {
-      ARROW_ASSIGN_OR_RAISE(out_arrays[i], Concatenate(arrays));
+          arrays[0], MakeArrayOfNull(output_schema->field(static_cast<int>(i))->type(),
+                                     /*length=*/0));
     }
+    out_arrays[i] = {std::move(arrays)};
   }
 
-  // The exec plan may reorder the output rows.  The tests are all setup to expect ouptut
-  // in ascending order of keys.  So we need to sort the result by the key columns.  To do
-  // that we create a table using the key columns, calculate the sort indices from that
-  // table (sorting on all fields) and then use those indices to calculate our result.
-  std::vector<std::shared_ptr<Field>> key_fields;
-  std::vector<std::shared_ptr<Array>> key_columns;
-  std::vector<SortKey> sort_keys;
-  for (std::size_t i = 0; i < key_names.size(); i++) {
-    const std::shared_ptr<Array>& arr = out_arrays[i + aggregates.size()];
-    if (arr->type_id() == Type::DICTIONARY) {
-      // Can't sort dictionary columns so need to decode
-      auto dict_arr = checked_pointer_cast<DictionaryArray>(arr);
-      ARROW_ASSIGN_OR_RAISE(auto decoded_arr,
-                            Take(*dict_arr->dictionary(), *dict_arr->indices()));
-      key_columns.push_back(decoded_arr);
-      key_fields.push_back(
-          field("name_does_not_matter", dict_arr->dict_type()->value_type()));
-    } else {
-      key_columns.push_back(arr);
-      key_fields.push_back(field("name_does_not_matter", arr->type()));
+  if (segmented && segment_key_names.size() > 0) {
+    ArrayVector struct_arrays;
+    struct_arrays.reserve(output_batches.size());
+    for (size_t j = 0; j < output_batches.size(); ++j) {
+      ArrayVector struct_fields;
+      struct_fields.reserve(out_arrays.size());
+      for (auto out_array : out_arrays) {
+        struct_fields.push_back(out_array[j]);
+      }
+      ARROW_ASSIGN_OR_RAISE(auto struct_array,
+                            StructArray::Make(struct_fields, output_schema->fields()));
+      struct_arrays.push_back(struct_array);
     }
-    sort_keys.emplace_back(static_cast<int>(i));
+    return ChunkedArray::Make(struct_arrays);

Review Comment:
   When the output corresponds to multiple segments, it streams for one segment at a time. Since there is one array per segment, a ChunkedArray is used to collect them.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rtpsw commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "rtpsw (via GitHub)" <gi...@apache.org>.
rtpsw commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1122287793


##########
cpp/src/arrow/compute/exec/options.h:
##########
@@ -199,21 +199,39 @@ class ARROW_EXPORT ProjectNodeOptions : public ExecNodeOptions {
   std::vector<std::string> names;
 };
 
-/// \brief Make a node which aggregates input batches, optionally grouped by keys.
+/// \brief Make a node which aggregates input batches, optionally grouped by keys and
+/// optionally segmented by segment-keys. Both keys and segment-keys determine the group.
+/// However segment-keys are also used for determining grouping segments, which should be
+/// large, and allow streaming a partial aggregation result after processing each segment.

Review Comment:
   Sounds right.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] icexelloss commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "icexelloss (via GitHub)" <gi...@apache.org>.
icexelloss commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1123374394


##########
cpp/src/arrow/compute/exec/aggregate_node.cc:
##########
@@ -169,35 +186,117 @@ void AggregatesToString(std::stringstream* ss, const Schema& input_schema,
   *ss << ']';
 }
 
+template <typename BatchHandler>
+Status HandleSegments(std::unique_ptr<GroupingSegmenter>& segmenter,
+                      const ExecBatch& batch, const std::vector<int>& ids,
+                      const BatchHandler& handle_batch) {
+  int64_t offset = 0;
+  ARROW_ASSIGN_OR_RAISE(auto segment_exec_batch, batch.SelectValues(ids));
+  ExecSpan segment_batch(segment_exec_batch);
+  while (true) {
+    ARROW_ASSIGN_OR_RAISE(auto segment, segmenter->GetNextSegment(segment_batch, offset));
+    if (segment.offset >= segment_batch.length) break;  // condition of no-next-segment
+    ARROW_RETURN_NOT_OK(handle_batch(batch, segment));
+    offset = segment.offset + segment.length;
+  }
+  return Status::OK();
+}
+
+Status GetScalarFields(std::vector<Datum>* values_ptr, const ExecBatch& input_batch,
+                       const std::vector<int>& field_ids) {
+  DCHECK_GT(input_batch.length, 0);
+  std::vector<Datum>& values = *values_ptr;
+  int64_t row = input_batch.length - 1;
+  values.clear();
+  values.resize(field_ids.size());
+  for (size_t i = 0; i < field_ids.size(); i++) {
+    const Datum& value = input_batch.values[field_ids[i]];
+    if (value.is_scalar()) {
+      values[i] = value;
+    } else if (value.is_array()) {
+      ARROW_ASSIGN_OR_RAISE(auto scalar, value.make_array()->GetScalar(row));
+      values[i] = scalar;
+    } else {
+      DCHECK(false);
+    }
+  }
+  return Status::OK();
+}
+
+void PlaceFields(ExecBatch& batch, size_t base, std::vector<Datum>& values) {
+  DCHECK_LE(base + values.size(), batch.values.size());
+  for (size_t i = 0; i < values.size(); i++) {
+    batch.values[base + i] = values[i];
+  }
+}
+
 class ScalarAggregateNode : public ExecNode, public TracedNode {
  public:
   ScalarAggregateNode(ExecPlan* plan, std::vector<ExecNode*> inputs,
                       std::shared_ptr<Schema> output_schema,
+                      std::unique_ptr<GroupingSegmenter> segmenter,
+                      std::vector<int> segment_field_ids,
                       std::vector<std::vector<int>> target_fieldsets,
                       std::vector<Aggregate> aggs,
                       std::vector<const ScalarAggregateKernel*> kernels,
                       std::vector<std::vector<std::unique_ptr<KernelState>>> states)
       : ExecNode(plan, std::move(inputs), {"target"},
                  /*output_schema=*/std::move(output_schema)),
         TracedNode(this),
+        segmenter_(std::move(segmenter)),
+        segment_field_ids_(std::move(segment_field_ids)),
         target_fieldsets_(std::move(target_fieldsets)),
         aggs_(std::move(aggs)),
         kernels_(std::move(kernels)),
-        states_(std::move(states)) {}
+        states_(std::move(states)) {
+    const auto& input_schema = *this->inputs()[0]->output_schema();
+    for (size_t i = 0; i < kernels_.size(); ++i) {
+      std::vector<TypeHolder> in_types;
+      for (const auto& target : target_fieldsets_[i]) {
+        in_types.emplace_back(input_schema.field(target)->type().get());
+      }
+      in_typesets_.push_back(std::move(in_types));
+    }
+  }
 
   static Result<ExecNode*> Make(ExecPlan* plan, std::vector<ExecNode*> inputs,
                                 const ExecNodeOptions& options) {
     RETURN_NOT_OK(ValidateExecNodeInputs(plan, inputs, 1, "ScalarAggregateNode"));
 
     const auto& aggregate_options = checked_cast<const AggregateNodeOptions&>(options);
     auto aggregates = aggregate_options.aggregates;
+    const auto& keys = aggregate_options.keys;
+    const auto& segment_keys = aggregate_options.segment_keys;
+
+    if (keys.size() > 0) {
+      return Status::Invalid("Scalar aggregation with some key");
+    }
+    if (plan->query_context()->exec_context()->executor()->GetCapacity() > 1 &&
+        segment_keys.size() > 0) {
+      return Status::NotImplemented("Segmented aggregation in a multi-threaded plan");
+    }
 
     const auto& input_schema = *inputs[0]->output_schema();
     auto exec_ctx = plan->query_context()->exec_context();
 
+    std::vector<int> segment_field_ids(segment_keys.size());
+    std::vector<TypeHolder> segment_key_types(segment_keys.size());
+    for (size_t i = 0; i < segment_keys.size(); i++) {
+      ARROW_ASSIGN_OR_RAISE(auto match, segment_keys[i].FindOne(input_schema));
+      if (match.indices().size() > 1) {
+        // ARROW-18369: Support nested references as segment ids

Review Comment:
   I see



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] icexelloss commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "icexelloss (via GitHub)" <gi...@apache.org>.
icexelloss commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1123611908


##########
cpp/src/arrow/compute/exec/aggregate_node.cc:
##########
@@ -283,28 +386,36 @@ class ScalarAggregateNode : public ExecNode, public TracedNode {
     DCHECK_EQ(input, inputs_[0]);
 
     auto thread_index = plan_->query_context()->GetThreadIndex();
-
-    ARROW_RETURN_NOT_OK(DoConsume(ExecSpan(batch), thread_index));
+    auto handler = [this, thread_index](const ExecBatch& full_batch,

Review Comment:
   Why do we pass the handler here instead of just implementing the logic inside `HandleSegments`. I found the indirection here a bit confusing/unnecessary. Is that a specific reason to do it this way?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rtpsw commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "rtpsw (via GitHub)" <gi...@apache.org>.
rtpsw commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1122292888


##########
cpp/src/arrow/compute/row/grouper.h:
##########
@@ -30,6 +30,69 @@
 namespace arrow {
 namespace compute {
 
+/// \brief A segment of contiguous rows for grouping
+struct ARROW_EXPORT GroupingSegment {
+  /// \brief the offset into the batch where the segment starts
+  int64_t offset;
+  /// \brief the length of the segment
+  int64_t length;
+  /// \brief whether the segment may be extended by a next one
+  bool is_open;
+  /// \brief whether the segment extends a preceeding one
+  bool extends;
+};
+
+inline bool operator==(const GroupingSegment& segment1, const GroupingSegment& segment2) {
+  return segment1.offset == segment2.offset && segment1.length == segment2.length &&
+         segment1.is_open == segment2.is_open && segment1.extends == segment2.extends;
+}
+inline bool operator!=(const GroupingSegment& segment1, const GroupingSegment& segment2) {
+  return !(segment1 == segment2);
+}

Review Comment:
   `TestSegments` in `hash_aggregate_test.cc` uses the above `operator==`. I don't think `operator~=` is used right now.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rtpsw commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "rtpsw (via GitHub)" <gi...@apache.org>.
rtpsw commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1122314126


##########
cpp/src/arrow/compute/exec/aggregate_node.cc:
##########
@@ -326,46 +446,86 @@ class ScalarAggregateNode : public ExecNode, public TracedNode {
   }
 
  private:
-  Status Finish() {
-    auto scope = TraceFinish();
+  Status ReconstructAggregates() {
+    const auto& input_schema = *inputs()[0]->output_schema();
+    auto exec_ctx = plan()->query_context()->exec_context();
+    for (size_t i = 0; i < kernels_.size(); ++i) {
+      std::vector<TypeHolder> in_types;
+      for (const auto& target : target_fieldsets_[i]) {
+        in_types.emplace_back(input_schema.field(target)->type().get());
+      }
+      states_[i].resize(plan()->query_context()->max_concurrency());

Review Comment:
   They state instances do need to be reset. They accumulate the aggregation per group. When the segment ends, the aggregation output is generated and the states are reset, ready for the next segment.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rtpsw commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "rtpsw (via GitHub)" <gi...@apache.org>.
rtpsw commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1124206218


##########
cpp/src/arrow/compute/exec/aggregate_node.cc:
##########
@@ -169,35 +185,117 @@ void AggregatesToString(std::stringstream* ss, const Schema& input_schema,
   *ss << ']';
 }
 
+template <typename BatchHandler>
+Status HandleSegments(std::unique_ptr<GroupingSegmenter>& segmenter,
+                      const ExecBatch& batch, const std::vector<int>& ids,
+                      const BatchHandler& handle_batch) {
+  int64_t offset = 0;
+  ARROW_ASSIGN_OR_RAISE(auto segment_exec_batch, batch.SelectValues(ids));
+  ExecSpan segment_batch(segment_exec_batch);
+  while (true) {
+    ARROW_ASSIGN_OR_RAISE(auto segment, segmenter->GetNextSegment(segment_batch, offset));
+    if (segment.offset >= segment_batch.length) break;  // condition of no-next-segment
+    ARROW_RETURN_NOT_OK(handle_batch(batch, segment));
+    offset = segment.offset + segment.length;
+  }
+  return Status::OK();
+}
+
+Status GetScalarFields(std::vector<Datum>* values_ptr, const ExecBatch& input_batch,
+                       const std::vector<int>& field_ids) {
+  DCHECK_GT(input_batch.length, 0);
+  std::vector<Datum>& values = *values_ptr;
+  int64_t row = input_batch.length - 1;
+  values.clear();
+  values.resize(field_ids.size());
+  for (size_t i = 0; i < field_ids.size(); i++) {
+    const Datum& value = input_batch.values[field_ids[i]];
+    if (value.is_scalar()) {

Review Comment:
   This value is a column of the input batch that may be a scalar - the batch is set by the previous node in the plan. When the value is a scalar, the column is constant.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] icexelloss commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "icexelloss (via GitHub)" <gi...@apache.org>.
icexelloss commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1126602059


##########
cpp/src/arrow/compute/kernels/hash_aggregate_test.cc:
##########
@@ -301,53 +426,143 @@ Result<Datum> GroupByTest(const std::vector<Datum>& arguments,
         {t_agg.function, t_agg.options, "agg_" + ToChars(idx), t_agg.function});
     idx = idx + 1;
   }
-  return RunGroupBy(arguments, keys, internal_aggregates, use_threads);
+  return group_by(arguments, keys, segment_keys, internal_aggregates, use_threads,
+                  /*naive=*/false);
 }
 
-}  // namespace
+Result<Datum> GroupByTest(GroupByFunction group_by, const std::vector<Datum>& arguments,
+                          const std::vector<Datum>& keys,
+                          const std::vector<TestAggregate>& aggregates,
+                          bool use_threads) {
+  return GroupByTest(group_by, arguments, keys, {}, aggregates, use_threads);
+}
 
-TEST(Grouper, SupportedKeys) {
-  ASSERT_OK(Grouper::Make({boolean()}));
+template <typename GroupClass>
+void TestGroupClassSupportedKeys() {
+  ASSERT_OK(GroupClass::Make({boolean()}));
 
-  ASSERT_OK(Grouper::Make({int8(), uint16(), int32(), uint64()}));
+  ASSERT_OK(GroupClass::Make({int8(), uint16(), int32(), uint64()}));
 
-  ASSERT_OK(Grouper::Make({dictionary(int64(), utf8())}));
+  ASSERT_OK(GroupClass::Make({dictionary(int64(), utf8())}));
 
-  ASSERT_OK(Grouper::Make({float16(), float32(), float64()}));
+  ASSERT_OK(GroupClass::Make({float16(), float32(), float64()}));
 
-  ASSERT_OK(Grouper::Make({utf8(), binary(), large_utf8(), large_binary()}));
+  ASSERT_OK(GroupClass::Make({utf8(), binary(), large_utf8(), large_binary()}));
 
-  ASSERT_OK(Grouper::Make({fixed_size_binary(16), fixed_size_binary(32)}));
+  ASSERT_OK(GroupClass::Make({fixed_size_binary(16), fixed_size_binary(32)}));
 
-  ASSERT_OK(Grouper::Make({decimal128(32, 10), decimal256(76, 20)}));
+  ASSERT_OK(GroupClass::Make({decimal128(32, 10), decimal256(76, 20)}));
 
-  ASSERT_OK(Grouper::Make({date32(), date64()}));
+  ASSERT_OK(GroupClass::Make({date32(), date64()}));
 
   for (auto unit : {
            TimeUnit::SECOND,
            TimeUnit::MILLI,
            TimeUnit::MICRO,
            TimeUnit::NANO,
        }) {
-    ASSERT_OK(Grouper::Make({timestamp(unit), duration(unit)}));
+    ASSERT_OK(GroupClass::Make({timestamp(unit), duration(unit)}));
   }
 
-  ASSERT_OK(
-      Grouper::Make({day_time_interval(), month_interval(), month_day_nano_interval()}));
+  ASSERT_OK(GroupClass::Make(
+      {day_time_interval(), month_interval(), month_day_nano_interval()}));
+
+  ASSERT_OK(GroupClass::Make({null()}));
+
+  ASSERT_RAISES(NotImplemented, GroupClass::Make({struct_({field("", int64())})}));
+
+  ASSERT_RAISES(NotImplemented, GroupClass::Make({struct_({})}));
 
-  ASSERT_OK(Grouper::Make({null()}));
+  ASSERT_RAISES(NotImplemented, GroupClass::Make({list(int32())}));
 
-  ASSERT_RAISES(NotImplemented, Grouper::Make({struct_({field("", int64())})}));
+  ASSERT_RAISES(NotImplemented, GroupClass::Make({fixed_size_list(int32(), 5)}));
 
-  ASSERT_RAISES(NotImplemented, Grouper::Make({struct_({})}));
+  ASSERT_RAISES(NotImplemented, GroupClass::Make({dense_union({field("", int32())})}));
+}
 
-  ASSERT_RAISES(NotImplemented, Grouper::Make({list(int32())}));
+void TestSegments(std::unique_ptr<RowSegmenter>& segmenter, const ExecSpan& batch,

Review Comment:
   Ok seems fine - but to be clear - we don't use the grouper class for "ordered aggregation" functionality?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] icexelloss commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "icexelloss (via GitHub)" <gi...@apache.org>.
icexelloss commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1126604172


##########
cpp/src/arrow/compute/kernels/hash_aggregate_test.cc:
##########
@@ -301,53 +426,143 @@ Result<Datum> GroupByTest(const std::vector<Datum>& arguments,
         {t_agg.function, t_agg.options, "agg_" + ToChars(idx), t_agg.function});
     idx = idx + 1;
   }
-  return RunGroupBy(arguments, keys, internal_aggregates, use_threads);
+  return group_by(arguments, keys, segment_keys, internal_aggregates, use_threads,
+                  /*naive=*/false);
 }
 
-}  // namespace
+Result<Datum> GroupByTest(GroupByFunction group_by, const std::vector<Datum>& arguments,
+                          const std::vector<Datum>& keys,
+                          const std::vector<TestAggregate>& aggregates,
+                          bool use_threads) {
+  return GroupByTest(group_by, arguments, keys, {}, aggregates, use_threads);
+}
 
-TEST(Grouper, SupportedKeys) {
-  ASSERT_OK(Grouper::Make({boolean()}));
+template <typename GroupClass>
+void TestGroupClassSupportedKeys() {
+  ASSERT_OK(GroupClass::Make({boolean()}));
 
-  ASSERT_OK(Grouper::Make({int8(), uint16(), int32(), uint64()}));
+  ASSERT_OK(GroupClass::Make({int8(), uint16(), int32(), uint64()}));
 
-  ASSERT_OK(Grouper::Make({dictionary(int64(), utf8())}));
+  ASSERT_OK(GroupClass::Make({dictionary(int64(), utf8())}));
 
-  ASSERT_OK(Grouper::Make({float16(), float32(), float64()}));
+  ASSERT_OK(GroupClass::Make({float16(), float32(), float64()}));
 
-  ASSERT_OK(Grouper::Make({utf8(), binary(), large_utf8(), large_binary()}));
+  ASSERT_OK(GroupClass::Make({utf8(), binary(), large_utf8(), large_binary()}));
 
-  ASSERT_OK(Grouper::Make({fixed_size_binary(16), fixed_size_binary(32)}));
+  ASSERT_OK(GroupClass::Make({fixed_size_binary(16), fixed_size_binary(32)}));
 
-  ASSERT_OK(Grouper::Make({decimal128(32, 10), decimal256(76, 20)}));
+  ASSERT_OK(GroupClass::Make({decimal128(32, 10), decimal256(76, 20)}));
 
-  ASSERT_OK(Grouper::Make({date32(), date64()}));
+  ASSERT_OK(GroupClass::Make({date32(), date64()}));
 
   for (auto unit : {
            TimeUnit::SECOND,
            TimeUnit::MILLI,
            TimeUnit::MICRO,
            TimeUnit::NANO,
        }) {
-    ASSERT_OK(Grouper::Make({timestamp(unit), duration(unit)}));
+    ASSERT_OK(GroupClass::Make({timestamp(unit), duration(unit)}));
   }
 
-  ASSERT_OK(
-      Grouper::Make({day_time_interval(), month_interval(), month_day_nano_interval()}));
+  ASSERT_OK(GroupClass::Make(
+      {day_time_interval(), month_interval(), month_day_nano_interval()}));
+
+  ASSERT_OK(GroupClass::Make({null()}));
+
+  ASSERT_RAISES(NotImplemented, GroupClass::Make({struct_({field("", int64())})}));
+
+  ASSERT_RAISES(NotImplemented, GroupClass::Make({struct_({})}));
 
-  ASSERT_OK(Grouper::Make({null()}));
+  ASSERT_RAISES(NotImplemented, GroupClass::Make({list(int32())}));
 
-  ASSERT_RAISES(NotImplemented, Grouper::Make({struct_({field("", int64())})}));
+  ASSERT_RAISES(NotImplemented, GroupClass::Make({fixed_size_list(int32(), 5)}));
 
-  ASSERT_RAISES(NotImplemented, Grouper::Make({struct_({})}));
+  ASSERT_RAISES(NotImplemented, GroupClass::Make({dense_union({field("", int32())})}));
+}
 
-  ASSERT_RAISES(NotImplemented, Grouper::Make({list(int32())}));
+void TestSegments(std::unique_ptr<RowSegmenter>& segmenter, const ExecSpan& batch,

Review Comment:
   I see - this is fine.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rtpsw commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "rtpsw (via GitHub)" <gi...@apache.org>.
rtpsw commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1126768877


##########
cpp/src/arrow/compute/row/grouper.h:
##########
@@ -30,6 +30,49 @@
 namespace arrow {
 namespace compute {
 
+/// \brief A segment of contiguous rows for grouping
+struct ARROW_EXPORT GroupingSegment {
+  int64_t offset;
+  int64_t length;
+  bool is_open;
+  bool extends;
+};
+
+inline bool operator==(const GroupingSegment& segment1, const GroupingSegment& segment2) {
+  return segment1.offset == segment2.offset && segment1.length == segment2.length &&
+         segment1.is_open == segment2.is_open && segment1.extends == segment2.extends;
+}
+inline bool operator!=(const GroupingSegment& segment1, const GroupingSegment& segment2) {
+  return !(segment1 == segment2);
+}
+
+/// \brief Computes grouping segments for a batch. Each segment covers rows with identical
+/// values in the batch. The values in the batch are often selected as keys from a larger
+/// batch.
+class ARROW_EXPORT GroupingSegmenter {
+ public:
+  virtual ~GroupingSegmenter() = default;
+
+  /// \brief Construct a GroupingSegmenter which receives the specified key types
+  static Result<std::unique_ptr<GroupingSegmenter>> Make(
+      const std::vector<TypeHolder>& key_types, bool nullable_keys = false,
+      ExecContext* ctx = default_exec_context());

Review Comment:
   I designed the default values to get a common interface before I had the specific template invocation in mind. If you feel strongly about removing the default values, one alternative is add functions in the tester file that would wrap the maker function with this common interface.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rtpsw commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "rtpsw (via GitHub)" <gi...@apache.org>.
rtpsw commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1126996953


##########
cpp/src/arrow/compute/row/grouper.cc:
##########
@@ -39,12 +43,330 @@
 namespace arrow {
 
 using internal::checked_cast;
+using internal::PrimitiveScalarBase;
 
 namespace compute {
 
 namespace {
 
-struct GrouperImpl : Grouper {
+constexpr uint32_t kNoGroupId = std::numeric_limits<uint32_t>::max();
+
+using group_id_t = std::remove_const<decltype(kNoGroupId)>::type;
+using GroupIdType = CTypeTraits<group_id_t>::ArrowType;
+auto g_group_id_type = std::make_shared<GroupIdType>();
+
+inline const uint8_t* GetValuesAsBytes(const ArraySpan& data, int64_t offset = 0) {
+  DCHECK_GT(data.type->byte_width(), 0);
+  int64_t absolute_byte_offset = (data.offset + offset) * data.type->byte_width();
+  return data.GetValues<uint8_t>(1, absolute_byte_offset);
+}
+
+template <typename Value>
+Status CheckForGetNextSegment(const std::vector<Value>& values, int64_t length,
+                              int64_t offset, const std::vector<TypeHolder>& key_types) {
+  if (offset < 0 || offset > length) {
+    return Status::Invalid("invalid grouping segmenter offset: ", offset);
+  }
+  if (values.size() != key_types.size()) {
+    return Status::Invalid("expected batch size ", key_types.size(), " but got ",
+                           values.size());
+  }
+  for (size_t i = 0; i < key_types.size(); i++) {
+    const auto& value = values[i];
+    const auto& key_type = key_types[i];
+    if (*value.type() != *key_type.type) {
+      return Status::Invalid("expected batch value ", i, " of type ", *key_type.type,
+                             " but got ", *value.type());
+    }
+  }
+  return Status::OK();
+}
+
+template <typename Batch>
+enable_if_t<std::is_same<Batch, ExecSpan>::value || std::is_same<Batch, ExecBatch>::value,
+            Status>
+CheckForGetNextSegment(const Batch& batch, int64_t offset,
+                       const std::vector<TypeHolder>& key_types) {
+  return CheckForGetNextSegment(batch.values, batch.length, offset, key_types);
+}
+
+struct BaseRowSegmenter : public RowSegmenter {
+  explicit BaseRowSegmenter(const std::vector<TypeHolder>& key_types)
+      : key_types_(key_types) {}
+
+  const std::vector<TypeHolder>& key_types() const override { return key_types_; }
+
+  std::vector<TypeHolder> key_types_;
+};
+
+Segment MakeSegment(int64_t batch_length, int64_t offset, int64_t length, bool extends) {
+  return Segment{offset, length, offset + length >= batch_length, extends};
+}
+
+int64_t GetMatchLength(const uint8_t* match_bytes, int64_t match_width,
+                       const uint8_t* array_bytes, int64_t offset, int64_t length) {
+  int64_t cursor, byte_cursor;
+  for (cursor = offset, byte_cursor = match_width * cursor; cursor < length;
+       cursor++, byte_cursor += match_width) {
+    if (memcmp(match_bytes, array_bytes + byte_cursor,
+               static_cast<size_t>(match_width)) != 0) {
+      break;
+    }
+  }
+  return std::min(cursor, length) - offset;
+}
+
+using ExtendFunc = std::function<bool(const void*)>;
+constexpr bool kDefaultExtends = true;
+constexpr bool kEmptyExtends = true;
+
+struct NoKeysSegmenter : public BaseRowSegmenter {
+  static std::unique_ptr<RowSegmenter> Make() {
+    return std::make_unique<NoKeysSegmenter>();
+  }
+
+  NoKeysSegmenter() : BaseRowSegmenter({}) {}
+
+  Status Reset() override { return Status::OK(); }
+
+  Result<Segment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {}));
+    return MakeSegment(batch.length, offset, batch.length - offset, kDefaultExtends);
+  }
+};
+
+struct SimpleKeySegmenter : public BaseRowSegmenter {
+  static Result<std::unique_ptr<RowSegmenter>> Make(TypeHolder key_type) {
+    return std::make_unique<SimpleKeySegmenter>(key_type);
+  }
+
+  explicit SimpleKeySegmenter(TypeHolder key_type)
+      : BaseRowSegmenter({key_type}), key_type_(key_types_[0]), save_key_data_() {}
+
+  Status CheckType(const DataType& type) {
+    if (!is_fixed_width(type)) {
+      return Status::Invalid("SimpleKeySegmenter does not support type ", type);
+    }
+    return Status::OK();
+  }
+
+  Status Reset() override {
+    save_key_data_.resize(0);
+    return Status::OK();
+  }
+
+  // Checks whether the given grouping data extends the current segment, i.e., is equal to
+  // previously seen grouping data, which is updated with each invocation.
+  bool Extend(const void* data) {
+    size_t byte_width = static_cast<size_t>(key_type_.type->byte_width());
+    bool extends = save_key_data_.size() != byte_width
+                       ? kDefaultExtends
+                       : 0 == memcmp(save_key_data_.data(), data, byte_width);
+    save_key_data_.resize(byte_width);
+    memcpy(save_key_data_.data(), data, byte_width);
+    return extends;
+  }
+
+  Result<Segment> GetNextSegment(const Scalar& scalar, int64_t offset, int64_t length) {
+    ARROW_RETURN_NOT_OK(CheckType(*scalar.type));
+    if (!scalar.is_valid) {
+      return Status::Invalid("segmenting an invalid scalar");
+    }
+    auto data = checked_cast<const PrimitiveScalarBase&>(scalar).data();
+    bool extends = length > 0 ? Extend(data) : kEmptyExtends;
+    return MakeSegment(length, offset, length, extends);
+  }
+
+  Result<Segment> GetNextSegment(const DataType& array_type, const uint8_t* array_bytes,
+                                 int64_t offset, int64_t length) {
+    RETURN_NOT_OK(CheckType(array_type));
+    int64_t byte_width = array_type.byte_width();
+    int64_t match_length = GetMatchLength(array_bytes + offset * byte_width, byte_width,
+                                          array_bytes, offset, length);
+    bool extends = length > 0 ? Extend(array_bytes + offset * byte_width) : kEmptyExtends;
+    return MakeSegment(length, offset, match_length, extends);
+  }
+
+  Result<Segment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {key_type_}));
+    if (offset == batch.length) {
+      return MakeSegment(batch.length, offset, 0, kEmptyExtends);
+    }
+    const auto& value = batch.values[0];
+    if (value.is_scalar()) {
+      return GetNextSegment(*value.scalar, offset, batch.length);
+    }
+    ARROW_DCHECK(value.is_array());
+    const auto& array = value.array;
+    if (array.GetNullCount() > 0) {
+      return Status::NotImplemented("segmenting a nullable array");
+    }
+    return GetNextSegment(*array.type, GetValuesAsBytes(array), offset, batch.length);
+  }
+
+ private:
+  TypeHolder key_type_;
+  std::vector<uint8_t> save_key_data_;
+};
+
+struct AnyKeysSegmenter : public BaseRowSegmenter {
+  static Result<std::unique_ptr<RowSegmenter>> Make(
+      const std::vector<TypeHolder>& key_types, ExecContext* ctx) {
+    ARROW_RETURN_NOT_OK(Grouper::Make(key_types, ctx));  // check types
+    return std::make_unique<AnyKeysSegmenter>(key_types, ctx);
+  }
+
+  AnyKeysSegmenter(const std::vector<TypeHolder>& key_types, ExecContext* ctx)
+      : BaseRowSegmenter(key_types),
+        ctx_(ctx),
+        grouper_(nullptr),
+        save_group_id_(kNoGroupId) {}
+
+  Status Reset() override {
+    grouper_ = nullptr;
+    save_group_id_ = kNoGroupId;
+    return Status::OK();
+  }
+
+  bool Extend(const void* data) {
+    auto group_id = *static_cast<const group_id_t*>(data);
+    bool extends =
+        save_group_id_ == kNoGroupId ? kDefaultExtends : save_group_id_ == group_id;
+    save_group_id_ = group_id;
+    return extends;
+  }
+
+  // Runs the grouper on a single row.  This is used to determine the group id of the
+  // first row of a new segment to see if it extends the previous segment.
+  template <typename Batch>
+  Result<group_id_t> MapGroupIdAt(const Batch& batch, int64_t offset) {
+    if (!grouper_) return kNoGroupId;
+    ARROW_ASSIGN_OR_RAISE(auto datum, grouper_->Consume(batch, offset,
+                                                        /*length=*/1));
+    if (!datum.is_array()) {
+      return Status::Invalid("accessing unsupported datum kind ", datum.kind());
+    }
+    const std::shared_ptr<ArrayData>& data = datum.array();
+    ARROW_DCHECK(data->GetNullCount() == 0);
+    DCHECK_EQ(data->type->id(), GroupIdType::type_id);
+    DCHECK_EQ(1, data->length);
+    const group_id_t* values = data->GetValues<group_id_t>(1);
+    return values[0];
+  }
+
+  Result<Segment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, key_types_));
+    if (offset == batch.length) {
+      return MakeSegment(batch.length, offset, 0, kEmptyExtends);
+    }
+    // ARROW-18311: make Grouper support Reset()
+    // so it can be cached instead of recreated below
+    //
+    // the group id must be computed prior to resetting the grouper, since it is compared
+    // to save_group_id_, and after resetting the grouper produces incomparable group ids
+    ARROW_ASSIGN_OR_RAISE(auto group_id, MapGroupIdAt(batch, offset));
+    ExtendFunc bound_extend = [this, group_id](const void* data) {
+      bool extends = Extend(&group_id);
+      save_group_id_ = *static_cast<const group_id_t*>(data);
+      return extends;
+    };
+    ARROW_ASSIGN_OR_RAISE(grouper_, Grouper::Make(key_types_, ctx_));  // TODO: reset it
+    ARROW_ASSIGN_OR_RAISE(auto datum, grouper_->Consume(batch, offset));
+    if (datum.is_array()) {
+      const std::shared_ptr<ArrayData>& data = datum.array();
+      ARROW_DCHECK(data->GetNullCount() == 0);
+      DCHECK_EQ(data->type->id(), GroupIdType::type_id);
+      const group_id_t* values = data->GetValues<group_id_t>(1);
+      int64_t cursor;
+      for (cursor = 1; cursor < data->length; cursor++) {
+        if (values[0] != values[cursor]) break;
+      }
+      int64_t length = std::min(cursor, batch.length - offset);

Review Comment:
   Turns out the change causes a hang. I'll take a closer look later.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] icexelloss commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "icexelloss (via GitHub)" <gi...@apache.org>.
icexelloss commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1127045355


##########
cpp/src/arrow/compute/kernels/hash_aggregate_test.cc:
##########
@@ -1917,57 +2243,7 @@ TEST(GroupBy, MinMaxBinary) {
   }
 }
 
-TEST(GroupBy, MinMaxFixedSizeBinary) {

Review Comment:
   The original test is called "MinMaxBinary" and it seems you renamed it to "MinMaxFixedSizeBinary" and deleted the original "MinMaxFixedSizeBinary" test



##########
cpp/src/arrow/compute/kernels/hash_aggregate_test.cc:
##########
@@ -1917,57 +2243,7 @@ TEST(GroupBy, MinMaxBinary) {
   }
 }
 
-TEST(GroupBy, MinMaxFixedSizeBinary) {

Review Comment:
   The original test above is called "MinMaxBinary" and it seems you renamed it to "MinMaxFixedSizeBinary" and deleted the original "MinMaxFixedSizeBinary" test



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rtpsw commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "rtpsw (via GitHub)" <gi...@apache.org>.
rtpsw commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1125563709


##########
cpp/src/arrow/compute/exec/aggregate_node.cc:
##########
@@ -326,46 +446,86 @@ class ScalarAggregateNode : public ExecNode, public TracedNode {
   }
 
  private:
-  Status Finish() {
-    auto scope = TraceFinish();
+  Status ReconstructAggregates() {
+    const auto& input_schema = *inputs()[0]->output_schema();
+    auto exec_ctx = plan()->query_context()->exec_context();
+    for (size_t i = 0; i < kernels_.size(); ++i) {
+      std::vector<TypeHolder> in_types;
+      for (const auto& target : target_fieldsets_[i]) {
+        in_types.emplace_back(input_schema.field(target)->type().get());
+      }
+      states_[i].resize(plan()->query_context()->max_concurrency());
+      KernelContext kernel_ctx{exec_ctx};
+      RETURN_NOT_OK(Kernel::InitAll(
+          &kernel_ctx, KernelInitArgs{kernels_[i], in_types, aggs_[i].options.get()},
+          &states_[i]));
+    }
+    return Status::OK();
+  }
+
+  Status OutputResult(bool is_last = false, bool traced = false) {
+    if (is_last && !traced) {
+      auto scope = TraceFinish();
+      return OutputResult(is_last, /*traced=*/true);
+    }
+    GatedUniqueLock lock(gated_shared_mutex_);
     ExecBatch batch{{}, 1};
-    batch.values.resize(kernels_.size());
+    batch.values.resize(kernels_.size() + segment_field_ids_.size());
 
     for (size_t i = 0; i < kernels_.size(); ++i) {
       util::tracing::Span span;
       START_COMPUTE_SPAN(span, aggs_[i].function,
                          {{"function.name", aggs_[i].function},
                           {"function.options",
                            aggs_[i].options ? aggs_[i].options->ToString() : "<NULLPTR>"},
-                          {"function.kind", std::string(kind_name()) + "::Finalize"}});
+                          {"function.kind", std::string(kind_name()) + "::Output"}});
       KernelContext ctx{plan()->query_context()->exec_context()};
       ARROW_ASSIGN_OR_RAISE(auto merged, ScalarAggregateKernel::MergeAll(
                                              kernels_[i], &ctx, std::move(states_[i])));
       RETURN_NOT_OK(kernels_[i]->finalize(&ctx, &batch.values[i]));
     }
+    PlaceFields(batch, kernels_.size(), segmenter_values_);
 
-    return output_->InputReceived(this, std::move(batch));
+    ARROW_RETURN_NOT_OK(output_->InputReceived(this, std::move(batch)));
+    total_output_batches_++;

Review Comment:
   Done.



##########
cpp/src/arrow/compute/exec/aggregate_node.cc:
##########
@@ -326,46 +446,86 @@ class ScalarAggregateNode : public ExecNode, public TracedNode {
   }
 
  private:
-  Status Finish() {
-    auto scope = TraceFinish();
+  Status ReconstructAggregates() {
+    const auto& input_schema = *inputs()[0]->output_schema();
+    auto exec_ctx = plan()->query_context()->exec_context();
+    for (size_t i = 0; i < kernels_.size(); ++i) {
+      std::vector<TypeHolder> in_types;
+      for (const auto& target : target_fieldsets_[i]) {
+        in_types.emplace_back(input_schema.field(target)->type().get());
+      }
+      states_[i].resize(plan()->query_context()->max_concurrency());

Review Comment:
   Fixed.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] icexelloss commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "icexelloss (via GitHub)" <gi...@apache.org>.
icexelloss commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1125574615


##########
cpp/src/arrow/compute/exec/aggregate_node.cc:
##########
@@ -584,29 +775,50 @@ class GroupByNode : public ExecNode, public TracedNode {
     ARROW_ASSIGN_OR_RAISE(out_data_, Finalize());
 
     int64_t num_output_batches = bit_util::CeilDiv(out_data_.length, output_batch_size());
-    RETURN_NOT_OK(output_->InputFinished(this, static_cast<int>(num_output_batches)));
-    return plan_->query_context()->StartTaskGroup(output_task_group_id_,
-                                                  num_output_batches);
+    total_output_batches_ += static_cast<int>(num_output_batches);
+    if (is_last) {
+      ARROW_RETURN_NOT_OK(output_->InputFinished(this, total_output_batches_));
+      RETURN_NOT_OK(plan_->query_context()->StartTaskGroup(output_task_group_id_,
+                                                           num_output_batches));
+    } else {
+      for (int64_t i = 0; i < num_output_batches; i++) {
+        ARROW_RETURN_NOT_OK(OutputNthBatch(i));
+      }
+      ARROW_RETURN_NOT_OK(ResetAggregates());
+    }
+    return Status::OK();
   }
 
   Status InputReceived(ExecNode* input, ExecBatch batch) override {
     auto scope = TraceInputReceived(batch);
 
     DCHECK_EQ(input, inputs_[0]);
 
-    ARROW_RETURN_NOT_OK(Consume(ExecSpan(batch)));
+    auto handler = [this](const ExecBatch& full_batch, const GroupingSegment& segment) {
+      if (!segment.extends && segment.offset == 0) RETURN_NOT_OK(OutputResult());
+      auto exec_batch = full_batch.Slice(segment.offset, segment.length);
+      auto batch = ExecSpan(exec_batch);
+      RETURN_NOT_OK(Consume(batch));
+      RETURN_NOT_OK(
+          GetScalarFields(&segmenter_values_, exec_batch, segment_key_field_ids_));

Review Comment:
   Feels to me this is very specific to segment aggregation



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] icexelloss commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "icexelloss (via GitHub)" <gi...@apache.org>.
icexelloss commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1125574448


##########
cpp/src/arrow/compute/exec/aggregate_node.cc:
##########
@@ -326,10 +437,22 @@ class ScalarAggregateNode : public ExecNode, public TracedNode {
   }
 
  private:
-  Status Finish() {
-    auto scope = TraceFinish();
+  Status ResetAggregates() {
+    auto exec_ctx = plan()->query_context()->exec_context();
+    for (size_t i = 0; i < kernels_.size(); ++i) {
+      const std::vector<TypeHolder>& in_types = in_typesets_[i];
+      states_[i].resize(plan()->query_context()->max_concurrency());
+      KernelContext kernel_ctx{exec_ctx};
+      RETURN_NOT_OK(Kernel::InitAll(
+          &kernel_ctx, KernelInitArgs{kernels_[i], in_types, aggs_[i].options.get()},
+          &states_[i]));
+    }
+    return Status::OK();
+  }
+
+  Status OutputResult(bool is_last = false) {
     ExecBatch batch{{}, 1};
-    batch.values.resize(kernels_.size());
+    batch.values.resize(kernels_.size() + segment_field_ids_.size());

Review Comment:
   I am confused here. How is the size of "kernels_" related to how many key columns we have here? 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rtpsw commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "rtpsw (via GitHub)" <gi...@apache.org>.
rtpsw commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1125648242


##########
cpp/src/arrow/compute/exec/aggregate_node.cc:
##########
@@ -584,29 +775,50 @@ class GroupByNode : public ExecNode, public TracedNode {
     ARROW_ASSIGN_OR_RAISE(out_data_, Finalize());
 
     int64_t num_output_batches = bit_util::CeilDiv(out_data_.length, output_batch_size());
-    RETURN_NOT_OK(output_->InputFinished(this, static_cast<int>(num_output_batches)));
-    return plan_->query_context()->StartTaskGroup(output_task_group_id_,
-                                                  num_output_batches);
+    total_output_batches_ += static_cast<int>(num_output_batches);
+    if (is_last) {
+      ARROW_RETURN_NOT_OK(output_->InputFinished(this, total_output_batches_));
+      RETURN_NOT_OK(plan_->query_context()->StartTaskGroup(output_task_group_id_,
+                                                           num_output_batches));
+    } else {
+      for (int64_t i = 0; i < num_output_batches; i++) {
+        ARROW_RETURN_NOT_OK(OutputNthBatch(i));
+      }
+      ARROW_RETURN_NOT_OK(ResetAggregates());
+    }
+    return Status::OK();
   }
 
   Status InputReceived(ExecNode* input, ExecBatch batch) override {
     auto scope = TraceInputReceived(batch);
 
     DCHECK_EQ(input, inputs_[0]);
 
-    ARROW_RETURN_NOT_OK(Consume(ExecSpan(batch)));
+    auto handler = [this](const ExecBatch& full_batch, const GroupingSegment& segment) {
+      if (!segment.extends && segment.offset == 0) RETURN_NOT_OK(OutputResult());
+      auto exec_batch = full_batch.Slice(segment.offset, segment.length);
+      auto batch = ExecSpan(exec_batch);
+      RETURN_NOT_OK(Consume(batch));
+      RETURN_NOT_OK(
+          GetScalarFields(&segmenter_values_, exec_batch, segment_key_field_ids_));

Review Comment:
   Whether it is specific to segmented aggregation is a matter of view; the implementation isn't but the use is. Regardless of view, I think this is a minor point.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rtpsw commented on pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "rtpsw (via GitHub)" <gi...@apache.org>.
rtpsw commented on PR #34311:
URL: https://github.com/apache/arrow/pull/34311#issuecomment-1442055371

   > @westonpace, in conflict resolution, I see `cpp/src/arrow/compute/exec/aggregate.{h,cc}` have been deleted. What was the story around that? I'm trying to figure out how best to resolve.
   
   I found the commit you made which removed these files and I generally figured it out. I pushed a resolution of the conflicts.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] github-actions[bot] commented on pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "github-actions[bot] (via GitHub)" <gi...@apache.org>.
github-actions[bot] commented on PR #34311:
URL: https://github.com/apache/arrow/pull/34311#issuecomment-1441511453

   * Closes: #32884


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rtpsw commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "rtpsw (via GitHub)" <gi...@apache.org>.
rtpsw commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1130039282


##########
cpp/src/arrow/compute/kernels/hash_aggregate_test.cc:
##########
@@ -174,81 +242,117 @@ Result<Datum> RunGroupBy(const BatchesWithSchema& input,
   ARROW_ASSIGN_OR_RAISE(std::vector<ExecBatch> output_batches,
                         start_and_collect.MoveResult());
 
-  ArrayVector out_arrays(aggregates.size() + key_names.size());
   const auto& output_schema = plan->nodes()[0]->output()->output_schema();
+  if (!segmented) {
+    return MakeGroupByOutput(output_batches, output_schema, aggregates.size(),
+                             key_names.size(), naive);
+  }
+
+  std::vector<ArrayVector> out_arrays(aggregates.size() + key_names.size() +
+                                      segment_key_names.size());
   for (size_t i = 0; i < out_arrays.size(); ++i) {
     std::vector<std::shared_ptr<Array>> arrays(output_batches.size());
     for (size_t j = 0; j < output_batches.size(); ++j) {
-      arrays[j] = output_batches[j].values[i].make_array();
+      auto& value = output_batches[j].values[i];
+      if (value.is_scalar()) {
+        ARROW_ASSIGN_OR_RAISE(
+            arrays[j], MakeArrayFromScalar(*value.scalar(), output_batches[j].length));
+      } else if (value.is_array()) {
+        arrays[j] = value.make_array();
+      } else {
+        return Status::Invalid("GroupByUsingExecPlan unsupported value kind ",
+                               ToString(value.kind()));
+      }
     }
     if (arrays.empty()) {
+      arrays.resize(1);
       ARROW_ASSIGN_OR_RAISE(
-          out_arrays[i],
-          MakeArrayOfNull(output_schema->field(static_cast<int>(i))->type(),
-                          /*length=*/0));
-    } else {
-      ARROW_ASSIGN_OR_RAISE(out_arrays[i], Concatenate(arrays));
+          arrays[0], MakeArrayOfNull(output_schema->field(static_cast<int>(i))->type(),
+                                     /*length=*/0));
     }
+    out_arrays[i] = {std::move(arrays)};
   }
 
-  // The exec plan may reorder the output rows.  The tests are all setup to expect ouptut
-  // in ascending order of keys.  So we need to sort the result by the key columns.  To do
-  // that we create a table using the key columns, calculate the sort indices from that
-  // table (sorting on all fields) and then use those indices to calculate our result.
-  std::vector<std::shared_ptr<Field>> key_fields;
-  std::vector<std::shared_ptr<Array>> key_columns;
-  std::vector<SortKey> sort_keys;
-  for (std::size_t i = 0; i < key_names.size(); i++) {
-    const std::shared_ptr<Array>& arr = out_arrays[i + aggregates.size()];
-    if (arr->type_id() == Type::DICTIONARY) {
-      // Can't sort dictionary columns so need to decode
-      auto dict_arr = checked_pointer_cast<DictionaryArray>(arr);
-      ARROW_ASSIGN_OR_RAISE(auto decoded_arr,
-                            Take(*dict_arr->dictionary(), *dict_arr->indices()));
-      key_columns.push_back(decoded_arr);
-      key_fields.push_back(
-          field("name_does_not_matter", dict_arr->dict_type()->value_type()));
-    } else {
-      key_columns.push_back(arr);
-      key_fields.push_back(field("name_does_not_matter", arr->type()));
+  if (segmented && segment_key_names.size() > 0) {
+    ArrayVector struct_arrays;
+    struct_arrays.reserve(output_batches.size());
+    for (size_t j = 0; j < output_batches.size(); ++j) {
+      ArrayVector struct_fields;
+      struct_fields.reserve(out_arrays.size());
+      for (auto out_array : out_arrays) {
+        struct_fields.push_back(out_array[j]);
+      }
+      ARROW_ASSIGN_OR_RAISE(auto struct_array,
+                            StructArray::Make(struct_fields, output_schema->fields()));
+      struct_arrays.push_back(struct_array);
     }
-    sort_keys.emplace_back(static_cast<int>(i));
+    return ChunkedArray::Make(struct_arrays);
+  } else {
+    ArrayVector struct_fields(out_arrays.size());
+    for (size_t i = 0; i < out_arrays.size(); ++i) {
+      ARROW_ASSIGN_OR_RAISE(struct_fields[i], Concatenate(out_arrays[i]));
+    }
+    return StructArray::Make(std::move(struct_fields), output_schema->fields());
   }
-  std::shared_ptr<Schema> key_schema = schema(std::move(key_fields));
-  std::shared_ptr<Table> key_table = Table::Make(std::move(key_schema), key_columns);
-  SortOptions sort_options(std::move(sort_keys));
-  ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Array> sort_indices,
-                        SortIndices(key_table, sort_options));
+}
 
-  ARROW_ASSIGN_OR_RAISE(
-      std::shared_ptr<Array> struct_arr,
-      StructArray::Make(std::move(out_arrays), output_schema->fields()));
+Result<Datum> RunGroupBy(const BatchesWithSchema& input,
+                         const std::vector<std::string>& key_names,
+                         const std::vector<std::string>& segment_key_names,
+                         const std::vector<Aggregate>& aggregates, bool use_threads,
+                         bool segmented = false, bool naive = false) {
+  if (segment_key_names.size() > 0) {
+    ARROW_ASSIGN_OR_RAISE(auto thread_pool, arrow::internal::ThreadPool::Make(1));
+    ExecContext seq_ctx(default_memory_pool(), thread_pool.get());
+    return RunGroupBy(input, key_names, segment_key_names, aggregates, &seq_ctx,
+                      use_threads, segmented, naive);
+  } else {
+    return RunGroupBy(input, key_names, segment_key_names, aggregates,
+                      threaded_exec_context(), use_threads, segmented, naive);
+  }
+}
 
-  return Take(struct_arr, sort_indices);
+Result<Datum> RunGroupBy(const BatchesWithSchema& input,
+                         const std::vector<std::string>& key_names,
+                         const std::vector<Aggregate>& aggregates, bool use_threads,
+                         bool segmented = false, bool naive = false) {
+  return RunGroupBy(input, key_names, {}, aggregates, use_threads, segmented);
 }
 
 /// Simpler overload where you can give the columns as datums
 Result<Datum> RunGroupBy(const std::vector<Datum>& arguments,
                          const std::vector<Datum>& keys,
-                         const std::vector<Aggregate>& aggregates,
-                         bool use_threads = false) {
+                         const std::vector<Datum>& segment_keys,
+                         const std::vector<Aggregate>& aggregates, bool use_threads,
+                         bool segmented = false, bool naive = false) {

Review Comment:
   I think this can be resolved since I documented the `naive` flag.



##########
cpp/src/arrow/compute/kernels/hash_aggregate_test.cc:
##########
@@ -1917,57 +2243,7 @@ TEST(GroupBy, MinMaxBinary) {
   }
 }
 
-TEST(GroupBy, MinMaxFixedSizeBinary) {

Review Comment:
   I fixed this, so I think this should be resolved.



##########
cpp/src/arrow/compute/kernels/hash_aggregate_test.cc:
##########
@@ -4202,5 +4551,260 @@ TEST(GroupBy, OnlyKeys) {
                       /*verbose=*/true);
   }
 }
+
+INSTANTIATE_TEST_SUITE_P(GroupBy, GroupBy, ::testing::Values(RunGroupByImpl));
+
+class SegmentedScalarGroupBy : public GroupBy {};
+
+class SegmentedKeyGroupBy : public GroupBy {};
+
+void TestSegment(GroupByFunction group_by, const std::shared_ptr<Table>& table,
+                 Datum output, const std::vector<Datum>& keys,
+                 const std::vector<Datum>& segment_keys, bool scalar) {
+  const char* names[] = {
+      scalar ? "count" : "hash_count",
+      scalar ? "sum" : "hash_sum",
+      scalar ? "min_max" : "hash_min_max",
+  };
+  ASSERT_OK_AND_ASSIGN(Datum aggregated_and_grouped,
+                       group_by(
+                           {
+                               table->GetColumnByName("argument"),
+                               table->GetColumnByName("argument"),
+                               table->GetColumnByName("argument"),
+                           },
+                           keys, segment_keys,
+                           {
+                               {names[0], nullptr, "agg_0", names[0]},
+                               {names[1], nullptr, "agg_1", names[1]},
+                               {names[2], nullptr, "agg_2", names[2]},
+                           },
+                           kDefaultUseThreads, /*naive=*/false));
+
+  AssertDatumsEqual(output, aggregated_and_grouped, /*verbose=*/true);
+}
+
+void TestSegmentScalar(GroupByFunction group_by, const std::shared_ptr<Table>& table,

Review Comment:
   Done.



##########
cpp/src/arrow/compute/kernels/hash_aggregate_test.cc:
##########
@@ -301,53 +426,184 @@ Result<Datum> GroupByTest(const std::vector<Datum>& arguments,
         {t_agg.function, t_agg.options, "agg_" + ToChars(idx), t_agg.function});
     idx = idx + 1;
   }
-  return RunGroupBy(arguments, keys, internal_aggregates, use_threads);
+  return group_by(arguments, keys, segment_keys, internal_aggregates, use_threads,
+                  /*naive=*/false);
 }
 
-}  // namespace
+Result<Datum> GroupByTest(GroupByFunction group_by, const std::vector<Datum>& arguments,
+                          const std::vector<Datum>& keys,
+                          const std::vector<TestAggregate>& aggregates,
+                          bool use_threads) {
+  return GroupByTest(group_by, arguments, keys, {}, aggregates, use_threads);
+}
 
-TEST(Grouper, SupportedKeys) {
-  ASSERT_OK(Grouper::Make({boolean()}));
+template <typename GroupClass>

Review Comment:
   I refactored this, so I think this can be resolved.
   



##########
cpp/src/arrow/compute/row/grouper.cc:
##########
@@ -39,12 +43,336 @@
 namespace arrow {
 
 using internal::checked_cast;
+using internal::PrimitiveScalarBase;
 
 namespace compute {
 
 namespace {
 
-struct GrouperImpl : Grouper {
+constexpr uint32_t kNoGroupId = std::numeric_limits<uint32_t>::max();
+
+using group_id_t = std::remove_const<decltype(kNoGroupId)>::type;
+using GroupIdType = CTypeTraits<group_id_t>::ArrowType;
+auto g_group_id_type = std::make_shared<GroupIdType>();
+
+inline const uint8_t* GetValuesAsBytes(const ArraySpan& data, int64_t offset = 0) {
+  DCHECK_GT(data.type->byte_width(), 0);
+  int64_t absolute_byte_offset = (data.offset + offset) * data.type->byte_width();
+  return data.GetValues<uint8_t>(1, absolute_byte_offset);
+}
+
+template <typename Value>
+Status CheckForGetNextSegment(const std::vector<Value>& values, int64_t length,
+                              int64_t offset, const std::vector<TypeHolder>& key_types) {
+  if (offset < 0 || offset > length) {

Review Comment:
   Error-checks mean the `EXPECT_RAISES_WITH_MESSAGE_THAT` checks in `TEST(RowSegmenter, Basics)`.
   
   > then it is invariant 
   
   It's only an invariant for segmented aggregation (a specific app) as the caller but not for future callers of the grouper API (as a library). 



##########
cpp/src/arrow/compute/row/grouper.cc:
##########
@@ -39,12 +43,336 @@
 namespace arrow {
 
 using internal::checked_cast;
+using internal::PrimitiveScalarBase;
 
 namespace compute {
 
 namespace {
 
-struct GrouperImpl : Grouper {
+constexpr uint32_t kNoGroupId = std::numeric_limits<uint32_t>::max();
+
+using group_id_t = std::remove_const<decltype(kNoGroupId)>::type;
+using GroupIdType = CTypeTraits<group_id_t>::ArrowType;
+auto g_group_id_type = std::make_shared<GroupIdType>();
+
+inline const uint8_t* GetValuesAsBytes(const ArraySpan& data, int64_t offset = 0) {
+  DCHECK_GT(data.type->byte_width(), 0);
+  int64_t absolute_byte_offset = (data.offset + offset) * data.type->byte_width();
+  return data.GetValues<uint8_t>(1, absolute_byte_offset);
+}
+
+template <typename Value>
+Status CheckForGetNextSegment(const std::vector<Value>& values, int64_t length,
+                              int64_t offset, const std::vector<TypeHolder>& key_types) {
+  if (offset < 0 || offset > length) {
+    return Status::Invalid("invalid grouping segmenter offset: ", offset);
+  }
+  if (values.size() != key_types.size()) {
+    return Status::Invalid("expected batch size ", key_types.size(), " but got ",
+                           values.size());
+  }
+  for (size_t i = 0; i < key_types.size(); i++) {
+    const auto& value = values[i];
+    const auto& key_type = key_types[i];
+    if (*value.type() != *key_type.type) {
+      return Status::Invalid("expected batch value ", i, " of type ", *key_type.type,
+                             " but got ", *value.type());
+    }
+  }
+  return Status::OK();
+}
+
+template <typename Batch>
+enable_if_t<std::is_same<Batch, ExecSpan>::value || std::is_same<Batch, ExecBatch>::value,
+            Status>
+CheckForGetNextSegment(const Batch& batch, int64_t offset,
+                       const std::vector<TypeHolder>& key_types) {
+  return CheckForGetNextSegment(batch.values, batch.length, offset, key_types);
+}
+
+struct BaseGroupingSegmenter : public GroupingSegmenter {
+  explicit BaseGroupingSegmenter(const std::vector<TypeHolder>& key_types)
+      : key_types_(key_types) {}
+
+  const std::vector<TypeHolder>& key_types() const override { return key_types_; }
+
+  std::vector<TypeHolder> key_types_;
+};
+
+GroupingSegment MakeSegment(int64_t batch_length, int64_t offset, int64_t length,
+                            bool extends) {
+  return GroupingSegment{offset, length, offset + length >= batch_length, extends};
+}
+
+int64_t GetMatchLength(const uint8_t* match_bytes, int64_t match_width,
+                       const uint8_t* array_bytes, int64_t offset, int64_t length) {
+  int64_t cursor, byte_cursor;
+  for (cursor = offset, byte_cursor = match_width * cursor; cursor < length;
+       cursor++, byte_cursor += match_width) {
+    if (memcmp(match_bytes, array_bytes + byte_cursor,
+               static_cast<size_t>(match_width)) != 0) {
+      break;
+    }
+  }
+  return std::min(cursor, length - offset);
+}
+
+using ExtendFunc = std::function<bool(const void*)>;
+constexpr bool kDefaultExtends = true;
+constexpr bool kEmptyExtends = true;
+
+struct NoKeysGroupingSegmenter : public BaseGroupingSegmenter {
+  static std::unique_ptr<GroupingSegmenter> Make() {
+    return std::make_unique<NoKeysGroupingSegmenter>();
+  }
+
+  NoKeysGroupingSegmenter() : BaseGroupingSegmenter({}) {}
+
+  Status Reset() override { return Status::OK(); }
+
+  Result<GroupingSegment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {}));
+    return MakeSegment(batch.length, offset, batch.length - offset, kDefaultExtends);
+  }
+};
+
+struct SimpleKeyGroupingSegmenter : public BaseGroupingSegmenter {
+  static Result<std::unique_ptr<GroupingSegmenter>> Make(TypeHolder key_type) {
+    return std::make_unique<SimpleKeyGroupingSegmenter>(key_type);
+  }
+
+  explicit SimpleKeyGroupingSegmenter(TypeHolder key_type)
+      : BaseGroupingSegmenter({key_type}), key_type_(key_types_[0]), save_key_data_() {}
+
+  Status CheckType(const DataType& type) {
+    if (!is_fixed_width(type)) {
+      return Status::Invalid("SimpleKeyGroupingSegmenter does not support type ", type);
+    }
+    return Status::OK();
+  }
+
+  Status Reset() override {
+    save_key_data_.resize(0);
+    return Status::OK();
+  }
+
+  // Checks whether the given grouping data extends the current segment, i.e., is equal to
+  // previously seen grouping data, which is updated with each invocation.
+  bool Extend(const void* data) {
+    size_t byte_width = static_cast<size_t>(key_type_.type->byte_width());
+    bool extends = save_key_data_.size() != byte_width
+                       ? kDefaultExtends
+                       : 0 == memcmp(save_key_data_.data(), data, byte_width);
+    save_key_data_.resize(byte_width);
+    memcpy(save_key_data_.data(), data, byte_width);
+    return extends;
+  }
+
+  Result<GroupingSegment> GetNextSegment(const Scalar& scalar, int64_t offset,
+                                         int64_t length) {
+    ARROW_RETURN_NOT_OK(CheckType(*scalar.type));
+    if (!scalar.is_valid) {
+      return Status::Invalid("segmenting an invalid scalar");
+    }
+    auto data = checked_cast<const PrimitiveScalarBase&>(scalar).data();
+    bool extends = length > 0 ? Extend(data) : kEmptyExtends;
+    return MakeSegment(length, offset, length, extends);
+  }
+
+  Result<GroupingSegment> GetNextSegment(const DataType& array_type,
+                                         const uint8_t* array_bytes, int64_t offset,
+                                         int64_t length) {
+    RETURN_NOT_OK(CheckType(array_type));
+    int64_t byte_width = array_type.byte_width();
+    int64_t match_length = GetMatchLength(array_bytes + offset * byte_width, byte_width,
+                                          array_bytes, offset, length);
+    bool extends = length > 0 ? Extend(array_bytes + offset * byte_width) : kEmptyExtends;
+    return MakeSegment(length, offset, match_length, extends);
+  }
+
+  Result<GroupingSegment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {key_type_}));
+    if (offset == batch.length) {
+      return MakeSegment(batch.length, offset, 0, kEmptyExtends);
+    }
+    const auto& value = batch.values[0];
+    if (value.is_scalar()) {
+      return GetNextSegment(*value.scalar, offset, batch.length);
+    }
+    ARROW_DCHECK(value.is_array());
+    const auto& array = value.array;
+    if (array.GetNullCount() > 0) {
+      return Status::NotImplemented("segmenting a nullable array");
+    }
+    return GetNextSegment(*array.type, GetValuesAsBytes(array), offset, batch.length);
+  }
+
+ private:
+  TypeHolder key_type_;
+  std::vector<uint8_t> save_key_data_;
+};
+
+struct AnyKeysGroupingSegmenter : public BaseGroupingSegmenter {
+  static Result<std::unique_ptr<GroupingSegmenter>> Make(
+      const std::vector<TypeHolder>& key_types, ExecContext* ctx) {
+    ARROW_RETURN_NOT_OK(Grouper::Make(key_types, ctx));  // check types
+    return std::make_unique<AnyKeysGroupingSegmenter>(key_types, ctx);
+  }
+
+  AnyKeysGroupingSegmenter(const std::vector<TypeHolder>& key_types, ExecContext* ctx)
+      : BaseGroupingSegmenter(key_types),
+        ctx_(ctx),
+        grouper_(nullptr),
+        save_group_id_(kNoGroupId) {}
+
+  Status Reset() override {
+    grouper_ = nullptr;
+    save_group_id_ = kNoGroupId;
+    return Status::OK();
+  }
+
+  bool Extend(const void* data) {
+    auto group_id = *static_cast<const group_id_t*>(data);
+    bool extends =
+        save_group_id_ == kNoGroupId ? kDefaultExtends : save_group_id_ == group_id;
+    save_group_id_ = group_id;
+    return extends;
+  }
+
+  // Runs the grouper on a single row.  This is used to determine the group id of the
+  // first row of a new segment to see if it extends the previous segment.
+  template <typename Batch>
+  Result<group_id_t> MapGroupIdAt(const Batch& batch, int64_t offset) {
+    if (!grouper_) return kNoGroupId;
+    ARROW_ASSIGN_OR_RAISE(auto datum, grouper_->Consume(batch, offset,
+                                                        /*consume_length=*/1));
+    if (!datum.is_array()) {
+      return Status::Invalid("accessing unsupported datum kind ", datum.kind());
+    }
+    const std::shared_ptr<ArrayData>& data = datum.array();
+    ARROW_DCHECK(data->GetNullCount() == 0);
+    DCHECK_EQ(data->type->id(), GroupIdType::type_id);
+    DCHECK_EQ(1, data->length);
+    const group_id_t* values = data->GetValues<group_id_t>(1);
+    return values[0];
+  }
+
+  Result<GroupingSegment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, key_types_));
+    if (offset == batch.length) {
+      return MakeSegment(batch.length, offset, 0, kEmptyExtends);
+    }
+    // ARROW-18311: make Grouper support Reset()
+    // so it can be cached instead of recreated below
+    //
+    // the group id must be computed prior to resetting the grouper, since it is compared
+    // to save_group_id_, and after resetting the grouper produces incomparable group ids
+    ARROW_ASSIGN_OR_RAISE(auto group_id, MapGroupIdAt(batch, offset));
+    ExtendFunc bound_extend = [this, group_id](const void* data) {
+      bool extends = Extend(&group_id);
+      save_group_id_ = *static_cast<const group_id_t*>(data);
+      return extends;
+    };
+    ARROW_ASSIGN_OR_RAISE(grouper_, Grouper::Make(key_types_, ctx_));  // TODO: reset it
+    ARROW_ASSIGN_OR_RAISE(auto datum, grouper_->Consume(batch, offset));
+    if (datum.is_array()) {
+      const std::shared_ptr<ArrayData>& data = datum.array();
+      ARROW_DCHECK(data->GetNullCount() == 0);
+      DCHECK_EQ(data->type->id(), GroupIdType::type_id);
+      const group_id_t* values = data->GetValues<group_id_t>(1);
+      int64_t cursor;
+      for (cursor = 1; cursor < data->length; cursor++) {
+        if (values[0] != values[cursor]) break;
+      }
+      int64_t length = std::min(cursor, batch.length - offset);
+      bool extends = length > 0 ? bound_extend(values) : kEmptyExtends;
+      return MakeSegment(batch.length, offset, length, extends);
+    } else {
+      return Status::Invalid("segmenting unsupported datum kind ", datum.kind());
+    }
+  }
+
+ private:
+  ExecContext* const ctx_;
+  std::unique_ptr<Grouper> grouper_;
+  group_id_t save_group_id_;
+};
+
+Status CheckForConsume(int64_t batch_length, int64_t& consume_offset,
+                       int64_t* consume_length) {
+  if (consume_offset < 0) {
+    return Status::Invalid("invalid grouper consume offset: ", consume_offset);
+  }
+  if (*consume_length < 0) {
+    *consume_length = batch_length - consume_offset;

Review Comment:
   I renamed to `CheckAndCapLengthForConsume`.



##########
cpp/src/arrow/compute/row/grouper.cc:
##########
@@ -39,12 +43,330 @@
 namespace arrow {
 
 using internal::checked_cast;
+using internal::PrimitiveScalarBase;
 
 namespace compute {
 
 namespace {
 
-struct GrouperImpl : Grouper {
+constexpr uint32_t kNoGroupId = std::numeric_limits<uint32_t>::max();
+
+using group_id_t = std::remove_const<decltype(kNoGroupId)>::type;
+using GroupIdType = CTypeTraits<group_id_t>::ArrowType;
+auto g_group_id_type = std::make_shared<GroupIdType>();
+
+inline const uint8_t* GetValuesAsBytes(const ArraySpan& data, int64_t offset = 0) {
+  DCHECK_GT(data.type->byte_width(), 0);
+  int64_t absolute_byte_offset = (data.offset + offset) * data.type->byte_width();
+  return data.GetValues<uint8_t>(1, absolute_byte_offset);
+}
+
+template <typename Value>
+Status CheckForGetNextSegment(const std::vector<Value>& values, int64_t length,
+                              int64_t offset, const std::vector<TypeHolder>& key_types) {
+  if (offset < 0 || offset > length) {
+    return Status::Invalid("invalid grouping segmenter offset: ", offset);
+  }
+  if (values.size() != key_types.size()) {
+    return Status::Invalid("expected batch size ", key_types.size(), " but got ",
+                           values.size());
+  }
+  for (size_t i = 0; i < key_types.size(); i++) {
+    const auto& value = values[i];
+    const auto& key_type = key_types[i];
+    if (*value.type() != *key_type.type) {
+      return Status::Invalid("expected batch value ", i, " of type ", *key_type.type,
+                             " but got ", *value.type());
+    }
+  }
+  return Status::OK();
+}
+
+template <typename Batch>
+enable_if_t<std::is_same<Batch, ExecSpan>::value || std::is_same<Batch, ExecBatch>::value,
+            Status>
+CheckForGetNextSegment(const Batch& batch, int64_t offset,
+                       const std::vector<TypeHolder>& key_types) {
+  return CheckForGetNextSegment(batch.values, batch.length, offset, key_types);
+}
+
+struct BaseRowSegmenter : public RowSegmenter {
+  explicit BaseRowSegmenter(const std::vector<TypeHolder>& key_types)
+      : key_types_(key_types) {}
+
+  const std::vector<TypeHolder>& key_types() const override { return key_types_; }
+
+  std::vector<TypeHolder> key_types_;
+};
+
+Segment MakeSegment(int64_t batch_length, int64_t offset, int64_t length, bool extends) {
+  return Segment{offset, length, offset + length >= batch_length, extends};
+}
+
+int64_t GetMatchLength(const uint8_t* match_bytes, int64_t match_width,
+                       const uint8_t* array_bytes, int64_t offset, int64_t length) {
+  int64_t cursor, byte_cursor;
+  for (cursor = offset, byte_cursor = match_width * cursor; cursor < length;
+       cursor++, byte_cursor += match_width) {
+    if (memcmp(match_bytes, array_bytes + byte_cursor,
+               static_cast<size_t>(match_width)) != 0) {
+      break;
+    }
+  }
+  return std::min(cursor, length) - offset;
+}
+
+using ExtendFunc = std::function<bool(const void*)>;
+constexpr bool kDefaultExtends = true;
+constexpr bool kEmptyExtends = true;
+
+struct NoKeysSegmenter : public BaseRowSegmenter {
+  static std::unique_ptr<RowSegmenter> Make() {
+    return std::make_unique<NoKeysSegmenter>();
+  }
+
+  NoKeysSegmenter() : BaseRowSegmenter({}) {}
+
+  Status Reset() override { return Status::OK(); }
+
+  Result<Segment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {}));
+    return MakeSegment(batch.length, offset, batch.length - offset, kDefaultExtends);
+  }
+};
+
+struct SimpleKeySegmenter : public BaseRowSegmenter {
+  static Result<std::unique_ptr<RowSegmenter>> Make(TypeHolder key_type) {
+    return std::make_unique<SimpleKeySegmenter>(key_type);
+  }
+
+  explicit SimpleKeySegmenter(TypeHolder key_type)
+      : BaseRowSegmenter({key_type}), key_type_(key_types_[0]), save_key_data_() {}
+
+  Status CheckType(const DataType& type) {
+    if (!is_fixed_width(type)) {
+      return Status::Invalid("SimpleKeySegmenter does not support type ", type);
+    }
+    return Status::OK();
+  }
+
+  Status Reset() override {
+    save_key_data_.resize(0);
+    return Status::OK();
+  }
+
+  // Checks whether the given grouping data extends the current segment, i.e., is equal to
+  // previously seen grouping data, which is updated with each invocation.
+  bool Extend(const void* data) {
+    size_t byte_width = static_cast<size_t>(key_type_.type->byte_width());
+    bool extends = save_key_data_.size() != byte_width
+                       ? kDefaultExtends
+                       : 0 == memcmp(save_key_data_.data(), data, byte_width);
+    save_key_data_.resize(byte_width);
+    memcpy(save_key_data_.data(), data, byte_width);
+    return extends;
+  }
+
+  Result<Segment> GetNextSegment(const Scalar& scalar, int64_t offset, int64_t length) {
+    ARROW_RETURN_NOT_OK(CheckType(*scalar.type));
+    if (!scalar.is_valid) {
+      return Status::Invalid("segmenting an invalid scalar");
+    }
+    auto data = checked_cast<const PrimitiveScalarBase&>(scalar).data();
+    bool extends = length > 0 ? Extend(data) : kEmptyExtends;
+    return MakeSegment(length, offset, length, extends);
+  }
+
+  Result<Segment> GetNextSegment(const DataType& array_type, const uint8_t* array_bytes,
+                                 int64_t offset, int64_t length) {
+    RETURN_NOT_OK(CheckType(array_type));
+    int64_t byte_width = array_type.byte_width();
+    int64_t match_length = GetMatchLength(array_bytes + offset * byte_width, byte_width,
+                                          array_bytes, offset, length);
+    bool extends = length > 0 ? Extend(array_bytes + offset * byte_width) : kEmptyExtends;
+    return MakeSegment(length, offset, match_length, extends);
+  }
+
+  Result<Segment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {key_type_}));
+    if (offset == batch.length) {
+      return MakeSegment(batch.length, offset, 0, kEmptyExtends);
+    }
+    const auto& value = batch.values[0];
+    if (value.is_scalar()) {
+      return GetNextSegment(*value.scalar, offset, batch.length);
+    }
+    ARROW_DCHECK(value.is_array());
+    const auto& array = value.array;
+    if (array.GetNullCount() > 0) {
+      return Status::NotImplemented("segmenting a nullable array");
+    }
+    return GetNextSegment(*array.type, GetValuesAsBytes(array), offset, batch.length);
+  }
+
+ private:
+  TypeHolder key_type_;
+  std::vector<uint8_t> save_key_data_;
+};
+
+struct AnyKeysSegmenter : public BaseRowSegmenter {
+  static Result<std::unique_ptr<RowSegmenter>> Make(
+      const std::vector<TypeHolder>& key_types, ExecContext* ctx) {
+    ARROW_RETURN_NOT_OK(Grouper::Make(key_types, ctx));  // check types
+    return std::make_unique<AnyKeysSegmenter>(key_types, ctx);
+  }
+
+  AnyKeysSegmenter(const std::vector<TypeHolder>& key_types, ExecContext* ctx)
+      : BaseRowSegmenter(key_types),
+        ctx_(ctx),
+        grouper_(nullptr),
+        save_group_id_(kNoGroupId) {}
+
+  Status Reset() override {
+    grouper_ = nullptr;
+    save_group_id_ = kNoGroupId;
+    return Status::OK();
+  }
+
+  bool Extend(const void* data) {
+    auto group_id = *static_cast<const group_id_t*>(data);
+    bool extends =
+        save_group_id_ == kNoGroupId ? kDefaultExtends : save_group_id_ == group_id;
+    save_group_id_ = group_id;
+    return extends;
+  }
+
+  // Runs the grouper on a single row.  This is used to determine the group id of the
+  // first row of a new segment to see if it extends the previous segment.
+  template <typename Batch>
+  Result<group_id_t> MapGroupIdAt(const Batch& batch, int64_t offset) {
+    if (!grouper_) return kNoGroupId;
+    ARROW_ASSIGN_OR_RAISE(auto datum, grouper_->Consume(batch, offset,
+                                                        /*length=*/1));
+    if (!datum.is_array()) {
+      return Status::Invalid("accessing unsupported datum kind ", datum.kind());
+    }
+    const std::shared_ptr<ArrayData>& data = datum.array();
+    ARROW_DCHECK(data->GetNullCount() == 0);
+    DCHECK_EQ(data->type->id(), GroupIdType::type_id);
+    DCHECK_EQ(1, data->length);
+    const group_id_t* values = data->GetValues<group_id_t>(1);
+    return values[0];
+  }
+
+  Result<Segment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, key_types_));
+    if (offset == batch.length) {
+      return MakeSegment(batch.length, offset, 0, kEmptyExtends);
+    }
+    // ARROW-18311: make Grouper support Reset()
+    // so it can be cached instead of recreated below
+    //
+    // the group id must be computed prior to resetting the grouper, since it is compared
+    // to save_group_id_, and after resetting the grouper produces incomparable group ids
+    ARROW_ASSIGN_OR_RAISE(auto group_id, MapGroupIdAt(batch, offset));
+    ExtendFunc bound_extend = [this, group_id](const void* data) {
+      bool extends = Extend(&group_id);
+      save_group_id_ = *static_cast<const group_id_t*>(data);
+      return extends;
+    };
+    ARROW_ASSIGN_OR_RAISE(grouper_, Grouper::Make(key_types_, ctx_));  // TODO: reset it

Review Comment:
   Done.



##########
cpp/src/arrow/compute/kernels/hash_aggregate_test.cc:
##########
@@ -174,81 +255,117 @@ Result<Datum> RunGroupBy(const BatchesWithSchema& input,
   ARROW_ASSIGN_OR_RAISE(std::vector<ExecBatch> output_batches,
                         start_and_collect.MoveResult());
 
-  ArrayVector out_arrays(aggregates.size() + key_names.size());
   const auto& output_schema = plan->nodes()[0]->output()->output_schema();
+  if (!segmented) {
+    return MakeGroupByOutput(output_batches, output_schema, aggregates.size(),
+                             key_names.size(), naive);
+  }
+
+  std::vector<ArrayVector> out_arrays(aggregates.size() + key_names.size() +
+                                      segment_key_names.size());
   for (size_t i = 0; i < out_arrays.size(); ++i) {
     std::vector<std::shared_ptr<Array>> arrays(output_batches.size());
     for (size_t j = 0; j < output_batches.size(); ++j) {
-      arrays[j] = output_batches[j].values[i].make_array();
+      auto& value = output_batches[j].values[i];
+      if (value.is_scalar()) {
+        ARROW_ASSIGN_OR_RAISE(
+            arrays[j], MakeArrayFromScalar(*value.scalar(), output_batches[j].length));
+      } else if (value.is_array()) {
+        arrays[j] = value.make_array();
+      } else {
+        return Status::Invalid("GroupByUsingExecPlan unsupported value kind ",
+                               ToString(value.kind()));
+      }
     }
     if (arrays.empty()) {
+      arrays.resize(1);
       ARROW_ASSIGN_OR_RAISE(
-          out_arrays[i],
-          MakeArrayOfNull(output_schema->field(static_cast<int>(i))->type(),
-                          /*length=*/0));
-    } else {
-      ARROW_ASSIGN_OR_RAISE(out_arrays[i], Concatenate(arrays));
+          arrays[0], MakeArrayOfNull(output_schema->field(static_cast<int>(i))->type(),
+                                     /*length=*/0));
     }
+    out_arrays[i] = {std::move(arrays)};
   }
 
-  // The exec plan may reorder the output rows.  The tests are all setup to expect ouptut
-  // in ascending order of keys.  So we need to sort the result by the key columns.  To do
-  // that we create a table using the key columns, calculate the sort indices from that
-  // table (sorting on all fields) and then use those indices to calculate our result.
-  std::vector<std::shared_ptr<Field>> key_fields;
-  std::vector<std::shared_ptr<Array>> key_columns;
-  std::vector<SortKey> sort_keys;
-  for (std::size_t i = 0; i < key_names.size(); i++) {
-    const std::shared_ptr<Array>& arr = out_arrays[i + aggregates.size()];
-    if (arr->type_id() == Type::DICTIONARY) {
-      // Can't sort dictionary columns so need to decode
-      auto dict_arr = checked_pointer_cast<DictionaryArray>(arr);
-      ARROW_ASSIGN_OR_RAISE(auto decoded_arr,
-                            Take(*dict_arr->dictionary(), *dict_arr->indices()));
-      key_columns.push_back(decoded_arr);
-      key_fields.push_back(
-          field("name_does_not_matter", dict_arr->dict_type()->value_type()));
-    } else {
-      key_columns.push_back(arr);
-      key_fields.push_back(field("name_does_not_matter", arr->type()));
+  if (segmented && segment_key_names.size() > 0) {
+    ArrayVector struct_arrays;
+    struct_arrays.reserve(output_batches.size());
+    for (size_t j = 0; j < output_batches.size(); ++j) {
+      ArrayVector struct_fields;
+      struct_fields.reserve(out_arrays.size());
+      for (auto out_array : out_arrays) {
+        struct_fields.push_back(out_array[j]);
+      }
+      ARROW_ASSIGN_OR_RAISE(auto struct_array,
+                            StructArray::Make(struct_fields, output_schema->fields()));
+      struct_arrays.push_back(struct_array);
     }
-    sort_keys.emplace_back(static_cast<int>(i));
+    return ChunkedArray::Make(struct_arrays);
+  } else {
+    ArrayVector struct_fields(out_arrays.size());
+    for (size_t i = 0; i < out_arrays.size(); ++i) {
+      ARROW_ASSIGN_OR_RAISE(struct_fields[i], Concatenate(out_arrays[i]));
+    }
+    return StructArray::Make(std::move(struct_fields), output_schema->fields());
   }
-  std::shared_ptr<Schema> key_schema = schema(std::move(key_fields));
-  std::shared_ptr<Table> key_table = Table::Make(std::move(key_schema), key_columns);
-  SortOptions sort_options(std::move(sort_keys));
-  ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Array> sort_indices,
-                        SortIndices(key_table, sort_options));
+}
 
-  ARROW_ASSIGN_OR_RAISE(
-      std::shared_ptr<Array> struct_arr,
-      StructArray::Make(std::move(out_arrays), output_schema->fields()));
+Result<Datum> RunGroupBy(const BatchesWithSchema& input,
+                         const std::vector<std::string>& key_names,
+                         const std::vector<std::string>& segment_key_names,
+                         const std::vector<Aggregate>& aggregates, bool use_threads,
+                         bool segmented = false, bool naive = false) {
+  if (segment_key_names.size() > 0) {
+    ARROW_ASSIGN_OR_RAISE(auto thread_pool, arrow::internal::ThreadPool::Make(1));
+    ExecContext seq_ctx(default_memory_pool(), thread_pool.get());
+    return RunGroupBy(input, key_names, segment_key_names, aggregates, &seq_ctx,

Review Comment:
   Done.



##########
cpp/src/arrow/compute/kernels/hash_aggregate_test.cc:
##########
@@ -301,53 +437,249 @@ Result<Datum> GroupByTest(const std::vector<Datum>& arguments,
         {t_agg.function, t_agg.options, "agg_" + ToChars(idx), t_agg.function});
     idx = idx + 1;
   }
-  return RunGroupBy(arguments, keys, internal_aggregates, use_threads);
+  return group_by(arguments, keys, segment_keys, internal_aggregates, use_threads,
+                  /*naive=*/false);
 }
 
-}  // namespace
+Result<Datum> GroupByTest(GroupByFunction group_by, const std::vector<Datum>& arguments,
+                          const std::vector<Datum>& keys,
+                          const std::vector<TestAggregate>& aggregates,
+                          bool use_threads) {
+  return GroupByTest(group_by, arguments, keys, {}, aggregates, use_threads);
+}
 
-TEST(Grouper, SupportedKeys) {
-  ASSERT_OK(Grouper::Make({boolean()}));
+template <typename GroupClass>
+void TestGroupClassSupportedKeys(
+    std::function<Result<std::unique_ptr<GroupClass>>(const std::vector<TypeHolder>&)>
+        make_func) {
+  ASSERT_OK(make_func({boolean()}));
 
-  ASSERT_OK(Grouper::Make({int8(), uint16(), int32(), uint64()}));
+  ASSERT_OK(make_func({int8(), uint16(), int32(), uint64()}));
 
-  ASSERT_OK(Grouper::Make({dictionary(int64(), utf8())}));
+  ASSERT_OK(make_func({dictionary(int64(), utf8())}));
 
-  ASSERT_OK(Grouper::Make({float16(), float32(), float64()}));
+  ASSERT_OK(make_func({float16(), float32(), float64()}));
 
-  ASSERT_OK(Grouper::Make({utf8(), binary(), large_utf8(), large_binary()}));
+  ASSERT_OK(make_func({utf8(), binary(), large_utf8(), large_binary()}));
 
-  ASSERT_OK(Grouper::Make({fixed_size_binary(16), fixed_size_binary(32)}));
+  ASSERT_OK(make_func({fixed_size_binary(16), fixed_size_binary(32)}));
 
-  ASSERT_OK(Grouper::Make({decimal128(32, 10), decimal256(76, 20)}));
+  ASSERT_OK(make_func({decimal128(32, 10), decimal256(76, 20)}));
 
-  ASSERT_OK(Grouper::Make({date32(), date64()}));
+  ASSERT_OK(make_func({date32(), date64()}));
 
   for (auto unit : {
            TimeUnit::SECOND,
            TimeUnit::MILLI,
            TimeUnit::MICRO,
            TimeUnit::NANO,
        }) {
-    ASSERT_OK(Grouper::Make({timestamp(unit), duration(unit)}));
+    ASSERT_OK(make_func({timestamp(unit), duration(unit)}));
   }
 
   ASSERT_OK(
-      Grouper::Make({day_time_interval(), month_interval(), month_day_nano_interval()}));
+      make_func({day_time_interval(), month_interval(), month_day_nano_interval()}));
+
+  ASSERT_OK(make_func({null()}));
+
+  ASSERT_RAISES(NotImplemented, make_func({struct_({field("", int64())})}));
+
+  ASSERT_RAISES(NotImplemented, make_func({struct_({})}));
+
+  ASSERT_RAISES(NotImplemented, make_func({list(int32())}));
+
+  ASSERT_RAISES(NotImplemented, make_func({fixed_size_list(int32(), 5)}));
+
+  ASSERT_RAISES(NotImplemented, make_func({dense_union({field("", int32())})}));
+}
+
+void TestSegments(std::unique_ptr<RowSegmenter>& segmenter, const ExecSpan& batch,
+                  std::vector<Segment> expected_segments) {
+  int64_t offset = 0, segment_num = 0;
+  for (auto expected_segment : expected_segments) {
+    SCOPED_TRACE("segment #" + ToChars(segment_num++));
+    ASSERT_OK_AND_ASSIGN(auto segment, segmenter->GetNextSegment(batch, offset));
+    ASSERT_EQ(expected_segment, segment);
+    offset = segment.offset + segment.length;
+  }
+}
+
+Result<std::unique_ptr<Grouper>> MakeGrouper(const std::vector<TypeHolder>& key_types) {
+  return Grouper::Make(key_types, default_exec_context());
+}
+
+Result<std::unique_ptr<RowSegmenter>> MakeRowSegmenter(
+    const std::vector<TypeHolder>& key_types) {
+  return RowSegmenter::Make(key_types, /*nullable_leys=*/false, default_exec_context());
+}
+
+Result<std::unique_ptr<RowSegmenter>> MakeGenericSegmenter(
+    const std::vector<TypeHolder>& key_types) {
+  return MakeAnyKeysSegmenter(key_types, default_exec_context());
+}
+
+}  // namespace
+
+TEST(RowSegmenter, SupportedKeys) {
+  TestGroupClassSupportedKeys<RowSegmenter>(MakeRowSegmenter);
+}
+
+TEST(RowSegmenter, Basics) {
+  std::vector<TypeHolder> bad_types2 = {int32(), float32()};
+  std::vector<TypeHolder> types2 = {int32(), int32()};
+  std::vector<TypeHolder> bad_types1 = {float32()};
+  std::vector<TypeHolder> types1 = {int32()};
+  std::vector<TypeHolder> types0 = {};
+  auto batch2 = ExecBatchFromJSON(types2, "[[1, 1], [1, 2], [2, 2]]");

Review Comment:
   I added the listed test cases.



##########
cpp/src/arrow/compute/row/grouper.cc:
##########
@@ -39,12 +44,329 @@
 namespace arrow {
 
 using internal::checked_cast;
+using internal::PrimitiveScalarBase;
 
 namespace compute {
 
 namespace {
 
-struct GrouperImpl : Grouper {
+constexpr uint32_t kNoGroupId = std::numeric_limits<uint32_t>::max();
+
+using group_id_t = std::remove_const<decltype(kNoGroupId)>::type;
+using GroupIdType = CTypeTraits<group_id_t>::ArrowType;
+auto g_group_id_type = std::make_shared<GroupIdType>();
+
+inline const uint8_t* GetValuesAsBytes(const ArraySpan& data, int64_t offset = 0) {
+  DCHECK_GT(data.type->byte_width(), 0);
+  int64_t absolute_byte_offset = (data.offset + offset) * data.type->byte_width();
+  return data.GetValues<uint8_t>(1, absolute_byte_offset);
+}
+
+template <typename Value>
+Status CheckForGetNextSegment(const std::vector<Value>& values, int64_t length,
+                              int64_t offset, const std::vector<TypeHolder>& key_types) {
+  if (offset < 0 || offset > length) {
+    return Status::Invalid("invalid grouping segmenter offset: ", offset);
+  }
+  if (values.size() != key_types.size()) {
+    return Status::Invalid("expected batch size ", key_types.size(), " but got ",
+                           values.size());
+  }
+  for (size_t i = 0; i < key_types.size(); i++) {
+    const auto& value = values[i];
+    const auto& key_type = key_types[i];
+    if (*value.type() != *key_type.type) {
+      return Status::Invalid("expected batch value ", i, " of type ", *key_type.type,
+                             " but got ", *value.type());
+    }
+  }
+  return Status::OK();
+}
+
+template <typename Batch>
+enable_if_t<std::is_same<Batch, ExecSpan>::value || std::is_same<Batch, ExecBatch>::value,
+            Status>
+CheckForGetNextSegment(const Batch& batch, int64_t offset,
+                       const std::vector<TypeHolder>& key_types) {
+  return CheckForGetNextSegment(batch.values, batch.length, offset, key_types);
+}
+
+struct BaseRowSegmenter : public RowSegmenter {
+  explicit BaseRowSegmenter(const std::vector<TypeHolder>& key_types)
+      : key_types_(key_types) {}
+
+  const std::vector<TypeHolder>& key_types() const override { return key_types_; }
+
+  std::vector<TypeHolder> key_types_;
+};
+
+Segment MakeSegment(int64_t batch_length, int64_t offset, int64_t length, bool extends) {
+  return Segment{offset, length, offset + length >= batch_length, extends};
+}
+
+// Used by SimpleKeySegmenter::GetNextSegment to find the match-length of a value within a
+// fixed-width buffer
+int64_t GetMatchLength(const uint8_t* match_bytes, int64_t match_width,
+                       const uint8_t* array_bytes, int64_t offset, int64_t length) {
+  int64_t cursor, byte_cursor;
+  for (cursor = offset, byte_cursor = match_width * cursor; cursor < length;
+       cursor++, byte_cursor += match_width) {
+    if (memcmp(match_bytes, array_bytes + byte_cursor,
+               static_cast<size_t>(match_width)) != 0) {
+      break;
+    }
+  }
+  return std::min(cursor, length) - offset;
+}
+
+using ExtendFunc = std::function<bool(const void*)>;
+constexpr bool kDefaultExtends = true;  // by default, the first segment extends
+constexpr bool kEmptyExtends = true;    // an empty segment extends too
+
+struct NoKeysSegmenter : public BaseRowSegmenter {
+  static std::unique_ptr<RowSegmenter> Make() {
+    return std::make_unique<NoKeysSegmenter>();
+  }
+
+  NoKeysSegmenter() : BaseRowSegmenter({}) {}
+
+  Status Reset() override { return Status::OK(); }
+
+  Result<Segment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {}));
+    return MakeSegment(batch.length, offset, batch.length - offset, kDefaultExtends);
+  }
+};
+
+struct SimpleKeySegmenter : public BaseRowSegmenter {
+  static Result<std::unique_ptr<RowSegmenter>> Make(TypeHolder key_type) {
+    return std::make_unique<SimpleKeySegmenter>(key_type);
+  }
+
+  explicit SimpleKeySegmenter(TypeHolder key_type)
+      : BaseRowSegmenter({key_type}),
+        key_type_(key_types_[0]),
+        save_key_data_(static_cast<size_t>(key_type_.type->byte_width())),
+        extend_was_called_(false) {}
+
+  Status CheckType(const DataType& type) {
+    if (!is_fixed_width(type)) {
+      return Status::Invalid("SimpleKeySegmenter does not support type ", type);
+    }
+    return Status::OK();
+  }
+
+  Status Reset() override {
+    extend_was_called_ = false;
+    return Status::OK();
+  }
+
+  // Checks whether the given grouping data extends the current segment, i.e., is equal to
+  // previously seen grouping data, which is updated with each invocation.
+  bool Extend(const void* data) {
+    bool extends = !extend_was_called_
+                       ? kDefaultExtends
+                       : 0 == memcmp(save_key_data_.data(), data, save_key_data_.size());
+    extend_was_called_ = true;
+    memcpy(save_key_data_.data(), data, save_key_data_.size());
+    return extends;
+  }
+
+  Result<Segment> GetNextSegment(const Scalar& scalar, int64_t offset, int64_t length) {
+    ARROW_RETURN_NOT_OK(CheckType(*scalar.type));
+    if (!scalar.is_valid) {
+      return Status::Invalid("segmenting an invalid scalar");
+    }
+    auto data = checked_cast<const PrimitiveScalarBase&>(scalar).data();
+    bool extends = length > 0 ? Extend(data) : kEmptyExtends;
+    return MakeSegment(length, offset, length, extends);
+  }
+
+  Result<Segment> GetNextSegment(const DataType& array_type, const uint8_t* array_bytes,
+                                 int64_t offset, int64_t length) {
+    RETURN_NOT_OK(CheckType(array_type));
+    DCHECK_LE(offset, length);
+    int64_t byte_width = array_type.byte_width();
+    int64_t match_length = GetMatchLength(array_bytes + offset * byte_width, byte_width,
+                                          array_bytes, offset, length);
+    bool extends = length > 0 ? Extend(array_bytes + offset * byte_width) : kEmptyExtends;
+    return MakeSegment(length, offset, match_length, extends);
+  }
+
+  Result<Segment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {key_type_}));
+    if (offset == batch.length) {
+      return MakeSegment(batch.length, offset, 0, kEmptyExtends);
+    }
+    const auto& value = batch.values[0];
+    if (value.is_scalar()) {
+      return GetNextSegment(*value.scalar, offset, batch.length);
+    }
+    ARROW_DCHECK(value.is_array());
+    const auto& array = value.array;
+    if (array.GetNullCount() > 0) {
+      return Status::NotImplemented("segmenting a nullable array");
+    }
+    return GetNextSegment(*array.type, GetValuesAsBytes(array), offset, batch.length);
+  }
+
+ private:
+  TypeHolder key_type_;
+  std::vector<uint8_t> save_key_data_;  // previusly seen segment-key grouping data
+  bool extend_was_called_;
+};
+
+struct AnyKeysSegmenter : public BaseRowSegmenter {
+  static Result<std::unique_ptr<RowSegmenter>> Make(
+      const std::vector<TypeHolder>& key_types, ExecContext* ctx) {
+    ARROW_RETURN_NOT_OK(Grouper::Make(key_types, ctx));  // check types
+    return std::make_unique<AnyKeysSegmenter>(key_types, ctx);
+  }
+
+  AnyKeysSegmenter(const std::vector<TypeHolder>& key_types, ExecContext* ctx)
+      : BaseRowSegmenter(key_types),
+        ctx_(ctx),
+        grouper_(nullptr),
+        save_group_id_(kNoGroupId) {}
+
+  Status Reset() override {
+    grouper_ = nullptr;
+    save_group_id_ = kNoGroupId;
+    return Status::OK();
+  }
+
+  bool Extend(const void* data) {
+    auto group_id = *static_cast<const group_id_t*>(data);
+    bool extends =
+        save_group_id_ == kNoGroupId ? kDefaultExtends : save_group_id_ == group_id;
+    save_group_id_ = group_id;
+    return extends;
+  }
+
+  // Runs the grouper on a single row.  This is used to determine the group id of the
+  // first row of a new segment to see if it extends the previous segment.
+  template <typename Batch>
+  Result<group_id_t> MapGroupIdAt(const Batch& batch, int64_t offset) {
+    if (!grouper_) return kNoGroupId;
+    ARROW_ASSIGN_OR_RAISE(auto datum, grouper_->Consume(batch, offset,
+                                                        /*length=*/1));
+    if (!datum.is_array()) {
+      return Status::Invalid("accessing unsupported datum kind ", datum.kind());
+    }
+    const std::shared_ptr<ArrayData>& data = datum.array();
+    ARROW_DCHECK(data->GetNullCount() == 0);
+    DCHECK_EQ(data->type->id(), GroupIdType::type_id);
+    DCHECK_EQ(1, data->length);
+    const group_id_t* values = data->GetValues<group_id_t>(1);
+    return values[0];
+  }
+
+  Result<Segment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, key_types_));
+    if (offset == batch.length) {
+      return MakeSegment(batch.length, offset, 0, kEmptyExtends);
+    }
+    // ARROW-18311: make Grouper support Reset()
+    // so it can be cached instead of recreated below
+    //
+    // the group id must be computed prior to resetting the grouper, since it is compared
+    // to save_group_id_, and after resetting the grouper produces incomparable group ids
+    ARROW_ASSIGN_OR_RAISE(auto group_id, MapGroupIdAt(batch, offset));
+    ExtendFunc bound_extend = [this, group_id](const void* data) {
+      bool extends = Extend(&group_id);
+      save_group_id_ = *static_cast<const group_id_t*>(data);
+      return extends;
+    };
+    ARROW_ASSIGN_OR_RAISE(grouper_, Grouper::Make(key_types_, ctx_));  // TODO: reset it
+    ARROW_ASSIGN_OR_RAISE(auto datum, grouper_->Consume(batch, offset));

Review Comment:
   Done.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rtpsw commented on pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "rtpsw (via GitHub)" <gi...@apache.org>.
rtpsw commented on PR #34311:
URL: https://github.com/apache/arrow/pull/34311#issuecomment-1460920927

   > @rtpsw I think this is getting close but there are still a number of unresolved thread. Please check those are resolved and ping me when it is ready for me to take another look.
   
   @icexelloss, I went over the unresolved discussions and commented; I think they can now be resolved. The deferred issues I found are in #34475.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rtpsw commented on pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "rtpsw (via GitHub)" <gi...@apache.org>.
rtpsw commented on PR #34311:
URL: https://github.com/apache/arrow/pull/34311#issuecomment-1462655518

   > Can you gather all the follow up issues and put them as a list in the PR description and the origin GH issue as well?
   
   Linked in both PR and original GH issue to #34475 which has the list.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] icexelloss commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "icexelloss (via GitHub)" <gi...@apache.org>.
icexelloss commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1125574240


##########
cpp/src/arrow/compute/exec/aggregate_node.cc:
##########
@@ -169,35 +185,117 @@ void AggregatesToString(std::stringstream* ss, const Schema& input_schema,
   *ss << ']';
 }
 
+template <typename BatchHandler>
+Status HandleSegments(std::unique_ptr<GroupingSegmenter>& segmenter,
+                      const ExecBatch& batch, const std::vector<int>& ids,
+                      const BatchHandler& handle_batch) {
+  int64_t offset = 0;
+  ARROW_ASSIGN_OR_RAISE(auto segment_exec_batch, batch.SelectValues(ids));
+  ExecSpan segment_batch(segment_exec_batch);
+  while (true) {
+    ARROW_ASSIGN_OR_RAISE(auto segment, segmenter->GetNextSegment(segment_batch, offset));
+    if (segment.offset >= segment_batch.length) break;  // condition of no-next-segment
+    ARROW_RETURN_NOT_OK(handle_batch(batch, segment));
+    offset = segment.offset + segment.length;
+  }
+  return Status::OK();
+}
+
+Status GetScalarFields(std::vector<Datum>* values_ptr, const ExecBatch& input_batch,
+                       const std::vector<int>& field_ids) {
+  DCHECK_GT(input_batch.length, 0);
+  std::vector<Datum>& values = *values_ptr;
+  int64_t row = input_batch.length - 1;
+  values.clear();
+  values.resize(field_ids.size());
+  for (size_t i = 0; i < field_ids.size(); i++) {
+    const Datum& value = input_batch.values[field_ids[i]];
+    if (value.is_scalar()) {
+      values[i] = value;
+    } else if (value.is_array()) {
+      ARROW_ASSIGN_OR_RAISE(auto scalar, value.make_array()->GetScalar(row));
+      values[i] = scalar;
+    } else {
+      DCHECK(false);
+    }
+  }
+  return Status::OK();
+}
+
+void PlaceFields(ExecBatch& batch, size_t base, std::vector<Datum>& values) {
+  DCHECK_LE(base + values.size(), batch.values.size());
+  for (size_t i = 0; i < values.size(); i++) {
+    batch.values[base + i] = values[i];
+  }
+}
+
 class ScalarAggregateNode : public ExecNode, public TracedNode {
  public:
   ScalarAggregateNode(ExecPlan* plan, std::vector<ExecNode*> inputs,
                       std::shared_ptr<Schema> output_schema,
+                      std::unique_ptr<GroupingSegmenter> segmenter,
+                      std::vector<int> segment_field_ids,
                       std::vector<std::vector<int>> target_fieldsets,
                       std::vector<Aggregate> aggs,
                       std::vector<const ScalarAggregateKernel*> kernels,
                       std::vector<std::vector<std::unique_ptr<KernelState>>> states)
       : ExecNode(plan, std::move(inputs), {"target"},
                  /*output_schema=*/std::move(output_schema)),
         TracedNode(this),
+        segmenter_(std::move(segmenter)),
+        segment_field_ids_(std::move(segment_field_ids)),
         target_fieldsets_(std::move(target_fieldsets)),
         aggs_(std::move(aggs)),
         kernels_(std::move(kernels)),
-        states_(std::move(states)) {}
+        states_(std::move(states)) {
+    const auto& input_schema = *this->inputs()[0]->output_schema();
+    for (size_t i = 0; i < kernels_.size(); ++i) {
+      std::vector<TypeHolder> in_types;
+      for (const auto& target : target_fieldsets_[i]) {
+        in_types.emplace_back(input_schema.field(target)->type().get());
+      }
+      in_typesets_.push_back(std::move(in_types));
+    }
+  }
 
   static Result<ExecNode*> Make(ExecPlan* plan, std::vector<ExecNode*> inputs,
                                 const ExecNodeOptions& options) {
     RETURN_NOT_OK(ValidateExecNodeInputs(plan, inputs, 1, "ScalarAggregateNode"));
 
     const auto& aggregate_options = checked_cast<const AggregateNodeOptions&>(options);
     auto aggregates = aggregate_options.aggregates;
+    const auto& keys = aggregate_options.keys;
+    const auto& segment_keys = aggregate_options.segment_keys;
+
+    if (keys.size() > 0) {
+      return Status::Invalid("Scalar aggregation with some key");
+    }
+    if (plan->query_context()->exec_context()->executor()->GetCapacity() > 1 &&
+        segment_keys.size() > 0) {
+      return Status::NotImplemented("Segmented aggregation in a multi-threaded plan");
+    }
 
     const auto& input_schema = *inputs[0]->output_schema();
     auto exec_ctx = plan->query_context()->exec_context();
 
+    std::vector<int> segment_field_ids(segment_keys.size());
+    std::vector<TypeHolder> segment_key_types(segment_keys.size());
+    for (size_t i = 0; i < segment_keys.size(); i++) {
+      ARROW_ASSIGN_OR_RAISE(auto match, segment_keys[i].FindOne(input_schema));
+      if (match.indices().size() > 1) {
+        // ARROW-18369: Support nested references as segment ids
+        return Status::Invalid("Nested references cannot be used as segment ids");
+      }
+      segment_field_ids[i] = match[0];
+      segment_key_types[i] = input_schema.field(match[0])->type().get();
+    }
+
+    ARROW_ASSIGN_OR_RAISE(
+        auto segmenter, GroupingSegmenter::Make(std::move(segment_key_types), exec_ctx));
+
     std::vector<const ScalarAggregateKernel*> kernels(aggregates.size());
     std::vector<std::vector<std::unique_ptr<KernelState>>> states(kernels.size());
-    FieldVector fields(kernels.size());
+    FieldVector fields(kernels.size() + segment_keys.size());

Review Comment:
   Oh I saw you replied in https://github.com/apache/arrow/pull/34311/files/01a946b3cfe955cbc6267895a6163a68b8638b01#r1124230571



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rtpsw commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "rtpsw (via GitHub)" <gi...@apache.org>.
rtpsw commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1124226306


##########
cpp/src/arrow/compute/row/grouper.cc:
##########
@@ -39,12 +43,336 @@
 namespace arrow {
 
 using internal::checked_cast;
+using internal::PrimitiveScalarBase;
 
 namespace compute {
 
 namespace {
 
-struct GrouperImpl : Grouper {
+constexpr uint32_t kNoGroupId = std::numeric_limits<uint32_t>::max();
+
+using group_id_t = std::remove_const<decltype(kNoGroupId)>::type;
+using GroupIdType = CTypeTraits<group_id_t>::ArrowType;
+auto g_group_id_type = std::make_shared<GroupIdType>();
+
+inline const uint8_t* GetValuesAsBytes(const ArraySpan& data, int64_t offset = 0) {
+  DCHECK_GT(data.type->byte_width(), 0);
+  int64_t absolute_byte_offset = (data.offset + offset) * data.type->byte_width();
+  return data.GetValues<uint8_t>(1, absolute_byte_offset);
+}
+
+template <typename Value>
+Status CheckForGetNextSegment(const std::vector<Value>& values, int64_t length,
+                              int64_t offset, const std::vector<TypeHolder>& key_types) {
+  if (offset < 0 || offset > length) {
+    return Status::Invalid("invalid grouping segmenter offset: ", offset);
+  }
+  if (values.size() != key_types.size()) {
+    return Status::Invalid("expected batch size ", key_types.size(), " but got ",
+                           values.size());
+  }
+  for (size_t i = 0; i < key_types.size(); i++) {
+    const auto& value = values[i];
+    const auto& key_type = key_types[i];
+    if (*value.type() != *key_type.type) {
+      return Status::Invalid("expected batch value ", i, " of type ", *key_type.type,
+                             " but got ", *value.type());
+    }
+  }
+  return Status::OK();
+}
+
+template <typename Batch>
+enable_if_t<std::is_same<Batch, ExecSpan>::value || std::is_same<Batch, ExecBatch>::value,
+            Status>
+CheckForGetNextSegment(const Batch& batch, int64_t offset,
+                       const std::vector<TypeHolder>& key_types) {
+  return CheckForGetNextSegment(batch.values, batch.length, offset, key_types);
+}
+
+struct BaseGroupingSegmenter : public GroupingSegmenter {
+  explicit BaseGroupingSegmenter(const std::vector<TypeHolder>& key_types)
+      : key_types_(key_types) {}
+
+  const std::vector<TypeHolder>& key_types() const override { return key_types_; }
+
+  std::vector<TypeHolder> key_types_;
+};
+
+GroupingSegment MakeSegment(int64_t batch_length, int64_t offset, int64_t length,
+                            bool extends) {
+  return GroupingSegment{offset, length, offset + length >= batch_length, extends};
+}
+
+int64_t GetMatchLength(const uint8_t* match_bytes, int64_t match_width,
+                       const uint8_t* array_bytes, int64_t offset, int64_t length) {
+  int64_t cursor, byte_cursor;
+  for (cursor = offset, byte_cursor = match_width * cursor; cursor < length;
+       cursor++, byte_cursor += match_width) {
+    if (memcmp(match_bytes, array_bytes + byte_cursor,
+               static_cast<size_t>(match_width)) != 0) {
+      break;
+    }
+  }
+  return std::min(cursor, length - offset);
+}
+
+using ExtendFunc = std::function<bool(const void*)>;
+constexpr bool kDefaultExtends = true;
+constexpr bool kEmptyExtends = true;
+
+struct NoKeysGroupingSegmenter : public BaseGroupingSegmenter {
+  static std::unique_ptr<GroupingSegmenter> Make() {
+    return std::make_unique<NoKeysGroupingSegmenter>();
+  }
+
+  NoKeysGroupingSegmenter() : BaseGroupingSegmenter({}) {}
+
+  Status Reset() override { return Status::OK(); }
+
+  Result<GroupingSegment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {}));

Review Comment:
   Because the current `NoKeysGroupingSegmenter` class has an empty set of keys. We could also pass `key_types_` here, as it is empty.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rtpsw commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "rtpsw (via GitHub)" <gi...@apache.org>.
rtpsw commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1124973802


##########
cpp/src/arrow/compute/exec/aggregate_node.cc:
##########
@@ -440,12 +613,29 @@ class GroupByNode : public ExecNode, public TracedNode {
       int key_field_id = key_field_ids[i];
       output_fields[base + i] = input_schema->field(key_field_id);
     }
+    base += keys.size();
+    for (size_t i = 0; i < segment_keys.size(); ++i) {
+      int segment_key_field_id = segment_key_field_ids[i];
+      output_fields[base + i] = input_schema->field(segment_key_field_id);
+    }
 
     return input->plan()->EmplaceNode<GroupByNode>(
         input, schema(std::move(output_fields)), std::move(key_field_ids),
+        std::move(segment_key_field_ids), std::move(segmenter), std::move(agg_src_types),
         std::move(agg_src_fieldsets), std::move(aggs), std::move(agg_kernels));
   }
 
+  Status ResetAggregates() {

Review Comment:
   One is in `ScalarAggregateNode` and the second is in `GroupByNode`.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rtpsw commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "rtpsw (via GitHub)" <gi...@apache.org>.
rtpsw commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1125561717


##########
cpp/src/arrow/compute/exec/aggregate_node.cc:
##########
@@ -16,9 +16,11 @@
 // under the License.
 
 #include <mutex>
+#include <shared_mutex>

Review Comment:
   Done.



##########
cpp/src/arrow/compute/exec/aggregate_node.cc:
##########
@@ -16,9 +16,11 @@
 // under the License.
 
 #include <mutex>
+#include <shared_mutex>

Review Comment:
   Done.



##########
cpp/src/arrow/compute/exec/aggregate_node.cc:
##########
@@ -326,46 +438,77 @@ class ScalarAggregateNode : public ExecNode, public TracedNode {
   }
 
  private:
-  Status Finish() {
-    auto scope = TraceFinish();
+  Status ResetAggregates() {
+    auto exec_ctx = plan()->query_context()->exec_context();
+    for (size_t i = 0; i < kernels_.size(); ++i) {
+      const std::vector<TypeHolder>& in_types = in_typesets_[i];
+      states_[i].resize(plan()->query_context()->max_concurrency());
+      KernelContext kernel_ctx{exec_ctx};
+      RETURN_NOT_OK(Kernel::InitAll(
+          &kernel_ctx, KernelInitArgs{kernels_[i], in_types, aggs_[i].options.get()},
+          &states_[i]));
+    }
+    return Status::OK();
+  }
+
+  Status OutputResult(bool is_last = false) {
     ExecBatch batch{{}, 1};
-    batch.values.resize(kernels_.size());
+    batch.values.resize(kernels_.size() + segment_field_ids_.size());
 
     for (size_t i = 0; i < kernels_.size(); ++i) {
       util::tracing::Span span;
       START_COMPUTE_SPAN(span, aggs_[i].function,
                          {{"function.name", aggs_[i].function},
                           {"function.options",
                            aggs_[i].options ? aggs_[i].options->ToString() : "<NULLPTR>"},
-                          {"function.kind", std::string(kind_name()) + "::Finalize"}});
+                          {"function.kind", std::string(kind_name()) + "::Output"}});

Review Comment:
   Done.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] icexelloss commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "icexelloss (via GitHub)" <gi...@apache.org>.
icexelloss commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1128197847


##########
cpp/src/arrow/compute/kernels/hash_aggregate_test.cc:
##########
@@ -1866,7 +2248,7 @@ TEST(GroupBy, MinMaxDecimal) {
   }
 }
 
-TEST(GroupBy, MinMaxBinary) {
+TEST_P(GroupBy, MinMaxFixedSizeBinary) {

Review Comment:
   This rename is wrong? This should still be "MinMaxBinary"



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rtpsw commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "rtpsw (via GitHub)" <gi...@apache.org>.
rtpsw commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1129102213


##########
cpp/src/arrow/compute/row/grouper.cc:
##########
@@ -39,12 +43,336 @@
 namespace arrow {
 
 using internal::checked_cast;
+using internal::PrimitiveScalarBase;
 
 namespace compute {
 
 namespace {
 
-struct GrouperImpl : Grouper {
+constexpr uint32_t kNoGroupId = std::numeric_limits<uint32_t>::max();
+
+using group_id_t = std::remove_const<decltype(kNoGroupId)>::type;
+using GroupIdType = CTypeTraits<group_id_t>::ArrowType;
+auto g_group_id_type = std::make_shared<GroupIdType>();
+
+inline const uint8_t* GetValuesAsBytes(const ArraySpan& data, int64_t offset = 0) {
+  DCHECK_GT(data.type->byte_width(), 0);
+  int64_t absolute_byte_offset = (data.offset + offset) * data.type->byte_width();
+  return data.GetValues<uint8_t>(1, absolute_byte_offset);
+}
+
+template <typename Value>
+Status CheckForGetNextSegment(const std::vector<Value>& values, int64_t length,
+                              int64_t offset, const std::vector<TypeHolder>& key_types) {
+  if (offset < 0 || offset > length) {

Review Comment:
   Changing this to a DCHECK breaks the error-checks in `TEST(RowSegmenter, Basics)`, so I won't make a change here for now. If you still want to change, you should say whether to defer on this, or to remove the error-checks, or to make internal APIs for the error-checks, or something else.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rtpsw commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "rtpsw (via GitHub)" <gi...@apache.org>.
rtpsw commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1127052647


##########
cpp/src/arrow/compute/row/grouper.cc:
##########
@@ -39,12 +43,330 @@
 namespace arrow {
 
 using internal::checked_cast;
+using internal::PrimitiveScalarBase;
 
 namespace compute {
 
 namespace {
 
-struct GrouperImpl : Grouper {
+constexpr uint32_t kNoGroupId = std::numeric_limits<uint32_t>::max();
+
+using group_id_t = std::remove_const<decltype(kNoGroupId)>::type;
+using GroupIdType = CTypeTraits<group_id_t>::ArrowType;
+auto g_group_id_type = std::make_shared<GroupIdType>();
+
+inline const uint8_t* GetValuesAsBytes(const ArraySpan& data, int64_t offset = 0) {
+  DCHECK_GT(data.type->byte_width(), 0);
+  int64_t absolute_byte_offset = (data.offset + offset) * data.type->byte_width();
+  return data.GetValues<uint8_t>(1, absolute_byte_offset);
+}
+
+template <typename Value>
+Status CheckForGetNextSegment(const std::vector<Value>& values, int64_t length,
+                              int64_t offset, const std::vector<TypeHolder>& key_types) {
+  if (offset < 0 || offset > length) {
+    return Status::Invalid("invalid grouping segmenter offset: ", offset);
+  }
+  if (values.size() != key_types.size()) {
+    return Status::Invalid("expected batch size ", key_types.size(), " but got ",
+                           values.size());
+  }
+  for (size_t i = 0; i < key_types.size(); i++) {
+    const auto& value = values[i];
+    const auto& key_type = key_types[i];
+    if (*value.type() != *key_type.type) {
+      return Status::Invalid("expected batch value ", i, " of type ", *key_type.type,
+                             " but got ", *value.type());
+    }
+  }
+  return Status::OK();
+}
+
+template <typename Batch>
+enable_if_t<std::is_same<Batch, ExecSpan>::value || std::is_same<Batch, ExecBatch>::value,
+            Status>
+CheckForGetNextSegment(const Batch& batch, int64_t offset,
+                       const std::vector<TypeHolder>& key_types) {
+  return CheckForGetNextSegment(batch.values, batch.length, offset, key_types);
+}
+
+struct BaseRowSegmenter : public RowSegmenter {
+  explicit BaseRowSegmenter(const std::vector<TypeHolder>& key_types)
+      : key_types_(key_types) {}
+
+  const std::vector<TypeHolder>& key_types() const override { return key_types_; }
+
+  std::vector<TypeHolder> key_types_;
+};
+
+Segment MakeSegment(int64_t batch_length, int64_t offset, int64_t length, bool extends) {
+  return Segment{offset, length, offset + length >= batch_length, extends};
+}
+
+int64_t GetMatchLength(const uint8_t* match_bytes, int64_t match_width,
+                       const uint8_t* array_bytes, int64_t offset, int64_t length) {
+  int64_t cursor, byte_cursor;
+  for (cursor = offset, byte_cursor = match_width * cursor; cursor < length;
+       cursor++, byte_cursor += match_width) {
+    if (memcmp(match_bytes, array_bytes + byte_cursor,
+               static_cast<size_t>(match_width)) != 0) {
+      break;
+    }
+  }
+  return std::min(cursor, length) - offset;
+}
+
+using ExtendFunc = std::function<bool(const void*)>;
+constexpr bool kDefaultExtends = true;
+constexpr bool kEmptyExtends = true;
+
+struct NoKeysSegmenter : public BaseRowSegmenter {
+  static std::unique_ptr<RowSegmenter> Make() {
+    return std::make_unique<NoKeysSegmenter>();
+  }
+
+  NoKeysSegmenter() : BaseRowSegmenter({}) {}
+
+  Status Reset() override { return Status::OK(); }
+
+  Result<Segment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {}));
+    return MakeSegment(batch.length, offset, batch.length - offset, kDefaultExtends);
+  }
+};
+
+struct SimpleKeySegmenter : public BaseRowSegmenter {
+  static Result<std::unique_ptr<RowSegmenter>> Make(TypeHolder key_type) {
+    return std::make_unique<SimpleKeySegmenter>(key_type);
+  }
+
+  explicit SimpleKeySegmenter(TypeHolder key_type)
+      : BaseRowSegmenter({key_type}), key_type_(key_types_[0]), save_key_data_() {}
+
+  Status CheckType(const DataType& type) {
+    if (!is_fixed_width(type)) {
+      return Status::Invalid("SimpleKeySegmenter does not support type ", type);
+    }
+    return Status::OK();
+  }
+
+  Status Reset() override {
+    save_key_data_.resize(0);
+    return Status::OK();
+  }
+
+  // Checks whether the given grouping data extends the current segment, i.e., is equal to
+  // previously seen grouping data, which is updated with each invocation.
+  bool Extend(const void* data) {
+    size_t byte_width = static_cast<size_t>(key_type_.type->byte_width());
+    bool extends = save_key_data_.size() != byte_width
+                       ? kDefaultExtends
+                       : 0 == memcmp(save_key_data_.data(), data, byte_width);
+    save_key_data_.resize(byte_width);
+    memcpy(save_key_data_.data(), data, byte_width);
+    return extends;
+  }
+
+  Result<Segment> GetNextSegment(const Scalar& scalar, int64_t offset, int64_t length) {
+    ARROW_RETURN_NOT_OK(CheckType(*scalar.type));
+    if (!scalar.is_valid) {
+      return Status::Invalid("segmenting an invalid scalar");
+    }
+    auto data = checked_cast<const PrimitiveScalarBase&>(scalar).data();
+    bool extends = length > 0 ? Extend(data) : kEmptyExtends;
+    return MakeSegment(length, offset, length, extends);
+  }
+
+  Result<Segment> GetNextSegment(const DataType& array_type, const uint8_t* array_bytes,
+                                 int64_t offset, int64_t length) {
+    RETURN_NOT_OK(CheckType(array_type));
+    int64_t byte_width = array_type.byte_width();
+    int64_t match_length = GetMatchLength(array_bytes + offset * byte_width, byte_width,
+                                          array_bytes, offset, length);
+    bool extends = length > 0 ? Extend(array_bytes + offset * byte_width) : kEmptyExtends;
+    return MakeSegment(length, offset, match_length, extends);
+  }
+
+  Result<Segment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {key_type_}));
+    if (offset == batch.length) {
+      return MakeSegment(batch.length, offset, 0, kEmptyExtends);
+    }
+    const auto& value = batch.values[0];
+    if (value.is_scalar()) {
+      return GetNextSegment(*value.scalar, offset, batch.length);
+    }
+    ARROW_DCHECK(value.is_array());
+    const auto& array = value.array;
+    if (array.GetNullCount() > 0) {
+      return Status::NotImplemented("segmenting a nullable array");
+    }
+    return GetNextSegment(*array.type, GetValuesAsBytes(array), offset, batch.length);
+  }
+
+ private:
+  TypeHolder key_type_;
+  std::vector<uint8_t> save_key_data_;
+};
+
+struct AnyKeysSegmenter : public BaseRowSegmenter {
+  static Result<std::unique_ptr<RowSegmenter>> Make(
+      const std::vector<TypeHolder>& key_types, ExecContext* ctx) {
+    ARROW_RETURN_NOT_OK(Grouper::Make(key_types, ctx));  // check types
+    return std::make_unique<AnyKeysSegmenter>(key_types, ctx);
+  }
+
+  AnyKeysSegmenter(const std::vector<TypeHolder>& key_types, ExecContext* ctx)
+      : BaseRowSegmenter(key_types),
+        ctx_(ctx),
+        grouper_(nullptr),
+        save_group_id_(kNoGroupId) {}
+
+  Status Reset() override {
+    grouper_ = nullptr;
+    save_group_id_ = kNoGroupId;
+    return Status::OK();
+  }
+
+  bool Extend(const void* data) {
+    auto group_id = *static_cast<const group_id_t*>(data);
+    bool extends =
+        save_group_id_ == kNoGroupId ? kDefaultExtends : save_group_id_ == group_id;
+    save_group_id_ = group_id;
+    return extends;
+  }
+
+  // Runs the grouper on a single row.  This is used to determine the group id of the
+  // first row of a new segment to see if it extends the previous segment.
+  template <typename Batch>
+  Result<group_id_t> MapGroupIdAt(const Batch& batch, int64_t offset) {
+    if (!grouper_) return kNoGroupId;
+    ARROW_ASSIGN_OR_RAISE(auto datum, grouper_->Consume(batch, offset,
+                                                        /*length=*/1));
+    if (!datum.is_array()) {
+      return Status::Invalid("accessing unsupported datum kind ", datum.kind());
+    }
+    const std::shared_ptr<ArrayData>& data = datum.array();
+    ARROW_DCHECK(data->GetNullCount() == 0);
+    DCHECK_EQ(data->type->id(), GroupIdType::type_id);
+    DCHECK_EQ(1, data->length);
+    const group_id_t* values = data->GetValues<group_id_t>(1);
+    return values[0];
+  }
+
+  Result<Segment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, key_types_));
+    if (offset == batch.length) {
+      return MakeSegment(batch.length, offset, 0, kEmptyExtends);
+    }
+    // ARROW-18311: make Grouper support Reset()
+    // so it can be cached instead of recreated below
+    //
+    // the group id must be computed prior to resetting the grouper, since it is compared
+    // to save_group_id_, and after resetting the grouper produces incomparable group ids
+    ARROW_ASSIGN_OR_RAISE(auto group_id, MapGroupIdAt(batch, offset));
+    ExtendFunc bound_extend = [this, group_id](const void* data) {
+      bool extends = Extend(&group_id);
+      save_group_id_ = *static_cast<const group_id_t*>(data);
+      return extends;
+    };
+    ARROW_ASSIGN_OR_RAISE(grouper_, Grouper::Make(key_types_, ctx_));  // TODO: reset it

Review Comment:
   We don't. As in the doc, the grouper just needs to be reset, and because resetting is not supported, recreating it is another (less efficient) way of doing it. As for the reason to do this, within each segment determined by `segment_keys` there is further grouping determined by `keys`. As I recall, this was discussed elsewhere. The discussions in this PR are by now numerous enough that it's hard to keep track.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] icexelloss commented on pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "icexelloss (via GitHub)" <gi...@apache.org>.
icexelloss commented on PR #34311:
URL: https://github.com/apache/arrow/pull/34311#issuecomment-1457023176

   @rtpsw I reviewed the grouper files. I still have the test file left to review.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rtpsw commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "rtpsw (via GitHub)" <gi...@apache.org>.
rtpsw commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1127092435


##########
cpp/src/arrow/compute/row/grouper.cc:
##########
@@ -39,12 +43,330 @@
 namespace arrow {
 
 using internal::checked_cast;
+using internal::PrimitiveScalarBase;
 
 namespace compute {
 
 namespace {
 
-struct GrouperImpl : Grouper {
+constexpr uint32_t kNoGroupId = std::numeric_limits<uint32_t>::max();
+
+using group_id_t = std::remove_const<decltype(kNoGroupId)>::type;
+using GroupIdType = CTypeTraits<group_id_t>::ArrowType;
+auto g_group_id_type = std::make_shared<GroupIdType>();
+
+inline const uint8_t* GetValuesAsBytes(const ArraySpan& data, int64_t offset = 0) {
+  DCHECK_GT(data.type->byte_width(), 0);
+  int64_t absolute_byte_offset = (data.offset + offset) * data.type->byte_width();
+  return data.GetValues<uint8_t>(1, absolute_byte_offset);
+}
+
+template <typename Value>
+Status CheckForGetNextSegment(const std::vector<Value>& values, int64_t length,
+                              int64_t offset, const std::vector<TypeHolder>& key_types) {
+  if (offset < 0 || offset > length) {
+    return Status::Invalid("invalid grouping segmenter offset: ", offset);
+  }
+  if (values.size() != key_types.size()) {
+    return Status::Invalid("expected batch size ", key_types.size(), " but got ",
+                           values.size());
+  }
+  for (size_t i = 0; i < key_types.size(); i++) {
+    const auto& value = values[i];
+    const auto& key_type = key_types[i];
+    if (*value.type() != *key_type.type) {
+      return Status::Invalid("expected batch value ", i, " of type ", *key_type.type,
+                             " but got ", *value.type());
+    }
+  }
+  return Status::OK();
+}
+
+template <typename Batch>
+enable_if_t<std::is_same<Batch, ExecSpan>::value || std::is_same<Batch, ExecBatch>::value,
+            Status>
+CheckForGetNextSegment(const Batch& batch, int64_t offset,
+                       const std::vector<TypeHolder>& key_types) {
+  return CheckForGetNextSegment(batch.values, batch.length, offset, key_types);
+}
+
+struct BaseRowSegmenter : public RowSegmenter {
+  explicit BaseRowSegmenter(const std::vector<TypeHolder>& key_types)
+      : key_types_(key_types) {}
+
+  const std::vector<TypeHolder>& key_types() const override { return key_types_; }
+
+  std::vector<TypeHolder> key_types_;
+};
+
+Segment MakeSegment(int64_t batch_length, int64_t offset, int64_t length, bool extends) {
+  return Segment{offset, length, offset + length >= batch_length, extends};
+}
+
+int64_t GetMatchLength(const uint8_t* match_bytes, int64_t match_width,
+                       const uint8_t* array_bytes, int64_t offset, int64_t length) {
+  int64_t cursor, byte_cursor;
+  for (cursor = offset, byte_cursor = match_width * cursor; cursor < length;
+       cursor++, byte_cursor += match_width) {
+    if (memcmp(match_bytes, array_bytes + byte_cursor,
+               static_cast<size_t>(match_width)) != 0) {
+      break;
+    }
+  }
+  return std::min(cursor, length) - offset;
+}
+
+using ExtendFunc = std::function<bool(const void*)>;
+constexpr bool kDefaultExtends = true;
+constexpr bool kEmptyExtends = true;
+
+struct NoKeysSegmenter : public BaseRowSegmenter {
+  static std::unique_ptr<RowSegmenter> Make() {
+    return std::make_unique<NoKeysSegmenter>();
+  }
+
+  NoKeysSegmenter() : BaseRowSegmenter({}) {}
+
+  Status Reset() override { return Status::OK(); }
+
+  Result<Segment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {}));
+    return MakeSegment(batch.length, offset, batch.length - offset, kDefaultExtends);
+  }
+};
+
+struct SimpleKeySegmenter : public BaseRowSegmenter {

Review Comment:
   > Can you add test to check SimpleKeySegmenter gives the same result as AnyKeysSegmenter with one key?
   
   Will do.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] icexelloss commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "icexelloss (via GitHub)" <gi...@apache.org>.
icexelloss commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1127140704


##########
cpp/src/arrow/compute/row/grouper.cc:
##########
@@ -39,12 +43,330 @@
 namespace arrow {
 
 using internal::checked_cast;
+using internal::PrimitiveScalarBase;
 
 namespace compute {
 
 namespace {
 
-struct GrouperImpl : Grouper {
+constexpr uint32_t kNoGroupId = std::numeric_limits<uint32_t>::max();
+
+using group_id_t = std::remove_const<decltype(kNoGroupId)>::type;
+using GroupIdType = CTypeTraits<group_id_t>::ArrowType;
+auto g_group_id_type = std::make_shared<GroupIdType>();
+
+inline const uint8_t* GetValuesAsBytes(const ArraySpan& data, int64_t offset = 0) {
+  DCHECK_GT(data.type->byte_width(), 0);
+  int64_t absolute_byte_offset = (data.offset + offset) * data.type->byte_width();
+  return data.GetValues<uint8_t>(1, absolute_byte_offset);
+}
+
+template <typename Value>
+Status CheckForGetNextSegment(const std::vector<Value>& values, int64_t length,
+                              int64_t offset, const std::vector<TypeHolder>& key_types) {
+  if (offset < 0 || offset > length) {
+    return Status::Invalid("invalid grouping segmenter offset: ", offset);
+  }
+  if (values.size() != key_types.size()) {
+    return Status::Invalid("expected batch size ", key_types.size(), " but got ",
+                           values.size());
+  }
+  for (size_t i = 0; i < key_types.size(); i++) {
+    const auto& value = values[i];
+    const auto& key_type = key_types[i];
+    if (*value.type() != *key_type.type) {
+      return Status::Invalid("expected batch value ", i, " of type ", *key_type.type,
+                             " but got ", *value.type());
+    }
+  }
+  return Status::OK();
+}
+
+template <typename Batch>
+enable_if_t<std::is_same<Batch, ExecSpan>::value || std::is_same<Batch, ExecBatch>::value,
+            Status>
+CheckForGetNextSegment(const Batch& batch, int64_t offset,
+                       const std::vector<TypeHolder>& key_types) {
+  return CheckForGetNextSegment(batch.values, batch.length, offset, key_types);
+}
+
+struct BaseRowSegmenter : public RowSegmenter {
+  explicit BaseRowSegmenter(const std::vector<TypeHolder>& key_types)
+      : key_types_(key_types) {}
+
+  const std::vector<TypeHolder>& key_types() const override { return key_types_; }
+
+  std::vector<TypeHolder> key_types_;
+};
+
+Segment MakeSegment(int64_t batch_length, int64_t offset, int64_t length, bool extends) {
+  return Segment{offset, length, offset + length >= batch_length, extends};
+}
+
+int64_t GetMatchLength(const uint8_t* match_bytes, int64_t match_width,
+                       const uint8_t* array_bytes, int64_t offset, int64_t length) {
+  int64_t cursor, byte_cursor;
+  for (cursor = offset, byte_cursor = match_width * cursor; cursor < length;
+       cursor++, byte_cursor += match_width) {
+    if (memcmp(match_bytes, array_bytes + byte_cursor,
+               static_cast<size_t>(match_width)) != 0) {
+      break;
+    }
+  }
+  return std::min(cursor, length) - offset;
+}
+
+using ExtendFunc = std::function<bool(const void*)>;
+constexpr bool kDefaultExtends = true;
+constexpr bool kEmptyExtends = true;
+
+struct NoKeysSegmenter : public BaseRowSegmenter {
+  static std::unique_ptr<RowSegmenter> Make() {
+    return std::make_unique<NoKeysSegmenter>();
+  }
+
+  NoKeysSegmenter() : BaseRowSegmenter({}) {}
+
+  Status Reset() override { return Status::OK(); }
+
+  Result<Segment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {}));
+    return MakeSegment(batch.length, offset, batch.length - offset, kDefaultExtends);
+  }
+};
+
+struct SimpleKeySegmenter : public BaseRowSegmenter {
+  static Result<std::unique_ptr<RowSegmenter>> Make(TypeHolder key_type) {
+    return std::make_unique<SimpleKeySegmenter>(key_type);
+  }
+
+  explicit SimpleKeySegmenter(TypeHolder key_type)
+      : BaseRowSegmenter({key_type}), key_type_(key_types_[0]), save_key_data_() {}
+
+  Status CheckType(const DataType& type) {
+    if (!is_fixed_width(type)) {
+      return Status::Invalid("SimpleKeySegmenter does not support type ", type);
+    }
+    return Status::OK();
+  }
+
+  Status Reset() override {
+    save_key_data_.resize(0);
+    return Status::OK();
+  }
+
+  // Checks whether the given grouping data extends the current segment, i.e., is equal to
+  // previously seen grouping data, which is updated with each invocation.
+  bool Extend(const void* data) {
+    size_t byte_width = static_cast<size_t>(key_type_.type->byte_width());
+    bool extends = save_key_data_.size() != byte_width
+                       ? kDefaultExtends
+                       : 0 == memcmp(save_key_data_.data(), data, byte_width);
+    save_key_data_.resize(byte_width);
+    memcpy(save_key_data_.data(), data, byte_width);
+    return extends;
+  }
+
+  Result<Segment> GetNextSegment(const Scalar& scalar, int64_t offset, int64_t length) {
+    ARROW_RETURN_NOT_OK(CheckType(*scalar.type));
+    if (!scalar.is_valid) {
+      return Status::Invalid("segmenting an invalid scalar");
+    }
+    auto data = checked_cast<const PrimitiveScalarBase&>(scalar).data();
+    bool extends = length > 0 ? Extend(data) : kEmptyExtends;
+    return MakeSegment(length, offset, length, extends);
+  }
+
+  Result<Segment> GetNextSegment(const DataType& array_type, const uint8_t* array_bytes,
+                                 int64_t offset, int64_t length) {
+    RETURN_NOT_OK(CheckType(array_type));
+    int64_t byte_width = array_type.byte_width();
+    int64_t match_length = GetMatchLength(array_bytes + offset * byte_width, byte_width,
+                                          array_bytes, offset, length);
+    bool extends = length > 0 ? Extend(array_bytes + offset * byte_width) : kEmptyExtends;
+    return MakeSegment(length, offset, match_length, extends);
+  }
+
+  Result<Segment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {key_type_}));
+    if (offset == batch.length) {
+      return MakeSegment(batch.length, offset, 0, kEmptyExtends);
+    }
+    const auto& value = batch.values[0];
+    if (value.is_scalar()) {
+      return GetNextSegment(*value.scalar, offset, batch.length);
+    }
+    ARROW_DCHECK(value.is_array());
+    const auto& array = value.array;
+    if (array.GetNullCount() > 0) {
+      return Status::NotImplemented("segmenting a nullable array");
+    }
+    return GetNextSegment(*array.type, GetValuesAsBytes(array), offset, batch.length);
+  }
+
+ private:
+  TypeHolder key_type_;
+  std::vector<uint8_t> save_key_data_;
+};
+
+struct AnyKeysSegmenter : public BaseRowSegmenter {
+  static Result<std::unique_ptr<RowSegmenter>> Make(
+      const std::vector<TypeHolder>& key_types, ExecContext* ctx) {
+    ARROW_RETURN_NOT_OK(Grouper::Make(key_types, ctx));  // check types
+    return std::make_unique<AnyKeysSegmenter>(key_types, ctx);
+  }
+
+  AnyKeysSegmenter(const std::vector<TypeHolder>& key_types, ExecContext* ctx)
+      : BaseRowSegmenter(key_types),
+        ctx_(ctx),
+        grouper_(nullptr),
+        save_group_id_(kNoGroupId) {}
+
+  Status Reset() override {
+    grouper_ = nullptr;
+    save_group_id_ = kNoGroupId;
+    return Status::OK();
+  }
+
+  bool Extend(const void* data) {
+    auto group_id = *static_cast<const group_id_t*>(data);
+    bool extends =
+        save_group_id_ == kNoGroupId ? kDefaultExtends : save_group_id_ == group_id;
+    save_group_id_ = group_id;
+    return extends;
+  }
+
+  // Runs the grouper on a single row.  This is used to determine the group id of the
+  // first row of a new segment to see if it extends the previous segment.
+  template <typename Batch>
+  Result<group_id_t> MapGroupIdAt(const Batch& batch, int64_t offset) {
+    if (!grouper_) return kNoGroupId;
+    ARROW_ASSIGN_OR_RAISE(auto datum, grouper_->Consume(batch, offset,
+                                                        /*length=*/1));
+    if (!datum.is_array()) {
+      return Status::Invalid("accessing unsupported datum kind ", datum.kind());
+    }
+    const std::shared_ptr<ArrayData>& data = datum.array();
+    ARROW_DCHECK(data->GetNullCount() == 0);
+    DCHECK_EQ(data->type->id(), GroupIdType::type_id);
+    DCHECK_EQ(1, data->length);
+    const group_id_t* values = data->GetValues<group_id_t>(1);
+    return values[0];
+  }
+
+  Result<Segment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, key_types_));
+    if (offset == batch.length) {
+      return MakeSegment(batch.length, offset, 0, kEmptyExtends);
+    }
+    // ARROW-18311: make Grouper support Reset()
+    // so it can be cached instead of recreated below
+    //
+    // the group id must be computed prior to resetting the grouper, since it is compared
+    // to save_group_id_, and after resetting the grouper produces incomparable group ids
+    ARROW_ASSIGN_OR_RAISE(auto group_id, MapGroupIdAt(batch, offset));
+    ExtendFunc bound_extend = [this, group_id](const void* data) {
+      bool extends = Extend(&group_id);
+      save_group_id_ = *static_cast<const group_id_t*>(data);
+      return extends;
+    };
+    ARROW_ASSIGN_OR_RAISE(grouper_, Grouper::Make(key_types_, ctx_));  // TODO: reset it

Review Comment:
   >  As for the reason to do this, within each segment determined by segment_keys there is further grouping determined by keys
   
   Sorry I still don't fully get - why do we need a new grouper every time?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] icexelloss commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "icexelloss (via GitHub)" <gi...@apache.org>.
icexelloss commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1126947659


##########
cpp/src/arrow/compute/kernels/hash_aggregate_test.cc:
##########
@@ -174,81 +242,117 @@ Result<Datum> RunGroupBy(const BatchesWithSchema& input,
   ARROW_ASSIGN_OR_RAISE(std::vector<ExecBatch> output_batches,
                         start_and_collect.MoveResult());
 
-  ArrayVector out_arrays(aggregates.size() + key_names.size());
   const auto& output_schema = plan->nodes()[0]->output()->output_schema();
+  if (!segmented) {
+    return MakeGroupByOutput(output_batches, output_schema, aggregates.size(),
+                             key_names.size(), naive);
+  }
+
+  std::vector<ArrayVector> out_arrays(aggregates.size() + key_names.size() +
+                                      segment_key_names.size());
   for (size_t i = 0; i < out_arrays.size(); ++i) {
     std::vector<std::shared_ptr<Array>> arrays(output_batches.size());
     for (size_t j = 0; j < output_batches.size(); ++j) {
-      arrays[j] = output_batches[j].values[i].make_array();
+      auto& value = output_batches[j].values[i];
+      if (value.is_scalar()) {
+        ARROW_ASSIGN_OR_RAISE(
+            arrays[j], MakeArrayFromScalar(*value.scalar(), output_batches[j].length));
+      } else if (value.is_array()) {
+        arrays[j] = value.make_array();
+      } else {
+        return Status::Invalid("GroupByUsingExecPlan unsupported value kind ",
+                               ToString(value.kind()));
+      }
     }
     if (arrays.empty()) {
+      arrays.resize(1);
       ARROW_ASSIGN_OR_RAISE(
-          out_arrays[i],
-          MakeArrayOfNull(output_schema->field(static_cast<int>(i))->type(),
-                          /*length=*/0));
-    } else {
-      ARROW_ASSIGN_OR_RAISE(out_arrays[i], Concatenate(arrays));
+          arrays[0], MakeArrayOfNull(output_schema->field(static_cast<int>(i))->type(),
+                                     /*length=*/0));
     }
+    out_arrays[i] = {std::move(arrays)};
   }
 
-  // The exec plan may reorder the output rows.  The tests are all setup to expect ouptut
-  // in ascending order of keys.  So we need to sort the result by the key columns.  To do
-  // that we create a table using the key columns, calculate the sort indices from that
-  // table (sorting on all fields) and then use those indices to calculate our result.
-  std::vector<std::shared_ptr<Field>> key_fields;
-  std::vector<std::shared_ptr<Array>> key_columns;
-  std::vector<SortKey> sort_keys;
-  for (std::size_t i = 0; i < key_names.size(); i++) {
-    const std::shared_ptr<Array>& arr = out_arrays[i + aggregates.size()];
-    if (arr->type_id() == Type::DICTIONARY) {
-      // Can't sort dictionary columns so need to decode
-      auto dict_arr = checked_pointer_cast<DictionaryArray>(arr);
-      ARROW_ASSIGN_OR_RAISE(auto decoded_arr,
-                            Take(*dict_arr->dictionary(), *dict_arr->indices()));
-      key_columns.push_back(decoded_arr);
-      key_fields.push_back(
-          field("name_does_not_matter", dict_arr->dict_type()->value_type()));
-    } else {
-      key_columns.push_back(arr);
-      key_fields.push_back(field("name_does_not_matter", arr->type()));
+  if (segmented && segment_key_names.size() > 0) {
+    ArrayVector struct_arrays;
+    struct_arrays.reserve(output_batches.size());
+    for (size_t j = 0; j < output_batches.size(); ++j) {
+      ArrayVector struct_fields;
+      struct_fields.reserve(out_arrays.size());
+      for (auto out_array : out_arrays) {
+        struct_fields.push_back(out_array[j]);
+      }
+      ARROW_ASSIGN_OR_RAISE(auto struct_array,
+                            StructArray::Make(struct_fields, output_schema->fields()));
+      struct_arrays.push_back(struct_array);
     }
-    sort_keys.emplace_back(static_cast<int>(i));
+    return ChunkedArray::Make(struct_arrays);
+  } else {
+    ArrayVector struct_fields(out_arrays.size());
+    for (size_t i = 0; i < out_arrays.size(); ++i) {
+      ARROW_ASSIGN_OR_RAISE(struct_fields[i], Concatenate(out_arrays[i]));
+    }
+    return StructArray::Make(std::move(struct_fields), output_schema->fields());
   }
-  std::shared_ptr<Schema> key_schema = schema(std::move(key_fields));
-  std::shared_ptr<Table> key_table = Table::Make(std::move(key_schema), key_columns);
-  SortOptions sort_options(std::move(sort_keys));
-  ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Array> sort_indices,
-                        SortIndices(key_table, sort_options));
+}
 
-  ARROW_ASSIGN_OR_RAISE(
-      std::shared_ptr<Array> struct_arr,
-      StructArray::Make(std::move(out_arrays), output_schema->fields()));
+Result<Datum> RunGroupBy(const BatchesWithSchema& input,
+                         const std::vector<std::string>& key_names,
+                         const std::vector<std::string>& segment_key_names,
+                         const std::vector<Aggregate>& aggregates, bool use_threads,
+                         bool segmented = false, bool naive = false) {
+  if (segment_key_names.size() > 0) {
+    ARROW_ASSIGN_OR_RAISE(auto thread_pool, arrow::internal::ThreadPool::Make(1));
+    ExecContext seq_ctx(default_memory_pool(), thread_pool.get());
+    return RunGroupBy(input, key_names, segment_key_names, aggregates, &seq_ctx,
+                      use_threads, segmented, naive);
+  } else {
+    return RunGroupBy(input, key_names, segment_key_names, aggregates,
+                      threaded_exec_context(), use_threads, segmented, naive);
+  }
+}
 
-  return Take(struct_arr, sort_indices);
+Result<Datum> RunGroupBy(const BatchesWithSchema& input,
+                         const std::vector<std::string>& key_names,
+                         const std::vector<Aggregate>& aggregates, bool use_threads,
+                         bool segmented = false, bool naive = false) {
+  return RunGroupBy(input, key_names, {}, aggregates, use_threads, segmented);
 }
 
 /// Simpler overload where you can give the columns as datums
 Result<Datum> RunGroupBy(const std::vector<Datum>& arguments,
                          const std::vector<Datum>& keys,
-                         const std::vector<Aggregate>& aggregates,
-                         bool use_threads = false) {
+                         const std::vector<Datum>& segment_keys,
+                         const std::vector<Aggregate>& aggregates, bool use_threads,
+                         bool segmented = false, bool naive = false) {

Review Comment:
   > NaiveGroupBy is [a tester function](https://github.com/apache/arrow/blob/d5b3b4737838774db658d3c488fcd3e72bc13f7e/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc#L75-L136) that computes the expected aggregation result (in a naive/simple/non-optimized way).
   
   I am still quite confused - what is this testing with `naive=true` vs `naive=false`? Are we test a tester function here?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rtpsw commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "rtpsw (via GitHub)" <gi...@apache.org>.
rtpsw commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1126955868


##########
cpp/src/arrow/compute/kernels/hash_aggregate_test.cc:
##########
@@ -174,81 +242,117 @@ Result<Datum> RunGroupBy(const BatchesWithSchema& input,
   ARROW_ASSIGN_OR_RAISE(std::vector<ExecBatch> output_batches,
                         start_and_collect.MoveResult());
 
-  ArrayVector out_arrays(aggregates.size() + key_names.size());
   const auto& output_schema = plan->nodes()[0]->output()->output_schema();
+  if (!segmented) {
+    return MakeGroupByOutput(output_batches, output_schema, aggregates.size(),
+                             key_names.size(), naive);
+  }
+
+  std::vector<ArrayVector> out_arrays(aggregates.size() + key_names.size() +
+                                      segment_key_names.size());
   for (size_t i = 0; i < out_arrays.size(); ++i) {
     std::vector<std::shared_ptr<Array>> arrays(output_batches.size());
     for (size_t j = 0; j < output_batches.size(); ++j) {
-      arrays[j] = output_batches[j].values[i].make_array();
+      auto& value = output_batches[j].values[i];
+      if (value.is_scalar()) {
+        ARROW_ASSIGN_OR_RAISE(
+            arrays[j], MakeArrayFromScalar(*value.scalar(), output_batches[j].length));
+      } else if (value.is_array()) {
+        arrays[j] = value.make_array();
+      } else {
+        return Status::Invalid("GroupByUsingExecPlan unsupported value kind ",
+                               ToString(value.kind()));
+      }
     }
     if (arrays.empty()) {
+      arrays.resize(1);
       ARROW_ASSIGN_OR_RAISE(
-          out_arrays[i],
-          MakeArrayOfNull(output_schema->field(static_cast<int>(i))->type(),
-                          /*length=*/0));
-    } else {
-      ARROW_ASSIGN_OR_RAISE(out_arrays[i], Concatenate(arrays));
+          arrays[0], MakeArrayOfNull(output_schema->field(static_cast<int>(i))->type(),
+                                     /*length=*/0));
     }
+    out_arrays[i] = {std::move(arrays)};
   }
 
-  // The exec plan may reorder the output rows.  The tests are all setup to expect ouptut
-  // in ascending order of keys.  So we need to sort the result by the key columns.  To do
-  // that we create a table using the key columns, calculate the sort indices from that
-  // table (sorting on all fields) and then use those indices to calculate our result.
-  std::vector<std::shared_ptr<Field>> key_fields;
-  std::vector<std::shared_ptr<Array>> key_columns;
-  std::vector<SortKey> sort_keys;
-  for (std::size_t i = 0; i < key_names.size(); i++) {
-    const std::shared_ptr<Array>& arr = out_arrays[i + aggregates.size()];
-    if (arr->type_id() == Type::DICTIONARY) {
-      // Can't sort dictionary columns so need to decode
-      auto dict_arr = checked_pointer_cast<DictionaryArray>(arr);
-      ARROW_ASSIGN_OR_RAISE(auto decoded_arr,
-                            Take(*dict_arr->dictionary(), *dict_arr->indices()));
-      key_columns.push_back(decoded_arr);
-      key_fields.push_back(
-          field("name_does_not_matter", dict_arr->dict_type()->value_type()));
-    } else {
-      key_columns.push_back(arr);
-      key_fields.push_back(field("name_does_not_matter", arr->type()));
+  if (segmented && segment_key_names.size() > 0) {
+    ArrayVector struct_arrays;
+    struct_arrays.reserve(output_batches.size());
+    for (size_t j = 0; j < output_batches.size(); ++j) {
+      ArrayVector struct_fields;
+      struct_fields.reserve(out_arrays.size());
+      for (auto out_array : out_arrays) {
+        struct_fields.push_back(out_array[j]);
+      }
+      ARROW_ASSIGN_OR_RAISE(auto struct_array,
+                            StructArray::Make(struct_fields, output_schema->fields()));
+      struct_arrays.push_back(struct_array);
     }
-    sort_keys.emplace_back(static_cast<int>(i));
+    return ChunkedArray::Make(struct_arrays);
+  } else {
+    ArrayVector struct_fields(out_arrays.size());
+    for (size_t i = 0; i < out_arrays.size(); ++i) {
+      ARROW_ASSIGN_OR_RAISE(struct_fields[i], Concatenate(out_arrays[i]));
+    }
+    return StructArray::Make(std::move(struct_fields), output_schema->fields());
   }
-  std::shared_ptr<Schema> key_schema = schema(std::move(key_fields));
-  std::shared_ptr<Table> key_table = Table::Make(std::move(key_schema), key_columns);
-  SortOptions sort_options(std::move(sort_keys));
-  ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Array> sort_indices,
-                        SortIndices(key_table, sort_options));
+}
 
-  ARROW_ASSIGN_OR_RAISE(
-      std::shared_ptr<Array> struct_arr,
-      StructArray::Make(std::move(out_arrays), output_schema->fields()));
+Result<Datum> RunGroupBy(const BatchesWithSchema& input,
+                         const std::vector<std::string>& key_names,
+                         const std::vector<std::string>& segment_key_names,
+                         const std::vector<Aggregate>& aggregates, bool use_threads,
+                         bool segmented = false, bool naive = false) {
+  if (segment_key_names.size() > 0) {
+    ARROW_ASSIGN_OR_RAISE(auto thread_pool, arrow::internal::ThreadPool::Make(1));
+    ExecContext seq_ctx(default_memory_pool(), thread_pool.get());
+    return RunGroupBy(input, key_names, segment_key_names, aggregates, &seq_ctx,
+                      use_threads, segmented, naive);
+  } else {
+    return RunGroupBy(input, key_names, segment_key_names, aggregates,
+                      threaded_exec_context(), use_threads, segmented, naive);
+  }
+}
 
-  return Take(struct_arr, sort_indices);
+Result<Datum> RunGroupBy(const BatchesWithSchema& input,
+                         const std::vector<std::string>& key_names,
+                         const std::vector<Aggregate>& aggregates, bool use_threads,
+                         bool segmented = false, bool naive = false) {
+  return RunGroupBy(input, key_names, {}, aggregates, use_threads, segmented);
 }
 
 /// Simpler overload where you can give the columns as datums
 Result<Datum> RunGroupBy(const std::vector<Datum>& arguments,
                          const std::vector<Datum>& keys,
-                         const std::vector<Aggregate>& aggregates,
-                         bool use_threads = false) {
+                         const std::vector<Datum>& segment_keys,
+                         const std::vector<Aggregate>& aggregates, bool use_threads,
+                         bool segmented = false, bool naive = false) {

Review Comment:
   > This will never happen in real scenario, right?
   
   The tester is trying to catch possible future errors, not just current ones. This scenario doesn't happen with the current implementation of the main code, but it could happen in a future implementation. I'll add documentation of the flag.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] icexelloss commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "icexelloss (via GitHub)" <gi...@apache.org>.
icexelloss commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1126951325


##########
cpp/src/arrow/compute/kernels/hash_aggregate_test.cc:
##########
@@ -301,53 +426,184 @@ Result<Datum> GroupByTest(const std::vector<Datum>& arguments,
         {t_agg.function, t_agg.options, "agg_" + ToChars(idx), t_agg.function});
     idx = idx + 1;
   }
-  return RunGroupBy(arguments, keys, internal_aggregates, use_threads);
+  return group_by(arguments, keys, segment_keys, internal_aggregates, use_threads,
+                  /*naive=*/false);
 }
 
-}  // namespace
+Result<Datum> GroupByTest(GroupByFunction group_by, const std::vector<Datum>& arguments,
+                          const std::vector<Datum>& keys,
+                          const std::vector<TestAggregate>& aggregates,
+                          bool use_threads) {
+  return GroupByTest(group_by, arguments, keys, {}, aggregates, use_threads);
+}
 
-TEST(Grouper, SupportedKeys) {
-  ASSERT_OK(Grouper::Make({boolean()}));
+template <typename GroupClass>

Review Comment:
   I think this templating probably causes more confusion/complexity than it is worth. There is also no guarantee that Grouper and RowSegmenter will always support the same set of datatypes.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] icexelloss commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "icexelloss (via GitHub)" <gi...@apache.org>.
icexelloss commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1127135554


##########
cpp/src/arrow/compute/row/grouper.cc:
##########
@@ -39,12 +43,330 @@
 namespace arrow {
 
 using internal::checked_cast;
+using internal::PrimitiveScalarBase;
 
 namespace compute {
 
 namespace {
 
-struct GrouperImpl : Grouper {
+constexpr uint32_t kNoGroupId = std::numeric_limits<uint32_t>::max();
+
+using group_id_t = std::remove_const<decltype(kNoGroupId)>::type;
+using GroupIdType = CTypeTraits<group_id_t>::ArrowType;
+auto g_group_id_type = std::make_shared<GroupIdType>();
+
+inline const uint8_t* GetValuesAsBytes(const ArraySpan& data, int64_t offset = 0) {
+  DCHECK_GT(data.type->byte_width(), 0);
+  int64_t absolute_byte_offset = (data.offset + offset) * data.type->byte_width();
+  return data.GetValues<uint8_t>(1, absolute_byte_offset);
+}
+
+template <typename Value>
+Status CheckForGetNextSegment(const std::vector<Value>& values, int64_t length,
+                              int64_t offset, const std::vector<TypeHolder>& key_types) {
+  if (offset < 0 || offset > length) {
+    return Status::Invalid("invalid grouping segmenter offset: ", offset);
+  }
+  if (values.size() != key_types.size()) {
+    return Status::Invalid("expected batch size ", key_types.size(), " but got ",
+                           values.size());
+  }
+  for (size_t i = 0; i < key_types.size(); i++) {
+    const auto& value = values[i];
+    const auto& key_type = key_types[i];
+    if (*value.type() != *key_type.type) {
+      return Status::Invalid("expected batch value ", i, " of type ", *key_type.type,
+                             " but got ", *value.type());
+    }
+  }
+  return Status::OK();
+}
+
+template <typename Batch>
+enable_if_t<std::is_same<Batch, ExecSpan>::value || std::is_same<Batch, ExecBatch>::value,
+            Status>
+CheckForGetNextSegment(const Batch& batch, int64_t offset,
+                       const std::vector<TypeHolder>& key_types) {
+  return CheckForGetNextSegment(batch.values, batch.length, offset, key_types);
+}
+
+struct BaseRowSegmenter : public RowSegmenter {
+  explicit BaseRowSegmenter(const std::vector<TypeHolder>& key_types)
+      : key_types_(key_types) {}
+
+  const std::vector<TypeHolder>& key_types() const override { return key_types_; }
+
+  std::vector<TypeHolder> key_types_;
+};
+
+Segment MakeSegment(int64_t batch_length, int64_t offset, int64_t length, bool extends) {
+  return Segment{offset, length, offset + length >= batch_length, extends};
+}
+
+int64_t GetMatchLength(const uint8_t* match_bytes, int64_t match_width,
+                       const uint8_t* array_bytes, int64_t offset, int64_t length) {
+  int64_t cursor, byte_cursor;
+  for (cursor = offset, byte_cursor = match_width * cursor; cursor < length;
+       cursor++, byte_cursor += match_width) {
+    if (memcmp(match_bytes, array_bytes + byte_cursor,
+               static_cast<size_t>(match_width)) != 0) {
+      break;
+    }
+  }
+  return std::min(cursor, length) - offset;
+}
+
+using ExtendFunc = std::function<bool(const void*)>;
+constexpr bool kDefaultExtends = true;
+constexpr bool kEmptyExtends = true;
+
+struct NoKeysSegmenter : public BaseRowSegmenter {
+  static std::unique_ptr<RowSegmenter> Make() {
+    return std::make_unique<NoKeysSegmenter>();
+  }
+
+  NoKeysSegmenter() : BaseRowSegmenter({}) {}
+
+  Status Reset() override { return Status::OK(); }
+
+  Result<Segment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {}));
+    return MakeSegment(batch.length, offset, batch.length - offset, kDefaultExtends);
+  }
+};
+
+struct SimpleKeySegmenter : public BaseRowSegmenter {
+  static Result<std::unique_ptr<RowSegmenter>> Make(TypeHolder key_type) {
+    return std::make_unique<SimpleKeySegmenter>(key_type);
+  }
+
+  explicit SimpleKeySegmenter(TypeHolder key_type)
+      : BaseRowSegmenter({key_type}), key_type_(key_types_[0]), save_key_data_() {}
+
+  Status CheckType(const DataType& type) {
+    if (!is_fixed_width(type)) {
+      return Status::Invalid("SimpleKeySegmenter does not support type ", type);
+    }
+    return Status::OK();
+  }
+
+  Status Reset() override {
+    save_key_data_.resize(0);
+    return Status::OK();
+  }
+
+  // Checks whether the given grouping data extends the current segment, i.e., is equal to
+  // previously seen grouping data, which is updated with each invocation.
+  bool Extend(const void* data) {
+    size_t byte_width = static_cast<size_t>(key_type_.type->byte_width());
+    bool extends = save_key_data_.size() != byte_width
+                       ? kDefaultExtends
+                       : 0 == memcmp(save_key_data_.data(), data, byte_width);
+    save_key_data_.resize(byte_width);
+    memcpy(save_key_data_.data(), data, byte_width);
+    return extends;
+  }
+
+  Result<Segment> GetNextSegment(const Scalar& scalar, int64_t offset, int64_t length) {
+    ARROW_RETURN_NOT_OK(CheckType(*scalar.type));
+    if (!scalar.is_valid) {
+      return Status::Invalid("segmenting an invalid scalar");
+    }
+    auto data = checked_cast<const PrimitiveScalarBase&>(scalar).data();
+    bool extends = length > 0 ? Extend(data) : kEmptyExtends;
+    return MakeSegment(length, offset, length, extends);
+  }
+
+  Result<Segment> GetNextSegment(const DataType& array_type, const uint8_t* array_bytes,
+                                 int64_t offset, int64_t length) {
+    RETURN_NOT_OK(CheckType(array_type));
+    int64_t byte_width = array_type.byte_width();
+    int64_t match_length = GetMatchLength(array_bytes + offset * byte_width, byte_width,

Review Comment:
   TBH I found it confusing at first because I didn't know this is used by which class, and I realized this is only used by SimpleKeySegmenter much later into reading the code. That's why I suggest making it into SimpleKeySegmenter to make the scope of the function clear. But if you feel strongly about this I wouldn't hold it.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rtpsw commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "rtpsw (via GitHub)" <gi...@apache.org>.
rtpsw commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1127461157


##########
cpp/src/arrow/compute/row/grouper.cc:
##########
@@ -39,12 +43,330 @@
 namespace arrow {
 
 using internal::checked_cast;
+using internal::PrimitiveScalarBase;
 
 namespace compute {
 
 namespace {
 
-struct GrouperImpl : Grouper {
+constexpr uint32_t kNoGroupId = std::numeric_limits<uint32_t>::max();
+
+using group_id_t = std::remove_const<decltype(kNoGroupId)>::type;
+using GroupIdType = CTypeTraits<group_id_t>::ArrowType;
+auto g_group_id_type = std::make_shared<GroupIdType>();
+
+inline const uint8_t* GetValuesAsBytes(const ArraySpan& data, int64_t offset = 0) {
+  DCHECK_GT(data.type->byte_width(), 0);
+  int64_t absolute_byte_offset = (data.offset + offset) * data.type->byte_width();
+  return data.GetValues<uint8_t>(1, absolute_byte_offset);
+}
+
+template <typename Value>
+Status CheckForGetNextSegment(const std::vector<Value>& values, int64_t length,
+                              int64_t offset, const std::vector<TypeHolder>& key_types) {
+  if (offset < 0 || offset > length) {
+    return Status::Invalid("invalid grouping segmenter offset: ", offset);
+  }
+  if (values.size() != key_types.size()) {
+    return Status::Invalid("expected batch size ", key_types.size(), " but got ",
+                           values.size());
+  }
+  for (size_t i = 0; i < key_types.size(); i++) {
+    const auto& value = values[i];
+    const auto& key_type = key_types[i];
+    if (*value.type() != *key_type.type) {
+      return Status::Invalid("expected batch value ", i, " of type ", *key_type.type,
+                             " but got ", *value.type());
+    }
+  }
+  return Status::OK();
+}
+
+template <typename Batch>
+enable_if_t<std::is_same<Batch, ExecSpan>::value || std::is_same<Batch, ExecBatch>::value,
+            Status>
+CheckForGetNextSegment(const Batch& batch, int64_t offset,
+                       const std::vector<TypeHolder>& key_types) {
+  return CheckForGetNextSegment(batch.values, batch.length, offset, key_types);
+}
+
+struct BaseRowSegmenter : public RowSegmenter {
+  explicit BaseRowSegmenter(const std::vector<TypeHolder>& key_types)
+      : key_types_(key_types) {}
+
+  const std::vector<TypeHolder>& key_types() const override { return key_types_; }
+
+  std::vector<TypeHolder> key_types_;
+};
+
+Segment MakeSegment(int64_t batch_length, int64_t offset, int64_t length, bool extends) {
+  return Segment{offset, length, offset + length >= batch_length, extends};
+}
+
+int64_t GetMatchLength(const uint8_t* match_bytes, int64_t match_width,
+                       const uint8_t* array_bytes, int64_t offset, int64_t length) {
+  int64_t cursor, byte_cursor;
+  for (cursor = offset, byte_cursor = match_width * cursor; cursor < length;
+       cursor++, byte_cursor += match_width) {
+    if (memcmp(match_bytes, array_bytes + byte_cursor,
+               static_cast<size_t>(match_width)) != 0) {
+      break;
+    }
+  }
+  return std::min(cursor, length) - offset;
+}
+
+using ExtendFunc = std::function<bool(const void*)>;
+constexpr bool kDefaultExtends = true;
+constexpr bool kEmptyExtends = true;
+
+struct NoKeysSegmenter : public BaseRowSegmenter {
+  static std::unique_ptr<RowSegmenter> Make() {
+    return std::make_unique<NoKeysSegmenter>();
+  }
+
+  NoKeysSegmenter() : BaseRowSegmenter({}) {}
+
+  Status Reset() override { return Status::OK(); }
+
+  Result<Segment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {}));
+    return MakeSegment(batch.length, offset, batch.length - offset, kDefaultExtends);
+  }
+};
+
+struct SimpleKeySegmenter : public BaseRowSegmenter {
+  static Result<std::unique_ptr<RowSegmenter>> Make(TypeHolder key_type) {
+    return std::make_unique<SimpleKeySegmenter>(key_type);
+  }
+
+  explicit SimpleKeySegmenter(TypeHolder key_type)
+      : BaseRowSegmenter({key_type}), key_type_(key_types_[0]), save_key_data_() {}
+
+  Status CheckType(const DataType& type) {
+    if (!is_fixed_width(type)) {
+      return Status::Invalid("SimpleKeySegmenter does not support type ", type);
+    }
+    return Status::OK();
+  }
+
+  Status Reset() override {
+    save_key_data_.resize(0);
+    return Status::OK();
+  }
+
+  // Checks whether the given grouping data extends the current segment, i.e., is equal to
+  // previously seen grouping data, which is updated with each invocation.
+  bool Extend(const void* data) {
+    size_t byte_width = static_cast<size_t>(key_type_.type->byte_width());
+    bool extends = save_key_data_.size() != byte_width
+                       ? kDefaultExtends
+                       : 0 == memcmp(save_key_data_.data(), data, byte_width);
+    save_key_data_.resize(byte_width);
+    memcpy(save_key_data_.data(), data, byte_width);
+    return extends;
+  }
+
+  Result<Segment> GetNextSegment(const Scalar& scalar, int64_t offset, int64_t length) {
+    ARROW_RETURN_NOT_OK(CheckType(*scalar.type));
+    if (!scalar.is_valid) {
+      return Status::Invalid("segmenting an invalid scalar");
+    }
+    auto data = checked_cast<const PrimitiveScalarBase&>(scalar).data();
+    bool extends = length > 0 ? Extend(data) : kEmptyExtends;
+    return MakeSegment(length, offset, length, extends);
+  }
+
+  Result<Segment> GetNextSegment(const DataType& array_type, const uint8_t* array_bytes,
+                                 int64_t offset, int64_t length) {
+    RETURN_NOT_OK(CheckType(array_type));
+    int64_t byte_width = array_type.byte_width();
+    int64_t match_length = GetMatchLength(array_bytes + offset * byte_width, byte_width,
+                                          array_bytes, offset, length);
+    bool extends = length > 0 ? Extend(array_bytes + offset * byte_width) : kEmptyExtends;
+    return MakeSegment(length, offset, match_length, extends);
+  }
+
+  Result<Segment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {key_type_}));
+    if (offset == batch.length) {
+      return MakeSegment(batch.length, offset, 0, kEmptyExtends);
+    }
+    const auto& value = batch.values[0];
+    if (value.is_scalar()) {
+      return GetNextSegment(*value.scalar, offset, batch.length);
+    }
+    ARROW_DCHECK(value.is_array());
+    const auto& array = value.array;
+    if (array.GetNullCount() > 0) {
+      return Status::NotImplemented("segmenting a nullable array");
+    }
+    return GetNextSegment(*array.type, GetValuesAsBytes(array), offset, batch.length);
+  }
+
+ private:
+  TypeHolder key_type_;
+  std::vector<uint8_t> save_key_data_;
+};
+
+struct AnyKeysSegmenter : public BaseRowSegmenter {
+  static Result<std::unique_ptr<RowSegmenter>> Make(
+      const std::vector<TypeHolder>& key_types, ExecContext* ctx) {
+    ARROW_RETURN_NOT_OK(Grouper::Make(key_types, ctx));  // check types
+    return std::make_unique<AnyKeysSegmenter>(key_types, ctx);
+  }
+
+  AnyKeysSegmenter(const std::vector<TypeHolder>& key_types, ExecContext* ctx)
+      : BaseRowSegmenter(key_types),
+        ctx_(ctx),
+        grouper_(nullptr),
+        save_group_id_(kNoGroupId) {}
+
+  Status Reset() override {

Review Comment:
   I'll add test coverage for segmenter resetting.
   
   > In your mind, how would this be used in the future?
   
   For example, outside the context of this PR, it makes sense to reset a segmenter when each batch is processed independently, rather than within a stream of batches, so that a segment must not cross into the next batch.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] icexelloss commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "icexelloss (via GitHub)" <gi...@apache.org>.
icexelloss commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1127974592


##########
cpp/src/arrow/compute/row/grouper.cc:
##########
@@ -39,12 +43,336 @@
 namespace arrow {
 
 using internal::checked_cast;
+using internal::PrimitiveScalarBase;
 
 namespace compute {
 
 namespace {
 
-struct GrouperImpl : Grouper {
+constexpr uint32_t kNoGroupId = std::numeric_limits<uint32_t>::max();
+
+using group_id_t = std::remove_const<decltype(kNoGroupId)>::type;
+using GroupIdType = CTypeTraits<group_id_t>::ArrowType;
+auto g_group_id_type = std::make_shared<GroupIdType>();
+
+inline const uint8_t* GetValuesAsBytes(const ArraySpan& data, int64_t offset = 0) {
+  DCHECK_GT(data.type->byte_width(), 0);
+  int64_t absolute_byte_offset = (data.offset + offset) * data.type->byte_width();
+  return data.GetValues<uint8_t>(1, absolute_byte_offset);
+}
+
+template <typename Value>
+Status CheckForGetNextSegment(const std::vector<Value>& values, int64_t length,
+                              int64_t offset, const std::vector<TypeHolder>& key_types) {
+  if (offset < 0 || offset > length) {
+    return Status::Invalid("invalid grouping segmenter offset: ", offset);
+  }
+  if (values.size() != key_types.size()) {
+    return Status::Invalid("expected batch size ", key_types.size(), " but got ",
+                           values.size());
+  }
+  for (size_t i = 0; i < key_types.size(); i++) {
+    const auto& value = values[i];
+    const auto& key_type = key_types[i];
+    if (*value.type() != *key_type.type) {
+      return Status::Invalid("expected batch value ", i, " of type ", *key_type.type,
+                             " but got ", *value.type());
+    }
+  }
+  return Status::OK();
+}
+
+template <typename Batch>
+enable_if_t<std::is_same<Batch, ExecSpan>::value || std::is_same<Batch, ExecBatch>::value,
+            Status>
+CheckForGetNextSegment(const Batch& batch, int64_t offset,
+                       const std::vector<TypeHolder>& key_types) {
+  return CheckForGetNextSegment(batch.values, batch.length, offset, key_types);
+}
+
+struct BaseGroupingSegmenter : public GroupingSegmenter {
+  explicit BaseGroupingSegmenter(const std::vector<TypeHolder>& key_types)
+      : key_types_(key_types) {}
+
+  const std::vector<TypeHolder>& key_types() const override { return key_types_; }
+
+  std::vector<TypeHolder> key_types_;
+};
+
+GroupingSegment MakeSegment(int64_t batch_length, int64_t offset, int64_t length,
+                            bool extends) {
+  return GroupingSegment{offset, length, offset + length >= batch_length, extends};
+}
+
+int64_t GetMatchLength(const uint8_t* match_bytes, int64_t match_width,
+                       const uint8_t* array_bytes, int64_t offset, int64_t length) {
+  int64_t cursor, byte_cursor;
+  for (cursor = offset, byte_cursor = match_width * cursor; cursor < length;
+       cursor++, byte_cursor += match_width) {
+    if (memcmp(match_bytes, array_bytes + byte_cursor,
+               static_cast<size_t>(match_width)) != 0) {
+      break;
+    }
+  }
+  return std::min(cursor, length - offset);
+}
+
+using ExtendFunc = std::function<bool(const void*)>;
+constexpr bool kDefaultExtends = true;
+constexpr bool kEmptyExtends = true;
+
+struct NoKeysGroupingSegmenter : public BaseGroupingSegmenter {
+  static std::unique_ptr<GroupingSegmenter> Make() {
+    return std::make_unique<NoKeysGroupingSegmenter>();
+  }
+
+  NoKeysGroupingSegmenter() : BaseGroupingSegmenter({}) {}
+
+  Status Reset() override { return Status::OK(); }
+
+  Result<GroupingSegment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {}));
+    return MakeSegment(batch.length, offset, batch.length - offset, kDefaultExtends);
+  }
+};
+
+struct SimpleKeyGroupingSegmenter : public BaseGroupingSegmenter {
+  static Result<std::unique_ptr<GroupingSegmenter>> Make(TypeHolder key_type) {
+    return std::make_unique<SimpleKeyGroupingSegmenter>(key_type);
+  }
+
+  explicit SimpleKeyGroupingSegmenter(TypeHolder key_type)
+      : BaseGroupingSegmenter({key_type}), key_type_(key_types_[0]), save_key_data_() {}
+
+  Status CheckType(const DataType& type) {
+    if (!is_fixed_width(type)) {
+      return Status::Invalid("SimpleKeyGroupingSegmenter does not support type ", type);
+    }
+    return Status::OK();
+  }
+
+  Status Reset() override {
+    save_key_data_.resize(0);
+    return Status::OK();
+  }
+
+  // Checks whether the given grouping data extends the current segment, i.e., is equal to
+  // previously seen grouping data, which is updated with each invocation.
+  bool Extend(const void* data) {
+    size_t byte_width = static_cast<size_t>(key_type_.type->byte_width());
+    bool extends = save_key_data_.size() != byte_width
+                       ? kDefaultExtends
+                       : 0 == memcmp(save_key_data_.data(), data, byte_width);
+    save_key_data_.resize(byte_width);
+    memcpy(save_key_data_.data(), data, byte_width);
+    return extends;
+  }
+
+  Result<GroupingSegment> GetNextSegment(const Scalar& scalar, int64_t offset,
+                                         int64_t length) {
+    ARROW_RETURN_NOT_OK(CheckType(*scalar.type));
+    if (!scalar.is_valid) {
+      return Status::Invalid("segmenting an invalid scalar");
+    }
+    auto data = checked_cast<const PrimitiveScalarBase&>(scalar).data();
+    bool extends = length > 0 ? Extend(data) : kEmptyExtends;
+    return MakeSegment(length, offset, length, extends);
+  }
+
+  Result<GroupingSegment> GetNextSegment(const DataType& array_type,
+                                         const uint8_t* array_bytes, int64_t offset,
+                                         int64_t length) {
+    RETURN_NOT_OK(CheckType(array_type));
+    int64_t byte_width = array_type.byte_width();
+    int64_t match_length = GetMatchLength(array_bytes + offset * byte_width, byte_width,
+                                          array_bytes, offset, length);
+    bool extends = length > 0 ? Extend(array_bytes + offset * byte_width) : kEmptyExtends;
+    return MakeSegment(length, offset, match_length, extends);
+  }
+
+  Result<GroupingSegment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {key_type_}));
+    if (offset == batch.length) {
+      return MakeSegment(batch.length, offset, 0, kEmptyExtends);
+    }
+    const auto& value = batch.values[0];
+    if (value.is_scalar()) {
+      return GetNextSegment(*value.scalar, offset, batch.length);
+    }
+    ARROW_DCHECK(value.is_array());
+    const auto& array = value.array;
+    if (array.GetNullCount() > 0) {
+      return Status::NotImplemented("segmenting a nullable array");
+    }
+    return GetNextSegment(*array.type, GetValuesAsBytes(array), offset, batch.length);
+  }
+
+ private:
+  TypeHolder key_type_;
+  std::vector<uint8_t> save_key_data_;
+};
+
+struct AnyKeysGroupingSegmenter : public BaseGroupingSegmenter {
+  static Result<std::unique_ptr<GroupingSegmenter>> Make(
+      const std::vector<TypeHolder>& key_types, ExecContext* ctx) {
+    ARROW_RETURN_NOT_OK(Grouper::Make(key_types, ctx));  // check types
+    return std::make_unique<AnyKeysGroupingSegmenter>(key_types, ctx);
+  }
+
+  AnyKeysGroupingSegmenter(const std::vector<TypeHolder>& key_types, ExecContext* ctx)
+      : BaseGroupingSegmenter(key_types),
+        ctx_(ctx),
+        grouper_(nullptr),
+        save_group_id_(kNoGroupId) {}
+
+  Status Reset() override {
+    grouper_ = nullptr;
+    save_group_id_ = kNoGroupId;
+    return Status::OK();
+  }
+
+  bool Extend(const void* data) {
+    auto group_id = *static_cast<const group_id_t*>(data);
+    bool extends =
+        save_group_id_ == kNoGroupId ? kDefaultExtends : save_group_id_ == group_id;
+    save_group_id_ = group_id;
+    return extends;
+  }
+
+  // Runs the grouper on a single row.  This is used to determine the group id of the
+  // first row of a new segment to see if it extends the previous segment.
+  template <typename Batch>
+  Result<group_id_t> MapGroupIdAt(const Batch& batch, int64_t offset) {
+    if (!grouper_) return kNoGroupId;
+    ARROW_ASSIGN_OR_RAISE(auto datum, grouper_->Consume(batch, offset,
+                                                        /*consume_length=*/1));
+    if (!datum.is_array()) {
+      return Status::Invalid("accessing unsupported datum kind ", datum.kind());
+    }
+    const std::shared_ptr<ArrayData>& data = datum.array();
+    ARROW_DCHECK(data->GetNullCount() == 0);
+    DCHECK_EQ(data->type->id(), GroupIdType::type_id);
+    DCHECK_EQ(1, data->length);
+    const group_id_t* values = data->GetValues<group_id_t>(1);
+    return values[0];
+  }
+
+  Result<GroupingSegment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, key_types_));
+    if (offset == batch.length) {
+      return MakeSegment(batch.length, offset, 0, kEmptyExtends);
+    }
+    // ARROW-18311: make Grouper support Reset()
+    // so it can be cached instead of recreated below
+    //
+    // the group id must be computed prior to resetting the grouper, since it is compared
+    // to save_group_id_, and after resetting the grouper produces incomparable group ids
+    ARROW_ASSIGN_OR_RAISE(auto group_id, MapGroupIdAt(batch, offset));
+    ExtendFunc bound_extend = [this, group_id](const void* data) {
+      bool extends = Extend(&group_id);
+      save_group_id_ = *static_cast<const group_id_t*>(data);
+      return extends;
+    };
+    ARROW_ASSIGN_OR_RAISE(grouper_, Grouper::Make(key_types_, ctx_));  // TODO: reset it
+    ARROW_ASSIGN_OR_RAISE(auto datum, grouper_->Consume(batch, offset));
+    if (datum.is_array()) {
+      const std::shared_ptr<ArrayData>& data = datum.array();
+      ARROW_DCHECK(data->GetNullCount() == 0);
+      DCHECK_EQ(data->type->id(), GroupIdType::type_id);
+      const group_id_t* values = data->GetValues<group_id_t>(1);
+      int64_t cursor;
+      for (cursor = 1; cursor < data->length; cursor++) {
+        if (values[0] != values[cursor]) break;
+      }
+      int64_t length = std::min(cursor, batch.length - offset);
+      bool extends = length > 0 ? bound_extend(values) : kEmptyExtends;
+      return MakeSegment(batch.length, offset, length, extends);
+    } else {
+      return Status::Invalid("segmenting unsupported datum kind ", datum.kind());
+    }
+  }
+
+ private:
+  ExecContext* const ctx_;
+  std::unique_ptr<Grouper> grouper_;
+  group_id_t save_group_id_;
+};
+
+Status CheckForConsume(int64_t batch_length, int64_t& consume_offset,
+                       int64_t* consume_length) {
+  if (consume_offset < 0) {
+    return Status::Invalid("invalid grouper consume offset: ", consume_offset);
+  }
+  if (*consume_length < 0) {
+    *consume_length = batch_length - consume_offset;

Review Comment:
   Maybe `UpdateConsumeLength` is a better name?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] icexelloss commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "icexelloss (via GitHub)" <gi...@apache.org>.
icexelloss commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1127984376


##########
cpp/src/arrow/compute/row/grouper.cc:
##########
@@ -39,12 +43,521 @@
 namespace arrow {
 
 using internal::checked_cast;
+using internal::PrimitiveScalarBase;
 
 namespace compute {
 
 namespace {
 
-struct GrouperImpl : Grouper {
+constexpr uint32_t kNoGroupId = std::numeric_limits<uint32_t>::max();
+
+using group_id_t = std::remove_const<decltype(kNoGroupId)>::type;
+using GroupIdType = CTypeTraits<group_id_t>::ArrowType;
+auto group_id_type = std::make_shared<GroupIdType>();
+
+inline const uint8_t* GetValuesAsBytes(const ArrayData& data, int64_t offset = 0) {
+  DCHECK_GT(data.type->byte_width(), 0);
+  int64_t absolute_byte_offset = (data.offset + offset) * data.type->byte_width();
+  return data.GetValues<uint8_t>(1, absolute_byte_offset);
+}
+
+inline const uint8_t* GetValuesAsBytes(const ArraySpan& data, int64_t offset = 0) {
+  DCHECK_GT(data.type->byte_width(), 0);
+  int64_t absolute_byte_offset = (data.offset + offset) * data.type->byte_width();
+  return data.GetValues<uint8_t>(1, absolute_byte_offset);
+}
+
+template <typename Value>
+Status CheckForGetNextSegment(const std::vector<Value>& values, int64_t length,
+                              int64_t offset, const std::vector<TypeHolder>& key_types) {
+  if (offset < 0 || offset > length) {
+    return Status::Invalid("invalid grouping segmenter offset: ", offset);
+  }
+  if (values.size() != key_types.size()) {
+    return Status::Invalid("expected batch size ", key_types.size(), " but got ",
+                           values.size());
+  }
+  for (size_t i = 0; i < key_types.size(); i++) {
+    const auto& value = values[i];
+    const auto& key_type = key_types[i];
+    if (*value.type() != *key_type.type) {
+      return Status::Invalid("expected batch value ", i, " of type ", *key_type.type,
+                             " but got ", *value.type());
+    }
+  }
+  return Status::OK();
+}
+
+template <typename Batch>
+enable_if_t<std::is_same<Batch, ExecSpan>::value || std::is_same<Batch, ExecBatch>::value,
+            Status>
+CheckForGetNextSegment(const Batch& batch, int64_t offset,
+                       const std::vector<TypeHolder>& key_types) {
+  return CheckForGetNextSegment(batch.values, batch.length, offset, key_types);
+}
+
+struct BaseGroupingSegmenter : public GroupingSegmenter {
+  explicit BaseGroupingSegmenter(const std::vector<TypeHolder>& key_types)
+      : key_types_(key_types) {}
+
+  const std::vector<TypeHolder>& key_types() const override { return key_types_; }
+
+  std::vector<TypeHolder> key_types_;
+};
+
+GroupingSegment MakeSegment(int64_t batch_length, int64_t offset, int64_t length,
+                            bool extends) {
+  return GroupingSegment{offset, length, offset + length >= batch_length, extends};
+}
+
+int64_t GetMatchLength(const uint8_t* match_bytes, int64_t match_width,

Review Comment:
   Thanks - I would agree that while generalization is usually good. However, in this case if it comes performance cost then I would think twice, especially when ordered aggregation is the main goal here and the more general form (segmented data) is not really a use we care about internally. But for now we can proceed with this.  



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] icexelloss commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "icexelloss (via GitHub)" <gi...@apache.org>.
icexelloss commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1127984376


##########
cpp/src/arrow/compute/row/grouper.cc:
##########
@@ -39,12 +43,521 @@
 namespace arrow {
 
 using internal::checked_cast;
+using internal::PrimitiveScalarBase;
 
 namespace compute {
 
 namespace {
 
-struct GrouperImpl : Grouper {
+constexpr uint32_t kNoGroupId = std::numeric_limits<uint32_t>::max();
+
+using group_id_t = std::remove_const<decltype(kNoGroupId)>::type;
+using GroupIdType = CTypeTraits<group_id_t>::ArrowType;
+auto group_id_type = std::make_shared<GroupIdType>();
+
+inline const uint8_t* GetValuesAsBytes(const ArrayData& data, int64_t offset = 0) {
+  DCHECK_GT(data.type->byte_width(), 0);
+  int64_t absolute_byte_offset = (data.offset + offset) * data.type->byte_width();
+  return data.GetValues<uint8_t>(1, absolute_byte_offset);
+}
+
+inline const uint8_t* GetValuesAsBytes(const ArraySpan& data, int64_t offset = 0) {
+  DCHECK_GT(data.type->byte_width(), 0);
+  int64_t absolute_byte_offset = (data.offset + offset) * data.type->byte_width();
+  return data.GetValues<uint8_t>(1, absolute_byte_offset);
+}
+
+template <typename Value>
+Status CheckForGetNextSegment(const std::vector<Value>& values, int64_t length,
+                              int64_t offset, const std::vector<TypeHolder>& key_types) {
+  if (offset < 0 || offset > length) {
+    return Status::Invalid("invalid grouping segmenter offset: ", offset);
+  }
+  if (values.size() != key_types.size()) {
+    return Status::Invalid("expected batch size ", key_types.size(), " but got ",
+                           values.size());
+  }
+  for (size_t i = 0; i < key_types.size(); i++) {
+    const auto& value = values[i];
+    const auto& key_type = key_types[i];
+    if (*value.type() != *key_type.type) {
+      return Status::Invalid("expected batch value ", i, " of type ", *key_type.type,
+                             " but got ", *value.type());
+    }
+  }
+  return Status::OK();
+}
+
+template <typename Batch>
+enable_if_t<std::is_same<Batch, ExecSpan>::value || std::is_same<Batch, ExecBatch>::value,
+            Status>
+CheckForGetNextSegment(const Batch& batch, int64_t offset,
+                       const std::vector<TypeHolder>& key_types) {
+  return CheckForGetNextSegment(batch.values, batch.length, offset, key_types);
+}
+
+struct BaseGroupingSegmenter : public GroupingSegmenter {
+  explicit BaseGroupingSegmenter(const std::vector<TypeHolder>& key_types)
+      : key_types_(key_types) {}
+
+  const std::vector<TypeHolder>& key_types() const override { return key_types_; }
+
+  std::vector<TypeHolder> key_types_;
+};
+
+GroupingSegment MakeSegment(int64_t batch_length, int64_t offset, int64_t length,
+                            bool extends) {
+  return GroupingSegment{offset, length, offset + length >= batch_length, extends};
+}
+
+int64_t GetMatchLength(const uint8_t* match_bytes, int64_t match_width,

Review Comment:
   Thanks - I would argue that while generalization is usually good, in this case if it comes performance cost then I would think twice, especially when ordered aggregation is the main goal here. But for now we can proceed with this.  



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rtpsw commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "rtpsw (via GitHub)" <gi...@apache.org>.
rtpsw commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1128232128


##########
cpp/src/arrow/compute/kernels/hash_aggregate_test.cc:
##########
@@ -4202,5 +4551,260 @@ TEST(GroupBy, OnlyKeys) {
                       /*verbose=*/true);
   }
 }
+
+INSTANTIATE_TEST_SUITE_P(GroupBy, GroupBy, ::testing::Values(RunGroupByImpl));
+
+class SegmentedScalarGroupBy : public GroupBy {};
+
+class SegmentedKeyGroupBy : public GroupBy {};
+
+void TestSegment(GroupByFunction group_by, const std::shared_ptr<Table>& table,
+                 Datum output, const std::vector<Datum>& keys,
+                 const std::vector<Datum>& segment_keys, bool scalar) {
+  const char* names[] = {
+      scalar ? "count" : "hash_count",
+      scalar ? "sum" : "hash_sum",
+      scalar ? "min_max" : "hash_min_max",
+  };
+  ASSERT_OK_AND_ASSIGN(Datum aggregated_and_grouped,
+                       group_by(
+                           {
+                               table->GetColumnByName("argument"),
+                               table->GetColumnByName("argument"),
+                               table->GetColumnByName("argument"),
+                           },
+                           keys, segment_keys,
+                           {
+                               {names[0], nullptr, "agg_0", names[0]},
+                               {names[1], nullptr, "agg_1", names[1]},
+                               {names[2], nullptr, "agg_2", names[2]},
+                           },
+                           kDefaultUseThreads, /*naive=*/false));
+
+  AssertDatumsEqual(output, aggregated_and_grouped, /*verbose=*/true);
+}
+
+void TestSegmentScalar(GroupByFunction group_by, const std::shared_ptr<Table>& table,

Review Comment:
   The former tests with empty `keys` (covering `ScalarAggregateNode`) and the latter using `{"key"}` as the (non-segment) keys.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rtpsw commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "rtpsw (via GitHub)" <gi...@apache.org>.
rtpsw commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1117891721


##########
cpp/src/arrow/compute/exec/aggregate_node.cc:
##########
@@ -169,17 +206,63 @@ void AggregatesToString(std::stringstream* ss, const Schema& input_schema,
   *ss << ']';
 }
 
+template <typename BatchHandler>
+Status HandleSegments(std::unique_ptr<GroupingSegmenter>& segmenter,
+                      const ExecBatch& batch, const std::vector<int>& ids,
+                      const BatchHandler& handle_batch) {
+  int64_t offset = 0;
+  ARROW_ASSIGN_OR_RAISE(auto segment_batch, batch.SelectValues(ids));
+  while (true) {
+    ARROW_ASSIGN_OR_RAISE(auto segment, segmenter->GetNextSegment(segment_batch, offset));
+    if (segment.offset >= segment_batch.length) break;

Review Comment:
   Done.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] icexelloss commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "icexelloss (via GitHub)" <gi...@apache.org>.
icexelloss commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1117338539


##########
cpp/src/arrow/compute/row/grouper.h:
##########
@@ -30,6 +30,49 @@
 namespace arrow {
 namespace compute {
 
+/// \brief A segment of contiguous rows for grouping
+struct ARROW_EXPORT GroupingSegment {

Review Comment:
   Can you explain what the four variables mean here



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rtpsw commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "rtpsw (via GitHub)" <gi...@apache.org>.
rtpsw commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1117884526


##########
cpp/src/arrow/compute/exec/options.h:
##########
@@ -199,21 +199,32 @@ class ARROW_EXPORT ProjectNodeOptions : public ExecNodeOptions {
   std::vector<std::string> names;
 };
 
-/// \brief Make a node which aggregates input batches, optionally grouped by keys.
+/// \brief Make a node which aggregates input batches, optionally grouped by keys and
+/// optionally segmented by segment-keys. Both keys and segment-keys determine the group.
+/// However segment-keys are also used for determining grouping segments, which should be
+/// large, and allow streaming a partial aggregation result after processing each segment.
+/// One common use-case for segment-keys is ordered aggregation, in which the segment-key
+/// attribute specifies a column with non-decreasing values or a lexigographically-ordered
+/// set of such columns.
 ///
 /// If the keys attribute is a non-empty vector, then each aggregate in `aggregates` is
 /// expected to be a HashAggregate function. If the keys attribute is an empty vector,
 /// then each aggregate is assumed to be a ScalarAggregate function.
 class ARROW_EXPORT AggregateNodeOptions : public ExecNodeOptions {
  public:
   explicit AggregateNodeOptions(std::vector<Aggregate> aggregates,
-                                std::vector<FieldRef> keys = {})
-      : aggregates(std::move(aggregates)), keys(std::move(keys)) {}
+                                std::vector<FieldRef> keys = {},
+                                std::vector<FieldRef> segment_keys = {})
+      : aggregates(std::move(aggregates)),
+        keys(std::move(keys)),
+        segment_keys(std::move(segment_keys)) {}
 
   // aggregations which will be applied to the targetted fields
   std::vector<Aggregate> aggregates;
   // keys by which aggregations will be grouped
   std::vector<FieldRef> keys;
+  // keys by which aggregations will be segmented
+  std::vector<FieldRef> segment_keys;

Review Comment:
   > So if I want to group by A, B, C and I know my data is sorted by A then I can do segments=A and keys=B C?
   
   Yes, that's what you'd do.
   
   I added the check to disallow.



##########
cpp/src/arrow/compute/kernels/hash_aggregate_test.cc:
##########
@@ -174,81 +242,117 @@ Result<Datum> RunGroupBy(const BatchesWithSchema& input,
   ARROW_ASSIGN_OR_RAISE(std::vector<ExecBatch> output_batches,
                         start_and_collect.MoveResult());
 
-  ArrayVector out_arrays(aggregates.size() + key_names.size());
   const auto& output_schema = plan->nodes()[0]->output()->output_schema();
+  if (!segmented) {
+    return MakeGroupByOutput(output_batches, output_schema, aggregates.size(),
+                             key_names.size(), naive);
+  }
+
+  std::vector<ArrayVector> out_arrays(aggregates.size() + key_names.size() +
+                                      segment_key_names.size());
   for (size_t i = 0; i < out_arrays.size(); ++i) {
     std::vector<std::shared_ptr<Array>> arrays(output_batches.size());
     for (size_t j = 0; j < output_batches.size(); ++j) {
-      arrays[j] = output_batches[j].values[i].make_array();
+      auto& value = output_batches[j].values[i];
+      if (value.is_scalar()) {
+        ARROW_ASSIGN_OR_RAISE(
+            arrays[j], MakeArrayFromScalar(*value.scalar(), output_batches[j].length));
+      } else if (value.is_array()) {
+        arrays[j] = value.make_array();
+      } else {
+        return Status::Invalid("GroupByUsingExecPlan unsupported value kind ",
+                               ToString(value.kind()));
+      }
     }
     if (arrays.empty()) {
+      arrays.resize(1);
       ARROW_ASSIGN_OR_RAISE(
-          out_arrays[i],
-          MakeArrayOfNull(output_schema->field(static_cast<int>(i))->type(),
-                          /*length=*/0));
-    } else {
-      ARROW_ASSIGN_OR_RAISE(out_arrays[i], Concatenate(arrays));
+          arrays[0], MakeArrayOfNull(output_schema->field(static_cast<int>(i))->type(),
+                                     /*length=*/0));
     }
+    out_arrays[i] = {std::move(arrays)};
   }
 
-  // The exec plan may reorder the output rows.  The tests are all setup to expect ouptut
-  // in ascending order of keys.  So we need to sort the result by the key columns.  To do
-  // that we create a table using the key columns, calculate the sort indices from that
-  // table (sorting on all fields) and then use those indices to calculate our result.
-  std::vector<std::shared_ptr<Field>> key_fields;
-  std::vector<std::shared_ptr<Array>> key_columns;
-  std::vector<SortKey> sort_keys;
-  for (std::size_t i = 0; i < key_names.size(); i++) {
-    const std::shared_ptr<Array>& arr = out_arrays[i + aggregates.size()];
-    if (arr->type_id() == Type::DICTIONARY) {
-      // Can't sort dictionary columns so need to decode
-      auto dict_arr = checked_pointer_cast<DictionaryArray>(arr);
-      ARROW_ASSIGN_OR_RAISE(auto decoded_arr,
-                            Take(*dict_arr->dictionary(), *dict_arr->indices()));
-      key_columns.push_back(decoded_arr);
-      key_fields.push_back(
-          field("name_does_not_matter", dict_arr->dict_type()->value_type()));
-    } else {
-      key_columns.push_back(arr);
-      key_fields.push_back(field("name_does_not_matter", arr->type()));
+  if (segmented && segment_key_names.size() > 0) {
+    ArrayVector struct_arrays;
+    struct_arrays.reserve(output_batches.size());
+    for (size_t j = 0; j < output_batches.size(); ++j) {
+      ArrayVector struct_fields;
+      struct_fields.reserve(out_arrays.size());
+      for (auto out_array : out_arrays) {
+        struct_fields.push_back(out_array[j]);
+      }
+      ARROW_ASSIGN_OR_RAISE(auto struct_array,
+                            StructArray::Make(struct_fields, output_schema->fields()));
+      struct_arrays.push_back(struct_array);
     }
-    sort_keys.emplace_back(static_cast<int>(i));
+    return ChunkedArray::Make(struct_arrays);
+  } else {
+    ArrayVector struct_fields(out_arrays.size());
+    for (size_t i = 0; i < out_arrays.size(); ++i) {
+      ARROW_ASSIGN_OR_RAISE(struct_fields[i], Concatenate(out_arrays[i]));
+    }
+    return StructArray::Make(std::move(struct_fields), output_schema->fields());
   }
-  std::shared_ptr<Schema> key_schema = schema(std::move(key_fields));
-  std::shared_ptr<Table> key_table = Table::Make(std::move(key_schema), key_columns);
-  SortOptions sort_options(std::move(sort_keys));
-  ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Array> sort_indices,
-                        SortIndices(key_table, sort_options));
+}
 
-  ARROW_ASSIGN_OR_RAISE(
-      std::shared_ptr<Array> struct_arr,
-      StructArray::Make(std::move(out_arrays), output_schema->fields()));
+Result<Datum> RunGroupBy(const BatchesWithSchema& input,
+                         const std::vector<std::string>& key_names,
+                         const std::vector<std::string>& segment_key_names,
+                         const std::vector<Aggregate>& aggregates, bool use_threads,
+                         bool segmented = false, bool naive = false) {
+  if (segment_key_names.size() > 0) {
+    ARROW_ASSIGN_OR_RAISE(auto thread_pool, arrow::internal::ThreadPool::Make(1));
+    ExecContext seq_ctx(default_memory_pool(), thread_pool.get());
+    return RunGroupBy(input, key_names, segment_key_names, aggregates, &seq_ctx,
+                      use_threads, segmented, naive);
+  } else {
+    return RunGroupBy(input, key_names, segment_key_names, aggregates,
+                      threaded_exec_context(), use_threads, segmented, naive);
+  }
+}
 
-  return Take(struct_arr, sort_indices);
+Result<Datum> RunGroupBy(const BatchesWithSchema& input,
+                         const std::vector<std::string>& key_names,
+                         const std::vector<Aggregate>& aggregates, bool use_threads,
+                         bool segmented = false, bool naive = false) {
+  return RunGroupBy(input, key_names, {}, aggregates, use_threads, segmented);
 }
 
 /// Simpler overload where you can give the columns as datums
 Result<Datum> RunGroupBy(const std::vector<Datum>& arguments,
                          const std::vector<Datum>& keys,
-                         const std::vector<Aggregate>& aggregates,
-                         bool use_threads = false) {
+                         const std::vector<Datum>& segment_keys,
+                         const std::vector<Aggregate>& aggregates, bool use_threads,
+                         bool segmented = false, bool naive = false) {
   using arrow::compute::detail::ExecSpanIterator;
 
-  FieldVector scan_fields(arguments.size() + keys.size());
+  FieldVector scan_fields(arguments.size() + keys.size() + segment_keys.size());
   std::vector<std::string> key_names(keys.size());
+  std::vector<std::string> segment_key_names(segment_keys.size());
   for (size_t i = 0; i < arguments.size(); ++i) {
     auto name = std::string("agg_") + ToChars(i);
     scan_fields[i] = field(name, arguments[i].type());
   }
+  size_t base = arguments.size();
   for (size_t i = 0; i < keys.size(); ++i) {
     auto name = std::string("key_") + ToChars(i);
-    scan_fields[arguments.size() + i] = field(name, keys[i].type());
+    scan_fields[base + i] = field(name, keys[i].type());
     key_names[i] = std::move(name);
   }
+  base += keys.size();
+  size_t j = segmented ? keys.size() : keys.size();

Review Comment:
   Done.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rtpsw commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "rtpsw (via GitHub)" <gi...@apache.org>.
rtpsw commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1117885883


##########
cpp/src/arrow/compute/row/grouper.h:
##########
@@ -30,6 +30,49 @@
 namespace arrow {
 namespace compute {
 
+/// \brief A segment of contiguous rows for grouping
+struct ARROW_EXPORT GroupingSegment {
+  int64_t offset;
+  int64_t length;
+  bool is_open;
+  bool extends;
+};
+
+inline bool operator==(const GroupingSegment& segment1, const GroupingSegment& segment2) {
+  return segment1.offset == segment2.offset && segment1.length == segment2.length &&
+         segment1.is_open == segment2.is_open && segment1.extends == segment2.extends;
+}
+inline bool operator!=(const GroupingSegment& segment1, const GroupingSegment& segment2) {
+  return !(segment1 == segment2);
+}
+
+/// \brief Computes grouping segments for a batch. Each segment covers rows with identical
+/// values in the batch. The values in the batch are often selected as keys from a larger
+/// batch.

Review Comment:
   Documented.



##########
cpp/src/arrow/compute/row/grouper.cc:
##########
@@ -39,12 +43,521 @@
 namespace arrow {
 
 using internal::checked_cast;
+using internal::PrimitiveScalarBase;
 
 namespace compute {
 
 namespace {
 
-struct GrouperImpl : Grouper {
+constexpr uint32_t kNoGroupId = std::numeric_limits<uint32_t>::max();
+
+using group_id_t = std::remove_const<decltype(kNoGroupId)>::type;
+using GroupIdType = CTypeTraits<group_id_t>::ArrowType;
+auto group_id_type = std::make_shared<GroupIdType>();
+
+inline const uint8_t* GetValuesAsBytes(const ArrayData& data, int64_t offset = 0) {
+  DCHECK_GT(data.type->byte_width(), 0);
+  int64_t absolute_byte_offset = (data.offset + offset) * data.type->byte_width();
+  return data.GetValues<uint8_t>(1, absolute_byte_offset);
+}
+
+inline const uint8_t* GetValuesAsBytes(const ArraySpan& data, int64_t offset = 0) {
+  DCHECK_GT(data.type->byte_width(), 0);
+  int64_t absolute_byte_offset = (data.offset + offset) * data.type->byte_width();
+  return data.GetValues<uint8_t>(1, absolute_byte_offset);
+}
+
+template <typename Value>
+Status CheckForGetNextSegment(const std::vector<Value>& values, int64_t length,
+                              int64_t offset, const std::vector<TypeHolder>& key_types) {
+  if (offset < 0 || offset > length) {
+    return Status::Invalid("invalid grouping segmenter offset: ", offset);
+  }
+  if (values.size() != key_types.size()) {
+    return Status::Invalid("expected batch size ", key_types.size(), " but got ",
+                           values.size());
+  }
+  for (size_t i = 0; i < key_types.size(); i++) {
+    const auto& value = values[i];
+    const auto& key_type = key_types[i];
+    if (*value.type() != *key_type.type) {
+      return Status::Invalid("expected batch value ", i, " of type ", *key_type.type,
+                             " but got ", *value.type());
+    }
+  }
+  return Status::OK();
+}
+
+template <typename Batch>
+enable_if_t<std::is_same<Batch, ExecSpan>::value || std::is_same<Batch, ExecBatch>::value,
+            Status>
+CheckForGetNextSegment(const Batch& batch, int64_t offset,
+                       const std::vector<TypeHolder>& key_types) {
+  return CheckForGetNextSegment(batch.values, batch.length, offset, key_types);
+}
+
+struct BaseGroupingSegmenter : public GroupingSegmenter {
+  explicit BaseGroupingSegmenter(const std::vector<TypeHolder>& key_types)
+      : key_types_(key_types) {}
+
+  const std::vector<TypeHolder>& key_types() const override { return key_types_; }
+
+  std::vector<TypeHolder> key_types_;
+};
+
+GroupingSegment MakeSegment(int64_t batch_length, int64_t offset, int64_t length,
+                            bool extends) {
+  return GroupingSegment{offset, length, offset + length >= batch_length, extends};
+}
+
+int64_t GetMatchLength(const uint8_t* match_bytes, int64_t match_width,
+                       const uint8_t* array_bytes, int64_t offset, int64_t length) {
+  int64_t cursor, byte_cursor;
+  for (cursor = offset, byte_cursor = match_width * cursor; cursor < length;
+       cursor++, byte_cursor += match_width) {
+    if (memcmp(match_bytes, array_bytes + byte_cursor,
+               static_cast<size_t>(match_width)) != 0) {
+      break;
+    }
+  }
+  return std::min(cursor, length - offset);
+}
+
+using ExtendFunc = std::function<bool(const void*)>;
+constexpr bool kDefaultExtends = true;
+constexpr bool kEmptyExtends = true;
+
+Result<GroupingSegment> GetNextSegmentChunked(
+    const std::shared_ptr<ChunkedArray>& chunked_array, int64_t offset,
+    ExtendFunc extend) {
+  if (offset >= chunked_array->length()) {
+    return MakeSegment(chunked_array->length(), chunked_array->length(), 0,
+                       kEmptyExtends);
+  }
+  int64_t remaining_offset = offset;
+  const auto& arrays = chunked_array->chunks();
+  for (size_t i = 0; remaining_offset >= 0 && i < arrays.size(); i++) {
+    // look up chunk containing offset
+    int64_t array_length = arrays[i]->length();
+    if (remaining_offset < array_length) {
+      // found - switch to matching
+      int64_t match_width = arrays[i]->type()->byte_width();
+      const uint8_t* match_bytes = GetValuesAsBytes(*arrays[i]->data(), remaining_offset);
+      int64_t total_match_length = 0;
+      for (; i < arrays.size(); i++) {
+        int64_t array_length = arrays[i]->length();
+        if (array_length <= 0) continue;
+        const uint8_t* array_bytes = GetValuesAsBytes(*arrays[i]->data());
+        int64_t match_length = GetMatchLength(match_bytes, match_width, array_bytes,
+                                              remaining_offset, array_length);
+        total_match_length += match_length;
+        remaining_offset = 0;
+        if (match_length < array_length - remaining_offset) break;
+      }
+      bool extends = extend(match_bytes);
+      return MakeSegment(chunked_array->length(), offset, total_match_length, extends);
+    }
+    remaining_offset -= array_length;
+  }
+  return Status::Invalid("segmenting invalid chunked array value");
+}
+
+struct NoKeysGroupingSegmenter : public BaseGroupingSegmenter {
+  static std::unique_ptr<GroupingSegmenter> Make() {
+    return std::make_unique<NoKeysGroupingSegmenter>();
+  }
+
+  NoKeysGroupingSegmenter() : BaseGroupingSegmenter({}) {}
+
+  Status Reset() override { return Status::OK(); }
+
+  template <typename Batch>
+  Result<GroupingSegment> GetNextSegmentImpl(const Batch& batch, int64_t offset) {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {}));
+    return MakeSegment(batch.length, offset, batch.length - offset, kDefaultExtends);
+  }
+
+  Result<GroupingSegment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    return GetNextSegmentImpl(batch, offset);
+  }
+
+  Result<GroupingSegment> GetNextSegment(const ExecBatch& batch,
+                                         int64_t offset) override {
+    return GetNextSegmentImpl(batch, offset);
+  }
+};
+
+struct SimpleKeyGroupingSegmenter : public BaseGroupingSegmenter {
+  static Result<std::unique_ptr<GroupingSegmenter>> Make(TypeHolder key_type) {
+    return std::make_unique<SimpleKeyGroupingSegmenter>(key_type);
+  }
+
+  explicit SimpleKeyGroupingSegmenter(TypeHolder key_type)
+      : BaseGroupingSegmenter({key_type}), key_type_(key_types_[0]), save_key_data_() {}
+
+  Status CheckType(const DataType& type) {
+    if (!is_fixed_width(type)) {
+      return Status::Invalid("SimpleKeyGroupingSegmenter does not support type ", type);
+    }
+    return Status::OK();
+  }
+
+  Status Reset() override {
+    save_key_data_.resize(0);
+    return Status::OK();
+  }
+
+  bool Extend(const void* data) {
+    size_t byte_width = static_cast<size_t>(key_type_.type->byte_width());
+    bool extends = save_key_data_.size() != byte_width
+                       ? kDefaultExtends
+                       : 0 == memcmp(save_key_data_.data(), data, byte_width);
+    save_key_data_.resize(byte_width);
+    memcpy(save_key_data_.data(), data, byte_width);
+    return extends;
+  }
+
+  Result<GroupingSegment> GetNextSegment(const Scalar& scalar, int64_t offset,
+                                         int64_t length) {
+    ARROW_RETURN_NOT_OK(CheckType(*scalar.type));
+    if (!scalar.is_valid) {
+      return Status::Invalid("segmenting an invalid scalar");
+    }
+    auto data = checked_cast<const PrimitiveScalarBase&>(scalar).data();
+    bool extends = length > 0 ? Extend(data) : kEmptyExtends;
+    return MakeSegment(length, 0, length, extends);
+  }
+
+  Result<GroupingSegment> GetNextSegment(const DataType& array_type,
+                                         const uint8_t* array_bytes, int64_t offset,
+                                         int64_t length) {
+    RETURN_NOT_OK(CheckType(array_type));
+    int64_t byte_width = array_type.byte_width();
+    int64_t match_length = GetMatchLength(array_bytes + offset * byte_width, byte_width,
+                                          array_bytes, offset, length);
+    bool extends = length > 0 ? Extend(array_bytes + offset * byte_width) : kEmptyExtends;
+    return MakeSegment(length, offset, match_length, extends);
+  }
+
+  Result<GroupingSegment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {key_type_}));
+    if (offset == batch.length) {
+      return MakeSegment(batch.length, offset, 0, kEmptyExtends);
+    }
+    const auto& value = batch.values[0];
+    if (value.is_scalar()) {
+      return GetNextSegment(*value.scalar, offset, batch.length);
+    }
+    ARROW_DCHECK(value.is_array());
+    const auto& array = value.array;
+    if (array.GetNullCount() > 0) {
+      return Status::NotImplemented("segmenting a nullable array");
+    }
+    return GetNextSegment(*array.type, GetValuesAsBytes(array), offset, batch.length);
+  }
+
+  Result<GroupingSegment> GetNextSegment(const ExecBatch& batch,
+                                         int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {key_type_}));
+    if (offset == batch.length) {
+      return MakeSegment(batch.length, offset, 0, kEmptyExtends);
+    }
+    const auto& value = batch.values[0];
+    if (value.is_scalar()) {
+      return GetNextSegment(*value.scalar(), offset, batch.length);
+    }
+    if (value.is_array()) {
+      auto array = value.array();
+      if (array->GetNullCount() > 0) {
+        return Status::NotImplemented("segmenting a nullable array");
+      }
+      return GetNextSegment(*array->type, GetValuesAsBytes(*array), offset, batch.length);
+    }
+    if (value.is_chunked_array()) {
+      auto array = value.chunked_array();
+      if (array->null_count() > 0) {
+        return Status::NotImplemented("segmenting a nullable array");
+      }
+      return GetNextSegmentChunked(array, offset, bound_extend_);
+    }
+    return Status::Invalid("segmenting unsupported value kind ", value.kind());
+  }
+
+ private:
+  TypeHolder key_type_;
+  std::vector<uint8_t> save_key_data_;
+  ExtendFunc bound_extend_ = [this](const void* data) { return Extend(data); };

Review Comment:
   Done.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rtpsw commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "rtpsw (via GitHub)" <gi...@apache.org>.
rtpsw commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1117886790


##########
cpp/src/arrow/compute/exec/aggregate_node.cc:
##########
@@ -326,46 +446,86 @@ class ScalarAggregateNode : public ExecNode, public TracedNode {
   }
 
  private:
-  Status Finish() {
-    auto scope = TraceFinish();
+  Status ReconstructAggregates() {

Review Comment:
   Done.



##########
cpp/src/arrow/compute/exec/aggregate_node.cc:
##########
@@ -326,46 +446,86 @@ class ScalarAggregateNode : public ExecNode, public TracedNode {
   }
 
  private:
-  Status Finish() {
-    auto scope = TraceFinish();
+  Status ReconstructAggregates() {
+    const auto& input_schema = *inputs()[0]->output_schema();
+    auto exec_ctx = plan()->query_context()->exec_context();
+    for (size_t i = 0; i < kernels_.size(); ++i) {
+      std::vector<TypeHolder> in_types;
+      for (const auto& target : target_fieldsets_[i]) {
+        in_types.emplace_back(input_schema.field(target)->type().get());
+      }
+      states_[i].resize(plan()->query_context()->max_concurrency());
+      KernelContext kernel_ctx{exec_ctx};
+      RETURN_NOT_OK(Kernel::InitAll(
+          &kernel_ctx, KernelInitArgs{kernels_[i], in_types, aggs_[i].options.get()},
+          &states_[i]));
+    }
+    return Status::OK();
+  }
+
+  Status OutputResult(bool is_last = false, bool traced = false) {
+    if (is_last && !traced) {
+      auto scope = TraceFinish();

Review Comment:
   Done.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rtpsw commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "rtpsw (via GitHub)" <gi...@apache.org>.
rtpsw commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1117886531


##########
cpp/src/arrow/compute/exec/aggregate_node.cc:
##########
@@ -169,17 +206,63 @@ void AggregatesToString(std::stringstream* ss, const Schema& input_schema,
   *ss << ']';
 }
 
+template <typename BatchHandler>
+Status HandleSegments(std::unique_ptr<GroupingSegmenter>& segmenter,
+                      const ExecBatch& batch, const std::vector<int>& ids,
+                      const BatchHandler& handle_batch) {
+  int64_t offset = 0;
+  ARROW_ASSIGN_OR_RAISE(auto segment_batch, batch.SelectValues(ids));
+  while (true) {
+    ARROW_ASSIGN_OR_RAISE(auto segment, segmenter->GetNextSegment(segment_batch, offset));
+    if (segment.offset >= segment_batch.length) break;
+    ARROW_RETURN_NOT_OK(handle_batch(batch, segment));
+    offset = segment.offset + segment.length;
+  }
+  return Status::OK();
+}
+
+Status GetScalarFields(std::vector<Datum>& values, const ExecBatch& input_batch,

Review Comment:
   Changed to pointer.
   
   > Do we expect this to modify "values"? The function is called "GetScalarFields" which seems like a non-modifying method
   
   Yes, it modifies `values`. What function name do you think would be better?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rtpsw commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "rtpsw (via GitHub)" <gi...@apache.org>.
rtpsw commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1117886935


##########
cpp/src/arrow/compute/row/grouper.h:
##########
@@ -30,6 +30,49 @@
 namespace arrow {
 namespace compute {
 
+/// \brief A segment of contiguous rows for grouping
+struct ARROW_EXPORT GroupingSegment {

Review Comment:
   Done.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] icexelloss commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "icexelloss (via GitHub)" <gi...@apache.org>.
icexelloss commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1116056377


##########
cpp/src/arrow/compute/row/grouper.h:
##########
@@ -30,6 +30,49 @@
 namespace arrow {
 namespace compute {
 
+/// \brief A segment of contiguous rows for grouping
+struct ARROW_EXPORT GroupingSegment {
+  int64_t offset;
+  int64_t length;
+  bool is_open;
+  bool extends;
+};
+
+inline bool operator==(const GroupingSegment& segment1, const GroupingSegment& segment2) {
+  return segment1.offset == segment2.offset && segment1.length == segment2.length &&
+         segment1.is_open == segment2.is_open && segment1.extends == segment2.extends;
+}
+inline bool operator!=(const GroupingSegment& segment1, const GroupingSegment& segment2) {
+  return !(segment1 == segment2);
+}
+
+/// \brief Computes grouping segments for a batch. Each segment covers rows with identical
+/// values in the batch. The values in the batch are often selected as keys from a larger
+/// batch.
+class ARROW_EXPORT GroupingSegmenter {
+ public:
+  virtual ~GroupingSegmenter() = default;
+
+  /// \brief Construct a GroupingSegmenter which receives the specified key types
+  static Result<std::unique_ptr<GroupingSegmenter>> Make(
+      const std::vector<TypeHolder>& key_types, bool nullable_keys = false,
+      ExecContext* ctx = default_exec_context());
+
+  /// \brief Return the key types of this segmenter
+  virtual const std::vector<TypeHolder>& key_types() const = 0;
+
+  /// \brief Reset this grouping segmenter
+  virtual Status Reset() = 0;
+
+  /// \brief Get the next segment for the given batch starting from the given offset
+  virtual Result<GroupingSegment> GetNextSegment(const ExecSpan& batch,
+                                                 int64_t offset) = 0;
+
+  /// \brief Get the next segment for the given batch starting from the given offset
+  virtual Result<GroupingSegment> GetNextSegment(const ExecBatch& batch,

Review Comment:
   What is the difference between ExecSpan and ExecBatch and why do we need to handle both?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] icexelloss commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "icexelloss (via GitHub)" <gi...@apache.org>.
icexelloss commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1116054419


##########
cpp/src/arrow/compute/row/grouper.h:
##########
@@ -30,6 +30,49 @@
 namespace arrow {
 namespace compute {
 
+/// \brief A segment of contiguous rows for grouping
+struct ARROW_EXPORT GroupingSegment {
+  int64_t offset;
+  int64_t length;
+  bool is_open;
+  bool extends;
+};
+
+inline bool operator==(const GroupingSegment& segment1, const GroupingSegment& segment2) {
+  return segment1.offset == segment2.offset && segment1.length == segment2.length &&
+         segment1.is_open == segment2.is_open && segment1.extends == segment2.extends;
+}
+inline bool operator!=(const GroupingSegment& segment1, const GroupingSegment& segment2) {
+  return !(segment1 == segment2);
+}
+
+/// \brief Computes grouping segments for a batch. Each segment covers rows with identical
+/// values in the batch. The values in the batch are often selected as keys from a larger
+/// batch.
+class ARROW_EXPORT GroupingSegmenter {
+ public:
+  virtual ~GroupingSegmenter() = default;
+
+  /// \brief Construct a GroupingSegmenter which receives the specified key types
+  static Result<std::unique_ptr<GroupingSegmenter>> Make(
+      const std::vector<TypeHolder>& key_types, bool nullable_keys = false,
+      ExecContext* ctx = default_exec_context());

Review Comment:
   I feel defaults like this can make it easier to make mistakes, i.e., the caller has a ctx that is not default_exec_context() and forget to pass that in.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rtpsw commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "rtpsw (via GitHub)" <gi...@apache.org>.
rtpsw commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1122260682


##########
cpp/src/arrow/compute/exec/aggregate_node.cc:
##########
@@ -169,35 +186,117 @@ void AggregatesToString(std::stringstream* ss, const Schema& input_schema,
   *ss << ']';
 }
 
+template <typename BatchHandler>
+Status HandleSegments(std::unique_ptr<GroupingSegmenter>& segmenter,
+                      const ExecBatch& batch, const std::vector<int>& ids,
+                      const BatchHandler& handle_batch) {
+  int64_t offset = 0;
+  ARROW_ASSIGN_OR_RAISE(auto segment_exec_batch, batch.SelectValues(ids));
+  ExecSpan segment_batch(segment_exec_batch);
+  while (true) {
+    ARROW_ASSIGN_OR_RAISE(auto segment, segmenter->GetNextSegment(segment_batch, offset));
+    if (segment.offset >= segment_batch.length) break;  // condition of no-next-segment
+    ARROW_RETURN_NOT_OK(handle_batch(batch, segment));
+    offset = segment.offset + segment.length;
+  }
+  return Status::OK();
+}
+
+Status GetScalarFields(std::vector<Datum>* values_ptr, const ExecBatch& input_batch,
+                       const std::vector<int>& field_ids) {
+  DCHECK_GT(input_batch.length, 0);
+  std::vector<Datum>& values = *values_ptr;
+  int64_t row = input_batch.length - 1;
+  values.clear();
+  values.resize(field_ids.size());
+  for (size_t i = 0; i < field_ids.size(); i++) {
+    const Datum& value = input_batch.values[field_ids[i]];
+    if (value.is_scalar()) {
+      values[i] = value;
+    } else if (value.is_array()) {
+      ARROW_ASSIGN_OR_RAISE(auto scalar, value.make_array()->GetScalar(row));
+      values[i] = scalar;
+    } else {
+      DCHECK(false);
+    }
+  }
+  return Status::OK();
+}
+
+void PlaceFields(ExecBatch& batch, size_t base, std::vector<Datum>& values) {
+  DCHECK_LE(base + values.size(), batch.values.size());
+  for (size_t i = 0; i < values.size(); i++) {
+    batch.values[base + i] = values[i];
+  }
+}
+
 class ScalarAggregateNode : public ExecNode, public TracedNode {
  public:
   ScalarAggregateNode(ExecPlan* plan, std::vector<ExecNode*> inputs,
                       std::shared_ptr<Schema> output_schema,
+                      std::unique_ptr<GroupingSegmenter> segmenter,
+                      std::vector<int> segment_field_ids,
                       std::vector<std::vector<int>> target_fieldsets,
                       std::vector<Aggregate> aggs,
                       std::vector<const ScalarAggregateKernel*> kernels,
                       std::vector<std::vector<std::unique_ptr<KernelState>>> states)
       : ExecNode(plan, std::move(inputs), {"target"},
                  /*output_schema=*/std::move(output_schema)),
         TracedNode(this),
+        segmenter_(std::move(segmenter)),
+        segment_field_ids_(std::move(segment_field_ids)),
         target_fieldsets_(std::move(target_fieldsets)),
         aggs_(std::move(aggs)),
         kernels_(std::move(kernels)),
-        states_(std::move(states)) {}
+        states_(std::move(states)) {
+    const auto& input_schema = *this->inputs()[0]->output_schema();
+    for (size_t i = 0; i < kernels_.size(); ++i) {
+      std::vector<TypeHolder> in_types;
+      for (const auto& target : target_fieldsets_[i]) {
+        in_types.emplace_back(input_schema.field(target)->type().get());
+      }
+      in_typesets_.push_back(std::move(in_types));
+    }
+  }
 
   static Result<ExecNode*> Make(ExecPlan* plan, std::vector<ExecNode*> inputs,
                                 const ExecNodeOptions& options) {
     RETURN_NOT_OK(ValidateExecNodeInputs(plan, inputs, 1, "ScalarAggregateNode"));
 
     const auto& aggregate_options = checked_cast<const AggregateNodeOptions&>(options);
     auto aggregates = aggregate_options.aggregates;
+    const auto& keys = aggregate_options.keys;
+    const auto& segment_keys = aggregate_options.segment_keys;
+
+    if (keys.size() > 0) {
+      return Status::Invalid("Scalar aggregation with some key");
+    }
+    if (plan->query_context()->exec_context()->executor()->GetCapacity() > 1 &&
+        segment_keys.size() > 0) {
+      return Status::NotImplemented("Segmented aggregation in a multi-threaded plan");
+    }
 
     const auto& input_schema = *inputs[0]->output_schema();
     auto exec_ctx = plan->query_context()->exec_context();
 
+    std::vector<int> segment_field_ids(segment_keys.size());
+    std::vector<TypeHolder> segment_key_types(segment_keys.size());
+    for (size_t i = 0; i < segment_keys.size(); i++) {
+      ARROW_ASSIGN_OR_RAISE(auto match, segment_keys[i].FindOne(input_schema));
+      if (match.indices().size() > 1) {
+        // ARROW-18369: Support nested references as segment ids

Review Comment:
   Nested references is a reference to a nested struct. It refers to `FieldRef` with a multi-component path, e.g. `[0, 2, 1]` selects the index-0 member, within it the index-2 field, and within it the index-1 field. The ARROW issue mentioned in the comment leads to [this post](https://github.com/apache/arrow/pull/14352#discussion_r1026945315).



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rtpsw commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "rtpsw (via GitHub)" <gi...@apache.org>.
rtpsw commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1122301127


##########
cpp/src/arrow/compute/row/grouper.cc:
##########
@@ -39,12 +43,336 @@
 namespace arrow {
 
 using internal::checked_cast;
+using internal::PrimitiveScalarBase;
 
 namespace compute {
 
 namespace {
 
-struct GrouperImpl : Grouper {
+constexpr uint32_t kNoGroupId = std::numeric_limits<uint32_t>::max();
+
+using group_id_t = std::remove_const<decltype(kNoGroupId)>::type;
+using GroupIdType = CTypeTraits<group_id_t>::ArrowType;
+auto group_id_type = std::make_shared<GroupIdType>();

Review Comment:
   For some reason, the above suggestion runs into a warning `returning reference to temporary`. I'll go with `g_group_id_type`.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] icexelloss commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "icexelloss (via GitHub)" <gi...@apache.org>.
icexelloss commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1123383479


##########
cpp/src/arrow/compute/exec/aggregate_node.cc:
##########
@@ -169,35 +186,117 @@ void AggregatesToString(std::stringstream* ss, const Schema& input_schema,
   *ss << ']';
 }
 
+template <typename BatchHandler>
+Status HandleSegments(std::unique_ptr<GroupingSegmenter>& segmenter,
+                      const ExecBatch& batch, const std::vector<int>& ids,
+                      const BatchHandler& handle_batch) {
+  int64_t offset = 0;
+  ARROW_ASSIGN_OR_RAISE(auto segment_exec_batch, batch.SelectValues(ids));
+  ExecSpan segment_batch(segment_exec_batch);
+  while (true) {
+    ARROW_ASSIGN_OR_RAISE(auto segment, segmenter->GetNextSegment(segment_batch, offset));
+    if (segment.offset >= segment_batch.length) break;  // condition of no-next-segment
+    ARROW_RETURN_NOT_OK(handle_batch(batch, segment));
+    offset = segment.offset + segment.length;
+  }
+  return Status::OK();
+}
+
+Status GetScalarFields(std::vector<Datum>* values_ptr, const ExecBatch& input_batch,
+                       const std::vector<int>& field_ids) {
+  DCHECK_GT(input_batch.length, 0);
+  std::vector<Datum>& values = *values_ptr;
+  int64_t row = input_batch.length - 1;
+  values.clear();
+  values.resize(field_ids.size());
+  for (size_t i = 0; i < field_ids.size(); i++) {
+    const Datum& value = input_batch.values[field_ids[i]];
+    if (value.is_scalar()) {
+      values[i] = value;
+    } else if (value.is_array()) {
+      ARROW_ASSIGN_OR_RAISE(auto scalar, value.make_array()->GetScalar(row));
+      values[i] = scalar;
+    } else {
+      DCHECK(false);
+    }
+  }
+  return Status::OK();
+}
+
+void PlaceFields(ExecBatch& batch, size_t base, std::vector<Datum>& values) {
+  DCHECK_LE(base + values.size(), batch.values.size());
+  for (size_t i = 0; i < values.size(); i++) {
+    batch.values[base + i] = values[i];
+  }
+}
+
 class ScalarAggregateNode : public ExecNode, public TracedNode {
  public:
   ScalarAggregateNode(ExecPlan* plan, std::vector<ExecNode*> inputs,
                       std::shared_ptr<Schema> output_schema,
+                      std::unique_ptr<GroupingSegmenter> segmenter,
+                      std::vector<int> segment_field_ids,
                       std::vector<std::vector<int>> target_fieldsets,
                       std::vector<Aggregate> aggs,
                       std::vector<const ScalarAggregateKernel*> kernels,
                       std::vector<std::vector<std::unique_ptr<KernelState>>> states)
       : ExecNode(plan, std::move(inputs), {"target"},
                  /*output_schema=*/std::move(output_schema)),
         TracedNode(this),
+        segmenter_(std::move(segmenter)),
+        segment_field_ids_(std::move(segment_field_ids)),
         target_fieldsets_(std::move(target_fieldsets)),
         aggs_(std::move(aggs)),
         kernels_(std::move(kernels)),
-        states_(std::move(states)) {}
+        states_(std::move(states)) {
+    const auto& input_schema = *this->inputs()[0]->output_schema();
+    for (size_t i = 0; i < kernels_.size(); ++i) {
+      std::vector<TypeHolder> in_types;
+      for (const auto& target : target_fieldsets_[i]) {
+        in_types.emplace_back(input_schema.field(target)->type().get());
+      }
+      in_typesets_.push_back(std::move(in_types));
+    }
+  }
 
   static Result<ExecNode*> Make(ExecPlan* plan, std::vector<ExecNode*> inputs,
                                 const ExecNodeOptions& options) {
     RETURN_NOT_OK(ValidateExecNodeInputs(plan, inputs, 1, "ScalarAggregateNode"));
 
     const auto& aggregate_options = checked_cast<const AggregateNodeOptions&>(options);
     auto aggregates = aggregate_options.aggregates;
+    const auto& keys = aggregate_options.keys;
+    const auto& segment_keys = aggregate_options.segment_keys;
+
+    if (keys.size() > 0) {
+      return Status::Invalid("Scalar aggregation with some key");
+    }
+    if (plan->query_context()->exec_context()->executor()->GetCapacity() > 1 &&
+        segment_keys.size() > 0) {
+      return Status::NotImplemented("Segmented aggregation in a multi-threaded plan");
+    }
 
     const auto& input_schema = *inputs[0]->output_schema();
     auto exec_ctx = plan->query_context()->exec_context();
 
+    std::vector<int> segment_field_ids(segment_keys.size());
+    std::vector<TypeHolder> segment_key_types(segment_keys.size());
+    for (size_t i = 0; i < segment_keys.size(); i++) {
+      ARROW_ASSIGN_OR_RAISE(auto match, segment_keys[i].FindOne(input_schema));
+      if (match.indices().size() > 1) {
+        // ARROW-18369: Support nested references as segment ids
+        return Status::Invalid("Nested references cannot be used as segment ids");
+      }
+      segment_field_ids[i] = match[0];
+      segment_key_types[i] = input_schema.field(match[0])->type().get();
+    }
+
+    ARROW_ASSIGN_OR_RAISE(
+        auto segmenter, GroupingSegmenter::Make(std::move(segment_key_types), exec_ctx));

Review Comment:
   Yeah I agree we should document and also add checks in the code if `null` segment key appears.
   
   I agree also to leave the handling of `null` segment key as follow-ups. From our use cases, this is not likely to happen because the data we have don't have "null" time values.



##########
cpp/src/arrow/compute/exec/aggregate_node.cc:
##########
@@ -169,35 +186,117 @@ void AggregatesToString(std::stringstream* ss, const Schema& input_schema,
   *ss << ']';
 }
 
+template <typename BatchHandler>
+Status HandleSegments(std::unique_ptr<GroupingSegmenter>& segmenter,
+                      const ExecBatch& batch, const std::vector<int>& ids,
+                      const BatchHandler& handle_batch) {
+  int64_t offset = 0;
+  ARROW_ASSIGN_OR_RAISE(auto segment_exec_batch, batch.SelectValues(ids));
+  ExecSpan segment_batch(segment_exec_batch);
+  while (true) {
+    ARROW_ASSIGN_OR_RAISE(auto segment, segmenter->GetNextSegment(segment_batch, offset));
+    if (segment.offset >= segment_batch.length) break;  // condition of no-next-segment
+    ARROW_RETURN_NOT_OK(handle_batch(batch, segment));
+    offset = segment.offset + segment.length;
+  }
+  return Status::OK();
+}
+
+Status GetScalarFields(std::vector<Datum>* values_ptr, const ExecBatch& input_batch,
+                       const std::vector<int>& field_ids) {
+  DCHECK_GT(input_batch.length, 0);
+  std::vector<Datum>& values = *values_ptr;
+  int64_t row = input_batch.length - 1;
+  values.clear();
+  values.resize(field_ids.size());
+  for (size_t i = 0; i < field_ids.size(); i++) {
+    const Datum& value = input_batch.values[field_ids[i]];
+    if (value.is_scalar()) {
+      values[i] = value;
+    } else if (value.is_array()) {
+      ARROW_ASSIGN_OR_RAISE(auto scalar, value.make_array()->GetScalar(row));
+      values[i] = scalar;
+    } else {
+      DCHECK(false);
+    }
+  }
+  return Status::OK();
+}
+
+void PlaceFields(ExecBatch& batch, size_t base, std::vector<Datum>& values) {
+  DCHECK_LE(base + values.size(), batch.values.size());
+  for (size_t i = 0; i < values.size(); i++) {
+    batch.values[base + i] = values[i];
+  }
+}
+
 class ScalarAggregateNode : public ExecNode, public TracedNode {
  public:
   ScalarAggregateNode(ExecPlan* plan, std::vector<ExecNode*> inputs,
                       std::shared_ptr<Schema> output_schema,
+                      std::unique_ptr<GroupingSegmenter> segmenter,
+                      std::vector<int> segment_field_ids,
                       std::vector<std::vector<int>> target_fieldsets,
                       std::vector<Aggregate> aggs,
                       std::vector<const ScalarAggregateKernel*> kernels,
                       std::vector<std::vector<std::unique_ptr<KernelState>>> states)
       : ExecNode(plan, std::move(inputs), {"target"},
                  /*output_schema=*/std::move(output_schema)),
         TracedNode(this),
+        segmenter_(std::move(segmenter)),
+        segment_field_ids_(std::move(segment_field_ids)),
         target_fieldsets_(std::move(target_fieldsets)),
         aggs_(std::move(aggs)),
         kernels_(std::move(kernels)),
-        states_(std::move(states)) {}
+        states_(std::move(states)) {
+    const auto& input_schema = *this->inputs()[0]->output_schema();
+    for (size_t i = 0; i < kernels_.size(); ++i) {
+      std::vector<TypeHolder> in_types;
+      for (const auto& target : target_fieldsets_[i]) {
+        in_types.emplace_back(input_schema.field(target)->type().get());
+      }
+      in_typesets_.push_back(std::move(in_types));
+    }
+  }
 
   static Result<ExecNode*> Make(ExecPlan* plan, std::vector<ExecNode*> inputs,
                                 const ExecNodeOptions& options) {
     RETURN_NOT_OK(ValidateExecNodeInputs(plan, inputs, 1, "ScalarAggregateNode"));
 
     const auto& aggregate_options = checked_cast<const AggregateNodeOptions&>(options);
     auto aggregates = aggregate_options.aggregates;
+    const auto& keys = aggregate_options.keys;
+    const auto& segment_keys = aggregate_options.segment_keys;
+
+    if (keys.size() > 0) {
+      return Status::Invalid("Scalar aggregation with some key");
+    }
+    if (plan->query_context()->exec_context()->executor()->GetCapacity() > 1 &&
+        segment_keys.size() > 0) {
+      return Status::NotImplemented("Segmented aggregation in a multi-threaded plan");
+    }
 
     const auto& input_schema = *inputs[0]->output_schema();
     auto exec_ctx = plan->query_context()->exec_context();
 
+    std::vector<int> segment_field_ids(segment_keys.size());
+    std::vector<TypeHolder> segment_key_types(segment_keys.size());
+    for (size_t i = 0; i < segment_keys.size(); i++) {
+      ARROW_ASSIGN_OR_RAISE(auto match, segment_keys[i].FindOne(input_schema));
+      if (match.indices().size() > 1) {
+        // ARROW-18369: Support nested references as segment ids
+        return Status::Invalid("Nested references cannot be used as segment ids");
+      }
+      segment_field_ids[i] = match[0];
+      segment_key_types[i] = input_schema.field(match[0])->type().get();
+    }
+
+    ARROW_ASSIGN_OR_RAISE(
+        auto segmenter, GroupingSegmenter::Make(std::move(segment_key_types), exec_ctx));

Review Comment:
   Yeah I agree we should document and also add checks to error out in the code if `null` segment key appears.
   
   I agree also to leave the handling of `null` segment key as follow-ups. From our use cases, this is not likely to happen because the data we have don't have "null" time values.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] icexelloss commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "icexelloss (via GitHub)" <gi...@apache.org>.
icexelloss commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1127010345


##########
cpp/src/arrow/compute/row/grouper.cc:
##########
@@ -39,12 +43,330 @@
 namespace arrow {
 
 using internal::checked_cast;
+using internal::PrimitiveScalarBase;
 
 namespace compute {
 
 namespace {
 
-struct GrouperImpl : Grouper {
+constexpr uint32_t kNoGroupId = std::numeric_limits<uint32_t>::max();
+
+using group_id_t = std::remove_const<decltype(kNoGroupId)>::type;
+using GroupIdType = CTypeTraits<group_id_t>::ArrowType;
+auto g_group_id_type = std::make_shared<GroupIdType>();
+
+inline const uint8_t* GetValuesAsBytes(const ArraySpan& data, int64_t offset = 0) {
+  DCHECK_GT(data.type->byte_width(), 0);
+  int64_t absolute_byte_offset = (data.offset + offset) * data.type->byte_width();
+  return data.GetValues<uint8_t>(1, absolute_byte_offset);
+}
+
+template <typename Value>
+Status CheckForGetNextSegment(const std::vector<Value>& values, int64_t length,
+                              int64_t offset, const std::vector<TypeHolder>& key_types) {
+  if (offset < 0 || offset > length) {
+    return Status::Invalid("invalid grouping segmenter offset: ", offset);
+  }
+  if (values.size() != key_types.size()) {
+    return Status::Invalid("expected batch size ", key_types.size(), " but got ",
+                           values.size());
+  }
+  for (size_t i = 0; i < key_types.size(); i++) {
+    const auto& value = values[i];
+    const auto& key_type = key_types[i];
+    if (*value.type() != *key_type.type) {
+      return Status::Invalid("expected batch value ", i, " of type ", *key_type.type,
+                             " but got ", *value.type());
+    }
+  }
+  return Status::OK();
+}
+
+template <typename Batch>
+enable_if_t<std::is_same<Batch, ExecSpan>::value || std::is_same<Batch, ExecBatch>::value,
+            Status>
+CheckForGetNextSegment(const Batch& batch, int64_t offset,
+                       const std::vector<TypeHolder>& key_types) {
+  return CheckForGetNextSegment(batch.values, batch.length, offset, key_types);
+}
+
+struct BaseRowSegmenter : public RowSegmenter {
+  explicit BaseRowSegmenter(const std::vector<TypeHolder>& key_types)
+      : key_types_(key_types) {}
+
+  const std::vector<TypeHolder>& key_types() const override { return key_types_; }
+
+  std::vector<TypeHolder> key_types_;
+};
+
+Segment MakeSegment(int64_t batch_length, int64_t offset, int64_t length, bool extends) {
+  return Segment{offset, length, offset + length >= batch_length, extends};
+}
+
+int64_t GetMatchLength(const uint8_t* match_bytes, int64_t match_width,
+                       const uint8_t* array_bytes, int64_t offset, int64_t length) {
+  int64_t cursor, byte_cursor;
+  for (cursor = offset, byte_cursor = match_width * cursor; cursor < length;
+       cursor++, byte_cursor += match_width) {
+    if (memcmp(match_bytes, array_bytes + byte_cursor,
+               static_cast<size_t>(match_width)) != 0) {
+      break;
+    }
+  }
+  return std::min(cursor, length) - offset;
+}
+
+using ExtendFunc = std::function<bool(const void*)>;
+constexpr bool kDefaultExtends = true;
+constexpr bool kEmptyExtends = true;
+
+struct NoKeysSegmenter : public BaseRowSegmenter {
+  static std::unique_ptr<RowSegmenter> Make() {
+    return std::make_unique<NoKeysSegmenter>();
+  }
+
+  NoKeysSegmenter() : BaseRowSegmenter({}) {}
+
+  Status Reset() override { return Status::OK(); }
+
+  Result<Segment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {}));
+    return MakeSegment(batch.length, offset, batch.length - offset, kDefaultExtends);
+  }
+};
+
+struct SimpleKeySegmenter : public BaseRowSegmenter {
+  static Result<std::unique_ptr<RowSegmenter>> Make(TypeHolder key_type) {
+    return std::make_unique<SimpleKeySegmenter>(key_type);
+  }
+
+  explicit SimpleKeySegmenter(TypeHolder key_type)
+      : BaseRowSegmenter({key_type}), key_type_(key_types_[0]), save_key_data_() {}
+
+  Status CheckType(const DataType& type) {
+    if (!is_fixed_width(type)) {
+      return Status::Invalid("SimpleKeySegmenter does not support type ", type);
+    }
+    return Status::OK();
+  }
+
+  Status Reset() override {
+    save_key_data_.resize(0);
+    return Status::OK();
+  }
+
+  // Checks whether the given grouping data extends the current segment, i.e., is equal to
+  // previously seen grouping data, which is updated with each invocation.
+  bool Extend(const void* data) {
+    size_t byte_width = static_cast<size_t>(key_type_.type->byte_width());
+    bool extends = save_key_data_.size() != byte_width
+                       ? kDefaultExtends
+                       : 0 == memcmp(save_key_data_.data(), data, byte_width);
+    save_key_data_.resize(byte_width);
+    memcpy(save_key_data_.data(), data, byte_width);
+    return extends;
+  }
+
+  Result<Segment> GetNextSegment(const Scalar& scalar, int64_t offset, int64_t length) {
+    ARROW_RETURN_NOT_OK(CheckType(*scalar.type));
+    if (!scalar.is_valid) {
+      return Status::Invalid("segmenting an invalid scalar");
+    }
+    auto data = checked_cast<const PrimitiveScalarBase&>(scalar).data();
+    bool extends = length > 0 ? Extend(data) : kEmptyExtends;
+    return MakeSegment(length, offset, length, extends);
+  }
+
+  Result<Segment> GetNextSegment(const DataType& array_type, const uint8_t* array_bytes,
+                                 int64_t offset, int64_t length) {
+    RETURN_NOT_OK(CheckType(array_type));
+    int64_t byte_width = array_type.byte_width();
+    int64_t match_length = GetMatchLength(array_bytes + offset * byte_width, byte_width,
+                                          array_bytes, offset, length);
+    bool extends = length > 0 ? Extend(array_bytes + offset * byte_width) : kEmptyExtends;
+    return MakeSegment(length, offset, match_length, extends);
+  }
+
+  Result<Segment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {key_type_}));
+    if (offset == batch.length) {
+      return MakeSegment(batch.length, offset, 0, kEmptyExtends);
+    }
+    const auto& value = batch.values[0];
+    if (value.is_scalar()) {
+      return GetNextSegment(*value.scalar, offset, batch.length);
+    }
+    ARROW_DCHECK(value.is_array());
+    const auto& array = value.array;
+    if (array.GetNullCount() > 0) {
+      return Status::NotImplemented("segmenting a nullable array");
+    }
+    return GetNextSegment(*array.type, GetValuesAsBytes(array), offset, batch.length);
+  }
+
+ private:
+  TypeHolder key_type_;
+  std::vector<uint8_t> save_key_data_;
+};
+
+struct AnyKeysSegmenter : public BaseRowSegmenter {
+  static Result<std::unique_ptr<RowSegmenter>> Make(
+      const std::vector<TypeHolder>& key_types, ExecContext* ctx) {
+    ARROW_RETURN_NOT_OK(Grouper::Make(key_types, ctx));  // check types
+    return std::make_unique<AnyKeysSegmenter>(key_types, ctx);
+  }
+
+  AnyKeysSegmenter(const std::vector<TypeHolder>& key_types, ExecContext* ctx)
+      : BaseRowSegmenter(key_types),
+        ctx_(ctx),
+        grouper_(nullptr),
+        save_group_id_(kNoGroupId) {}
+
+  Status Reset() override {

Review Comment:
   Is this used anywhere? When do we want to reset a segmenter?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rtpsw commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "rtpsw (via GitHub)" <gi...@apache.org>.
rtpsw commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1127092002


##########
cpp/src/arrow/compute/row/grouper.cc:
##########
@@ -39,12 +43,330 @@
 namespace arrow {
 
 using internal::checked_cast;
+using internal::PrimitiveScalarBase;
 
 namespace compute {
 
 namespace {
 
-struct GrouperImpl : Grouper {
+constexpr uint32_t kNoGroupId = std::numeric_limits<uint32_t>::max();
+
+using group_id_t = std::remove_const<decltype(kNoGroupId)>::type;
+using GroupIdType = CTypeTraits<group_id_t>::ArrowType;
+auto g_group_id_type = std::make_shared<GroupIdType>();
+
+inline const uint8_t* GetValuesAsBytes(const ArraySpan& data, int64_t offset = 0) {
+  DCHECK_GT(data.type->byte_width(), 0);
+  int64_t absolute_byte_offset = (data.offset + offset) * data.type->byte_width();
+  return data.GetValues<uint8_t>(1, absolute_byte_offset);
+}
+
+template <typename Value>
+Status CheckForGetNextSegment(const std::vector<Value>& values, int64_t length,
+                              int64_t offset, const std::vector<TypeHolder>& key_types) {
+  if (offset < 0 || offset > length) {
+    return Status::Invalid("invalid grouping segmenter offset: ", offset);
+  }
+  if (values.size() != key_types.size()) {
+    return Status::Invalid("expected batch size ", key_types.size(), " but got ",
+                           values.size());
+  }
+  for (size_t i = 0; i < key_types.size(); i++) {
+    const auto& value = values[i];
+    const auto& key_type = key_types[i];
+    if (*value.type() != *key_type.type) {
+      return Status::Invalid("expected batch value ", i, " of type ", *key_type.type,
+                             " but got ", *value.type());
+    }
+  }
+  return Status::OK();
+}
+
+template <typename Batch>
+enable_if_t<std::is_same<Batch, ExecSpan>::value || std::is_same<Batch, ExecBatch>::value,
+            Status>
+CheckForGetNextSegment(const Batch& batch, int64_t offset,
+                       const std::vector<TypeHolder>& key_types) {
+  return CheckForGetNextSegment(batch.values, batch.length, offset, key_types);
+}
+
+struct BaseRowSegmenter : public RowSegmenter {
+  explicit BaseRowSegmenter(const std::vector<TypeHolder>& key_types)
+      : key_types_(key_types) {}
+
+  const std::vector<TypeHolder>& key_types() const override { return key_types_; }
+
+  std::vector<TypeHolder> key_types_;
+};
+
+Segment MakeSegment(int64_t batch_length, int64_t offset, int64_t length, bool extends) {
+  return Segment{offset, length, offset + length >= batch_length, extends};
+}
+
+int64_t GetMatchLength(const uint8_t* match_bytes, int64_t match_width,
+                       const uint8_t* array_bytes, int64_t offset, int64_t length) {
+  int64_t cursor, byte_cursor;
+  for (cursor = offset, byte_cursor = match_width * cursor; cursor < length;
+       cursor++, byte_cursor += match_width) {
+    if (memcmp(match_bytes, array_bytes + byte_cursor,
+               static_cast<size_t>(match_width)) != 0) {
+      break;
+    }
+  }
+  return std::min(cursor, length) - offset;
+}
+
+using ExtendFunc = std::function<bool(const void*)>;
+constexpr bool kDefaultExtends = true;
+constexpr bool kEmptyExtends = true;
+
+struct NoKeysSegmenter : public BaseRowSegmenter {
+  static std::unique_ptr<RowSegmenter> Make() {
+    return std::make_unique<NoKeysSegmenter>();
+  }
+
+  NoKeysSegmenter() : BaseRowSegmenter({}) {}
+
+  Status Reset() override { return Status::OK(); }
+
+  Result<Segment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {}));
+    return MakeSegment(batch.length, offset, batch.length - offset, kDefaultExtends);
+  }
+};
+
+struct SimpleKeySegmenter : public BaseRowSegmenter {

Review Comment:
   Yes, we do. `test_row_segmenter_constant_batch` (and the test-cases using it) covers all 3 segmenter classes using batches with 0, 1, and 2 columns corresponding to `NoKeySegmenter`, `SimpleKeySegmenter`, and `AnyKeysSegmenter`.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rtpsw commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "rtpsw (via GitHub)" <gi...@apache.org>.
rtpsw commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1125645306


##########
cpp/src/arrow/compute/row/grouper.h:
##########
@@ -30,6 +30,49 @@
 namespace arrow {
 namespace compute {
 
+/// \brief A segment of contiguous rows for grouping
+struct ARROW_EXPORT GroupingSegment {
+  int64_t offset;
+  int64_t length;
+  bool is_open;
+  bool extends;
+};
+
+inline bool operator==(const GroupingSegment& segment1, const GroupingSegment& segment2) {
+  return segment1.offset == segment2.offset && segment1.length == segment2.length &&
+         segment1.is_open == segment2.is_open && segment1.extends == segment2.extends;
+}
+inline bool operator!=(const GroupingSegment& segment1, const GroupingSegment& segment2) {
+  return !(segment1 == segment2);
+}
+
+/// \brief Computes grouping segments for a batch. Each segment covers rows with identical
+/// values in the batch. The values in the batch are often selected as keys from a larger
+/// batch.
+class ARROW_EXPORT GroupingSegmenter {
+ public:
+  virtual ~GroupingSegmenter() = default;
+
+  /// \brief Construct a GroupingSegmenter which receives the specified key types
+  static Result<std::unique_ptr<GroupingSegmenter>> Make(
+      const std::vector<TypeHolder>& key_types, bool nullable_keys = false,
+      ExecContext* ctx = default_exec_context());

Review Comment:
   The template-invocation `GroupClass::Make(...)` passes a single `const std::vector<TypeHolder>&` parameter. In one case it invokes `Grouper::Make` and in the second case `RowSegmenter::Make`. These doesn't have the same signature; in particular they differ on the default parameters. However, both accept a single `const std::vector<TypeHolder>&` parameter, which can be seen as an overload-signature that the invocation relies on. If the default values were removed, this overload-signature would be removed, and the invocation would break.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] icexelloss commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "icexelloss (via GitHub)" <gi...@apache.org>.
icexelloss commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1126946765


##########
cpp/src/arrow/compute/kernels/hash_aggregate_test.cc:
##########
@@ -174,81 +242,117 @@ Result<Datum> RunGroupBy(const BatchesWithSchema& input,
   ARROW_ASSIGN_OR_RAISE(std::vector<ExecBatch> output_batches,
                         start_and_collect.MoveResult());
 
-  ArrayVector out_arrays(aggregates.size() + key_names.size());
   const auto& output_schema = plan->nodes()[0]->output()->output_schema();
+  if (!segmented) {
+    return MakeGroupByOutput(output_batches, output_schema, aggregates.size(),
+                             key_names.size(), naive);
+  }
+
+  std::vector<ArrayVector> out_arrays(aggregates.size() + key_names.size() +
+                                      segment_key_names.size());
   for (size_t i = 0; i < out_arrays.size(); ++i) {
     std::vector<std::shared_ptr<Array>> arrays(output_batches.size());
     for (size_t j = 0; j < output_batches.size(); ++j) {
-      arrays[j] = output_batches[j].values[i].make_array();
+      auto& value = output_batches[j].values[i];
+      if (value.is_scalar()) {
+        ARROW_ASSIGN_OR_RAISE(
+            arrays[j], MakeArrayFromScalar(*value.scalar(), output_batches[j].length));
+      } else if (value.is_array()) {
+        arrays[j] = value.make_array();
+      } else {
+        return Status::Invalid("GroupByUsingExecPlan unsupported value kind ",
+                               ToString(value.kind()));
+      }
     }
     if (arrays.empty()) {
+      arrays.resize(1);
       ARROW_ASSIGN_OR_RAISE(
-          out_arrays[i],
-          MakeArrayOfNull(output_schema->field(static_cast<int>(i))->type(),
-                          /*length=*/0));
-    } else {
-      ARROW_ASSIGN_OR_RAISE(out_arrays[i], Concatenate(arrays));
+          arrays[0], MakeArrayOfNull(output_schema->field(static_cast<int>(i))->type(),
+                                     /*length=*/0));
     }
+    out_arrays[i] = {std::move(arrays)};
   }
 
-  // The exec plan may reorder the output rows.  The tests are all setup to expect ouptut
-  // in ascending order of keys.  So we need to sort the result by the key columns.  To do
-  // that we create a table using the key columns, calculate the sort indices from that
-  // table (sorting on all fields) and then use those indices to calculate our result.
-  std::vector<std::shared_ptr<Field>> key_fields;
-  std::vector<std::shared_ptr<Array>> key_columns;
-  std::vector<SortKey> sort_keys;
-  for (std::size_t i = 0; i < key_names.size(); i++) {
-    const std::shared_ptr<Array>& arr = out_arrays[i + aggregates.size()];
-    if (arr->type_id() == Type::DICTIONARY) {
-      // Can't sort dictionary columns so need to decode
-      auto dict_arr = checked_pointer_cast<DictionaryArray>(arr);
-      ARROW_ASSIGN_OR_RAISE(auto decoded_arr,
-                            Take(*dict_arr->dictionary(), *dict_arr->indices()));
-      key_columns.push_back(decoded_arr);
-      key_fields.push_back(
-          field("name_does_not_matter", dict_arr->dict_type()->value_type()));
-    } else {
-      key_columns.push_back(arr);
-      key_fields.push_back(field("name_does_not_matter", arr->type()));
+  if (segmented && segment_key_names.size() > 0) {
+    ArrayVector struct_arrays;
+    struct_arrays.reserve(output_batches.size());
+    for (size_t j = 0; j < output_batches.size(); ++j) {
+      ArrayVector struct_fields;
+      struct_fields.reserve(out_arrays.size());
+      for (auto out_array : out_arrays) {
+        struct_fields.push_back(out_array[j]);
+      }
+      ARROW_ASSIGN_OR_RAISE(auto struct_array,
+                            StructArray::Make(struct_fields, output_schema->fields()));
+      struct_arrays.push_back(struct_array);
     }
-    sort_keys.emplace_back(static_cast<int>(i));
+    return ChunkedArray::Make(struct_arrays);
+  } else {
+    ArrayVector struct_fields(out_arrays.size());
+    for (size_t i = 0; i < out_arrays.size(); ++i) {
+      ARROW_ASSIGN_OR_RAISE(struct_fields[i], Concatenate(out_arrays[i]));
+    }
+    return StructArray::Make(std::move(struct_fields), output_schema->fields());
   }
-  std::shared_ptr<Schema> key_schema = schema(std::move(key_fields));
-  std::shared_ptr<Table> key_table = Table::Make(std::move(key_schema), key_columns);
-  SortOptions sort_options(std::move(sort_keys));
-  ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Array> sort_indices,
-                        SortIndices(key_table, sort_options));
+}
 
-  ARROW_ASSIGN_OR_RAISE(
-      std::shared_ptr<Array> struct_arr,
-      StructArray::Make(std::move(out_arrays), output_schema->fields()));
+Result<Datum> RunGroupBy(const BatchesWithSchema& input,
+                         const std::vector<std::string>& key_names,
+                         const std::vector<std::string>& segment_key_names,
+                         const std::vector<Aggregate>& aggregates, bool use_threads,
+                         bool segmented = false, bool naive = false) {
+  if (segment_key_names.size() > 0) {
+    ARROW_ASSIGN_OR_RAISE(auto thread_pool, arrow::internal::ThreadPool::Make(1));
+    ExecContext seq_ctx(default_memory_pool(), thread_pool.get());
+    return RunGroupBy(input, key_names, segment_key_names, aggregates, &seq_ctx,
+                      use_threads, segmented, naive);
+  } else {
+    return RunGroupBy(input, key_names, segment_key_names, aggregates,
+                      threaded_exec_context(), use_threads, segmented, naive);
+  }
+}
 
-  return Take(struct_arr, sort_indices);
+Result<Datum> RunGroupBy(const BatchesWithSchema& input,
+                         const std::vector<std::string>& key_names,
+                         const std::vector<Aggregate>& aggregates, bool use_threads,
+                         bool segmented = false, bool naive = false) {
+  return RunGroupBy(input, key_names, {}, aggregates, use_threads, segmented);
 }
 
 /// Simpler overload where you can give the columns as datums
 Result<Datum> RunGroupBy(const std::vector<Datum>& arguments,
                          const std::vector<Datum>& keys,
-                         const std::vector<Aggregate>& aggregates,
-                         bool use_threads = false) {
+                         const std::vector<Datum>& segment_keys,
+                         const std::vector<Aggregate>& aggregates, bool use_threads,
+                         bool segmented = false, bool naive = false) {

Review Comment:
   > therwise (when empty), it may still be set to true, and this is the case we're discussing here.
   
   This will never happen in real scenario, right? If that is the case, I think we probably don't need to test that case. If you really want to test this then we should make it very clear in the test comment because this is quite confusing IMO.  



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] icexelloss commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "icexelloss (via GitHub)" <gi...@apache.org>.
icexelloss commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1127000695


##########
cpp/src/arrow/compute/row/grouper.cc:
##########
@@ -39,12 +43,330 @@
 namespace arrow {
 
 using internal::checked_cast;
+using internal::PrimitiveScalarBase;
 
 namespace compute {
 
 namespace {
 
-struct GrouperImpl : Grouper {
+constexpr uint32_t kNoGroupId = std::numeric_limits<uint32_t>::max();
+
+using group_id_t = std::remove_const<decltype(kNoGroupId)>::type;
+using GroupIdType = CTypeTraits<group_id_t>::ArrowType;
+auto g_group_id_type = std::make_shared<GroupIdType>();
+
+inline const uint8_t* GetValuesAsBytes(const ArraySpan& data, int64_t offset = 0) {
+  DCHECK_GT(data.type->byte_width(), 0);
+  int64_t absolute_byte_offset = (data.offset + offset) * data.type->byte_width();
+  return data.GetValues<uint8_t>(1, absolute_byte_offset);
+}
+
+template <typename Value>
+Status CheckForGetNextSegment(const std::vector<Value>& values, int64_t length,
+                              int64_t offset, const std::vector<TypeHolder>& key_types) {
+  if (offset < 0 || offset > length) {
+    return Status::Invalid("invalid grouping segmenter offset: ", offset);
+  }
+  if (values.size() != key_types.size()) {
+    return Status::Invalid("expected batch size ", key_types.size(), " but got ",
+                           values.size());
+  }
+  for (size_t i = 0; i < key_types.size(); i++) {
+    const auto& value = values[i];
+    const auto& key_type = key_types[i];
+    if (*value.type() != *key_type.type) {
+      return Status::Invalid("expected batch value ", i, " of type ", *key_type.type,
+                             " but got ", *value.type());
+    }
+  }
+  return Status::OK();
+}
+
+template <typename Batch>
+enable_if_t<std::is_same<Batch, ExecSpan>::value || std::is_same<Batch, ExecBatch>::value,
+            Status>
+CheckForGetNextSegment(const Batch& batch, int64_t offset,
+                       const std::vector<TypeHolder>& key_types) {
+  return CheckForGetNextSegment(batch.values, batch.length, offset, key_types);
+}
+
+struct BaseRowSegmenter : public RowSegmenter {
+  explicit BaseRowSegmenter(const std::vector<TypeHolder>& key_types)
+      : key_types_(key_types) {}
+
+  const std::vector<TypeHolder>& key_types() const override { return key_types_; }
+
+  std::vector<TypeHolder> key_types_;
+};
+
+Segment MakeSegment(int64_t batch_length, int64_t offset, int64_t length, bool extends) {
+  return Segment{offset, length, offset + length >= batch_length, extends};
+}
+
+int64_t GetMatchLength(const uint8_t* match_bytes, int64_t match_width,
+                       const uint8_t* array_bytes, int64_t offset, int64_t length) {
+  int64_t cursor, byte_cursor;
+  for (cursor = offset, byte_cursor = match_width * cursor; cursor < length;
+       cursor++, byte_cursor += match_width) {
+    if (memcmp(match_bytes, array_bytes + byte_cursor,
+               static_cast<size_t>(match_width)) != 0) {
+      break;
+    }
+  }
+  return std::min(cursor, length) - offset;
+}
+
+using ExtendFunc = std::function<bool(const void*)>;
+constexpr bool kDefaultExtends = true;
+constexpr bool kEmptyExtends = true;
+
+struct NoKeysSegmenter : public BaseRowSegmenter {
+  static std::unique_ptr<RowSegmenter> Make() {
+    return std::make_unique<NoKeysSegmenter>();
+  }
+
+  NoKeysSegmenter() : BaseRowSegmenter({}) {}
+
+  Status Reset() override { return Status::OK(); }
+
+  Result<Segment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {}));
+    return MakeSegment(batch.length, offset, batch.length - offset, kDefaultExtends);
+  }
+};
+
+struct SimpleKeySegmenter : public BaseRowSegmenter {
+  static Result<std::unique_ptr<RowSegmenter>> Make(TypeHolder key_type) {
+    return std::make_unique<SimpleKeySegmenter>(key_type);
+  }
+
+  explicit SimpleKeySegmenter(TypeHolder key_type)
+      : BaseRowSegmenter({key_type}), key_type_(key_types_[0]), save_key_data_() {}
+
+  Status CheckType(const DataType& type) {
+    if (!is_fixed_width(type)) {
+      return Status::Invalid("SimpleKeySegmenter does not support type ", type);
+    }
+    return Status::OK();
+  }
+
+  Status Reset() override {
+    save_key_data_.resize(0);
+    return Status::OK();
+  }
+
+  // Checks whether the given grouping data extends the current segment, i.e., is equal to
+  // previously seen grouping data, which is updated with each invocation.
+  bool Extend(const void* data) {
+    size_t byte_width = static_cast<size_t>(key_type_.type->byte_width());
+    bool extends = save_key_data_.size() != byte_width
+                       ? kDefaultExtends
+                       : 0 == memcmp(save_key_data_.data(), data, byte_width);
+    save_key_data_.resize(byte_width);
+    memcpy(save_key_data_.data(), data, byte_width);
+    return extends;
+  }
+
+  Result<Segment> GetNextSegment(const Scalar& scalar, int64_t offset, int64_t length) {
+    ARROW_RETURN_NOT_OK(CheckType(*scalar.type));
+    if (!scalar.is_valid) {
+      return Status::Invalid("segmenting an invalid scalar");
+    }
+    auto data = checked_cast<const PrimitiveScalarBase&>(scalar).data();
+    bool extends = length > 0 ? Extend(data) : kEmptyExtends;
+    return MakeSegment(length, offset, length, extends);
+  }
+
+  Result<Segment> GetNextSegment(const DataType& array_type, const uint8_t* array_bytes,
+                                 int64_t offset, int64_t length) {
+    RETURN_NOT_OK(CheckType(array_type));
+    int64_t byte_width = array_type.byte_width();
+    int64_t match_length = GetMatchLength(array_bytes + offset * byte_width, byte_width,
+                                          array_bytes, offset, length);
+    bool extends = length > 0 ? Extend(array_bytes + offset * byte_width) : kEmptyExtends;
+    return MakeSegment(length, offset, match_length, extends);
+  }
+
+  Result<Segment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {key_type_}));
+    if (offset == batch.length) {
+      return MakeSegment(batch.length, offset, 0, kEmptyExtends);
+    }
+    const auto& value = batch.values[0];
+    if (value.is_scalar()) {
+      return GetNextSegment(*value.scalar, offset, batch.length);
+    }
+    ARROW_DCHECK(value.is_array());
+    const auto& array = value.array;
+    if (array.GetNullCount() > 0) {
+      return Status::NotImplemented("segmenting a nullable array");
+    }
+    return GetNextSegment(*array.type, GetValuesAsBytes(array), offset, batch.length);
+  }
+
+ private:
+  TypeHolder key_type_;
+  std::vector<uint8_t> save_key_data_;
+};
+
+struct AnyKeysSegmenter : public BaseRowSegmenter {
+  static Result<std::unique_ptr<RowSegmenter>> Make(
+      const std::vector<TypeHolder>& key_types, ExecContext* ctx) {
+    ARROW_RETURN_NOT_OK(Grouper::Make(key_types, ctx));  // check types
+    return std::make_unique<AnyKeysSegmenter>(key_types, ctx);
+  }
+
+  AnyKeysSegmenter(const std::vector<TypeHolder>& key_types, ExecContext* ctx)
+      : BaseRowSegmenter(key_types),
+        ctx_(ctx),
+        grouper_(nullptr),
+        save_group_id_(kNoGroupId) {}
+
+  Status Reset() override {
+    grouper_ = nullptr;
+    save_group_id_ = kNoGroupId;
+    return Status::OK();
+  }
+
+  bool Extend(const void* data) {
+    auto group_id = *static_cast<const group_id_t*>(data);
+    bool extends =
+        save_group_id_ == kNoGroupId ? kDefaultExtends : save_group_id_ == group_id;
+    save_group_id_ = group_id;
+    return extends;
+  }
+
+  // Runs the grouper on a single row.  This is used to determine the group id of the
+  // first row of a new segment to see if it extends the previous segment.
+  template <typename Batch>
+  Result<group_id_t> MapGroupIdAt(const Batch& batch, int64_t offset) {
+    if (!grouper_) return kNoGroupId;
+    ARROW_ASSIGN_OR_RAISE(auto datum, grouper_->Consume(batch, offset,
+                                                        /*length=*/1));
+    if (!datum.is_array()) {
+      return Status::Invalid("accessing unsupported datum kind ", datum.kind());
+    }
+    const std::shared_ptr<ArrayData>& data = datum.array();
+    ARROW_DCHECK(data->GetNullCount() == 0);
+    DCHECK_EQ(data->type->id(), GroupIdType::type_id);
+    DCHECK_EQ(1, data->length);
+    const group_id_t* values = data->GetValues<group_id_t>(1);

Review Comment:
   Why index "1" here?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rtpsw commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "rtpsw (via GitHub)" <gi...@apache.org>.
rtpsw commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1116869071


##########
cpp/src/arrow/compute/row/grouper.h:
##########
@@ -30,6 +30,49 @@
 namespace arrow {
 namespace compute {
 
+/// \brief A segment of contiguous rows for grouping
+struct ARROW_EXPORT GroupingSegment {
+  int64_t offset;
+  int64_t length;
+  bool is_open;
+  bool extends;
+};
+
+inline bool operator==(const GroupingSegment& segment1, const GroupingSegment& segment2) {
+  return segment1.offset == segment2.offset && segment1.length == segment2.length &&
+         segment1.is_open == segment2.is_open && segment1.extends == segment2.extends;
+}
+inline bool operator!=(const GroupingSegment& segment1, const GroupingSegment& segment2) {
+  return !(segment1 == segment2);
+}
+
+/// \brief Computes grouping segments for a batch. Each segment covers rows with identical
+/// values in the batch. The values in the batch are often selected as keys from a larger
+/// batch.
+class ARROW_EXPORT GroupingSegmenter {
+ public:
+  virtual ~GroupingSegmenter() = default;
+
+  /// \brief Construct a GroupingSegmenter which receives the specified key types
+  static Result<std::unique_ptr<GroupingSegmenter>> Make(
+      const std::vector<TypeHolder>& key_types, bool nullable_keys = false,
+      ExecContext* ctx = default_exec_context());
+
+  /// \brief Return the key types of this segmenter
+  virtual const std::vector<TypeHolder>& key_types() const = 0;
+
+  /// \brief Reset this grouping segmenter
+  virtual Status Reset() = 0;
+
+  /// \brief Get the next segment for the given batch starting from the given offset
+  virtual Result<GroupingSegment> GetNextSegment(const ExecSpan& batch,
+                                                 int64_t offset) = 0;
+
+  /// \brief Get the next segment for the given batch starting from the given offset
+  virtual Result<GroupingSegment> GetNextSegment(const ExecBatch& batch,

Review Comment:
   An `ExecSpan` is [a view](https://github.com/apache/arrow/blob/e8e50a3ca96ec8eb2f2beb383cf9005ce1c93486/cpp/src/arrow/compute/exec.h#L348-L350) into an `ExecBatch`. The latter owns the memory of its members while the former doesn't. Another difference is that an `ExecSpan` cannot be chunked, as its values (i.e., columns) can only be a view into an array, but `ExecBatch` can have chunked array values.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] icexelloss commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "icexelloss (via GitHub)" <gi...@apache.org>.
icexelloss commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1125573523


##########
cpp/src/arrow/compute/row/grouper.h:
##########
@@ -30,6 +30,49 @@
 namespace arrow {
 namespace compute {
 
+/// \brief A segment of contiguous rows for grouping
+struct ARROW_EXPORT GroupingSegment {
+  int64_t offset;
+  int64_t length;
+  bool is_open;
+  bool extends;
+};
+
+inline bool operator==(const GroupingSegment& segment1, const GroupingSegment& segment2) {
+  return segment1.offset == segment2.offset && segment1.length == segment2.length &&
+         segment1.is_open == segment2.is_open && segment1.extends == segment2.extends;
+}
+inline bool operator!=(const GroupingSegment& segment1, const GroupingSegment& segment2) {
+  return !(segment1 == segment2);
+}
+
+/// \brief Computes grouping segments for a batch. Each segment covers rows with identical
+/// values in the batch. The values in the batch are often selected as keys from a larger
+/// batch.
+class ARROW_EXPORT GroupingSegmenter {
+ public:
+  virtual ~GroupingSegmenter() = default;
+
+  /// \brief Construct a GroupingSegmenter which receives the specified key types
+  static Result<std::unique_ptr<GroupingSegmenter>> Make(
+      const std::vector<TypeHolder>& key_types, bool nullable_keys = false,
+      ExecContext* ctx = default_exec_context());

Review Comment:
   I am not sure why this is related, can you elaborate



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] icexelloss commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "icexelloss (via GitHub)" <gi...@apache.org>.
icexelloss commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1125574829


##########
cpp/src/arrow/compute/exec/aggregate_node.cc:
##########
@@ -440,12 +613,29 @@ class GroupByNode : public ExecNode, public TracedNode {
       int key_field_id = key_field_ids[i];
       output_fields[base + i] = input_schema->field(key_field_id);
     }
+    base += keys.size();
+    for (size_t i = 0; i < segment_keys.size(); ++i) {
+      int segment_key_field_id = segment_key_field_ids[i];
+      output_fields[base + i] = input_schema->field(segment_key_field_id);
+    }
 
     return input->plan()->EmplaceNode<GroupByNode>(
         input, schema(std::move(output_fields)), std::move(key_field_ids),
+        std::move(segment_key_field_ids), std::move(segmenter), std::move(agg_src_types),
         std::move(agg_src_fieldsets), std::move(aggs), std::move(agg_kernels));
   }
 
+  Status ResetAggregates() {

Review Comment:
   What are the difference between them? And which one do we use for ordered aggregation?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] icexelloss commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "icexelloss (via GitHub)" <gi...@apache.org>.
icexelloss commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1126714774


##########
cpp/src/arrow/compute/exec/aggregate_node.cc:
##########
@@ -326,10 +437,22 @@ class ScalarAggregateNode : public ExecNode, public TracedNode {
   }
 
  private:
-  Status Finish() {
-    auto scope = TraceFinish();
+  Status ResetAggregates() {
+    auto exec_ctx = plan()->query_context()->exec_context();
+    for (size_t i = 0; i < kernels_.size(); ++i) {
+      const std::vector<TypeHolder>& in_types = in_typesets_[i];
+      states_[i].resize(plan()->query_context()->max_concurrency());
+      KernelContext kernel_ctx{exec_ctx};
+      RETURN_NOT_OK(Kernel::InitAll(
+          &kernel_ctx, KernelInitArgs{kernels_[i], in_types, aggs_[i].options.get()},
+          &states_[i]));
+    }
+    return Status::OK();
+  }
+
+  Status OutputResult(bool is_last = false) {
     ExecBatch batch{{}, 1};
-    batch.values.resize(kernels_.size());
+    batch.values.resize(kernels_.size() + segment_field_ids_.size());

Review Comment:
   I see



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rtpsw commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "rtpsw (via GitHub)" <gi...@apache.org>.
rtpsw commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1127413760


##########
cpp/src/arrow/compute/row/grouper.cc:
##########
@@ -39,12 +43,330 @@
 namespace arrow {
 
 using internal::checked_cast;
+using internal::PrimitiveScalarBase;
 
 namespace compute {
 
 namespace {
 
-struct GrouperImpl : Grouper {
+constexpr uint32_t kNoGroupId = std::numeric_limits<uint32_t>::max();
+
+using group_id_t = std::remove_const<decltype(kNoGroupId)>::type;
+using GroupIdType = CTypeTraits<group_id_t>::ArrowType;
+auto g_group_id_type = std::make_shared<GroupIdType>();
+
+inline const uint8_t* GetValuesAsBytes(const ArraySpan& data, int64_t offset = 0) {
+  DCHECK_GT(data.type->byte_width(), 0);
+  int64_t absolute_byte_offset = (data.offset + offset) * data.type->byte_width();
+  return data.GetValues<uint8_t>(1, absolute_byte_offset);
+}
+
+template <typename Value>
+Status CheckForGetNextSegment(const std::vector<Value>& values, int64_t length,
+                              int64_t offset, const std::vector<TypeHolder>& key_types) {
+  if (offset < 0 || offset > length) {
+    return Status::Invalid("invalid grouping segmenter offset: ", offset);
+  }
+  if (values.size() != key_types.size()) {
+    return Status::Invalid("expected batch size ", key_types.size(), " but got ",
+                           values.size());
+  }
+  for (size_t i = 0; i < key_types.size(); i++) {
+    const auto& value = values[i];
+    const auto& key_type = key_types[i];
+    if (*value.type() != *key_type.type) {
+      return Status::Invalid("expected batch value ", i, " of type ", *key_type.type,
+                             " but got ", *value.type());
+    }
+  }
+  return Status::OK();
+}
+
+template <typename Batch>
+enable_if_t<std::is_same<Batch, ExecSpan>::value || std::is_same<Batch, ExecBatch>::value,
+            Status>
+CheckForGetNextSegment(const Batch& batch, int64_t offset,
+                       const std::vector<TypeHolder>& key_types) {
+  return CheckForGetNextSegment(batch.values, batch.length, offset, key_types);
+}
+
+struct BaseRowSegmenter : public RowSegmenter {
+  explicit BaseRowSegmenter(const std::vector<TypeHolder>& key_types)
+      : key_types_(key_types) {}
+
+  const std::vector<TypeHolder>& key_types() const override { return key_types_; }
+
+  std::vector<TypeHolder> key_types_;
+};
+
+Segment MakeSegment(int64_t batch_length, int64_t offset, int64_t length, bool extends) {
+  return Segment{offset, length, offset + length >= batch_length, extends};
+}
+
+int64_t GetMatchLength(const uint8_t* match_bytes, int64_t match_width,
+                       const uint8_t* array_bytes, int64_t offset, int64_t length) {
+  int64_t cursor, byte_cursor;
+  for (cursor = offset, byte_cursor = match_width * cursor; cursor < length;
+       cursor++, byte_cursor += match_width) {
+    if (memcmp(match_bytes, array_bytes + byte_cursor,
+               static_cast<size_t>(match_width)) != 0) {
+      break;
+    }
+  }
+  return std::min(cursor, length) - offset;
+}
+
+using ExtendFunc = std::function<bool(const void*)>;
+constexpr bool kDefaultExtends = true;
+constexpr bool kEmptyExtends = true;
+
+struct NoKeysSegmenter : public BaseRowSegmenter {
+  static std::unique_ptr<RowSegmenter> Make() {
+    return std::make_unique<NoKeysSegmenter>();
+  }
+
+  NoKeysSegmenter() : BaseRowSegmenter({}) {}
+
+  Status Reset() override { return Status::OK(); }
+
+  Result<Segment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {}));
+    return MakeSegment(batch.length, offset, batch.length - offset, kDefaultExtends);
+  }
+};
+
+struct SimpleKeySegmenter : public BaseRowSegmenter {
+  static Result<std::unique_ptr<RowSegmenter>> Make(TypeHolder key_type) {
+    return std::make_unique<SimpleKeySegmenter>(key_type);
+  }
+
+  explicit SimpleKeySegmenter(TypeHolder key_type)
+      : BaseRowSegmenter({key_type}), key_type_(key_types_[0]), save_key_data_() {}
+
+  Status CheckType(const DataType& type) {
+    if (!is_fixed_width(type)) {
+      return Status::Invalid("SimpleKeySegmenter does not support type ", type);
+    }
+    return Status::OK();
+  }
+
+  Status Reset() override {
+    save_key_data_.resize(0);
+    return Status::OK();
+  }
+
+  // Checks whether the given grouping data extends the current segment, i.e., is equal to
+  // previously seen grouping data, which is updated with each invocation.
+  bool Extend(const void* data) {
+    size_t byte_width = static_cast<size_t>(key_type_.type->byte_width());
+    bool extends = save_key_data_.size() != byte_width
+                       ? kDefaultExtends
+                       : 0 == memcmp(save_key_data_.data(), data, byte_width);
+    save_key_data_.resize(byte_width);

Review Comment:
   While I don't think it's critical, I'll refactor.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] icexelloss commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "icexelloss (via GitHub)" <gi...@apache.org>.
icexelloss commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1127032532


##########
cpp/src/arrow/compute/row/grouper.cc:
##########
@@ -39,12 +43,521 @@
 namespace arrow {
 
 using internal::checked_cast;
+using internal::PrimitiveScalarBase;
 
 namespace compute {
 
 namespace {
 
-struct GrouperImpl : Grouper {
+constexpr uint32_t kNoGroupId = std::numeric_limits<uint32_t>::max();
+
+using group_id_t = std::remove_const<decltype(kNoGroupId)>::type;
+using GroupIdType = CTypeTraits<group_id_t>::ArrowType;
+auto group_id_type = std::make_shared<GroupIdType>();
+
+inline const uint8_t* GetValuesAsBytes(const ArrayData& data, int64_t offset = 0) {
+  DCHECK_GT(data.type->byte_width(), 0);
+  int64_t absolute_byte_offset = (data.offset + offset) * data.type->byte_width();
+  return data.GetValues<uint8_t>(1, absolute_byte_offset);
+}
+
+inline const uint8_t* GetValuesAsBytes(const ArraySpan& data, int64_t offset = 0) {
+  DCHECK_GT(data.type->byte_width(), 0);
+  int64_t absolute_byte_offset = (data.offset + offset) * data.type->byte_width();
+  return data.GetValues<uint8_t>(1, absolute_byte_offset);
+}
+
+template <typename Value>
+Status CheckForGetNextSegment(const std::vector<Value>& values, int64_t length,
+                              int64_t offset, const std::vector<TypeHolder>& key_types) {
+  if (offset < 0 || offset > length) {
+    return Status::Invalid("invalid grouping segmenter offset: ", offset);
+  }
+  if (values.size() != key_types.size()) {
+    return Status::Invalid("expected batch size ", key_types.size(), " but got ",
+                           values.size());
+  }
+  for (size_t i = 0; i < key_types.size(); i++) {
+    const auto& value = values[i];
+    const auto& key_type = key_types[i];
+    if (*value.type() != *key_type.type) {
+      return Status::Invalid("expected batch value ", i, " of type ", *key_type.type,
+                             " but got ", *value.type());
+    }
+  }
+  return Status::OK();
+}
+
+template <typename Batch>
+enable_if_t<std::is_same<Batch, ExecSpan>::value || std::is_same<Batch, ExecBatch>::value,
+            Status>
+CheckForGetNextSegment(const Batch& batch, int64_t offset,
+                       const std::vector<TypeHolder>& key_types) {
+  return CheckForGetNextSegment(batch.values, batch.length, offset, key_types);
+}
+
+struct BaseGroupingSegmenter : public GroupingSegmenter {
+  explicit BaseGroupingSegmenter(const std::vector<TypeHolder>& key_types)
+      : key_types_(key_types) {}
+
+  const std::vector<TypeHolder>& key_types() const override { return key_types_; }
+
+  std::vector<TypeHolder> key_types_;
+};
+
+GroupingSegment MakeSegment(int64_t batch_length, int64_t offset, int64_t length,
+                            bool extends) {
+  return GroupingSegment{offset, length, offset + length >= batch_length, extends};
+}
+
+int64_t GetMatchLength(const uint8_t* match_bytes, int64_t match_width,

Review Comment:
   I feel we are writing algorithms for the wrong use cases here.
   
   `A, A, B, B, A, A, A, A, A, A, A` is not our use case to support and we should optimize the code for "ordered data" rather then "segment but not ordered data". Even in the AnyKeysSegmenter case I think we can do binary search instead of linear search for finding segment boundaries. Let's create a follow up to fix this.
   



##########
cpp/src/arrow/compute/row/grouper.cc:
##########
@@ -39,12 +43,521 @@
 namespace arrow {
 
 using internal::checked_cast;
+using internal::PrimitiveScalarBase;
 
 namespace compute {
 
 namespace {
 
-struct GrouperImpl : Grouper {
+constexpr uint32_t kNoGroupId = std::numeric_limits<uint32_t>::max();
+
+using group_id_t = std::remove_const<decltype(kNoGroupId)>::type;
+using GroupIdType = CTypeTraits<group_id_t>::ArrowType;
+auto group_id_type = std::make_shared<GroupIdType>();
+
+inline const uint8_t* GetValuesAsBytes(const ArrayData& data, int64_t offset = 0) {
+  DCHECK_GT(data.type->byte_width(), 0);
+  int64_t absolute_byte_offset = (data.offset + offset) * data.type->byte_width();
+  return data.GetValues<uint8_t>(1, absolute_byte_offset);
+}
+
+inline const uint8_t* GetValuesAsBytes(const ArraySpan& data, int64_t offset = 0) {
+  DCHECK_GT(data.type->byte_width(), 0);
+  int64_t absolute_byte_offset = (data.offset + offset) * data.type->byte_width();
+  return data.GetValues<uint8_t>(1, absolute_byte_offset);
+}
+
+template <typename Value>
+Status CheckForGetNextSegment(const std::vector<Value>& values, int64_t length,
+                              int64_t offset, const std::vector<TypeHolder>& key_types) {
+  if (offset < 0 || offset > length) {
+    return Status::Invalid("invalid grouping segmenter offset: ", offset);
+  }
+  if (values.size() != key_types.size()) {
+    return Status::Invalid("expected batch size ", key_types.size(), " but got ",
+                           values.size());
+  }
+  for (size_t i = 0; i < key_types.size(); i++) {
+    const auto& value = values[i];
+    const auto& key_type = key_types[i];
+    if (*value.type() != *key_type.type) {
+      return Status::Invalid("expected batch value ", i, " of type ", *key_type.type,
+                             " but got ", *value.type());
+    }
+  }
+  return Status::OK();
+}
+
+template <typename Batch>
+enable_if_t<std::is_same<Batch, ExecSpan>::value || std::is_same<Batch, ExecBatch>::value,
+            Status>
+CheckForGetNextSegment(const Batch& batch, int64_t offset,
+                       const std::vector<TypeHolder>& key_types) {
+  return CheckForGetNextSegment(batch.values, batch.length, offset, key_types);
+}
+
+struct BaseGroupingSegmenter : public GroupingSegmenter {
+  explicit BaseGroupingSegmenter(const std::vector<TypeHolder>& key_types)
+      : key_types_(key_types) {}
+
+  const std::vector<TypeHolder>& key_types() const override { return key_types_; }
+
+  std::vector<TypeHolder> key_types_;
+};
+
+GroupingSegment MakeSegment(int64_t batch_length, int64_t offset, int64_t length,
+                            bool extends) {
+  return GroupingSegment{offset, length, offset + length >= batch_length, extends};
+}
+
+int64_t GetMatchLength(const uint8_t* match_bytes, int64_t match_width,

Review Comment:
   I feel we are writing algorithms for the wrong use cases here.
   
   `A, A, B, B, A, A, A, A, A, A, A` is not our main use case to support and we should optimize the code for "ordered data" rather then "segment but not ordered data". Even in the AnyKeysSegmenter case I think we can do binary search instead of linear search for finding segment boundaries. Let's create a follow up to fix this.
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] icexelloss commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "icexelloss (via GitHub)" <gi...@apache.org>.
icexelloss commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1126608425


##########
cpp/src/arrow/compute/exec/aggregate_node.cc:
##########
@@ -440,12 +613,29 @@ class GroupByNode : public ExecNode, public TracedNode {
       int key_field_id = key_field_ids[i];
       output_fields[base + i] = input_schema->field(key_field_id);
     }
+    base += keys.size();
+    for (size_t i = 0; i < segment_keys.size(); ++i) {
+      int segment_key_field_id = segment_key_field_ids[i];
+      output_fields[base + i] = input_schema->field(segment_key_field_id);
+    }
 
     return input->plan()->EmplaceNode<GroupByNode>(
         input, schema(std::move(output_fields)), std::move(key_field_ids),
+        std::move(segment_key_field_ids), std::move(segmenter), std::move(agg_src_types),
         std::move(agg_src_fieldsets), std::move(aggs), std::move(agg_kernels));
   }
 
+  Status ResetAggregates() {

Review Comment:
   To clarify - if a non-segment key exists,  we use GroupByNode, if non-segment key is empty, we use ScalarAggregateNode?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rtpsw commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "rtpsw (via GitHub)" <gi...@apache.org>.
rtpsw commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1124230571


##########
cpp/src/arrow/compute/exec/aggregate_node.cc:
##########
@@ -326,10 +437,22 @@ class ScalarAggregateNode : public ExecNode, public TracedNode {
   }
 
  private:
-  Status Finish() {
-    auto scope = TraceFinish();
+  Status ResetAggregates() {
+    auto exec_ctx = plan()->query_context()->exec_context();
+    for (size_t i = 0; i < kernels_.size(); ++i) {
+      const std::vector<TypeHolder>& in_types = in_typesets_[i];
+      states_[i].resize(plan()->query_context()->max_concurrency());
+      KernelContext kernel_ctx{exec_ctx};
+      RETURN_NOT_OK(Kernel::InitAll(
+          &kernel_ctx, KernelInitArgs{kernels_[i], in_types, aggs_[i].options.get()},
+          &states_[i]));
+    }
+    return Status::OK();
+  }
+
+  Status OutputResult(bool is_last = false) {
     ExecBatch batch{{}, 1};
-    batch.values.resize(kernels_.size());
+    batch.values.resize(kernels_.size() + segment_field_ids_.size());

Review Comment:
   These are additional keys passed to the output. For example, if you have `time` as a segment-key and `key` as a key, then both will be passed to the output.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rtpsw commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "rtpsw (via GitHub)" <gi...@apache.org>.
rtpsw commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1126764743


##########
cpp/src/arrow/compute/exec/aggregate_node.cc:
##########
@@ -440,12 +613,29 @@ class GroupByNode : public ExecNode, public TracedNode {
       int key_field_id = key_field_ids[i];
       output_fields[base + i] = input_schema->field(key_field_id);
     }
+    base += keys.size();
+    for (size_t i = 0; i < segment_keys.size(); ++i) {
+      int segment_key_field_id = segment_key_field_ids[i];
+      output_fields[base + i] = input_schema->field(segment_key_field_id);
+    }
 
     return input->plan()->EmplaceNode<GroupByNode>(
         input, schema(std::move(output_fields)), std::move(key_field_ids),
+        std::move(segment_key_field_ids), std::move(segmenter), std::move(agg_src_types),
         std::move(agg_src_fieldsets), std::move(aggs), std::move(agg_kernels));
   }
 
+  Status ResetAggregates() {

Review Comment:
   Exactly.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] icexelloss commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "icexelloss (via GitHub)" <gi...@apache.org>.
icexelloss commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1127039705


##########
cpp/src/arrow/compute/row/grouper.cc:
##########
@@ -39,12 +43,330 @@
 namespace arrow {
 
 using internal::checked_cast;
+using internal::PrimitiveScalarBase;
 
 namespace compute {
 
 namespace {
 
-struct GrouperImpl : Grouper {
+constexpr uint32_t kNoGroupId = std::numeric_limits<uint32_t>::max();
+
+using group_id_t = std::remove_const<decltype(kNoGroupId)>::type;
+using GroupIdType = CTypeTraits<group_id_t>::ArrowType;
+auto g_group_id_type = std::make_shared<GroupIdType>();
+
+inline const uint8_t* GetValuesAsBytes(const ArraySpan& data, int64_t offset = 0) {
+  DCHECK_GT(data.type->byte_width(), 0);
+  int64_t absolute_byte_offset = (data.offset + offset) * data.type->byte_width();
+  return data.GetValues<uint8_t>(1, absolute_byte_offset);
+}
+
+template <typename Value>
+Status CheckForGetNextSegment(const std::vector<Value>& values, int64_t length,
+                              int64_t offset, const std::vector<TypeHolder>& key_types) {
+  if (offset < 0 || offset > length) {
+    return Status::Invalid("invalid grouping segmenter offset: ", offset);
+  }
+  if (values.size() != key_types.size()) {
+    return Status::Invalid("expected batch size ", key_types.size(), " but got ",
+                           values.size());
+  }
+  for (size_t i = 0; i < key_types.size(); i++) {
+    const auto& value = values[i];
+    const auto& key_type = key_types[i];
+    if (*value.type() != *key_type.type) {
+      return Status::Invalid("expected batch value ", i, " of type ", *key_type.type,
+                             " but got ", *value.type());
+    }
+  }
+  return Status::OK();
+}
+
+template <typename Batch>
+enable_if_t<std::is_same<Batch, ExecSpan>::value || std::is_same<Batch, ExecBatch>::value,
+            Status>
+CheckForGetNextSegment(const Batch& batch, int64_t offset,
+                       const std::vector<TypeHolder>& key_types) {
+  return CheckForGetNextSegment(batch.values, batch.length, offset, key_types);
+}
+
+struct BaseRowSegmenter : public RowSegmenter {
+  explicit BaseRowSegmenter(const std::vector<TypeHolder>& key_types)
+      : key_types_(key_types) {}
+
+  const std::vector<TypeHolder>& key_types() const override { return key_types_; }
+
+  std::vector<TypeHolder> key_types_;
+};
+
+Segment MakeSegment(int64_t batch_length, int64_t offset, int64_t length, bool extends) {
+  return Segment{offset, length, offset + length >= batch_length, extends};
+}
+
+int64_t GetMatchLength(const uint8_t* match_bytes, int64_t match_width,
+                       const uint8_t* array_bytes, int64_t offset, int64_t length) {
+  int64_t cursor, byte_cursor;
+  for (cursor = offset, byte_cursor = match_width * cursor; cursor < length;
+       cursor++, byte_cursor += match_width) {
+    if (memcmp(match_bytes, array_bytes + byte_cursor,
+               static_cast<size_t>(match_width)) != 0) {
+      break;
+    }
+  }
+  return std::min(cursor, length) - offset;
+}
+
+using ExtendFunc = std::function<bool(const void*)>;
+constexpr bool kDefaultExtends = true;
+constexpr bool kEmptyExtends = true;
+
+struct NoKeysSegmenter : public BaseRowSegmenter {
+  static std::unique_ptr<RowSegmenter> Make() {
+    return std::make_unique<NoKeysSegmenter>();
+  }
+
+  NoKeysSegmenter() : BaseRowSegmenter({}) {}
+
+  Status Reset() override { return Status::OK(); }
+
+  Result<Segment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {}));
+    return MakeSegment(batch.length, offset, batch.length - offset, kDefaultExtends);
+  }
+};
+
+struct SimpleKeySegmenter : public BaseRowSegmenter {
+  static Result<std::unique_ptr<RowSegmenter>> Make(TypeHolder key_type) {
+    return std::make_unique<SimpleKeySegmenter>(key_type);
+  }
+
+  explicit SimpleKeySegmenter(TypeHolder key_type)
+      : BaseRowSegmenter({key_type}), key_type_(key_types_[0]), save_key_data_() {}
+
+  Status CheckType(const DataType& type) {
+    if (!is_fixed_width(type)) {
+      return Status::Invalid("SimpleKeySegmenter does not support type ", type);
+    }
+    return Status::OK();
+  }
+
+  Status Reset() override {
+    save_key_data_.resize(0);
+    return Status::OK();
+  }
+
+  // Checks whether the given grouping data extends the current segment, i.e., is equal to
+  // previously seen grouping data, which is updated with each invocation.
+  bool Extend(const void* data) {
+    size_t byte_width = static_cast<size_t>(key_type_.type->byte_width());
+    bool extends = save_key_data_.size() != byte_width
+                       ? kDefaultExtends
+                       : 0 == memcmp(save_key_data_.data(), data, byte_width);
+    save_key_data_.resize(byte_width);
+    memcpy(save_key_data_.data(), data, byte_width);
+    return extends;
+  }
+
+  Result<Segment> GetNextSegment(const Scalar& scalar, int64_t offset, int64_t length) {
+    ARROW_RETURN_NOT_OK(CheckType(*scalar.type));
+    if (!scalar.is_valid) {
+      return Status::Invalid("segmenting an invalid scalar");
+    }
+    auto data = checked_cast<const PrimitiveScalarBase&>(scalar).data();
+    bool extends = length > 0 ? Extend(data) : kEmptyExtends;
+    return MakeSegment(length, offset, length, extends);
+  }
+
+  Result<Segment> GetNextSegment(const DataType& array_type, const uint8_t* array_bytes,

Review Comment:
   Can we add DCHECK here that offset / length doesn't go out of bound? We are doing pointer math here so we should be *very* careful.



##########
cpp/src/arrow/compute/row/grouper.cc:
##########
@@ -39,12 +43,330 @@
 namespace arrow {
 
 using internal::checked_cast;
+using internal::PrimitiveScalarBase;
 
 namespace compute {
 
 namespace {
 
-struct GrouperImpl : Grouper {
+constexpr uint32_t kNoGroupId = std::numeric_limits<uint32_t>::max();
+
+using group_id_t = std::remove_const<decltype(kNoGroupId)>::type;
+using GroupIdType = CTypeTraits<group_id_t>::ArrowType;
+auto g_group_id_type = std::make_shared<GroupIdType>();
+
+inline const uint8_t* GetValuesAsBytes(const ArraySpan& data, int64_t offset = 0) {
+  DCHECK_GT(data.type->byte_width(), 0);
+  int64_t absolute_byte_offset = (data.offset + offset) * data.type->byte_width();
+  return data.GetValues<uint8_t>(1, absolute_byte_offset);
+}
+
+template <typename Value>
+Status CheckForGetNextSegment(const std::vector<Value>& values, int64_t length,
+                              int64_t offset, const std::vector<TypeHolder>& key_types) {
+  if (offset < 0 || offset > length) {
+    return Status::Invalid("invalid grouping segmenter offset: ", offset);
+  }
+  if (values.size() != key_types.size()) {
+    return Status::Invalid("expected batch size ", key_types.size(), " but got ",
+                           values.size());
+  }
+  for (size_t i = 0; i < key_types.size(); i++) {
+    const auto& value = values[i];
+    const auto& key_type = key_types[i];
+    if (*value.type() != *key_type.type) {
+      return Status::Invalid("expected batch value ", i, " of type ", *key_type.type,
+                             " but got ", *value.type());
+    }
+  }
+  return Status::OK();
+}
+
+template <typename Batch>
+enable_if_t<std::is_same<Batch, ExecSpan>::value || std::is_same<Batch, ExecBatch>::value,
+            Status>
+CheckForGetNextSegment(const Batch& batch, int64_t offset,
+                       const std::vector<TypeHolder>& key_types) {
+  return CheckForGetNextSegment(batch.values, batch.length, offset, key_types);
+}
+
+struct BaseRowSegmenter : public RowSegmenter {
+  explicit BaseRowSegmenter(const std::vector<TypeHolder>& key_types)
+      : key_types_(key_types) {}
+
+  const std::vector<TypeHolder>& key_types() const override { return key_types_; }
+
+  std::vector<TypeHolder> key_types_;
+};
+
+Segment MakeSegment(int64_t batch_length, int64_t offset, int64_t length, bool extends) {
+  return Segment{offset, length, offset + length >= batch_length, extends};
+}
+
+int64_t GetMatchLength(const uint8_t* match_bytes, int64_t match_width,
+                       const uint8_t* array_bytes, int64_t offset, int64_t length) {
+  int64_t cursor, byte_cursor;
+  for (cursor = offset, byte_cursor = match_width * cursor; cursor < length;
+       cursor++, byte_cursor += match_width) {
+    if (memcmp(match_bytes, array_bytes + byte_cursor,
+               static_cast<size_t>(match_width)) != 0) {
+      break;
+    }
+  }
+  return std::min(cursor, length) - offset;
+}
+
+using ExtendFunc = std::function<bool(const void*)>;
+constexpr bool kDefaultExtends = true;
+constexpr bool kEmptyExtends = true;
+
+struct NoKeysSegmenter : public BaseRowSegmenter {
+  static std::unique_ptr<RowSegmenter> Make() {
+    return std::make_unique<NoKeysSegmenter>();
+  }
+
+  NoKeysSegmenter() : BaseRowSegmenter({}) {}
+
+  Status Reset() override { return Status::OK(); }
+
+  Result<Segment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {}));
+    return MakeSegment(batch.length, offset, batch.length - offset, kDefaultExtends);
+  }
+};
+
+struct SimpleKeySegmenter : public BaseRowSegmenter {
+  static Result<std::unique_ptr<RowSegmenter>> Make(TypeHolder key_type) {
+    return std::make_unique<SimpleKeySegmenter>(key_type);
+  }
+
+  explicit SimpleKeySegmenter(TypeHolder key_type)
+      : BaseRowSegmenter({key_type}), key_type_(key_types_[0]), save_key_data_() {}
+
+  Status CheckType(const DataType& type) {
+    if (!is_fixed_width(type)) {
+      return Status::Invalid("SimpleKeySegmenter does not support type ", type);
+    }
+    return Status::OK();
+  }
+
+  Status Reset() override {
+    save_key_data_.resize(0);
+    return Status::OK();
+  }
+
+  // Checks whether the given grouping data extends the current segment, i.e., is equal to
+  // previously seen grouping data, which is updated with each invocation.
+  bool Extend(const void* data) {
+    size_t byte_width = static_cast<size_t>(key_type_.type->byte_width());
+    bool extends = save_key_data_.size() != byte_width
+                       ? kDefaultExtends
+                       : 0 == memcmp(save_key_data_.data(), data, byte_width);
+    save_key_data_.resize(byte_width);
+    memcpy(save_key_data_.data(), data, byte_width);
+    return extends;
+  }
+
+  Result<Segment> GetNextSegment(const Scalar& scalar, int64_t offset, int64_t length) {
+    ARROW_RETURN_NOT_OK(CheckType(*scalar.type));
+    if (!scalar.is_valid) {
+      return Status::Invalid("segmenting an invalid scalar");
+    }
+    auto data = checked_cast<const PrimitiveScalarBase&>(scalar).data();
+    bool extends = length > 0 ? Extend(data) : kEmptyExtends;
+    return MakeSegment(length, offset, length, extends);
+  }
+
+  Result<Segment> GetNextSegment(const DataType& array_type, const uint8_t* array_bytes,

Review Comment:
   Can we add DCHECK here that offset / length doesn't go out of bound? We are doing pointer math here so we should be **very** careful.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] icexelloss commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "icexelloss (via GitHub)" <gi...@apache.org>.
icexelloss commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1127036454


##########
cpp/src/arrow/compute/row/grouper.cc:
##########
@@ -39,12 +43,330 @@
 namespace arrow {
 
 using internal::checked_cast;
+using internal::PrimitiveScalarBase;
 
 namespace compute {
 
 namespace {
 
-struct GrouperImpl : Grouper {
+constexpr uint32_t kNoGroupId = std::numeric_limits<uint32_t>::max();
+
+using group_id_t = std::remove_const<decltype(kNoGroupId)>::type;
+using GroupIdType = CTypeTraits<group_id_t>::ArrowType;
+auto g_group_id_type = std::make_shared<GroupIdType>();
+
+inline const uint8_t* GetValuesAsBytes(const ArraySpan& data, int64_t offset = 0) {
+  DCHECK_GT(data.type->byte_width(), 0);
+  int64_t absolute_byte_offset = (data.offset + offset) * data.type->byte_width();
+  return data.GetValues<uint8_t>(1, absolute_byte_offset);
+}
+
+template <typename Value>
+Status CheckForGetNextSegment(const std::vector<Value>& values, int64_t length,
+                              int64_t offset, const std::vector<TypeHolder>& key_types) {
+  if (offset < 0 || offset > length) {
+    return Status::Invalid("invalid grouping segmenter offset: ", offset);
+  }
+  if (values.size() != key_types.size()) {
+    return Status::Invalid("expected batch size ", key_types.size(), " but got ",
+                           values.size());
+  }
+  for (size_t i = 0; i < key_types.size(); i++) {
+    const auto& value = values[i];
+    const auto& key_type = key_types[i];
+    if (*value.type() != *key_type.type) {
+      return Status::Invalid("expected batch value ", i, " of type ", *key_type.type,
+                             " but got ", *value.type());
+    }
+  }
+  return Status::OK();
+}
+
+template <typename Batch>
+enable_if_t<std::is_same<Batch, ExecSpan>::value || std::is_same<Batch, ExecBatch>::value,
+            Status>
+CheckForGetNextSegment(const Batch& batch, int64_t offset,
+                       const std::vector<TypeHolder>& key_types) {
+  return CheckForGetNextSegment(batch.values, batch.length, offset, key_types);
+}
+
+struct BaseRowSegmenter : public RowSegmenter {
+  explicit BaseRowSegmenter(const std::vector<TypeHolder>& key_types)
+      : key_types_(key_types) {}
+
+  const std::vector<TypeHolder>& key_types() const override { return key_types_; }
+
+  std::vector<TypeHolder> key_types_;
+};
+
+Segment MakeSegment(int64_t batch_length, int64_t offset, int64_t length, bool extends) {
+  return Segment{offset, length, offset + length >= batch_length, extends};
+}
+
+int64_t GetMatchLength(const uint8_t* match_bytes, int64_t match_width,
+                       const uint8_t* array_bytes, int64_t offset, int64_t length) {
+  int64_t cursor, byte_cursor;
+  for (cursor = offset, byte_cursor = match_width * cursor; cursor < length;
+       cursor++, byte_cursor += match_width) {
+    if (memcmp(match_bytes, array_bytes + byte_cursor,
+               static_cast<size_t>(match_width)) != 0) {
+      break;
+    }
+  }
+  return std::min(cursor, length) - offset;
+}
+
+using ExtendFunc = std::function<bool(const void*)>;
+constexpr bool kDefaultExtends = true;
+constexpr bool kEmptyExtends = true;
+
+struct NoKeysSegmenter : public BaseRowSegmenter {
+  static std::unique_ptr<RowSegmenter> Make() {
+    return std::make_unique<NoKeysSegmenter>();
+  }
+
+  NoKeysSegmenter() : BaseRowSegmenter({}) {}
+
+  Status Reset() override { return Status::OK(); }
+
+  Result<Segment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {}));
+    return MakeSegment(batch.length, offset, batch.length - offset, kDefaultExtends);
+  }
+};
+
+struct SimpleKeySegmenter : public BaseRowSegmenter {
+  static Result<std::unique_ptr<RowSegmenter>> Make(TypeHolder key_type) {
+    return std::make_unique<SimpleKeySegmenter>(key_type);
+  }
+
+  explicit SimpleKeySegmenter(TypeHolder key_type)
+      : BaseRowSegmenter({key_type}), key_type_(key_types_[0]), save_key_data_() {}
+
+  Status CheckType(const DataType& type) {
+    if (!is_fixed_width(type)) {
+      return Status::Invalid("SimpleKeySegmenter does not support type ", type);
+    }
+    return Status::OK();
+  }
+
+  Status Reset() override {
+    save_key_data_.resize(0);
+    return Status::OK();
+  }
+
+  // Checks whether the given grouping data extends the current segment, i.e., is equal to
+  // previously seen grouping data, which is updated with each invocation.
+  bool Extend(const void* data) {
+    size_t byte_width = static_cast<size_t>(key_type_.type->byte_width());
+    bool extends = save_key_data_.size() != byte_width
+                       ? kDefaultExtends
+                       : 0 == memcmp(save_key_data_.data(), data, byte_width);
+    save_key_data_.resize(byte_width);

Review Comment:
   Why resize here? Byte_width of key_type never changes right?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] icexelloss commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "icexelloss (via GitHub)" <gi...@apache.org>.
icexelloss commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1127154899


##########
cpp/src/arrow/compute/row/grouper.cc:
##########
@@ -39,12 +43,330 @@
 namespace arrow {
 
 using internal::checked_cast;
+using internal::PrimitiveScalarBase;
 
 namespace compute {
 
 namespace {
 
-struct GrouperImpl : Grouper {
+constexpr uint32_t kNoGroupId = std::numeric_limits<uint32_t>::max();
+
+using group_id_t = std::remove_const<decltype(kNoGroupId)>::type;
+using GroupIdType = CTypeTraits<group_id_t>::ArrowType;
+auto g_group_id_type = std::make_shared<GroupIdType>();
+
+inline const uint8_t* GetValuesAsBytes(const ArraySpan& data, int64_t offset = 0) {
+  DCHECK_GT(data.type->byte_width(), 0);
+  int64_t absolute_byte_offset = (data.offset + offset) * data.type->byte_width();
+  return data.GetValues<uint8_t>(1, absolute_byte_offset);
+}
+
+template <typename Value>
+Status CheckForGetNextSegment(const std::vector<Value>& values, int64_t length,
+                              int64_t offset, const std::vector<TypeHolder>& key_types) {
+  if (offset < 0 || offset > length) {
+    return Status::Invalid("invalid grouping segmenter offset: ", offset);
+  }
+  if (values.size() != key_types.size()) {
+    return Status::Invalid("expected batch size ", key_types.size(), " but got ",
+                           values.size());
+  }
+  for (size_t i = 0; i < key_types.size(); i++) {
+    const auto& value = values[i];
+    const auto& key_type = key_types[i];
+    if (*value.type() != *key_type.type) {
+      return Status::Invalid("expected batch value ", i, " of type ", *key_type.type,
+                             " but got ", *value.type());
+    }
+  }
+  return Status::OK();
+}
+
+template <typename Batch>
+enable_if_t<std::is_same<Batch, ExecSpan>::value || std::is_same<Batch, ExecBatch>::value,
+            Status>
+CheckForGetNextSegment(const Batch& batch, int64_t offset,
+                       const std::vector<TypeHolder>& key_types) {
+  return CheckForGetNextSegment(batch.values, batch.length, offset, key_types);
+}
+
+struct BaseRowSegmenter : public RowSegmenter {
+  explicit BaseRowSegmenter(const std::vector<TypeHolder>& key_types)
+      : key_types_(key_types) {}
+
+  const std::vector<TypeHolder>& key_types() const override { return key_types_; }
+
+  std::vector<TypeHolder> key_types_;
+};
+
+Segment MakeSegment(int64_t batch_length, int64_t offset, int64_t length, bool extends) {
+  return Segment{offset, length, offset + length >= batch_length, extends};
+}
+
+int64_t GetMatchLength(const uint8_t* match_bytes, int64_t match_width,
+                       const uint8_t* array_bytes, int64_t offset, int64_t length) {
+  int64_t cursor, byte_cursor;
+  for (cursor = offset, byte_cursor = match_width * cursor; cursor < length;
+       cursor++, byte_cursor += match_width) {
+    if (memcmp(match_bytes, array_bytes + byte_cursor,
+               static_cast<size_t>(match_width)) != 0) {
+      break;
+    }
+  }
+  return std::min(cursor, length) - offset;
+}
+
+using ExtendFunc = std::function<bool(const void*)>;
+constexpr bool kDefaultExtends = true;
+constexpr bool kEmptyExtends = true;
+
+struct NoKeysSegmenter : public BaseRowSegmenter {
+  static std::unique_ptr<RowSegmenter> Make() {
+    return std::make_unique<NoKeysSegmenter>();
+  }
+
+  NoKeysSegmenter() : BaseRowSegmenter({}) {}
+
+  Status Reset() override { return Status::OK(); }
+
+  Result<Segment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {}));
+    return MakeSegment(batch.length, offset, batch.length - offset, kDefaultExtends);
+  }
+};
+
+struct SimpleKeySegmenter : public BaseRowSegmenter {
+  static Result<std::unique_ptr<RowSegmenter>> Make(TypeHolder key_type) {
+    return std::make_unique<SimpleKeySegmenter>(key_type);
+  }
+
+  explicit SimpleKeySegmenter(TypeHolder key_type)
+      : BaseRowSegmenter({key_type}), key_type_(key_types_[0]), save_key_data_() {}
+
+  Status CheckType(const DataType& type) {
+    if (!is_fixed_width(type)) {
+      return Status::Invalid("SimpleKeySegmenter does not support type ", type);
+    }
+    return Status::OK();
+  }
+
+  Status Reset() override {
+    save_key_data_.resize(0);
+    return Status::OK();
+  }
+
+  // Checks whether the given grouping data extends the current segment, i.e., is equal to
+  // previously seen grouping data, which is updated with each invocation.
+  bool Extend(const void* data) {
+    size_t byte_width = static_cast<size_t>(key_type_.type->byte_width());
+    bool extends = save_key_data_.size() != byte_width
+                       ? kDefaultExtends
+                       : 0 == memcmp(save_key_data_.data(), data, byte_width);
+    save_key_data_.resize(byte_width);
+    memcpy(save_key_data_.data(), data, byte_width);
+    return extends;
+  }
+
+  Result<Segment> GetNextSegment(const Scalar& scalar, int64_t offset, int64_t length) {
+    ARROW_RETURN_NOT_OK(CheckType(*scalar.type));
+    if (!scalar.is_valid) {
+      return Status::Invalid("segmenting an invalid scalar");
+    }
+    auto data = checked_cast<const PrimitiveScalarBase&>(scalar).data();
+    bool extends = length > 0 ? Extend(data) : kEmptyExtends;
+    return MakeSegment(length, offset, length, extends);
+  }
+
+  Result<Segment> GetNextSegment(const DataType& array_type, const uint8_t* array_bytes,
+                                 int64_t offset, int64_t length) {
+    RETURN_NOT_OK(CheckType(array_type));
+    int64_t byte_width = array_type.byte_width();
+    int64_t match_length = GetMatchLength(array_bytes + offset * byte_width, byte_width,
+                                          array_bytes, offset, length);
+    bool extends = length > 0 ? Extend(array_bytes + offset * byte_width) : kEmptyExtends;
+    return MakeSegment(length, offset, match_length, extends);
+  }
+
+  Result<Segment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {key_type_}));
+    if (offset == batch.length) {
+      return MakeSegment(batch.length, offset, 0, kEmptyExtends);
+    }
+    const auto& value = batch.values[0];
+    if (value.is_scalar()) {
+      return GetNextSegment(*value.scalar, offset, batch.length);
+    }
+    ARROW_DCHECK(value.is_array());
+    const auto& array = value.array;
+    if (array.GetNullCount() > 0) {
+      return Status::NotImplemented("segmenting a nullable array");
+    }
+    return GetNextSegment(*array.type, GetValuesAsBytes(array), offset, batch.length);
+  }
+
+ private:
+  TypeHolder key_type_;
+  std::vector<uint8_t> save_key_data_;
+};
+
+struct AnyKeysSegmenter : public BaseRowSegmenter {
+  static Result<std::unique_ptr<RowSegmenter>> Make(
+      const std::vector<TypeHolder>& key_types, ExecContext* ctx) {
+    ARROW_RETURN_NOT_OK(Grouper::Make(key_types, ctx));  // check types
+    return std::make_unique<AnyKeysSegmenter>(key_types, ctx);
+  }
+
+  AnyKeysSegmenter(const std::vector<TypeHolder>& key_types, ExecContext* ctx)
+      : BaseRowSegmenter(key_types),
+        ctx_(ctx),
+        grouper_(nullptr),
+        save_group_id_(kNoGroupId) {}
+
+  Status Reset() override {
+    grouper_ = nullptr;
+    save_group_id_ = kNoGroupId;
+    return Status::OK();
+  }
+
+  bool Extend(const void* data) {
+    auto group_id = *static_cast<const group_id_t*>(data);
+    bool extends =
+        save_group_id_ == kNoGroupId ? kDefaultExtends : save_group_id_ == group_id;
+    save_group_id_ = group_id;
+    return extends;
+  }
+
+  // Runs the grouper on a single row.  This is used to determine the group id of the
+  // first row of a new segment to see if it extends the previous segment.
+  template <typename Batch>
+  Result<group_id_t> MapGroupIdAt(const Batch& batch, int64_t offset) {
+    if (!grouper_) return kNoGroupId;
+    ARROW_ASSIGN_OR_RAISE(auto datum, grouper_->Consume(batch, offset,
+                                                        /*length=*/1));
+    if (!datum.is_array()) {
+      return Status::Invalid("accessing unsupported datum kind ", datum.kind());
+    }
+    const std::shared_ptr<ArrayData>& data = datum.array();
+    ARROW_DCHECK(data->GetNullCount() == 0);
+    DCHECK_EQ(data->type->id(), GroupIdType::type_id);
+    DCHECK_EQ(1, data->length);
+    const group_id_t* values = data->GetValues<group_id_t>(1);

Review Comment:
   Can you add comments here? 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] icexelloss commented on pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "icexelloss (via GitHub)" <gi...@apache.org>.
icexelloss commented on PR #34311:
URL: https://github.com/apache/arrow/pull/34311#issuecomment-1452616880

   @rtpsw I have finished reviewing the aggregate node changes and half way through the grouper changes. In addition to the comments I made here, I've also consolidated some the my comments into this PR to you branch:
   https://github.com/apache/arrow/pull/34311
   Please take a look and merge it if you can.
   
   So far the general approach looks good I will try to finish up reviewing grouper changes tomorrow.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rtpsw commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "rtpsw (via GitHub)" <gi...@apache.org>.
rtpsw commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1122260682


##########
cpp/src/arrow/compute/exec/aggregate_node.cc:
##########
@@ -169,35 +186,117 @@ void AggregatesToString(std::stringstream* ss, const Schema& input_schema,
   *ss << ']';
 }
 
+template <typename BatchHandler>
+Status HandleSegments(std::unique_ptr<GroupingSegmenter>& segmenter,
+                      const ExecBatch& batch, const std::vector<int>& ids,
+                      const BatchHandler& handle_batch) {
+  int64_t offset = 0;
+  ARROW_ASSIGN_OR_RAISE(auto segment_exec_batch, batch.SelectValues(ids));
+  ExecSpan segment_batch(segment_exec_batch);
+  while (true) {
+    ARROW_ASSIGN_OR_RAISE(auto segment, segmenter->GetNextSegment(segment_batch, offset));
+    if (segment.offset >= segment_batch.length) break;  // condition of no-next-segment
+    ARROW_RETURN_NOT_OK(handle_batch(batch, segment));
+    offset = segment.offset + segment.length;
+  }
+  return Status::OK();
+}
+
+Status GetScalarFields(std::vector<Datum>* values_ptr, const ExecBatch& input_batch,
+                       const std::vector<int>& field_ids) {
+  DCHECK_GT(input_batch.length, 0);
+  std::vector<Datum>& values = *values_ptr;
+  int64_t row = input_batch.length - 1;
+  values.clear();
+  values.resize(field_ids.size());
+  for (size_t i = 0; i < field_ids.size(); i++) {
+    const Datum& value = input_batch.values[field_ids[i]];
+    if (value.is_scalar()) {
+      values[i] = value;
+    } else if (value.is_array()) {
+      ARROW_ASSIGN_OR_RAISE(auto scalar, value.make_array()->GetScalar(row));
+      values[i] = scalar;
+    } else {
+      DCHECK(false);
+    }
+  }
+  return Status::OK();
+}
+
+void PlaceFields(ExecBatch& batch, size_t base, std::vector<Datum>& values) {
+  DCHECK_LE(base + values.size(), batch.values.size());
+  for (size_t i = 0; i < values.size(); i++) {
+    batch.values[base + i] = values[i];
+  }
+}
+
 class ScalarAggregateNode : public ExecNode, public TracedNode {
  public:
   ScalarAggregateNode(ExecPlan* plan, std::vector<ExecNode*> inputs,
                       std::shared_ptr<Schema> output_schema,
+                      std::unique_ptr<GroupingSegmenter> segmenter,
+                      std::vector<int> segment_field_ids,
                       std::vector<std::vector<int>> target_fieldsets,
                       std::vector<Aggregate> aggs,
                       std::vector<const ScalarAggregateKernel*> kernels,
                       std::vector<std::vector<std::unique_ptr<KernelState>>> states)
       : ExecNode(plan, std::move(inputs), {"target"},
                  /*output_schema=*/std::move(output_schema)),
         TracedNode(this),
+        segmenter_(std::move(segmenter)),
+        segment_field_ids_(std::move(segment_field_ids)),
         target_fieldsets_(std::move(target_fieldsets)),
         aggs_(std::move(aggs)),
         kernels_(std::move(kernels)),
-        states_(std::move(states)) {}
+        states_(std::move(states)) {
+    const auto& input_schema = *this->inputs()[0]->output_schema();
+    for (size_t i = 0; i < kernels_.size(); ++i) {
+      std::vector<TypeHolder> in_types;
+      for (const auto& target : target_fieldsets_[i]) {
+        in_types.emplace_back(input_schema.field(target)->type().get());
+      }
+      in_typesets_.push_back(std::move(in_types));
+    }
+  }
 
   static Result<ExecNode*> Make(ExecPlan* plan, std::vector<ExecNode*> inputs,
                                 const ExecNodeOptions& options) {
     RETURN_NOT_OK(ValidateExecNodeInputs(plan, inputs, 1, "ScalarAggregateNode"));
 
     const auto& aggregate_options = checked_cast<const AggregateNodeOptions&>(options);
     auto aggregates = aggregate_options.aggregates;
+    const auto& keys = aggregate_options.keys;
+    const auto& segment_keys = aggregate_options.segment_keys;
+
+    if (keys.size() > 0) {
+      return Status::Invalid("Scalar aggregation with some key");
+    }
+    if (plan->query_context()->exec_context()->executor()->GetCapacity() > 1 &&
+        segment_keys.size() > 0) {
+      return Status::NotImplemented("Segmented aggregation in a multi-threaded plan");
+    }
 
     const auto& input_schema = *inputs[0]->output_schema();
     auto exec_ctx = plan->query_context()->exec_context();
 
+    std::vector<int> segment_field_ids(segment_keys.size());
+    std::vector<TypeHolder> segment_key_types(segment_keys.size());
+    for (size_t i = 0; i < segment_keys.size(); i++) {
+      ARROW_ASSIGN_OR_RAISE(auto match, segment_keys[i].FindOne(input_schema));
+      if (match.indices().size() > 1) {
+        // ARROW-18369: Support nested references as segment ids

Review Comment:
   A nested reference is a reference to a nested struct. It refers to `FieldRef` with a multi-component path, e.g. `[0, 2, 1]` selects the index-0 member, within it the index-2 field, and within it the index-1 field. The ARROW issue mentioned in the comment leads to [this post](https://github.com/apache/arrow/pull/14352#discussion_r1026945315).



##########
cpp/src/arrow/compute/exec/aggregate_node.cc:
##########
@@ -169,35 +186,117 @@ void AggregatesToString(std::stringstream* ss, const Schema& input_schema,
   *ss << ']';
 }
 
+template <typename BatchHandler>
+Status HandleSegments(std::unique_ptr<GroupingSegmenter>& segmenter,
+                      const ExecBatch& batch, const std::vector<int>& ids,
+                      const BatchHandler& handle_batch) {
+  int64_t offset = 0;
+  ARROW_ASSIGN_OR_RAISE(auto segment_exec_batch, batch.SelectValues(ids));
+  ExecSpan segment_batch(segment_exec_batch);
+  while (true) {
+    ARROW_ASSIGN_OR_RAISE(auto segment, segmenter->GetNextSegment(segment_batch, offset));
+    if (segment.offset >= segment_batch.length) break;  // condition of no-next-segment
+    ARROW_RETURN_NOT_OK(handle_batch(batch, segment));
+    offset = segment.offset + segment.length;
+  }
+  return Status::OK();
+}
+
+Status GetScalarFields(std::vector<Datum>* values_ptr, const ExecBatch& input_batch,
+                       const std::vector<int>& field_ids) {
+  DCHECK_GT(input_batch.length, 0);
+  std::vector<Datum>& values = *values_ptr;
+  int64_t row = input_batch.length - 1;
+  values.clear();
+  values.resize(field_ids.size());
+  for (size_t i = 0; i < field_ids.size(); i++) {
+    const Datum& value = input_batch.values[field_ids[i]];
+    if (value.is_scalar()) {
+      values[i] = value;
+    } else if (value.is_array()) {
+      ARROW_ASSIGN_OR_RAISE(auto scalar, value.make_array()->GetScalar(row));
+      values[i] = scalar;
+    } else {
+      DCHECK(false);
+    }
+  }
+  return Status::OK();
+}
+
+void PlaceFields(ExecBatch& batch, size_t base, std::vector<Datum>& values) {
+  DCHECK_LE(base + values.size(), batch.values.size());
+  for (size_t i = 0; i < values.size(); i++) {
+    batch.values[base + i] = values[i];
+  }
+}
+
 class ScalarAggregateNode : public ExecNode, public TracedNode {
  public:
   ScalarAggregateNode(ExecPlan* plan, std::vector<ExecNode*> inputs,
                       std::shared_ptr<Schema> output_schema,
+                      std::unique_ptr<GroupingSegmenter> segmenter,
+                      std::vector<int> segment_field_ids,
                       std::vector<std::vector<int>> target_fieldsets,
                       std::vector<Aggregate> aggs,
                       std::vector<const ScalarAggregateKernel*> kernels,
                       std::vector<std::vector<std::unique_ptr<KernelState>>> states)
       : ExecNode(plan, std::move(inputs), {"target"},
                  /*output_schema=*/std::move(output_schema)),
         TracedNode(this),
+        segmenter_(std::move(segmenter)),
+        segment_field_ids_(std::move(segment_field_ids)),
         target_fieldsets_(std::move(target_fieldsets)),
         aggs_(std::move(aggs)),
         kernels_(std::move(kernels)),
-        states_(std::move(states)) {}
+        states_(std::move(states)) {
+    const auto& input_schema = *this->inputs()[0]->output_schema();
+    for (size_t i = 0; i < kernels_.size(); ++i) {
+      std::vector<TypeHolder> in_types;
+      for (const auto& target : target_fieldsets_[i]) {
+        in_types.emplace_back(input_schema.field(target)->type().get());
+      }
+      in_typesets_.push_back(std::move(in_types));
+    }
+  }
 
   static Result<ExecNode*> Make(ExecPlan* plan, std::vector<ExecNode*> inputs,
                                 const ExecNodeOptions& options) {
     RETURN_NOT_OK(ValidateExecNodeInputs(plan, inputs, 1, "ScalarAggregateNode"));
 
     const auto& aggregate_options = checked_cast<const AggregateNodeOptions&>(options);
     auto aggregates = aggregate_options.aggregates;
+    const auto& keys = aggregate_options.keys;
+    const auto& segment_keys = aggregate_options.segment_keys;
+
+    if (keys.size() > 0) {
+      return Status::Invalid("Scalar aggregation with some key");
+    }
+    if (plan->query_context()->exec_context()->executor()->GetCapacity() > 1 &&
+        segment_keys.size() > 0) {
+      return Status::NotImplemented("Segmented aggregation in a multi-threaded plan");
+    }
 
     const auto& input_schema = *inputs[0]->output_schema();
     auto exec_ctx = plan->query_context()->exec_context();
 
+    std::vector<int> segment_field_ids(segment_keys.size());
+    std::vector<TypeHolder> segment_key_types(segment_keys.size());
+    for (size_t i = 0; i < segment_keys.size(); i++) {
+      ARROW_ASSIGN_OR_RAISE(auto match, segment_keys[i].FindOne(input_schema));
+      if (match.indices().size() > 1) {
+        // ARROW-18369: Support nested references as segment ids

Review Comment:
   A nested reference is a reference into a nested struct. It refers to `FieldRef` with a multi-component path, e.g. `[0, 2, 1]` selects the index-0 member, within it the index-2 field, and within it the index-1 field. The ARROW issue mentioned in the comment leads to [this post](https://github.com/apache/arrow/pull/14352#discussion_r1026945315).



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] icexelloss commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "icexelloss (via GitHub)" <gi...@apache.org>.
icexelloss commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1123680876


##########
cpp/src/arrow/compute/exec/aggregate_node.cc:
##########
@@ -326,46 +446,86 @@ class ScalarAggregateNode : public ExecNode, public TracedNode {
   }
 
  private:
-  Status Finish() {
-    auto scope = TraceFinish();
+  Status ReconstructAggregates() {
+    const auto& input_schema = *inputs()[0]->output_schema();
+    auto exec_ctx = plan()->query_context()->exec_context();
+    for (size_t i = 0; i < kernels_.size(); ++i) {
+      std::vector<TypeHolder> in_types;
+      for (const auto& target : target_fieldsets_[i]) {
+        in_types.emplace_back(input_schema.field(target)->type().get());
+      }
+      states_[i].resize(plan()->query_context()->max_concurrency());

Review Comment:
   I understand the state needs to be reset - my point is that we don't need to resize this because different segments should not change the size here



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] icexelloss commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "icexelloss (via GitHub)" <gi...@apache.org>.
icexelloss commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1123695688


##########
cpp/src/arrow/compute/exec/aggregate_node.cc:
##########
@@ -697,16 +925,22 @@ class GroupByNode : public ExecNode, public TracedNode {
   }
 
   int output_task_group_id_;
+  std::unique_ptr<GroupingSegmenter> segmenter_;
+  std::vector<Datum> segmenter_values_;
 
   const std::vector<int> key_field_ids_;
+  const std::vector<int> segment_key_field_ids_;
+  const std::vector<std::vector<TypeHolder>> agg_src_types_;
   const std::vector<std::vector<int>> agg_src_fieldsets_;
   const std::vector<Aggregate> aggs_;
-  const std::vector<const HashAggregateKernel*> agg_kernels_;
+  std::vector<const HashAggregateKernel*> agg_kernels_;

Review Comment:
   why do we need to reset kernels? Is resetting kernel states not enough?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] westonpace commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "westonpace (via GitHub)" <gi...@apache.org>.
westonpace commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1119133819


##########
cpp/src/arrow/compute/exec/aggregate_node.cc:
##########
@@ -16,9 +16,11 @@
 // under the License.
 
 #include <mutex>
+#include <shared_mutex>

Review Comment:
   ```suggestion
   ```



##########
cpp/src/arrow/compute/exec/aggregate_node.cc:
##########
@@ -326,46 +438,77 @@ class ScalarAggregateNode : public ExecNode, public TracedNode {
   }
 
  private:
-  Status Finish() {
-    auto scope = TraceFinish();
+  Status ResetAggregates() {
+    auto exec_ctx = plan()->query_context()->exec_context();
+    for (size_t i = 0; i < kernels_.size(); ++i) {
+      const std::vector<TypeHolder>& in_types = in_typesets_[i];
+      states_[i].resize(plan()->query_context()->max_concurrency());
+      KernelContext kernel_ctx{exec_ctx};
+      RETURN_NOT_OK(Kernel::InitAll(
+          &kernel_ctx, KernelInitArgs{kernels_[i], in_types, aggs_[i].options.get()},
+          &states_[i]));
+    }
+    return Status::OK();
+  }
+
+  Status OutputResult(bool is_last = false) {
     ExecBatch batch{{}, 1};
-    batch.values.resize(kernels_.size());
+    batch.values.resize(kernels_.size() + segment_field_ids_.size());
 
     for (size_t i = 0; i < kernels_.size(); ++i) {
       util::tracing::Span span;
       START_COMPUTE_SPAN(span, aggs_[i].function,
                          {{"function.name", aggs_[i].function},
                           {"function.options",
                            aggs_[i].options ? aggs_[i].options->ToString() : "<NULLPTR>"},
-                          {"function.kind", std::string(kind_name()) + "::Finalize"}});
+                          {"function.kind", std::string(kind_name()) + "::Output"}});

Review Comment:
   Hmm, maybe keep the old name here?  This is tracking how much time is spent in the two kernel calls `MergeAll` and `Finalize` so I think `::Finalize` is more accurate.



##########
cpp/src/arrow/compute/exec/aggregate_node.cc:
##########
@@ -16,9 +16,11 @@
 // under the License.
 
 #include <mutex>
+#include <shared_mutex>

Review Comment:
   ```suggestion
   ```



##########
cpp/src/arrow/compute/exec/aggregate_node.cc:
##########
@@ -584,29 +778,50 @@ class GroupByNode : public ExecNode, public TracedNode {
     ARROW_ASSIGN_OR_RAISE(out_data_, Finalize());
 
     int64_t num_output_batches = bit_util::CeilDiv(out_data_.length, output_batch_size());
-    RETURN_NOT_OK(output_->InputFinished(this, static_cast<int>(num_output_batches)));
-    return plan_->query_context()->StartTaskGroup(output_task_group_id_,
-                                                  num_output_batches);
+    total_output_batches_ += static_cast<int>(num_output_batches);
+    if (is_last) {
+      ARROW_RETURN_NOT_OK(output_->InputFinished(this, total_output_batches_));
+      RETURN_NOT_OK(plan_->query_context()->StartTaskGroup(output_task_group_id_,
+                                                           num_output_batches));
+    } else {
+      for (int64_t i = 0; i < num_output_batches; i++) {
+        ARROW_RETURN_NOT_OK(OutputNthBatch(i));

Review Comment:
   This is a little bizarre (though not introduced by you).  If we are outputting an Nth batch we just output it immediately.  However, when outputting the final batch we divide it into 32k chunks and schedule tasks for each one.  I think, longer term, we will want to implement accumulation for segmented group-by, since some segments may be quite small (and we might only output a few rows per segment depending on the grouping keys).
   
   We have some prototype for this in `ExecBatchBuilder` (in `light_array.h`).  However, let's definitely save for a follow-up.  Do you want to create an issue or should I?



##########
cpp/src/arrow/compute/exec/options.h:
##########
@@ -199,21 +199,39 @@ class ARROW_EXPORT ProjectNodeOptions : public ExecNodeOptions {
   std::vector<std::string> names;
 };
 
-/// \brief Make a node which aggregates input batches, optionally grouped by keys.
+/// \brief Make a node which aggregates input batches, optionally grouped by keys and
+/// optionally segmented by segment-keys. Both keys and segment-keys determine the group.
+/// However segment-keys are also used for determining grouping segments, which should be
+/// large, and allow streaming a partial aggregation result after processing each segment.

Review Comment:
   Perhaps, if implement accumulation of groups, we could remove the "which should be large" from the advice here?



##########
cpp/src/arrow/compute/exec/aggregate_node.cc:
##########
@@ -326,46 +438,77 @@ class ScalarAggregateNode : public ExecNode, public TracedNode {
   }
 
  private:
-  Status Finish() {
-    auto scope = TraceFinish();
+  Status ResetAggregates() {
+    auto exec_ctx = plan()->query_context()->exec_context();
+    for (size_t i = 0; i < kernels_.size(); ++i) {
+      const std::vector<TypeHolder>& in_types = in_typesets_[i];
+      states_[i].resize(plan()->query_context()->max_concurrency());
+      KernelContext kernel_ctx{exec_ctx};
+      RETURN_NOT_OK(Kernel::InitAll(
+          &kernel_ctx, KernelInitArgs{kernels_[i], in_types, aggs_[i].options.get()},
+          &states_[i]));
+    }
+    return Status::OK();
+  }
+
+  Status OutputResult(bool is_last = false) {
     ExecBatch batch{{}, 1};
-    batch.values.resize(kernels_.size());
+    batch.values.resize(kernels_.size() + segment_field_ids_.size());
 
     for (size_t i = 0; i < kernels_.size(); ++i) {
       util::tracing::Span span;
       START_COMPUTE_SPAN(span, aggs_[i].function,
                          {{"function.name", aggs_[i].function},
                           {"function.options",
                            aggs_[i].options ? aggs_[i].options->ToString() : "<NULLPTR>"},
-                          {"function.kind", std::string(kind_name()) + "::Finalize"}});
+                          {"function.kind", std::string(kind_name()) + "::Output"}});
       KernelContext ctx{plan()->query_context()->exec_context()};
       ARROW_ASSIGN_OR_RAISE(auto merged, ScalarAggregateKernel::MergeAll(
                                              kernels_[i], &ctx, std::move(states_[i])));
       RETURN_NOT_OK(kernels_[i]->finalize(&ctx, &batch.values[i]));
     }
+    PlaceFields(batch, kernels_.size(), segmenter_values_);
 
-    return output_->InputReceived(this, std::move(batch));
+    ARROW_RETURN_NOT_OK(output_->InputReceived(this, std::move(batch)));
+    total_output_batches_++;
+    if (is_last) {
+      ARROW_RETURN_NOT_OK(output_->InputFinished(this, total_output_batches_));
+    } else {
+      ARROW_RETURN_NOT_OK(ResetAggregates());
+    }
+    return Status::OK();

Review Comment:
   ```suggestion
       if (is_last) {
         return output_->InputFinished(this, total_output_batches_);
       }
       return ResetAggregates();
   ```
   
   Minor nit: it's just a touch more compact.



##########
cpp/src/arrow/compute/row/grouper.cc:
##########
@@ -39,12 +43,336 @@
 namespace arrow {
 
 using internal::checked_cast;
+using internal::PrimitiveScalarBase;
 
 namespace compute {
 
 namespace {
 
-struct GrouperImpl : Grouper {
+constexpr uint32_t kNoGroupId = std::numeric_limits<uint32_t>::max();
+
+using group_id_t = std::remove_const<decltype(kNoGroupId)>::type;
+using GroupIdType = CTypeTraits<group_id_t>::ArrowType;
+auto group_id_type = std::make_shared<GroupIdType>();

Review Comment:
   ```suggestion
   const std::shared_ptr<DataType>& group_id_type() {
     static auto instance = std::make_shared<GroupIdType>();
     return instance;
   }
   ```
   
   Just a general aversion to global state.  However, if you want to keep it then it's probably ok.  Maybe just rename to `g_group_id_type`?



##########
cpp/src/arrow/compute/exec/options.h:
##########
@@ -199,21 +199,39 @@ class ARROW_EXPORT ProjectNodeOptions : public ExecNodeOptions {
   std::vector<std::string> names;
 };
 
-/// \brief Make a node which aggregates input batches, optionally grouped by keys.
+/// \brief Make a node which aggregates input batches, optionally grouped by keys and
+/// optionally segmented by segment-keys. Both keys and segment-keys determine the group.
+/// However segment-keys are also used for determining grouping segments, which should be
+/// large, and allow streaming a partial aggregation result after processing each segment.
+/// One common use-case for segment-keys is ordered aggregation, in which the segment-key
+/// attribute specifies a column with non-decreasing values or a lexigographically-ordered

Review Comment:
   ```suggestion
   /// attribute specifies a column with non-decreasing values or a lexicographically-ordered
   ```



##########
cpp/src/arrow/compute/row/grouper.h:
##########
@@ -30,6 +30,69 @@
 namespace arrow {
 namespace compute {
 
+/// \brief A segment of contiguous rows for grouping
+struct ARROW_EXPORT GroupingSegment {
+  /// \brief the offset into the batch where the segment starts
+  int64_t offset;
+  /// \brief the length of the segment
+  int64_t length;
+  /// \brief whether the segment may be extended by a next one
+  bool is_open;
+  /// \brief whether the segment extends a preceeding one
+  bool extends;
+};
+
+inline bool operator==(const GroupingSegment& segment1, const GroupingSegment& segment2) {
+  return segment1.offset == segment2.offset && segment1.length == segment2.length &&
+         segment1.is_open == segment2.is_open && segment1.extends == segment2.extends;
+}
+inline bool operator!=(const GroupingSegment& segment1, const GroupingSegment& segment2) {
+  return !(segment1 == segment2);
+}

Review Comment:
   Is this used anywhere today?  It might be useful to have for unit tests so I don't think we should get rid of it.  Just checking my understanding.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] icexelloss commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "icexelloss (via GitHub)" <gi...@apache.org>.
icexelloss commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1128109837


##########
cpp/src/arrow/compute/kernels/hash_aggregate_test.cc:
##########
@@ -72,6 +73,12 @@ using internal::ToChars;
 namespace compute {
 namespace {
 
+bool kDefaultUseThreads = false;

Review Comment:
   This is a bit confusing - it seems that we only use this default in segment aggregation in two places, and this is also not the default for hash aggregation? We can probably remove this variable and specify explicitly in segment aggregation



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] icexelloss commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "icexelloss (via GitHub)" <gi...@apache.org>.
icexelloss commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1128375752


##########
cpp/src/arrow/compute/kernels/hash_aggregate_test.cc:
##########
@@ -301,53 +437,249 @@ Result<Datum> GroupByTest(const std::vector<Datum>& arguments,
         {t_agg.function, t_agg.options, "agg_" + ToChars(idx), t_agg.function});
     idx = idx + 1;
   }
-  return RunGroupBy(arguments, keys, internal_aggregates, use_threads);
+  return group_by(arguments, keys, segment_keys, internal_aggregates, use_threads,
+                  /*naive=*/false);
 }
 
-}  // namespace
+Result<Datum> GroupByTest(GroupByFunction group_by, const std::vector<Datum>& arguments,
+                          const std::vector<Datum>& keys,
+                          const std::vector<TestAggregate>& aggregates,
+                          bool use_threads) {
+  return GroupByTest(group_by, arguments, keys, {}, aggregates, use_threads);
+}
 
-TEST(Grouper, SupportedKeys) {
-  ASSERT_OK(Grouper::Make({boolean()}));
+template <typename GroupClass>
+void TestGroupClassSupportedKeys(
+    std::function<Result<std::unique_ptr<GroupClass>>(const std::vector<TypeHolder>&)>
+        make_func) {
+  ASSERT_OK(make_func({boolean()}));
 
-  ASSERT_OK(Grouper::Make({int8(), uint16(), int32(), uint64()}));
+  ASSERT_OK(make_func({int8(), uint16(), int32(), uint64()}));
 
-  ASSERT_OK(Grouper::Make({dictionary(int64(), utf8())}));
+  ASSERT_OK(make_func({dictionary(int64(), utf8())}));
 
-  ASSERT_OK(Grouper::Make({float16(), float32(), float64()}));
+  ASSERT_OK(make_func({float16(), float32(), float64()}));
 
-  ASSERT_OK(Grouper::Make({utf8(), binary(), large_utf8(), large_binary()}));
+  ASSERT_OK(make_func({utf8(), binary(), large_utf8(), large_binary()}));
 
-  ASSERT_OK(Grouper::Make({fixed_size_binary(16), fixed_size_binary(32)}));
+  ASSERT_OK(make_func({fixed_size_binary(16), fixed_size_binary(32)}));
 
-  ASSERT_OK(Grouper::Make({decimal128(32, 10), decimal256(76, 20)}));
+  ASSERT_OK(make_func({decimal128(32, 10), decimal256(76, 20)}));
 
-  ASSERT_OK(Grouper::Make({date32(), date64()}));
+  ASSERT_OK(make_func({date32(), date64()}));
 
   for (auto unit : {
            TimeUnit::SECOND,
            TimeUnit::MILLI,
            TimeUnit::MICRO,
            TimeUnit::NANO,
        }) {
-    ASSERT_OK(Grouper::Make({timestamp(unit), duration(unit)}));
+    ASSERT_OK(make_func({timestamp(unit), duration(unit)}));
   }
 
   ASSERT_OK(
-      Grouper::Make({day_time_interval(), month_interval(), month_day_nano_interval()}));
+      make_func({day_time_interval(), month_interval(), month_day_nano_interval()}));
+
+  ASSERT_OK(make_func({null()}));
+
+  ASSERT_RAISES(NotImplemented, make_func({struct_({field("", int64())})}));
+
+  ASSERT_RAISES(NotImplemented, make_func({struct_({})}));
+
+  ASSERT_RAISES(NotImplemented, make_func({list(int32())}));
+
+  ASSERT_RAISES(NotImplemented, make_func({fixed_size_list(int32(), 5)}));
+
+  ASSERT_RAISES(NotImplemented, make_func({dense_union({field("", int32())})}));
+}
+
+void TestSegments(std::unique_ptr<RowSegmenter>& segmenter, const ExecSpan& batch,
+                  std::vector<Segment> expected_segments) {
+  int64_t offset = 0, segment_num = 0;
+  for (auto expected_segment : expected_segments) {
+    SCOPED_TRACE("segment #" + ToChars(segment_num++));
+    ASSERT_OK_AND_ASSIGN(auto segment, segmenter->GetNextSegment(batch, offset));
+    ASSERT_EQ(expected_segment, segment);
+    offset = segment.offset + segment.length;
+  }
+}
+
+Result<std::unique_ptr<Grouper>> MakeGrouper(const std::vector<TypeHolder>& key_types) {
+  return Grouper::Make(key_types, default_exec_context());
+}
+
+Result<std::unique_ptr<RowSegmenter>> MakeRowSegmenter(
+    const std::vector<TypeHolder>& key_types) {
+  return RowSegmenter::Make(key_types, /*nullable_leys=*/false, default_exec_context());
+}
+
+Result<std::unique_ptr<RowSegmenter>> MakeGenericSegmenter(
+    const std::vector<TypeHolder>& key_types) {
+  return MakeAnyKeysSegmenter(key_types, default_exec_context());
+}
+
+}  // namespace
+
+TEST(RowSegmenter, SupportedKeys) {
+  TestGroupClassSupportedKeys<RowSegmenter>(MakeRowSegmenter);
+}
+
+TEST(RowSegmenter, Basics) {
+  std::vector<TypeHolder> bad_types2 = {int32(), float32()};
+  std::vector<TypeHolder> types2 = {int32(), int32()};
+  std::vector<TypeHolder> bad_types1 = {float32()};
+  std::vector<TypeHolder> types1 = {int32()};
+  std::vector<TypeHolder> types0 = {};
+  auto batch2 = ExecBatchFromJSON(types2, "[[1, 1], [1, 2], [2, 2]]");

Review Comment:
   Can you add a few more test cases here? The current test case seems trivial. 
   
   A few ideas:
   (1) Non-ordered segment case, e.g., `1, 1, 2, 2, 1, 1`
   (2) Empty batches (This should included in the end-to-end test)
   (3) More than 2 segments inside one partition
   
   I suggest try to cover as much edge case you can here



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] icexelloss commented on pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "icexelloss (via GitHub)" <gi...@apache.org>.
icexelloss commented on PR #34311:
URL: https://github.com/apache/arrow/pull/34311#issuecomment-1460706829

   @rtpsw there are many follow up items from this PR, can you include the list of follows up in PR title so we have at least some ways to track it? If you have a GH issue, please list follow ups here as well.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] icexelloss commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "icexelloss (via GitHub)" <gi...@apache.org>.
icexelloss commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1127984376


##########
cpp/src/arrow/compute/row/grouper.cc:
##########
@@ -39,12 +43,521 @@
 namespace arrow {
 
 using internal::checked_cast;
+using internal::PrimitiveScalarBase;
 
 namespace compute {
 
 namespace {
 
-struct GrouperImpl : Grouper {
+constexpr uint32_t kNoGroupId = std::numeric_limits<uint32_t>::max();
+
+using group_id_t = std::remove_const<decltype(kNoGroupId)>::type;
+using GroupIdType = CTypeTraits<group_id_t>::ArrowType;
+auto group_id_type = std::make_shared<GroupIdType>();
+
+inline const uint8_t* GetValuesAsBytes(const ArrayData& data, int64_t offset = 0) {
+  DCHECK_GT(data.type->byte_width(), 0);
+  int64_t absolute_byte_offset = (data.offset + offset) * data.type->byte_width();
+  return data.GetValues<uint8_t>(1, absolute_byte_offset);
+}
+
+inline const uint8_t* GetValuesAsBytes(const ArraySpan& data, int64_t offset = 0) {
+  DCHECK_GT(data.type->byte_width(), 0);
+  int64_t absolute_byte_offset = (data.offset + offset) * data.type->byte_width();
+  return data.GetValues<uint8_t>(1, absolute_byte_offset);
+}
+
+template <typename Value>
+Status CheckForGetNextSegment(const std::vector<Value>& values, int64_t length,
+                              int64_t offset, const std::vector<TypeHolder>& key_types) {
+  if (offset < 0 || offset > length) {
+    return Status::Invalid("invalid grouping segmenter offset: ", offset);
+  }
+  if (values.size() != key_types.size()) {
+    return Status::Invalid("expected batch size ", key_types.size(), " but got ",
+                           values.size());
+  }
+  for (size_t i = 0; i < key_types.size(); i++) {
+    const auto& value = values[i];
+    const auto& key_type = key_types[i];
+    if (*value.type() != *key_type.type) {
+      return Status::Invalid("expected batch value ", i, " of type ", *key_type.type,
+                             " but got ", *value.type());
+    }
+  }
+  return Status::OK();
+}
+
+template <typename Batch>
+enable_if_t<std::is_same<Batch, ExecSpan>::value || std::is_same<Batch, ExecBatch>::value,
+            Status>
+CheckForGetNextSegment(const Batch& batch, int64_t offset,
+                       const std::vector<TypeHolder>& key_types) {
+  return CheckForGetNextSegment(batch.values, batch.length, offset, key_types);
+}
+
+struct BaseGroupingSegmenter : public GroupingSegmenter {
+  explicit BaseGroupingSegmenter(const std::vector<TypeHolder>& key_types)
+      : key_types_(key_types) {}
+
+  const std::vector<TypeHolder>& key_types() const override { return key_types_; }
+
+  std::vector<TypeHolder> key_types_;
+};
+
+GroupingSegment MakeSegment(int64_t batch_length, int64_t offset, int64_t length,
+                            bool extends) {
+  return GroupingSegment{offset, length, offset + length >= batch_length, extends};
+}
+
+int64_t GetMatchLength(const uint8_t* match_bytes, int64_t match_width,

Review Comment:
   Thanks - I would argue that while generalization is usually good, in this case if it comes performance cost then I would think twice, especially when ordered aggregation is the main goal here and the more general form (segmented data) is not really a use we care about internally. But for now we can proceed with this.  



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] icexelloss commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "icexelloss (via GitHub)" <gi...@apache.org>.
icexelloss commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1128122232


##########
cpp/src/arrow/compute/kernels/hash_aggregate_test.cc:
##########
@@ -174,81 +255,117 @@ Result<Datum> RunGroupBy(const BatchesWithSchema& input,
   ARROW_ASSIGN_OR_RAISE(std::vector<ExecBatch> output_batches,
                         start_and_collect.MoveResult());
 
-  ArrayVector out_arrays(aggregates.size() + key_names.size());
   const auto& output_schema = plan->nodes()[0]->output()->output_schema();
+  if (!segmented) {
+    return MakeGroupByOutput(output_batches, output_schema, aggregates.size(),
+                             key_names.size(), naive);
+  }
+
+  std::vector<ArrayVector> out_arrays(aggregates.size() + key_names.size() +
+                                      segment_key_names.size());
   for (size_t i = 0; i < out_arrays.size(); ++i) {
     std::vector<std::shared_ptr<Array>> arrays(output_batches.size());
     for (size_t j = 0; j < output_batches.size(); ++j) {
-      arrays[j] = output_batches[j].values[i].make_array();
+      auto& value = output_batches[j].values[i];
+      if (value.is_scalar()) {
+        ARROW_ASSIGN_OR_RAISE(
+            arrays[j], MakeArrayFromScalar(*value.scalar(), output_batches[j].length));
+      } else if (value.is_array()) {
+        arrays[j] = value.make_array();
+      } else {
+        return Status::Invalid("GroupByUsingExecPlan unsupported value kind ",
+                               ToString(value.kind()));
+      }
     }
     if (arrays.empty()) {
+      arrays.resize(1);
       ARROW_ASSIGN_OR_RAISE(
-          out_arrays[i],
-          MakeArrayOfNull(output_schema->field(static_cast<int>(i))->type(),
-                          /*length=*/0));
-    } else {
-      ARROW_ASSIGN_OR_RAISE(out_arrays[i], Concatenate(arrays));
+          arrays[0], MakeArrayOfNull(output_schema->field(static_cast<int>(i))->type(),
+                                     /*length=*/0));
     }
+    out_arrays[i] = {std::move(arrays)};
   }
 
-  // The exec plan may reorder the output rows.  The tests are all setup to expect ouptut
-  // in ascending order of keys.  So we need to sort the result by the key columns.  To do
-  // that we create a table using the key columns, calculate the sort indices from that
-  // table (sorting on all fields) and then use those indices to calculate our result.
-  std::vector<std::shared_ptr<Field>> key_fields;
-  std::vector<std::shared_ptr<Array>> key_columns;
-  std::vector<SortKey> sort_keys;
-  for (std::size_t i = 0; i < key_names.size(); i++) {
-    const std::shared_ptr<Array>& arr = out_arrays[i + aggregates.size()];
-    if (arr->type_id() == Type::DICTIONARY) {
-      // Can't sort dictionary columns so need to decode
-      auto dict_arr = checked_pointer_cast<DictionaryArray>(arr);
-      ARROW_ASSIGN_OR_RAISE(auto decoded_arr,
-                            Take(*dict_arr->dictionary(), *dict_arr->indices()));
-      key_columns.push_back(decoded_arr);
-      key_fields.push_back(
-          field("name_does_not_matter", dict_arr->dict_type()->value_type()));
-    } else {
-      key_columns.push_back(arr);
-      key_fields.push_back(field("name_does_not_matter", arr->type()));
+  if (segmented && segment_key_names.size() > 0) {
+    ArrayVector struct_arrays;
+    struct_arrays.reserve(output_batches.size());
+    for (size_t j = 0; j < output_batches.size(); ++j) {
+      ArrayVector struct_fields;
+      struct_fields.reserve(out_arrays.size());
+      for (auto out_array : out_arrays) {
+        struct_fields.push_back(out_array[j]);
+      }
+      ARROW_ASSIGN_OR_RAISE(auto struct_array,
+                            StructArray::Make(struct_fields, output_schema->fields()));
+      struct_arrays.push_back(struct_array);
     }
-    sort_keys.emplace_back(static_cast<int>(i));
+    return ChunkedArray::Make(struct_arrays);
+  } else {
+    ArrayVector struct_fields(out_arrays.size());
+    for (size_t i = 0; i < out_arrays.size(); ++i) {
+      ARROW_ASSIGN_OR_RAISE(struct_fields[i], Concatenate(out_arrays[i]));
+    }
+    return StructArray::Make(std::move(struct_fields), output_schema->fields());
   }
-  std::shared_ptr<Schema> key_schema = schema(std::move(key_fields));
-  std::shared_ptr<Table> key_table = Table::Make(std::move(key_schema), key_columns);
-  SortOptions sort_options(std::move(sort_keys));
-  ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Array> sort_indices,
-                        SortIndices(key_table, sort_options));
+}
 
-  ARROW_ASSIGN_OR_RAISE(
-      std::shared_ptr<Array> struct_arr,
-      StructArray::Make(std::move(out_arrays), output_schema->fields()));
+Result<Datum> RunGroupBy(const BatchesWithSchema& input,
+                         const std::vector<std::string>& key_names,
+                         const std::vector<std::string>& segment_key_names,
+                         const std::vector<Aggregate>& aggregates, bool use_threads,
+                         bool segmented = false, bool naive = false) {
+  if (segment_key_names.size() > 0) {
+    ARROW_ASSIGN_OR_RAISE(auto thread_pool, arrow::internal::ThreadPool::Make(1));
+    ExecContext seq_ctx(default_memory_pool(), thread_pool.get());
+    return RunGroupBy(input, key_names, segment_key_names, aggregates, &seq_ctx,

Review Comment:
   Why do we pass both single threaded thread pool and "use_threads" flag here? Doesn't "use_threads=False" means to use a single threaded pool?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] icexelloss commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "icexelloss (via GitHub)" <gi...@apache.org>.
icexelloss commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1128194139


##########
cpp/src/arrow/compute/kernels/hash_aggregate_test.cc:
##########
@@ -1917,57 +2243,7 @@ TEST(GroupBy, MinMaxBinary) {
   }
 }
 
-TEST(GroupBy, MinMaxFixedSizeBinary) {

Review Comment:
   Looks like https://github.com/apache/arrow/blob/main/cpp/src/arrow/compute/kernels/hash_aggregate_test.cc#L1869 exists in main branch but you removed the test in this PR - can you restore it?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rtpsw commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "rtpsw (via GitHub)" <gi...@apache.org>.
rtpsw commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1128194798


##########
cpp/src/arrow/compute/kernels/hash_aggregate_test.cc:
##########
@@ -135,22 +142,96 @@ Result<Datum> NaiveGroupBy(std::vector<Datum> arguments, std::vector<Datum> keys
   return Take(struct_arr, sorted_indices);
 }
 
+Result<Datum> MakeGroupByOutput(const std::vector<ExecBatch>& output_batches,
+                                const std::shared_ptr<Schema> output_schema,
+                                size_t num_aggregates, size_t num_keys, bool naive) {
+  ArrayVector out_arrays(num_aggregates + num_keys);
+  for (size_t i = 0; i < out_arrays.size(); ++i) {
+    std::vector<std::shared_ptr<Array>> arrays(output_batches.size());
+    for (size_t j = 0; j < output_batches.size(); ++j) {
+      arrays[j] = output_batches[j].values[i].make_array();
+    }
+    if (arrays.empty()) {
+      ARROW_ASSIGN_OR_RAISE(
+          out_arrays[i],
+          MakeArrayOfNull(output_schema->field(static_cast<int>(i))->type(),
+                          /*length=*/0));
+    } else {
+      ARROW_ASSIGN_OR_RAISE(out_arrays[i], Concatenate(arrays));
+    }
+  }
+
+  ARROW_ASSIGN_OR_RAISE(
+      std::shared_ptr<Array> struct_arr,
+      StructArray::Make(std::move(out_arrays), output_schema->fields()));
+
+  bool need_sort = !naive;
+  for (size_t i = num_aggregates; need_sort && i < out_arrays.size(); i++) {
+    if (output_schema->field(static_cast<int>(i))->type()->id() == Type::DICTIONARY) {
+      need_sort = false;
+    }
+  }
+  if (!need_sort) {
+    return struct_arr;
+  }
+
+  // The exec plan may reorder the output rows.  The tests are all setup to expect ouptut
+  // in ascending order of keys.  So we need to sort the result by the key columns.  To do
+  // that we create a table using the key columns, calculate the sort indices from that
+  // table (sorting on all fields) and then use those indices to calculate our result.
+  std::vector<std::shared_ptr<Field>> key_fields;
+  std::vector<std::shared_ptr<Array>> key_columns;
+  std::vector<SortKey> sort_keys;
+  for (std::size_t i = 0; i < num_keys; i++) {
+    const std::shared_ptr<Array>& arr = out_arrays[i + num_aggregates];
+    key_columns.push_back(arr);
+    key_fields.push_back(field("name_does_not_matter", arr->type()));
+    sort_keys.emplace_back(static_cast<int>(i));
+  }
+  std::shared_ptr<Schema> key_schema = schema(std::move(key_fields));
+  std::shared_ptr<Table> key_table = Table::Make(std::move(key_schema), key_columns);
+  SortOptions sort_options(std::move(sort_keys));
+  ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Array> sort_indices,
+                        SortIndices(key_table, sort_options));
+
+  return Take(struct_arr, sort_indices);
+}
+
 Result<Datum> RunGroupBy(const BatchesWithSchema& input,
                          const std::vector<std::string>& key_names,
-                         const std::vector<Aggregate>& aggregates, bool use_threads) {
+                         const std::vector<std::string>& segment_key_names,
+                         const std::vector<Aggregate>& aggregates, ExecContext* ctx,
+                         bool use_threads, bool segmented = false, bool naive = false) {
+  // When segment_keys is non-empty the `segmented` flag is always true; otherwise (when
+  // empty), it may still be set to true. In this case, the tester restructures (without
+  // changing the data of) the result of RunGroupBy from `std::vector<ExecBatch>`
+  // (output_batches) to `std::vector<ArrayVector>` (out_arrays), which have the structure
+  // typical of the case of a non-empty segment_keys (with multiple arrays per column, one
+  // array per segment) but only one array per column (because, technically, there is only
+  // one segment in this case). Thus, this case focuses on the structure of the result.
+  //
+  // The `naive` flag means that the output is expected to be like that of `NaiveGroupBy`,
+  // which in particular doesn't require sorting. The reason for the naive flag is that

Review Comment:
   Yes, the output is unsorted.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] icexelloss commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "icexelloss (via GitHub)" <gi...@apache.org>.
icexelloss commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1129549157


##########
cpp/src/arrow/compute/row/grouper.cc:
##########
@@ -39,12 +43,336 @@
 namespace arrow {
 
 using internal::checked_cast;
+using internal::PrimitiveScalarBase;
 
 namespace compute {
 
 namespace {
 
-struct GrouperImpl : Grouper {
+constexpr uint32_t kNoGroupId = std::numeric_limits<uint32_t>::max();
+
+using group_id_t = std::remove_const<decltype(kNoGroupId)>::type;
+using GroupIdType = CTypeTraits<group_id_t>::ArrowType;
+auto g_group_id_type = std::make_shared<GroupIdType>();
+
+inline const uint8_t* GetValuesAsBytes(const ArraySpan& data, int64_t offset = 0) {
+  DCHECK_GT(data.type->byte_width(), 0);
+  int64_t absolute_byte_offset = (data.offset + offset) * data.type->byte_width();
+  return data.GetValues<uint8_t>(1, absolute_byte_offset);
+}
+
+template <typename Value>
+Status CheckForGetNextSegment(const std::vector<Value>& values, int64_t length,
+                              int64_t offset, const std::vector<TypeHolder>& key_types) {
+  if (offset < 0 || offset > length) {

Review Comment:
   Which error-check is this breaking? If this is a DCHECK, then it is invariant that shouldn't happen and I don't think we need to have specific test case for it.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] icexelloss commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "icexelloss (via GitHub)" <gi...@apache.org>.
icexelloss commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1129570181


##########
cpp/src/arrow/compute/row/grouper.cc:
##########
@@ -39,12 +44,329 @@
 namespace arrow {
 
 using internal::checked_cast;
+using internal::PrimitiveScalarBase;
 
 namespace compute {
 
 namespace {
 
-struct GrouperImpl : Grouper {
+constexpr uint32_t kNoGroupId = std::numeric_limits<uint32_t>::max();
+
+using group_id_t = std::remove_const<decltype(kNoGroupId)>::type;
+using GroupIdType = CTypeTraits<group_id_t>::ArrowType;
+auto g_group_id_type = std::make_shared<GroupIdType>();
+
+inline const uint8_t* GetValuesAsBytes(const ArraySpan& data, int64_t offset = 0) {
+  DCHECK_GT(data.type->byte_width(), 0);
+  int64_t absolute_byte_offset = (data.offset + offset) * data.type->byte_width();
+  return data.GetValues<uint8_t>(1, absolute_byte_offset);
+}
+
+template <typename Value>
+Status CheckForGetNextSegment(const std::vector<Value>& values, int64_t length,
+                              int64_t offset, const std::vector<TypeHolder>& key_types) {
+  if (offset < 0 || offset > length) {
+    return Status::Invalid("invalid grouping segmenter offset: ", offset);
+  }
+  if (values.size() != key_types.size()) {
+    return Status::Invalid("expected batch size ", key_types.size(), " but got ",
+                           values.size());
+  }
+  for (size_t i = 0; i < key_types.size(); i++) {
+    const auto& value = values[i];
+    const auto& key_type = key_types[i];
+    if (*value.type() != *key_type.type) {
+      return Status::Invalid("expected batch value ", i, " of type ", *key_type.type,
+                             " but got ", *value.type());
+    }
+  }
+  return Status::OK();
+}
+
+template <typename Batch>
+enable_if_t<std::is_same<Batch, ExecSpan>::value || std::is_same<Batch, ExecBatch>::value,
+            Status>
+CheckForGetNextSegment(const Batch& batch, int64_t offset,
+                       const std::vector<TypeHolder>& key_types) {
+  return CheckForGetNextSegment(batch.values, batch.length, offset, key_types);
+}
+
+struct BaseRowSegmenter : public RowSegmenter {
+  explicit BaseRowSegmenter(const std::vector<TypeHolder>& key_types)
+      : key_types_(key_types) {}
+
+  const std::vector<TypeHolder>& key_types() const override { return key_types_; }
+
+  std::vector<TypeHolder> key_types_;
+};
+
+Segment MakeSegment(int64_t batch_length, int64_t offset, int64_t length, bool extends) {
+  return Segment{offset, length, offset + length >= batch_length, extends};
+}
+
+// Used by SimpleKeySegmenter::GetNextSegment to find the match-length of a value within a
+// fixed-width buffer
+int64_t GetMatchLength(const uint8_t* match_bytes, int64_t match_width,
+                       const uint8_t* array_bytes, int64_t offset, int64_t length) {
+  int64_t cursor, byte_cursor;
+  for (cursor = offset, byte_cursor = match_width * cursor; cursor < length;
+       cursor++, byte_cursor += match_width) {
+    if (memcmp(match_bytes, array_bytes + byte_cursor,
+               static_cast<size_t>(match_width)) != 0) {
+      break;
+    }
+  }
+  return std::min(cursor, length) - offset;
+}
+
+using ExtendFunc = std::function<bool(const void*)>;
+constexpr bool kDefaultExtends = true;  // by default, the first segment extends
+constexpr bool kEmptyExtends = true;    // an empty segment extends too
+
+struct NoKeysSegmenter : public BaseRowSegmenter {
+  static std::unique_ptr<RowSegmenter> Make() {
+    return std::make_unique<NoKeysSegmenter>();
+  }
+
+  NoKeysSegmenter() : BaseRowSegmenter({}) {}
+
+  Status Reset() override { return Status::OK(); }
+
+  Result<Segment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {}));
+    return MakeSegment(batch.length, offset, batch.length - offset, kDefaultExtends);
+  }
+};
+
+struct SimpleKeySegmenter : public BaseRowSegmenter {
+  static Result<std::unique_ptr<RowSegmenter>> Make(TypeHolder key_type) {
+    return std::make_unique<SimpleKeySegmenter>(key_type);
+  }
+
+  explicit SimpleKeySegmenter(TypeHolder key_type)
+      : BaseRowSegmenter({key_type}),
+        key_type_(key_types_[0]),
+        save_key_data_(static_cast<size_t>(key_type_.type->byte_width())),
+        extend_was_called_(false) {}
+
+  Status CheckType(const DataType& type) {
+    if (!is_fixed_width(type)) {
+      return Status::Invalid("SimpleKeySegmenter does not support type ", type);
+    }
+    return Status::OK();
+  }
+
+  Status Reset() override {
+    extend_was_called_ = false;
+    return Status::OK();
+  }
+
+  // Checks whether the given grouping data extends the current segment, i.e., is equal to
+  // previously seen grouping data, which is updated with each invocation.
+  bool Extend(const void* data) {
+    bool extends = !extend_was_called_
+                       ? kDefaultExtends
+                       : 0 == memcmp(save_key_data_.data(), data, save_key_data_.size());
+    extend_was_called_ = true;
+    memcpy(save_key_data_.data(), data, save_key_data_.size());
+    return extends;
+  }
+
+  Result<Segment> GetNextSegment(const Scalar& scalar, int64_t offset, int64_t length) {
+    ARROW_RETURN_NOT_OK(CheckType(*scalar.type));
+    if (!scalar.is_valid) {
+      return Status::Invalid("segmenting an invalid scalar");
+    }
+    auto data = checked_cast<const PrimitiveScalarBase&>(scalar).data();
+    bool extends = length > 0 ? Extend(data) : kEmptyExtends;
+    return MakeSegment(length, offset, length, extends);
+  }
+
+  Result<Segment> GetNextSegment(const DataType& array_type, const uint8_t* array_bytes,
+                                 int64_t offset, int64_t length) {
+    RETURN_NOT_OK(CheckType(array_type));
+    DCHECK_LE(offset, length);
+    int64_t byte_width = array_type.byte_width();
+    int64_t match_length = GetMatchLength(array_bytes + offset * byte_width, byte_width,
+                                          array_bytes, offset, length);
+    bool extends = length > 0 ? Extend(array_bytes + offset * byte_width) : kEmptyExtends;
+    return MakeSegment(length, offset, match_length, extends);
+  }
+
+  Result<Segment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {key_type_}));
+    if (offset == batch.length) {
+      return MakeSegment(batch.length, offset, 0, kEmptyExtends);
+    }
+    const auto& value = batch.values[0];
+    if (value.is_scalar()) {
+      return GetNextSegment(*value.scalar, offset, batch.length);
+    }
+    ARROW_DCHECK(value.is_array());
+    const auto& array = value.array;
+    if (array.GetNullCount() > 0) {
+      return Status::NotImplemented("segmenting a nullable array");
+    }
+    return GetNextSegment(*array.type, GetValuesAsBytes(array), offset, batch.length);
+  }
+
+ private:
+  TypeHolder key_type_;
+  std::vector<uint8_t> save_key_data_;  // previusly seen segment-key grouping data
+  bool extend_was_called_;
+};
+
+struct AnyKeysSegmenter : public BaseRowSegmenter {
+  static Result<std::unique_ptr<RowSegmenter>> Make(
+      const std::vector<TypeHolder>& key_types, ExecContext* ctx) {
+    ARROW_RETURN_NOT_OK(Grouper::Make(key_types, ctx));  // check types
+    return std::make_unique<AnyKeysSegmenter>(key_types, ctx);
+  }
+
+  AnyKeysSegmenter(const std::vector<TypeHolder>& key_types, ExecContext* ctx)
+      : BaseRowSegmenter(key_types),
+        ctx_(ctx),
+        grouper_(nullptr),
+        save_group_id_(kNoGroupId) {}
+
+  Status Reset() override {
+    grouper_ = nullptr;
+    save_group_id_ = kNoGroupId;
+    return Status::OK();
+  }
+
+  bool Extend(const void* data) {
+    auto group_id = *static_cast<const group_id_t*>(data);
+    bool extends =
+        save_group_id_ == kNoGroupId ? kDefaultExtends : save_group_id_ == group_id;
+    save_group_id_ = group_id;
+    return extends;
+  }
+
+  // Runs the grouper on a single row.  This is used to determine the group id of the
+  // first row of a new segment to see if it extends the previous segment.
+  template <typename Batch>
+  Result<group_id_t> MapGroupIdAt(const Batch& batch, int64_t offset) {
+    if (!grouper_) return kNoGroupId;
+    ARROW_ASSIGN_OR_RAISE(auto datum, grouper_->Consume(batch, offset,
+                                                        /*length=*/1));
+    if (!datum.is_array()) {
+      return Status::Invalid("accessing unsupported datum kind ", datum.kind());
+    }
+    const std::shared_ptr<ArrayData>& data = datum.array();
+    ARROW_DCHECK(data->GetNullCount() == 0);
+    DCHECK_EQ(data->type->id(), GroupIdType::type_id);
+    DCHECK_EQ(1, data->length);
+    const group_id_t* values = data->GetValues<group_id_t>(1);
+    return values[0];
+  }
+
+  Result<Segment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, key_types_));
+    if (offset == batch.length) {
+      return MakeSegment(batch.length, offset, 0, kEmptyExtends);
+    }
+    // ARROW-18311: make Grouper support Reset()
+    // so it can be cached instead of recreated below
+    //
+    // the group id must be computed prior to resetting the grouper, since it is compared
+    // to save_group_id_, and after resetting the grouper produces incomparable group ids
+    ARROW_ASSIGN_OR_RAISE(auto group_id, MapGroupIdAt(batch, offset));
+    ExtendFunc bound_extend = [this, group_id](const void* data) {
+      bool extends = Extend(&group_id);
+      save_group_id_ = *static_cast<const group_id_t*>(data);
+      return extends;
+    };
+    ARROW_ASSIGN_OR_RAISE(grouper_, Grouper::Make(key_types_, ctx_));  // TODO: reset it
+    ARROW_ASSIGN_OR_RAISE(auto datum, grouper_->Consume(batch, offset));

Review Comment:
   Let's add a follow up here to make this more efficient - currently I believe we are mapping the group id of the entire batch (from offset) every time, making this O(n2) (n is the number of rows in the batch)



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] icexelloss commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "icexelloss (via GitHub)" <gi...@apache.org>.
icexelloss commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1129575336


##########
cpp/src/arrow/compute/row/grouper.cc:
##########
@@ -39,12 +43,521 @@
 namespace arrow {
 
 using internal::checked_cast;
+using internal::PrimitiveScalarBase;
 
 namespace compute {
 
 namespace {
 
-struct GrouperImpl : Grouper {
+constexpr uint32_t kNoGroupId = std::numeric_limits<uint32_t>::max();
+
+using group_id_t = std::remove_const<decltype(kNoGroupId)>::type;
+using GroupIdType = CTypeTraits<group_id_t>::ArrowType;
+auto group_id_type = std::make_shared<GroupIdType>();
+
+inline const uint8_t* GetValuesAsBytes(const ArrayData& data, int64_t offset = 0) {
+  DCHECK_GT(data.type->byte_width(), 0);
+  int64_t absolute_byte_offset = (data.offset + offset) * data.type->byte_width();
+  return data.GetValues<uint8_t>(1, absolute_byte_offset);
+}
+
+inline const uint8_t* GetValuesAsBytes(const ArraySpan& data, int64_t offset = 0) {
+  DCHECK_GT(data.type->byte_width(), 0);
+  int64_t absolute_byte_offset = (data.offset + offset) * data.type->byte_width();
+  return data.GetValues<uint8_t>(1, absolute_byte_offset);
+}
+
+template <typename Value>
+Status CheckForGetNextSegment(const std::vector<Value>& values, int64_t length,
+                              int64_t offset, const std::vector<TypeHolder>& key_types) {
+  if (offset < 0 || offset > length) {
+    return Status::Invalid("invalid grouping segmenter offset: ", offset);
+  }
+  if (values.size() != key_types.size()) {
+    return Status::Invalid("expected batch size ", key_types.size(), " but got ",
+                           values.size());
+  }
+  for (size_t i = 0; i < key_types.size(); i++) {
+    const auto& value = values[i];
+    const auto& key_type = key_types[i];
+    if (*value.type() != *key_type.type) {
+      return Status::Invalid("expected batch value ", i, " of type ", *key_type.type,
+                             " but got ", *value.type());
+    }
+  }
+  return Status::OK();
+}
+
+template <typename Batch>
+enable_if_t<std::is_same<Batch, ExecSpan>::value || std::is_same<Batch, ExecBatch>::value,
+            Status>
+CheckForGetNextSegment(const Batch& batch, int64_t offset,
+                       const std::vector<TypeHolder>& key_types) {
+  return CheckForGetNextSegment(batch.values, batch.length, offset, key_types);
+}
+
+struct BaseGroupingSegmenter : public GroupingSegmenter {
+  explicit BaseGroupingSegmenter(const std::vector<TypeHolder>& key_types)
+      : key_types_(key_types) {}
+
+  const std::vector<TypeHolder>& key_types() const override { return key_types_; }
+
+  std::vector<TypeHolder> key_types_;
+};
+
+GroupingSegment MakeSegment(int64_t batch_length, int64_t offset, int64_t length,
+                            bool extends) {
+  return GroupingSegment{offset, length, offset + length >= batch_length, extends};
+}
+
+int64_t GetMatchLength(const uint8_t* match_bytes, int64_t match_width,

Review Comment:
   Agree



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] icexelloss commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "icexelloss (via GitHub)" <gi...@apache.org>.
icexelloss commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1132444876


##########
cpp/src/arrow/compute/row/grouper.cc:
##########
@@ -39,12 +43,336 @@
 namespace arrow {
 
 using internal::checked_cast;
+using internal::PrimitiveScalarBase;
 
 namespace compute {
 
 namespace {
 
-struct GrouperImpl : Grouper {
+constexpr uint32_t kNoGroupId = std::numeric_limits<uint32_t>::max();
+
+using group_id_t = std::remove_const<decltype(kNoGroupId)>::type;
+using GroupIdType = CTypeTraits<group_id_t>::ArrowType;
+auto g_group_id_type = std::make_shared<GroupIdType>();
+
+inline const uint8_t* GetValuesAsBytes(const ArraySpan& data, int64_t offset = 0) {
+  DCHECK_GT(data.type->byte_width(), 0);
+  int64_t absolute_byte_offset = (data.offset + offset) * data.type->byte_width();
+  return data.GetValues<uint8_t>(1, absolute_byte_offset);
+}
+
+template <typename Value>
+Status CheckForGetNextSegment(const std::vector<Value>& values, int64_t length,
+                              int64_t offset, const std::vector<TypeHolder>& key_types) {
+  if (offset < 0 || offset > length) {

Review Comment:
   I think this should still be a DCheck here because I see the class here being "internal util that used for segment aggregation" and not a "library for users to use". But not a big issue so I wouldn't hold on to it.
   
   I will create a follow up issue to track this



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rtpsw commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "rtpsw (via GitHub)" <gi...@apache.org>.
rtpsw commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1117567499


##########
cpp/src/arrow/compute/exec/aggregate_node.cc:
##########
@@ -282,12 +396,19 @@ class ScalarAggregateNode : public ExecNode, public TracedNode {
     auto scope = TraceInputReceived(batch);
     DCHECK_EQ(input, inputs_[0]);
 
-    auto thread_index = plan_->query_context()->GetThreadIndex();
-
-    ARROW_RETURN_NOT_OK(DoConsume(ExecSpan(batch), thread_index));
+    auto handler = [this](const ExecBatch& full_batch, const GroupingSegment& segment) {
+      if (!segment.extends && segment.offset == 0) RETURN_NOT_OK(OutputResult());
+      auto exec_batch = full_batch.Slice(segment.offset, segment.length);
+      auto batch = ExecSpan(exec_batch);
+      RETURN_NOT_OK(DoConsume(batch, plan_->query_context()->GetThreadIndex()));

Review Comment:
   Before, `thread_index` was not within a lambda. I'll fix this to be similar to the original code.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rtpsw commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "rtpsw (via GitHub)" <gi...@apache.org>.
rtpsw commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1117884526


##########
cpp/src/arrow/compute/exec/options.h:
##########
@@ -199,21 +199,32 @@ class ARROW_EXPORT ProjectNodeOptions : public ExecNodeOptions {
   std::vector<std::string> names;
 };
 
-/// \brief Make a node which aggregates input batches, optionally grouped by keys.
+/// \brief Make a node which aggregates input batches, optionally grouped by keys and
+/// optionally segmented by segment-keys. Both keys and segment-keys determine the group.
+/// However segment-keys are also used for determining grouping segments, which should be
+/// large, and allow streaming a partial aggregation result after processing each segment.
+/// One common use-case for segment-keys is ordered aggregation, in which the segment-key
+/// attribute specifies a column with non-decreasing values or a lexigographically-ordered
+/// set of such columns.
 ///
 /// If the keys attribute is a non-empty vector, then each aggregate in `aggregates` is
 /// expected to be a HashAggregate function. If the keys attribute is an empty vector,
 /// then each aggregate is assumed to be a ScalarAggregate function.
 class ARROW_EXPORT AggregateNodeOptions : public ExecNodeOptions {
  public:
   explicit AggregateNodeOptions(std::vector<Aggregate> aggregates,
-                                std::vector<FieldRef> keys = {})
-      : aggregates(std::move(aggregates)), keys(std::move(keys)) {}
+                                std::vector<FieldRef> keys = {},
+                                std::vector<FieldRef> segment_keys = {})
+      : aggregates(std::move(aggregates)),
+        keys(std::move(keys)),
+        segment_keys(std::move(segment_keys)) {}
 
   // aggregations which will be applied to the targetted fields
   std::vector<Aggregate> aggregates;
   // keys by which aggregations will be grouped
   std::vector<FieldRef> keys;
+  // keys by which aggregations will be segmented
+  std::vector<FieldRef> segment_keys;

Review Comment:
   I added the check to disallow.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rtpsw commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "rtpsw (via GitHub)" <gi...@apache.org>.
rtpsw commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1117883799


##########
cpp/src/arrow/compute/row/grouper.h:
##########
@@ -30,6 +30,49 @@
 namespace arrow {
 namespace compute {
 
+/// \brief A segment of contiguous rows for grouping
+struct ARROW_EXPORT GroupingSegment {
+  int64_t offset;
+  int64_t length;
+  bool is_open;
+  bool extends;
+};
+
+inline bool operator==(const GroupingSegment& segment1, const GroupingSegment& segment2) {
+  return segment1.offset == segment2.offset && segment1.length == segment2.length &&
+         segment1.is_open == segment2.is_open && segment1.extends == segment2.extends;
+}
+inline bool operator!=(const GroupingSegment& segment1, const GroupingSegment& segment2) {
+  return !(segment1 == segment2);
+}
+
+/// \brief Computes grouping segments for a batch. Each segment covers rows with identical
+/// values in the batch. The values in the batch are often selected as keys from a larger
+/// batch.
+class ARROW_EXPORT GroupingSegmenter {
+ public:
+  virtual ~GroupingSegmenter() = default;
+
+  /// \brief Construct a GroupingSegmenter which receives the specified key types
+  static Result<std::unique_ptr<GroupingSegmenter>> Make(
+      const std::vector<TypeHolder>& key_types, bool nullable_keys = false,
+      ExecContext* ctx = default_exec_context());
+
+  /// \brief Return the key types of this segmenter
+  virtual const std::vector<TypeHolder>& key_types() const = 0;

Review Comment:
   A `TypeHolder` can hold a `shared_ptr<DataType>` or a `DataType*`, i.e., it can own or not own the instance pointed to.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rtpsw commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "rtpsw (via GitHub)" <gi...@apache.org>.
rtpsw commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1117883799


##########
cpp/src/arrow/compute/row/grouper.h:
##########
@@ -30,6 +30,49 @@
 namespace arrow {
 namespace compute {
 
+/// \brief A segment of contiguous rows for grouping
+struct ARROW_EXPORT GroupingSegment {
+  int64_t offset;
+  int64_t length;
+  bool is_open;
+  bool extends;
+};
+
+inline bool operator==(const GroupingSegment& segment1, const GroupingSegment& segment2) {
+  return segment1.offset == segment2.offset && segment1.length == segment2.length &&
+         segment1.is_open == segment2.is_open && segment1.extends == segment2.extends;
+}
+inline bool operator!=(const GroupingSegment& segment1, const GroupingSegment& segment2) {
+  return !(segment1 == segment2);
+}
+
+/// \brief Computes grouping segments for a batch. Each segment covers rows with identical
+/// values in the batch. The values in the batch are often selected as keys from a larger
+/// batch.
+class ARROW_EXPORT GroupingSegmenter {
+ public:
+  virtual ~GroupingSegmenter() = default;
+
+  /// \brief Construct a GroupingSegmenter which receives the specified key types
+  static Result<std::unique_ptr<GroupingSegmenter>> Make(
+      const std::vector<TypeHolder>& key_types, bool nullable_keys = false,
+      ExecContext* ctx = default_exec_context());
+
+  /// \brief Return the key types of this segmenter
+  virtual const std::vector<TypeHolder>& key_types() const = 0;

Review Comment:
   A `TypeHolder` can hold a `shared_ptr<DataType>` or a `DataType*`, so it can own or not own the instance pointed to.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rtpsw commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "rtpsw (via GitHub)" <gi...@apache.org>.
rtpsw commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1117884396


##########
cpp/src/arrow/compute/kernels/hash_aggregate_test.cc:
##########
@@ -135,22 +141,84 @@ Result<Datum> NaiveGroupBy(std::vector<Datum> arguments, std::vector<Datum> keys
   return Take(struct_arr, sorted_indices);
 }
 
+Result<Datum> MakeGroupByOutput(const std::vector<ExecBatch>& output_batches,
+                                const std::shared_ptr<Schema> output_schema,
+                                size_t num_aggregates, size_t num_keys, bool naive) {
+  ArrayVector out_arrays(num_aggregates + num_keys);
+  for (size_t i = 0; i < out_arrays.size(); ++i) {
+    std::vector<std::shared_ptr<Array>> arrays(output_batches.size());
+    for (size_t j = 0; j < output_batches.size(); ++j) {
+      arrays[j] = output_batches[j].values[i].make_array();
+    }
+    if (arrays.empty()) {
+      ARROW_ASSIGN_OR_RAISE(
+          out_arrays[i],
+          MakeArrayOfNull(output_schema->field(static_cast<int>(i))->type(),
+                          /*length=*/0));
+    } else {
+      ARROW_ASSIGN_OR_RAISE(out_arrays[i], Concatenate(arrays));
+    }
+  }
+
+  ARROW_ASSIGN_OR_RAISE(
+      std::shared_ptr<Array> struct_arr,
+      StructArray::Make(std::move(out_arrays), output_schema->fields()));
+
+  bool need_sort = !naive;
+  for (size_t i = num_aggregates; need_sort && i < out_arrays.size(); i++) {
+    if (output_schema->field(i)->type()->id() == Type::DICTIONARY) {

Review Comment:
   Because the API is `Schema::field(int i)`. Around here, `i` is used in both `int` (here) and `size_t` (at the for-loop) contexts, so a cast is needed.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rtpsw commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "rtpsw (via GitHub)" <gi...@apache.org>.
rtpsw commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1117883593


##########
cpp/src/arrow/compute/exec/aggregate_node.cc:
##########
@@ -43,7 +44,45 @@ namespace compute {
 
 namespace {
 
-namespace {
+/// \brief A gated shared mutex is similar to a shared mutex, in that it allows either
+/// multiple shared readers or a unique writer access to the mutex, except that a waiting
+/// writer gates future readers by preventing them from reacquiring shared access until it
+/// has acquired and released the mutex. This is useful for ensuring a writer is never
+/// starved by readers.
+struct GatedSharedMutex {
+  std::mutex gate;
+  std::shared_mutex mutex;
+};
+
+/// \brief Acquires unique access to a gatex mutex. This is useful for a unique writer.
+class GatedUniqueLock {
+ public:
+  // acquires the gate first, to ensure future readers will wait for its release
+  explicit GatedUniqueLock(GatedSharedMutex& gated_shared_mutex)
+      : lock_gate_(gated_shared_mutex.gate), lock_mutex_(gated_shared_mutex.mutex) {}
+
+ private:
+  std::unique_lock<std::mutex> lock_gate_;
+  std::unique_lock<std::shared_mutex> lock_mutex_;
+};
+
+/// \brief Acquires shared access to a gatex mutex. This is useful for a shared reader.
+class GatedSharedLock {

Review Comment:
   Done.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] icexelloss commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "icexelloss (via GitHub)" <gi...@apache.org>.
icexelloss commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1116044576


##########
cpp/src/arrow/compute/exec.h:
##########
@@ -240,6 +246,8 @@ struct ARROW_EXPORT ExecBatch {
 
   ExecBatch Slice(int64_t offset, int64_t length) const;
 
+  Result<ExecBatch> SelectValues(const std::vector<int>& ids) const;

Review Comment:
   What is this for?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rtpsw commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "rtpsw (via GitHub)" <gi...@apache.org>.
rtpsw commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1117528943


##########
cpp/src/arrow/compute/row/grouper.cc:
##########
@@ -39,12 +43,521 @@
 namespace arrow {
 
 using internal::checked_cast;
+using internal::PrimitiveScalarBase;
 
 namespace compute {
 
 namespace {
 
-struct GrouperImpl : Grouper {
+constexpr uint32_t kNoGroupId = std::numeric_limits<uint32_t>::max();
+
+using group_id_t = std::remove_const<decltype(kNoGroupId)>::type;
+using GroupIdType = CTypeTraits<group_id_t>::ArrowType;
+auto group_id_type = std::make_shared<GroupIdType>();
+
+inline const uint8_t* GetValuesAsBytes(const ArrayData& data, int64_t offset = 0) {
+  DCHECK_GT(data.type->byte_width(), 0);
+  int64_t absolute_byte_offset = (data.offset + offset) * data.type->byte_width();
+  return data.GetValues<uint8_t>(1, absolute_byte_offset);
+}
+
+inline const uint8_t* GetValuesAsBytes(const ArraySpan& data, int64_t offset = 0) {
+  DCHECK_GT(data.type->byte_width(), 0);
+  int64_t absolute_byte_offset = (data.offset + offset) * data.type->byte_width();
+  return data.GetValues<uint8_t>(1, absolute_byte_offset);
+}
+
+template <typename Value>
+Status CheckForGetNextSegment(const std::vector<Value>& values, int64_t length,
+                              int64_t offset, const std::vector<TypeHolder>& key_types) {
+  if (offset < 0 || offset > length) {
+    return Status::Invalid("invalid grouping segmenter offset: ", offset);
+  }
+  if (values.size() != key_types.size()) {
+    return Status::Invalid("expected batch size ", key_types.size(), " but got ",
+                           values.size());
+  }
+  for (size_t i = 0; i < key_types.size(); i++) {
+    const auto& value = values[i];
+    const auto& key_type = key_types[i];
+    if (*value.type() != *key_type.type) {
+      return Status::Invalid("expected batch value ", i, " of type ", *key_type.type,
+                             " but got ", *value.type());
+    }
+  }
+  return Status::OK();
+}
+
+template <typename Batch>
+enable_if_t<std::is_same<Batch, ExecSpan>::value || std::is_same<Batch, ExecBatch>::value,
+            Status>
+CheckForGetNextSegment(const Batch& batch, int64_t offset,
+                       const std::vector<TypeHolder>& key_types) {
+  return CheckForGetNextSegment(batch.values, batch.length, offset, key_types);
+}
+
+struct BaseGroupingSegmenter : public GroupingSegmenter {
+  explicit BaseGroupingSegmenter(const std::vector<TypeHolder>& key_types)
+      : key_types_(key_types) {}
+
+  const std::vector<TypeHolder>& key_types() const override { return key_types_; }
+
+  std::vector<TypeHolder> key_types_;
+};
+
+GroupingSegment MakeSegment(int64_t batch_length, int64_t offset, int64_t length,
+                            bool extends) {
+  return GroupingSegment{offset, length, offset + length >= batch_length, extends};
+}
+
+int64_t GetMatchLength(const uint8_t* match_bytes, int64_t match_width,
+                       const uint8_t* array_bytes, int64_t offset, int64_t length) {
+  int64_t cursor, byte_cursor;
+  for (cursor = offset, byte_cursor = match_width * cursor; cursor < length;
+       cursor++, byte_cursor += match_width) {
+    if (memcmp(match_bytes, array_bytes + byte_cursor,
+               static_cast<size_t>(match_width)) != 0) {
+      break;
+    }
+  }
+  return std::min(cursor, length - offset);
+}
+
+using ExtendFunc = std::function<bool(const void*)>;
+constexpr bool kDefaultExtends = true;
+constexpr bool kEmptyExtends = true;
+
+Result<GroupingSegment> GetNextSegmentChunked(
+    const std::shared_ptr<ChunkedArray>& chunked_array, int64_t offset,
+    ExtendFunc extend) {
+  if (offset >= chunked_array->length()) {
+    return MakeSegment(chunked_array->length(), chunked_array->length(), 0,
+                       kEmptyExtends);
+  }
+  int64_t remaining_offset = offset;
+  const auto& arrays = chunked_array->chunks();
+  for (size_t i = 0; remaining_offset >= 0 && i < arrays.size(); i++) {
+    // look up chunk containing offset
+    int64_t array_length = arrays[i]->length();
+    if (remaining_offset < array_length) {
+      // found - switch to matching
+      int64_t match_width = arrays[i]->type()->byte_width();
+      const uint8_t* match_bytes = GetValuesAsBytes(*arrays[i]->data(), remaining_offset);
+      int64_t total_match_length = 0;
+      for (; i < arrays.size(); i++) {
+        int64_t array_length = arrays[i]->length();
+        if (array_length <= 0) continue;
+        const uint8_t* array_bytes = GetValuesAsBytes(*arrays[i]->data());
+        int64_t match_length = GetMatchLength(match_bytes, match_width, array_bytes,
+                                              remaining_offset, array_length);
+        total_match_length += match_length;
+        remaining_offset = 0;
+        if (match_length < array_length - remaining_offset) break;
+      }
+      bool extends = extend(match_bytes);
+      return MakeSegment(chunked_array->length(), offset, total_match_length, extends);
+    }
+    remaining_offset -= array_length;
+  }
+  return Status::Invalid("segmenting invalid chunked array value");
+}
+
+struct NoKeysGroupingSegmenter : public BaseGroupingSegmenter {
+  static std::unique_ptr<GroupingSegmenter> Make() {
+    return std::make_unique<NoKeysGroupingSegmenter>();
+  }
+
+  NoKeysGroupingSegmenter() : BaseGroupingSegmenter({}) {}
+
+  Status Reset() override { return Status::OK(); }
+
+  template <typename Batch>
+  Result<GroupingSegment> GetNextSegmentImpl(const Batch& batch, int64_t offset) {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {}));
+    return MakeSegment(batch.length, offset, batch.length - offset, kDefaultExtends);
+  }
+
+  Result<GroupingSegment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    return GetNextSegmentImpl(batch, offset);
+  }
+
+  Result<GroupingSegment> GetNextSegment(const ExecBatch& batch,
+                                         int64_t offset) override {
+    return GetNextSegmentImpl(batch, offset);
+  }
+};
+
+struct SimpleKeyGroupingSegmenter : public BaseGroupingSegmenter {
+  static Result<std::unique_ptr<GroupingSegmenter>> Make(TypeHolder key_type) {
+    return std::make_unique<SimpleKeyGroupingSegmenter>(key_type);
+  }
+
+  explicit SimpleKeyGroupingSegmenter(TypeHolder key_type)
+      : BaseGroupingSegmenter({key_type}), key_type_(key_types_[0]), save_key_data_() {}
+
+  Status CheckType(const DataType& type) {
+    if (!is_fixed_width(type)) {
+      return Status::Invalid("SimpleKeyGroupingSegmenter does not support type ", type);
+    }
+    return Status::OK();
+  }
+
+  Status Reset() override {
+    save_key_data_.resize(0);
+    return Status::OK();
+  }
+
+  bool Extend(const void* data) {
+    size_t byte_width = static_cast<size_t>(key_type_.type->byte_width());
+    bool extends = save_key_data_.size() != byte_width
+                       ? kDefaultExtends
+                       : 0 == memcmp(save_key_data_.data(), data, byte_width);
+    save_key_data_.resize(byte_width);
+    memcpy(save_key_data_.data(), data, byte_width);
+    return extends;
+  }
+
+  Result<GroupingSegment> GetNextSegment(const Scalar& scalar, int64_t offset,
+                                         int64_t length) {
+    ARROW_RETURN_NOT_OK(CheckType(*scalar.type));
+    if (!scalar.is_valid) {
+      return Status::Invalid("segmenting an invalid scalar");
+    }
+    auto data = checked_cast<const PrimitiveScalarBase&>(scalar).data();
+    bool extends = length > 0 ? Extend(data) : kEmptyExtends;
+    return MakeSegment(length, 0, length, extends);
+  }
+
+  Result<GroupingSegment> GetNextSegment(const DataType& array_type,
+                                         const uint8_t* array_bytes, int64_t offset,
+                                         int64_t length) {
+    RETURN_NOT_OK(CheckType(array_type));
+    int64_t byte_width = array_type.byte_width();
+    int64_t match_length = GetMatchLength(array_bytes + offset * byte_width, byte_width,
+                                          array_bytes, offset, length);
+    bool extends = length > 0 ? Extend(array_bytes + offset * byte_width) : kEmptyExtends;
+    return MakeSegment(length, offset, match_length, extends);
+  }
+
+  Result<GroupingSegment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {key_type_}));
+    if (offset == batch.length) {
+      return MakeSegment(batch.length, offset, 0, kEmptyExtends);
+    }
+    const auto& value = batch.values[0];
+    if (value.is_scalar()) {
+      return GetNextSegment(*value.scalar, offset, batch.length);
+    }
+    ARROW_DCHECK(value.is_array());
+    const auto& array = value.array;
+    if (array.GetNullCount() > 0) {
+      return Status::NotImplemented("segmenting a nullable array");
+    }
+    return GetNextSegment(*array.type, GetValuesAsBytes(array), offset, batch.length);
+  }
+
+  Result<GroupingSegment> GetNextSegment(const ExecBatch& batch,
+                                         int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {key_type_}));
+    if (offset == batch.length) {
+      return MakeSegment(batch.length, offset, 0, kEmptyExtends);
+    }
+    const auto& value = batch.values[0];
+    if (value.is_scalar()) {
+      return GetNextSegment(*value.scalar(), offset, batch.length);
+    }
+    if (value.is_array()) {
+      auto array = value.array();
+      if (array->GetNullCount() > 0) {
+        return Status::NotImplemented("segmenting a nullable array");
+      }
+      return GetNextSegment(*array->type, GetValuesAsBytes(*array), offset, batch.length);
+    }
+    if (value.is_chunked_array()) {
+      auto array = value.chunked_array();
+      if (array->null_count() > 0) {
+        return Status::NotImplemented("segmenting a nullable array");
+      }
+      return GetNextSegmentChunked(array, offset, bound_extend_);
+    }
+    return Status::Invalid("segmenting unsupported value kind ", value.kind());
+  }
+
+ private:
+  TypeHolder key_type_;
+  std::vector<uint8_t> save_key_data_;
+  ExtendFunc bound_extend_ = [this](const void* data) { return Extend(data); };
+};
+
+struct AnyKeysGroupingSegmenter : public BaseGroupingSegmenter {
+  static Result<std::unique_ptr<GroupingSegmenter>> Make(
+      const std::vector<TypeHolder>& key_types, ExecContext* ctx) {
+    ARROW_RETURN_NOT_OK(Grouper::Make(key_types, ctx));  // check types
+    return std::make_unique<AnyKeysGroupingSegmenter>(key_types, ctx);
+  }
+
+  AnyKeysGroupingSegmenter(const std::vector<TypeHolder>& key_types, ExecContext* ctx)
+      : BaseGroupingSegmenter(key_types),
+        ctx_(ctx),
+        grouper_(nullptr),
+        save_group_id_(kNoGroupId) {}
+
+  Status Reset() override {
+    grouper_ = nullptr;
+    save_group_id_ = kNoGroupId;
+    return Status::OK();
+  }
+
+  bool Extend(const void* data) {
+    auto group_id = *static_cast<const group_id_t*>(data);
+    bool extends =
+        save_group_id_ == kNoGroupId ? kDefaultExtends : save_group_id_ == group_id;
+    save_group_id_ = group_id;
+    return extends;
+  }
+
+  template <typename Batch>
+  Result<group_id_t> MapGroupIdAt(const Batch& batch, int64_t offset) {
+    if (offset < 0 || offset >= batch.length) {
+      return Status::Invalid("requesting group id out of bounds");
+    }
+    if (!grouper_) return kNoGroupId;
+    ARROW_ASSIGN_OR_RAISE(auto datum, grouper_->Consume(batch, offset,
+                                                        /*consume_length=*/1));
+    if (!(datum.is_array() || datum.is_chunked_array())) {
+      return Status::Invalid("accessing unsupported datum kind ", datum.kind());
+    }
+    const std::shared_ptr<ArrayData>& data =
+        datum.is_array() ? datum.array() : datum.chunked_array()->chunk(0)->data();
+    ARROW_DCHECK(data->GetNullCount() == 0);
+    DCHECK_EQ(data->type->id(), GroupIdType::type_id);
+    DCHECK_EQ(1, data->length);
+    const group_id_t* values = data->GetValues<group_id_t>(1);
+    return values[0];
+  }
+
+  template <typename Batch>
+  Result<GroupingSegment> GetNextSegmentImpl(const Batch& batch, int64_t offset) {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, key_types_));
+    if (offset == batch.length) {
+      return MakeSegment(batch.length, offset, 0, kEmptyExtends);
+    }
+    // ARROW-18311: make Grouper support Reset()
+    // so it can be cached instead of recreated below
+    //
+    // the group id must be computed prior to resetting the grouper, since it is compared
+    // to save_group_id_, and after resetting the grouper produces incomparable group ids
+    ARROW_ASSIGN_OR_RAISE(auto group_id, MapGroupIdAt(batch, offset));
+    ExtendFunc bound_extend = [this, group_id](const void* data) {
+      bool extends = Extend(&group_id);
+      save_group_id_ = *static_cast<const group_id_t*>(data);
+      return extends;
+    };
+    ARROW_ASSIGN_OR_RAISE(grouper_, Grouper::Make(key_types_, ctx_));  // TODO: reset it
+    ARROW_ASSIGN_OR_RAISE(auto datum, grouper_->Consume(batch, offset));
+    if (datum.is_array()) {

Review Comment:
   This is the array of group ids; it can't be a scalar.



##########
cpp/src/arrow/compute/row/grouper.cc:
##########
@@ -39,12 +43,521 @@
 namespace arrow {
 
 using internal::checked_cast;
+using internal::PrimitiveScalarBase;
 
 namespace compute {
 
 namespace {
 
-struct GrouperImpl : Grouper {
+constexpr uint32_t kNoGroupId = std::numeric_limits<uint32_t>::max();
+
+using group_id_t = std::remove_const<decltype(kNoGroupId)>::type;
+using GroupIdType = CTypeTraits<group_id_t>::ArrowType;
+auto group_id_type = std::make_shared<GroupIdType>();
+
+inline const uint8_t* GetValuesAsBytes(const ArrayData& data, int64_t offset = 0) {
+  DCHECK_GT(data.type->byte_width(), 0);
+  int64_t absolute_byte_offset = (data.offset + offset) * data.type->byte_width();
+  return data.GetValues<uint8_t>(1, absolute_byte_offset);
+}
+
+inline const uint8_t* GetValuesAsBytes(const ArraySpan& data, int64_t offset = 0) {
+  DCHECK_GT(data.type->byte_width(), 0);
+  int64_t absolute_byte_offset = (data.offset + offset) * data.type->byte_width();
+  return data.GetValues<uint8_t>(1, absolute_byte_offset);
+}
+
+template <typename Value>
+Status CheckForGetNextSegment(const std::vector<Value>& values, int64_t length,
+                              int64_t offset, const std::vector<TypeHolder>& key_types) {
+  if (offset < 0 || offset > length) {
+    return Status::Invalid("invalid grouping segmenter offset: ", offset);
+  }
+  if (values.size() != key_types.size()) {
+    return Status::Invalid("expected batch size ", key_types.size(), " but got ",
+                           values.size());
+  }
+  for (size_t i = 0; i < key_types.size(); i++) {
+    const auto& value = values[i];
+    const auto& key_type = key_types[i];
+    if (*value.type() != *key_type.type) {
+      return Status::Invalid("expected batch value ", i, " of type ", *key_type.type,
+                             " but got ", *value.type());
+    }
+  }
+  return Status::OK();
+}
+
+template <typename Batch>
+enable_if_t<std::is_same<Batch, ExecSpan>::value || std::is_same<Batch, ExecBatch>::value,
+            Status>
+CheckForGetNextSegment(const Batch& batch, int64_t offset,
+                       const std::vector<TypeHolder>& key_types) {
+  return CheckForGetNextSegment(batch.values, batch.length, offset, key_types);
+}
+
+struct BaseGroupingSegmenter : public GroupingSegmenter {
+  explicit BaseGroupingSegmenter(const std::vector<TypeHolder>& key_types)
+      : key_types_(key_types) {}
+
+  const std::vector<TypeHolder>& key_types() const override { return key_types_; }
+
+  std::vector<TypeHolder> key_types_;
+};
+
+GroupingSegment MakeSegment(int64_t batch_length, int64_t offset, int64_t length,
+                            bool extends) {
+  return GroupingSegment{offset, length, offset + length >= batch_length, extends};
+}
+
+int64_t GetMatchLength(const uint8_t* match_bytes, int64_t match_width,
+                       const uint8_t* array_bytes, int64_t offset, int64_t length) {
+  int64_t cursor, byte_cursor;
+  for (cursor = offset, byte_cursor = match_width * cursor; cursor < length;
+       cursor++, byte_cursor += match_width) {
+    if (memcmp(match_bytes, array_bytes + byte_cursor,
+               static_cast<size_t>(match_width)) != 0) {
+      break;
+    }
+  }
+  return std::min(cursor, length - offset);
+}
+
+using ExtendFunc = std::function<bool(const void*)>;
+constexpr bool kDefaultExtends = true;
+constexpr bool kEmptyExtends = true;
+
+Result<GroupingSegment> GetNextSegmentChunked(
+    const std::shared_ptr<ChunkedArray>& chunked_array, int64_t offset,
+    ExtendFunc extend) {
+  if (offset >= chunked_array->length()) {
+    return MakeSegment(chunked_array->length(), chunked_array->length(), 0,
+                       kEmptyExtends);
+  }
+  int64_t remaining_offset = offset;
+  const auto& arrays = chunked_array->chunks();
+  for (size_t i = 0; remaining_offset >= 0 && i < arrays.size(); i++) {
+    // look up chunk containing offset
+    int64_t array_length = arrays[i]->length();
+    if (remaining_offset < array_length) {
+      // found - switch to matching
+      int64_t match_width = arrays[i]->type()->byte_width();
+      const uint8_t* match_bytes = GetValuesAsBytes(*arrays[i]->data(), remaining_offset);
+      int64_t total_match_length = 0;
+      for (; i < arrays.size(); i++) {
+        int64_t array_length = arrays[i]->length();
+        if (array_length <= 0) continue;
+        const uint8_t* array_bytes = GetValuesAsBytes(*arrays[i]->data());
+        int64_t match_length = GetMatchLength(match_bytes, match_width, array_bytes,
+                                              remaining_offset, array_length);
+        total_match_length += match_length;
+        remaining_offset = 0;
+        if (match_length < array_length - remaining_offset) break;
+      }
+      bool extends = extend(match_bytes);
+      return MakeSegment(chunked_array->length(), offset, total_match_length, extends);
+    }
+    remaining_offset -= array_length;
+  }
+  return Status::Invalid("segmenting invalid chunked array value");
+}
+
+struct NoKeysGroupingSegmenter : public BaseGroupingSegmenter {
+  static std::unique_ptr<GroupingSegmenter> Make() {
+    return std::make_unique<NoKeysGroupingSegmenter>();
+  }
+
+  NoKeysGroupingSegmenter() : BaseGroupingSegmenter({}) {}
+
+  Status Reset() override { return Status::OK(); }
+
+  template <typename Batch>
+  Result<GroupingSegment> GetNextSegmentImpl(const Batch& batch, int64_t offset) {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {}));
+    return MakeSegment(batch.length, offset, batch.length - offset, kDefaultExtends);
+  }
+
+  Result<GroupingSegment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    return GetNextSegmentImpl(batch, offset);
+  }
+
+  Result<GroupingSegment> GetNextSegment(const ExecBatch& batch,
+                                         int64_t offset) override {
+    return GetNextSegmentImpl(batch, offset);
+  }
+};
+
+struct SimpleKeyGroupingSegmenter : public BaseGroupingSegmenter {
+  static Result<std::unique_ptr<GroupingSegmenter>> Make(TypeHolder key_type) {
+    return std::make_unique<SimpleKeyGroupingSegmenter>(key_type);
+  }
+
+  explicit SimpleKeyGroupingSegmenter(TypeHolder key_type)
+      : BaseGroupingSegmenter({key_type}), key_type_(key_types_[0]), save_key_data_() {}
+
+  Status CheckType(const DataType& type) {
+    if (!is_fixed_width(type)) {
+      return Status::Invalid("SimpleKeyGroupingSegmenter does not support type ", type);
+    }
+    return Status::OK();
+  }
+
+  Status Reset() override {
+    save_key_data_.resize(0);
+    return Status::OK();
+  }
+
+  bool Extend(const void* data) {
+    size_t byte_width = static_cast<size_t>(key_type_.type->byte_width());
+    bool extends = save_key_data_.size() != byte_width
+                       ? kDefaultExtends
+                       : 0 == memcmp(save_key_data_.data(), data, byte_width);
+    save_key_data_.resize(byte_width);
+    memcpy(save_key_data_.data(), data, byte_width);
+    return extends;
+  }
+
+  Result<GroupingSegment> GetNextSegment(const Scalar& scalar, int64_t offset,
+                                         int64_t length) {
+    ARROW_RETURN_NOT_OK(CheckType(*scalar.type));
+    if (!scalar.is_valid) {
+      return Status::Invalid("segmenting an invalid scalar");
+    }
+    auto data = checked_cast<const PrimitiveScalarBase&>(scalar).data();
+    bool extends = length > 0 ? Extend(data) : kEmptyExtends;
+    return MakeSegment(length, 0, length, extends);
+  }
+
+  Result<GroupingSegment> GetNextSegment(const DataType& array_type,
+                                         const uint8_t* array_bytes, int64_t offset,
+                                         int64_t length) {
+    RETURN_NOT_OK(CheckType(array_type));
+    int64_t byte_width = array_type.byte_width();
+    int64_t match_length = GetMatchLength(array_bytes + offset * byte_width, byte_width,
+                                          array_bytes, offset, length);
+    bool extends = length > 0 ? Extend(array_bytes + offset * byte_width) : kEmptyExtends;
+    return MakeSegment(length, offset, match_length, extends);
+  }
+
+  Result<GroupingSegment> GetNextSegment(const ExecSpan& batch, int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {key_type_}));
+    if (offset == batch.length) {
+      return MakeSegment(batch.length, offset, 0, kEmptyExtends);
+    }
+    const auto& value = batch.values[0];
+    if (value.is_scalar()) {
+      return GetNextSegment(*value.scalar, offset, batch.length);
+    }
+    ARROW_DCHECK(value.is_array());
+    const auto& array = value.array;
+    if (array.GetNullCount() > 0) {
+      return Status::NotImplemented("segmenting a nullable array");
+    }
+    return GetNextSegment(*array.type, GetValuesAsBytes(array), offset, batch.length);
+  }
+
+  Result<GroupingSegment> GetNextSegment(const ExecBatch& batch,
+                                         int64_t offset) override {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, {key_type_}));
+    if (offset == batch.length) {
+      return MakeSegment(batch.length, offset, 0, kEmptyExtends);
+    }
+    const auto& value = batch.values[0];
+    if (value.is_scalar()) {
+      return GetNextSegment(*value.scalar(), offset, batch.length);
+    }
+    if (value.is_array()) {
+      auto array = value.array();
+      if (array->GetNullCount() > 0) {
+        return Status::NotImplemented("segmenting a nullable array");
+      }
+      return GetNextSegment(*array->type, GetValuesAsBytes(*array), offset, batch.length);
+    }
+    if (value.is_chunked_array()) {
+      auto array = value.chunked_array();
+      if (array->null_count() > 0) {
+        return Status::NotImplemented("segmenting a nullable array");
+      }
+      return GetNextSegmentChunked(array, offset, bound_extend_);
+    }
+    return Status::Invalid("segmenting unsupported value kind ", value.kind());
+  }
+
+ private:
+  TypeHolder key_type_;
+  std::vector<uint8_t> save_key_data_;
+  ExtendFunc bound_extend_ = [this](const void* data) { return Extend(data); };
+};
+
+struct AnyKeysGroupingSegmenter : public BaseGroupingSegmenter {
+  static Result<std::unique_ptr<GroupingSegmenter>> Make(
+      const std::vector<TypeHolder>& key_types, ExecContext* ctx) {
+    ARROW_RETURN_NOT_OK(Grouper::Make(key_types, ctx));  // check types
+    return std::make_unique<AnyKeysGroupingSegmenter>(key_types, ctx);
+  }
+
+  AnyKeysGroupingSegmenter(const std::vector<TypeHolder>& key_types, ExecContext* ctx)
+      : BaseGroupingSegmenter(key_types),
+        ctx_(ctx),
+        grouper_(nullptr),
+        save_group_id_(kNoGroupId) {}
+
+  Status Reset() override {
+    grouper_ = nullptr;
+    save_group_id_ = kNoGroupId;
+    return Status::OK();
+  }
+
+  bool Extend(const void* data) {
+    auto group_id = *static_cast<const group_id_t*>(data);
+    bool extends =
+        save_group_id_ == kNoGroupId ? kDefaultExtends : save_group_id_ == group_id;
+    save_group_id_ = group_id;
+    return extends;
+  }
+
+  template <typename Batch>
+  Result<group_id_t> MapGroupIdAt(const Batch& batch, int64_t offset) {
+    if (offset < 0 || offset >= batch.length) {
+      return Status::Invalid("requesting group id out of bounds");
+    }
+    if (!grouper_) return kNoGroupId;
+    ARROW_ASSIGN_OR_RAISE(auto datum, grouper_->Consume(batch, offset,
+                                                        /*consume_length=*/1));
+    if (!(datum.is_array() || datum.is_chunked_array())) {
+      return Status::Invalid("accessing unsupported datum kind ", datum.kind());
+    }
+    const std::shared_ptr<ArrayData>& data =
+        datum.is_array() ? datum.array() : datum.chunked_array()->chunk(0)->data();
+    ARROW_DCHECK(data->GetNullCount() == 0);
+    DCHECK_EQ(data->type->id(), GroupIdType::type_id);
+    DCHECK_EQ(1, data->length);
+    const group_id_t* values = data->GetValues<group_id_t>(1);
+    return values[0];
+  }
+
+  template <typename Batch>
+  Result<GroupingSegment> GetNextSegmentImpl(const Batch& batch, int64_t offset) {
+    ARROW_RETURN_NOT_OK(CheckForGetNextSegment(batch, offset, key_types_));
+    if (offset == batch.length) {
+      return MakeSegment(batch.length, offset, 0, kEmptyExtends);
+    }
+    // ARROW-18311: make Grouper support Reset()
+    // so it can be cached instead of recreated below
+    //
+    // the group id must be computed prior to resetting the grouper, since it is compared
+    // to save_group_id_, and after resetting the grouper produces incomparable group ids
+    ARROW_ASSIGN_OR_RAISE(auto group_id, MapGroupIdAt(batch, offset));
+    ExtendFunc bound_extend = [this, group_id](const void* data) {
+      bool extends = Extend(&group_id);
+      save_group_id_ = *static_cast<const group_id_t*>(data);
+      return extends;
+    };
+    ARROW_ASSIGN_OR_RAISE(grouper_, Grouper::Make(key_types_, ctx_));  // TODO: reset it
+    ARROW_ASSIGN_OR_RAISE(auto datum, grouper_->Consume(batch, offset));
+    if (datum.is_array()) {
+      const std::shared_ptr<ArrayData>& data = datum.array();
+      ARROW_DCHECK(data->GetNullCount() == 0);

Review Comment:
   This is the array of group ids; it can't have nulls.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] icexelloss commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "icexelloss (via GitHub)" <gi...@apache.org>.
icexelloss commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1116055429


##########
cpp/src/arrow/compute/row/grouper.h:
##########
@@ -30,6 +30,49 @@
 namespace arrow {
 namespace compute {
 
+/// \brief A segment of contiguous rows for grouping
+struct ARROW_EXPORT GroupingSegment {
+  int64_t offset;
+  int64_t length;
+  bool is_open;
+  bool extends;
+};
+
+inline bool operator==(const GroupingSegment& segment1, const GroupingSegment& segment2) {
+  return segment1.offset == segment2.offset && segment1.length == segment2.length &&
+         segment1.is_open == segment2.is_open && segment1.extends == segment2.extends;
+}
+inline bool operator!=(const GroupingSegment& segment1, const GroupingSegment& segment2) {
+  return !(segment1 == segment2);
+}
+
+/// \brief Computes grouping segments for a batch. Each segment covers rows with identical
+/// values in the batch. The values in the batch are often selected as keys from a larger
+/// batch.
+class ARROW_EXPORT GroupingSegmenter {
+ public:
+  virtual ~GroupingSegmenter() = default;
+
+  /// \brief Construct a GroupingSegmenter which receives the specified key types
+  static Result<std::unique_ptr<GroupingSegmenter>> Make(
+      const std::vector<TypeHolder>& key_types, bool nullable_keys = false,
+      ExecContext* ctx = default_exec_context());
+
+  /// \brief Return the key types of this segmenter
+  virtual const std::vector<TypeHolder>& key_types() const = 0;

Review Comment:
   I am not familiar with TypeHolder - how it is different from just datatypes?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rtpsw commented on a diff in pull request #34311: GH-32884: [C++] Add ordered aggregation

Posted by "rtpsw (via GitHub)" <gi...@apache.org>.
rtpsw commented on code in PR #34311:
URL: https://github.com/apache/arrow/pull/34311#discussion_r1117092736


##########
cpp/src/arrow/compute/exec/options.h:
##########
@@ -199,21 +199,32 @@ class ARROW_EXPORT ProjectNodeOptions : public ExecNodeOptions {
   std::vector<std::string> names;
 };
 
-/// \brief Make a node which aggregates input batches, optionally grouped by keys.
+/// \brief Make a node which aggregates input batches, optionally grouped by keys and
+/// optionally segmented by segment-keys. Both keys and segment-keys determine the group.
+/// However segment-keys are also used for determining grouping segments, which should be
+/// large, and allow streaming a partial aggregation result after processing each segment.
+/// One common use-case for segment-keys is ordered aggregation, in which the segment-key
+/// attribute specifies a column with non-decreasing values or a lexigographically-ordered
+/// set of such columns.
 ///
 /// If the keys attribute is a non-empty vector, then each aggregate in `aggregates` is
 /// expected to be a HashAggregate function. If the keys attribute is an empty vector,
 /// then each aggregate is assumed to be a ScalarAggregate function.
 class ARROW_EXPORT AggregateNodeOptions : public ExecNodeOptions {
  public:
   explicit AggregateNodeOptions(std::vector<Aggregate> aggregates,
-                                std::vector<FieldRef> keys = {})
-      : aggregates(std::move(aggregates)), keys(std::move(keys)) {}
+                                std::vector<FieldRef> keys = {},
+                                std::vector<FieldRef> segment_keys = {})
+      : aggregates(std::move(aggregates)),
+        keys(std::move(keys)),
+        segment_keys(std::move(segment_keys)) {}
 
   // aggregations which will be applied to the targetted fields
   std::vector<Aggregate> aggregates;
   // keys by which aggregations will be grouped
   std::vector<FieldRef> keys;
+  // keys by which aggregations will be segmented

Review Comment:
   Both `keys` and `segment_keys` are optional.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org