You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@arrow.apache.org by GitBox <gi...@apache.org> on 2020/08/10 14:09:03 UTC

[GitHub] [arrow] pitrou commented on a change in pull request #7920: ARROW-9638: [C++][Compute] Implement mode kernel

pitrou commented on a change in pull request #7920:
URL: https://github.com/apache/arrow/pull/7920#discussion_r467911033



##########
File path: cpp/src/arrow/compute/api_aggregate.h
##########
@@ -147,5 +147,20 @@ Result<Datum> MinMax(const Array& array,
                      const MinMaxOptions& options = MinMaxOptions::Defaults(),
                      ExecContext* ctx = NULLPTR);
 
+/// \brief Calculate the modal (most common) value of a numeric array
+///
+/// This function returns both mode and count as a struct scalar, with type
+/// struct<mode: T, count: int64>, where T is the input type.
+/// If there is more than one such value, the smallest one is returned.
+///
+/// \param[in] value input datum, expecting Array
+/// \param[in] ctx the function execution context, optional
+/// \return resulting datum as a struct<mode: T, count: int64> scalar
+///
+/// \since 1.0.0

Review comment:
       Rather 2.0.0.

##########
File path: cpp/src/arrow/compute/kernels/aggregate_test.cc
##########
@@ -594,5 +594,92 @@ TYPED_TEST(TestFloatingMinMaxKernel, DefaultOptions) {
   AssertDatumsEqual(explicit_defaults, no_options_provided);
 }
 
+///
+/// Mode
+///
+
+template <typename ArrowType>
+class TestPrimitiveModeKernel : public ::testing::Test {
+  using Traits = TypeTraits<ArrowType>;
+  using c_type = typename ArrowType::c_type;
+  using ModeType = typename Traits::ScalarType;
+  using CountType = typename TypeTraits<Int64Type>::ScalarType;
+
+ public:
+  void AssertModeIs(const Datum& array, c_type expected_mode, int64_t expected_count) {
+    ASSERT_OK_AND_ASSIGN(Datum out, Mode(array));
+    const StructScalar& value = out.scalar_as<StructScalar>();
+
+    const auto& out_mode = checked_cast<const ModeType&>(*value.value[0]);
+    ASSERT_EQ(expected_mode, out_mode.value);
+
+    const auto& out_count = checked_cast<const CountType&>(*value.value[1]);
+    ASSERT_EQ(expected_count, out_count.value);
+  }
+
+  void AssertModeIs(const std::string& json, c_type expected_mode,
+                    int64_t expected_count) {
+    auto array = ArrayFromJSON(type_singleton(), json);
+    AssertModeIs(array, expected_mode, expected_count);
+  }
+
+  void AssertModeIsNull(const Datum& array) {
+    ASSERT_OK_AND_ASSIGN(Datum out, Mode(array));
+    const StructScalar& value = out.scalar_as<StructScalar>();
+
+    for (const auto& val : value.value) {
+      ASSERT_FALSE(val->is_valid);
+    }
+  }
+
+  void AssertModeIsNull(const std::string& json) {
+    auto array = ArrayFromJSON(type_singleton(), json);
+    AssertModeIsNull(array);
+  }
+
+  std::shared_ptr<DataType> type_singleton() { return Traits::type_singleton(); }
+};
+
+template <typename ArrowType>
+class TestIntegerModeKernel : public TestPrimitiveModeKernel<ArrowType> {};
+
+template <typename ArrowType>
+class TestFloatingModeKernel : public TestPrimitiveModeKernel<ArrowType> {};
+
+class TestBooleanModeKernel : public TestPrimitiveModeKernel<BooleanType> {};
+
+TEST_F(TestBooleanModeKernel, Basics) {
+  this->AssertModeIs("[false, false]", false, 2);
+  this->AssertModeIs("[false, false, true, true, true]", true, 3);
+  this->AssertModeIs("[true, false, false, true, true]", true, 3);
+  this->AssertModeIs("[false, false, true, true, true, false]", false, 3);
+
+  this->AssertModeIs("[true, null, false, false, null, true, null, null, true]", true, 3);
+  this->AssertModeIsNull("[null, null, null]");

Review comment:
       Also test an empty array?

##########
File path: cpp/src/arrow/compute/kernels/aggregate_basic_internal.h
##########
@@ -44,6 +48,9 @@ void AddMeanAvx2AggKernels(ScalarAggregateFunction* func);
 void AddSumAvx512AggKernels(ScalarAggregateFunction* func);
 void AddMeanAvx512AggKernels(ScalarAggregateFunction* func);
 
+// Aggregate kernels in separated source files

Review comment:
       I don't understand what this comment means?

##########
File path: cpp/src/arrow/compute/kernels/aggregate_mode.cc
##########
@@ -0,0 +1,182 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <cmath>
+#include <unordered_map>
+
+#include "arrow/compute/kernels/aggregate_basic_internal.h"
+
+namespace arrow {
+namespace compute {
+namespace aggregate {
+
+namespace {
+
+template <typename ArrowType>
+struct ModeState {
+  using ThisType = ModeState<ArrowType>;
+  using T = typename ArrowType::c_type;
+
+  void MergeFrom(const ThisType& state) {
+    for (const auto& value_count : state.value_counts) {
+      auto value = value_count.first;
+      auto count = value_count.second;
+      this->value_counts[value] += count;
+    }
+    this->has_values |= state.has_values;
+  }
+
+  template <typename ArrowType_ = ArrowType>
+  enable_if_t<!is_floating_type<ArrowType_>::value> MergeOne(T value) {
+    ++this->value_counts[value];
+  }
+
+  template <typename ArrowType_ = ArrowType>
+  enable_if_t<is_floating_type<ArrowType_>::value> MergeOne(T value) {
+    if (!std::isnan(value)) {

Review comment:
       +1

##########
File path: cpp/src/arrow/compute/kernels/aggregate_mode.cc
##########
@@ -0,0 +1,182 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <cmath>
+#include <unordered_map>

Review comment:
       For the record, we have our own hash table in `arrow/util/hashing.h`. It's definitely not as user-friendly, though...

##########
File path: cpp/src/arrow/compute/kernels/aggregate_mode.cc
##########
@@ -0,0 +1,182 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <cmath>
+#include <unordered_map>
+
+#include "arrow/compute/kernels/aggregate_basic_internal.h"
+
+namespace arrow {
+namespace compute {
+namespace aggregate {
+
+namespace {
+
+template <typename ArrowType>
+struct ModeState {
+  using ThisType = ModeState<ArrowType>;
+  using T = typename ArrowType::c_type;
+
+  void MergeFrom(const ThisType& state) {
+    for (const auto& value_count : state.value_counts) {
+      auto value = value_count.first;
+      auto count = value_count.second;
+      this->value_counts[value] += count;
+    }
+    this->has_values |= state.has_values;
+  }
+
+  template <typename ArrowType_ = ArrowType>
+  enable_if_t<!is_floating_type<ArrowType_>::value> MergeOne(T value) {
+    ++this->value_counts[value];
+  }
+
+  template <typename ArrowType_ = ArrowType>
+  enable_if_t<is_floating_type<ArrowType_>::value> MergeOne(T value) {
+    if (!std::isnan(value)) {
+      ++this->value_counts[value];
+    }
+  }
+
+  std::pair<T, int64_t> Finalize() {
+    T mode = std::numeric_limits<T>::min();
+    int64_t count = 0;
+
+    for (const auto& value_count : this->value_counts) {
+      auto this_value = value_count.first;
+      auto this_count = value_count.second;
+      if (this_count > count || (this_count == count && this_value < mode)) {
+        count = this_count;
+        mode = this_value;
+      }
+    }
+    return std::make_pair(mode, count);
+  }
+
+  bool has_values = false;
+  std::unordered_map<T, int64_t> value_counts{};
+};
+
+template <typename ArrowType>
+struct ModeImpl : public ScalarAggregator {
+  using ThisType = ModeImpl<ArrowType>;
+  using ArrayType = typename TypeTraits<ArrowType>::ArrayType;
+
+  explicit ModeImpl(const std::shared_ptr<DataType>& out_type) : out_type(out_type) {}
+
+  void Consume(KernelContext*, const ExecBatch& batch) override {
+    ModeState<ArrowType> local_state;
+    ArrayType arr(batch[0].array());
+
+    local_state.has_values = (arr.length() - arr.null_count()) > 0;
+    if (arr.null_count() > 0) {
+      BitmapReader reader(arr.null_bitmap_data(), arr.offset(), arr.length());
+      for (int64_t i = 0; i < arr.length(); i++) {
+        if (reader.IsSet()) {
+          local_state.MergeOne(arr.Value(i));
+        }
+        reader.Next();
+      }
+    } else {
+      for (int64_t i = 0; i < arr.length(); i++) {
+        local_state.MergeOne(arr.Value(i));
+      }
+    }
+    this->state = std::move(local_state);
+  }
+
+  void MergeFrom(KernelContext*, const KernelState& src) override {
+    const auto& other = checked_cast<const ThisType&>(src);
+    this->state.MergeFrom(other.state);
+  }
+
+  void Finalize(KernelContext*, Datum* out) override {
+    using ModeType = typename TypeTraits<ArrowType>::ScalarType;
+    using CountType = typename TypeTraits<Int64Type>::ScalarType;
+
+    std::vector<std::shared_ptr<Scalar>> values;
+    if (!this->state.has_values) {

Review comment:
       Can this simply be written `if (state.value_counts.empty())`?

##########
File path: cpp/src/arrow/compute/kernels/aggregate_test.cc
##########
@@ -594,5 +594,92 @@ TYPED_TEST(TestFloatingMinMaxKernel, DefaultOptions) {
   AssertDatumsEqual(explicit_defaults, no_options_provided);
 }
 
+///

Review comment:
       Nit: "///" (triple-slash comments) should be reserved for Doxygen docstrings.

##########
File path: cpp/src/arrow/compute/kernels/aggregate_mode.cc
##########
@@ -0,0 +1,182 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <cmath>
+#include <unordered_map>
+
+#include "arrow/compute/kernels/aggregate_basic_internal.h"
+
+namespace arrow {
+namespace compute {
+namespace aggregate {
+
+namespace {
+
+template <typename ArrowType>
+struct ModeState {
+  using ThisType = ModeState<ArrowType>;
+  using T = typename ArrowType::c_type;
+
+  void MergeFrom(const ThisType& state) {
+    for (const auto& value_count : state.value_counts) {
+      auto value = value_count.first;
+      auto count = value_count.second;
+      this->value_counts[value] += count;
+    }
+    this->has_values |= state.has_values;
+  }
+
+  template <typename ArrowType_ = ArrowType>
+  enable_if_t<!is_floating_type<ArrowType_>::value> MergeOne(T value) {
+    ++this->value_counts[value];
+  }
+
+  template <typename ArrowType_ = ArrowType>
+  enable_if_t<is_floating_type<ArrowType_>::value> MergeOne(T value) {
+    if (!std::isnan(value)) {
+      ++this->value_counts[value];
+    }
+  }
+
+  std::pair<T, int64_t> Finalize() {
+    T mode = std::numeric_limits<T>::min();
+    int64_t count = 0;
+
+    for (const auto& value_count : this->value_counts) {
+      auto this_value = value_count.first;
+      auto this_count = value_count.second;
+      if (this_count > count || (this_count == count && this_value < mode)) {
+        count = this_count;
+        mode = this_value;
+      }
+    }
+    return std::make_pair(mode, count);
+  }
+
+  bool has_values = false;
+  std::unordered_map<T, int64_t> value_counts{};
+};
+
+template <typename ArrowType>
+struct ModeImpl : public ScalarAggregator {
+  using ThisType = ModeImpl<ArrowType>;
+  using ArrayType = typename TypeTraits<ArrowType>::ArrayType;
+
+  explicit ModeImpl(const std::shared_ptr<DataType>& out_type) : out_type(out_type) {}
+
+  void Consume(KernelContext*, const ExecBatch& batch) override {
+    ModeState<ArrowType> local_state;
+    ArrayType arr(batch[0].array());
+
+    local_state.has_values = (arr.length() - arr.null_count()) > 0;

Review comment:
       Note that at the end, `state.has_values` should simply be `!state.value_counts.empty()`.

##########
File path: cpp/src/arrow/compute/kernels/aggregate_test.cc
##########
@@ -594,5 +594,92 @@ TYPED_TEST(TestFloatingMinMaxKernel, DefaultOptions) {
   AssertDatumsEqual(explicit_defaults, no_options_provided);
 }
 
+///
+/// Mode
+///
+
+template <typename ArrowType>
+class TestPrimitiveModeKernel : public ::testing::Test {
+  using Traits = TypeTraits<ArrowType>;
+  using c_type = typename ArrowType::c_type;
+  using ModeType = typename Traits::ScalarType;
+  using CountType = typename TypeTraits<Int64Type>::ScalarType;
+
+ public:
+  void AssertModeIs(const Datum& array, c_type expected_mode, int64_t expected_count) {
+    ASSERT_OK_AND_ASSIGN(Datum out, Mode(array));
+    const StructScalar& value = out.scalar_as<StructScalar>();
+
+    const auto& out_mode = checked_cast<const ModeType&>(*value.value[0]);
+    ASSERT_EQ(expected_mode, out_mode.value);
+
+    const auto& out_count = checked_cast<const CountType&>(*value.value[1]);
+    ASSERT_EQ(expected_count, out_count.value);
+  }
+
+  void AssertModeIs(const std::string& json, c_type expected_mode,
+                    int64_t expected_count) {
+    auto array = ArrayFromJSON(type_singleton(), json);
+    AssertModeIs(array, expected_mode, expected_count);
+  }
+
+  void AssertModeIsNull(const Datum& array) {
+    ASSERT_OK_AND_ASSIGN(Datum out, Mode(array));
+    const StructScalar& value = out.scalar_as<StructScalar>();
+
+    for (const auto& val : value.value) {
+      ASSERT_FALSE(val->is_valid);
+    }
+  }
+
+  void AssertModeIsNull(const std::string& json) {
+    auto array = ArrayFromJSON(type_singleton(), json);
+    AssertModeIsNull(array);
+  }
+
+  std::shared_ptr<DataType> type_singleton() { return Traits::type_singleton(); }
+};
+
+template <typename ArrowType>
+class TestIntegerModeKernel : public TestPrimitiveModeKernel<ArrowType> {};
+
+template <typename ArrowType>
+class TestFloatingModeKernel : public TestPrimitiveModeKernel<ArrowType> {};
+
+class TestBooleanModeKernel : public TestPrimitiveModeKernel<BooleanType> {};
+
+TEST_F(TestBooleanModeKernel, Basics) {
+  this->AssertModeIs("[false, false]", false, 2);
+  this->AssertModeIs("[false, false, true, true, true]", true, 3);
+  this->AssertModeIs("[true, false, false, true, true]", true, 3);
+  this->AssertModeIs("[false, false, true, true, true, false]", false, 3);
+
+  this->AssertModeIs("[true, null, false, false, null, true, null, null, true]", true, 3);
+  this->AssertModeIsNull("[null, null, null]");
+}
+
+TYPED_TEST_SUITE(TestIntegerModeKernel, IntegralArrowTypes);
+TYPED_TEST(TestIntegerModeKernel, Basics) {
+  this->AssertModeIs("[5, 1, 1, 5, 5]", 5, 3);
+  this->AssertModeIs("[5, 1, 1, 5, 5, 1]", 1, 3);
+  this->AssertModeIs("[127, 0, 127, 127, 0, 1, 0, 127]", 127, 4);
+
+  this->AssertModeIs("[null, null, 2, null, 1]", 1, 1);
+  this->AssertModeIsNull("[null, null, null]");
+  this->AssertModeIsNull("[]");
+}
+
+TYPED_TEST_SUITE(TestFloatingModeKernel, RealArrowTypes);
+TYPED_TEST(TestFloatingModeKernel, Floats) {
+  this->AssertModeIs("[5, 1, 1, 5, 5]", 5, 3);
+  this->AssertModeIs("[5, 1, 1, 5, 5, 1]", 1, 3);
+  this->AssertModeIs("[Inf, 100, Inf, 100, Inf]", INFINITY, 3);
+  this->AssertModeIs("[Inf, -Inf, Inf, -Inf]", -INFINITY, 2);
+
+  this->AssertModeIs("[null, null, 2, null, 1]", 1, 1);
+  this->AssertModeIs("[NaN, NaN, 1]", 1, 1);
+  this->AssertModeIsNull("[null, null, null]");

Review comment:
       And also test an empty array?

##########
File path: cpp/src/arrow/compute/api_aggregate.cc
##########
@@ -41,5 +41,9 @@ Result<Datum> MinMax(const Datum& value, const MinMaxOptions& options, ExecConte
   return CallFunction("min_max", {value}, &options, ctx);
 }
 
+Result<Datum> Mode(const Datum& value, ExecContext* ctx) {
+  return CallFunction("mode", {value}, ctx);

Review comment:
       We have a comprehensive doc for compute functions here:
   https://arrow.apache.org/docs/cpp/compute.html
   
   You should add your mode function to `docs/source/cpp/compute.rst`.

##########
File path: cpp/src/arrow/compute/kernels/aggregate_mode.cc
##########
@@ -0,0 +1,182 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <cmath>
+#include <unordered_map>
+
+#include "arrow/compute/kernels/aggregate_basic_internal.h"
+
+namespace arrow {
+namespace compute {
+namespace aggregate {
+
+namespace {
+
+template <typename ArrowType>
+struct ModeState {
+  using ThisType = ModeState<ArrowType>;
+  using T = typename ArrowType::c_type;
+
+  void MergeFrom(const ThisType& state) {
+    for (const auto& value_count : state.value_counts) {
+      auto value = value_count.first;
+      auto count = value_count.second;
+      this->value_counts[value] += count;
+    }
+    this->has_values |= state.has_values;
+  }
+
+  template <typename ArrowType_ = ArrowType>
+  enable_if_t<!is_floating_type<ArrowType_>::value> MergeOne(T value) {
+    ++this->value_counts[value];
+  }
+
+  template <typename ArrowType_ = ArrowType>
+  enable_if_t<is_floating_type<ArrowType_>::value> MergeOne(T value) {
+    if (!std::isnan(value)) {
+      ++this->value_counts[value];
+    }
+  }
+
+  std::pair<T, int64_t> Finalize() {
+    T mode = std::numeric_limits<T>::min();
+    int64_t count = 0;
+
+    for (const auto& value_count : this->value_counts) {
+      auto this_value = value_count.first;
+      auto this_count = value_count.second;
+      if (this_count > count || (this_count == count && this_value < mode)) {
+        count = this_count;
+        mode = this_value;
+      }
+    }
+    return std::make_pair(mode, count);
+  }
+
+  bool has_values = false;
+  std::unordered_map<T, int64_t> value_counts{};
+};
+
+template <typename ArrowType>
+struct ModeImpl : public ScalarAggregator {
+  using ThisType = ModeImpl<ArrowType>;
+  using ArrayType = typename TypeTraits<ArrowType>::ArrayType;
+
+  explicit ModeImpl(const std::shared_ptr<DataType>& out_type) : out_type(out_type) {}
+
+  void Consume(KernelContext*, const ExecBatch& batch) override {
+    ModeState<ArrowType> local_state;
+    ArrayType arr(batch[0].array());
+
+    local_state.has_values = (arr.length() - arr.null_count()) > 0;

Review comment:
       This is not correct if you have a floating point array full of NaNs.

##########
File path: cpp/src/arrow/compute/kernels/aggregate_test.cc
##########
@@ -594,5 +594,92 @@ TYPED_TEST(TestFloatingMinMaxKernel, DefaultOptions) {
   AssertDatumsEqual(explicit_defaults, no_options_provided);
 }
 
+///
+/// Mode
+///
+
+template <typename ArrowType>
+class TestPrimitiveModeKernel : public ::testing::Test {
+  using Traits = TypeTraits<ArrowType>;
+  using c_type = typename ArrowType::c_type;
+  using ModeType = typename Traits::ScalarType;
+  using CountType = typename TypeTraits<Int64Type>::ScalarType;
+
+ public:
+  void AssertModeIs(const Datum& array, c_type expected_mode, int64_t expected_count) {
+    ASSERT_OK_AND_ASSIGN(Datum out, Mode(array));
+    const StructScalar& value = out.scalar_as<StructScalar>();
+
+    const auto& out_mode = checked_cast<const ModeType&>(*value.value[0]);
+    ASSERT_EQ(expected_mode, out_mode.value);
+
+    const auto& out_count = checked_cast<const CountType&>(*value.value[1]);
+    ASSERT_EQ(expected_count, out_count.value);
+  }
+
+  void AssertModeIs(const std::string& json, c_type expected_mode,
+                    int64_t expected_count) {
+    auto array = ArrayFromJSON(type_singleton(), json);
+    AssertModeIs(array, expected_mode, expected_count);
+  }
+
+  void AssertModeIsNull(const Datum& array) {
+    ASSERT_OK_AND_ASSIGN(Datum out, Mode(array));
+    const StructScalar& value = out.scalar_as<StructScalar>();
+
+    for (const auto& val : value.value) {
+      ASSERT_FALSE(val->is_valid);
+    }
+  }
+
+  void AssertModeIsNull(const std::string& json) {
+    auto array = ArrayFromJSON(type_singleton(), json);
+    AssertModeIsNull(array);
+  }
+
+  std::shared_ptr<DataType> type_singleton() { return Traits::type_singleton(); }
+};
+
+template <typename ArrowType>
+class TestIntegerModeKernel : public TestPrimitiveModeKernel<ArrowType> {};
+
+template <typename ArrowType>
+class TestFloatingModeKernel : public TestPrimitiveModeKernel<ArrowType> {};
+
+class TestBooleanModeKernel : public TestPrimitiveModeKernel<BooleanType> {};
+
+TEST_F(TestBooleanModeKernel, Basics) {
+  this->AssertModeIs("[false, false]", false, 2);
+  this->AssertModeIs("[false, false, true, true, true]", true, 3);
+  this->AssertModeIs("[true, false, false, true, true]", true, 3);
+  this->AssertModeIs("[false, false, true, true, true, false]", false, 3);
+
+  this->AssertModeIs("[true, null, false, false, null, true, null, null, true]", true, 3);
+  this->AssertModeIsNull("[null, null, null]");
+}
+
+TYPED_TEST_SUITE(TestIntegerModeKernel, IntegralArrowTypes);
+TYPED_TEST(TestIntegerModeKernel, Basics) {
+  this->AssertModeIs("[5, 1, 1, 5, 5]", 5, 3);
+  this->AssertModeIs("[5, 1, 1, 5, 5, 1]", 1, 3);
+  this->AssertModeIs("[127, 0, 127, 127, 0, 1, 0, 127]", 127, 4);
+
+  this->AssertModeIs("[null, null, 2, null, 1]", 1, 1);
+  this->AssertModeIsNull("[null, null, null]");
+  this->AssertModeIsNull("[]");
+}
+
+TYPED_TEST_SUITE(TestFloatingModeKernel, RealArrowTypes);
+TYPED_TEST(TestFloatingModeKernel, Floats) {
+  this->AssertModeIs("[5, 1, 1, 5, 5]", 5, 3);
+  this->AssertModeIs("[5, 1, 1, 5, 5, 1]", 1, 3);
+  this->AssertModeIs("[Inf, 100, Inf, 100, Inf]", INFINITY, 3);
+  this->AssertModeIs("[Inf, -Inf, Inf, -Inf]", -INFINITY, 2);
+
+  this->AssertModeIs("[null, null, 2, null, 1]", 1, 1);
+  this->AssertModeIs("[NaN, NaN, 1]", 1, 1);
+  this->AssertModeIsNull("[null, null, null]");

Review comment:
       Can you also add a test for e.g. `[NaN, NaN, null]`?




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

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