You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@arrow.apache.org by we...@apache.org on 2020/06/24 03:32:15 UTC

[arrow] branch master updated: ARROW-8934: [C++] Enable `compute::Subtract` with timestamp inputs to return duration

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

wesm 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 dd94a58  ARROW-8934: [C++] Enable `compute::Subtract` with timestamp inputs to return duration
dd94a58 is described below

commit dd94a5809b56b32fe2fb538f688bf568d9642e3b
Author: Wes McKinney <we...@apache.org>
AuthorDate: Tue Jun 23 22:31:51 2020 -0500

    ARROW-8934: [C++] Enable `compute::Subtract` with timestamp inputs to return duration
    
    I also did a little bit of cleaning, moving some stuff into `arrow::compute::internal`.
    
    Closes #7530 from wesm/ARROW-8934
    
    Authored-by: Wes McKinney <we...@apache.org>
    Signed-off-by: Wes McKinney <we...@apache.org>
---
 cpp/src/arrow/compute/kernels/codegen_internal.cc  |   8 +
 cpp/src/arrow/compute/kernels/codegen_internal.h   |   5 +
 cpp/src/arrow/compute/kernels/scalar_arithmetic.cc |  46 +++-
 .../compute/kernels/scalar_arithmetic_test.cc      | 241 +++++++++++----------
 cpp/src/arrow/compute/kernels/scalar_compare.cc    |   4 +-
 cpp/src/arrow/compute/kernels/scalar_set_lookup.cc |   3 +-
 cpp/src/arrow/compute/kernels/vector_hash.cc       |   3 +-
 7 files changed, 181 insertions(+), 129 deletions(-)

diff --git a/cpp/src/arrow/compute/kernels/codegen_internal.cc b/cpp/src/arrow/compute/kernels/codegen_internal.cc
index 2606c97..641998f 100644
--- a/cpp/src/arrow/compute/kernels/codegen_internal.cc
+++ b/cpp/src/arrow/compute/kernels/codegen_internal.cc
@@ -26,6 +26,7 @@
 
 namespace arrow {
 namespace compute {
+namespace internal {
 
 void ExecFail(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
   ctx->SetStatus(Status::NotImplemented("This kernel is malformed"));
@@ -131,6 +132,12 @@ const std::vector<std::shared_ptr<DataType>>& FloatingPointTypes() {
   return g_floating_types;
 }
 
+const std::vector<TimeUnit::type>& AllTimeUnits() {
+  static std::vector<TimeUnit::type> units = {TimeUnit::SECOND, TimeUnit::MILLI,
+                                              TimeUnit::MICRO, TimeUnit::NANO};
+  return units;
+}
+
 const std::vector<std::shared_ptr<DataType>>& NumericTypes() {
   std::call_once(codegen_static_initialized, InitStaticData);
   return g_numeric_types;
@@ -172,5 +179,6 @@ Result<ValueDescr> FirstType(KernelContext*, const std::vector<ValueDescr>& desc
   return descrs[0];
 }
 
+}  // namespace internal
 }  // namespace compute
 }  // namespace arrow
diff --git a/cpp/src/arrow/compute/kernels/codegen_internal.h b/cpp/src/arrow/compute/kernels/codegen_internal.h
index c808196..fdceb53 100644
--- a/cpp/src/arrow/compute/kernels/codegen_internal.h
+++ b/cpp/src/arrow/compute/kernels/codegen_internal.h
@@ -54,6 +54,7 @@ using internal::FirstTimeBitmapWriter;
 using internal::GenerateBitsUnrolled;
 
 namespace compute {
+namespace internal {
 
 #ifdef ARROW_EXTRA_ERROR_CONTEXT
 
@@ -257,6 +258,9 @@ const std::vector<std::shared_ptr<DataType>>& UnsignedIntTypes();
 const std::vector<std::shared_ptr<DataType>>& IntTypes();
 const std::vector<std::shared_ptr<DataType>>& FloatingPointTypes();
 
+ARROW_EXPORT
+const std::vector<TimeUnit::type>& AllTimeUnits();
+
 // Returns a vector of example instances of parametric types such as
 //
 // * Decimal
@@ -885,5 +889,6 @@ ArrayKernelExec GenerateTemporal(detail::GetTypeId get_id) {
 // END of kernel generator-dispatchers
 // ----------------------------------------------------------------------
 
+}  // namespace internal
 }  // namespace compute
 }  // namespace arrow
diff --git a/cpp/src/arrow/compute/kernels/scalar_arithmetic.cc b/cpp/src/arrow/compute/kernels/scalar_arithmetic.cc
index 5100323..82a8f15 100644
--- a/cpp/src/arrow/compute/kernels/scalar_arithmetic.cc
+++ b/cpp/src/arrow/compute/kernels/scalar_arithmetic.cc
@@ -24,6 +24,7 @@
 
 namespace arrow {
 namespace compute {
+namespace internal {
 namespace {
 
 template <typename T>
@@ -241,6 +242,7 @@ ArrayKernelExec NumericEqualTypesBinary(detail::GetTypeId get_id) {
     case Type::UINT32:
       return ScalarBinaryEqualTypes<UInt32Type, UInt32Type, Op>::Exec;
     case Type::INT64:
+    case Type::TIMESTAMP:
       return ScalarBinaryEqualTypes<Int64Type, Int64Type, Op>::Exec;
     case Type::UINT64:
       return ScalarBinaryEqualTypes<UInt64Type, UInt64Type, Op>::Exec;
@@ -255,26 +257,50 @@ ArrayKernelExec NumericEqualTypesBinary(detail::GetTypeId get_id) {
 }
 
 template <typename Op>
-void AddBinaryFunction(std::string name, FunctionRegistry* registry) {
+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);
     DCHECK_OK(func->AddKernel({ty, ty}, ty, exec));
   }
-  DCHECK_OK(registry->AddFunction(std::move(func)));
+  return func;
 }
 
 }  // namespace
 
-namespace internal {
-
 void RegisterScalarArithmetic(FunctionRegistry* registry) {
-  AddBinaryFunction<Add>("add", registry);
-  AddBinaryFunction<AddChecked>("add_checked", registry);
-  AddBinaryFunction<Subtract>("subtract", registry);
-  AddBinaryFunction<SubtractChecked>("subtract_checked", registry);
-  AddBinaryFunction<Multiply>("multiply", registry);
-  AddBinaryFunction<MultiplyChecked>("multiply_checked", registry);
+  // ----------------------------------------------------------------------
+  auto add = MakeArithmeticFunction<Add>("add");
+  DCHECK_OK(registry->AddFunction(std::move(add)));
+
+  // ----------------------------------------------------------------------
+  auto add_checked = MakeArithmeticFunction<AddChecked>("add_checked");
+  DCHECK_OK(registry->AddFunction(std::move(add_checked)));
+
+  // ----------------------------------------------------------------------
+  // subtract
+  auto subtract = MakeArithmeticFunction<Subtract>("subtract");
+
+  // Add subtract(timestamp, timestamp) -> duration
+  for (auto unit : AllTimeUnits()) {
+    InputType in_type(match::TimestampTypeUnit(unit));
+    auto exec = NumericEqualTypesBinary<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");
+  DCHECK_OK(registry->AddFunction(std::move(subtract_checked)));
+
+  // ----------------------------------------------------------------------
+  auto multiply = MakeArithmeticFunction<Multiply>("multiply");
+  DCHECK_OK(registry->AddFunction(std::move(multiply)));
+
+  // ----------------------------------------------------------------------
+  auto multiply_checked = MakeArithmeticFunction<MultiplyChecked>("multiply_checked");
+  DCHECK_OK(registry->AddFunction(std::move(multiply_checked)));
 }
 
 }  // namespace internal
diff --git a/cpp/src/arrow/compute/kernels/scalar_arithmetic_test.cc b/cpp/src/arrow/compute/kernels/scalar_arithmetic_test.cc
index 8b9b5e4..e0f4890 100644
--- a/cpp/src/arrow/compute/kernels/scalar_arithmetic_test.cc
+++ b/cpp/src/arrow/compute/kernels/scalar_arithmetic_test.cc
@@ -25,6 +25,7 @@
 
 #include "arrow/array.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"
@@ -39,7 +40,7 @@ namespace arrow {
 namespace compute {
 
 template <typename ArrowType>
-class TestBinaryArithmetics : public TestBase {
+class TestBinaryArithmetic : public TestBase {
  protected:
   using CType = typename ArrowType::c_type;
 
@@ -116,16 +117,16 @@ std::string MakeArray(Elements... elements) {
 }
 
 template <typename T>
-class TestBinaryArithmeticsIntegral : public TestBinaryArithmetics<T> {};
+class TestBinaryArithmeticIntegral : public TestBinaryArithmetic<T> {};
 
 template <typename T>
-class TestBinaryArithmeticsSigned : public TestBinaryArithmeticsIntegral<T> {};
+class TestBinaryArithmeticSigned : public TestBinaryArithmeticIntegral<T> {};
 
 template <typename T>
-class TestBinaryArithmeticsUnsigned : public TestBinaryArithmeticsIntegral<T> {};
+class TestBinaryArithmeticUnsigned : public TestBinaryArithmeticIntegral<T> {};
 
 template <typename T>
-class TestBinaryArithmeticsFloating : public TestBinaryArithmetics<T> {};
+class TestBinaryArithmeticFloating : public TestBinaryArithmetic<T> {};
 
 // InputType - OutputType pairs
 using IntegralTypes = testing::Types<Int8Type, Int16Type, Int32Type, Int64Type, UInt8Type,
@@ -139,90 +140,111 @@ using UnsignedIntegerTypes =
 // TODO(kszucs): add half-float
 using FloatingTypes = testing::Types<FloatType, DoubleType>;
 
-TYPED_TEST_SUITE(TestBinaryArithmeticsIntegral, IntegralTypes);
-TYPED_TEST_SUITE(TestBinaryArithmeticsSigned, SignedIntegerTypes);
-TYPED_TEST_SUITE(TestBinaryArithmeticsUnsigned, UnsignedIntegerTypes);
-TYPED_TEST_SUITE(TestBinaryArithmeticsFloating, FloatingTypes);
+TYPED_TEST_SUITE(TestBinaryArithmeticIntegral, IntegralTypes);
+TYPED_TEST_SUITE(TestBinaryArithmeticSigned, SignedIntegerTypes);
+TYPED_TEST_SUITE(TestBinaryArithmeticUnsigned, UnsignedIntegerTypes);
+TYPED_TEST_SUITE(TestBinaryArithmeticFloating, FloatingTypes);
 
-TYPED_TEST(TestBinaryArithmeticsIntegral, Add) {
+TYPED_TEST(TestBinaryArithmeticIntegral, Add) {
   for (auto check_overflow : {false, true}) {
     this->SetOverflowCheck(check_overflow);
 
-    this->AssertBinop(arrow::compute::Add, "[]", "[]", "[]");
-    this->AssertBinop(arrow::compute::Add, "[null]", "[null]", "[null]");
-    this->AssertBinop(arrow::compute::Add, "[3, 2, 6]", "[1, 0, 2]", "[4, 2, 8]");
+    this->AssertBinop(Add, "[]", "[]", "[]");
+    this->AssertBinop(Add, "[null]", "[null]", "[null]");
+    this->AssertBinop(Add, "[3, 2, 6]", "[1, 0, 2]", "[4, 2, 8]");
 
-    this->AssertBinop(arrow::compute::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, "[1, 2, 3, 4, 5, 6, 7]", "[0, 1, 2, 3, 4, 5, 6]",
+                      "[1, 3, 5, 7, 9, 11, 13]");
 
-    this->AssertBinop(arrow::compute::Add, "[10, 12, 4, 50, 50, 32, 11]",
-                      "[2, 0, 6, 1, 5, 3, 4]", "[12, 12, 10, 51, 55, 35, 15]");
-    this->AssertBinop(arrow::compute::Add, "[null, 1, 3, null, 2, 5]",
-                      "[1, 4, 2, 5, 0, 3]", "[null, 5, 5, null, 2, 8]");
-    this->AssertBinop(arrow::compute::Add, 10, "[null, 1, 3, null, 2, 5]",
+    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(arrow::compute::Add, 17, 42, 59);
+    this->AssertBinop(Add, 17, 42, 59);
   }
 }
 
-TYPED_TEST(TestBinaryArithmeticsIntegral, Sub) {
+TYPED_TEST(TestBinaryArithmeticIntegral, Sub) {
   for (auto check_overflow : {false, true}) {
     this->SetOverflowCheck(check_overflow);
 
-    this->AssertBinop(arrow::compute::Subtract, "[]", "[]", "[]");
-    this->AssertBinop(arrow::compute::Subtract, "[null]", "[null]", "[null]");
-    this->AssertBinop(arrow::compute::Subtract, "[3, 2, 6]", "[1, 0, 2]", "[2, 2, 4]");
-    this->AssertBinop(arrow::compute::Subtract, "[1, 2, 3, 4, 5, 6, 7]",
-                      "[0, 1, 2, 3, 4, 5, 6]", "[1, 1, 1, 1, 1, 1, 1]");
-    this->AssertBinop(arrow::compute::Subtract, 10, "[null, 1, 3, null, 2, 5]",
+    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(arrow::compute::Subtract, 20, 9, 11);
+    this->AssertBinop(Subtract, 20, 9, 11);
   }
 }
 
-TYPED_TEST(TestBinaryArithmeticsIntegral, Mul) {
+TEST(TestBinaryArithmetic, SubtractTimestamps) {
+  random::RandomArrayGenerator rand(kRandomSeed);
+
+  const int64_t length = 100;
+
+  auto lhs = rand.Int64(length, 0, 100000000);
+  auto rhs = rand.Int64(length, 0, 100000000);
+  auto expected_int64 = (*Subtract(lhs, rhs)).make_array();
+
+  for (auto unit : internal::AllTimeUnits()) {
+    auto timestamp_ty = timestamp(unit);
+    auto duration_ty = duration(unit);
+
+    auto lhs_timestamp = *lhs->View(timestamp_ty);
+    auto rhs_timestamp = *rhs->View(timestamp_ty);
+
+    auto result = (*Subtract(lhs_timestamp, rhs_timestamp)).make_array();
+    ASSERT_TRUE(result->type()->Equals(*duration_ty));
+    AssertArraysEqual(**result->View(int64()), *expected_int64);
+  }
+}
+
+TYPED_TEST(TestBinaryArithmeticIntegral, Mul) {
   for (auto check_overflow : {false, true}) {
     this->SetOverflowCheck(check_overflow);
 
-    this->AssertBinop(arrow::compute::Multiply, "[]", "[]", "[]");
-    this->AssertBinop(arrow::compute::Multiply, "[null]", "[null]", "[null]");
-    this->AssertBinop(arrow::compute::Multiply, "[3, 2, 6]", "[1, 0, 2]", "[3, 0, 12]");
-    this->AssertBinop(arrow::compute::Multiply, "[1, 2, 3, 4, 5, 6, 7]",
-                      "[0, 1, 2, 3, 4, 5, 6]", "[0, 2, 6, 12, 20, 30, 42]");
-    this->AssertBinop(arrow::compute::Multiply, "[7, 6, 5, 4, 3, 2, 1]",
-                      "[6, 5, 4, 3, 2, 1, 0]", "[42, 30, 20, 12, 6, 2, 0]");
-    this->AssertBinop(arrow::compute::Multiply, "[null, 1, 3, null, 2, 5]",
-                      "[1, 4, 2, 5, 0, 3]", "[null, 4, 6, null, 0, 15]");
-    this->AssertBinop(arrow::compute::Multiply, 3, "[null, 1, 3, null, 2, 5]",
+    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(arrow::compute::Multiply, 6, 7, 42);
+    this->AssertBinop(Multiply, 6, 7, 42);
   }
 }
 
-TYPED_TEST(TestBinaryArithmeticsSigned, Add) {
-  this->AssertBinop(arrow::compute::Add, "[-7, 6, 5, 4, 3, 2, 1]",
-                    "[-6, 5, -4, 3, -2, 1, 0]", "[-13, 11, 1, 7, 1, 3, 1]");
-  this->AssertBinop(arrow::compute::Add, -1, "[-6, 5, -4, 3, -2, 1, 0]",
-                    "[-7, 4, -5, 2, -3, 0, -1]");
-  this->AssertBinop(arrow::compute::Add, -10, 5, -5);
+TYPED_TEST(TestBinaryArithmeticSigned, Add) {
+  this->AssertBinop(Add, "[-7, 6, 5, 4, 3, 2, 1]", "[-6, 5, -4, 3, -2, 1, 0]",
+                    "[-13, 11, 1, 7, 1, 3, 1]");
+  this->AssertBinop(Add, -1, "[-6, 5, -4, 3, -2, 1, 0]", "[-7, 4, -5, 2, -3, 0, -1]");
+  this->AssertBinop(Add, -10, 5, -5);
 }
 
-TYPED_TEST(TestBinaryArithmeticsSigned, OverflowWraps) {
+TYPED_TEST(TestBinaryArithmeticSigned, OverflowWraps) {
   using CType = typename TestFixture::CType;
 
   auto min = std::numeric_limits<CType>::lowest();
   auto max = std::numeric_limits<CType>::max();
 
-  this->AssertBinop(arrow::compute::Add, MakeArray(min, max, max),
-                    MakeArray(CType(-1), 1, max), MakeArray(max, min, CType(-2)));
+  this->AssertBinop(Add, MakeArray(min, max, max), MakeArray(CType(-1), 1, max),
+                    MakeArray(max, min, CType(-2)));
 
-  this->AssertBinop(arrow::compute::Subtract, MakeArray(min, max, min),
-                    MakeArray(1, max, max), MakeArray(max, 0, 1));
-  this->AssertBinop(arrow::compute::Multiply, MakeArray(min, max, max),
-                    MakeArray(max, 2, max), MakeArray(min, CType(-2), 1));
+  this->AssertBinop(Subtract, MakeArray(min, max, min), MakeArray(1, max, max),
+                    MakeArray(max, 0, 1));
+  this->AssertBinop(Multiply, MakeArray(min, max, max), MakeArray(max, 2, max),
+                    MakeArray(min, CType(-2), 1));
 }
 
-TYPED_TEST(TestBinaryArithmeticsIntegral, OverflowRaises) {
+TYPED_TEST(TestBinaryArithmeticIntegral, OverflowRaises) {
   using CType = typename TestFixture::CType;
 
   auto min = std::numeric_limits<CType>::lowest();
@@ -230,18 +252,16 @@ TYPED_TEST(TestBinaryArithmeticsIntegral, OverflowRaises) {
 
   this->SetOverflowCheck(true);
 
-  this->AssertBinopRaises(arrow::compute::Add, MakeArray(min, max, max),
-                          MakeArray(CType(-1), 1, max), "overflow");
-  this->AssertBinopRaises(arrow::compute::Subtract, MakeArray(min, max),
-                          MakeArray(1, max), "overflow");
-  this->AssertBinopRaises(arrow::compute::Subtract, MakeArray(min), MakeArray(max),
+  this->AssertBinopRaises(Add, MakeArray(min, max, max), MakeArray(CType(-1), 1, max),
                           "overflow");
+  this->AssertBinopRaises(Subtract, MakeArray(min, max), MakeArray(1, max), "overflow");
+  this->AssertBinopRaises(Subtract, MakeArray(min), MakeArray(max), "overflow");
 
-  this->AssertBinopRaises(arrow::compute::Multiply, MakeArray(min, max, max),
-                          MakeArray(max, 2, max), "overflow");
+  this->AssertBinopRaises(Multiply, MakeArray(min, max, max), MakeArray(max, 2, max),
+                          "overflow");
 }
 
-TYPED_TEST(TestBinaryArithmeticsSigned, OverflowRaises) {
+TYPED_TEST(TestBinaryArithmeticSigned, OverflowRaises) {
   using CType = typename TestFixture::CType;
 
   auto min = std::numeric_limits<CType>::lowest();
@@ -249,95 +269,90 @@ TYPED_TEST(TestBinaryArithmeticsSigned, OverflowRaises) {
 
   this->SetOverflowCheck(true);
 
-  this->AssertBinop(arrow::compute::Multiply, MakeArray(max), MakeArray(-1),
-                    MakeArray(min + 1));
-  this->AssertBinopRaises(arrow::compute::Multiply, MakeArray(max), MakeArray(2),
-                          "overflow");
-  this->AssertBinopRaises(arrow::compute::Multiply, MakeArray(min), MakeArray(-1),
-                          "overflow");
+  this->AssertBinop(Multiply, MakeArray(max), MakeArray(-1), MakeArray(min + 1));
+  this->AssertBinopRaises(Multiply, MakeArray(max), MakeArray(2), "overflow");
+  this->AssertBinopRaises(Multiply, MakeArray(min), MakeArray(-1), "overflow");
 }
 
-TYPED_TEST(TestBinaryArithmeticsUnsigned, OverflowWraps) {
+TYPED_TEST(TestBinaryArithmeticUnsigned, OverflowWraps) {
   using CType = typename TestFixture::CType;
 
   auto min = std::numeric_limits<CType>::lowest();
   auto max = std::numeric_limits<CType>::max();
 
   this->SetOverflowCheck(false);
-  this->AssertBinop(arrow::compute::Add, MakeArray(min, max, max),
-                    MakeArray(CType(-1), 1, max), MakeArray(max, min, CType(-2)));
+  this->AssertBinop(Add, MakeArray(min, max, max), MakeArray(CType(-1), 1, max),
+                    MakeArray(max, min, CType(-2)));
 
-  this->AssertBinop(arrow::compute::Subtract, MakeArray(min, max, min),
-                    MakeArray(1, max, max), MakeArray(max, 0, 1));
+  this->AssertBinop(Subtract, MakeArray(min, max, min), MakeArray(1, max, max),
+                    MakeArray(max, 0, 1));
 
-  this->AssertBinop(arrow::compute::Multiply, MakeArray(min, max, max),
-                    MakeArray(max, 2, max), MakeArray(min, CType(-2), 1));
+  this->AssertBinop(Multiply, MakeArray(min, max, max), MakeArray(max, 2, max),
+                    MakeArray(min, CType(-2), 1));
 }
 
-TYPED_TEST(TestBinaryArithmeticsSigned, Sub) {
-  this->AssertBinop(arrow::compute::Subtract, "[0, 1, 2, 3, 4, 5, 6]",
-                    "[1, 2, 3, 4, 5, 6, 7]", "[-1, -1, -1, -1, -1, -1, -1]");
+TYPED_TEST(TestBinaryArithmeticSigned, Sub) {
+  this->AssertBinop(Subtract, "[0, 1, 2, 3, 4, 5, 6]", "[1, 2, 3, 4, 5, 6, 7]",
+                    "[-1, -1, -1, -1, -1, -1, -1]");
 
-  this->AssertBinop(arrow::compute::Subtract, "[0, 0, 0, 0, 0, 0, 0]",
-                    "[6, 5, 4, 3, 2, 1, 0]", "[-6, -5, -4, -3, -2, -1, 0]");
+  this->AssertBinop(Subtract, "[0, 0, 0, 0, 0, 0, 0]", "[6, 5, 4, 3, 2, 1, 0]",
+                    "[-6, -5, -4, -3, -2, -1, 0]");
 
-  this->AssertBinop(arrow::compute::Subtract, "[10, 12, 4, 50, 50, 32, 11]",
-                    "[2, 0, 6, 1, 5, 3, 4]", "[8, 12, -2, 49, 45, 29, 7]");
+  this->AssertBinop(Subtract, "[10, 12, 4, 50, 50, 32, 11]", "[2, 0, 6, 1, 5, 3, 4]",
+                    "[8, 12, -2, 49, 45, 29, 7]");
 
-  this->AssertBinop(arrow::compute::Subtract, "[null, 1, 3, null, 2, 5]",
-                    "[1, 4, 2, 5, 0, 3]", "[null, -3, 1, null, 2, 2]");
+  this->AssertBinop(Subtract, "[null, 1, 3, null, 2, 5]", "[1, 4, 2, 5, 0, 3]",
+                    "[null, -3, 1, null, 2, 2]");
 }
 
-TYPED_TEST(TestBinaryArithmeticsSigned, Mul) {
-  this->AssertBinop(arrow::compute::Multiply, "[-10, 12, 4, 50, -5, 32, 11]",
-                    "[-2, 0, -6, 1, 5, 3, 4]", "[20, 0, -24, 50, -25, 96, 44]");
-  this->AssertBinop(arrow::compute::Multiply, -2, "[-10, 12, 4, 50, -5, 32, 11]",
+TYPED_TEST(TestBinaryArithmeticSigned, Mul) {
+  this->AssertBinop(Multiply, "[-10, 12, 4, 50, -5, 32, 11]", "[-2, 0, -6, 1, 5, 3, 4]",
+                    "[20, 0, -24, 50, -25, 96, 44]");
+  this->AssertBinop(Multiply, -2, "[-10, 12, 4, 50, -5, 32, 11]",
                     "[20, -24, -8, -100, 10, -64, -22]");
-  this->AssertBinop(arrow::compute::Multiply, -5, -5, 25);
+  this->AssertBinop(Multiply, -5, -5, 25);
 }
 
-TYPED_TEST(TestBinaryArithmeticsFloating, Add) {
-  this->AssertBinop(arrow::compute::Add, "[]", "[]", "[]");
+TYPED_TEST(TestBinaryArithmeticFloating, Add) {
+  this->AssertBinop(Add, "[]", "[]", "[]");
 
-  this->AssertBinop(arrow::compute::Add, "[3.4, 2.6, 6.3]", "[1, 0, 2]",
-                    "[4.4, 2.6, 8.3]");
+  this->AssertBinop(Add, "[3.4, 2.6, 6.3]", "[1, 0, 2]", "[4.4, 2.6, 8.3]");
 
-  this->AssertBinop(arrow::compute::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, "[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(arrow::compute::Add, "[7, 6, 5, 4, 3, 2, 1]", "[6, 5, 4, 3, 2, 1, 0]",
+  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(arrow::compute::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, "[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(arrow::compute::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, "[null, 1, 3.3, null, 2, 5.3]", "[1, 4, 2, 5, 0, 3]",
+                    "[null, 5, 5.3, null, 2, 8.3]");
 
-  this->AssertBinop(arrow::compute::Add, 1.1F, "[null, 1, 3.3, null, 2, 5.3]",
+  this->AssertBinop(Add, 1.1F, "[null, 1, 3.3, null, 2, 5.3]",
                     "[null, 2.1, 4.4, null, 3.1, 6.4]");
 }
 
-TYPED_TEST(TestBinaryArithmeticsFloating, Sub) {
-  this->AssertBinop(arrow::compute::Subtract, "[]", "[]", "[]");
+TYPED_TEST(TestBinaryArithmeticFloating, Sub) {
+  this->AssertBinop(Subtract, "[]", "[]", "[]");
 
-  this->AssertBinop(arrow::compute::Subtract, "[3.4, 2.6, 6.3]", "[1, 0, 2]",
-                    "[2.4, 2.6, 4.3]");
+  this->AssertBinop(Subtract, "[3.4, 2.6, 6.3]", "[1, 0, 2]", "[2.4, 2.6, 4.3]");
 
-  this->AssertBinop(arrow::compute::Subtract, "[1.1, 2.4, 3.5, 4.3, 5.1, 6.8, 7.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(arrow::compute::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, "[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(arrow::compute::Subtract, "[10.4, 12, 4.2, 50, 50.3, 32, 11]",
+  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(arrow::compute::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, "[null, 1, 3.3, null, 2, 5.3]", "[1, 4, 2, 5, 0, 3]",
+                    "[null, -3, 1.3, null, 2, 2.3]");
 
-  this->AssertBinop(arrow::compute::Subtract, 0.1F, "[null, 1, 3.3, null, 2, 5.3]",
+  this->AssertBinop(Subtract, 0.1F, "[null, 1, 3.3, null, 2, 5.3]",
                     "[null, -0.9, -3.2, null, -1.9, -5.2]");
 }
 
diff --git a/cpp/src/arrow/compute/kernels/scalar_compare.cc b/cpp/src/arrow/compute/kernels/scalar_compare.cc
index bf67dba..1e44a35 100644
--- a/cpp/src/arrow/compute/kernels/scalar_compare.cc
+++ b/cpp/src/arrow/compute/kernels/scalar_compare.cc
@@ -90,7 +90,7 @@ std::shared_ptr<ScalarFunction> MakeCompareFunction(std::string name) {
   AddGenericCompare<DoubleType, Op>(float64(), func.get());
 
   // Add timestamp kernels
-  for (auto unit : {TimeUnit::SECOND, TimeUnit::MILLI, TimeUnit::MICRO, TimeUnit::NANO}) {
+  for (auto unit : AllTimeUnits()) {
     InputType in_type(match::TimestampTypeUnit(unit));
     auto exec =
         GeneratePhysicalInteger<applicator::ScalarBinaryEqualTypes, BooleanType, Op>(
@@ -99,7 +99,7 @@ std::shared_ptr<ScalarFunction> MakeCompareFunction(std::string name) {
   }
 
   // Duration
-  for (auto unit : {TimeUnit::SECOND, TimeUnit::MILLI, TimeUnit::MICRO, TimeUnit::NANO}) {
+  for (auto unit : AllTimeUnits()) {
     InputType in_type(match::DurationTypeUnit(unit));
     auto exec =
         GeneratePhysicalInteger<applicator::ScalarBinaryEqualTypes, BooleanType, Op>(
diff --git a/cpp/src/arrow/compute/kernels/scalar_set_lookup.cc b/cpp/src/arrow/compute/kernels/scalar_set_lookup.cc
index 27c4809..75a68dc 100644
--- a/cpp/src/arrow/compute/kernels/scalar_set_lookup.cc
+++ b/cpp/src/arrow/compute/kernels/scalar_set_lookup.cc
@@ -31,6 +31,7 @@ using internal::checked_cast;
 using internal::HashTraits;
 
 namespace compute {
+namespace internal {
 namespace {
 
 template <typename T, typename R = void>
@@ -316,8 +317,6 @@ void AddBasicSetLookupKernels(ScalarKernel kernel,
 
 }  // namespace
 
-namespace internal {
-
 void RegisterScalarSetLookup(FunctionRegistry* registry) {
   // IsIn always writes into preallocated memory
   {
diff --git a/cpp/src/arrow/compute/kernels/vector_hash.cc b/cpp/src/arrow/compute/kernels/vector_hash.cc
index b51124e..1cd677e 100644
--- a/cpp/src/arrow/compute/kernels/vector_hash.cc
+++ b/cpp/src/arrow/compute/kernels/vector_hash.cc
@@ -35,6 +35,7 @@ using internal::DictionaryTraits;
 using internal::HashTraits;
 
 namespace compute {
+namespace internal {
 
 namespace {
 
@@ -539,8 +540,6 @@ void AddHashKernels(VectorFunction* func, VectorKernel base,
 
 }  // namespace
 
-namespace internal {
-
 void RegisterVectorHash(FunctionRegistry* registry) {
   VectorKernel base;
   base.exec = HashExec;