pitrou commented on code in PR #12460:
URL: https://github.com/apache/arrow/pull/12460#discussion_r873810599


##########
cpp/src/arrow/compute/kernels/vector_cumulative_ops.cc:
##########
@@ -0,0 +1,224 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "arrow/array/array_base.h"
+#include "arrow/array/builder_primitive.h"
+#include "arrow/compute/api_scalar.h"
+#include "arrow/compute/api_vector.h"
+#include "arrow/compute/cast.h"
+#include "arrow/compute/kernels/base_arithmetic_internal.h"
+#include "arrow/compute/kernels/codegen_internal.h"
+#include "arrow/compute/kernels/common.h"
+#include "arrow/result.h"
+#include "arrow/util/bit_util.h"
+#include "arrow/visit_type_inline.h"
+
+namespace arrow {
+namespace compute {
+namespace internal {
+namespace {
+template <typename OptionsType>
+struct CumulativeOptionsWrapper : public OptionsWrapper<OptionsType> {
+  using State = CumulativeOptionsWrapper<OptionsType>;
+
+  explicit CumulativeOptionsWrapper(OptionsType options)
+      : OptionsWrapper<OptionsType>(std::move(options)) {}
+
+  static Result<std::unique_ptr<KernelState>> Init(KernelContext* ctx,
+                                                   const KernelInitArgs& args) 
{
+    auto options = checked_cast<const OptionsType*>(args.options);
+    if (!options) {
+      return Status::Invalid(
+          "Attempted to initialize KernelState from null FunctionOptions");
+    }
+
+    const auto& start = options->start;
+    if (!start || !start->is_valid) {
+      return Status::Invalid("Cumulative `start` option must be non-null and 
valid");
+    }
+
+    // Ensure `start` option matches input type
+    if (!start->type->Equals(args.inputs[0].type)) {
+      ARROW_ASSIGN_OR_RAISE(auto casted_start,
+                            Cast(Datum(start), args.inputs[0].type, 
CastOptions::Safe(),
+                                 ctx->exec_context()));
+      auto new_options = OptionsType(casted_start.scalar(), 
options->skip_nulls);
+      return ::arrow::internal::make_unique<State>(new_options);
+    }
+    return ::arrow::internal::make_unique<State>(*options);
+  }
+};
+
+// The driver kernel for all cumulative compute functions. Op is a compute 
kernel
+// representing any binary associative operation (add, product, min, max, 
etc.) and
+// OptionsType the options type corresponding to Op. ArgType and OutType are 
the input
+// and output types, which will normally be the same (e.g. the cumulative sum 
of an array
+// of Int64Type will result in an array of Int64Type).
+template <typename OutType, typename ArgType, typename Op, typename 
OptionsType>
+struct CumulativeGeneric {
+  using OutValue = typename GetOutputType<OutType>::T;
+  using ArgValue = typename GetViewType<ArgType>::T;
+
+  KernelContext* ctx;
+  ArgValue accumulator;
+  bool skip_nulls;
+  bool encountered_null = false;
+  Datum values;
+  NumericBuilder<OutType>* builder;
+
+  Status Cumulate(std::shared_ptr<ArrayData>* out_arr) {
+    switch (values.kind()) {
+      case Datum::SCALAR: {
+        auto in_value = UnboxScalar<OutType>::Unbox(*(values.scalar()));
+        Status st;
+        auto result = Op::template Call<OutValue, ArgValue, ArgValue>(ctx, 
accumulator,
+                                                                      
in_value, &st);
+        RETURN_NOT_OK(st);
+        RETURN_NOT_OK(builder->Append(result));
+        break;
+      }
+      case Datum::ARRAY: {
+        auto arr_input = values.array();
+        RETURN_NOT_OK(builder->Reserve(arr_input->length));
+        RETURN_NOT_OK(Call(*arr_input));
+        break;
+      }
+      case Datum::CHUNKED_ARRAY: {
+        const auto& chunked_input = values.chunked_array();
+
+        for (const auto& chunk : chunked_input->chunks()) {
+          RETURN_NOT_OK(builder->Reserve(chunk->length()));

Review Comment:
   It would certainly be more efficient to reserve once and for all instead of 
growing the output at every chunk:
   ```suggestion
           RETURN_NOT_OK(builder->Reserve(chunked_input.length()));
           for (const auto& chunk : chunked_input->chunks()) {
   ```



##########
cpp/src/arrow/compute/kernels/vector_cumulative_ops_test.cc:
##########
@@ -0,0 +1,314 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <memory>
+#include <string>
+#include <vector>
+
+#include <gtest/gtest.h>
+
+#include "arrow/array.h"
+#include "arrow/chunked_array.h"
+#include "arrow/testing/gtest_util.h"
+#include "arrow/testing/util.h"
+#include "arrow/type.h"
+
+#include "arrow/array/builder_primitive.h"
+#include "arrow/compute/api.h"
+#include "arrow/compute/kernels/test_util.h"
+
+namespace arrow {
+namespace compute {
+
+TEST(TestCumulativeSum, Empty) {
+  CumulativeSumOptions options;
+  for (auto ty : NumericTypes()) {
+    auto empty_arr = ArrayFromJSON(ty, "[]");
+    auto empty_chunked = ChunkedArrayFromJSON(ty, {"[]"});
+    CheckVectorUnary("cumulative_sum", empty_arr, empty_arr, &options);
+    CheckVectorUnary("cumulative_sum_checked", empty_arr, empty_arr, &options);
+
+    CheckVectorUnary("cumulative_sum", empty_chunked, empty_chunked, &options);
+    CheckVectorUnary("cumulative_sum_checked", empty_chunked, empty_chunked, 
&options);
+  }
+}
+
+TEST(TestCumulativeSum, AllNulls) {
+  CumulativeSumOptions options;
+  for (auto ty : NumericTypes()) {
+    auto nulls_arr = ArrayFromJSON(ty, "[null, null, null]");
+    auto nulls_one_chunk = ChunkedArrayFromJSON(ty, {"[null, null, null]"});
+    auto nulls_three_chunks = ChunkedArrayFromJSON(ty, {"[null]", "[null]", 
"[null]"});
+    CheckVectorUnary("cumulative_sum", nulls_arr, nulls_arr, &options);
+    CheckVectorUnary("cumulative_sum_checked", nulls_arr, nulls_arr, &options);
+
+    CheckVectorUnary("cumulative_sum", nulls_one_chunk, nulls_one_chunk, 
&options);
+    CheckVectorUnary("cumulative_sum_checked", nulls_one_chunk, 
nulls_one_chunk,
+                     &options);
+
+    CheckVectorUnary("cumulative_sum", nulls_three_chunks, nulls_one_chunk, 
&options);
+    CheckVectorUnary("cumulative_sum_checked", nulls_three_chunks, 
nulls_one_chunk,
+                     &options);
+  }
+}
+
+TEST(TestCumulativeSum, ScalarInput) {
+  CumulativeSumOptions no_start;
+  CumulativeSumOptions with_start(10);
+  for (auto ty : NumericTypes()) {
+    CheckVectorUnary("cumulative_sum", ScalarFromJSON(ty, "10"),
+                     ArrayFromJSON(ty, "[10]"), &no_start);
+    CheckVectorUnary("cumulative_sum_checked", ScalarFromJSON(ty, "10"),
+                     ArrayFromJSON(ty, "[10]"), &no_start);
+
+    CheckVectorUnary("cumulative_sum", ScalarFromJSON(ty, "10"),
+                     ArrayFromJSON(ty, "[20]"), &with_start);
+    CheckVectorUnary("cumulative_sum_checked", ScalarFromJSON(ty, "10"),
+                     ArrayFromJSON(ty, "[20]"), &with_start);
+  }

Review Comment:
   Can you add a test with a null scalar as input?



##########
cpp/src/arrow/compute/kernels/vector_cumulative_ops_test.cc:
##########
@@ -0,0 +1,314 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <memory>
+#include <string>
+#include <vector>
+
+#include <gtest/gtest.h>
+
+#include "arrow/array.h"
+#include "arrow/chunked_array.h"
+#include "arrow/testing/gtest_util.h"
+#include "arrow/testing/util.h"
+#include "arrow/type.h"
+
+#include "arrow/array/builder_primitive.h"
+#include "arrow/compute/api.h"
+#include "arrow/compute/kernels/test_util.h"
+
+namespace arrow {
+namespace compute {
+
+TEST(TestCumulativeSum, Empty) {
+  CumulativeSumOptions options;
+  for (auto ty : NumericTypes()) {
+    auto empty_arr = ArrayFromJSON(ty, "[]");
+    auto empty_chunked = ChunkedArrayFromJSON(ty, {"[]"});
+    CheckVectorUnary("cumulative_sum", empty_arr, empty_arr, &options);
+    CheckVectorUnary("cumulative_sum_checked", empty_arr, empty_arr, &options);
+
+    CheckVectorUnary("cumulative_sum", empty_chunked, empty_chunked, &options);
+    CheckVectorUnary("cumulative_sum_checked", empty_chunked, empty_chunked, 
&options);
+  }
+}
+
+TEST(TestCumulativeSum, AllNulls) {
+  CumulativeSumOptions options;
+  for (auto ty : NumericTypes()) {
+    auto nulls_arr = ArrayFromJSON(ty, "[null, null, null]");
+    auto nulls_one_chunk = ChunkedArrayFromJSON(ty, {"[null, null, null]"});
+    auto nulls_three_chunks = ChunkedArrayFromJSON(ty, {"[null]", "[null]", 
"[null]"});
+    CheckVectorUnary("cumulative_sum", nulls_arr, nulls_arr, &options);
+    CheckVectorUnary("cumulative_sum_checked", nulls_arr, nulls_arr, &options);
+
+    CheckVectorUnary("cumulative_sum", nulls_one_chunk, nulls_one_chunk, 
&options);
+    CheckVectorUnary("cumulative_sum_checked", nulls_one_chunk, 
nulls_one_chunk,
+                     &options);
+
+    CheckVectorUnary("cumulative_sum", nulls_three_chunks, nulls_one_chunk, 
&options);
+    CheckVectorUnary("cumulative_sum_checked", nulls_three_chunks, 
nulls_one_chunk,
+                     &options);
+  }
+}
+
+TEST(TestCumulativeSum, ScalarInput) {
+  CumulativeSumOptions no_start;
+  CumulativeSumOptions with_start(10);
+  for (auto ty : NumericTypes()) {
+    CheckVectorUnary("cumulative_sum", ScalarFromJSON(ty, "10"),
+                     ArrayFromJSON(ty, "[10]"), &no_start);
+    CheckVectorUnary("cumulative_sum_checked", ScalarFromJSON(ty, "10"),
+                     ArrayFromJSON(ty, "[10]"), &no_start);
+
+    CheckVectorUnary("cumulative_sum", ScalarFromJSON(ty, "10"),
+                     ArrayFromJSON(ty, "[20]"), &with_start);
+    CheckVectorUnary("cumulative_sum_checked", ScalarFromJSON(ty, "10"),
+                     ArrayFromJSON(ty, "[20]"), &with_start);
+  }
+}
+
+using testing::HasSubstr;
+
+#define CHECK_CUMULATIVE_SUM_UNSIGNED_OVERFLOW(FUNC, TYPENAME, CTYPE)          
  \
+  {                                                                            
  \
+    CumulativeSumOptions pos_overflow(1);                                      
  \
+    auto max = std::numeric_limits<CTYPE>::max();                              
  \
+    TYPENAME##Builder builder;                                                 
  \
+    std::shared_ptr<Array> array;                                              
  \
+    ASSERT_OK(builder.Append(max));                                            
  \
+    ASSERT_OK(builder.Finish(&array));                                         
  \
+    EXPECT_RAISES_WITH_MESSAGE_THAT(                                           
  \
+        Invalid, HasSubstr("overflow"),                                        
  \
+        CallFunction(FUNC, {TypeTraits<TYPENAME##Type>::ScalarType(max)},      
  \
+                     &pos_overflow));                                          
  \
+    EXPECT_RAISES_WITH_MESSAGE_THAT(Invalid, HasSubstr("overflow"),            
  \
+                                    CallFunction(FUNC, {array}, 
&pos_overflow)); \
+  }
+
+#define CHECK_CUMULATIVE_SUM_SIGNED_OVERFLOW(FUNC, TYPENAME, CTYPE)            
  \
+  CHECK_CUMULATIVE_SUM_UNSIGNED_OVERFLOW(FUNC, TYPENAME, CTYPE);               
  \
+  {                                                                            
  \
+    CumulativeSumOptions neg_overflow(-1);                                     
  \
+    auto min = std::numeric_limits<CTYPE>::lowest();                           
  \
+    TYPENAME##Builder builder;                                                 
  \
+    std::shared_ptr<Array> array;                                              
  \
+    ASSERT_OK(builder.Append(min));                                            
  \
+    ASSERT_OK(builder.Finish(&array));                                         
  \
+    EXPECT_RAISES_WITH_MESSAGE_THAT(                                           
  \
+        Invalid, HasSubstr("overflow"),                                        
  \
+        CallFunction(FUNC, {TypeTraits<TYPENAME##Type>::ScalarType(min)},      
  \
+                     &neg_overflow));                                          
  \
+    EXPECT_RAISES_WITH_MESSAGE_THAT(Invalid, HasSubstr("overflow"),            
  \
+                                    CallFunction(FUNC, {array}, 
&neg_overflow)); \
+  }
+
+TEST(TestCumulativeSum, IntegerOverflow) {
+  CHECK_CUMULATIVE_SUM_UNSIGNED_OVERFLOW("cumulative_sum_checked", UInt8, 
uint8_t);

Review Comment:
   Macros are tedious to maintain and debug. You can use a C++ template 
function instead, or perhaps it's easier to hardcode the test values for each 
type.



##########
python/pyarrow/tests/test_compute.py:
##########
@@ -2510,6 +2511,63 @@ def test_min_max_element_wise():
     assert result == pa.array([1, 2, None])
 
 
[email protected]('start', (1.1, 10.5, -10.5))
[email protected]('skip_nulls', (True, False))
+def test_cumulative_sum(start, skip_nulls):
+    # Exact tests (e.g., integral types)
+    start_int = int(start)
+    starts = [start_int, pa.scalar(start_int, type=pa.int8()),
+              pa.scalar(start_int, type=pa.int64())]
+    for strt in starts:
+        arrays = [
+            pa.array([1, 2, 3]),
+            pa.array([0, None, 20, 30]),
+            pa.chunked_array([[0, None], [20, 30]])
+        ]
+        expected_arrays = [
+            pa.array([1, 3, 6]),
+            pa.array([0, None, 20, 50])
+            if skip_nulls else pa.array([0, None, None, None]),
+            pa.chunked_array([[0, None, 20, 50]])
+            if skip_nulls else pa.chunked_array([[0, None, None, None]])
+        ]
+        for i, arr in enumerate(arrays):
+            result = pc.cumulative_sum(arr, start=strt, skip_nulls=skip_nulls)
+            # Add `start` offset to expected array before comparing
+            expected = pc.add(expected_arrays[i], strt)
+            assert result.equals(expected)
+
+    # Approximate and NaN tests (e.g., floating-point types)
+    # NOTE: Conversion from Arrow to NumPy replaces null slots with NaNs
+    # which prevent fully validating both states of `skip_nulls` for
+    # floating-point values. Ideally, equality comparisons make use of Arrow's
+    # `equals()` functions but an approximate version is not exposed in Python.

Review Comment:
   To avoid floating-point inaccuracy issues, you can just carefully choose the 
inputs. For example, make sure the fractional part is an inverse power of two 
(`0.5`, `0.25`...).



##########
cpp/src/arrow/compute/kernels/vector_cumulative_ops.cc:
##########
@@ -0,0 +1,224 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "arrow/array/array_base.h"
+#include "arrow/array/builder_primitive.h"
+#include "arrow/compute/api_scalar.h"
+#include "arrow/compute/api_vector.h"
+#include "arrow/compute/cast.h"
+#include "arrow/compute/kernels/base_arithmetic_internal.h"
+#include "arrow/compute/kernels/codegen_internal.h"
+#include "arrow/compute/kernels/common.h"
+#include "arrow/result.h"
+#include "arrow/util/bit_util.h"
+#include "arrow/visit_type_inline.h"
+
+namespace arrow {
+namespace compute {
+namespace internal {
+namespace {
+template <typename OptionsType>
+struct CumulativeOptionsWrapper : public OptionsWrapper<OptionsType> {
+  using State = CumulativeOptionsWrapper<OptionsType>;
+
+  explicit CumulativeOptionsWrapper(OptionsType options)
+      : OptionsWrapper<OptionsType>(std::move(options)) {}
+
+  static Result<std::unique_ptr<KernelState>> Init(KernelContext* ctx,
+                                                   const KernelInitArgs& args) 
{
+    auto options = checked_cast<const OptionsType*>(args.options);
+    if (!options) {
+      return Status::Invalid(
+          "Attempted to initialize KernelState from null FunctionOptions");
+    }
+
+    const auto& start = options->start;
+    if (!start || !start->is_valid) {
+      return Status::Invalid("Cumulative `start` option must be non-null and 
valid");
+    }
+
+    // Ensure `start` option matches input type
+    if (!start->type->Equals(args.inputs[0].type)) {
+      ARROW_ASSIGN_OR_RAISE(auto casted_start,
+                            Cast(Datum(start), args.inputs[0].type, 
CastOptions::Safe(),
+                                 ctx->exec_context()));
+      auto new_options = OptionsType(casted_start.scalar(), 
options->skip_nulls);
+      return ::arrow::internal::make_unique<State>(new_options);
+    }
+    return ::arrow::internal::make_unique<State>(*options);
+  }
+};
+
+// The driver kernel for all cumulative compute functions. Op is a compute 
kernel
+// representing any binary associative operation (add, product, min, max, 
etc.) and
+// OptionsType the options type corresponding to Op. ArgType and OutType are 
the input
+// and output types, which will normally be the same (e.g. the cumulative sum 
of an array
+// of Int64Type will result in an array of Int64Type).
+template <typename OutType, typename ArgType, typename Op, typename 
OptionsType>
+struct CumulativeGeneric {
+  using OutValue = typename GetOutputType<OutType>::T;
+  using ArgValue = typename GetViewType<ArgType>::T;
+
+  KernelContext* ctx;
+  ArgValue accumulator;
+  bool skip_nulls;
+  bool encountered_null = false;
+  Datum values;
+  NumericBuilder<OutType>* builder;
+
+  Status Cumulate(std::shared_ptr<ArrayData>* out_arr) {
+    switch (values.kind()) {
+      case Datum::SCALAR: {
+        auto in_value = UnboxScalar<OutType>::Unbox(*(values.scalar()));
+        Status st;
+        auto result = Op::template Call<OutValue, ArgValue, ArgValue>(ctx, 
accumulator,
+                                                                      
in_value, &st);
+        RETURN_NOT_OK(st);
+        RETURN_NOT_OK(builder->Append(result));
+        break;
+      }
+      case Datum::ARRAY: {
+        auto arr_input = values.array();
+        RETURN_NOT_OK(builder->Reserve(arr_input->length));
+        RETURN_NOT_OK(Call(*arr_input));
+        break;
+      }
+      case Datum::CHUNKED_ARRAY: {
+        const auto& chunked_input = values.chunked_array();
+
+        for (const auto& chunk : chunked_input->chunks()) {
+          RETURN_NOT_OK(builder->Reserve(chunk->length()));
+          RETURN_NOT_OK(Call(*chunk->data()));
+        }
+        break;
+      }
+      default:
+        return Status::NotImplemented(
+            "Unsupported input type for function 'cumulative_<operator>': ",
+            values.ToString());
+    }
+
+    RETURN_NOT_OK(builder->FinishInternal(out_arr));
+    return Status::OK();
+  }
+
+  Status Call(const ArrayData& input) {
+    Status st = Status::OK();
+
+    if (skip_nulls || (input.GetNullCount() == 0 && !encountered_null)) {
+      VisitArrayValuesInline<ArgType>(
+          input,
+          [&](ArgValue v) {
+            accumulator =
+                Op::template Call<OutValue, ArgValue, ArgValue>(ctx, v, 
accumulator, &st);
+            builder->UnsafeAppend(accumulator);
+          },
+          [&]() { builder->UnsafeAppendNull(); });
+    } else {
+      auto nulls_start_idx = 0;

Review Comment:
   Should be `int64_t`.



##########
cpp/src/arrow/compute/kernels/vector_cumulative_ops_test.cc:
##########
@@ -0,0 +1,314 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <memory>
+#include <string>
+#include <vector>
+
+#include <gtest/gtest.h>
+
+#include "arrow/array.h"
+#include "arrow/chunked_array.h"
+#include "arrow/testing/gtest_util.h"
+#include "arrow/testing/util.h"
+#include "arrow/type.h"
+
+#include "arrow/array/builder_primitive.h"
+#include "arrow/compute/api.h"
+#include "arrow/compute/kernels/test_util.h"
+
+namespace arrow {
+namespace compute {
+
+TEST(TestCumulativeSum, Empty) {
+  CumulativeSumOptions options;
+  for (auto ty : NumericTypes()) {
+    auto empty_arr = ArrayFromJSON(ty, "[]");
+    auto empty_chunked = ChunkedArrayFromJSON(ty, {"[]"});
+    CheckVectorUnary("cumulative_sum", empty_arr, empty_arr, &options);
+    CheckVectorUnary("cumulative_sum_checked", empty_arr, empty_arr, &options);
+
+    CheckVectorUnary("cumulative_sum", empty_chunked, empty_chunked, &options);
+    CheckVectorUnary("cumulative_sum_checked", empty_chunked, empty_chunked, 
&options);
+  }
+}
+
+TEST(TestCumulativeSum, AllNulls) {
+  CumulativeSumOptions options;
+  for (auto ty : NumericTypes()) {
+    auto nulls_arr = ArrayFromJSON(ty, "[null, null, null]");
+    auto nulls_one_chunk = ChunkedArrayFromJSON(ty, {"[null, null, null]"});
+    auto nulls_three_chunks = ChunkedArrayFromJSON(ty, {"[null]", "[null]", 
"[null]"});
+    CheckVectorUnary("cumulative_sum", nulls_arr, nulls_arr, &options);
+    CheckVectorUnary("cumulative_sum_checked", nulls_arr, nulls_arr, &options);
+
+    CheckVectorUnary("cumulative_sum", nulls_one_chunk, nulls_one_chunk, 
&options);
+    CheckVectorUnary("cumulative_sum_checked", nulls_one_chunk, 
nulls_one_chunk,
+                     &options);
+
+    CheckVectorUnary("cumulative_sum", nulls_three_chunks, nulls_one_chunk, 
&options);
+    CheckVectorUnary("cumulative_sum_checked", nulls_three_chunks, 
nulls_one_chunk,
+                     &options);
+  }
+}
+
+TEST(TestCumulativeSum, ScalarInput) {
+  CumulativeSumOptions no_start;
+  CumulativeSumOptions with_start(10);
+  for (auto ty : NumericTypes()) {
+    CheckVectorUnary("cumulative_sum", ScalarFromJSON(ty, "10"),
+                     ArrayFromJSON(ty, "[10]"), &no_start);
+    CheckVectorUnary("cumulative_sum_checked", ScalarFromJSON(ty, "10"),
+                     ArrayFromJSON(ty, "[10]"), &no_start);
+
+    CheckVectorUnary("cumulative_sum", ScalarFromJSON(ty, "10"),
+                     ArrayFromJSON(ty, "[20]"), &with_start);
+    CheckVectorUnary("cumulative_sum_checked", ScalarFromJSON(ty, "10"),
+                     ArrayFromJSON(ty, "[20]"), &with_start);
+  }
+}
+
+using testing::HasSubstr;
+
+#define CHECK_CUMULATIVE_SUM_UNSIGNED_OVERFLOW(FUNC, TYPENAME, CTYPE)          
  \
+  {                                                                            
  \
+    CumulativeSumOptions pos_overflow(1);                                      
  \
+    auto max = std::numeric_limits<CTYPE>::max();                              
  \
+    TYPENAME##Builder builder;                                                 
  \
+    std::shared_ptr<Array> array;                                              
  \
+    ASSERT_OK(builder.Append(max));                                            
  \
+    ASSERT_OK(builder.Finish(&array));                                         
  \
+    EXPECT_RAISES_WITH_MESSAGE_THAT(                                           
  \
+        Invalid, HasSubstr("overflow"),                                        
  \
+        CallFunction(FUNC, {TypeTraits<TYPENAME##Type>::ScalarType(max)},      
  \
+                     &pos_overflow));                                          
  \
+    EXPECT_RAISES_WITH_MESSAGE_THAT(Invalid, HasSubstr("overflow"),            
  \
+                                    CallFunction(FUNC, {array}, 
&pos_overflow)); \
+  }
+
+#define CHECK_CUMULATIVE_SUM_SIGNED_OVERFLOW(FUNC, TYPENAME, CTYPE)            
  \
+  CHECK_CUMULATIVE_SUM_UNSIGNED_OVERFLOW(FUNC, TYPENAME, CTYPE);               
  \
+  {                                                                            
  \
+    CumulativeSumOptions neg_overflow(-1);                                     
  \
+    auto min = std::numeric_limits<CTYPE>::lowest();                           
  \
+    TYPENAME##Builder builder;                                                 
  \
+    std::shared_ptr<Array> array;                                              
  \
+    ASSERT_OK(builder.Append(min));                                            
  \
+    ASSERT_OK(builder.Finish(&array));                                         
  \
+    EXPECT_RAISES_WITH_MESSAGE_THAT(                                           
  \
+        Invalid, HasSubstr("overflow"),                                        
  \
+        CallFunction(FUNC, {TypeTraits<TYPENAME##Type>::ScalarType(min)},      
  \
+                     &neg_overflow));                                          
  \
+    EXPECT_RAISES_WITH_MESSAGE_THAT(Invalid, HasSubstr("overflow"),            
  \
+                                    CallFunction(FUNC, {array}, 
&neg_overflow)); \
+  }
+
+TEST(TestCumulativeSum, IntegerOverflow) {
+  CHECK_CUMULATIVE_SUM_UNSIGNED_OVERFLOW("cumulative_sum_checked", UInt8, 
uint8_t);
+  CHECK_CUMULATIVE_SUM_UNSIGNED_OVERFLOW("cumulative_sum_checked", UInt16, 
uint16_t);
+  CHECK_CUMULATIVE_SUM_UNSIGNED_OVERFLOW("cumulative_sum_checked", UInt32, 
uint32_t);
+  CHECK_CUMULATIVE_SUM_UNSIGNED_OVERFLOW("cumulative_sum_checked", UInt64, 
uint64_t);
+  CHECK_CUMULATIVE_SUM_SIGNED_OVERFLOW("cumulative_sum_checked", Int8, int8_t);
+  CHECK_CUMULATIVE_SUM_SIGNED_OVERFLOW("cumulative_sum_checked", Int16, 
int16_t);
+  CHECK_CUMULATIVE_SUM_SIGNED_OVERFLOW("cumulative_sum_checked", Int32, 
int32_t);
+  CHECK_CUMULATIVE_SUM_SIGNED_OVERFLOW("cumulative_sum_checked", Int64, 
int64_t);

Review Comment:
   But also, you should check the results of "cumulative_sum" on these 
overflowing 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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to