You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@arrow.apache.org by ap...@apache.org on 2020/08/10 13:33:59 UTC

[arrow] branch master updated: ARROW-9402: [C++] Rework portable wrappers for checked integer arithmetic

This is an automated email from the ASF dual-hosted git repository.

apitrou pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/arrow.git


The following commit(s) were added to refs/heads/master by this push:
     new 811d8f6  ARROW-9402: [C++] Rework portable wrappers for checked integer arithmetic
811d8f6 is described below

commit 811d8f6c5ca5e5bbf70a4d8c93d4c7fe0a9b9847
Author: Antoine Pitrou <an...@python.org>
AuthorDate: Mon Aug 10 15:33:28 2020 +0200

    ARROW-9402: [C++] Rework portable wrappers for checked integer arithmetic
    
    Vendor relevant code from the portable-snippets library (~ public domain):
    https://github.com/nemequ/portable-snippets/tree/master/safe-math
    
    Also fix some bugs in checked arithmetic (null values had their value slots checked).
    Add compute scaffolding for stateful binary scalar functions.
    
    Closes #7784 from pitrou/ARROW-9402-overflow-arith
    
    Authored-by: Antoine Pitrou <an...@python.org>
    Signed-off-by: Antoine Pitrou <an...@python.org>
---
 LICENSE.txt                                        |   16 +
 cpp/src/arrow/array/concatenate.cc                 |    2 +-
 cpp/src/arrow/array/data.cc                        |    2 +-
 cpp/src/arrow/array/validate.cc                    |   16 +-
 cpp/src/arrow/buffer.cc                            |    2 +-
 cpp/src/arrow/compute/kernels/codegen_internal.h   |  353 +++++--
 cpp/src/arrow/compute/kernels/scalar_arithmetic.cc |  148 ++-
 .../compute/kernels/scalar_arithmetic_benchmark.cc |   57 +-
 .../compute/kernels/scalar_arithmetic_test.cc      |  280 +++--
 .../arrow/compute/kernels/scalar_cast_boolean.cc   |    8 +-
 .../arrow/compute/kernels/scalar_cast_numeric.cc   |   50 +-
 .../arrow/compute/kernels/scalar_cast_temporal.cc  |    6 +-
 cpp/src/arrow/compute/kernels/scalar_nested.cc     |    4 +-
 cpp/src/arrow/compute/kernels/scalar_string.cc     |    6 +-
 cpp/src/arrow/compute/kernels/test_util.cc         |  113 ++-
 cpp/src/arrow/compute/kernels/test_util.h          |   10 +
 cpp/src/arrow/dataset/filter.cc                    |    5 +-
 cpp/src/arrow/pretty_print.cc                      |    2 +-
 cpp/src/arrow/scalar.h                             |   16 +-
 cpp/src/arrow/util/basic_decimal.cc                |    2 +-
 cpp/src/arrow/util/bit_block_counter.cc            |   23 +-
 cpp/src/arrow/util/bit_block_counter.h             |  186 +++-
 cpp/src/arrow/util/bit_block_counter_test.cc       |   92 ++
 cpp/src/arrow/util/decimal.cc                      |    2 +-
 cpp/src/arrow/util/int_util.h                      |   94 --
 cpp/src/arrow/util/int_util_internal.h             |  124 +++
 cpp/src/arrow/util/int_util_test.cc                |    1 +
 cpp/src/arrow/util/windows_fixup.h                 |   11 +
 cpp/src/arrow/vendored/portable-snippets/README.md |   10 +
 .../arrow/vendored/portable-snippets/safe-math.h   | 1071 ++++++++++++++++++++
 cpp/src/arrow/visitor_inline.h                     |   90 +-
 cpp/src/parquet/arrow/reader_internal.cc           |    4 +-
 cpp/src/parquet/column_reader.cc                   |   24 +-
 cpp/src/parquet/types.h                            |    1 -
 34 files changed, 2285 insertions(+), 546 deletions(-)

diff --git a/LICENSE.txt b/LICENSE.txt
index 673b146..e79841d 100644
--- a/LICENSE.txt
+++ b/LICENSE.txt
@@ -2207,3 +2207,19 @@ AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
 LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
 OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
 SOFTWARE.
+
+--------------------------------------------------------------------------------
+
+The files in cpp/src/arrow/vendored/portable-snippets/ contain code from
+
+https://github.com/nemequ/portable-snippets
+
+and have the following copyright notice:
+
+Each source file contains a preamble explaining the license situation
+for that file, which takes priority over this file.  With the
+exception of some code pulled in from other repositories (such as
+µnit, an MIT-licensed project which is used for testing), the code is
+public domain, released using the CC0 1.0 Universal dedication (*).
+
+(*) https://creativecommons.org/publicdomain/zero/1.0/legalcode
diff --git a/cpp/src/arrow/array/concatenate.cc b/cpp/src/arrow/array/concatenate.cc
index 923bf30..8456154 100644
--- a/cpp/src/arrow/array/concatenate.cc
+++ b/cpp/src/arrow/array/concatenate.cc
@@ -36,7 +36,7 @@
 #include "arrow/util/bit_util.h"
 #include "arrow/util/bitmap_ops.h"
 #include "arrow/util/checked_cast.h"
-#include "arrow/util/int_util.h"
+#include "arrow/util/int_util_internal.h"
 #include "arrow/util/logging.h"
 #include "arrow/visitor_inline.h"
 
diff --git a/cpp/src/arrow/array/data.cc b/cpp/src/arrow/array/data.cc
index 6af1c44..7bfb395 100644
--- a/cpp/src/arrow/array/data.cc
+++ b/cpp/src/arrow/array/data.cc
@@ -29,7 +29,7 @@
 #include "arrow/status.h"
 #include "arrow/type.h"
 #include "arrow/util/bitmap_ops.h"
-#include "arrow/util/int_util.h"
+#include "arrow/util/int_util_internal.h"
 #include "arrow/util/logging.h"
 #include "arrow/util/macros.h"
 
diff --git a/cpp/src/arrow/array/validate.cc b/cpp/src/arrow/array/validate.cc
index 8fb8b59..4ba6a34 100644
--- a/cpp/src/arrow/array/validate.cc
+++ b/cpp/src/arrow/array/validate.cc
@@ -26,7 +26,7 @@
 #include "arrow/type_traits.h"
 #include "arrow/util/bit_util.h"
 #include "arrow/util/checked_cast.h"
-#include "arrow/util/int_util.h"
+#include "arrow/util/int_util_internal.h"
 #include "arrow/util/logging.h"
 #include "arrow/visitor_inline.h"
 
@@ -98,8 +98,9 @@ struct ValidateArrayVisitor {
     if (value_size < 0) {
       return Status::Invalid("FixedSizeListArray has negative value size ", value_size);
     }
-    if (HasPositiveMultiplyOverflow(len, value_size) ||
-        array.values()->length() != len * value_size) {
+    int64_t expected_values_length = -1;
+    if (MultiplyWithOverflow(len, value_size, &expected_values_length) ||
+        array.values()->length() != expected_values_length) {
       return Status::Invalid("Values Length (", array.values()->length(),
                              ") is not equal to the length (", len,
                              ") multiplied by the value size (", value_size, ")");
@@ -329,7 +330,8 @@ Status ValidateArray(const Array& array) {
                            type.ToString(), ", got ", data.buffers.size());
   }
   // This check is required to avoid addition overflow below
-  if (HasPositiveAdditionOverflow(array.length(), array.offset())) {
+  int64_t length_plus_offset = -1;
+  if (AddWithOverflow(array.length(), array.offset(), &length_plus_offset)) {
     return Status::Invalid("Array of type ", type.ToString(),
                            " has impossibly large length and offset");
   }
@@ -343,15 +345,13 @@ Status ValidateArray(const Array& array) {
     int64_t min_buffer_size = -1;
     switch (spec.kind) {
       case DataTypeLayout::BITMAP:
-        min_buffer_size = BitUtil::BytesForBits(array.length() + array.offset());
+        min_buffer_size = BitUtil::BytesForBits(length_plus_offset);
         break;
       case DataTypeLayout::FIXED_WIDTH:
-        if (HasPositiveMultiplyOverflow(array.length() + array.offset(),
-                                        spec.byte_width)) {
+        if (MultiplyWithOverflow(length_plus_offset, spec.byte_width, &min_buffer_size)) {
           return Status::Invalid("Array of type ", type.ToString(),
                                  " has impossibly large length and offset");
         }
-        min_buffer_size = spec.byte_width * (array.length() + array.offset());
         break;
       case DataTypeLayout::ALWAYS_NULL:
         // XXX Should we raise on non-null buffer?
diff --git a/cpp/src/arrow/buffer.cc b/cpp/src/arrow/buffer.cc
index 2614cd1..6aac70e 100644
--- a/cpp/src/arrow/buffer.cc
+++ b/cpp/src/arrow/buffer.cc
@@ -25,7 +25,7 @@
 #include "arrow/result.h"
 #include "arrow/status.h"
 #include "arrow/util/bit_util.h"
-#include "arrow/util/int_util.h"
+#include "arrow/util/int_util_internal.h"
 #include "arrow/util/logging.h"
 #include "arrow/util/string.h"
 
diff --git a/cpp/src/arrow/compute/kernels/codegen_internal.h b/cpp/src/arrow/compute/kernels/codegen_internal.h
index 4d8918a..a4e11fe 100644
--- a/cpp/src/arrow/compute/kernels/codegen_internal.h
+++ b/cpp/src/arrow/compute/kernels/codegen_internal.h
@@ -35,6 +35,7 @@
 #include "arrow/status.h"
 #include "arrow/type.h"
 #include "arrow/type_traits.h"
+#include "arrow/util/bit_block_counter.h"
 #include "arrow/util/bit_util.h"
 #include "arrow/util/bitmap_generate.h"
 #include "arrow/util/bitmap_reader.h"
@@ -50,10 +51,14 @@
 
 namespace arrow {
 
+using internal::BinaryBitBlockCounter;
+using internal::BitBlockCount;
 using internal::BitmapReader;
 using internal::checked_cast;
 using internal::FirstTimeBitmapWriter;
 using internal::GenerateBitsUnrolled;
+using internal::VisitBitBlocksVoid;
+using internal::VisitTwoBitBlocksVoid;
 
 namespace compute {
 namespace internal {
@@ -122,12 +127,65 @@ struct OptionsWrapper : public KernelState {
 };
 
 // ----------------------------------------------------------------------
+// Input and output value type definitions
+
+template <typename Type, typename Enable = void>
+struct GetViewType;
+
+template <typename Type>
+struct GetViewType<Type, enable_if_has_c_type<Type>> {
+  using T = typename Type::c_type;
+  using PhysicalType = T;
+
+  static T LogicalValue(PhysicalType value) { return value; }
+};
+
+template <typename Type>
+struct GetViewType<Type, enable_if_t<is_base_binary_type<Type>::value ||
+                                     is_fixed_size_binary_type<Type>::value>> {
+  using T = util::string_view;
+  using PhysicalType = T;
+
+  static T LogicalValue(PhysicalType value) { return value; }
+};
+
+template <>
+struct GetViewType<Decimal128Type> {
+  using T = Decimal128;
+  using PhysicalType = util::string_view;
+
+  static T LogicalValue(PhysicalType value) {
+    return Decimal128(reinterpret_cast<const uint8_t*>(value.data()));
+  }
+};
+
+template <typename Type, typename Enable = void>
+struct GetOutputType;
+
+template <typename Type>
+struct GetOutputType<Type, enable_if_has_c_type<Type>> {
+  using T = typename Type::c_type;
+};
+
+template <typename Type>
+struct GetOutputType<Type, enable_if_t<is_string_like_type<Type>::value>> {
+  using T = std::string;
+};
+
+template <>
+struct GetOutputType<Decimal128Type> {
+  using T = Decimal128;
+};
+
+// ----------------------------------------------------------------------
 // Iteration / value access utilities
 
 template <typename T, typename R = void>
 using enable_if_has_c_type_not_boolean =
     enable_if_t<has_c_type<T>::value && !is_boolean_type<T>::value, R>;
 
+// Iterator over various input array types, yielding a GetViewType<Type>
+
 template <typename Type, typename Enable = void>
 struct ArrayIterator;
 
@@ -135,6 +193,7 @@ template <typename Type>
 struct ArrayIterator<Type, enable_if_has_c_type_not_boolean<Type>> {
   using T = typename Type::c_type;
   const T* values;
+
   explicit ArrayIterator(const ArrayData& data) : values(data.GetValues<T>(1)) {}
   T operator()() { return *values++; }
 };
@@ -142,6 +201,7 @@ struct ArrayIterator<Type, enable_if_has_c_type_not_boolean<Type>> {
 template <typename Type>
 struct ArrayIterator<Type, enable_if_boolean<Type>> {
   BitmapReader reader;
+
   explicit ArrayIterator(const ArrayData& data)
       : reader(data.buffers[1]->data(), data.offset, data.length) {}
   bool operator()() {
@@ -159,6 +219,7 @@ struct ArrayIterator<Type, enable_if_base_binary<Type>> {
   offset_type cur_offset;
   const char* data;
   int64_t position;
+
   explicit ArrayIterator(const ArrayData& arr)
       : arr(arr),
         offsets(reinterpret_cast<const offset_type*>(arr.buffers[1]->data()) +
@@ -168,13 +229,34 @@ struct ArrayIterator<Type, enable_if_base_binary<Type>> {
         position(0) {}
 
   util::string_view operator()() {
-    offset_type next_offset = offsets[position++ + 1];
+    offset_type next_offset = offsets[++position];
     auto result = util::string_view(data + cur_offset, next_offset - cur_offset);
     cur_offset = next_offset;
     return result;
   }
 };
 
+// Iterator over various output array types, taking a GetOutputType<Type>
+
+template <typename Type, typename Enable = void>
+struct OutputArrayWriter;
+
+template <typename Type>
+struct OutputArrayWriter<Type, enable_if_has_c_type_not_boolean<Type>> {
+  using T = typename Type::c_type;
+  T* values;
+
+  explicit OutputArrayWriter(ArrayData* data) : values(data->GetMutableValues<T>(1)) {}
+
+  void Write(T value) { *values++ = value; }
+
+  // Note that this doesn't write the null bitmap, which should be consistent
+  // with Write / WriteNull calls
+  void WriteNull() { *values++ = T{}; }
+};
+
+// (Un)box Scalar to / from C++ value
+
 template <typename Type, typename Enable = void>
 struct UnboxScalar;
 
@@ -202,43 +284,6 @@ struct UnboxScalar<Decimal128Type> {
 };
 
 template <typename Type, typename Enable = void>
-struct GetViewType;
-
-template <typename Type>
-struct GetViewType<Type, enable_if_has_c_type<Type>> {
-  using T = typename Type::c_type;
-};
-
-template <typename Type>
-struct GetViewType<Type, enable_if_t<is_base_binary_type<Type>::value ||
-                                     is_fixed_size_binary_type<Type>::value>> {
-  using T = util::string_view;
-};
-
-template <>
-struct GetViewType<Decimal128Type> {
-  using T = Decimal128;
-};
-
-template <typename Type, typename Enable = void>
-struct GetOutputType;
-
-template <typename Type>
-struct GetOutputType<Type, enable_if_has_c_type<Type>> {
-  using T = typename Type::c_type;
-};
-
-template <typename Type>
-struct GetOutputType<Type, enable_if_t<is_string_like_type<Type>::value>> {
-  using T = std::string;
-};
-
-template <>
-struct GetOutputType<Decimal128Type> {
-  using T = Decimal128;
-};
-
-template <typename Type, typename Enable = void>
 struct BoxScalar;
 
 template <typename Type>
@@ -264,6 +309,41 @@ struct BoxScalar<Decimal128Type> {
   static void Box(T val, Scalar* out) { checked_cast<ScalarType*>(out)->value = val; }
 };
 
+// A VisitArrayDataInline variant that calls its visitor function with logical
+// values, such as Decimal128 rather than util::string_view.
+
+template <typename T, typename VisitFunc, typename NullFunc>
+static void VisitArrayValuesInline(const ArrayData& arr, VisitFunc&& valid_func,
+                                   NullFunc&& null_func) {
+  VisitArrayDataInline<T>(
+      arr,
+      [&](typename GetViewType<T>::PhysicalType v) {
+        valid_func(GetViewType<T>::LogicalValue(std::move(v)));
+      },
+      std::forward<NullFunc>(null_func));
+}
+
+// Like VisitArrayValuesInline, but for binary functions.
+
+template <typename Arg0Type, typename Arg1Type, typename VisitFunc, typename NullFunc>
+static void VisitTwoArrayValuesInline(const ArrayData& arr0, const ArrayData& arr1,
+                                      VisitFunc&& valid_func, NullFunc&& null_func) {
+  ArrayIterator<Arg0Type> arr0_it(arr0);
+  ArrayIterator<Arg1Type> arr1_it(arr1);
+
+  auto visit_valid = [&](int64_t i) {
+    valid_func(GetViewType<Arg0Type>::LogicalValue(arr0_it()),
+               GetViewType<Arg1Type>::LogicalValue(arr1_it()));
+  };
+  auto visit_null = [&]() {
+    arr0_it();
+    arr1_it();
+    null_func();
+  };
+  VisitTwoBitBlocksVoid(arr0.buffers[0], arr0.offset, arr1.buffers[0], arr1.offset,
+                        arr0.length, std::move(visit_valid), std::move(visit_null));
+}
+
 // ----------------------------------------------------------------------
 // Reusable type resolvers
 
@@ -406,26 +486,27 @@ struct OutputAdapter<Type, enable_if_base_binary<Type>> {
 // The "Op" functor should have the form
 //
 // struct Op {
-//   template <typename OUT, typename ARG0>
-//   static OUT Call(KernelContext* ctx, ARG0 val) {
+//   template <typename OutValue, typename Arg0Value>
+//   static OutValue Call(KernelContext* ctx, Arg0Value val) {
 //     // implementation
 //   }
 // };
 template <typename OutType, typename Arg0Type, typename Op>
 struct ScalarUnary {
-  using OUT = typename GetOutputType<OutType>::T;
-  using ARG0 = typename GetViewType<Arg0Type>::T;
+  using OutValue = typename GetOutputType<OutType>::T;
+  using Arg0Value = typename GetViewType<Arg0Type>::T;
 
   static void Array(KernelContext* ctx, const ArrayData& arg0, Datum* out) {
     ArrayIterator<Arg0Type> arg0_it(arg0);
-    OutputAdapter<OutType>::Write(
-        ctx, out, [&]() -> OUT { return Op::template Call<OUT, ARG0>(ctx, arg0_it()); });
+    OutputAdapter<OutType>::Write(ctx, out, [&]() -> OutValue {
+      return Op::template Call<OutValue, Arg0Value>(ctx, arg0_it());
+    });
   }
 
   static void Scalar(KernelContext* ctx, const Scalar& arg0, Datum* out) {
     if (arg0.is_valid) {
-      ARG0 arg0_val = UnboxScalar<Arg0Type>::Unbox(arg0);
-      BoxScalar<OutType>::Box(Op::template Call<OUT, ARG0>(ctx, arg0_val),
+      Arg0Value arg0_val = UnboxScalar<Arg0Type>::Unbox(arg0);
+      BoxScalar<OutType>::Box(Op::template Call<OutValue, Arg0Value>(ctx, arg0_val),
                               out->scalar().get());
     } else {
       out->value = MakeNullScalar(arg0.type);
@@ -441,37 +522,13 @@ struct ScalarUnary {
   }
 };
 
-// A VisitArrayDataInline variant that passes a Decimal128 value,
-// not util::string_view, for decimal128 arrays,
-
-template <typename T, typename VisitFunc, typename NullFunc>
-static typename std::enable_if<!std::is_same<T, Decimal128Type>::value, void>::type
-VisitArrayValuesInline(const ArrayData& arr, VisitFunc&& valid_func,
-                       NullFunc&& null_func) {
-  VisitArrayDataInline<T>(arr, std::forward<VisitFunc>(valid_func),
-                          std::forward<NullFunc>(null_func));
-}
-
-template <typename T, typename VisitFunc, typename NullFunc>
-static typename std::enable_if<std::is_same<T, Decimal128Type>::value, void>::type
-VisitArrayValuesInline(const ArrayData& arr, VisitFunc&& valid_func,
-                       NullFunc&& null_func) {
-  VisitArrayDataInline<T>(
-      arr,
-      [&](util::string_view v) {
-        const auto dec_value = Decimal128(reinterpret_cast<const uint8_t*>(v.data()));
-        valid_func(dec_value);
-      },
-      std::forward<NullFunc>(null_func));
-}
-
 // An alternative to ScalarUnary that Applies a scalar operation with state on
 // only the not-null values of a single array
 template <typename OutType, typename Arg0Type, typename Op>
 struct ScalarUnaryNotNullStateful {
   using ThisType = ScalarUnaryNotNullStateful<OutType, Arg0Type, Op>;
-  using OUT = typename GetOutputType<OutType>::T;
-  using ARG0 = typename GetViewType<Arg0Type>::T;
+  using OutValue = typename GetOutputType<OutType>::T;
+  using Arg0Value = typename GetViewType<Arg0Type>::T;
 
   Op op;
   explicit ScalarUnaryNotNullStateful(Op op) : op(std::move(op)) {}
@@ -493,10 +550,12 @@ struct ScalarUnaryNotNullStateful {
     static void Exec(const ThisType& functor, KernelContext* ctx, const ArrayData& arg0,
                      Datum* out) {
       ArrayData* out_arr = out->mutable_array();
-      auto out_data = out_arr->GetMutableValues<OUT>(1);
+      auto out_data = out_arr->GetMutableValues<OutValue>(1);
       VisitArrayValuesInline<Arg0Type>(
           arg0,
-          [&](ARG0 v) { *out_data++ = functor.op.template Call<OUT, ARG0>(ctx, v); },
+          [&](Arg0Value v) {
+            *out_data++ = functor.op.template Call<OutValue, Arg0Value>(ctx, v);
+          },
           [&]() {
             // null
             ++out_data;
@@ -515,7 +574,7 @@ struct ScalarUnaryNotNullStateful {
       typename TypeTraits<Type>::BuilderType builder;
       VisitArrayValuesInline<Arg0Type>(
           arg0,
-          [&](ARG0 v) {
+          [&](Arg0Value v) {
             KERNEL_RETURN_IF_ERROR(ctx, builder.Append(functor.op.Call(ctx, v)));
           },
           [&]() { KERNEL_RETURN_IF_ERROR(ctx, builder.AppendNull()); });
@@ -536,8 +595,8 @@ struct ScalarUnaryNotNullStateful {
                                        out_arr->offset, out_arr->length);
       VisitArrayValuesInline<Arg0Type>(
           arg0,
-          [&](ARG0 v) {
-            if (functor.op.template Call<OUT, ARG0>(ctx, v)) {
+          [&](Arg0Value v) {
+            if (functor.op.template Call<OutValue, Arg0Value>(ctx, v)) {
               out_writer.Set();
             }
             out_writer.Next();
@@ -559,8 +618,8 @@ struct ScalarUnaryNotNullStateful {
       auto out_data = out_arr->GetMutableValues<uint8_t>(1);
       VisitArrayValuesInline<Arg0Type>(
           arg0,
-          [&](ARG0 v) {
-            functor.op.template Call<OUT, ARG0>(ctx, v).ToBytes(out_data);
+          [&](Arg0Value v) {
+            functor.op.template Call<OutValue, Arg0Value>(ctx, v).ToBytes(out_data);
             out_data += 16;
           },
           [&]() { out_data += 16; });
@@ -569,8 +628,8 @@ struct ScalarUnaryNotNullStateful {
 
   void Scalar(KernelContext* ctx, const Scalar& arg0, Datum* out) {
     if (arg0.is_valid) {
-      ARG0 arg0_val = UnboxScalar<Arg0Type>::Unbox(arg0);
-      BoxScalar<OutType>::Box(this->op.template Call<OUT, ARG0>(ctx, arg0_val),
+      Arg0Value arg0_val = UnboxScalar<Arg0Type>::Unbox(arg0);
+      BoxScalar<OutType>::Box(this->op.template Call<OutValue, Arg0Value>(ctx, arg0_val),
                               out->scalar().get());
     } else {
       out->value = MakeNullScalar(arg0.type);
@@ -591,8 +650,8 @@ struct ScalarUnaryNotNullStateful {
 // operator requires some initialization use ScalarUnaryNotNullStateful
 template <typename OutType, typename Arg0Type, typename Op>
 struct ScalarUnaryNotNull {
-  using OUT = typename GetOutputType<OutType>::T;
-  using ARG0 = typename GetViewType<Arg0Type>::T;
+  using OutValue = typename GetOutputType<OutType>::T;
+  using Arg0Value = typename GetViewType<Arg0Type>::T;
 
   static void Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
     // Seed kernel with dummy state
@@ -612,39 +671,42 @@ struct ScalarUnaryNotNull {
 // The "Op" functor should have the form
 //
 // struct Op {
-//   template <typename OUT, typename ARG0, typename ARG1>
-//   static OUT Call(KernelContext* ctx, ARG0 arg0, ARG1 arg1) {
+//   template <typename OutValue, typename Arg0Value, typename Arg1Value>
+//   static OutValue Call(KernelContext* ctx, Arg0Value arg0, Arg1Value arg1) {
 //     // implementation
 //   }
 // };
 template <typename OutType, typename Arg0Type, typename Arg1Type, typename Op>
 struct ScalarBinary {
-  using OUT = typename GetOutputType<OutType>::T;
-  using ARG0 = typename GetViewType<Arg0Type>::T;
-  using ARG1 = typename GetViewType<Arg1Type>::T;
+  using OutValue = typename GetOutputType<OutType>::T;
+  using Arg0Value = typename GetViewType<Arg0Type>::T;
+  using Arg1Value = typename GetViewType<Arg1Type>::T;
 
   static void ArrayArray(KernelContext* ctx, const ArrayData& arg0, const ArrayData& arg1,
                          Datum* out) {
     ArrayIterator<Arg0Type> arg0_it(arg0);
     ArrayIterator<Arg1Type> arg1_it(arg1);
-    OutputAdapter<OutType>::Write(
-        ctx, out, [&]() -> OUT { return Op::template Call(ctx, arg0_it(), arg1_it()); });
+    OutputAdapter<OutType>::Write(ctx, out, [&]() -> OutValue {
+      return Op::template Call(ctx, arg0_it(), arg1_it());
+    });
   }
 
   static void ArrayScalar(KernelContext* ctx, const ArrayData& arg0, const Scalar& arg1,
                           Datum* out) {
     ArrayIterator<Arg0Type> arg0_it(arg0);
     auto arg1_val = UnboxScalar<Arg1Type>::Unbox(arg1);
-    OutputAdapter<OutType>::Write(
-        ctx, out, [&]() -> OUT { return Op::template Call(ctx, arg0_it(), arg1_val); });
+    OutputAdapter<OutType>::Write(ctx, out, [&]() -> OutValue {
+      return Op::template Call(ctx, arg0_it(), arg1_val);
+    });
   }
 
   static void ScalarArray(KernelContext* ctx, const Scalar& arg0, const ArrayData& arg1,
                           Datum* out) {
     auto arg0_val = UnboxScalar<Arg0Type>::Unbox(arg0);
     ArrayIterator<Arg1Type> arg1_it(arg1);
-    OutputAdapter<OutType>::Write(
-        ctx, out, [&]() -> OUT { return Op::template Call(ctx, arg0_val, arg1_it()); });
+    OutputAdapter<OutType>::Write(ctx, out, [&]() -> OutValue {
+      return Op::template Call(ctx, arg0_val, arg1_it());
+    });
   }
 
   static void ScalarScalar(KernelContext* ctx, const Scalar& arg0, const Scalar& arg1,
@@ -675,11 +737,114 @@ struct ScalarBinary {
   }
 };
 
+// An alternative to ScalarBinary that Applies a scalar operation with state on
+// only the value pairs which are not-null in both arrays
+template <typename OutType, typename Arg0Type, typename Arg1Type, typename Op>
+struct ScalarBinaryNotNullStateful {
+  using ThisType = ScalarBinaryNotNullStateful<OutType, Arg0Type, Arg1Type, Op>;
+  using OutValue = typename GetOutputType<OutType>::T;
+  using Arg0Value = typename GetViewType<Arg0Type>::T;
+  using Arg1Value = typename GetViewType<Arg1Type>::T;
+
+  Op op;
+  explicit ScalarBinaryNotNullStateful(Op op) : op(std::move(op)) {}
+
+  // NOTE: In ArrayExec<Type>, Type is really OutputType
+
+  void ArrayArray(KernelContext* ctx, const ArrayData& arg0, const ArrayData& arg1,
+                  Datum* out) {
+    OutputArrayWriter<OutType> writer(out->mutable_array());
+    VisitTwoArrayValuesInline<Arg0Type, Arg1Type>(
+        arg0, arg1,
+        [&](Arg0Value u, Arg1Value v) {
+          writer.Write(op.template Call<OutValue, Arg0Value, Arg1Value>(ctx, u, v));
+        },
+        [&]() { writer.WriteNull(); });
+  }
+
+  void ArrayScalar(KernelContext* ctx, const ArrayData& arg0, const Scalar& arg1,
+                   Datum* out) {
+    OutputArrayWriter<OutType> writer(out->mutable_array());
+    if (arg1.is_valid) {
+      const auto arg1_val = UnboxScalar<Arg1Type>::Unbox(arg1);
+      VisitArrayValuesInline<Arg0Type>(
+          arg0,
+          [&](Arg0Value u) {
+            writer.Write(
+                op.template Call<OutValue, Arg0Value, Arg1Value>(ctx, u, arg1_val));
+          },
+          [&]() { writer.WriteNull(); });
+    }
+  }
+
+  void ScalarArray(KernelContext* ctx, const Scalar& arg0, const ArrayData& arg1,
+                   Datum* out) {
+    OutputArrayWriter<OutType> writer(out->mutable_array());
+    if (arg0.is_valid) {
+      const auto arg0_val = UnboxScalar<Arg0Type>::Unbox(arg0);
+      VisitArrayValuesInline<Arg1Type>(
+          arg1,
+          [&](Arg1Value v) {
+            writer.Write(
+                op.template Call<OutValue, Arg0Value, Arg1Value>(ctx, arg0_val, v));
+          },
+          [&]() { writer.WriteNull(); });
+    }
+  }
+
+  void ScalarScalar(KernelContext* ctx, const Scalar& arg0, const Scalar& arg1,
+                    Datum* out) {
+    if (arg0.is_valid && arg1.is_valid) {
+      const auto arg0_val = UnboxScalar<Arg0Type>::Unbox(arg0);
+      const auto arg1_val = UnboxScalar<Arg1Type>::Unbox(arg1);
+      BoxScalar<OutType>::Box(
+          op.template Call<OutValue, Arg0Value, Arg1Value>(ctx, arg0_val, arg1_val),
+          out->scalar().get());
+    }
+  }
+
+  void Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    if (batch[0].kind() == Datum::ARRAY) {
+      if (batch[1].kind() == Datum::ARRAY) {
+        return ArrayArray(ctx, *batch[0].array(), *batch[1].array(), out);
+      } else {
+        return ArrayScalar(ctx, *batch[0].array(), *batch[1].scalar(), out);
+      }
+    } else {
+      if (batch[1].kind() == Datum::ARRAY) {
+        return ScalarArray(ctx, *batch[0].scalar(), *batch[1].array(), out);
+      } else {
+        return ScalarScalar(ctx, *batch[0].scalar(), *batch[1].scalar(), out);
+      }
+    }
+  }
+};
+
+// An alternative to ScalarBinary that Applies a scalar operation on only
+// the value pairs which are not-null in both arrays.
+// The operator is not stateful; if the operator requires some initialization
+// use ScalarBinaryNotNullStateful.
+template <typename OutType, typename Arg0Type, typename Arg1Type, typename Op>
+struct ScalarBinaryNotNull {
+  using OutValue = typename GetOutputType<OutType>::T;
+  using Arg0Value = typename GetViewType<Arg0Type>::T;
+  using Arg1Value = typename GetViewType<Arg1Type>::T;
+
+  static void Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    // Seed kernel with dummy state
+    ScalarBinaryNotNullStateful<OutType, Arg0Type, Arg1Type, Op> kernel({});
+    return kernel.Exec(ctx, batch, out);
+  }
+};
+
 // A kernel exec generator for binary kernels where both input types are the
 // same
 template <typename OutType, typename ArgType, typename Op>
 using ScalarBinaryEqualTypes = ScalarBinary<OutType, ArgType, ArgType, Op>;
 
+template <typename OutType, typename ArgType, typename Op>
+using ScalarBinaryNotNullEqualTypes = ScalarBinaryNotNull<OutType, ArgType, ArgType, Op>;
+
 }  // namespace applicator
 
 // ----------------------------------------------------------------------
diff --git a/cpp/src/arrow/compute/kernels/scalar_arithmetic.cc b/cpp/src/arrow/compute/kernels/scalar_arithmetic.cc
index 1f0cd37..299f652 100644
--- a/cpp/src/arrow/compute/kernels/scalar_arithmetic.cc
+++ b/cpp/src/arrow/compute/kernels/scalar_arithmetic.cc
@@ -16,16 +16,21 @@
 // under the License.
 
 #include "arrow/compute/kernels/common.h"
-#include "arrow/util/int_util.h"
+#include "arrow/util/int_util_internal.h"
 #include "arrow/util/macros.h"
 
-#ifndef __has_builtin
-#define __has_builtin(x) 0
-#endif
-
 namespace arrow {
+
+using internal::AddWithOverflow;
+using internal::MultiplyWithOverflow;
+using internal::SubtractWithOverflow;
+
 namespace compute {
 namespace internal {
+
+using applicator::ScalarBinaryEqualTypes;
+using applicator::ScalarBinaryNotNullEqualTypes;
+
 namespace {
 
 template <typename T>
@@ -72,35 +77,19 @@ struct Add {
 };
 
 struct AddChecked {
-#if __has_builtin(__builtin_add_overflow)
-  template <typename T>
-  static enable_if_integer<T> Call(KernelContext* ctx, T left, T right) {
+  template <typename T, typename Arg0, typename Arg1>
+  enable_if_integer<T> Call(KernelContext* ctx, Arg0 left, Arg1 right) {
+    static_assert(std::is_same<T, Arg0>::value && std::is_same<T, Arg1>::value, "");
     T result;
-    if (ARROW_PREDICT_FALSE(__builtin_add_overflow(left, right, &result))) {
+    if (ARROW_PREDICT_FALSE(AddWithOverflow(left, right, &result))) {
       ctx->SetStatus(Status::Invalid("overflow"));
     }
     return result;
   }
-#else
-  template <typename T>
-  static enable_if_unsigned_integer<T> Call(KernelContext* ctx, T left, T right) {
-    if (ARROW_PREDICT_FALSE(arrow::internal::HasPositiveAdditionOverflow(left, right))) {
-      ctx->SetStatus(Status::Invalid("overflow"));
-    }
-    return left + right;
-  }
-
-  template <typename T>
-  static enable_if_signed_integer<T> Call(KernelContext* ctx, T left, T right) {
-    if (ARROW_PREDICT_FALSE(arrow::internal::HasSignedAdditionOverflow(left, right))) {
-      ctx->SetStatus(Status::Invalid("overflow"));
-    }
-    return left + right;
-  }
-#endif
 
-  template <typename T>
-  static constexpr enable_if_floating_point<T> Call(KernelContext*, T left, T right) {
+  template <typename T, typename Arg0, typename Arg1>
+  enable_if_floating_point<T> Call(KernelContext*, Arg0 left, Arg1 right) {
+    static_assert(std::is_same<T, Arg0>::value && std::is_same<T, Arg1>::value, "");
     return left + right;
   }
 };
@@ -123,36 +112,19 @@ struct Subtract {
 };
 
 struct SubtractChecked {
-#if __has_builtin(__builtin_sub_overflow)
-  template <typename T>
-  static enable_if_integer<T> Call(KernelContext* ctx, T left, T right) {
+  template <typename T, typename Arg0, typename Arg1>
+  enable_if_integer<T> Call(KernelContext* ctx, Arg0 left, Arg1 right) {
+    static_assert(std::is_same<T, Arg0>::value && std::is_same<T, Arg1>::value, "");
     T result;
-    if (ARROW_PREDICT_FALSE(__builtin_sub_overflow(left, right, &result))) {
+    if (ARROW_PREDICT_FALSE(SubtractWithOverflow(left, right, &result))) {
       ctx->SetStatus(Status::Invalid("overflow"));
     }
     return result;
   }
-#else
-  template <typename T>
-  static enable_if_unsigned_integer<T> Call(KernelContext* ctx, T left, T right) {
-    if (ARROW_PREDICT_FALSE(
-            arrow::internal::HasPositiveSubtractionOverflow(left, right))) {
-      ctx->SetStatus(Status::Invalid("overflow"));
-    }
-    return left - right;
-  }
 
-  template <typename T>
-  static enable_if_signed_integer<T> Call(KernelContext* ctx, T left, T right) {
-    if (ARROW_PREDICT_FALSE(arrow::internal::HasSignedSubtractionOverflow(left, right))) {
-      ctx->SetStatus(Status::Invalid("overflow"));
-    }
-    return left - right;
-  }
-#endif
-
-  template <typename T>
-  static constexpr enable_if_floating_point<T> Call(KernelContext*, T left, T right) {
+  template <typename T, typename Arg0, typename Arg1>
+  enable_if_floating_point<T> Call(KernelContext*, Arg0 left, Arg1 right) {
+    static_assert(std::is_same<T, Arg0>::value && std::is_same<T, Arg1>::value, "");
     return left - right;
   }
 };
@@ -197,59 +169,48 @@ struct Multiply {
 };
 
 struct MultiplyChecked {
-  template <typename T>
-  static enable_if_integer<T> Call(KernelContext* ctx, T left, T right) {
+  template <typename T, typename Arg0, typename Arg1>
+  enable_if_integer<T> Call(KernelContext* ctx, Arg0 left, Arg1 right) {
+    static_assert(std::is_same<T, Arg0>::value && std::is_same<T, Arg1>::value, "");
     T result;
-#if __has_builtin(__builtin_mul_overflow)
-    if (ARROW_PREDICT_FALSE(__builtin_mul_overflow(left, right, &result))) {
-      ctx->SetStatus(Status::Invalid("overflow"));
-    }
-#else
-    result = Multiply::Call(ctx, left, right);
-    if (left != 0 && ARROW_PREDICT_FALSE(result / left != right)) {
+    if (ARROW_PREDICT_FALSE(MultiplyWithOverflow(left, right, &result))) {
       ctx->SetStatus(Status::Invalid("overflow"));
     }
-#endif
     return result;
   }
 
-  template <typename T>
-  static constexpr enable_if_floating_point<T> Call(KernelContext*, T left, T right) {
+  template <typename T, typename Arg0, typename Arg1>
+  enable_if_floating_point<T> Call(KernelContext*, Arg0 left, Arg1 right) {
+    static_assert(std::is_same<T, Arg0>::value && std::is_same<T, Arg1>::value, "");
     return left * right;
   }
 };
 
-using applicator::ScalarBinaryEqualTypes;
-
 // Generate a kernel given an arithmetic functor
-//
-// To avoid undefined behaviour of signed integer overflow treat the signed
-// input argument values as unsigned then cast them to signed making them wrap
-// around.
-template <typename Op>
+template <template <typename... Args> class KernelGenerator, typename Op>
 ArrayKernelExec NumericEqualTypesBinary(detail::GetTypeId get_id) {
   switch (get_id.id) {
     case Type::INT8:
-      return ScalarBinaryEqualTypes<Int8Type, Int8Type, Op>::Exec;
+      return KernelGenerator<Int8Type, Int8Type, Op>::Exec;
     case Type::UINT8:
-      return ScalarBinaryEqualTypes<UInt8Type, UInt8Type, Op>::Exec;
+      return KernelGenerator<UInt8Type, UInt8Type, Op>::Exec;
     case Type::INT16:
-      return ScalarBinaryEqualTypes<Int16Type, Int16Type, Op>::Exec;
+      return KernelGenerator<Int16Type, Int16Type, Op>::Exec;
     case Type::UINT16:
-      return ScalarBinaryEqualTypes<UInt16Type, UInt16Type, Op>::Exec;
+      return KernelGenerator<UInt16Type, UInt16Type, Op>::Exec;
     case Type::INT32:
-      return ScalarBinaryEqualTypes<Int32Type, Int32Type, Op>::Exec;
+      return KernelGenerator<Int32Type, Int32Type, Op>::Exec;
     case Type::UINT32:
-      return ScalarBinaryEqualTypes<UInt32Type, UInt32Type, Op>::Exec;
+      return KernelGenerator<UInt32Type, UInt32Type, Op>::Exec;
     case Type::INT64:
     case Type::TIMESTAMP:
-      return ScalarBinaryEqualTypes<Int64Type, Int64Type, Op>::Exec;
+      return KernelGenerator<Int64Type, Int64Type, Op>::Exec;
     case Type::UINT64:
-      return ScalarBinaryEqualTypes<UInt64Type, UInt64Type, Op>::Exec;
+      return KernelGenerator<UInt64Type, UInt64Type, Op>::Exec;
     case Type::FLOAT:
-      return ScalarBinaryEqualTypes<FloatType, FloatType, Op>::Exec;
+      return KernelGenerator<FloatType, FloatType, Op>::Exec;
     case Type::DOUBLE:
-      return ScalarBinaryEqualTypes<DoubleType, DoubleType, Op>::Exec;
+      return KernelGenerator<DoubleType, DoubleType, Op>::Exec;
     default:
       DCHECK(false);
       return ExecFail;
@@ -260,7 +221,19 @@ template <typename Op>
 std::shared_ptr<ScalarFunction> MakeArithmeticFunction(std::string name) {
   auto func = std::make_shared<ScalarFunction>(name, Arity::Binary());
   for (const auto& ty : NumericTypes()) {
-    auto exec = NumericEqualTypesBinary<Op>(ty);
+    auto exec = NumericEqualTypesBinary<ScalarBinaryEqualTypes, Op>(ty);
+    DCHECK_OK(func->AddKernel({ty, ty}, ty, exec));
+  }
+  return func;
+}
+
+// Like MakeArithmeticFunction, but for arithmetic ops that need to run
+// only on non-null output.
+template <typename Op>
+std::shared_ptr<ScalarFunction> MakeArithmeticFunctionNotNull(std::string name) {
+  auto func = std::make_shared<ScalarFunction>(name, Arity::Binary());
+  for (const auto& ty : NumericTypes()) {
+    auto exec = NumericEqualTypesBinary<ScalarBinaryNotNullEqualTypes, Op>(ty);
     DCHECK_OK(func->AddKernel({ty, ty}, ty, exec));
   }
   return func;
@@ -274,7 +247,7 @@ void RegisterScalarArithmetic(FunctionRegistry* registry) {
   DCHECK_OK(registry->AddFunction(std::move(add)));
 
   // ----------------------------------------------------------------------
-  auto add_checked = MakeArithmeticFunction<AddChecked>("add_checked");
+  auto add_checked = MakeArithmeticFunctionNotNull<AddChecked>("add_checked");
   DCHECK_OK(registry->AddFunction(std::move(add_checked)));
 
   // ----------------------------------------------------------------------
@@ -284,14 +257,16 @@ void RegisterScalarArithmetic(FunctionRegistry* registry) {
   // Add subtract(timestamp, timestamp) -> duration
   for (auto unit : AllTimeUnits()) {
     InputType in_type(match::TimestampTypeUnit(unit));
-    auto exec = NumericEqualTypesBinary<Subtract>(Type::TIMESTAMP);
+    auto exec =
+        NumericEqualTypesBinary<ScalarBinaryEqualTypes, Subtract>(Type::TIMESTAMP);
     DCHECK_OK(subtract->AddKernel({in_type, in_type}, duration(unit), std::move(exec)));
   }
 
   DCHECK_OK(registry->AddFunction(std::move(subtract)));
 
   // ----------------------------------------------------------------------
-  auto subtract_checked = MakeArithmeticFunction<SubtractChecked>("subtract_checked");
+  auto subtract_checked =
+      MakeArithmeticFunctionNotNull<SubtractChecked>("subtract_checked");
   DCHECK_OK(registry->AddFunction(std::move(subtract_checked)));
 
   // ----------------------------------------------------------------------
@@ -299,7 +274,8 @@ void RegisterScalarArithmetic(FunctionRegistry* registry) {
   DCHECK_OK(registry->AddFunction(std::move(multiply)));
 
   // ----------------------------------------------------------------------
-  auto multiply_checked = MakeArithmeticFunction<MultiplyChecked>("multiply_checked");
+  auto multiply_checked =
+      MakeArithmeticFunctionNotNull<MultiplyChecked>("multiply_checked");
   DCHECK_OK(registry->AddFunction(std::move(multiply_checked)));
 }
 
diff --git a/cpp/src/arrow/compute/kernels/scalar_arithmetic_benchmark.cc b/cpp/src/arrow/compute/kernels/scalar_arithmetic_benchmark.cc
index cac5679..8dec794 100644
--- a/cpp/src/arrow/compute/kernels/scalar_arithmetic_benchmark.cc
+++ b/cpp/src/arrow/compute/kernels/scalar_arithmetic_benchmark.cc
@@ -33,21 +33,45 @@ constexpr auto kSeed = 0x94378165;
 using BinaryOp = Result<Datum>(const Datum&, const Datum&, ArithmeticOptions,
                                ExecContext*);
 
+// Add explicit overflow-checked shortcuts, for easy benchmark parametering.
+static Result<Datum> AddChecked(const Datum& left, const Datum& right,
+                                ArithmeticOptions options = ArithmeticOptions(),
+                                ExecContext* ctx = NULLPTR) {
+  options.check_overflow = true;
+  return Add(left, right, std::move(options), ctx);
+}
+
+static Result<Datum> SubtractChecked(const Datum& left, const Datum& right,
+                                     ArithmeticOptions options = ArithmeticOptions(),
+                                     ExecContext* ctx = NULLPTR) {
+  options.check_overflow = true;
+  return Subtract(left, right, std::move(options), ctx);
+}
+
+static Result<Datum> MultiplyChecked(const Datum& left, const Datum& right,
+                                     ArithmeticOptions options = ArithmeticOptions(),
+                                     ExecContext* ctx = NULLPTR) {
+  options.check_overflow = true;
+  return Multiply(left, right, std::move(options), ctx);
+}
+
 template <BinaryOp& Op, typename ArrowType, typename CType = typename ArrowType::c_type>
 static void ArrayScalarKernel(benchmark::State& state) {
   RegressionArgs args(state);
 
   const int64_t array_size = args.size / sizeof(CType);
-  auto min = std::numeric_limits<CType>::lowest();
-  auto max = std::numeric_limits<CType>::max();
+
+  // Choose values so as to avoid overflow on all ops and types
+  auto min = static_cast<CType>(6);
+  auto max = static_cast<CType>(min + 15);
+  Datum rhs(static_cast<CType>(6));
 
   auto rand = random::RandomArrayGenerator(kSeed);
   auto lhs = std::static_pointer_cast<NumericArray<ArrowType>>(
       rand.Numeric<ArrowType>(array_size, min, max, args.null_proportion));
 
-  Datum fifteen(CType(15));
   for (auto _ : state) {
-    ABORT_NOT_OK(Op(lhs, fifteen, ArithmeticOptions(), nullptr).status());
+    ABORT_NOT_OK(Op(lhs, rhs, ArithmeticOptions(), nullptr).status());
   }
   state.SetItemsProcessed(state.iterations() * array_size);
 }
@@ -56,9 +80,10 @@ template <BinaryOp& Op, typename ArrowType, typename CType = typename ArrowType:
 static void ArrayArrayKernel(benchmark::State& state) {
   RegressionArgs args(state);
 
+  // Choose values so as to avoid overflow on all ops
   const int64_t array_size = args.size / sizeof(CType);
-  auto min = std::numeric_limits<CType>::lowest();
-  auto max = std::numeric_limits<CType>::max();
+  auto min = static_cast<CType>(0);
+  auto max = static_cast<CType>(11);
 
   auto rand = random::RandomArrayGenerator(kSeed);
   auto lhs = std::static_pointer_cast<NumericArray<ArrowType>>(
@@ -92,6 +117,19 @@ void SetArgs(benchmark::internal::Benchmark* bench) {
   BENCHMARK_TEMPLATE(BENCHMARK, OP, FloatType)->Apply(SetArgs);  \
   BENCHMARK_TEMPLATE(BENCHMARK, OP, DoubleType)->Apply(SetArgs)
 
+// Checked floating-point variants of arithmetic operations are identical to
+// non-checked variants, so do not bother measuring them.
+
+#define DECLARE_ARITHMETIC_CHECKED_BENCHMARKS(BENCHMARK, OP)     \
+  BENCHMARK_TEMPLATE(BENCHMARK, OP, Int64Type)->Apply(SetArgs);  \
+  BENCHMARK_TEMPLATE(BENCHMARK, OP, Int32Type)->Apply(SetArgs);  \
+  BENCHMARK_TEMPLATE(BENCHMARK, OP, Int16Type)->Apply(SetArgs);  \
+  BENCHMARK_TEMPLATE(BENCHMARK, OP, Int8Type)->Apply(SetArgs);   \
+  BENCHMARK_TEMPLATE(BENCHMARK, OP, UInt64Type)->Apply(SetArgs); \
+  BENCHMARK_TEMPLATE(BENCHMARK, OP, UInt32Type)->Apply(SetArgs); \
+  BENCHMARK_TEMPLATE(BENCHMARK, OP, UInt16Type)->Apply(SetArgs); \
+  BENCHMARK_TEMPLATE(BENCHMARK, OP, UInt8Type)->Apply(SetArgs);
+
 DECLARE_ARITHMETIC_BENCHMARKS(ArrayArrayKernel, Add);
 DECLARE_ARITHMETIC_BENCHMARKS(ArrayScalarKernel, Add);
 DECLARE_ARITHMETIC_BENCHMARKS(ArrayArrayKernel, Subtract);
@@ -99,5 +137,12 @@ DECLARE_ARITHMETIC_BENCHMARKS(ArrayScalarKernel, Subtract);
 DECLARE_ARITHMETIC_BENCHMARKS(ArrayArrayKernel, Multiply);
 DECLARE_ARITHMETIC_BENCHMARKS(ArrayScalarKernel, Multiply);
 
+DECLARE_ARITHMETIC_CHECKED_BENCHMARKS(ArrayArrayKernel, AddChecked);
+DECLARE_ARITHMETIC_CHECKED_BENCHMARKS(ArrayScalarKernel, AddChecked);
+DECLARE_ARITHMETIC_CHECKED_BENCHMARKS(ArrayArrayKernel, SubtractChecked);
+DECLARE_ARITHMETIC_CHECKED_BENCHMARKS(ArrayScalarKernel, SubtractChecked);
+DECLARE_ARITHMETIC_CHECKED_BENCHMARKS(ArrayArrayKernel, MultiplyChecked);
+DECLARE_ARITHMETIC_CHECKED_BENCHMARKS(ArrayScalarKernel, MultiplyChecked);
+
 }  // namespace compute
 }  // namespace arrow
diff --git a/cpp/src/arrow/compute/kernels/scalar_arithmetic_test.cc b/cpp/src/arrow/compute/kernels/scalar_arithmetic_test.cc
index ceb4623..4593b79 100644
--- a/cpp/src/arrow/compute/kernels/scalar_arithmetic_test.cc
+++ b/cpp/src/arrow/compute/kernels/scalar_arithmetic_test.cc
@@ -24,11 +24,13 @@
 #include <gtest/gtest.h>
 
 #include "arrow/array.h"
+#include "arrow/buffer.h"
 #include "arrow/compute/api.h"
 #include "arrow/compute/kernels/codegen_internal.h"
 #include "arrow/compute/kernels/test_util.h"
 #include "arrow/type.h"
 #include "arrow/type_traits.h"
+#include "arrow/util/bit_util.h"
 #include "arrow/util/checked_cast.h"
 #include "arrow/util/string.h"
 
@@ -39,9 +41,23 @@
 namespace arrow {
 namespace compute {
 
-template <typename ArrowType>
+std::shared_ptr<Array> TweakValidityBit(const std::shared_ptr<Array>& array,
+                                        int64_t index, bool validity) {
+  auto data = array->data()->Copy();
+  if (data->buffers[0] == nullptr) {
+    data->buffers[0] = *AllocateBitmap(data->length);
+    BitUtil::SetBitsTo(data->buffers[0]->mutable_data(), 0, data->length, true);
+  }
+  BitUtil::SetBitTo(data->buffers[0]->mutable_data(), index, validity);
+  data->null_count = kUnknownNullCount;
+  // Need to return a new array, because Array caches the null bitmap pointer
+  return MakeArray(data);
+}
+
+template <typename T>
 class TestBinaryArithmetic : public TestBase {
  protected:
+  using ArrowType = T;
   using CType = typename ArrowType::c_type;
 
   static std::shared_ptr<DataType> type_singleton() {
@@ -53,20 +69,34 @@ class TestBinaryArithmetic : public TestBase {
 
   void SetUp() { options_.check_overflow = false; }
 
+  std::shared_ptr<Scalar> MakeNullScalar() {
+    return arrow::MakeNullScalar(type_singleton());
+  }
+
+  std::shared_ptr<Scalar> MakeScalar(CType value) {
+    return *arrow::MakeScalar(type_singleton(), value);
+  }
+
   // (Scalar, Scalar)
   void AssertBinop(BinaryFunction func, CType lhs, CType rhs, CType expected) {
-    ASSERT_OK_AND_ASSIGN(auto left, MakeScalar(type_singleton(), lhs));
-    ASSERT_OK_AND_ASSIGN(auto right, MakeScalar(type_singleton(), rhs));
-    ASSERT_OK_AND_ASSIGN(auto exp, MakeScalar(type_singleton(), expected));
+    auto left = MakeScalar(lhs);
+    auto right = MakeScalar(rhs);
+    auto exp = MakeScalar(expected);
 
     ASSERT_OK_AND_ASSIGN(auto actual, func(left, right, options_, nullptr));
-    AssertScalarsEqual(*exp, *actual.scalar(), true);
+    AssertScalarsEqual(*exp, *actual.scalar(), /*verbose=*/true);
   }
 
   // (Scalar, Array)
   void AssertBinop(BinaryFunction func, CType lhs, const std::string& rhs,
                    const std::string& expected) {
-    ASSERT_OK_AND_ASSIGN(auto left, MakeScalar(type_singleton(), lhs));
+    auto left = MakeScalar(lhs);
+    AssertBinop(func, left, rhs, expected);
+  }
+
+  // (Scalar, Array)
+  void AssertBinop(BinaryFunction func, const std::shared_ptr<Scalar>& left,
+                   const std::string& rhs, const std::string& expected) {
     auto right = ArrayFromJSON(type_singleton(), rhs);
     auto exp = ArrayFromJSON(type_singleton(), expected);
 
@@ -74,14 +104,48 @@ class TestBinaryArithmetic : public TestBase {
     ValidateAndAssertApproxEqual(actual.make_array(), expected);
   }
 
+  // (Array, Scalar)
+  void AssertBinop(BinaryFunction func, const std::string& lhs, CType rhs,
+                   const std::string& expected) {
+    auto right = MakeScalar(rhs);
+    AssertBinop(func, lhs, right, expected);
+  }
+
+  // (Array, Scalar)
+  void AssertBinop(BinaryFunction func, const std::string& lhs,
+                   const std::shared_ptr<Scalar>& right, const std::string& expected) {
+    auto left = ArrayFromJSON(type_singleton(), lhs);
+    auto exp = ArrayFromJSON(type_singleton(), expected);
+
+    ASSERT_OK_AND_ASSIGN(auto actual, func(left, right, options_, nullptr));
+    ValidateAndAssertApproxEqual(actual.make_array(), expected);
+  }
+
   // (Array, Array)
   void AssertBinop(BinaryFunction func, const std::string& lhs, const std::string& rhs,
                    const std::string& expected) {
     auto left = ArrayFromJSON(type_singleton(), lhs);
     auto right = ArrayFromJSON(type_singleton(), rhs);
 
+    AssertBinop(func, left, right, expected);
+  }
+
+  // (Array, Array)
+  void AssertBinop(BinaryFunction func, const std::shared_ptr<Array>& left,
+                   const std::shared_ptr<Array>& right,
+                   const std::string& expected_json) {
+    const auto expected = ArrayFromJSON(type_singleton(), expected_json);
     ASSERT_OK_AND_ASSIGN(Datum actual, func(left, right, options_, nullptr));
     ValidateAndAssertApproxEqual(actual.make_array(), expected);
+
+    // Also check (Scalar, Scalar) operations
+    const int64_t length = expected->length();
+    for (int64_t i = 0; i < length; ++i) {
+      const auto expected_scalar = *expected->GetScalar(i);
+      ASSERT_OK_AND_ASSIGN(
+          actual, func(*left->GetScalar(i), *right->GetScalar(i), options_, nullptr));
+      AssertScalarsEqual(*expected_scalar, *actual.scalar(), /*verbose=*/true);
+    }
   }
 
   void AssertBinopRaises(BinaryFunction func, const std::string& lhs,
@@ -93,11 +157,15 @@ class TestBinaryArithmetic : public TestBase {
                                     func(left, right, options_, nullptr));
   }
 
-  void ValidateAndAssertApproxEqual(std::shared_ptr<Array> actual,
+  void ValidateAndAssertApproxEqual(const std::shared_ptr<Array>& actual,
                                     const std::string& expected) {
-    auto exp = ArrayFromJSON(type_singleton(), expected);
+    ValidateAndAssertApproxEqual(actual, ArrayFromJSON(type_singleton(), expected));
+  }
+
+  void ValidateAndAssertApproxEqual(const std::shared_ptr<Array>& actual,
+                                    const std::shared_ptr<Array>& expected) {
     ASSERT_OK(actual->ValidateFull());
-    AssertArraysApproxEqual(*exp, *actual);
+    AssertArraysApproxEqual(*expected, *actual, /*verbose=*/true);
   }
 
   void SetOverflowCheck(bool value = true) { options_.check_overflow = value; }
@@ -150,19 +218,25 @@ TYPED_TEST(TestBinaryArithmeticIntegral, Add) {
     this->SetOverflowCheck(check_overflow);
 
     this->AssertBinop(Add, "[]", "[]", "[]");
-    this->AssertBinop(Add, "[null]", "[null]", "[null]");
     this->AssertBinop(Add, "[3, 2, 6]", "[1, 0, 2]", "[4, 2, 8]");
+    // Nulls on left side
+    this->AssertBinop(Add, "[null, 1, null]", "[3, 4, 5]", "[null, 5, null]");
+    this->AssertBinop(Add, "[3, 4, 5]", "[null, 1, null]", "[null, 5, null]");
+    // Nulls on both sides
+    this->AssertBinop(Add, "[null, 1, 2]", "[3, 4, null]", "[null, 5, null]");
+    // All nulls
+    this->AssertBinop(Add, "[null]", "[null]", "[null]");
 
-    this->AssertBinop(Add, "[1, 2, 3, 4, 5, 6, 7]", "[0, 1, 2, 3, 4, 5, 6]",
-                      "[1, 3, 5, 7, 9, 11, 13]");
-
-    this->AssertBinop(Add, "[10, 12, 4, 50, 50, 32, 11]", "[2, 0, 6, 1, 5, 3, 4]",
-                      "[12, 12, 10, 51, 55, 35, 15]");
-    this->AssertBinop(Add, "[null, 1, 3, null, 2, 5]", "[1, 4, 2, 5, 0, 3]",
-                      "[null, 5, 5, null, 2, 8]");
-    this->AssertBinop(Add, 10, "[null, 1, 3, null, 2, 5]",
-                      "[null, 11, 13, null, 12, 15]");
-    this->AssertBinop(Add, 17, 42, 59);
+    // Scalar on the left
+    this->AssertBinop(Add, 3, "[1, 2]", "[4, 5]");
+    this->AssertBinop(Add, 3, "[null, 2]", "[null, 5]");
+    this->AssertBinop(Add, this->MakeNullScalar(), "[1, 2]", "[null, null]");
+    this->AssertBinop(Add, this->MakeNullScalar(), "[null, 2]", "[null, null]");
+    // Scalar on the right
+    this->AssertBinop(Add, "[1, 2]", 3, "[4, 5]");
+    this->AssertBinop(Add, "[null, 2]", 3, "[null, 5]");
+    this->AssertBinop(Add, "[1, 2]", this->MakeNullScalar(), "[null, null]");
+    this->AssertBinop(Add, "[null, 2]", this->MakeNullScalar(), "[null, null]");
   }
 }
 
@@ -171,13 +245,25 @@ TYPED_TEST(TestBinaryArithmeticIntegral, Sub) {
     this->SetOverflowCheck(check_overflow);
 
     this->AssertBinop(Subtract, "[]", "[]", "[]");
-    this->AssertBinop(Subtract, "[null]", "[null]", "[null]");
     this->AssertBinop(Subtract, "[3, 2, 6]", "[1, 0, 2]", "[2, 2, 4]");
-    this->AssertBinop(Subtract, "[1, 2, 3, 4, 5, 6, 7]", "[0, 1, 2, 3, 4, 5, 6]",
-                      "[1, 1, 1, 1, 1, 1, 1]");
-    this->AssertBinop(Subtract, 10, "[null, 1, 3, null, 2, 5]",
-                      "[null, 9, 7, null, 8, 5]");
-    this->AssertBinop(Subtract, 20, 9, 11);
+    // Nulls on left side
+    this->AssertBinop(Subtract, "[null, 4, null]", "[2, 1, 0]", "[null, 3, null]");
+    this->AssertBinop(Subtract, "[5, 4, 3]", "[null, 1, null]", "[null, 3, null]");
+    // Nulls on both sides
+    this->AssertBinop(Subtract, "[null, 4, 3]", "[2, 1, null]", "[null, 3, null]");
+    // All nulls
+    this->AssertBinop(Subtract, "[null]", "[null]", "[null]");
+
+    // Scalar on the left
+    this->AssertBinop(Subtract, 3, "[1, 2]", "[2, 1]");
+    this->AssertBinop(Subtract, 3, "[null, 2]", "[null, 1]");
+    this->AssertBinop(Subtract, this->MakeNullScalar(), "[1, 2]", "[null, null]");
+    this->AssertBinop(Subtract, this->MakeNullScalar(), "[null, 2]", "[null, null]");
+    // Scalar on the right
+    this->AssertBinop(Subtract, "[4, 5]", 3, "[1, 2]");
+    this->AssertBinop(Subtract, "[null, 5]", 3, "[null, 2]");
+    this->AssertBinop(Subtract, "[1, 2]", this->MakeNullScalar(), "[null, null]");
+    this->AssertBinop(Subtract, "[null, 2]", this->MakeNullScalar(), "[null, null]");
   }
 }
 
@@ -208,17 +294,25 @@ TYPED_TEST(TestBinaryArithmeticIntegral, Mul) {
     this->SetOverflowCheck(check_overflow);
 
     this->AssertBinop(Multiply, "[]", "[]", "[]");
-    this->AssertBinop(Multiply, "[null]", "[null]", "[null]");
     this->AssertBinop(Multiply, "[3, 2, 6]", "[1, 0, 2]", "[3, 0, 12]");
-    this->AssertBinop(Multiply, "[1, 2, 3, 4, 5, 6, 7]", "[0, 1, 2, 3, 4, 5, 6]",
-                      "[0, 2, 6, 12, 20, 30, 42]");
-    this->AssertBinop(Multiply, "[7, 6, 5, 4, 3, 2, 1]", "[6, 5, 4, 3, 2, 1, 0]",
-                      "[42, 30, 20, 12, 6, 2, 0]");
-    this->AssertBinop(Multiply, "[null, 1, 3, null, 2, 5]", "[1, 4, 2, 5, 0, 3]",
-                      "[null, 4, 6, null, 0, 15]");
-    this->AssertBinop(Multiply, 3, "[null, 1, 3, null, 2, 5]",
-                      "[null, 3, 9, null, 6, 15]");
-    this->AssertBinop(Multiply, 6, 7, 42);
+    // Nulls on left side
+    this->AssertBinop(Multiply, "[null, 2, null]", "[4, 5, 6]", "[null, 10, null]");
+    this->AssertBinop(Multiply, "[4, 5, 6]", "[null, 2, null]", "[null, 10, null]");
+    // Nulls on both sides
+    this->AssertBinop(Multiply, "[null, 2, 3]", "[4, 5, null]", "[null, 10, null]");
+    // All nulls
+    this->AssertBinop(Multiply, "[null]", "[null]", "[null]");
+
+    // Scalar on the left
+    this->AssertBinop(Multiply, 3, "[4, 5]", "[12, 15]");
+    this->AssertBinop(Multiply, 3, "[null, 5]", "[null, 15]");
+    this->AssertBinop(Multiply, this->MakeNullScalar(), "[1, 2]", "[null, null]");
+    this->AssertBinop(Multiply, this->MakeNullScalar(), "[null, 2]", "[null, null]");
+    // Scalar on the right
+    this->AssertBinop(Multiply, "[4, 5]", 3, "[12, 15]");
+    this->AssertBinop(Multiply, "[null, 5]", 3, "[null, 15]");
+    this->AssertBinop(Multiply, "[1, 2]", this->MakeNullScalar(), "[null, null]");
+    this->AssertBinop(Multiply, "[null, 2]", this->MakeNullScalar(), "[null, null]");
   }
 }
 
@@ -273,6 +367,13 @@ TYPED_TEST(TestBinaryArithmeticSigned, AddOverflowRaises) {
 
   this->AssertBinopRaises(Add, MakeArray(max), MakeArray(1), "overflow");
   this->AssertBinopRaises(Add, MakeArray(min), MakeArray(-1), "overflow");
+
+  // Overflow should not be checked on underlying value slots when output would be null
+  auto left = ArrayFromJSON(this->type_singleton(), MakeArray(1, max, min));
+  auto right = ArrayFromJSON(this->type_singleton(), MakeArray(1, 1, -1));
+  left = TweakValidityBit(left, 1, false);
+  right = TweakValidityBit(right, 2, false);
+  this->AssertBinop(Add, left, right, "[2, null, null]");
 }
 
 TYPED_TEST(TestBinaryArithmeticSigned, SubOverflowRaises) {
@@ -290,6 +391,13 @@ TYPED_TEST(TestBinaryArithmeticSigned, SubOverflowRaises) {
 
   this->AssertBinopRaises(Subtract, MakeArray(max), MakeArray(-1), "overflow");
   this->AssertBinopRaises(Subtract, MakeArray(min), MakeArray(1), "overflow");
+
+  // Overflow should not be checked on underlying value slots when output would be null
+  auto left = ArrayFromJSON(this->type_singleton(), MakeArray(2, max, min));
+  auto right = ArrayFromJSON(this->type_singleton(), MakeArray(1, -1, 1));
+  left = TweakValidityBit(left, 1, false);
+  right = TweakValidityBit(right, 2, false);
+  this->AssertBinop(Subtract, left, right, "[1, null, null]");
 }
 
 TYPED_TEST(TestBinaryArithmeticSigned, MulOverflowRaises) {
@@ -311,6 +419,13 @@ TYPED_TEST(TestBinaryArithmeticSigned, MulOverflowRaises) {
   this->AssertBinopRaises(Multiply, MakeArray(min / 2), MakeArray(3), "overflow");
   this->AssertBinopRaises(Multiply, MakeArray(min), MakeArray(-1), "overflow");
   this->AssertBinopRaises(Multiply, MakeArray(min / 2), MakeArray(-2), "overflow");
+
+  // Overflow should not be checked on underlying value slots when output would be null
+  auto left = ArrayFromJSON(this->type_singleton(), MakeArray(2, max, min / 2));
+  auto right = ArrayFromJSON(this->type_singleton(), MakeArray(1, 2, 3));
+  left = TweakValidityBit(left, 1, false);
+  right = TweakValidityBit(right, 2, false);
+  this->AssertBinop(Multiply, left, right, "[2, null, null]");
 }
 
 TYPED_TEST(TestBinaryArithmeticUnsigned, OverflowWraps) {
@@ -352,47 +467,76 @@ TYPED_TEST(TestBinaryArithmeticSigned, Mul) {
   this->AssertBinop(Multiply, -5, -5, 25);
 }
 
+// NOTE: cannot test Inf / -Inf (ARROW-9495)
+
 TYPED_TEST(TestBinaryArithmeticFloating, Add) {
   this->AssertBinop(Add, "[]", "[]", "[]");
 
-  this->AssertBinop(Add, "[3.4, 2.6, 6.3]", "[1, 0, 2]", "[4.4, 2.6, 8.3]");
-
-  this->AssertBinop(Add, "[1.1, 2.4, 3.5, 4.3, 5.1, 6.8, 7.3]", "[0, 1, 2, 3, 4, 5, 6]",
-                    "[1.1, 3.4, 5.5, 7.3, 9.1, 11.8, 13.3]");
-
-  this->AssertBinop(Add, "[7, 6, 5, 4, 3, 2, 1]", "[6, 5, 4, 3, 2, 1, 0]",
-                    "[13, 11, 9, 7, 5, 3, 1]");
-
-  this->AssertBinop(Add, "[10.4, 12, 4.2, 50, 50.3, 32, 11]", "[2, 0, 6, 1, 5, 3, 4]",
-                    "[12.4, 12, 10.2, 51, 55.3, 35, 15]");
-
-  this->AssertBinop(Add, "[null, 1, 3.3, null, 2, 5.3]", "[1, 4, 2, 5, 0, 3]",
-                    "[null, 5, 5.3, null, 2, 8.3]");
-
-  this->AssertBinop(Add, 1.1F, "[null, 1, 3.3, null, 2, 5.3]",
-                    "[null, 2.1, 4.4, null, 3.1, 6.4]");
+  this->AssertBinop(Add, "[1.5, 0.5]", "[2.0, -3]", "[3.5, -2.5]");
+  // Nulls on the left
+  this->AssertBinop(Add, "[null, 0.5]", "[2.0, -3]", "[null, -2.5]");
+  // Nulls on the right
+  this->AssertBinop(Add, "[1.5, 0.5]", "[null, -3]", "[null, -2.5]");
+  // Nulls on both sides
+  this->AssertBinop(Add, "[null, 1.5, 0.5]", "[2.0, -3, null]", "[null, -1.5, null]");
+
+  // Scalar on the left
+  this->AssertBinop(Add, -1.5f, "[0.0, 2.0]", "[-1.5, 0.5]");
+  this->AssertBinop(Add, -1.5f, "[null, 2.0]", "[null, 0.5]");
+  this->AssertBinop(Add, this->MakeNullScalar(), "[0.0, 2.0]", "[null, null]");
+  this->AssertBinop(Add, this->MakeNullScalar(), "[null, 2.0]", "[null, null]");
+  // Scalar on the right
+  this->AssertBinop(Add, "[0.0, 2.0]", -1.5f, "[-1.5, 0.5]");
+  this->AssertBinop(Add, "[null, 2.0]", -1.5f, "[null, 0.5]");
+  this->AssertBinop(Add, "[0.0, 2.0]", this->MakeNullScalar(), "[null, null]");
+  this->AssertBinop(Add, "[null, 2.0]", this->MakeNullScalar(), "[null, null]");
 }
 
 TYPED_TEST(TestBinaryArithmeticFloating, Sub) {
   this->AssertBinop(Subtract, "[]", "[]", "[]");
 
-  this->AssertBinop(Subtract, "[3.4, 2.6, 6.3]", "[1, 0, 2]", "[2.4, 2.6, 4.3]");
-
-  this->AssertBinop(Subtract, "[1.1, 2.4, 3.5, 4.3, 5.1, 6.8, 7.3]",
-                    "[0.1, 1.2, 2.3, 3.4, 4.5, 5.6, 6.7]",
-                    "[1.0, 1.2, 1.2, 0.9, 0.6, 1.2, 0.6]");
-
-  this->AssertBinop(Subtract, "[7, 6, 5, 4, 3, 2, 1]", "[6, 5, 4, 3, 2, 1, 0]",
-                    "[1.0, 1.0, 1.0, 1.0, 1.0, 1.0, 1.0]");
-
-  this->AssertBinop(Subtract, "[10.4, 12, 4.2, 50, 50.3, 32, 11]",
-                    "[2, 0, 6, 1, 5, 3, 4]", "[8.4, 12, -1.8, 49, 45.3, 29, 7]");
-
-  this->AssertBinop(Subtract, "[null, 1, 3.3, null, 2, 5.3]", "[1, 4, 2, 5, 0, 3]",
-                    "[null, -3, 1.3, null, 2, 2.3]");
+  this->AssertBinop(Subtract, "[1.5, 0.5]", "[2.0, -3]", "[-0.5, 3.5]");
+  // Nulls on the left
+  this->AssertBinop(Subtract, "[null, 0.5]", "[2.0, -3]", "[null, 3.5]");
+  // Nulls on the right
+  this->AssertBinop(Subtract, "[1.5, 0.5]", "[null, -3]", "[null, 3.5]");
+  // Nulls on both sides
+  this->AssertBinop(Subtract, "[null, 1.5, 0.5]", "[2.0, -3, null]", "[null, 4.5, null]");
+
+  // Scalar on the left
+  this->AssertBinop(Subtract, -1.5f, "[0.0, 2.0]", "[-1.5, -3.5]");
+  this->AssertBinop(Subtract, -1.5f, "[null, 2.0]", "[null, -3.5]");
+  this->AssertBinop(Subtract, this->MakeNullScalar(), "[0.0, 2.0]", "[null, null]");
+  this->AssertBinop(Subtract, this->MakeNullScalar(), "[null, 2.0]", "[null, null]");
+  // Scalar on the right
+  this->AssertBinop(Subtract, "[0.0, 2.0]", -1.5f, "[1.5, 3.5]");
+  this->AssertBinop(Subtract, "[null, 2.0]", -1.5f, "[null, 3.5]");
+  this->AssertBinop(Subtract, "[0.0, 2.0]", this->MakeNullScalar(), "[null, null]");
+  this->AssertBinop(Subtract, "[null, 2.0]", this->MakeNullScalar(), "[null, null]");
+}
 
-  this->AssertBinop(Subtract, 0.1F, "[null, 1, 3.3, null, 2, 5.3]",
-                    "[null, -0.9, -3.2, null, -1.9, -5.2]");
+TYPED_TEST(TestBinaryArithmeticFloating, Mul) {
+  this->AssertBinop(Multiply, "[]", "[]", "[]");
+
+  this->AssertBinop(Multiply, "[1.5, 0.5]", "[2.0, -3]", "[3.0, -1.5]");
+  // Nulls on the left
+  this->AssertBinop(Multiply, "[null, 0.5]", "[2.0, -3]", "[null, -1.5]");
+  // Nulls on the right
+  this->AssertBinop(Multiply, "[1.5, 0.5]", "[null, -3]", "[null, -1.5]");
+  // Nulls on both sides
+  this->AssertBinop(Multiply, "[null, 1.5, 0.5]", "[2.0, -3, null]",
+                    "[null, -4.5, null]");
+
+  // Scalar on the left
+  this->AssertBinop(Multiply, -1.5f, "[0.0, 2.0]", "[0.0, -3.0]");
+  this->AssertBinop(Multiply, -1.5f, "[null, 2.0]", "[null, -3.0]");
+  this->AssertBinop(Multiply, this->MakeNullScalar(), "[0.0, 2.0]", "[null, null]");
+  this->AssertBinop(Multiply, this->MakeNullScalar(), "[null, 2.0]", "[null, null]");
+  // Scalar on the right
+  this->AssertBinop(Multiply, "[0.0, 2.0]", -1.5f, "[0.0, -3.0]");
+  this->AssertBinop(Multiply, "[null, 2.0]", -1.5f, "[null, -3.0]");
+  this->AssertBinop(Multiply, "[0.0, 2.0]", this->MakeNullScalar(), "[null, null]");
+  this->AssertBinop(Multiply, "[null, 2.0]", this->MakeNullScalar(), "[null, null]");
 }
 
 }  // namespace compute
diff --git a/cpp/src/arrow/compute/kernels/scalar_cast_boolean.cc b/cpp/src/arrow/compute/kernels/scalar_cast_boolean.cc
index d786a33..207dd30 100644
--- a/cpp/src/arrow/compute/kernels/scalar_cast_boolean.cc
+++ b/cpp/src/arrow/compute/kernels/scalar_cast_boolean.cc
@@ -29,15 +29,15 @@ namespace compute {
 namespace internal {
 
 struct IsNonZero {
-  template <typename OUT, typename ARG0>
-  static OUT Call(KernelContext*, ARG0 val) {
+  template <typename OutValue, typename Arg0Value>
+  static OutValue Call(KernelContext*, Arg0Value val) {
     return val != 0;
   }
 };
 
 struct ParseBooleanString {
-  template <typename OUT, typename ARG0>
-  static OUT Call(KernelContext* ctx, ARG0 val) {
+  template <typename OutValue, typename Arg0Value>
+  static OutValue Call(KernelContext* ctx, Arg0Value val) {
     bool result = false;
     if (ARROW_PREDICT_FALSE(!ParseValue<BooleanType>(val.data(), val.size(), &result))) {
       ctx->SetStatus(Status::Invalid("Failed to parse value: ", val));
diff --git a/cpp/src/arrow/compute/kernels/scalar_cast_numeric.cc b/cpp/src/arrow/compute/kernels/scalar_cast_numeric.cc
index 4778019..078902a 100644
--- a/cpp/src/arrow/compute/kernels/scalar_cast_numeric.cc
+++ b/cpp/src/arrow/compute/kernels/scalar_cast_numeric.cc
@@ -257,10 +257,10 @@ void CastIntegerToFloating(KernelContext* ctx, const ExecBatch& batch, Datum* ou
 // Boolean to number
 
 struct BooleanToNumber {
-  template <typename OUT, typename ARG0>
-  static OUT Call(KernelContext*, ARG0 val) {
-    constexpr auto kOne = static_cast<OUT>(1);
-    constexpr auto kZero = static_cast<OUT>(0);
+  template <typename OutValue, typename Arg0Value>
+  static OutValue Call(KernelContext*, Arg0Value val) {
+    constexpr auto kOne = static_cast<OutValue>(1);
+    constexpr auto kZero = static_cast<OutValue>(0);
     return val ? kOne : kZero;
   }
 };
@@ -277,9 +277,9 @@ struct CastFunctor<O, BooleanType, enable_if_number<O>> {
 
 template <typename OutType>
 struct ParseString {
-  template <typename OUT, typename ARG0>
-  OUT Call(KernelContext* ctx, ARG0 val) const {
-    OUT result = OUT(0);
+  template <typename OutValue, typename Arg0Value>
+  OutValue Call(KernelContext* ctx, Arg0Value val) const {
+    OutValue result = OutValue(0);
     if (ARROW_PREDICT_FALSE(!ParseValue<OutType>(val.data(), val.size(), &result))) {
       ctx->SetStatus(Status::Invalid("Failed to parse string: ", val));
     }
@@ -298,16 +298,16 @@ struct CastFunctor<O, I, enable_if_base_binary<I>> {
 // Decimal to integer
 
 struct DecimalToIntegerMixin {
-  template <typename OUT>
-  OUT ToInteger(KernelContext* ctx, const Decimal128& val) const {
-    constexpr auto min_value = std::numeric_limits<OUT>::min();
-    constexpr auto max_value = std::numeric_limits<OUT>::max();
+  template <typename OutValue>
+  OutValue ToInteger(KernelContext* ctx, const Decimal128& val) const {
+    constexpr auto min_value = std::numeric_limits<OutValue>::min();
+    constexpr auto max_value = std::numeric_limits<OutValue>::max();
 
     if (!allow_int_overflow_ && ARROW_PREDICT_FALSE(val < min_value || val > max_value)) {
       ctx->SetStatus(Status::Invalid("Integer value out of bounds"));
-      return OUT{};  // Zero
+      return OutValue{};  // Zero
     } else {
-      return static_cast<OUT>(val.low_bits());
+      return static_cast<OutValue>(val.low_bits());
     }
   }
 
@@ -321,32 +321,32 @@ struct DecimalToIntegerMixin {
 struct UnsafeUpscaleDecimalToInteger : public DecimalToIntegerMixin {
   using DecimalToIntegerMixin::DecimalToIntegerMixin;
 
-  template <typename OUT, typename ARG0>
-  OUT Call(KernelContext* ctx, Decimal128 val) const {
-    return ToInteger<OUT>(ctx, val.IncreaseScaleBy(-in_scale_));
+  template <typename OutValue, typename Arg0Value>
+  OutValue Call(KernelContext* ctx, Decimal128 val) const {
+    return ToInteger<OutValue>(ctx, val.IncreaseScaleBy(-in_scale_));
   }
 };
 
 struct UnsafeDownscaleDecimalToInteger : public DecimalToIntegerMixin {
   using DecimalToIntegerMixin::DecimalToIntegerMixin;
 
-  template <typename OUT, typename ARG0>
-  OUT Call(KernelContext* ctx, Decimal128 val) const {
-    return ToInteger<OUT>(ctx, val.ReduceScaleBy(in_scale_, false));
+  template <typename OutValue, typename Arg0Value>
+  OutValue Call(KernelContext* ctx, Decimal128 val) const {
+    return ToInteger<OutValue>(ctx, val.ReduceScaleBy(in_scale_, false));
   }
 };
 
 struct SafeRescaleDecimalToInteger : public DecimalToIntegerMixin {
   using DecimalToIntegerMixin::DecimalToIntegerMixin;
 
-  template <typename OUT, typename ARG0>
-  OUT Call(KernelContext* ctx, Decimal128 val) const {
+  template <typename OutValue, typename Arg0Value>
+  OutValue Call(KernelContext* ctx, Decimal128 val) const {
     auto result = val.Rescale(in_scale_, 0);
     if (ARROW_PREDICT_FALSE(!result.ok())) {
       ctx->SetStatus(result.status());
-      return OUT{};  // Zero
+      return OutValue{};  // Zero
     } else {
-      return ToInteger<OUT>(ctx, *result);
+      return ToInteger<OutValue>(ctx, *result);
     }
   }
 };
@@ -466,7 +466,7 @@ struct CastFunctor<Decimal128Type, Decimal128Type> {
 // Real to decimal
 
 struct RealToDecimal {
-  template <typename OUT, typename RealType>
+  template <typename OutValue, typename RealType>
   Decimal128 Call(KernelContext* ctx, RealType val) const {
     auto result = Decimal128::FromReal(val, out_precision_, out_scale_);
     if (ARROW_PREDICT_FALSE(!result.ok())) {
@@ -502,7 +502,7 @@ struct CastFunctor<Decimal128Type, I, enable_if_t<is_floating_type<I>::value>> {
 // Decimal to real
 
 struct DecimalToReal {
-  template <typename RealType, typename ARG0>
+  template <typename RealType, typename Arg0Value>
   RealType Call(KernelContext* ctx, const Decimal128& val) const {
     return val.ToReal<RealType>(in_scale_);
   }
diff --git a/cpp/src/arrow/compute/kernels/scalar_cast_temporal.cc b/cpp/src/arrow/compute/kernels/scalar_cast_temporal.cc
index 60b72b3..96f0b87 100644
--- a/cpp/src/arrow/compute/kernels/scalar_cast_temporal.cc
+++ b/cpp/src/arrow/compute/kernels/scalar_cast_temporal.cc
@@ -267,9 +267,9 @@ struct CastFunctor<Date32Type, Date64Type> {
 // String to Timestamp
 
 struct ParseTimestamp {
-  template <typename OUT, typename ARG0>
-  OUT Call(KernelContext* ctx, ARG0 val) const {
-    OUT result = 0;
+  template <typename OutValue, typename Arg0Value>
+  OutValue Call(KernelContext* ctx, Arg0Value val) const {
+    OutValue result = 0;
     if (ARROW_PREDICT_FALSE(!ParseValue(type, val.data(), val.size(), &result))) {
       ctx->SetStatus(Status::Invalid("Failed to parse string: ", val));
     }
diff --git a/cpp/src/arrow/compute/kernels/scalar_nested.cc b/cpp/src/arrow/compute/kernels/scalar_nested.cc
index 677c577..201d501 100644
--- a/cpp/src/arrow/compute/kernels/scalar_nested.cc
+++ b/cpp/src/arrow/compute/kernels/scalar_nested.cc
@@ -20,7 +20,7 @@
 #include "arrow/array/array_base.h"
 #include "arrow/compute/kernels/common.h"
 #include "arrow/result.h"
-#include "arrow/visitor_inline.h"
+#include "arrow/util/bit_block_counter.h"
 
 namespace arrow {
 namespace compute {
@@ -37,7 +37,7 @@ void ListValueLength(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
     ArrayData* out_arr = out->mutable_array();
     auto out_values = out_arr->GetMutableValues<offset_type>(1);
     const offset_type* offsets = list.raw_value_offsets();
-    ::arrow::internal::detail::VisitBitBlocksVoid(
+    ::arrow::internal::VisitBitBlocksVoid(
         list.data()->buffers[0], list.offset(), list.length(),
         [&](int64_t position) {
           *out_values++ = offsets[position + 1] - offsets[position];
diff --git a/cpp/src/arrow/compute/kernels/scalar_string.cc b/cpp/src/arrow/compute/kernels/scalar_string.cc
index a0ea240..7e61617 100644
--- a/cpp/src/arrow/compute/kernels/scalar_string.cc
+++ b/cpp/src/arrow/compute/kernels/scalar_string.cc
@@ -56,9 +56,9 @@ static inline bool IsAsciiCharacter(T character) {
 }
 
 struct BinaryLength {
-  template <typename OUT, typename ARG0 = util::string_view>
-  static OUT Call(KernelContext*, ARG0 val) {
-    return static_cast<OUT>(val.size());
+  template <typename OutValue, typename Arg0Value = util::string_view>
+  static OutValue Call(KernelContext*, Arg0Value val) {
+    return static_cast<OutValue>(val.size());
   }
 };
 
diff --git a/cpp/src/arrow/compute/kernels/test_util.cc b/cpp/src/arrow/compute/kernels/test_util.cc
index dcc7c2d..f8cc1c0 100644
--- a/cpp/src/arrow/compute/kernels/test_util.cc
+++ b/cpp/src/arrow/compute/kernels/test_util.cc
@@ -33,68 +33,105 @@ namespace compute {
 
 namespace {
 
-void CheckScalarUnaryNonRecursive(const std::string& func_name,
-                                  const std::shared_ptr<Array>& input,
-                                  const std::shared_ptr<Array>& expected,
-                                  const FunctionOptions* options) {
-  ASSERT_OK_AND_ASSIGN(Datum out, CallFunction(func_name, {input}, options));
+template <typename T>
+std::vector<Datum> GetDatums(const std::vector<T>& inputs) {
+  std::vector<Datum> datums;
+  for (const auto& input : inputs) {
+    datums.emplace_back(input);
+  }
+  return datums;
+}
+
+void CheckScalarNonRecursive(const std::string& func_name, const ArrayVector& inputs,
+                             const std::shared_ptr<Array>& expected,
+                             const FunctionOptions* options) {
+  ASSERT_OK_AND_ASSIGN(Datum out, CallFunction(func_name, GetDatums(inputs), options));
   std::shared_ptr<Array> actual = std::move(out).make_array();
   ASSERT_OK(actual->ValidateFull());
   AssertArraysEqual(*expected, *actual, /*verbose=*/true);
 }
 
-}  // namespace
+template <typename... SliceArgs>
+ArrayVector SliceAll(const ArrayVector& inputs, SliceArgs... slice_args) {
+  ArrayVector sliced;
+  for (const auto& input : inputs) {
+    sliced.push_back(input->Slice(slice_args...));
+  }
+  return sliced;
+}
 
-void CheckScalarUnary(std::string func_name, std::shared_ptr<Array> input,
-                      std::shared_ptr<Array> expected, const FunctionOptions* options) {
-  CheckScalarUnaryNonRecursive(func_name, input, expected, options);
+ScalarVector GetScalars(const ArrayVector& inputs, int64_t index) {
+  ScalarVector scalars;
+  for (const auto& input : inputs) {
+    scalars.push_back(*input->GetScalar(index));
+  }
+  return scalars;
+}
+
+void CheckScalar(std::string func_name, const ScalarVector& inputs,
+                 std::shared_ptr<Scalar> expected, const FunctionOptions* options) {
+  ASSERT_OK_AND_ASSIGN(Datum out, CallFunction(func_name, GetDatums(inputs), options));
+  AssertScalarsEqual(*expected, *out.scalar(), /*verbose=*/true);
+}
+
+void CheckScalar(std::string func_name, const ArrayVector& inputs,
+                 std::shared_ptr<Array> expected, const FunctionOptions* options) {
+  CheckScalarNonRecursive(func_name, inputs, expected, options);
 
   // Check all the input scalars
-  for (int64_t i = 0; i < input->length(); ++i) {
-    ASSERT_OK_AND_ASSIGN(auto val, input->GetScalar(i));
-    ASSERT_OK_AND_ASSIGN(auto ex_val, expected->GetScalar(i));
-    CheckScalarUnary(func_name, val, ex_val, options);
+  for (int64_t i = 0; i < inputs[0]->length(); ++i) {
+    CheckScalar(func_name, GetScalars(inputs, i), *expected->GetScalar(i), options);
   }
 
-  const auto slice_length = input->length() / 3;
-  // Since it's a scalar function, calling it on a sliced input should
+  // Since it's a scalar function, calling it on sliced inputs should
   // result in the sliced expected output.
+  const auto slice_length = inputs[0]->length() / 3;
   if (slice_length > 0) {
-    CheckScalarUnaryNonRecursive(func_name, input->Slice(0, slice_length),
-                                 expected->Slice(0, slice_length), options);
+    CheckScalarNonRecursive(func_name, SliceAll(inputs, 0, slice_length),
+                            expected->Slice(0, slice_length), options);
 
-    CheckScalarUnaryNonRecursive(func_name, input->Slice(slice_length, slice_length),
-                                 expected->Slice(slice_length, slice_length), options);
+    CheckScalarNonRecursive(func_name, SliceAll(inputs, slice_length, slice_length),
+                            expected->Slice(slice_length, slice_length), options);
 
-    CheckScalarUnaryNonRecursive(func_name, input->Slice(2 * slice_length),
-                                 expected->Slice(2 * slice_length), options);
+    CheckScalarNonRecursive(func_name, SliceAll(inputs, 2 * slice_length),
+                            expected->Slice(2 * slice_length), options);
   }
 
-  // Ditto with a ChunkedArray input
+  // Ditto with ChunkedArray inputs
   if (slice_length > 0) {
-    ArrayVector input_chunks{input->Slice(0, slice_length), input->Slice(slice_length)},
-        expected_chunks{expected->Slice(0, 2 * slice_length),
-                        expected->Slice(2 * slice_length)};
-
-    ASSERT_OK_AND_ASSIGN(
-        Datum out,
-        CallFunction(func_name, {std::make_shared<ChunkedArray>(input_chunks)}, options));
+    std::vector<std::shared_ptr<ChunkedArray>> chunked_inputs;
+    chunked_inputs.reserve(inputs.size());
+    for (const auto& input : inputs) {
+      chunked_inputs.push_back(std::make_shared<ChunkedArray>(
+          ArrayVector{input->Slice(0, slice_length), input->Slice(slice_length)}));
+    }
+    ArrayVector expected_chunks{expected->Slice(0, slice_length),
+                                expected->Slice(slice_length)};
+
+    ASSERT_OK_AND_ASSIGN(Datum out,
+                         CallFunction(func_name, GetDatums(chunked_inputs), options));
     ASSERT_OK(out.chunked_array()->ValidateFull());
     AssertDatumsEqual(std::make_shared<ChunkedArray>(expected_chunks), out);
   }
 }
 
+}  // namespace
+
+void CheckScalarUnary(std::string func_name, std::shared_ptr<Array> input,
+                      std::shared_ptr<Array> expected, const FunctionOptions* options) {
+  CheckScalar(std::move(func_name), {input}, expected, options);
+}
+
 void CheckScalarUnary(std::string func_name, std::shared_ptr<DataType> in_ty,
                       std::string json_input, std::shared_ptr<DataType> out_ty,
                       std::string json_expected, const FunctionOptions* options) {
-  CheckScalarUnary(func_name, ArrayFromJSON(in_ty, json_input),
+  CheckScalarUnary(std::move(func_name), ArrayFromJSON(in_ty, json_input),
                    ArrayFromJSON(out_ty, json_expected), options);
 }
 
 void CheckScalarUnary(std::string func_name, std::shared_ptr<Scalar> input,
                       std::shared_ptr<Scalar> expected, const FunctionOptions* options) {
-  ASSERT_OK_AND_ASSIGN(Datum out, CallFunction(func_name, {input}, options));
-  AssertScalarsEqual(*expected, *out.scalar(), /*verbose=*/true);
+  CheckScalar(std::move(func_name), {input}, expected, options);
 }
 
 void CheckVectorUnary(std::string func_name, Datum input, std::shared_ptr<Array> expected,
@@ -105,5 +142,17 @@ void CheckVectorUnary(std::string func_name, Datum input, std::shared_ptr<Array>
   AssertArraysEqual(*expected, *actual, /*verbose=*/true);
 }
 
+void CheckScalarBinary(std::string func_name, std::shared_ptr<Scalar> left_input,
+                       std::shared_ptr<Scalar> right_input,
+                       std::shared_ptr<Scalar> expected, const FunctionOptions* options) {
+  CheckScalar(std::move(func_name), {left_input, right_input}, expected, options);
+}
+
+void CheckScalarBinary(std::string func_name, std::shared_ptr<Array> left_input,
+                       std::shared_ptr<Array> right_input,
+                       std::shared_ptr<Array> expected, const FunctionOptions* options) {
+  CheckScalar(std::move(func_name), {left_input, right_input}, expected, options);
+}
+
 }  // namespace compute
 }  // namespace arrow
diff --git a/cpp/src/arrow/compute/kernels/test_util.h b/cpp/src/arrow/compute/kernels/test_util.h
index 9786c89..c38c0ce 100644
--- a/cpp/src/arrow/compute/kernels/test_util.h
+++ b/cpp/src/arrow/compute/kernels/test_util.h
@@ -103,6 +103,16 @@ void CheckScalarUnary(std::string func_name, std::shared_ptr<Scalar> input,
                       std::shared_ptr<Scalar> expected,
                       const FunctionOptions* options = nullptr);
 
+void CheckScalarBinary(std::string func_name, std::shared_ptr<Scalar> left_input,
+                       std::shared_ptr<Scalar> right_input,
+                       std::shared_ptr<Scalar> expected,
+                       const FunctionOptions* options = nullptr);
+
+void CheckScalarBinary(std::string func_name, std::shared_ptr<Array> left_input,
+                       std::shared_ptr<Array> right_input,
+                       std::shared_ptr<Array> expected,
+                       const FunctionOptions* options = nullptr);
+
 void CheckVectorUnary(std::string func_name, Datum input, std::shared_ptr<Array> expected,
                       const FunctionOptions* options = nullptr);
 
diff --git a/cpp/src/arrow/dataset/filter.cc b/cpp/src/arrow/dataset/filter.cc
index d99d624..ddfee24 100644
--- a/cpp/src/arrow/dataset/filter.cc
+++ b/cpp/src/arrow/dataset/filter.cc
@@ -39,7 +39,7 @@
 #include "arrow/scalar.h"
 #include "arrow/type_fwd.h"
 #include "arrow/util/checked_cast.h"
-#include "arrow/util/int_util.h"
+#include "arrow/util/int_util_internal.h"
 #include "arrow/util/iterator.h"
 #include "arrow/util/logging.h"
 #include "arrow/util/string.h"
@@ -1606,8 +1606,7 @@ class StructDictionary {
     *fused_indices = checked_pointer_cast<Int32Array>(new_fused_indices.make_array());
 
     // XXX should probably cap this at 2**15 or so
-    DCHECK(!internal::HasPositiveMultiplyOverflow(size_, dictionary_size));
-    size_ *= dictionary_size;
+    ARROW_CHECK(!internal::MultiplyWithOverflow(size_, dictionary_size, &size_));
     return Status::OK();
   }
 
diff --git a/cpp/src/arrow/pretty_print.cc b/cpp/src/arrow/pretty_print.cc
index bb8b09f..4b3fd86 100644
--- a/cpp/src/arrow/pretty_print.cc
+++ b/cpp/src/arrow/pretty_print.cc
@@ -35,7 +35,7 @@
 #include "arrow/type.h"
 #include "arrow/type_traits.h"
 #include "arrow/util/checked_cast.h"
-#include "arrow/util/int_util.h"
+#include "arrow/util/int_util_internal.h"
 #include "arrow/util/key_value_metadata.h"
 #include "arrow/util/string.h"
 #include "arrow/vendored/datetime.h"
diff --git a/cpp/src/arrow/scalar.h b/cpp/src/arrow/scalar.h
index 1a079bb..d15c44c 100644
--- a/cpp/src/arrow/scalar.h
+++ b/cpp/src/arrow/scalar.h
@@ -439,16 +439,6 @@ struct ARROW_EXPORT ExtensionScalar : public Scalar {
 
 /// @}
 
-/// \defgroup scalar-factories Scalar factory functions
-///
-/// @{
-
-/// \brief Scalar factory for null scalars
-ARROW_EXPORT
-std::shared_ptr<Scalar> MakeNullScalar(std::shared_ptr<DataType> type);
-
-/// @}
-
 namespace internal {
 
 inline Status CheckBufferLength(...) { return Status::OK(); }
@@ -488,10 +478,14 @@ struct MakeScalarImpl {
   std::shared_ptr<Scalar> out_;
 };
 
-/// \addtogroup scalar-factories
+/// \defgroup scalar-factories Scalar factory functions
 ///
 /// @{
 
+/// \brief Scalar factory for null scalars
+ARROW_EXPORT
+std::shared_ptr<Scalar> MakeNullScalar(std::shared_ptr<DataType> type);
+
 /// \brief Scalar factory for non-null scalars
 template <typename Value>
 Result<std::shared_ptr<Scalar>> MakeScalar(std::shared_ptr<DataType> type,
diff --git a/cpp/src/arrow/util/basic_decimal.cc b/cpp/src/arrow/util/basic_decimal.cc
index fccc3b4..e46834d 100644
--- a/cpp/src/arrow/util/basic_decimal.cc
+++ b/cpp/src/arrow/util/basic_decimal.cc
@@ -28,7 +28,7 @@
 #include <string>
 
 #include "arrow/util/bit_util.h"
-#include "arrow/util/int_util.h"
+#include "arrow/util/int_util_internal.h"
 #include "arrow/util/logging.h"
 #include "arrow/util/macros.h"
 
diff --git a/cpp/src/arrow/util/bit_block_counter.cc b/cpp/src/arrow/util/bit_block_counter.cc
index 3bf5080..516695e 100644
--- a/cpp/src/arrow/util/bit_block_counter.cc
+++ b/cpp/src/arrow/util/bit_block_counter.cc
@@ -113,10 +113,10 @@ BitBlockCount BitBlockCounter::NextFourWords() {
 
 OptionalBitBlockCounter::OptionalBitBlockCounter(const uint8_t* validity_bitmap,
                                                  int64_t offset, int64_t length)
-    : counter_(validity_bitmap, offset, length),
+    : has_bitmap_(validity_bitmap != nullptr),
       position_(0),
       length_(length),
-      has_bitmap_(validity_bitmap != nullptr) {}
+      counter_(validity_bitmap, offset, length) {}
 
 OptionalBitBlockCounter::OptionalBitBlockCounter(
     const std::shared_ptr<Buffer>& validity_bitmap, int64_t offset, int64_t length)
@@ -180,5 +180,24 @@ BitBlockCount BinaryBitBlockCounter::NextOrNotWord() {
   return NextWord<detail::BitBlockOrNot>();
 }
 
+OptionalBinaryBitBlockCounter::OptionalBinaryBitBlockCounter(const uint8_t* left_bitmap,
+                                                             int64_t left_offset,
+                                                             const uint8_t* right_bitmap,
+                                                             int64_t right_offset,
+                                                             int64_t length)
+    : has_bitmap_(HasBitmapFromBitmaps(left_bitmap != nullptr, right_bitmap != nullptr)),
+      position_(0),
+      length_(length),
+      unary_counter_(left_bitmap != nullptr ? left_bitmap : right_bitmap,
+                     left_bitmap != nullptr ? left_offset : right_offset, length),
+      binary_counter_(left_bitmap, left_offset, right_bitmap, right_offset, length) {}
+
+OptionalBinaryBitBlockCounter::OptionalBinaryBitBlockCounter(
+    const std::shared_ptr<Buffer>& left_bitmap, int64_t left_offset,
+    const std::shared_ptr<Buffer>& right_bitmap, int64_t right_offset, int64_t length)
+    : OptionalBinaryBitBlockCounter(
+          left_bitmap ? left_bitmap->data() : nullptr, left_offset,
+          right_bitmap ? right_bitmap->data() : nullptr, right_offset, length) {}
+
 }  // namespace internal
 }  // namespace arrow
diff --git a/cpp/src/arrow/util/bit_block_counter.h b/cpp/src/arrow/util/bit_block_counter.h
index 52a3985..8f2bd0a 100644
--- a/cpp/src/arrow/util/bit_block_counter.h
+++ b/cpp/src/arrow/util/bit_block_counter.h
@@ -22,14 +22,14 @@
 #include <limits>
 #include <memory>
 
+#include "arrow/buffer.h"
+#include "arrow/status.h"
+#include "arrow/util/bit_util.h"
+#include "arrow/util/macros.h"
 #include "arrow/util/visibility.h"
 
 namespace arrow {
-
-class Buffer;
-
 namespace internal {
-
 namespace detail {
 
 // These templates are here to help with unit tests
@@ -121,7 +121,7 @@ class ARROW_EXPORT BitBlockCounter {
 /// cases without giving up a lot of performance.
 class ARROW_EXPORT OptionalBitBlockCounter {
  public:
-  // validity_bitmap may be nullptr
+  // validity_bitmap may be NULLPTR
   OptionalBitBlockCounter(const uint8_t* validity_bitmap, int64_t offset, int64_t length);
 
   // validity_bitmap may be null
@@ -163,10 +163,10 @@ class ARROW_EXPORT OptionalBitBlockCounter {
   }
 
  private:
-  BitBlockCounter counter_;
+  const bool has_bitmap_;
   int64_t position_;
   int64_t length_;
-  bool has_bitmap_;
+  BitBlockCounter counter_;
 };
 
 /// \brief A class that computes popcounts on the result of bitwise operations
@@ -207,5 +207,177 @@ class ARROW_EXPORT BinaryBitBlockCounter {
   int64_t bits_remaining_;
 };
 
+class ARROW_EXPORT OptionalBinaryBitBlockCounter {
+ public:
+  // Any bitmap may be NULLPTR
+  OptionalBinaryBitBlockCounter(const uint8_t* left_bitmap, int64_t left_offset,
+                                const uint8_t* right_bitmap, int64_t right_offset,
+                                int64_t length);
+
+  // Any bitmap may be null
+  OptionalBinaryBitBlockCounter(const std::shared_ptr<Buffer>& left_bitmap,
+                                int64_t left_offset,
+                                const std::shared_ptr<Buffer>& right_bitmap,
+                                int64_t right_offset, int64_t length);
+
+  BitBlockCount NextAndBlock() {
+    static constexpr int64_t kMaxBlockSize = std::numeric_limits<int16_t>::max();
+    switch (has_bitmap_) {
+      case HasBitmap::BOTH: {
+        BitBlockCount block = binary_counter_.NextAndWord();
+        position_ += block.length;
+        return block;
+      }
+      case HasBitmap::ONE: {
+        BitBlockCount block = unary_counter_.NextWord();
+        position_ += block.length;
+        return block;
+      }
+      case HasBitmap::NONE:
+      default: {
+        const int16_t block_size =
+            static_cast<int16_t>(std::min(kMaxBlockSize, length_ - position_));
+        position_ += block_size;
+        // All values are non-null
+        return {block_size, block_size};
+      }
+    }
+  }
+
+ private:
+  enum class HasBitmap : int { BOTH, ONE, NONE };
+
+  const HasBitmap has_bitmap_;
+  int64_t position_;
+  int64_t length_;
+  BitBlockCounter unary_counter_;
+  BinaryBitBlockCounter binary_counter_;
+
+  static HasBitmap HasBitmapFromBitmaps(bool has_left, bool has_right) {
+    switch (static_cast<int>(has_left) + static_cast<int>(has_right)) {
+      case 0:
+        return HasBitmap::NONE;
+      case 1:
+        return HasBitmap::ONE;
+      default:  // 2
+        return HasBitmap::BOTH;
+    }
+  }
+};
+
+// Functional-style bit block visitors.
+
+template <typename VisitNotNull, typename VisitNull>
+Status VisitBitBlocks(const std::shared_ptr<Buffer>& bitmap_buf, int64_t offset,
+                      int64_t length, VisitNotNull&& visit_not_null,
+                      VisitNull&& visit_null) {
+  const uint8_t* bitmap = NULLPTR;
+  if (bitmap_buf != NULLPTR) {
+    bitmap = bitmap_buf->data();
+  }
+  internal::OptionalBitBlockCounter bit_counter(bitmap, offset, length);
+  int64_t position = 0;
+  while (position < length) {
+    internal::BitBlockCount block = bit_counter.NextBlock();
+    if (block.AllSet()) {
+      for (int64_t i = 0; i < block.length; ++i, ++position) {
+        ARROW_RETURN_NOT_OK(visit_not_null(position));
+      }
+    } else if (block.NoneSet()) {
+      for (int64_t i = 0; i < block.length; ++i, ++position) {
+        ARROW_RETURN_NOT_OK(visit_null());
+      }
+    } else {
+      for (int64_t i = 0; i < block.length; ++i, ++position) {
+        if (BitUtil::GetBit(bitmap, offset + position)) {
+          ARROW_RETURN_NOT_OK(visit_not_null(position));
+        } else {
+          ARROW_RETURN_NOT_OK(visit_null());
+        }
+      }
+    }
+  }
+  return Status::OK();
+}
+
+template <typename VisitNotNull, typename VisitNull>
+void VisitBitBlocksVoid(const std::shared_ptr<Buffer>& bitmap_buf, int64_t offset,
+                        int64_t length, VisitNotNull&& visit_not_null,
+                        VisitNull&& visit_null) {
+  const uint8_t* bitmap = NULLPTR;
+  if (bitmap_buf != NULLPTR) {
+    bitmap = bitmap_buf->data();
+  }
+  internal::OptionalBitBlockCounter bit_counter(bitmap, offset, length);
+  int64_t position = 0;
+  while (position < length) {
+    internal::BitBlockCount block = bit_counter.NextBlock();
+    if (block.AllSet()) {
+      for (int64_t i = 0; i < block.length; ++i, ++position) {
+        visit_not_null(position);
+      }
+    } else if (block.NoneSet()) {
+      for (int64_t i = 0; i < block.length; ++i, ++position) {
+        visit_null();
+      }
+    } else {
+      for (int64_t i = 0; i < block.length; ++i, ++position) {
+        if (BitUtil::GetBit(bitmap, offset + position)) {
+          visit_not_null(position);
+        } else {
+          visit_null();
+        }
+      }
+    }
+  }
+}
+
+template <typename VisitNotNull, typename VisitNull>
+void VisitTwoBitBlocksVoid(const std::shared_ptr<Buffer>& left_bitmap_buf,
+                           int64_t left_offset,
+                           const std::shared_ptr<Buffer>& right_bitmap_buf,
+                           int64_t right_offset, int64_t length,
+                           VisitNotNull&& visit_not_null, VisitNull&& visit_null) {
+  if (left_bitmap_buf == NULLPTR || right_bitmap_buf == NULLPTR) {
+    // At most one bitmap is present
+    if (left_bitmap_buf == NULLPTR) {
+      return VisitBitBlocksVoid(right_bitmap_buf, right_offset, length,
+                                std::forward<VisitNotNull>(visit_not_null),
+                                std::forward<VisitNull>(visit_null));
+    } else {
+      return VisitBitBlocksVoid(left_bitmap_buf, left_offset, length,
+                                std::forward<VisitNotNull>(visit_not_null),
+                                std::forward<VisitNull>(visit_null));
+    }
+  }
+  // Both bitmaps are present
+  const uint8_t* left_bitmap = left_bitmap_buf->data();
+  const uint8_t* right_bitmap = right_bitmap_buf->data();
+  BinaryBitBlockCounter bit_counter(left_bitmap, left_offset, right_bitmap, right_offset,
+                                    length);
+  int64_t position = 0;
+  while (position < length) {
+    BitBlockCount block = bit_counter.NextAndWord();
+    if (block.AllSet()) {
+      for (int64_t i = 0; i < block.length; ++i, ++position) {
+        visit_not_null(position);
+      }
+    } else if (block.NoneSet()) {
+      for (int64_t i = 0; i < block.length; ++i, ++position) {
+        visit_null();
+      }
+    } else {
+      for (int64_t i = 0; i < block.length; ++i, ++position) {
+        if (BitUtil::GetBit(left_bitmap, left_offset + position) &&
+            BitUtil::GetBit(right_bitmap, right_offset + position)) {
+          visit_not_null(position);
+        } else {
+          visit_null();
+        }
+      }
+    }
+  }
+}
+
 }  // namespace internal
 }  // namespace arrow
diff --git a/cpp/src/arrow/util/bit_block_counter_test.cc b/cpp/src/arrow/util/bit_block_counter_test.cc
index 924222c..3fdfa3e 100644
--- a/cpp/src/arrow/util/bit_block_counter_test.cc
+++ b/cpp/src/arrow/util/bit_block_counter_test.cc
@@ -321,5 +321,97 @@ TEST(TestOptionalBitBlockCounter, NextWord) {
   ASSERT_EQ(optional_block.popcount, 0);
 }
 
+class TestOptionalBinaryBitBlockCounter : public ::testing::Test {
+ public:
+  void SetUp() {
+    const int64_t nbytes = 5000;
+    ASSERT_OK_AND_ASSIGN(left_bitmap_, AllocateBitmap(nbytes * 8));
+    ASSERT_OK_AND_ASSIGN(right_bitmap_, AllocateBitmap(nbytes * 8));
+    random_bytes(nbytes, 0, left_bitmap_->mutable_data());
+    random_bytes(nbytes, 0, right_bitmap_->mutable_data());
+
+    left_offset_ = 12;
+    right_offset_ = 23;
+    length_ = nbytes * 8 - std::max(left_offset_, right_offset_);
+  }
+
+ protected:
+  std::shared_ptr<Buffer> left_bitmap_, right_bitmap_;
+  int64_t left_offset_;
+  int64_t right_offset_;
+  int64_t length_;
+};
+
+TEST_F(TestOptionalBinaryBitBlockCounter, NextBlockBothBitmaps) {
+  // Both bitmaps present
+  OptionalBinaryBitBlockCounter optional_counter(left_bitmap_, left_offset_,
+                                                 right_bitmap_, right_offset_, length_);
+  BinaryBitBlockCounter bit_counter(left_bitmap_->data(), left_offset_,
+                                    right_bitmap_->data(), right_offset_, length_);
+
+  while (true) {
+    BitBlockCount block = bit_counter.NextAndWord();
+    BitBlockCount optional_block = optional_counter.NextAndBlock();
+    ASSERT_EQ(optional_block.length, block.length);
+    ASSERT_EQ(optional_block.popcount, block.popcount);
+    if (block.length == 0) {
+      break;
+    }
+  }
+}
+
+TEST_F(TestOptionalBinaryBitBlockCounter, NextBlockLeftBitmap) {
+  // Left bitmap present
+  OptionalBinaryBitBlockCounter optional_counter(left_bitmap_, left_offset_, nullptr,
+                                                 right_offset_, length_);
+  BitBlockCounter bit_counter(left_bitmap_->data(), left_offset_, length_);
+
+  while (true) {
+    BitBlockCount block = bit_counter.NextWord();
+    BitBlockCount optional_block = optional_counter.NextAndBlock();
+    ASSERT_EQ(optional_block.length, block.length);
+    ASSERT_EQ(optional_block.popcount, block.popcount);
+    if (block.length == 0) {
+      break;
+    }
+  }
+}
+
+TEST_F(TestOptionalBinaryBitBlockCounter, NextBlockRightBitmap) {
+  // Right bitmap present
+  OptionalBinaryBitBlockCounter optional_counter(nullptr, left_offset_, right_bitmap_,
+                                                 right_offset_, length_);
+  BitBlockCounter bit_counter(right_bitmap_->data(), right_offset_, length_);
+
+  while (true) {
+    BitBlockCount block = bit_counter.NextWord();
+    BitBlockCount optional_block = optional_counter.NextAndBlock();
+    ASSERT_EQ(optional_block.length, block.length);
+    ASSERT_EQ(optional_block.popcount, block.popcount);
+    if (block.length == 0) {
+      break;
+    }
+  }
+}
+
+TEST_F(TestOptionalBinaryBitBlockCounter, NextBlockNoBitmap) {
+  // No bitmap present
+  OptionalBinaryBitBlockCounter optional_counter(nullptr, left_offset_, nullptr,
+                                                 right_offset_, length_);
+
+  BitBlockCount block = optional_counter.NextAndBlock();
+  ASSERT_EQ(block.length, std::numeric_limits<int16_t>::max());
+  ASSERT_EQ(block.popcount, block.length);
+
+  const int64_t remaining_length = length_ - block.length;
+  block = optional_counter.NextAndBlock();
+  ASSERT_EQ(block.length, remaining_length);
+  ASSERT_EQ(block.popcount, block.length);
+
+  block = optional_counter.NextAndBlock();
+  ASSERT_EQ(block.length, 0);
+  ASSERT_EQ(block.popcount, 0);
+}
+
 }  // namespace internal
 }  // namespace arrow
diff --git a/cpp/src/arrow/util/decimal.cc b/cpp/src/arrow/util/decimal.cc
index 20331ba..c9c63f8 100644
--- a/cpp/src/arrow/util/decimal.cc
+++ b/cpp/src/arrow/util/decimal.cc
@@ -31,7 +31,7 @@
 #include "arrow/status.h"
 #include "arrow/util/bit_util.h"
 #include "arrow/util/decimal.h"
-#include "arrow/util/int_util.h"
+#include "arrow/util/int_util_internal.h"
 #include "arrow/util/logging.h"
 #include "arrow/util/macros.h"
 #include "arrow/util/value_parsing.h"
diff --git a/cpp/src/arrow/util/int_util.h b/cpp/src/arrow/util/int_util.h
index 5a2be72..11fd974 100644
--- a/cpp/src/arrow/util/int_util.h
+++ b/cpp/src/arrow/util/int_util.h
@@ -18,8 +18,6 @@
 #pragma once
 
 #include <cstdint>
-#include <limits>
-#include <type_traits>
 
 #include "arrow/status.h"
 #include "arrow/util/visibility.h"
@@ -75,98 +73,6 @@ template <typename InputInt, typename OutputInt>
 ARROW_EXPORT void TransposeInts(const InputInt* source, OutputInt* dest, int64_t length,
                                 const int32_t* transpose_map);
 
-/// Signed addition with well-defined behaviour on overflow (as unsigned)
-template <typename SignedInt>
-SignedInt SafeSignedAdd(SignedInt u, SignedInt v) {
-  using UnsignedInt = typename std::make_unsigned<SignedInt>::type;
-  return static_cast<SignedInt>(static_cast<UnsignedInt>(u) +
-                                static_cast<UnsignedInt>(v));
-}
-
-/// Signed subtraction with well-defined behaviour on overflow (as unsigned)
-template <typename SignedInt>
-SignedInt SafeSignedSubtract(SignedInt u, SignedInt v) {
-  using UnsignedInt = typename std::make_unsigned<SignedInt>::type;
-  return static_cast<SignedInt>(static_cast<UnsignedInt>(u) -
-                                static_cast<UnsignedInt>(v));
-}
-
-/// Signed left shift with well-defined behaviour on negative numbers or overflow
-template <typename SignedInt, typename Shift>
-SignedInt SafeLeftShift(SignedInt u, Shift shift) {
-  using UnsignedInt = typename std::make_unsigned<SignedInt>::type;
-  return static_cast<SignedInt>(static_cast<UnsignedInt>(u) << shift);
-}
-
-// TODO Add portable wrappers for __builtin_add_overflow and friends
-// see http://www.open-std.org/jtc1/sc22/wg14/www/docs/n2428.pdf
-
-/// Detect multiplication overflow between *positive* integers
-template <typename Integer>
-bool HasPositiveMultiplyOverflow(Integer value, Integer multiplicand) {
-  return (multiplicand != 0 &&
-          value > std::numeric_limits<Integer>::max() / multiplicand);
-}
-
-/// Detect addition overflow between *positive* integers
-template <typename Integer>
-bool HasPositiveAdditionOverflow(Integer value, Integer addend) {
-  return (value > std::numeric_limits<Integer>::max() - addend);
-}
-
-/// Detect addition overflow between signed integers
-template <typename Integer>
-bool HasSignedAdditionOverflow(Integer value, Integer addend) {
-  return (addend > 0) ? (value > std::numeric_limits<Integer>::max() - addend)
-                      : (value < std::numeric_limits<Integer>::min() - addend);
-}
-
-/// Detect subtraction overflow between *positive* integers
-template <typename Integer>
-bool HasPositiveSubtractionOverflow(Integer value, Integer minuend) {
-  return (value < minuend);
-}
-
-/// Detect subtraction overflow between signed integers
-template <typename Integer>
-bool HasSignedSubtractionOverflow(Integer value, Integer subtrahend) {
-  return (subtrahend > 0) ? (value < std::numeric_limits<Integer>::min() + subtrahend)
-                          : (value > std::numeric_limits<Integer>::max() + subtrahend);
-}
-
-/// Upcast an integer to the largest possible width (currently 64 bits)
-
-template <typename Integer>
-typename std::enable_if<
-    std::is_integral<Integer>::value && std::is_signed<Integer>::value, int64_t>::type
-UpcastInt(Integer v) {
-  return v;
-}
-
-template <typename Integer>
-typename std::enable_if<
-    std::is_integral<Integer>::value && std::is_unsigned<Integer>::value, uint64_t>::type
-UpcastInt(Integer v) {
-  return v;
-}
-
-static inline Status CheckSliceParams(int64_t object_length, int64_t slice_offset,
-                                      int64_t slice_length, const char* object_name) {
-  if (slice_offset < 0) {
-    return Status::Invalid("Negative ", object_name, " slice offset");
-  }
-  if (slice_length < 0) {
-    return Status::Invalid("Negative ", object_name, " slice length");
-  }
-  if (internal::HasPositiveAdditionOverflow(slice_offset, slice_length)) {
-    return Status::Invalid(object_name, " slice would overflow");
-  }
-  if (slice_offset + slice_length > object_length) {
-    return Status::Invalid(object_name, " slice would exceed ", object_name, " length");
-  }
-  return Status::OK();
-}
-
 /// \brief Do vectorized boundschecking of integer-type array indices. The
 /// indices must be non-nonnegative and strictly less than the passed upper
 /// limit (which is usually the length of an array that is being indexed-into).
diff --git a/cpp/src/arrow/util/int_util_internal.h b/cpp/src/arrow/util/int_util_internal.h
new file mode 100644
index 0000000..5de92a9
--- /dev/null
+++ b/cpp/src/arrow/util/int_util_internal.h
@@ -0,0 +1,124 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#pragma once
+
+#include <cstdint>
+#include <limits>
+#include <type_traits>
+
+#include "arrow/status.h"
+#include "arrow/util/macros.h"
+#include "arrow/util/visibility.h"
+
+// "safe-math.h" includes <intsafe.h> from the Windows headers.
+#include "arrow/util/windows_compatibility.h"
+#include "arrow/vendored/portable-snippets/safe-math.h"
+// clang-format off (avoid include reordering)
+#include "arrow/util/windows_fixup.h"
+// clang-format on
+
+namespace arrow {
+namespace internal {
+
+// Define functions AddWithOverflow, SubtractWithOverflow, MultiplyWithOverflow
+// with the signature `bool(T u, T v, T* out)` where T is an integer type.
+// On overflow, these functions return true.  Otherwise, false is returned
+// and `out` is updated with the result of the operation.
+
+#define OP_WITH_OVERFLOW(_func_name, _psnip_op, _type, _psnip_type) \
+  static inline bool _func_name(_type u, _type v, _type* out) {     \
+    return !psnip_safe_##_psnip_type##_##_psnip_op(out, u, v);      \
+  }
+
+#define OPS_WITH_OVERFLOW(_func_name, _psnip_op)            \
+  OP_WITH_OVERFLOW(_func_name, _psnip_op, int8_t, int8)     \
+  OP_WITH_OVERFLOW(_func_name, _psnip_op, int16_t, int16)   \
+  OP_WITH_OVERFLOW(_func_name, _psnip_op, int32_t, int32)   \
+  OP_WITH_OVERFLOW(_func_name, _psnip_op, int64_t, int64)   \
+  OP_WITH_OVERFLOW(_func_name, _psnip_op, uint8_t, uint8)   \
+  OP_WITH_OVERFLOW(_func_name, _psnip_op, uint16_t, uint16) \
+  OP_WITH_OVERFLOW(_func_name, _psnip_op, uint32_t, uint32) \
+  OP_WITH_OVERFLOW(_func_name, _psnip_op, uint64_t, uint64)
+
+OPS_WITH_OVERFLOW(AddWithOverflow, add)
+OPS_WITH_OVERFLOW(SubtractWithOverflow, sub)
+OPS_WITH_OVERFLOW(MultiplyWithOverflow, mul)
+
+#undef OP_WITH_OVERFLOW
+#undef OPS_WITH_OVERFLOW
+
+/// Signed addition with well-defined behaviour on overflow (as unsigned)
+template <typename SignedInt>
+SignedInt SafeSignedAdd(SignedInt u, SignedInt v) {
+  using UnsignedInt = typename std::make_unsigned<SignedInt>::type;
+  return static_cast<SignedInt>(static_cast<UnsignedInt>(u) +
+                                static_cast<UnsignedInt>(v));
+}
+
+/// Signed subtraction with well-defined behaviour on overflow (as unsigned)
+template <typename SignedInt>
+SignedInt SafeSignedSubtract(SignedInt u, SignedInt v) {
+  using UnsignedInt = typename std::make_unsigned<SignedInt>::type;
+  return static_cast<SignedInt>(static_cast<UnsignedInt>(u) -
+                                static_cast<UnsignedInt>(v));
+}
+
+/// Signed left shift with well-defined behaviour on negative numbers or overflow
+template <typename SignedInt, typename Shift>
+SignedInt SafeLeftShift(SignedInt u, Shift shift) {
+  using UnsignedInt = typename std::make_unsigned<SignedInt>::type;
+  return static_cast<SignedInt>(static_cast<UnsignedInt>(u) << shift);
+}
+
+/// Upcast an integer to the largest possible width (currently 64 bits)
+
+template <typename Integer>
+typename std::enable_if<
+    std::is_integral<Integer>::value && std::is_signed<Integer>::value, int64_t>::type
+UpcastInt(Integer v) {
+  return v;
+}
+
+template <typename Integer>
+typename std::enable_if<
+    std::is_integral<Integer>::value && std::is_unsigned<Integer>::value, uint64_t>::type
+UpcastInt(Integer v) {
+  return v;
+}
+
+static inline Status CheckSliceParams(int64_t object_length, int64_t slice_offset,
+                                      int64_t slice_length, const char* object_name) {
+  if (ARROW_PREDICT_FALSE(slice_offset < 0)) {
+    return Status::Invalid("Negative ", object_name, " slice offset");
+  }
+  if (ARROW_PREDICT_FALSE(slice_length < 0)) {
+    return Status::Invalid("Negative ", object_name, " slice length");
+  }
+  int64_t offset_plus_length;
+  if (ARROW_PREDICT_FALSE(
+          internal::AddWithOverflow(slice_offset, slice_length, &offset_plus_length))) {
+    return Status::Invalid(object_name, " slice would overflow");
+  }
+  if (ARROW_PREDICT_FALSE(slice_offset + slice_length > object_length)) {
+    return Status::Invalid(object_name, " slice would exceed ", object_name, " length");
+  }
+  return Status::OK();
+}
+
+}  // namespace internal
+}  // namespace arrow
diff --git a/cpp/src/arrow/util/int_util_test.cc b/cpp/src/arrow/util/int_util_test.cc
index 6cb0e25..333154c 100644
--- a/cpp/src/arrow/util/int_util_test.cc
+++ b/cpp/src/arrow/util/int_util_test.cc
@@ -29,6 +29,7 @@
 #include "arrow/testing/random.h"
 #include "arrow/type.h"
 #include "arrow/util/int_util.h"
+#include "arrow/util/int_util_internal.h"
 
 namespace arrow {
 namespace internal {
diff --git a/cpp/src/arrow/util/windows_fixup.h b/cpp/src/arrow/util/windows_fixup.h
index 80c39c7..0afa53c 100644
--- a/cpp/src/arrow/util/windows_fixup.h
+++ b/cpp/src/arrow/util/windows_fixup.h
@@ -31,4 +31,15 @@
 #undef DeleteFile
 #endif
 
+// Other annoying Windows macro definitions...
+#ifdef IN
+#undef IN
+#endif
+#ifdef OUT
+#undef OUT
+#endif
+
+// Note that we can't undefine OPTIONAL, because it can be used in other
+// Windows headers...
+
 #endif  // _WIN32
diff --git a/cpp/src/arrow/vendored/portable-snippets/README.md b/cpp/src/arrow/vendored/portable-snippets/README.md
new file mode 100644
index 0000000..9c67b7b
--- /dev/null
+++ b/cpp/src/arrow/vendored/portable-snippets/README.md
@@ -0,0 +1,10 @@
+<!---
+Each source file contains a preamble explaining the license situation
+for that file, which takes priority over this file.  With the
+exception of some code pulled in from other repositories (such as
+µnit, an MIT-licensed project which is used for testing), the code is
+public domain, released using the CC0 1.0 Universal dedication.
+-->
+
+The files in this directory are vendored from portable-snippets
+git changeset f596f8b0a4b8a6ea1166c2361a5cb7e6f802c5ea.
diff --git a/cpp/src/arrow/vendored/portable-snippets/safe-math.h b/cpp/src/arrow/vendored/portable-snippets/safe-math.h
new file mode 100644
index 0000000..670b539
--- /dev/null
+++ b/cpp/src/arrow/vendored/portable-snippets/safe-math.h
@@ -0,0 +1,1071 @@
+/* Overflow-safe math functions
+ * Portable Snippets - https://github.com/nemequ/portable-snippets
+ * Created by Evan Nemerson <ev...@nemerson.com>
+ *
+ *   To the extent possible under law, the authors have waived all
+ *   copyright and related or neighboring rights to this code.  For
+ *   details, see the Creative Commons Zero 1.0 Universal license at
+ *   https://creativecommons.org/publicdomain/zero/1.0/
+ */
+
+#if !defined(PSNIP_SAFE_H)
+#define PSNIP_SAFE_H
+
+#if !defined(PSNIP_SAFE_FORCE_PORTABLE)
+#  if defined(__has_builtin)
+#    if __has_builtin(__builtin_add_overflow) && !defined(__ibmxl__)
+#      define PSNIP_SAFE_HAVE_BUILTIN_OVERFLOW
+#    endif
+#  elif defined(__GNUC__) && (__GNUC__ >= 5) && !defined(__INTEL_COMPILER)
+#    define PSNIP_SAFE_HAVE_BUILTIN_OVERFLOW
+#  endif
+#  if defined(__has_include)
+#    if __has_include(<intsafe.h>)
+#      define PSNIP_SAFE_HAVE_INTSAFE_H
+#    endif
+#  elif defined(_WIN32)
+#    define PSNIP_SAFE_HAVE_INTSAFE_H
+#  endif
+#endif /* !defined(PSNIP_SAFE_FORCE_PORTABLE) */
+
+#if defined(__GNUC__)
+#  define PSNIP_SAFE_LIKELY(expr)   __builtin_expect(!!(expr), 1)
+#  define PSNIP_SAFE_UNLIKELY(expr) __builtin_expect(!!(expr), 0)
+#else
+#  define PSNIP_SAFE_LIKELY(expr) !!(expr)
+#  define PSNIP_SAFE_UNLIKELY(expr) !!(expr)
+#endif /* defined(__GNUC__) */
+
+#if !defined(PSNIP_SAFE_STATIC_INLINE)
+#  if defined(__GNUC__)
+#    define PSNIP_SAFE__COMPILER_ATTRIBUTES __attribute__((__unused__))
+#  else
+#    define PSNIP_SAFE__COMPILER_ATTRIBUTES
+#  endif
+
+#  if defined(HEDLEY_INLINE)
+#    define PSNIP_SAFE__INLINE HEDLEY_INLINE
+#  elif defined(__STDC_VERSION__) && __STDC_VERSION__ >= 199901L
+#    define PSNIP_SAFE__INLINE inline
+#  elif defined(__GNUC_STDC_INLINE__)
+#    define PSNIP_SAFE__INLINE __inline__
+#  elif defined(_MSC_VER) && _MSC_VER >= 1200
+#    define PSNIP_SAFE__INLINE __inline
+#  else
+#    define PSNIP_SAFE__INLINE
+#  endif
+
+#  define PSNIP_SAFE__FUNCTION PSNIP_SAFE__COMPILER_ATTRIBUTES static PSNIP_SAFE__INLINE
+#endif
+
+#if defined(__STDC_VERSION__) && __STDC_VERSION__ >= 199901L
+#  define psnip_safe_bool _Bool
+#else
+#  define psnip_safe_bool int
+#endif
+
+#if !defined(PSNIP_SAFE_NO_FIXED)
+/* For maximum portability include the exact-int module from
+   portable snippets. */
+#  if \
+    !defined(psnip_int64_t) || !defined(psnip_uint64_t) || \
+    !defined(psnip_int32_t) || !defined(psnip_uint32_t) || \
+    !defined(psnip_int16_t) || !defined(psnip_uint16_t) || \
+    !defined(psnip_int8_t)  || !defined(psnip_uint8_t)
+#    include <stdint.h>
+#    if !defined(psnip_int64_t)
+#      define psnip_int64_t int64_t
+#    endif
+#    if !defined(psnip_uint64_t)
+#      define psnip_uint64_t uint64_t
+#    endif
+#    if !defined(psnip_int32_t)
+#      define psnip_int32_t int32_t
+#    endif
+#    if !defined(psnip_uint32_t)
+#      define psnip_uint32_t uint32_t
+#    endif
+#    if !defined(psnip_int16_t)
+#      define psnip_int16_t int16_t
+#    endif
+#    if !defined(psnip_uint16_t)
+#      define psnip_uint16_t uint16_t
+#    endif
+#    if !defined(psnip_int8_t)
+#      define psnip_int8_t int8_t
+#    endif
+#    if !defined(psnip_uint8_t)
+#      define psnip_uint8_t uint8_t
+#    endif
+#  endif
+#endif /* !defined(PSNIP_SAFE_NO_FIXED) */
+#include <limits.h>
+#include <stdlib.h>
+
+#if !defined(PSNIP_SAFE_SIZE_MAX)
+#  if defined(__SIZE_MAX__)
+#    define PSNIP_SAFE_SIZE_MAX __SIZE_MAX__
+#  elif defined(PSNIP_EXACT_INT_HAVE_STDINT)
+#    include <stdint.h>
+#  endif
+#endif
+
+#if defined(PSNIP_SAFE_SIZE_MAX)
+#  define PSNIP_SAFE__SIZE_MAX_RT PSNIP_SAFE_SIZE_MAX
+#else
+#  define PSNIP_SAFE__SIZE_MAX_RT (~((size_t) 0))
+#endif
+
+#if defined(PSNIP_SAFE_HAVE_INTSAFE_H)
+/* In VS 10, stdint.h and intsafe.h both define (U)INTN_MIN/MAX, which
+   triggers warning C4005 (level 1). */
+#  if defined(_MSC_VER) && (_MSC_VER == 1600)
+#    pragma warning(push)
+#    pragma warning(disable:4005)
+#  endif
+#  include <intsafe.h>
+#  if defined(_MSC_VER) && (_MSC_VER == 1600)
+#    pragma warning(pop)
+#  endif
+#endif /* defined(PSNIP_SAFE_HAVE_INTSAFE_H) */
+
+/* If there is a type larger than the one we're concerned with it's
+ * likely much faster to simply promote the operands, perform the
+ * requested operation, verify that the result falls within the
+ * original type, then cast the result back to the original type. */
+
+#if !defined(PSNIP_SAFE_NO_PROMOTIONS)
+
+#define PSNIP_SAFE_DEFINE_LARGER_BINARY_OP(T, name, op_name, op) \
+  PSNIP_SAFE__FUNCTION psnip_safe_##name##_larger \
+  psnip_safe_larger_##name##_##op_name (T a, T b) { \
+    return ((psnip_safe_##name##_larger) a) op ((psnip_safe_##name##_larger) b); \
+  }
+
+#define PSNIP_SAFE_DEFINE_LARGER_UNARY_OP(T, name, op_name, op) \
+  PSNIP_SAFE__FUNCTION psnip_safe_##name##_larger \
+  psnip_safe_larger_##name##_##op_name (T value) { \
+    return (op ((psnip_safe_##name##_larger) value)); \
+  }
+
+#define PSNIP_SAFE_DEFINE_LARGER_SIGNED_OPS(T, name) \
+  PSNIP_SAFE_DEFINE_LARGER_BINARY_OP(T, name, add, +) \
+  PSNIP_SAFE_DEFINE_LARGER_BINARY_OP(T, name, sub, -) \
+  PSNIP_SAFE_DEFINE_LARGER_BINARY_OP(T, name, mul, *) \
+  PSNIP_SAFE_DEFINE_LARGER_BINARY_OP(T, name, div, /) \
+  PSNIP_SAFE_DEFINE_LARGER_BINARY_OP(T, name, mod, %) \
+  PSNIP_SAFE_DEFINE_LARGER_UNARY_OP (T, name, neg, -)
+
+#define PSNIP_SAFE_DEFINE_LARGER_UNSIGNED_OPS(T, name) \
+  PSNIP_SAFE_DEFINE_LARGER_BINARY_OP(T, name, add, +) \
+  PSNIP_SAFE_DEFINE_LARGER_BINARY_OP(T, name, sub, -) \
+  PSNIP_SAFE_DEFINE_LARGER_BINARY_OP(T, name, mul, *) \
+  PSNIP_SAFE_DEFINE_LARGER_BINARY_OP(T, name, div, /) \
+  PSNIP_SAFE_DEFINE_LARGER_BINARY_OP(T, name, mod, %)
+
+#define PSNIP_SAFE_IS_LARGER(ORIG_MAX, DEST_MAX) ((DEST_MAX / ORIG_MAX) >= ORIG_MAX)
+
+#if defined(__GNUC__) && ((__GNUC__ >= 4) || (__GNUC__ == 4 && __GNUC_MINOR__ >= 6)) && defined(__SIZEOF_INT128__) && !defined(__ibmxl__)
+#define PSNIP_SAFE_HAVE_128
+typedef __int128  psnip_safe_int128_t;
+typedef unsigned __int128 psnip_safe_uint128_t;
+#endif /* defined(__GNUC__) */
+
+#if !defined(PSNIP_SAFE_NO_FIXED)
+#define PSNIP_SAFE_HAVE_INT8_LARGER
+#define PSNIP_SAFE_HAVE_UINT8_LARGER
+typedef psnip_int16_t  psnip_safe_int8_larger;
+typedef psnip_uint16_t psnip_safe_uint8_larger;
+
+#define PSNIP_SAFE_HAVE_INT16_LARGER
+typedef psnip_int32_t  psnip_safe_int16_larger;
+typedef psnip_uint32_t psnip_safe_uint16_larger;
+
+#define PSNIP_SAFE_HAVE_INT32_LARGER
+typedef psnip_int64_t  psnip_safe_int32_larger;
+typedef psnip_uint64_t psnip_safe_uint32_larger;
+
+#if defined(PSNIP_SAFE_HAVE_128)
+#define PSNIP_SAFE_HAVE_INT64_LARGER
+typedef psnip_safe_int128_t psnip_safe_int64_larger;
+typedef psnip_safe_uint128_t psnip_safe_uint64_larger;
+#endif /* defined(PSNIP_SAFE_HAVE_128) */
+#endif /* !defined(PSNIP_SAFE_NO_FIXED) */
+
+#define PSNIP_SAFE_HAVE_LARGER_SCHAR
+#if PSNIP_SAFE_IS_LARGER(SCHAR_MAX, SHRT_MAX)
+typedef short psnip_safe_schar_larger;
+#elif PSNIP_SAFE_IS_LARGER(SCHAR_MAX, INT_MAX)
+typedef int psnip_safe_schar_larger;
+#elif PSNIP_SAFE_IS_LARGER(SCHAR_MAX, LONG_MAX)
+typedef long psnip_safe_schar_larger;
+#elif PSNIP_SAFE_IS_LARGER(SCHAR_MAX, LLONG_MAX)
+typedef long long psnip_safe_schar_larger;
+#elif !defined(PSNIP_SAFE_NO_FIXED) && PSNIP_SAFE_IS_LARGER(SCHAR_MAX, 0x7fff)
+typedef psnip_int16_t psnip_safe_schar_larger;
+#elif !defined(PSNIP_SAFE_NO_FIXED) && PSNIP_SAFE_IS_LARGER(SCHAR_MAX, 0x7fffffffLL)
+typedef psnip_int32_t psnip_safe_schar_larger;
+#elif !defined(PSNIP_SAFE_NO_FIXED) && PSNIP_SAFE_IS_LARGER(SCHAR_MAX, 0x7fffffffffffffffLL)
+typedef psnip_int64_t psnip_safe_schar_larger;
+#elif !defined(PSNIP_SAFE_NO_FIXED) && defined(PSNIP_SAFE_HAVE_128) && (SCHAR_MAX <= 0x7fffffffffffffffLL)
+typedef psnip_safe_int128_t psnip_safe_schar_larger;
+#else
+#undef PSNIP_SAFE_HAVE_LARGER_SCHAR
+#endif
+
+#define PSNIP_SAFE_HAVE_LARGER_UCHAR
+#if PSNIP_SAFE_IS_LARGER(UCHAR_MAX, USHRT_MAX)
+typedef unsigned short psnip_safe_uchar_larger;
+#elif PSNIP_SAFE_IS_LARGER(UCHAR_MAX, UINT_MAX)
+typedef unsigned int psnip_safe_uchar_larger;
+#elif PSNIP_SAFE_IS_LARGER(UCHAR_MAX, ULONG_MAX)
+typedef unsigned long psnip_safe_uchar_larger;
+#elif PSNIP_SAFE_IS_LARGER(UCHAR_MAX, ULLONG_MAX)
+typedef unsigned long long psnip_safe_uchar_larger;
+#elif !defined(PSNIP_SAFE_NO_FIXED) && PSNIP_SAFE_IS_LARGER(UCHAR_MAX, 0xffffU)
+typedef psnip_uint16_t psnip_safe_uchar_larger;
+#elif !defined(PSNIP_SAFE_NO_FIXED) && PSNIP_SAFE_IS_LARGER(UCHAR_MAX, 0xffffffffUL)
+typedef psnip_uint32_t psnip_safe_uchar_larger;
+#elif !defined(PSNIP_SAFE_NO_FIXED) && PSNIP_SAFE_IS_LARGER(UCHAR_MAX, 0xffffffffffffffffULL)
+typedef psnip_uint64_t psnip_safe_uchar_larger;
+#elif !defined(PSNIP_SAFE_NO_FIXED) && defined(PSNIP_SAFE_HAVE_128) && (UCHAR_MAX <= 0xffffffffffffffffULL)
+typedef psnip_safe_uint128_t psnip_safe_uchar_larger;
+#else
+#undef PSNIP_SAFE_HAVE_LARGER_UCHAR
+#endif
+
+#if CHAR_MIN == 0 && defined(PSNIP_SAFE_HAVE_LARGER_UCHAR)
+#define PSNIP_SAFE_HAVE_LARGER_CHAR
+typedef psnip_safe_uchar_larger psnip_safe_char_larger;
+#elif CHAR_MIN < 0 && defined(PSNIP_SAFE_HAVE_LARGER_SCHAR)
+#define PSNIP_SAFE_HAVE_LARGER_CHAR
+typedef psnip_safe_schar_larger psnip_safe_char_larger;
+#endif
+
+#define PSNIP_SAFE_HAVE_LARGER_SHRT
+#if PSNIP_SAFE_IS_LARGER(SHRT_MAX, INT_MAX)
+typedef int psnip_safe_short_larger;
+#elif PSNIP_SAFE_IS_LARGER(SHRT_MAX, LONG_MAX)
+typedef long psnip_safe_short_larger;
+#elif PSNIP_SAFE_IS_LARGER(SHRT_MAX, LLONG_MAX)
+typedef long long psnip_safe_short_larger;
+#elif !defined(PSNIP_SAFE_NO_FIXED) && PSNIP_SAFE_IS_LARGER(SHRT_MAX, 0x7fff)
+typedef psnip_int16_t psnip_safe_short_larger;
+#elif !defined(PSNIP_SAFE_NO_FIXED) && PSNIP_SAFE_IS_LARGER(SHRT_MAX, 0x7fffffffLL)
+typedef psnip_int32_t psnip_safe_short_larger;
+#elif !defined(PSNIP_SAFE_NO_FIXED) && PSNIP_SAFE_IS_LARGER(SHRT_MAX, 0x7fffffffffffffffLL)
+typedef psnip_int64_t psnip_safe_short_larger;
+#elif !defined(PSNIP_SAFE_NO_FIXED) && defined(PSNIP_SAFE_HAVE_128) && (SHRT_MAX <= 0x7fffffffffffffffLL)
+typedef psnip_safe_int128_t psnip_safe_short_larger;
+#else
+#undef PSNIP_SAFE_HAVE_LARGER_SHRT
+#endif
+
+#define PSNIP_SAFE_HAVE_LARGER_USHRT
+#if PSNIP_SAFE_IS_LARGER(USHRT_MAX, UINT_MAX)
+typedef unsigned int psnip_safe_ushort_larger;
+#elif PSNIP_SAFE_IS_LARGER(USHRT_MAX, ULONG_MAX)
+typedef unsigned long psnip_safe_ushort_larger;
+#elif PSNIP_SAFE_IS_LARGER(USHRT_MAX, ULLONG_MAX)
+typedef unsigned long long psnip_safe_ushort_larger;
+#elif !defined(PSNIP_SAFE_NO_FIXED) && PSNIP_SAFE_IS_LARGER(USHRT_MAX, 0xffff)
+typedef psnip_uint16_t psnip_safe_ushort_larger;
+#elif !defined(PSNIP_SAFE_NO_FIXED) && PSNIP_SAFE_IS_LARGER(USHRT_MAX, 0xffffffffUL)
+typedef psnip_uint32_t psnip_safe_ushort_larger;
+#elif !defined(PSNIP_SAFE_NO_FIXED) && PSNIP_SAFE_IS_LARGER(USHRT_MAX, 0xffffffffffffffffULL)
+typedef psnip_uint64_t psnip_safe_ushort_larger;
+#elif !defined(PSNIP_SAFE_NO_FIXED) && defined(PSNIP_SAFE_HAVE_128) && (USHRT_MAX <= 0xffffffffffffffffULL)
+typedef psnip_safe_uint128_t psnip_safe_ushort_larger;
+#else
+#undef PSNIP_SAFE_HAVE_LARGER_USHRT
+#endif
+
+#define PSNIP_SAFE_HAVE_LARGER_INT
+#if PSNIP_SAFE_IS_LARGER(INT_MAX, LONG_MAX)
+typedef long psnip_safe_int_larger;
+#elif PSNIP_SAFE_IS_LARGER(INT_MAX, LLONG_MAX)
+typedef long long psnip_safe_int_larger;
+#elif !defined(PSNIP_SAFE_NO_FIXED) && PSNIP_SAFE_IS_LARGER(INT_MAX, 0x7fff)
+typedef psnip_int16_t psnip_safe_int_larger;
+#elif !defined(PSNIP_SAFE_NO_FIXED) && PSNIP_SAFE_IS_LARGER(INT_MAX, 0x7fffffffLL)
+typedef psnip_int32_t psnip_safe_int_larger;
+#elif !defined(PSNIP_SAFE_NO_FIXED) && PSNIP_SAFE_IS_LARGER(INT_MAX, 0x7fffffffffffffffLL)
+typedef psnip_int64_t psnip_safe_int_larger;
+#elif !defined(PSNIP_SAFE_NO_FIXED) && defined(PSNIP_SAFE_HAVE_128) && (INT_MAX <= 0x7fffffffffffffffLL)
+typedef psnip_safe_int128_t psnip_safe_int_larger;
+#else
+#undef PSNIP_SAFE_HAVE_LARGER_INT
+#endif
+
+#define PSNIP_SAFE_HAVE_LARGER_UINT
+#if PSNIP_SAFE_IS_LARGER(UINT_MAX, ULONG_MAX)
+typedef unsigned long psnip_safe_uint_larger;
+#elif PSNIP_SAFE_IS_LARGER(UINT_MAX, ULLONG_MAX)
+typedef unsigned long long psnip_safe_uint_larger;
+#elif !defined(PSNIP_SAFE_NO_FIXED) && PSNIP_SAFE_IS_LARGER(UINT_MAX, 0xffff)
+typedef psnip_uint16_t psnip_safe_uint_larger;
+#elif !defined(PSNIP_SAFE_NO_FIXED) && PSNIP_SAFE_IS_LARGER(UINT_MAX, 0xffffffffUL)
+typedef psnip_uint32_t psnip_safe_uint_larger;
+#elif !defined(PSNIP_SAFE_NO_FIXED) && PSNIP_SAFE_IS_LARGER(UINT_MAX, 0xffffffffffffffffULL)
+typedef psnip_uint64_t psnip_safe_uint_larger;
+#elif !defined(PSNIP_SAFE_NO_FIXED) && defined(PSNIP_SAFE_HAVE_128) && (UINT_MAX <= 0xffffffffffffffffULL)
+typedef psnip_safe_uint128_t psnip_safe_uint_larger;
+#else
+#undef PSNIP_SAFE_HAVE_LARGER_UINT
+#endif
+
+#define PSNIP_SAFE_HAVE_LARGER_LONG
+#if PSNIP_SAFE_IS_LARGER(LONG_MAX, LLONG_MAX)
+typedef long long psnip_safe_long_larger;
+#elif !defined(PSNIP_SAFE_NO_FIXED) && PSNIP_SAFE_IS_LARGER(LONG_MAX, 0x7fff)
+typedef psnip_int16_t psnip_safe_long_larger;
+#elif !defined(PSNIP_SAFE_NO_FIXED) && PSNIP_SAFE_IS_LARGER(LONG_MAX, 0x7fffffffLL)
+typedef psnip_int32_t psnip_safe_long_larger;
+#elif !defined(PSNIP_SAFE_NO_FIXED) && PSNIP_SAFE_IS_LARGER(LONG_MAX, 0x7fffffffffffffffLL)
+typedef psnip_int64_t psnip_safe_long_larger;
+#elif !defined(PSNIP_SAFE_NO_FIXED) && defined(PSNIP_SAFE_HAVE_128) && (LONG_MAX <= 0x7fffffffffffffffLL)
+typedef psnip_safe_int128_t psnip_safe_long_larger;
+#else
+#undef PSNIP_SAFE_HAVE_LARGER_LONG
+#endif
+
+#define PSNIP_SAFE_HAVE_LARGER_ULONG
+#if PSNIP_SAFE_IS_LARGER(ULONG_MAX, ULLONG_MAX)
+typedef unsigned long long psnip_safe_ulong_larger;
+#elif !defined(PSNIP_SAFE_NO_FIXED) && PSNIP_SAFE_IS_LARGER(ULONG_MAX, 0xffff)
+typedef psnip_uint16_t psnip_safe_ulong_larger;
+#elif !defined(PSNIP_SAFE_NO_FIXED) && PSNIP_SAFE_IS_LARGER(ULONG_MAX, 0xffffffffUL)
+typedef psnip_uint32_t psnip_safe_ulong_larger;
+#elif !defined(PSNIP_SAFE_NO_FIXED) && PSNIP_SAFE_IS_LARGER(ULONG_MAX, 0xffffffffffffffffULL)
+typedef psnip_uint64_t psnip_safe_ulong_larger;
+#elif !defined(PSNIP_SAFE_NO_FIXED) && defined(PSNIP_SAFE_HAVE_128) && (ULONG_MAX <= 0xffffffffffffffffULL)
+typedef psnip_safe_uint128_t psnip_safe_ulong_larger;
+#else
+#undef PSNIP_SAFE_HAVE_LARGER_ULONG
+#endif
+
+#define PSNIP_SAFE_HAVE_LARGER_LLONG
+#if !defined(PSNIP_SAFE_NO_FIXED) && PSNIP_SAFE_IS_LARGER(LLONG_MAX, 0x7fff)
+typedef psnip_int16_t psnip_safe_llong_larger;
+#elif !defined(PSNIP_SAFE_NO_FIXED) && PSNIP_SAFE_IS_LARGER(LLONG_MAX, 0x7fffffffLL)
+typedef psnip_int32_t psnip_safe_llong_larger;
+#elif !defined(PSNIP_SAFE_NO_FIXED) && PSNIP_SAFE_IS_LARGER(LLONG_MAX, 0x7fffffffffffffffLL)
+typedef psnip_int64_t psnip_safe_llong_larger;
+#elif !defined(PSNIP_SAFE_NO_FIXED) && defined(PSNIP_SAFE_HAVE_128) && (LLONG_MAX <= 0x7fffffffffffffffLL)
+typedef psnip_safe_int128_t psnip_safe_llong_larger;
+#else
+#undef PSNIP_SAFE_HAVE_LARGER_LLONG
+#endif
+
+#define PSNIP_SAFE_HAVE_LARGER_ULLONG
+#if !defined(PSNIP_SAFE_NO_FIXED) && PSNIP_SAFE_IS_LARGER(ULLONG_MAX, 0xffff)
+typedef psnip_uint16_t psnip_safe_ullong_larger;
+#elif !defined(PSNIP_SAFE_NO_FIXED) && PSNIP_SAFE_IS_LARGER(ULLONG_MAX, 0xffffffffUL)
+typedef psnip_uint32_t psnip_safe_ullong_larger;
+#elif !defined(PSNIP_SAFE_NO_FIXED) && PSNIP_SAFE_IS_LARGER(ULLONG_MAX, 0xffffffffffffffffULL)
+typedef psnip_uint64_t psnip_safe_ullong_larger;
+#elif !defined(PSNIP_SAFE_NO_FIXED) && defined(PSNIP_SAFE_HAVE_128) && (ULLONG_MAX <= 0xffffffffffffffffULL)
+typedef psnip_safe_uint128_t psnip_safe_ullong_larger;
+#else
+#undef PSNIP_SAFE_HAVE_LARGER_ULLONG
+#endif
+
+#if defined(PSNIP_SAFE_SIZE_MAX)
+#define PSNIP_SAFE_HAVE_LARGER_SIZE
+#if PSNIP_SAFE_IS_LARGER(PSNIP_SAFE_SIZE_MAX, USHRT_MAX)
+typedef unsigned short psnip_safe_size_larger;
+#elif PSNIP_SAFE_IS_LARGER(PSNIP_SAFE_SIZE_MAX, UINT_MAX)
+typedef unsigned int psnip_safe_size_larger;
+#elif PSNIP_SAFE_IS_LARGER(PSNIP_SAFE_SIZE_MAX, ULONG_MAX)
+typedef unsigned long psnip_safe_size_larger;
+#elif PSNIP_SAFE_IS_LARGER(PSNIP_SAFE_SIZE_MAX, ULLONG_MAX)
+typedef unsigned long long psnip_safe_size_larger;
+#elif !defined(PSNIP_SAFE_NO_FIXED) && PSNIP_SAFE_IS_LARGER(PSNIP_SAFE_SIZE_MAX, 0xffff)
+typedef psnip_uint16_t psnip_safe_size_larger;
+#elif !defined(PSNIP_SAFE_NO_FIXED) && PSNIP_SAFE_IS_LARGER(PSNIP_SAFE_SIZE_MAX, 0xffffffffUL)
+typedef psnip_uint32_t psnip_safe_size_larger;
+#elif !defined(PSNIP_SAFE_NO_FIXED) && PSNIP_SAFE_IS_LARGER(PSNIP_SAFE_SIZE_MAX, 0xffffffffffffffffULL)
+typedef psnip_uint64_t psnip_safe_size_larger;
+#elif !defined(PSNIP_SAFE_NO_FIXED) && defined(PSNIP_SAFE_HAVE_128) && (PSNIP_SAFE_SIZE_MAX <= 0xffffffffffffffffULL)
+typedef psnip_safe_uint128_t psnip_safe_size_larger;
+#else
+#undef PSNIP_SAFE_HAVE_LARGER_SIZE
+#endif
+#endif
+
+#if defined(PSNIP_SAFE_HAVE_LARGER_SCHAR)
+PSNIP_SAFE_DEFINE_LARGER_SIGNED_OPS(signed char, schar)
+#endif
+
+#if defined(PSNIP_SAFE_HAVE_LARGER_UCHAR)
+PSNIP_SAFE_DEFINE_LARGER_UNSIGNED_OPS(unsigned char, uchar)
+#endif
+
+#if defined(PSNIP_SAFE_HAVE_LARGER_CHAR)
+#if CHAR_MIN == 0
+PSNIP_SAFE_DEFINE_LARGER_UNSIGNED_OPS(char, char)
+#else
+PSNIP_SAFE_DEFINE_LARGER_SIGNED_OPS(char, char)
+#endif
+#endif
+
+#if defined(PSNIP_SAFE_HAVE_LARGER_SHORT)
+PSNIP_SAFE_DEFINE_LARGER_SIGNED_OPS(short, short)
+#endif
+
+#if defined(PSNIP_SAFE_HAVE_LARGER_USHORT)
+PSNIP_SAFE_DEFINE_LARGER_UNSIGNED_OPS(unsigned short, ushort)
+#endif
+
+#if defined(PSNIP_SAFE_HAVE_LARGER_INT)
+PSNIP_SAFE_DEFINE_LARGER_SIGNED_OPS(int, int)
+#endif
+
+#if defined(PSNIP_SAFE_HAVE_LARGER_UINT)
+PSNIP_SAFE_DEFINE_LARGER_UNSIGNED_OPS(unsigned int, uint)
+#endif
+
+#if defined(PSNIP_SAFE_HAVE_LARGER_LONG)
+PSNIP_SAFE_DEFINE_LARGER_SIGNED_OPS(long, long)
+#endif
+
+#if defined(PSNIP_SAFE_HAVE_LARGER_ULONG)
+PSNIP_SAFE_DEFINE_LARGER_UNSIGNED_OPS(unsigned long, ulong)
+#endif
+
+#if defined(PSNIP_SAFE_HAVE_LARGER_LLONG)
+PSNIP_SAFE_DEFINE_LARGER_SIGNED_OPS(long long, llong)
+#endif
+
+#if defined(PSNIP_SAFE_HAVE_LARGER_ULLONG)
+PSNIP_SAFE_DEFINE_LARGER_UNSIGNED_OPS(unsigned long long, ullong)
+#endif
+
+#if defined(PSNIP_SAFE_HAVE_LARGER_SIZE)
+PSNIP_SAFE_DEFINE_LARGER_UNSIGNED_OPS(size_t, size)
+#endif
+
+#if !defined(PSNIP_SAFE_NO_FIXED)
+PSNIP_SAFE_DEFINE_LARGER_SIGNED_OPS(psnip_int8_t,   int8)
+PSNIP_SAFE_DEFINE_LARGER_UNSIGNED_OPS(psnip_uint8_t,  uint8)
+PSNIP_SAFE_DEFINE_LARGER_SIGNED_OPS(psnip_int16_t,  int16)
+PSNIP_SAFE_DEFINE_LARGER_UNSIGNED_OPS(psnip_uint16_t, uint16)
+PSNIP_SAFE_DEFINE_LARGER_SIGNED_OPS(psnip_int32_t,  int32)
+PSNIP_SAFE_DEFINE_LARGER_UNSIGNED_OPS(psnip_uint32_t, uint32)
+#if defined(PSNIP_SAFE_HAVE_128)
+PSNIP_SAFE_DEFINE_LARGER_SIGNED_OPS(psnip_int64_t,  int64)
+PSNIP_SAFE_DEFINE_LARGER_UNSIGNED_OPS(psnip_uint64_t, uint64)
+#endif
+#endif
+
+#endif /* !defined(PSNIP_SAFE_NO_PROMOTIONS) */
+
+#define PSNIP_SAFE_DEFINE_BUILTIN_BINARY_OP(T, name, op_name) \
+  PSNIP_SAFE__FUNCTION psnip_safe_bool \
+  psnip_safe_##name##_##op_name(T* res, T a, T b) { \
+    return !__builtin_##op_name##_overflow(a, b, res); \
+  }
+
+#define PSNIP_SAFE_DEFINE_PROMOTED_SIGNED_BINARY_OP(T, name, op_name, min, max) \
+  PSNIP_SAFE__FUNCTION psnip_safe_bool \
+  psnip_safe_##name##_##op_name(T* res, T a, T b) { \
+    const psnip_safe_##name##_larger r = psnip_safe_larger_##name##_##op_name(a, b); \
+    *res = (T) r; \
+    return (r >= min) && (r <= max); \
+  }
+
+#define PSNIP_SAFE_DEFINE_PROMOTED_UNSIGNED_BINARY_OP(T, name, op_name, max) \
+  PSNIP_SAFE__FUNCTION psnip_safe_bool \
+  psnip_safe_##name##_##op_name(T* res, T a, T b) { \
+    const psnip_safe_##name##_larger r = psnip_safe_larger_##name##_##op_name(a, b); \
+    *res = (T) r; \
+    return (r <= max); \
+  }
+
+#define PSNIP_SAFE_DEFINE_SIGNED_ADD(T, name, min, max) \
+  PSNIP_SAFE__FUNCTION psnip_safe_bool \
+  psnip_safe_##name##_add (T* res, T a, T b) { \
+    psnip_safe_bool r = !( ((b > 0) && (a > (max - b))) ||   \
+                 ((b < 0) && (a < (min - b))) ); \
+    if(PSNIP_SAFE_LIKELY(r)) \
+        *res = a + b; \
+    return r; \
+  }
+
+#define PSNIP_SAFE_DEFINE_UNSIGNED_ADD(T, name, max) \
+  PSNIP_SAFE__FUNCTION psnip_safe_bool \
+  psnip_safe_##name##_add (T* res, T a, T b) { \
+    *res = (T) (a + b); \
+    return !PSNIP_SAFE_UNLIKELY((b > 0) && (a > (max - b))); \
+  }
+
+#define PSNIP_SAFE_DEFINE_SIGNED_SUB(T, name, min, max) \
+  PSNIP_SAFE__FUNCTION psnip_safe_bool \
+  psnip_safe_##name##_sub (T* res, T a, T b) { \
+      psnip_safe_bool r = !((b > 0 && a < (min + b)) || \
+                  (b < 0 && a > (max + b))); \
+      if(PSNIP_SAFE_LIKELY(r)) \
+          *res = a - b; \
+      return r; \
+  }
+
+#define PSNIP_SAFE_DEFINE_UNSIGNED_SUB(T, name, max) \
+  PSNIP_SAFE__FUNCTION psnip_safe_bool \
+  psnip_safe_##name##_sub (T* res, T a, T b) { \
+      *res = a - b; \
+      return !PSNIP_SAFE_UNLIKELY(b > a); \
+  }
+
+#define PSNIP_SAFE_DEFINE_SIGNED_MUL(T, name, min, max) \
+  PSNIP_SAFE__FUNCTION psnip_safe_bool \
+  psnip_safe_##name##_mul (T* res, T a, T b) { \
+    psnip_safe_bool r = 1;  \
+    if (a > 0) { \
+      if (b > 0) { \
+        if (a > (max / b)) { \
+          r = 0; \
+        } \
+      } else { \
+        if (b < (min / a)) { \
+          r = 0; \
+        } \
+      } \
+    } else { \
+      if (b > 0) { \
+        if (a < (min / b)) { \
+          r = 0; \
+        } \
+      } else { \
+        if ( (a != 0) && (b < (max / a))) { \
+          r = 0; \
+        } \
+      } \
+    } \
+    if(PSNIP_SAFE_LIKELY(r)) \
+        *res = a * b; \
+    return r; \
+  }
+
+#define PSNIP_SAFE_DEFINE_UNSIGNED_MUL(T, name, max) \
+  PSNIP_SAFE__FUNCTION psnip_safe_bool \
+  psnip_safe_##name##_mul (T* res, T a, T b) { \
+    *res = (T) (a * b); \
+    return !PSNIP_SAFE_UNLIKELY((a > 0) && (b > 0) && (a > (max / b))); \
+  }
+
+#define PSNIP_SAFE_DEFINE_SIGNED_DIV(T, name, min, max)   \
+  PSNIP_SAFE__FUNCTION psnip_safe_bool \
+  psnip_safe_##name##_div (T* res, T a, T b) { \
+    if (PSNIP_SAFE_UNLIKELY(b == 0)) { \
+      *res = 0; \
+      return 0; \
+    } else if (PSNIP_SAFE_UNLIKELY(a == min && b == -1)) {    \
+      *res = min; \
+      return 0; \
+    } else { \
+      *res = (T) (a / b); \
+      return 1; \
+    } \
+  }
+
+#define PSNIP_SAFE_DEFINE_UNSIGNED_DIV(T, name, max) \
+  PSNIP_SAFE__FUNCTION psnip_safe_bool \
+  psnip_safe_##name##_div (T* res, T a, T b) { \
+    if (PSNIP_SAFE_UNLIKELY(b == 0)) { \
+      *res = 0; \
+      return 0; \
+    } else { \
+      *res = a / b; \
+      return 1; \
+    } \
+  }
+
+#define PSNIP_SAFE_DEFINE_SIGNED_MOD(T, name, min, max) \
+  PSNIP_SAFE__FUNCTION psnip_safe_bool \
+  psnip_safe_##name##_mod (T* res, T a, T b) { \
+    if (PSNIP_SAFE_UNLIKELY(b == 0)) { \
+      *res = 0; \
+      return 0; \
+    } else if (PSNIP_SAFE_UNLIKELY(a == min && b == -1)) { \
+      *res = min; \
+      return 0; \
+    } else { \
+      *res = (T) (a % b); \
+      return 1; \
+    } \
+  }
+
+#define PSNIP_SAFE_DEFINE_UNSIGNED_MOD(T, name, max) \
+  PSNIP_SAFE__FUNCTION psnip_safe_bool \
+  psnip_safe_##name##_mod (T* res, T a, T b) { \
+    if (PSNIP_SAFE_UNLIKELY(b == 0)) { \
+      *res = 0; \
+      return 0; \
+    } else { \
+      *res = a % b; \
+      return 1; \
+    } \
+  }
+
+#define PSNIP_SAFE_DEFINE_SIGNED_NEG(T, name, min, max) \
+  PSNIP_SAFE__FUNCTION psnip_safe_bool \
+  psnip_safe_##name##_neg (T* res, T value) { \
+    psnip_safe_bool r = value != min; \
+    *res = PSNIP_SAFE_LIKELY(r) ? -value : max; \
+    return r; \
+  }
+
+#define PSNIP_SAFE_DEFINE_INTSAFE(T, name, op, isf) \
+  PSNIP_SAFE__FUNCTION psnip_safe_bool \
+  psnip_safe_##name##_##op (T* res, T a, T b) { \
+    return isf(a, b, res) == S_OK; \
+  }
+
+#if CHAR_MIN == 0
+#if defined(PSNIP_SAFE_HAVE_BUILTIN_OVERFLOW)
+PSNIP_SAFE_DEFINE_BUILTIN_BINARY_OP(char, char, add)
+PSNIP_SAFE_DEFINE_BUILTIN_BINARY_OP(char, char, sub)
+PSNIP_SAFE_DEFINE_BUILTIN_BINARY_OP(char, char, mul)
+#elif defined(PSNIP_SAFE_HAVE_LARGER_CHAR)
+PSNIP_SAFE_DEFINE_PROMOTED_UNSIGNED_BINARY_OP(char, char, add, CHAR_MAX)
+PSNIP_SAFE_DEFINE_PROMOTED_UNSIGNED_BINARY_OP(char, char, sub, CHAR_MAX)
+PSNIP_SAFE_DEFINE_PROMOTED_UNSIGNED_BINARY_OP(char, char, mul, CHAR_MAX)
+#else
+PSNIP_SAFE_DEFINE_UNSIGNED_ADD(char, char, CHAR_MAX)
+PSNIP_SAFE_DEFINE_UNSIGNED_SUB(char, char, CHAR_MAX)
+PSNIP_SAFE_DEFINE_UNSIGNED_MUL(char, char, CHAR_MAX)
+#endif
+PSNIP_SAFE_DEFINE_UNSIGNED_DIV(char, char, CHAR_MAX)
+PSNIP_SAFE_DEFINE_UNSIGNED_MOD(char, char, CHAR_MAX)
+#else /* CHAR_MIN != 0 */
+#if defined(PSNIP_SAFE_HAVE_BUILTIN_OVERFLOW)
+PSNIP_SAFE_DEFINE_BUILTIN_BINARY_OP(char, char, add)
+PSNIP_SAFE_DEFINE_BUILTIN_BINARY_OP(char, char, sub)
+PSNIP_SAFE_DEFINE_BUILTIN_BINARY_OP(char, char, mul)
+#elif defined(PSNIP_SAFE_HAVE_LARGER_CHAR)
+PSNIP_SAFE_DEFINE_PROMOTED_SIGNED_BINARY_OP(char, char, add, CHAR_MIN, CHAR_MAX)
+PSNIP_SAFE_DEFINE_PROMOTED_SIGNED_BINARY_OP(char, char, sub, CHAR_MIN, CHAR_MAX)
+PSNIP_SAFE_DEFINE_PROMOTED_SIGNED_BINARY_OP(char, char, mul, CHAR_MIN, CHAR_MAX)
+#else
+PSNIP_SAFE_DEFINE_SIGNED_ADD(char, char, CHAR_MIN, CHAR_MAX)
+PSNIP_SAFE_DEFINE_SIGNED_SUB(char, char, CHAR_MIN, CHAR_MAX)
+PSNIP_SAFE_DEFINE_SIGNED_MUL(char, char, CHAR_MIN, CHAR_MAX)
+#endif
+PSNIP_SAFE_DEFINE_SIGNED_DIV(char, char, CHAR_MIN, CHAR_MAX)
+PSNIP_SAFE_DEFINE_SIGNED_MOD(char, char, CHAR_MIN, CHAR_MAX)
+PSNIP_SAFE_DEFINE_SIGNED_NEG(char, char, CHAR_MIN, CHAR_MAX)
+#endif
+
+#if defined(PSNIP_SAFE_HAVE_BUILTIN_OVERFLOW)
+PSNIP_SAFE_DEFINE_BUILTIN_BINARY_OP(signed char, schar, add)
+PSNIP_SAFE_DEFINE_BUILTIN_BINARY_OP(signed char, schar, sub)
+PSNIP_SAFE_DEFINE_BUILTIN_BINARY_OP(signed char, schar, mul)
+#elif defined(PSNIP_SAFE_HAVE_LARGER_SCHAR)
+PSNIP_SAFE_DEFINE_PROMOTED_SIGNED_BINARY_OP(signed char, schar, add, SCHAR_MIN, SCHAR_MAX)
+PSNIP_SAFE_DEFINE_PROMOTED_SIGNED_BINARY_OP(signed char, schar, sub, SCHAR_MIN, SCHAR_MAX)
+PSNIP_SAFE_DEFINE_PROMOTED_SIGNED_BINARY_OP(signed char, schar, mul, SCHAR_MIN, SCHAR_MAX)
+#else
+PSNIP_SAFE_DEFINE_SIGNED_ADD(signed char, schar, SCHAR_MIN, SCHAR_MAX)
+PSNIP_SAFE_DEFINE_SIGNED_SUB(signed char, schar, SCHAR_MIN, SCHAR_MAX)
+PSNIP_SAFE_DEFINE_SIGNED_MUL(signed char, schar, SCHAR_MIN, SCHAR_MAX)
+#endif
+PSNIP_SAFE_DEFINE_SIGNED_DIV(signed char, schar, SCHAR_MIN, SCHAR_MAX)
+PSNIP_SAFE_DEFINE_SIGNED_MOD(signed char, schar, SCHAR_MIN, SCHAR_MAX)
+PSNIP_SAFE_DEFINE_SIGNED_NEG(signed char, schar, SCHAR_MIN, SCHAR_MAX)
+
+#if defined(PSNIP_SAFE_HAVE_BUILTIN_OVERFLOW)
+PSNIP_SAFE_DEFINE_BUILTIN_BINARY_OP(unsigned char, uchar, add)
+PSNIP_SAFE_DEFINE_BUILTIN_BINARY_OP(unsigned char, uchar, sub)
+PSNIP_SAFE_DEFINE_BUILTIN_BINARY_OP(unsigned char, uchar, mul)
+#elif defined(PSNIP_SAFE_HAVE_LARGER_UCHAR)
+PSNIP_SAFE_DEFINE_PROMOTED_UNSIGNED_BINARY_OP(unsigned char, uchar, add, UCHAR_MAX)
+PSNIP_SAFE_DEFINE_PROMOTED_UNSIGNED_BINARY_OP(unsigned char, uchar, sub, UCHAR_MAX)
+PSNIP_SAFE_DEFINE_PROMOTED_UNSIGNED_BINARY_OP(unsigned char, uchar, mul, UCHAR_MAX)
+#else
+PSNIP_SAFE_DEFINE_UNSIGNED_ADD(unsigned char, uchar, UCHAR_MAX)
+PSNIP_SAFE_DEFINE_UNSIGNED_SUB(unsigned char, uchar, UCHAR_MAX)
+PSNIP_SAFE_DEFINE_UNSIGNED_MUL(unsigned char, uchar, UCHAR_MAX)
+#endif
+PSNIP_SAFE_DEFINE_UNSIGNED_DIV(unsigned char, uchar, UCHAR_MAX)
+PSNIP_SAFE_DEFINE_UNSIGNED_MOD(unsigned char, uchar, UCHAR_MAX)
+
+#if defined(PSNIP_SAFE_HAVE_BUILTIN_OVERFLOW)
+PSNIP_SAFE_DEFINE_BUILTIN_BINARY_OP(short, short, add)
+PSNIP_SAFE_DEFINE_BUILTIN_BINARY_OP(short, short, sub)
+PSNIP_SAFE_DEFINE_BUILTIN_BINARY_OP(short, short, mul)
+#elif defined(PSNIP_SAFE_HAVE_LARGER_SHORT)
+PSNIP_SAFE_DEFINE_PROMOTED_SIGNED_BINARY_OP(short, short, add, SHRT_MIN, SHRT_MAX)
+PSNIP_SAFE_DEFINE_PROMOTED_SIGNED_BINARY_OP(short, short, sub, SHRT_MIN, SHRT_MAX)
+PSNIP_SAFE_DEFINE_PROMOTED_SIGNED_BINARY_OP(short, short, mul, SHRT_MIN, SHRT_MAX)
+#else
+PSNIP_SAFE_DEFINE_SIGNED_ADD(short, short, SHRT_MIN, SHRT_MAX)
+PSNIP_SAFE_DEFINE_SIGNED_SUB(short, short, SHRT_MIN, SHRT_MAX)
+PSNIP_SAFE_DEFINE_SIGNED_MUL(short, short, SHRT_MIN, SHRT_MAX)
+#endif
+PSNIP_SAFE_DEFINE_SIGNED_DIV(short, short, SHRT_MIN, SHRT_MAX)
+PSNIP_SAFE_DEFINE_SIGNED_MOD(short, short, SHRT_MIN, SHRT_MAX)
+PSNIP_SAFE_DEFINE_SIGNED_NEG(short, short, SHRT_MIN, SHRT_MAX)
+
+#if defined(PSNIP_SAFE_HAVE_BUILTIN_OVERFLOW)
+PSNIP_SAFE_DEFINE_BUILTIN_BINARY_OP(unsigned short, ushort, add)
+PSNIP_SAFE_DEFINE_BUILTIN_BINARY_OP(unsigned short, ushort, sub)
+PSNIP_SAFE_DEFINE_BUILTIN_BINARY_OP(unsigned short, ushort, mul)
+#elif defined(PSNIP_SAFE_HAVE_INTSAFE_H)
+PSNIP_SAFE_DEFINE_INTSAFE(unsigned short, ushort, add, UShortAdd)
+PSNIP_SAFE_DEFINE_INTSAFE(unsigned short, ushort, sub, UShortSub)
+PSNIP_SAFE_DEFINE_INTSAFE(unsigned short, ushort, mul, UShortMult)
+#elif defined(PSNIP_SAFE_HAVE_LARGER_USHORT)
+PSNIP_SAFE_DEFINE_PROMOTED_UNSIGNED_BINARY_OP(unsigned short, ushort, add, USHRT_MAX)
+PSNIP_SAFE_DEFINE_PROMOTED_UNSIGNED_BINARY_OP(unsigned short, ushort, sub, USHRT_MAX)
+PSNIP_SAFE_DEFINE_PROMOTED_UNSIGNED_BINARY_OP(unsigned short, ushort, mul, USHRT_MAX)
+#else
+PSNIP_SAFE_DEFINE_UNSIGNED_ADD(unsigned short, ushort, USHRT_MAX)
+PSNIP_SAFE_DEFINE_UNSIGNED_SUB(unsigned short, ushort, USHRT_MAX)
+PSNIP_SAFE_DEFINE_UNSIGNED_MUL(unsigned short, ushort, USHRT_MAX)
+#endif
+PSNIP_SAFE_DEFINE_UNSIGNED_DIV(unsigned short, ushort, USHRT_MAX)
+PSNIP_SAFE_DEFINE_UNSIGNED_MOD(unsigned short, ushort, USHRT_MAX)
+
+#if defined(PSNIP_SAFE_HAVE_BUILTIN_OVERFLOW)
+PSNIP_SAFE_DEFINE_BUILTIN_BINARY_OP(int, int, add)
+PSNIP_SAFE_DEFINE_BUILTIN_BINARY_OP(int, int, sub)
+PSNIP_SAFE_DEFINE_BUILTIN_BINARY_OP(int, int, mul)
+#elif defined(PSNIP_SAFE_HAVE_LARGER_INT)
+PSNIP_SAFE_DEFINE_PROMOTED_SIGNED_BINARY_OP(int, int, add, INT_MIN, INT_MAX)
+PSNIP_SAFE_DEFINE_PROMOTED_SIGNED_BINARY_OP(int, int, sub, INT_MIN, INT_MAX)
+PSNIP_SAFE_DEFINE_PROMOTED_SIGNED_BINARY_OP(int, int, mul, INT_MIN, INT_MAX)
+#else
+PSNIP_SAFE_DEFINE_SIGNED_ADD(int, int, INT_MIN, INT_MAX)
+PSNIP_SAFE_DEFINE_SIGNED_SUB(int, int, INT_MIN, INT_MAX)
+PSNIP_SAFE_DEFINE_SIGNED_MUL(int, int, INT_MIN, INT_MAX)
+#endif
+PSNIP_SAFE_DEFINE_SIGNED_DIV(int, int, INT_MIN, INT_MAX)
+PSNIP_SAFE_DEFINE_SIGNED_MOD(int, int, INT_MIN, INT_MAX)
+PSNIP_SAFE_DEFINE_SIGNED_NEG(int, int, INT_MIN, INT_MAX)
+
+#if defined(PSNIP_SAFE_HAVE_BUILTIN_OVERFLOW)
+PSNIP_SAFE_DEFINE_BUILTIN_BINARY_OP(unsigned int, uint, add)
+PSNIP_SAFE_DEFINE_BUILTIN_BINARY_OP(unsigned int, uint, sub)
+PSNIP_SAFE_DEFINE_BUILTIN_BINARY_OP(unsigned int, uint, mul)
+#elif defined(PSNIP_SAFE_HAVE_INTSAFE_H)
+PSNIP_SAFE_DEFINE_INTSAFE(unsigned int, uint, add, UIntAdd)
+PSNIP_SAFE_DEFINE_INTSAFE(unsigned int, uint, sub, UIntSub)
+PSNIP_SAFE_DEFINE_INTSAFE(unsigned int, uint, mul, UIntMult)
+#elif defined(PSNIP_SAFE_HAVE_LARGER_UINT)
+PSNIP_SAFE_DEFINE_PROMOTED_UNSIGNED_BINARY_OP(unsigned int, uint, add, UINT_MAX)
+PSNIP_SAFE_DEFINE_PROMOTED_UNSIGNED_BINARY_OP(unsigned int, uint, sub, UINT_MAX)
+PSNIP_SAFE_DEFINE_PROMOTED_UNSIGNED_BINARY_OP(unsigned int, uint, mul, UINT_MAX)
+#else
+PSNIP_SAFE_DEFINE_UNSIGNED_ADD(unsigned int, uint, UINT_MAX)
+PSNIP_SAFE_DEFINE_UNSIGNED_SUB(unsigned int, uint, UINT_MAX)
+PSNIP_SAFE_DEFINE_UNSIGNED_MUL(unsigned int, uint, UINT_MAX)
+#endif
+PSNIP_SAFE_DEFINE_UNSIGNED_DIV(unsigned int, uint, UINT_MAX)
+PSNIP_SAFE_DEFINE_UNSIGNED_MOD(unsigned int, uint, UINT_MAX)
+
+#if defined(PSNIP_SAFE_HAVE_BUILTIN_OVERFLOW)
+PSNIP_SAFE_DEFINE_BUILTIN_BINARY_OP(long, long, add)
+PSNIP_SAFE_DEFINE_BUILTIN_BINARY_OP(long, long, sub)
+PSNIP_SAFE_DEFINE_BUILTIN_BINARY_OP(long, long, mul)
+#elif defined(PSNIP_SAFE_HAVE_LARGER_LONG)
+PSNIP_SAFE_DEFINE_PROMOTED_SIGNED_BINARY_OP(long, long, add, LONG_MIN, LONG_MAX)
+PSNIP_SAFE_DEFINE_PROMOTED_SIGNED_BINARY_OP(long, long, sub, LONG_MIN, LONG_MAX)
+PSNIP_SAFE_DEFINE_PROMOTED_SIGNED_BINARY_OP(long, long, mul, LONG_MIN, LONG_MAX)
+#else
+PSNIP_SAFE_DEFINE_SIGNED_ADD(long, long, LONG_MIN, LONG_MAX)
+PSNIP_SAFE_DEFINE_SIGNED_SUB(long, long, LONG_MIN, LONG_MAX)
+PSNIP_SAFE_DEFINE_SIGNED_MUL(long, long, LONG_MIN, LONG_MAX)
+#endif
+PSNIP_SAFE_DEFINE_SIGNED_DIV(long, long, LONG_MIN, LONG_MAX)
+PSNIP_SAFE_DEFINE_SIGNED_MOD(long, long, LONG_MIN, LONG_MAX)
+PSNIP_SAFE_DEFINE_SIGNED_NEG(long, long, LONG_MIN, LONG_MAX)
+
+#if defined(PSNIP_SAFE_HAVE_BUILTIN_OVERFLOW)
+PSNIP_SAFE_DEFINE_BUILTIN_BINARY_OP(unsigned long, ulong, add)
+PSNIP_SAFE_DEFINE_BUILTIN_BINARY_OP(unsigned long, ulong, sub)
+PSNIP_SAFE_DEFINE_BUILTIN_BINARY_OP(unsigned long, ulong, mul)
+#elif defined(PSNIP_SAFE_HAVE_INTSAFE_H)
+PSNIP_SAFE_DEFINE_INTSAFE(unsigned long, ulong, add, ULongAdd)
+PSNIP_SAFE_DEFINE_INTSAFE(unsigned long, ulong, sub, ULongSub)
+PSNIP_SAFE_DEFINE_INTSAFE(unsigned long, ulong, mul, ULongMult)
+#elif defined(PSNIP_SAFE_HAVE_LARGER_ULONG)
+PSNIP_SAFE_DEFINE_PROMOTED_UNSIGNED_BINARY_OP(unsigned long, ulong, add, ULONG_MAX)
+PSNIP_SAFE_DEFINE_PROMOTED_UNSIGNED_BINARY_OP(unsigned long, ulong, sub, ULONG_MAX)
+PSNIP_SAFE_DEFINE_PROMOTED_UNSIGNED_BINARY_OP(unsigned long, ulong, mul, ULONG_MAX)
+#else
+PSNIP_SAFE_DEFINE_UNSIGNED_ADD(unsigned long, ulong, ULONG_MAX)
+PSNIP_SAFE_DEFINE_UNSIGNED_SUB(unsigned long, ulong, ULONG_MAX)
+PSNIP_SAFE_DEFINE_UNSIGNED_MUL(unsigned long, ulong, ULONG_MAX)
+#endif
+PSNIP_SAFE_DEFINE_UNSIGNED_DIV(unsigned long, ulong, ULONG_MAX)
+PSNIP_SAFE_DEFINE_UNSIGNED_MOD(unsigned long, ulong, ULONG_MAX)
+
+#if defined(PSNIP_SAFE_HAVE_BUILTIN_OVERFLOW)
+PSNIP_SAFE_DEFINE_BUILTIN_BINARY_OP(long long, llong, add)
+PSNIP_SAFE_DEFINE_BUILTIN_BINARY_OP(long long, llong, sub)
+PSNIP_SAFE_DEFINE_BUILTIN_BINARY_OP(long long, llong, mul)
+#elif defined(PSNIP_SAFE_HAVE_LARGER_LLONG)
+PSNIP_SAFE_DEFINE_PROMOTED_SIGNED_BINARY_OP(long long, llong, add, LLONG_MIN, LLONG_MAX)
+PSNIP_SAFE_DEFINE_PROMOTED_SIGNED_BINARY_OP(long long, llong, sub, LLONG_MIN, LLONG_MAX)
+PSNIP_SAFE_DEFINE_PROMOTED_SIGNED_BINARY_OP(long long, llong, mul, LLONG_MIN, LLONG_MAX)
+#else
+PSNIP_SAFE_DEFINE_SIGNED_ADD(long long, llong, LLONG_MIN, LLONG_MAX)
+PSNIP_SAFE_DEFINE_SIGNED_SUB(long long, llong, LLONG_MIN, LLONG_MAX)
+PSNIP_SAFE_DEFINE_SIGNED_MUL(long long, llong, LLONG_MIN, LLONG_MAX)
+#endif
+PSNIP_SAFE_DEFINE_SIGNED_DIV(long long, llong, LLONG_MIN, LLONG_MAX)
+PSNIP_SAFE_DEFINE_SIGNED_MOD(long long, llong, LLONG_MIN, LLONG_MAX)
+PSNIP_SAFE_DEFINE_SIGNED_NEG(long long, llong, LLONG_MIN, LLONG_MAX)
+
+#if defined(PSNIP_SAFE_HAVE_BUILTIN_OVERFLOW)
+PSNIP_SAFE_DEFINE_BUILTIN_BINARY_OP(unsigned long long, ullong, add)
+PSNIP_SAFE_DEFINE_BUILTIN_BINARY_OP(unsigned long long, ullong, sub)
+PSNIP_SAFE_DEFINE_BUILTIN_BINARY_OP(unsigned long long, ullong, mul)
+#elif defined(PSNIP_SAFE_HAVE_INTSAFE_H)
+PSNIP_SAFE_DEFINE_INTSAFE(unsigned long long, ullong, add, ULongLongAdd)
+PSNIP_SAFE_DEFINE_INTSAFE(unsigned long long, ullong, sub, ULongLongSub)
+PSNIP_SAFE_DEFINE_INTSAFE(unsigned long long, ullong, mul, ULongLongMult)
+#elif defined(PSNIP_SAFE_HAVE_LARGER_ULLONG)
+PSNIP_SAFE_DEFINE_PROMOTED_UNSIGNED_BINARY_OP(unsigned long long, ullong, add, ULLONG_MAX)
+PSNIP_SAFE_DEFINE_PROMOTED_UNSIGNED_BINARY_OP(unsigned long long, ullong, sub, ULLONG_MAX)
+PSNIP_SAFE_DEFINE_PROMOTED_UNSIGNED_BINARY_OP(unsigned long long, ullong, mul, ULLONG_MAX)
+#else
+PSNIP_SAFE_DEFINE_UNSIGNED_ADD(unsigned long long, ullong, ULLONG_MAX)
+PSNIP_SAFE_DEFINE_UNSIGNED_SUB(unsigned long long, ullong, ULLONG_MAX)
+PSNIP_SAFE_DEFINE_UNSIGNED_MUL(unsigned long long, ullong, ULLONG_MAX)
+#endif
+PSNIP_SAFE_DEFINE_UNSIGNED_DIV(unsigned long long, ullong, ULLONG_MAX)
+PSNIP_SAFE_DEFINE_UNSIGNED_MOD(unsigned long long, ullong, ULLONG_MAX)
+
+#if defined(PSNIP_SAFE_HAVE_BUILTIN_OVERFLOW)
+PSNIP_SAFE_DEFINE_BUILTIN_BINARY_OP(size_t, size, add)
+PSNIP_SAFE_DEFINE_BUILTIN_BINARY_OP(size_t, size, sub)
+PSNIP_SAFE_DEFINE_BUILTIN_BINARY_OP(size_t, size, mul)
+#elif defined(PSNIP_SAFE_HAVE_INTSAFE_H)
+PSNIP_SAFE_DEFINE_INTSAFE(size_t, size, add, SizeTAdd)
+PSNIP_SAFE_DEFINE_INTSAFE(size_t, size, sub, SizeTSub)
+PSNIP_SAFE_DEFINE_INTSAFE(size_t, size, mul, SizeTMult)
+#elif defined(PSNIP_SAFE_HAVE_LARGER_SIZE)
+PSNIP_SAFE_DEFINE_PROMOTED_UNSIGNED_BINARY_OP(size_t, size, add, PSNIP_SAFE__SIZE_MAX_RT)
+PSNIP_SAFE_DEFINE_PROMOTED_UNSIGNED_BINARY_OP(size_t, size, sub, PSNIP_SAFE__SIZE_MAX_RT)
+PSNIP_SAFE_DEFINE_PROMOTED_UNSIGNED_BINARY_OP(size_t, size, mul, PSNIP_SAFE__SIZE_MAX_RT)
+#else
+PSNIP_SAFE_DEFINE_UNSIGNED_ADD(size_t, size, PSNIP_SAFE__SIZE_MAX_RT)
+PSNIP_SAFE_DEFINE_UNSIGNED_SUB(size_t, size, PSNIP_SAFE__SIZE_MAX_RT)
+PSNIP_SAFE_DEFINE_UNSIGNED_MUL(size_t, size, PSNIP_SAFE__SIZE_MAX_RT)
+#endif
+PSNIP_SAFE_DEFINE_UNSIGNED_DIV(size_t, size, PSNIP_SAFE__SIZE_MAX_RT)
+PSNIP_SAFE_DEFINE_UNSIGNED_MOD(size_t, size, PSNIP_SAFE__SIZE_MAX_RT)
+
+#if !defined(PSNIP_SAFE_NO_FIXED)
+
+#if defined(PSNIP_SAFE_HAVE_BUILTIN_OVERFLOW)
+PSNIP_SAFE_DEFINE_BUILTIN_BINARY_OP(psnip_int8_t, int8, add)
+PSNIP_SAFE_DEFINE_BUILTIN_BINARY_OP(psnip_int8_t, int8, sub)
+PSNIP_SAFE_DEFINE_BUILTIN_BINARY_OP(psnip_int8_t, int8, mul)
+#elif defined(PSNIP_SAFE_HAVE_LARGER_INT8)
+PSNIP_SAFE_DEFINE_PROMOTED_SIGNED_BINARY_OP(psnip_int8_t, int8, add, (-0x7fLL-1), 0x7f)
+PSNIP_SAFE_DEFINE_PROMOTED_SIGNED_BINARY_OP(psnip_int8_t, int8, sub, (-0x7fLL-1), 0x7f)
+PSNIP_SAFE_DEFINE_PROMOTED_SIGNED_BINARY_OP(psnip_int8_t, int8, mul, (-0x7fLL-1), 0x7f)
+#else
+PSNIP_SAFE_DEFINE_SIGNED_ADD(psnip_int8_t, int8, (-0x7fLL-1), 0x7f)
+PSNIP_SAFE_DEFINE_SIGNED_SUB(psnip_int8_t, int8, (-0x7fLL-1), 0x7f)
+PSNIP_SAFE_DEFINE_SIGNED_MUL(psnip_int8_t, int8, (-0x7fLL-1), 0x7f)
+#endif
+PSNIP_SAFE_DEFINE_SIGNED_DIV(psnip_int8_t, int8, (-0x7fLL-1), 0x7f)
+PSNIP_SAFE_DEFINE_SIGNED_MOD(psnip_int8_t, int8, (-0x7fLL-1), 0x7f)
+PSNIP_SAFE_DEFINE_SIGNED_NEG(psnip_int8_t, int8, (-0x7fLL-1), 0x7f)
+
+#if defined(PSNIP_SAFE_HAVE_BUILTIN_OVERFLOW)
+PSNIP_SAFE_DEFINE_BUILTIN_BINARY_OP(psnip_uint8_t, uint8, add)
+PSNIP_SAFE_DEFINE_BUILTIN_BINARY_OP(psnip_uint8_t, uint8, sub)
+PSNIP_SAFE_DEFINE_BUILTIN_BINARY_OP(psnip_uint8_t, uint8, mul)
+#elif defined(PSNIP_SAFE_HAVE_LARGER_UINT8)
+PSNIP_SAFE_DEFINE_PROMOTED_UNSIGNED_BINARY_OP(psnip_uint8_t, uint8, add, 0xff)
+PSNIP_SAFE_DEFINE_PROMOTED_UNSIGNED_BINARY_OP(psnip_uint8_t, uint8, sub, 0xff)
+PSNIP_SAFE_DEFINE_PROMOTED_UNSIGNED_BINARY_OP(psnip_uint8_t, uint8, mul, 0xff)
+#else
+PSNIP_SAFE_DEFINE_UNSIGNED_ADD(psnip_uint8_t, uint8, 0xff)
+PSNIP_SAFE_DEFINE_UNSIGNED_SUB(psnip_uint8_t, uint8, 0xff)
+PSNIP_SAFE_DEFINE_UNSIGNED_MUL(psnip_uint8_t, uint8, 0xff)
+#endif
+PSNIP_SAFE_DEFINE_UNSIGNED_DIV(psnip_uint8_t, uint8, 0xff)
+PSNIP_SAFE_DEFINE_UNSIGNED_MOD(psnip_uint8_t, uint8, 0xff)
+
+#if defined(PSNIP_SAFE_HAVE_BUILTIN_OVERFLOW)
+PSNIP_SAFE_DEFINE_BUILTIN_BINARY_OP(psnip_int16_t, int16, add)
+PSNIP_SAFE_DEFINE_BUILTIN_BINARY_OP(psnip_int16_t, int16, sub)
+PSNIP_SAFE_DEFINE_BUILTIN_BINARY_OP(psnip_int16_t, int16, mul)
+#elif defined(PSNIP_SAFE_HAVE_LARGER_INT16)
+PSNIP_SAFE_DEFINE_PROMOTED_SIGNED_BINARY_OP(psnip_int16_t, int16, add, (-32767-1), 0x7fff)
+PSNIP_SAFE_DEFINE_PROMOTED_SIGNED_BINARY_OP(psnip_int16_t, int16, sub, (-32767-1), 0x7fff)
+PSNIP_SAFE_DEFINE_PROMOTED_SIGNED_BINARY_OP(psnip_int16_t, int16, mul, (-32767-1), 0x7fff)
+#else
+PSNIP_SAFE_DEFINE_SIGNED_ADD(psnip_int16_t, int16, (-32767-1), 0x7fff)
+PSNIP_SAFE_DEFINE_SIGNED_SUB(psnip_int16_t, int16, (-32767-1), 0x7fff)
+PSNIP_SAFE_DEFINE_SIGNED_MUL(psnip_int16_t, int16, (-32767-1), 0x7fff)
+#endif
+PSNIP_SAFE_DEFINE_SIGNED_DIV(psnip_int16_t, int16, (-32767-1), 0x7fff)
+PSNIP_SAFE_DEFINE_SIGNED_MOD(psnip_int16_t, int16, (-32767-1), 0x7fff)
+PSNIP_SAFE_DEFINE_SIGNED_NEG(psnip_int16_t, int16, (-32767-1), 0x7fff)
+
+#if defined(PSNIP_SAFE_HAVE_BUILTIN_OVERFLOW)
+PSNIP_SAFE_DEFINE_BUILTIN_BINARY_OP(psnip_uint16_t, uint16, add)
+PSNIP_SAFE_DEFINE_BUILTIN_BINARY_OP(psnip_uint16_t, uint16, sub)
+PSNIP_SAFE_DEFINE_BUILTIN_BINARY_OP(psnip_uint16_t, uint16, mul)
+#elif defined(PSNIP_SAFE_HAVE_INTSAFE_H) && defined(_WIN32)
+PSNIP_SAFE_DEFINE_INTSAFE(psnip_uint16_t, uint16, add, UShortAdd)
+PSNIP_SAFE_DEFINE_INTSAFE(psnip_uint16_t, uint16, sub, UShortSub)
+PSNIP_SAFE_DEFINE_INTSAFE(psnip_uint16_t, uint16, mul, UShortMult)
+#elif defined(PSNIP_SAFE_HAVE_LARGER_UINT16)
+PSNIP_SAFE_DEFINE_PROMOTED_UNSIGNED_BINARY_OP(psnip_uint16_t, uint16, add, 0xffff)
+PSNIP_SAFE_DEFINE_PROMOTED_UNSIGNED_BINARY_OP(psnip_uint16_t, uint16, sub, 0xffff)
+PSNIP_SAFE_DEFINE_PROMOTED_UNSIGNED_BINARY_OP(psnip_uint16_t, uint16, mul, 0xffff)
+#else
+PSNIP_SAFE_DEFINE_UNSIGNED_ADD(psnip_uint16_t, uint16, 0xffff)
+PSNIP_SAFE_DEFINE_UNSIGNED_SUB(psnip_uint16_t, uint16, 0xffff)
+PSNIP_SAFE_DEFINE_UNSIGNED_MUL(psnip_uint16_t, uint16, 0xffff)
+#endif
+PSNIP_SAFE_DEFINE_UNSIGNED_DIV(psnip_uint16_t, uint16, 0xffff)
+PSNIP_SAFE_DEFINE_UNSIGNED_MOD(psnip_uint16_t, uint16, 0xffff)
+
+#if defined(PSNIP_SAFE_HAVE_BUILTIN_OVERFLOW)
+PSNIP_SAFE_DEFINE_BUILTIN_BINARY_OP(psnip_int32_t, int32, add)
+PSNIP_SAFE_DEFINE_BUILTIN_BINARY_OP(psnip_int32_t, int32, sub)
+PSNIP_SAFE_DEFINE_BUILTIN_BINARY_OP(psnip_int32_t, int32, mul)
+#elif defined(PSNIP_SAFE_HAVE_LARGER_INT32)
+PSNIP_SAFE_DEFINE_PROMOTED_SIGNED_BINARY_OP(psnip_int32_t, int32, add, (-0x7fffffffLL-1), 0x7fffffffLL)
+PSNIP_SAFE_DEFINE_PROMOTED_SIGNED_BINARY_OP(psnip_int32_t, int32, sub, (-0x7fffffffLL-1), 0x7fffffffLL)
+PSNIP_SAFE_DEFINE_PROMOTED_SIGNED_BINARY_OP(psnip_int32_t, int32, mul, (-0x7fffffffLL-1), 0x7fffffffLL)
+#else
+PSNIP_SAFE_DEFINE_SIGNED_ADD(psnip_int32_t, int32, (-0x7fffffffLL-1), 0x7fffffffLL)
+PSNIP_SAFE_DEFINE_SIGNED_SUB(psnip_int32_t, int32, (-0x7fffffffLL-1), 0x7fffffffLL)
+PSNIP_SAFE_DEFINE_SIGNED_MUL(psnip_int32_t, int32, (-0x7fffffffLL-1), 0x7fffffffLL)
+#endif
+PSNIP_SAFE_DEFINE_SIGNED_DIV(psnip_int32_t, int32, (-0x7fffffffLL-1), 0x7fffffffLL)
+PSNIP_SAFE_DEFINE_SIGNED_MOD(psnip_int32_t, int32, (-0x7fffffffLL-1), 0x7fffffffLL)
+PSNIP_SAFE_DEFINE_SIGNED_NEG(psnip_int32_t, int32, (-0x7fffffffLL-1), 0x7fffffffLL)
+
+#if defined(PSNIP_SAFE_HAVE_BUILTIN_OVERFLOW)
+PSNIP_SAFE_DEFINE_BUILTIN_BINARY_OP(psnip_uint32_t, uint32, add)
+PSNIP_SAFE_DEFINE_BUILTIN_BINARY_OP(psnip_uint32_t, uint32, sub)
+PSNIP_SAFE_DEFINE_BUILTIN_BINARY_OP(psnip_uint32_t, uint32, mul)
+#elif defined(PSNIP_SAFE_HAVE_INTSAFE_H) && defined(_WIN32)
+PSNIP_SAFE_DEFINE_INTSAFE(psnip_uint32_t, uint32, add, UIntAdd)
+PSNIP_SAFE_DEFINE_INTSAFE(psnip_uint32_t, uint32, sub, UIntSub)
+PSNIP_SAFE_DEFINE_INTSAFE(psnip_uint32_t, uint32, mul, UIntMult)
+#elif defined(PSNIP_SAFE_HAVE_LARGER_UINT32)
+PSNIP_SAFE_DEFINE_PROMOTED_UNSIGNED_BINARY_OP(psnip_uint32_t, uint32, add, 0xffffffffUL)
+PSNIP_SAFE_DEFINE_PROMOTED_UNSIGNED_BINARY_OP(psnip_uint32_t, uint32, sub, 0xffffffffUL)
+PSNIP_SAFE_DEFINE_PROMOTED_UNSIGNED_BINARY_OP(psnip_uint32_t, uint32, mul, 0xffffffffUL)
+#else
+PSNIP_SAFE_DEFINE_UNSIGNED_ADD(psnip_uint32_t, uint32, 0xffffffffUL)
+PSNIP_SAFE_DEFINE_UNSIGNED_SUB(psnip_uint32_t, uint32, 0xffffffffUL)
+PSNIP_SAFE_DEFINE_UNSIGNED_MUL(psnip_uint32_t, uint32, 0xffffffffUL)
+#endif
+PSNIP_SAFE_DEFINE_UNSIGNED_DIV(psnip_uint32_t, uint32, 0xffffffffUL)
+PSNIP_SAFE_DEFINE_UNSIGNED_MOD(psnip_uint32_t, uint32, 0xffffffffUL)
+
+#if defined(PSNIP_SAFE_HAVE_BUILTIN_OVERFLOW)
+PSNIP_SAFE_DEFINE_BUILTIN_BINARY_OP(psnip_int64_t, int64, add)
+PSNIP_SAFE_DEFINE_BUILTIN_BINARY_OP(psnip_int64_t, int64, sub)
+PSNIP_SAFE_DEFINE_BUILTIN_BINARY_OP(psnip_int64_t, int64, mul)
+#elif defined(PSNIP_SAFE_HAVE_LARGER_INT64)
+PSNIP_SAFE_DEFINE_PROMOTED_SIGNED_BINARY_OP(psnip_int64_t, int64, add, (-0x7fffffffffffffffLL-1), 0x7fffffffffffffffLL)
+PSNIP_SAFE_DEFINE_PROMOTED_SIGNED_BINARY_OP(psnip_int64_t, int64, sub, (-0x7fffffffffffffffLL-1), 0x7fffffffffffffffLL)
+PSNIP_SAFE_DEFINE_PROMOTED_SIGNED_BINARY_OP(psnip_int64_t, int64, mul, (-0x7fffffffffffffffLL-1), 0x7fffffffffffffffLL)
+#else
+PSNIP_SAFE_DEFINE_SIGNED_ADD(psnip_int64_t, int64, (-0x7fffffffffffffffLL-1), 0x7fffffffffffffffLL)
+PSNIP_SAFE_DEFINE_SIGNED_SUB(psnip_int64_t, int64, (-0x7fffffffffffffffLL-1), 0x7fffffffffffffffLL)
+PSNIP_SAFE_DEFINE_SIGNED_MUL(psnip_int64_t, int64, (-0x7fffffffffffffffLL-1), 0x7fffffffffffffffLL)
+#endif
+PSNIP_SAFE_DEFINE_SIGNED_DIV(psnip_int64_t, int64, (-0x7fffffffffffffffLL-1), 0x7fffffffffffffffLL)
+PSNIP_SAFE_DEFINE_SIGNED_MOD(psnip_int64_t, int64, (-0x7fffffffffffffffLL-1), 0x7fffffffffffffffLL)
+PSNIP_SAFE_DEFINE_SIGNED_NEG(psnip_int64_t, int64, (-0x7fffffffffffffffLL-1), 0x7fffffffffffffffLL)
+
+#if defined(PSNIP_SAFE_HAVE_BUILTIN_OVERFLOW)
+PSNIP_SAFE_DEFINE_BUILTIN_BINARY_OP(psnip_uint64_t, uint64, add)
+PSNIP_SAFE_DEFINE_BUILTIN_BINARY_OP(psnip_uint64_t, uint64, sub)
+PSNIP_SAFE_DEFINE_BUILTIN_BINARY_OP(psnip_uint64_t, uint64, mul)
+#elif defined(PSNIP_SAFE_HAVE_INTSAFE_H) && defined(_WIN32)
+PSNIP_SAFE_DEFINE_INTSAFE(psnip_uint64_t, uint64, add, ULongLongAdd)
+PSNIP_SAFE_DEFINE_INTSAFE(psnip_uint64_t, uint64, sub, ULongLongSub)
+PSNIP_SAFE_DEFINE_INTSAFE(psnip_uint64_t, uint64, mul, ULongLongMult)
+#elif defined(PSNIP_SAFE_HAVE_LARGER_UINT64)
+PSNIP_SAFE_DEFINE_PROMOTED_UNSIGNED_BINARY_OP(psnip_uint64_t, uint64, add, 0xffffffffffffffffULL)
+PSNIP_SAFE_DEFINE_PROMOTED_UNSIGNED_BINARY_OP(psnip_uint64_t, uint64, sub, 0xffffffffffffffffULL)
+PSNIP_SAFE_DEFINE_PROMOTED_UNSIGNED_BINARY_OP(psnip_uint64_t, uint64, mul, 0xffffffffffffffffULL)
+#else
+PSNIP_SAFE_DEFINE_UNSIGNED_ADD(psnip_uint64_t, uint64, 0xffffffffffffffffULL)
+PSNIP_SAFE_DEFINE_UNSIGNED_SUB(psnip_uint64_t, uint64, 0xffffffffffffffffULL)
+PSNIP_SAFE_DEFINE_UNSIGNED_MUL(psnip_uint64_t, uint64, 0xffffffffffffffffULL)
+#endif
+PSNIP_SAFE_DEFINE_UNSIGNED_DIV(psnip_uint64_t, uint64, 0xffffffffffffffffULL)
+PSNIP_SAFE_DEFINE_UNSIGNED_MOD(psnip_uint64_t, uint64, 0xffffffffffffffffULL)
+
+#endif /* !defined(PSNIP_SAFE_NO_FIXED) */
+
+#define PSNIP_SAFE_C11_GENERIC_SELECTION(res, op) \
+  _Generic((*res), \
+	   char: psnip_safe_char_##op, \
+	   unsigned char: psnip_safe_uchar_##op, \
+	   short: psnip_safe_short_##op, \
+	   unsigned short: psnip_safe_ushort_##op, \
+	   int: psnip_safe_int_##op, \
+	   unsigned int: psnip_safe_uint_##op, \
+	   long: psnip_safe_long_##op, \
+	   unsigned long: psnip_safe_ulong_##op, \
+	   long long: psnip_safe_llong_##op, \
+	   unsigned long long: psnip_safe_ullong_##op)
+
+#define PSNIP_SAFE_C11_GENERIC_BINARY_OP(op, res, a, b) \
+  PSNIP_SAFE_C11_GENERIC_SELECTION(res, op)(res, a, b)
+#define PSNIP_SAFE_C11_GENERIC_UNARY_OP(op, res, v) \
+  PSNIP_SAFE_C11_GENERIC_SELECTION(res, op)(res, v)
+
+#if defined(PSNIP_SAFE_HAVE_BUILTIN_OVERFLOW)
+#define psnip_safe_add(res, a, b) !__builtin_add_overflow(a, b, res)
+#define psnip_safe_sub(res, a, b) !__builtin_sub_overflow(a, b, res)
+#define psnip_safe_mul(res, a, b) !__builtin_mul_overflow(a, b, res)
+#define psnip_safe_div(res, a, b) !__builtin_div_overflow(a, b, res)
+#define psnip_safe_mod(res, a, b) !__builtin_mod_overflow(a, b, res)
+#define psnip_safe_neg(res, v)    PSNIP_SAFE_C11_GENERIC_UNARY_OP (neg, res, v)
+
+#elif defined(__STDC_VERSION__) && (__STDC_VERSION__ >= 201112L)
+/* The are no fixed-length or size selections because they cause an
+ * error about _Generic specifying two compatible types.  Hopefully
+ * this doesn't cause problems on exotic platforms, but if it does
+ * please let me know and I'll try to figure something out. */
+
+#define psnip_safe_add(res, a, b) PSNIP_SAFE_C11_GENERIC_BINARY_OP(add, res, a, b)
+#define psnip_safe_sub(res, a, b) PSNIP_SAFE_C11_GENERIC_BINARY_OP(sub, res, a, b)
+#define psnip_safe_mul(res, a, b) PSNIP_SAFE_C11_GENERIC_BINARY_OP(mul, res, a, b)
+#define psnip_safe_div(res, a, b) PSNIP_SAFE_C11_GENERIC_BINARY_OP(div, res, a, b)
+#define psnip_safe_mod(res, a, b) PSNIP_SAFE_C11_GENERIC_BINARY_OP(mod, res, a, b)
+#define psnip_safe_neg(res, v)    PSNIP_SAFE_C11_GENERIC_UNARY_OP (neg, res, v)
+#endif
+
+#if !defined(PSNIP_SAFE_HAVE_BUILTINS) && (defined(PSNIP_SAFE_EMULATE_NATIVE) || defined(PSNIP_BUILTIN_EMULATE_NATIVE))
+#  define __builtin_sadd_overflow(a, b, res)   (!psnip_safe_int_add(res, a, b))
+#  define __builtin_saddl_overflow(a, b, res)  (!psnip_safe_long_add(res, a, b))
+#  define __builtin_saddll_overflow(a, b, res) (!psnip_safe_llong_add(res, a, b))
+#  define __builtin_uadd_overflow(a, b, res)   (!psnip_safe_uint_add(res, a, b))
+#  define __builtin_uaddl_overflow(a, b, res)  (!psnip_safe_ulong_add(res, a, b))
+#  define __builtin_uaddll_overflow(a, b, res) (!psnip_safe_ullong_add(res, a, b))
+
+#  define __builtin_ssub_overflow(a, b, res)   (!psnip_safe_int_sub(res, a, b))
+#  define __builtin_ssubl_overflow(a, b, res)  (!psnip_safe_long_sub(res, a, b))
+#  define __builtin_ssubll_overflow(a, b, res) (!psnip_safe_llong_sub(res, a, b))
+#  define __builtin_usub_overflow(a, b, res)   (!psnip_safe_uint_sub(res, a, b))
+#  define __builtin_usubl_overflow(a, b, res)  (!psnip_safe_ulong_sub(res, a, b))
+#  define __builtin_usubll_overflow(a, b, res) (!psnip_safe_ullong_sub(res, a, b))
+
+#  define __builtin_smul_overflow(a, b, res)   (!psnip_safe_int_mul(res, a, b))
+#  define __builtin_smull_overflow(a, b, res)  (!psnip_safe_long_mul(res, a, b))
+#  define __builtin_smulll_overflow(a, b, res) (!psnip_safe_llong_mul(res, a, b))
+#  define __builtin_umul_overflow(a, b, res)   (!psnip_safe_uint_mul(res, a, b))
+#  define __builtin_umull_overflow(a, b, res)  (!psnip_safe_ulong_mul(res, a, b))
+#  define __builtin_umulll_overflow(a, b, res) (!psnip_safe_ullong_mul(res, a, b))
+#endif
+
+#endif /* !defined(PSNIP_SAFE_H) */
diff --git a/cpp/src/arrow/visitor_inline.h b/cpp/src/arrow/visitor_inline.h
index 233b105..bff97fc 100644
--- a/cpp/src/arrow/visitor_inline.h
+++ b/cpp/src/arrow/visitor_inline.h
@@ -134,75 +134,6 @@ namespace internal {
 template <typename T, typename Enable = void>
 struct ArrayDataInlineVisitor {};
 
-namespace detail {
-
-template <typename VisitNotNull, typename VisitNull>
-Status VisitBitBlocks(const std::shared_ptr<Buffer>& bitmap_buf, int64_t offset,
-                      int64_t length, VisitNotNull&& visit_not_null,
-                      VisitNull&& visit_null) {
-  const uint8_t* bitmap = nullptr;
-  if (bitmap_buf != nullptr) {
-    bitmap = bitmap_buf->data();
-  }
-  internal::OptionalBitBlockCounter bit_counter(bitmap, offset, length);
-  int64_t position = 0;
-  while (position < length) {
-    internal::BitBlockCount block = bit_counter.NextBlock();
-    if (block.AllSet()) {
-      for (int64_t i = 0; i < block.length; ++i, ++position) {
-        ARROW_RETURN_NOT_OK(visit_not_null(position));
-      }
-    } else if (block.NoneSet()) {
-      for (int64_t i = 0; i < block.length; ++i, ++position) {
-        ARROW_RETURN_NOT_OK(visit_null());
-      }
-    } else {
-      for (int64_t i = 0; i < block.length; ++i, ++position) {
-        if (BitUtil::GetBit(bitmap, offset + position)) {
-          ARROW_RETURN_NOT_OK(visit_not_null(position));
-        } else {
-          ARROW_RETURN_NOT_OK(visit_null());
-        }
-      }
-    }
-  }
-  return Status::OK();
-}
-
-template <typename VisitNotNull, typename VisitNull>
-void VisitBitBlocksVoid(const std::shared_ptr<Buffer>& bitmap_buf, int64_t offset,
-                        int64_t length, VisitNotNull&& visit_not_null,
-                        VisitNull&& visit_null) {
-  const uint8_t* bitmap = nullptr;
-  if (bitmap_buf != nullptr) {
-    bitmap = bitmap_buf->data();
-  }
-  internal::OptionalBitBlockCounter bit_counter(bitmap, offset, length);
-  int64_t position = 0;
-  while (position < length) {
-    internal::BitBlockCount block = bit_counter.NextBlock();
-    if (block.AllSet()) {
-      for (int64_t i = 0; i < block.length; ++i, ++position) {
-        visit_not_null(position);
-      }
-    } else if (block.NoneSet()) {
-      for (int64_t i = 0; i < block.length; ++i, ++position) {
-        visit_null();
-      }
-    } else {
-      for (int64_t i = 0; i < block.length; ++i, ++position) {
-        if (BitUtil::GetBit(bitmap, offset + position)) {
-          visit_not_null(position);
-        } else {
-          visit_null();
-        }
-      }
-    }
-  }
-}
-
-}  // namespace detail
-
 // Numeric and primitive C-compatible types
 template <typename T>
 struct ArrayDataInlineVisitor<T, enable_if_has_c_type<T>> {
@@ -213,9 +144,8 @@ struct ArrayDataInlineVisitor<T, enable_if_has_c_type<T>> {
                             NullFunc&& null_func) {
     const c_type* data = arr.GetValues<c_type>(1);
     auto visit_valid = [&](int64_t i) { return valid_func(data[i]); };
-    return detail::VisitBitBlocks(arr.buffers[0], arr.offset, arr.length,
-                                  std::move(visit_valid),
-                                  std::forward<NullFunc>(null_func));
+    return VisitBitBlocks(arr.buffers[0], arr.offset, arr.length, std::move(visit_valid),
+                          std::forward<NullFunc>(null_func));
   }
 
   template <typename ValidFunc, typename NullFunc>
@@ -224,8 +154,8 @@ struct ArrayDataInlineVisitor<T, enable_if_has_c_type<T>> {
     using c_type = typename T::c_type;
     const c_type* data = arr.GetValues<c_type>(1);
     auto visit_valid = [&](int64_t i) { valid_func(data[i]); };
-    detail::VisitBitBlocksVoid(arr.buffers[0], arr.offset, arr.length,
-                               std::move(visit_valid), std::forward<NullFunc>(null_func));
+    VisitBitBlocksVoid(arr.buffers[0], arr.offset, arr.length, std::move(visit_valid),
+                       std::forward<NullFunc>(null_func));
   }
 };
 
@@ -239,7 +169,7 @@ struct ArrayDataInlineVisitor<BooleanType> {
                             NullFunc&& null_func) {
     int64_t offset = arr.offset;
     const uint8_t* data = arr.buffers[1]->data();
-    return detail::VisitBitBlocks(
+    return VisitBitBlocks(
         arr.buffers[0], offset, arr.length,
         [&](int64_t i) { return valid_func(BitUtil::GetBit(data, offset + i)); },
         std::forward<NullFunc>(null_func));
@@ -250,7 +180,7 @@ struct ArrayDataInlineVisitor<BooleanType> {
                         NullFunc&& null_func) {
     int64_t offset = arr.offset;
     const uint8_t* data = arr.buffers[1]->data();
-    detail::VisitBitBlocksVoid(
+    VisitBitBlocksVoid(
         arr.buffers[0], offset, arr.length,
         [&](int64_t i) { valid_func(BitUtil::GetBit(data, offset + i)); },
         std::forward<NullFunc>(null_func));
@@ -278,7 +208,7 @@ struct ArrayDataInlineVisitor<T, enable_if_base_binary<T>> {
       data = arr.GetValues<char>(2, /*absolute_offset=*/0);
     }
     offset_type cur_offset = *offsets++;
-    return detail::VisitBitBlocks(
+    return VisitBitBlocks(
         arr.buffers[0], arr.offset, arr.length,
         [&](int64_t i) {
           ARROW_UNUSED(i);
@@ -308,7 +238,7 @@ struct ArrayDataInlineVisitor<T, enable_if_base_binary<T>> {
       data = arr.GetValues<uint8_t>(2, /*absolute_offset=*/0);
     }
 
-    detail::VisitBitBlocksVoid(
+    VisitBitBlocksVoid(
         arr.buffers[0], arr.offset, arr.length,
         [&](int64_t i) {
           auto value = util::string_view(reinterpret_cast<const char*>(data + offsets[i]),
@@ -333,7 +263,7 @@ struct ArrayDataInlineVisitor<T, enable_if_fixed_size_binary<T>> {
     const char* data = arr.GetValues<char>(1,
                                            /*absolute_offset=*/arr.offset * byte_width);
 
-    return detail::VisitBitBlocks(
+    return VisitBitBlocks(
         arr.buffers[0], arr.offset, arr.length,
         [&](int64_t i) {
           auto value = util::string_view(data, byte_width);
@@ -355,7 +285,7 @@ struct ArrayDataInlineVisitor<T, enable_if_fixed_size_binary<T>> {
     const char* data = arr.GetValues<char>(1,
                                            /*absolute_offset=*/arr.offset * byte_width);
 
-    detail::VisitBitBlocksVoid(
+    VisitBitBlocksVoid(
         arr.buffers[0], arr.offset, arr.length,
         [&](int64_t i) {
           valid_func(util::string_view(data, byte_width));
diff --git a/cpp/src/parquet/arrow/reader_internal.cc b/cpp/src/parquet/arrow/reader_internal.cc
index 3d68bf8..903cbab 100644
--- a/cpp/src/parquet/arrow/reader_internal.cc
+++ b/cpp/src/parquet/arrow/reader_internal.cc
@@ -41,7 +41,7 @@
 #include "arrow/type_traits.h"
 #include "arrow/util/base64.h"
 #include "arrow/util/checked_cast.h"
-#include "arrow/util/int_util.h"
+#include "arrow/util/int_util_internal.h"
 #include "arrow/util/logging.h"
 #include "arrow/util/string_view.h"
 #include "arrow/util/ubsan.h"
@@ -55,6 +55,8 @@
 #include "parquet/schema.h"
 #include "parquet/statistics.h"
 #include "parquet/types.h"
+// Required after "arrow/util/int_util_internal.h" (for OPTIONAL)
+#include "parquet/windows_compatibility.h"
 
 using arrow::Array;
 using arrow::BooleanArray;
diff --git a/cpp/src/parquet/column_reader.cc b/cpp/src/parquet/column_reader.cc
index cd1b288..8d9a0f7 100644
--- a/cpp/src/parquet/column_reader.cc
+++ b/cpp/src/parquet/column_reader.cc
@@ -35,7 +35,7 @@
 #include "arrow/util/bit_stream_utils.h"
 #include "arrow/util/checked_cast.h"
 #include "arrow/util/compression.h"
-#include "arrow/util/int_util.h"
+#include "arrow/util/int_util_internal.h"
 #include "arrow/util/logging.h"
 #include "arrow/util/rle_encoding.h"
 #include "parquet/column_page.h"
@@ -46,9 +46,13 @@
 #include "parquet/properties.h"
 #include "parquet/statistics.h"
 #include "parquet/thrift_internal.h"  // IWYU pragma: keep
+// Required after "arrow/util/int_util_internal.h" (for OPTIONAL)
+#include "parquet/windows_compatibility.h"
 
 using arrow::MemoryPool;
+using arrow::internal::AddWithOverflow;
 using arrow::internal::checked_cast;
+using arrow::internal::MultiplyWithOverflow;
 
 namespace parquet {
 
@@ -1028,10 +1032,11 @@ class TypedRecordReader : public ColumnReaderImplBase<DType>,
   // Compute the values capacity in bytes for the given number of elements
   int64_t bytes_for_values(int64_t nitems) const {
     int64_t type_size = GetTypeByteSize(this->descr_->physical_type());
-    if (::arrow::internal::HasPositiveMultiplyOverflow(nitems, type_size)) {
+    int64_t bytes_for_values = -1;
+    if (MultiplyWithOverflow(nitems, type_size, &bytes_for_values)) {
       throw ParquetException("Total size of items too large");
     }
-    return nitems * type_size;
+    return bytes_for_values;
   }
 
   int64_t ReadRecords(int64_t num_records) override {
@@ -1196,10 +1201,10 @@ class TypedRecordReader : public ColumnReaderImplBase<DType>,
     if (extra_size < 0) {
       throw ParquetException("Negative size (corrupt file?)");
     }
-    if (::arrow::internal::HasPositiveAdditionOverflow(size, extra_size)) {
+    int64_t target_size = -1;
+    if (AddWithOverflow(size, extra_size, &target_size)) {
       throw ParquetException("Allocation size too large (corrupt file?)");
     }
-    const int64_t target_size = size + extra_size;
     if (target_size >= (1LL << 62)) {
       throw ParquetException("Allocation size too large (corrupt file?)");
     }
@@ -1215,14 +1220,13 @@ class TypedRecordReader : public ColumnReaderImplBase<DType>,
           UpdateCapacity(levels_capacity_, levels_written_, extra_levels);
       if (new_levels_capacity > levels_capacity_) {
         constexpr auto kItemSize = static_cast<int64_t>(sizeof(int16_t));
-        if (::arrow::internal::HasPositiveMultiplyOverflow(new_levels_capacity,
-                                                           kItemSize)) {
+        int64_t capacity_in_bytes = -1;
+        if (MultiplyWithOverflow(new_levels_capacity, kItemSize, &capacity_in_bytes)) {
           throw ParquetException("Allocation size too large (corrupt file?)");
         }
-        PARQUET_THROW_NOT_OK(def_levels_->Resize(new_levels_capacity * kItemSize, false));
+        PARQUET_THROW_NOT_OK(def_levels_->Resize(capacity_in_bytes, false));
         if (this->max_rep_level_ > 0) {
-          PARQUET_THROW_NOT_OK(
-              rep_levels_->Resize(new_levels_capacity * kItemSize, false));
+          PARQUET_THROW_NOT_OK(rep_levels_->Resize(capacity_in_bytes, false));
         }
         levels_capacity_ = new_levels_capacity;
       }
diff --git a/cpp/src/parquet/types.h b/cpp/src/parquet/types.h
index f7f7bb7..950f038 100644
--- a/cpp/src/parquet/types.h
+++ b/cpp/src/parquet/types.h
@@ -24,7 +24,6 @@
 #include <sstream>
 #include <string>
 
-#include "arrow/util/int_util.h"
 #include "arrow/util/string_view.h"
 
 #include "parquet/platform.h"