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 2021/12/07 12:53:31 UTC

[GitHub] [arrow] bkmgit opened a new pull request #11882: ARROW-9843: [C++] Between Kernel

bkmgit opened a new pull request #11882:
URL: https://github.com/apache/arrow/pull/11882


   See if can get a clean history


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

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

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



[GitHub] [arrow] bkmgit commented on a change in pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
bkmgit commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r767672873



##########
File path: cpp/src/arrow/util/bit_block_counter.h
##########
@@ -586,5 +961,144 @@ static void VisitTwoBitBlocksVoid(const std::shared_ptr<Buffer>& left_bitmap_buf
   }
 }
 
+template <typename VisitNotNull, typename VisitNull>
+static void VisitThreeBitBlocksVoid(
+    const std::shared_ptr<Buffer>& left_bitmap_buf, int64_t left_offset,
+    const std::shared_ptr<Buffer>& mid_bitmap_buf, int64_t mid_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) && (mid_bitmap_buf == NULLPTR)) ||
+      ((mid_bitmap_buf == NULLPTR) && (right_bitmap_buf == NULLPTR)) ||
+      ((left_bitmap_buf == NULLPTR) && (right_bitmap_buf == NULLPTR))) {
+    // At most one bitmap is present
+    if ((left_bitmap_buf == NULLPTR) && (mid_bitmap_buf == NULLPTR)) {
+      return VisitBitBlocksVoid(right_bitmap_buf, right_offset, length,
+                                std::forward<VisitNotNull>(visit_not_null),
+                                std::forward<VisitNull>(visit_null));
+    } else if ((mid_bitmap_buf == NULLPTR) && (right_bitmap_buf == NULLPTR)) {
+      return VisitBitBlocksVoid(left_bitmap_buf, left_offset, length,
+                                std::forward<VisitNotNull>(visit_not_null),
+                                std::forward<VisitNull>(visit_null));
+    } else {
+      return VisitBitBlocksVoid(mid_bitmap_buf, mid_offset, length,
+                                std::forward<VisitNotNull>(visit_not_null),
+                                std::forward<VisitNull>(visit_null));
+    }
+  }
+  // Two bitmaps are present
+  if (left_bitmap_buf == NULLPTR) {
+    const uint8_t* mid_bitmap = mid_bitmap_buf->data();
+    const uint8_t* right_bitmap = right_bitmap_buf->data();
+    BinaryBitBlockCounter bit_counter(mid_bitmap, mid_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 (bit_util::GetBit(mid_bitmap, mid_offset + position) &&
+              bit_util::GetBit(right_bitmap, right_offset + position)) {
+            visit_not_null(position);
+          } else {
+            visit_null();
+          }
+        }
+      }
+    }
+  } else if (mid_bitmap_buf == NULLPTR) {
+    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 (bit_util::GetBit(left_bitmap, left_offset + position) &&
+              bit_util::GetBit(right_bitmap, right_offset + position)) {
+            visit_not_null(position);
+          } else {
+            visit_null();
+          }
+        }
+      }
+    }
+  } else {
+    const uint8_t* left_bitmap = left_bitmap_buf->data();

Review comment:
       Thanks. Better to test each of the cases separately.




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

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

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



[GitHub] [arrow] bkmgit commented on a change in pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
bkmgit commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r768863041



##########
File path: cpp/src/arrow/compute/kernels/scalar_compare_test.cc
##########
@@ -1850,5 +1851,154 @@ TEST(TestMaxElementWiseMinElementWise, CommonTemporal) {
               ResultWith(ScalarFromJSON(date64(), "86400000")));
 }
 
+template <typename ArrowType>
+static void ValidateBetween(const Datum& val, const Datum& lhs, const Datum& rhs,
+                            const Datum& expected) {
+  ASSERT_OK_AND_ASSIGN(Datum result, Between(val, lhs, rhs));
+  AssertArraysEqual(*expected.make_array(), *result.make_array(),
+                    /*verbose=*/true);
+}
+
+template <typename ArrowType>
+static void ValidateBetween(const char* value_str, const Datum& lhs, const Datum& rhs,
+                            const char* expected_str) {
+  auto value = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), value_str);
+  auto expected = ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), expected_str);
+  ValidateBetween<ArrowType>(value, lhs, rhs, expected);
+}
+
+template <>
+void ValidateBetween<StringType>(const char* value_str, const Datum& lhs,
+                                 const Datum& rhs, const char* expected_str) {
+  auto value = ArrayFromJSON(utf8(), value_str);
+  auto expected = ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), expected_str);
+  ValidateBetween<StringType>(value, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+class TestNumericBetweenKernel : public ::testing::Test {};
+
+TYPED_TEST_SUITE(TestNumericBetweenKernel, NumericArrowTypes);
+TYPED_TEST(TestNumericBetweenKernel, SimpleBetweenArrayScalarScalar) {
+  using ScalarType = typename TypeTraits<TypeParam>::ScalarType;
+  using CType = typename TypeTraits<TypeParam>::CType;
+
+  Datum zero(std::make_shared<ScalarType>(CType(0)));
+  Datum four(std::make_shared<ScalarType>(CType(4)));
+  ValidateBetween<TypeParam>("[]", zero, four, "[]");
+  ValidateBetween<TypeParam>("[null]", zero, four, "[null]");
+  ValidateBetween<TypeParam>("[0,0,1,1,2,2]", zero, four, "[0,0,1,1,1,1]");
+  ValidateBetween<TypeParam>("[0,1,2,3,4,5]", zero, four, "[0,1,1,1,0,0]");
+  ValidateBetween<TypeParam>("[5,4,3,2,1,0]", zero, four, "[0,0,1,1,1,0]");
+  ValidateBetween<TypeParam>("[null,0,1,1]", zero, four, "[null,0,1,1]");
+}
+
+TEST(TestSimpleBetweenKernel, SimpleStringTest) {
+  using ScalarType = typename TypeTraits<StringType>::ScalarType;
+  auto l = Datum(std::make_shared<ScalarType>("abc"));
+  auto r = Datum(std::make_shared<ScalarType>("zzz"));
+  ValidateBetween<StringType>("[]", l, r, "[]");
+  ValidateBetween<StringType>("[null]", l, r, "[null]");
+  ValidateBetween<StringType>(R"(["aaa", "aaaa", "ccc", "z"])", l, r,
+                              R"([false, false, true, true])");
+  ValidateBetween<StringType>(R"(["a", "aaaa", "c", "z"])", l, r,
+                              R"([false, false, true, true])");
+  ValidateBetween<StringType>(R"(["a", "aaaa", "fff", "zzzz"])", l, r,
+                              R"([false, false, true, false])");
+  ValidateBetween<StringType>(R"(["abd", null, null, "zzx"])", l, r,
+                              R"([true, null, null, true])");
+}
+
+TEST(TestSimpleBetweenKernel, SimpleTimestampTest) {
+  using ScalarType = typename TypeTraits<TimestampType>::ScalarType;
+  auto checkTimestampArray = [](std::shared_ptr<DataType> type, const char* input_str,
+                                const Datum& lhs, const Datum& rhs,
+                                const char* expected_str) {
+    auto value = ArrayFromJSON(type, input_str);
+    auto expected = ArrayFromJSON(boolean(), expected_str);
+    ValidateBetween<TimestampType>(value, lhs, rhs, expected);
+  };
+  auto unit = TimeUnit::SECOND;
+  auto l = Datum(std::make_shared<ScalarType>(923184000, timestamp(unit)));
+  auto r = Datum(std::make_shared<ScalarType>(1602032602, timestamp(unit)));
+  checkTimestampArray(timestamp(unit), "[]", l, r, "[]");
+  checkTimestampArray(timestamp(unit), "[null]", l, r, "[null]");
+  checkTimestampArray(timestamp(unit), R"(["1970-01-01","2000-02-29","1900-02-28"])", l,
+                      r, "[false,true,false]");
+  checkTimestampArray(timestamp(unit), R"(["1970-01-01","2000-02-29","2004-02-28"])", l,
+                      r, "[false,true,true]");
+  checkTimestampArray(timestamp(unit), R"(["2018-01-01","1999-04-04","1900-02-28"])", l,
+                      r, "[true,false,false]");
+}
+
+TYPED_TEST(TestNumericBetweenKernel, SimpleBetweenArrayArrayArray) {
+  ValidateBetween<TypeParam>(
+      "[]", ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"), "[]");
+  ValidateBetween<TypeParam>(
+      "[null]", ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"), "[null]");
+  ValidateBetween<TypeParam>(
+      "[1,1,2,2,2]",
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      "[true,true,false,false,false]");
+  ValidateBetween<TypeParam>(
+      "[1,1,2,2,2,2]",
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,2,5,5]"),
+      "[true,true,false,null,false,false]");
+}
+
+TEST(TestSimpleBetweenKernel, StringArrayArrayArrayTest) {
+  ValidateBetween<StringType>(
+      R"(["david","hello","world"])",
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["adam","hi","whirl"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                    R"(["robert","goeiemoreen","whirlwind"])"),
+      "[true, false, false]");
+  ValidateBetween<StringType>(
+      R"(["x","a","f"])",
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["w","a","e"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["z","a","g"])"),
+      "[true, false, true]");
+  ValidateBetween<StringType>(
+      R"(["block","bit","binary"])",
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                    R"(["bit","nibble","ternary"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["word","d","xyz"])"),
+      "[true, false, false]");
+  ValidateBetween<StringType>(R"(["Ayumi","アユミ","王梦莹"])",
+                              ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                            R"(["たなか","あゆみ","歩美"])"),
+                              ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                            R"(["李平之","田中","たなか"])"),
+                              "[false, true, false]");
+}

Review comment:
       Done.




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

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

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



[GitHub] [arrow] bkmgit commented on a change in pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
bkmgit commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r767629466



##########
File path: cpp/src/arrow/compute/kernels/codegen_internal.h
##########
@@ -583,7 +606,23 @@ struct OutputAdapter<Type, enable_if_base_binary<Type>> {
     return Status::NotImplemented("NYI");
   }
 };
+/*
+template <typename Type>
+struct OutputAdapter<Type, enable_if_base_binary<Type>> {
+  using T = typename TypeTraits<Type>::ScalarType::ValueType;
 
+  template <typename Generator>
+  static Status Write(KernelContext*, Datum* out, Generator&& generator) {
+    ArrayData* out_arr = out->mutable_array();
+    auto out_data = out_arr->GetMutableValues<T>(1);
+    // TODO: Is this as fast as a more explicitly inlined function?
+    for (int64_t i = 0; i < out_arr->length; ++i) {
+      *out_data++ = generator();
+    }
+    return Status::OK();
+  }
+};
+*/

Review comment:
       This can be removed.




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

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

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



[GitHub] [arrow] bkmgit commented on a change in pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
bkmgit commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r771991046



##########
File path: cpp/src/arrow/compute/kernels/codegen_internal.h
##########
@@ -982,6 +1021,407 @@ template <typename OutType, typename ArgType, typename Op>
 using ScalarBinaryNotNullStatefulEqualTypes =
     ScalarBinaryNotNullStateful<OutType, ArgType, ArgType, Op>;
 
+// A kernel exec generator for ternary functions that addresses both array and
+// scalar inputs and dispatches input iteration and output writing to other
+// templates
+//
+// This template executes the operator even on the data behind null values,
+// therefore it is generally only suitable for operators that are safe to apply
+// even on the null slot values.
+//
+// The "Op" functor should have the form
+//
+// struct Op {
+//   template <typename OutValue, typename Arg0Value, typename Arg1Value, typename
+//   Arg2Value> static OutValue Call(KernelContext* ctx, Arg0Value arg0, Arg1Value arg1,
+//   Arg2Value arg2, Status *st) {
+//     // implementation
+//     // NOTE: "status" should only be populated with errors,
+//     //       leave it unmodified to indicate Status::OK()
+//   }
+// };
+template <typename OutType, typename Arg0Type, typename Arg1Type, typename Arg2Type,
+          typename Op>
+struct ScalarTernary {
+  using OutValue = typename GetOutputType<OutType>::T;
+  using Arg0Value = typename GetViewType<Arg0Type>::T;
+  using Arg1Value = typename GetViewType<Arg1Type>::T;
+  using Arg2Value = typename GetViewType<Arg2Type>::T;
+
+  static Status ArrayArrayArray(KernelContext* ctx, const ArrayData& arg0,
+                                const ArrayData& arg1, const ArrayData& arg2,
+                                Datum* out) {
+    Status st = Status::OK();
+    ArrayIterator<Arg0Type> arg0_it(arg0);
+    ArrayIterator<Arg1Type> arg1_it(arg1);
+    ArrayIterator<Arg2Type> arg2_it(arg2);
+    RETURN_NOT_OK(OutputAdapter<OutType>::Write(ctx, out, [&]() -> OutValue {
+      return Op::template Call<OutValue, Arg0Value, Arg1Value, Arg2Value>(
+          ctx, arg0_it(), arg1_it(), arg2_it(), &st);
+    }));
+    return st;
+  }
+
+  static Status ArrayArrayScalar(KernelContext* ctx, const ArrayData& arg0,
+                                 const ArrayData& arg1, const Scalar& arg2, Datum* out) {
+    Status st = Status::OK();
+    ArrayIterator<Arg0Type> arg0_it(arg0);
+    ArrayIterator<Arg1Type> arg1_it(arg1);
+    auto arg2_val = UnboxScalar<Arg2Type>::Unbox(arg2);
+    RETURN_NOT_OK(OutputAdapter<OutType>::Write(ctx, out, [&]() -> OutValue {
+      return Op::template Call<OutValue, Arg0Value, Arg1Value, Arg2Value>(
+          ctx, arg0_it(), arg1_it(), arg2_val, &st);
+    }));
+    return st;
+  }
+
+  static Status ArrayScalarArray(KernelContext* ctx, const ArrayData& arg0,
+                                 const Scalar& arg1, const ArrayData& arg2, Datum* out) {
+    Status st = Status::OK();
+    ArrayIterator<Arg0Type> arg0_it(arg0);
+    auto arg1_val = UnboxScalar<Arg1Type>::Unbox(arg1);
+    ArrayIterator<Arg2Type> arg2_it(arg2);
+    RETURN_NOT_OK(OutputAdapter<OutType>::Write(ctx, out, [&]() -> OutValue {
+      return Op::template Call<OutValue, Arg0Value, Arg1Value, Arg2Value>(
+          ctx, arg0_it(), arg1_val, arg2_it(), &st);
+    }));
+    return st;
+  }
+
+  static Status ScalarArrayArray(KernelContext* ctx, const Scalar& arg0,
+                                 const ArrayData& arg1, const ArrayData& arg2,
+                                 Datum* out) {
+    Status st = Status::OK();
+    auto arg0_val = UnboxScalar<Arg0Type>::Unbox(arg0);
+    ArrayIterator<Arg1Type> arg1_it(arg1);
+    ArrayIterator<Arg2Type> arg2_it(arg2);
+    RETURN_NOT_OK(OutputAdapter<OutType>::Write(ctx, out, [&]() -> OutValue {
+      return Op::template Call<OutValue, Arg0Value, Arg1Value, Arg2Value>(
+          ctx, arg0_val, arg1_it(), arg2_it(), &st);
+    }));
+    return st;
+  }
+
+  static Status ArrayScalarScalar(KernelContext* ctx, const ArrayData& arg0,
+                                  const Scalar& arg1, const Scalar& arg2, Datum* out) {
+    Status st = Status::OK();
+    ArrayIterator<Arg0Type> arg0_it(arg0);
+    auto arg1_val = UnboxScalar<Arg1Type>::Unbox(arg1);
+    auto arg2_val = UnboxScalar<Arg2Type>::Unbox(arg2);
+    RETURN_NOT_OK(OutputAdapter<OutType>::Write(ctx, out, [&]() -> OutValue {
+      return Op::template Call<OutValue, Arg0Value, Arg1Value, Arg2Value>(
+          ctx, arg0_it(), arg1_val, arg2_val, &st);
+    }));
+    return st;
+  }
+
+  static Status ScalarScalarArray(KernelContext* ctx, const Scalar& arg0,
+                                  const Scalar& arg1, const ArrayData& arg2, Datum* out) {
+    Status st = Status::OK();
+    auto arg0_val = UnboxScalar<Arg0Type>::Unbox(arg0);
+    auto arg1_val = UnboxScalar<Arg1Type>::Unbox(arg1);
+    ArrayIterator<Arg2Type> arg2_it(arg2);
+    RETURN_NOT_OK(OutputAdapter<OutType>::Write(ctx, out, [&]() -> OutValue {
+      return Op::template Call<OutValue, Arg0Value, Arg1Value, Arg2Value>(
+          ctx, arg0_val, arg1_val, arg2_it(), &st);
+    }));
+    return st;
+  }
+
+  static Status ScalarArrayScalar(KernelContext* ctx, const Scalar& arg0,
+                                  const ArrayData& arg1, const Scalar& arg2, Datum* out) {
+    Status st = Status::OK();
+    auto arg0_val = UnboxScalar<Arg0Type>::Unbox(arg0);
+    ArrayIterator<Arg1Type> arg1_it(arg1);
+    auto arg2_val = UnboxScalar<Arg2Type>::Unbox(arg2);
+    RETURN_NOT_OK(OutputAdapter<OutType>::Write(ctx, out, [&]() -> OutValue {
+      return Op::template Call<OutValue, Arg0Value, Arg1Value, Arg2Value>(
+          ctx, arg0_val, arg1_it(), arg2_val, &st);
+    }));
+    return st;
+  }
+
+  static Status ScalarScalarScalar(KernelContext* ctx, const Scalar& arg0,
+                                   const Scalar& arg1, const Scalar& arg2, Datum* out) {
+    Status st = Status::OK();
+    if (out->scalar()->is_valid) {
+      auto arg0_val = UnboxScalar<Arg0Type>::Unbox(arg0);
+      auto arg1_val = UnboxScalar<Arg1Type>::Unbox(arg1);
+      auto arg2_val = UnboxScalar<Arg2Type>::Unbox(arg2);
+      BoxScalar<OutType>::Box(
+          Op::template Call<OutValue, Arg0Value, Arg1Value, Arg2Value>(
+              ctx, arg0_val, arg1_val, arg2_val, &st),
+          out->scalar().get());
+    }
+    return st;
+  }
+
+  static Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    if (batch[0].kind() == Datum::ARRAY) {
+      if (batch[1].kind() == Datum::ARRAY) {
+        if (batch[2].kind() == Datum::ARRAY) {
+          return ArrayArrayArray(ctx, *batch[0].array(), *batch[1].array(),
+                                 *batch[2].array(), out);
+        } else {
+          return ArrayArrayScalar(ctx, *batch[0].array(), *batch[1].array(),
+                                  *batch[2].scalar(), out);
+        }
+      } else {
+        if (batch[2].kind() == Datum::ARRAY) {
+          return ArrayScalarArray(ctx, *batch[0].array(), *batch[1].scalar(),
+                                  *batch[2].array(), out);
+        } else {
+          return ArrayScalarScalar(ctx, *batch[0].array(), *batch[1].scalar(),
+                                   *batch[2].scalar(), out);
+        }
+      }
+    } else {
+      if (batch[1].kind() == Datum::ARRAY) {
+        if (batch[2].kind() == Datum::ARRAY) {
+          return ScalarArrayArray(ctx, *batch[0].scalar(), *batch[1].array(),
+                                  *batch[2].array(), out);
+        } else {
+          return ScalarArrayScalar(ctx, *batch[0].scalar(), *batch[1].array(),
+                                   *batch[2].scalar(), out);
+        }
+      } else {
+        if (batch[2].kind() == Datum::ARRAY) {
+          return ScalarScalarArray(ctx, *batch[0].scalar(), *batch[1].scalar(),
+                                   *batch[2].array(), out);
+        } else {
+          return ScalarScalarScalar(ctx, *batch[0].scalar(), *batch[1].scalar(),
+                                    *batch[2].scalar(), out);
+        }
+      }
+    }
+  }
+};
+
+// An alternative to ScalarTernary 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 Arg2Type,
+          typename Op>
+struct ScalarTernaryNotNullStateful {

Review comment:
       Decimal support added.




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

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

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



[GitHub] [arrow] lidavidm commented on a change in pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
lidavidm commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r772361652



##########
File path: cpp/src/arrow/compute/kernels/scalar_compare.cc
##########
@@ -68,6 +72,50 @@ struct GreaterEqual {
   }
 };
 
+struct BetweenLessEqualLessEqual {
+  template <typename T, typename Arg0, typename Arg1, typename Arg2>
+  static constexpr T Call(KernelContext*, const Arg0& middle, const Arg1& left,
+                          const Arg2& right, Status*) {
+    static_assert(std::is_same<T, bool>::value && std::is_same<Arg0, Arg1>::value &&
+                      std::is_same<Arg1, Arg2>::value,
+                  "");
+    return (left <= middle) && (middle <= right);
+  }
+};
+
+struct BetweenLessThanLessEqual {
+  template <typename T, typename Arg0, typename Arg1, typename Arg2>
+  static constexpr T Call(KernelContext*, const Arg0& middle, const Arg1& left,
+                          const Arg2& right, Status*) {
+    static_assert(std::is_same<T, bool>::value && std::is_same<Arg0, Arg1>::value &&
+                      std::is_same<Arg1, Arg2>::value,
+                  "");
+    return (left < middle) && (middle <= right);
+  }
+};
+
+struct BetweenLessEqualLessThan {
+  template <typename T, typename Arg0, typename Arg1, typename Arg2>
+  static constexpr T Call(KernelContext*, const Arg0& middle, const Arg1& left,
+                          const Arg2& right, Status*) {
+    static_assert(std::is_same<T, bool>::value && std::is_same<Arg0, Arg1>::value &&
+                      std::is_same<Arg1, Arg2>::value,
+                  "");
+    return (left <= middle) && (middle < right);
+  }
+};
+
+struct BetweenLessThanLessThan {
+  template <typename T, typename Arg0, typename Arg1, typename Arg2>
+  static constexpr T Call(KernelContext*, const Arg0& middle, const Arg1& left,
+                          const Arg2& right, Status*) {
+    static_assert(std::is_same<T, bool>::value && std::is_same<Arg0, Arg1>::value &&
+                      std::is_same<Arg1, Arg2>::value,
+                  "");
+    return (left < middle) && (middle < right);
+  }
+};

Review comment:
       Compare is awkward since there is no one compare function, instead each of `less`, `greater`, etc. is its own function. There is an "option" for it but it was semi-removed a while back. I think we can leave compare as-is here.




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

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

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



[GitHub] [arrow] edponce commented on a change in pull request #11882: ARROW-9843: [C++][Python] Implement Between ternary kernel and Python bindings

Posted by GitBox <gi...@apache.org>.
edponce commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r786359948



##########
File path: cpp/src/arrow/compute/kernels/scalar_compare.cc
##########
@@ -156,39 +212,52 @@ struct Maximum {
   }
 };
 
+// Check if timestamp timezones are comparable (either all are empty or none is).
+Status CheckCompareTimestamps(const ExecBatch& batch) {
+  if (batch.num_values() > 0) {
+    for (int i = 0; i < batch.num_values() - 1; ++i) {
+      const auto& tsi = checked_cast<const TimestampType&>(*batch[i].type());
+      for (int j = i + 1; j < batch.num_values(); ++j) {

Review comment:
       This check can be done with a linear search, comparing pairs as @pitrou pointed out. No need to do an all-to-all comparisons. I missed this when doing the change.

##########
File path: cpp/src/arrow/compute/kernels/codegen_internal.h
##########
@@ -1134,8 +1340,43 @@ ArrayKernelExec GeneratePhysicalInteger(detail::GetTypeId get_id) {
   }
 }
 
-template <template <typename... Args> class Generator, typename... Args>
+template <template <typename... Args> class Generator, typename Type0, typename... Args>
 ArrayKernelExec GeneratePhysicalNumeric(detail::GetTypeId get_id) {

Review comment:
       Maybe something like:
   ```c++
   // Generate a kernel given a templated functor for numeric types.
   // The functor is expected to have a constant output type specified via its first template parameter.
   GeneratePhysicalNumeric()
   
   // Generate a kernel given a templated functor for numeric types.
   // The functor is expected to have the same input and output numeric types.
   GeneratePhysicalNumericToNumeric()
   ```




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

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

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



[GitHub] [arrow] bkmgit commented on pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
bkmgit commented on pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#issuecomment-1013738181


   @lidavidm @edponce @kou Thanks for all your feedback. Any further suggestions?


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

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

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



[GitHub] [arrow] bkmgit commented on a change in pull request #11882: ARROW-9843: [C++][Python] Implement Between ternary kernel and Python bindings

Posted by GitBox <gi...@apache.org>.
bkmgit commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r787042444



##########
File path: cpp/src/arrow/compute/kernels/codegen_internal.h
##########
@@ -1134,8 +1340,43 @@ ArrayKernelExec GeneratePhysicalInteger(detail::GetTypeId get_id) {
   }
 }
 
-template <template <typename... Args> class Generator, typename... Args>
+template <template <typename... Args> class Generator, typename Type0, typename... Args>
 ArrayKernelExec GeneratePhysicalNumeric(detail::GetTypeId get_id) {

Review comment:
       Applied edponce suggestions.




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

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

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



[GitHub] [arrow] bkmgit commented on a change in pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
bkmgit commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r780786506



##########
File path: cpp/src/arrow/compute/kernels/scalar_compare.cc
##########
@@ -156,39 +210,50 @@ struct Maximum {
   }
 };
 
+// Check if timestamp timezones are comparable (either all are empty or none is).
+Status CheckCompareTimestamps(const ExecBatch& batch) {
+  if (batch.num_values() > 0) {
+    const auto& ts0 = checked_cast<const TimestampType&>(*batch[0].type());
+    bool invalid_state = ts0.timezone().empty();
+    for (int i = 1; i < batch.num_values(); ++i) {
+      const auto& ts = checked_cast<const TimestampType&>(*batch[i].type());
+      invalid_state ^= ts.timezone().empty();
+    }
+    if (invalid_state) {
+      return Status::Invalid(
+          "Cannot compare timestamp with timezone to timestamp without timezone");
+    }

Review comment:
       Fixed.




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

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

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



[GitHub] [arrow] jorisvandenbossche commented on a change in pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
jorisvandenbossche commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r782759913



##########
File path: python/pyarrow/tests/test_compute.py
##########
@@ -1262,6 +1263,300 @@ def test_filter_null_type():
     assert len(table.filter(mask).column(0)) == 5
 
 
+@pytest.mark.parametrize("ty", ["inclusive"])
+def test_between_array_array_array(ty):
+    BetweenOptions = partial(pc.BetweenOptions)
+
+    val = pa.array([1, 1, 4, 3, 2, 6])
+    arr1 = pa.array([1, 1, 3, 4, None, 5])
+    arr2 = pa.array([1, 2, 4, None, 4, 7])
+
+    inclusive_and_expected = {
+        "both": [True, True, True, None, None, True],
+        "left": [False, True, False, None, None, True],
+        "right": [False, False, True, None, None, True],
+        "neither": [False, False, False, None, None, True],
+    }
+
+    for inclusive, expected in inclusive_and_expected.items():
+        options = BetweenOptions(inclusive=inclusive)
+        result = pc.between(val, arr1, arr2, options=options)
+        np.testing.assert_array_equal(result, pa.array(expected))

Review comment:
       ```suggestion
           assert result.equals(pa.array(expected))
   ```
   
   Since you are comparing pyarrow arrays, this is better than coercing to numpy arrays with np.testing
   
   Same for the occurrences below




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

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

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



[GitHub] [arrow] jorisvandenbossche commented on a change in pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
jorisvandenbossche commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r782762409



##########
File path: python/pyarrow/tests/test_compute.py
##########
@@ -1262,6 +1263,300 @@ def test_filter_null_type():
     assert len(table.filter(mask).column(0)) == 5
 
 
+@pytest.mark.parametrize("ty", ["inclusive"])
+def test_between_array_array_array(ty):
+    BetweenOptions = partial(pc.BetweenOptions)
+
+    val = pa.array([1, 1, 4, 3, 2, 6])
+    arr1 = pa.array([1, 1, 3, 4, None, 5])
+    arr2 = pa.array([1, 2, 4, None, 4, 7])
+
+    inclusive_and_expected = {
+        "both": [True, True, True, None, None, True],
+        "left": [False, True, False, None, None, True],
+        "right": [False, False, True, None, None, True],
+        "neither": [False, False, False, None, None, True],
+    }
+
+    for inclusive, expected in inclusive_and_expected.items():
+        options = BetweenOptions(inclusive=inclusive)
+        result = pc.between(val, arr1, arr2, options=options)
+        np.testing.assert_array_equal(result, pa.array(expected))

Review comment:
       This is adding a lot of testing code, though (it's of course good to have thorough tests, but too much tests make it also harder to maintain the tests). 
   I was wondering, is the string version needed here in Python? (strings are already tested in C++ tests, and the integer tests in Python already ensure that arrays vs scalars and options are correctly passed). Leaving out those would already cut the test code here in half?




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

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

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



[GitHub] [arrow] bkmgit commented on a change in pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
bkmgit commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r782785820



##########
File path: python/pyarrow/tests/test_compute.py
##########
@@ -1262,6 +1263,300 @@ def test_filter_null_type():
     assert len(table.filter(mask).column(0)) == 5
 
 
+@pytest.mark.parametrize("ty", ["inclusive"])
+def test_between_array_array_array(ty):
+    BetweenOptions = partial(pc.BetweenOptions)
+
+    val = pa.array([1, 1, 4, 3, 2, 6])
+    arr1 = pa.array([1, 1, 3, 4, None, 5])
+    arr2 = pa.array([1, 2, 4, None, 4, 7])
+
+    inclusive_and_expected = {
+        "both": [True, True, True, None, None, True],
+        "left": [False, True, False, None, None, True],
+        "right": [False, False, True, None, None, True],
+        "neither": [False, False, False, None, None, True],
+    }
+
+    for inclusive, expected in inclusive_and_expected.items():
+        options = BetweenOptions(inclusive=inclusive)
+        result = pc.between(val, arr1, arr2, options=options)
+        np.testing.assert_array_equal(result, pa.array(expected))

Review comment:
       Ok. Can remove string tests if deemed not required.




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

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

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



[GitHub] [arrow] bkmgit commented on a change in pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
bkmgit commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r776654198



##########
File path: cpp/src/arrow/util/bit_block_counter.h
##########
@@ -87,6 +87,147 @@ struct BitBlockOrNot<bool> {
   static bool Call(bool left, bool right) { return left || !right; }
 };
 
+// Three Arguments
+template <typename T>
+struct BitBlockAndAnd {
+  static T Call(T left, T mid, T right) { return left & mid & right; }
+};
+
+template <>
+struct BitBlockAndAnd<bool> {
+  static bool Call(bool left, bool mid, bool right) { return left && mid && right; }

Review comment:
       Done




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

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

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



[GitHub] [arrow] lidavidm commented on pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
lidavidm commented on pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#issuecomment-1003017650


   Consistency is important but not at the cost of everything else.
   
   @edponce I think I'm missing something, what is the difference between 1) and 2b)?


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

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

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



[GitHub] [arrow] bkmgit commented on pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
bkmgit commented on pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#issuecomment-1009992012


   @edponce Thanks. Finalizing tests. @lidavidm Let us know if ok


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

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

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



[GitHub] [arrow] bkmgit commented on pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
bkmgit commented on pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#issuecomment-1010646334


   @lidavidm Thanks for the feedback. Inclusive is in global space so that it can also be used in NotBetween - follow up issue https://issues.apache.org/jira/browse/ARROW-15223


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

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

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



[GitHub] [arrow] bkmgit commented on a change in pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
bkmgit commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r782685428



##########
File path: docs/source/python/api/compute.rst
##########
@@ -158,6 +158,7 @@ they return ``null``.
    less
    less_equal
    not_equal
+   between

Review comment:
       Done.




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

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

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



[GitHub] [arrow] bkmgit commented on pull request #11882: ARROW-9843: [C++][Python] Implement Between ternary kernel and Python bindings

Posted by GitBox <gi...@apache.org>.
bkmgit commented on pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#issuecomment-1020947191


   Comparisons of the form:
   ```
     ValidateBetween(Datum(std::make_shared<ScalarType>("zzz")),
         Datum(ArrayFromJSON(TypeTraits<StringType>::type_singleton(), "[]")),
         Datum(ArrayFromJSON(TypeTraits<StringType>::type_singleton(), "[null]")));
   ```
   and
   ```
   ValidateCompare<TypeParam>(eq, "[null]", "[]", "[]");
   ```
   fail with `failed with Invalid: Array arguments must all be the same length` from   [exec.cc](https://github.com/apache/arrow/blob/master/cpp/src/arrow/compute/exec.cc#L265), while comparisons of the form   
   ```
     ValidateBetween(Datum(std::make_shared<ScalarType>("zzz")),
         Datum(ArrayFromJSON(TypeTraits<StringType>::type_singleton(), "[]")),
         Datum(ArrayFromJSON(TypeTraits<StringType>::type_singleton(), "[]")));
   ```
   and
   ```
     ValidateBetween(Datum(std::make_shared<ScalarType>("zzz")),
         Datum(ArrayFromJSON(TypeTraits<StringType>::type_singleton(), "[null]")),
         Datum(ArrayFromJSON(TypeTraits<StringType>::type_singleton(), "[null]")));
   ```
   are valid. Is this the behavior that is desired? Would expect comparing null with empty to give empty for scalar comparisons.


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

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

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



[GitHub] [arrow] edponce edited a comment on pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
edponce edited a comment on pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#issuecomment-1013718034


   @bkmgit I am curious why there are many changes to Java, Go, R, Makefile files made alongside this Between PR?
   Preferably, this PR should have kept scoped to the C++ Between implementation and its Python bindings.


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

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

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



[GitHub] [arrow] bkmgit commented on a change in pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
bkmgit commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r785091500



##########
File path: cpp/src/arrow/compute/kernels/scalar_compare_test.cc
##########
@@ -1865,5 +1861,636 @@ TEST(TestMaxElementWiseMinElementWise, CommonTemporal) {
               ResultWith(ScalarFromJSON(date64(), "86400000")));
 }
 
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const Datum& val, const Datum& lhs,
+                            const Datum& rhs, const Datum& expected) {
+  ASSERT_OK_AND_ASSIGN(Datum result, Between(val, lhs, rhs, options, nullptr));
+  AssertArraysEqual(*expected.make_array(), *result.make_array(),
+                    /*verbose=*/true);
+}
+
+template <typename ArrowType>
+void ValidateBetween(BetweenOptions options, const Datum& val, const Datum& lhs,
+                     const Datum& rhs) {
+  CompareOperator lhs_val;
+  CompareOperator val_rhs;
+  BetweenOptions::Inclusive include_endpoints = options.inclusive;
+
+  if (include_endpoints == BetweenOptions::Inclusive::NEITHER) {
+    lhs_val = LESS;
+    val_rhs = LESS;
+  } else if (include_endpoints == BetweenOptions::Inclusive::LEFT) {
+    lhs_val = LESS_EQUAL;
+    val_rhs = LESS;
+  } else if (include_endpoints == BetweenOptions::Inclusive::RIGHT) {
+    lhs_val = LESS;
+    val_rhs = LESS_EQUAL;
+  } else {
+    lhs_val = LESS_EQUAL;
+    val_rhs = LESS_EQUAL;
+  }
+
+  ASSERT_OK_AND_ASSIGN(Datum resultl,
+                       CallFunction(CompareOperatorToFunctionName(lhs_val), {lhs, val}));
+  ASSERT_OK_AND_ASSIGN(Datum resultr,
+                       CallFunction(CompareOperatorToFunctionName(val_rhs), {val, rhs}));
+  ASSERT_OK_AND_ASSIGN(Datum expected, CallFunction("and", {resultl, resultr}));
+
+  ValidateBetween<ArrowType>(options, val, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+class TestNumericBetweenKernel : public ::testing::Test {};
+
+TYPED_TEST_SUITE(TestNumericBetweenKernel, NumericArrowTypes);
+TYPED_TEST(TestNumericBetweenKernel, SimpleBetweenArrayScalarScalar) {
+  using ScalarType = typename TypeTraits<TypeParam>::ScalarType;
+  using CType = typename TypeTraits<TypeParam>::CType;
+
+  Datum zero(std::make_shared<ScalarType>(CType(0)));
+  Datum four(std::make_shared<ScalarType>(CType(4)));
+  Datum null(std::make_shared<ScalarType>());
+  BetweenOptions InclusiveBoth(BetweenOptions::Inclusive::BOTH);
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"), zero,
+      four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      zero, four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,1,2,2]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[1,1,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,3,4,5]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[1,1,1,1,1,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,1,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null,0,1,1]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), null, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, null,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  BetweenOptions InclusiveLeft(BetweenOptions::Inclusive::LEFT);
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"), zero,
+      four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      zero, four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,1,2,2]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[1,1,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,3,4,5]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[1,1,1,1,0,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,0,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null,0,1,1]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), null, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, null,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  BetweenOptions InclusiveRight(BetweenOptions::Inclusive::RIGHT);
+  ValidateBetween<TypeParam>(
+      InclusiveRight, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"), zero,
+      four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      zero, four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,1,2,2]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,0,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,3,4,5]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,1,1,1,1,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,1,1,1,1,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null,0,1,1]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null,0,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), null, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, null,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  BetweenOptions InclusiveNeither(BetweenOptions::Inclusive::NEITHER);
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      zero, four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      zero, four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,1,2,2]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,0,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,3,4,5]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,1,1,1,0,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,0,1,1,1,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null,0,1,1]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null,0,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), null, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, null,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+}
+
+TYPED_TEST(TestNumericBetweenKernel, SimpleBetweenScalarArrayArray) {
+  using ScalarType = typename TypeTraits<TypeParam>::ScalarType;
+  using CType = typename TypeTraits<TypeParam>::CType;
+
+  Datum one(std::make_shared<ScalarType>(CType(1)));
+
+  BetweenOptions InclusiveBoth(BetweenOptions::Inclusive::BOTH);
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, one, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,true,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,true,null,false,false]"));
+  BetweenOptions InclusiveLeft(BetweenOptions::Inclusive::LEFT);
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, one, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,true,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,true,null,false,false]"));
+  BetweenOptions InclusiveRight(BetweenOptions::Inclusive::RIGHT);
+  ValidateBetween<TypeParam>(
+      InclusiveRight, one, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,false,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,false,null,false,false]"));
+  BetweenOptions InclusiveNeither(BetweenOptions::Inclusive::NEITHER);
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, one, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,false,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,false,null,false,false]"));
+}
+
+TYPED_TEST(TestNumericBetweenKernel, SimpleBetweenArrayArrayArray) {
+  BetweenOptions InclusiveBoth(BetweenOptions::Inclusive::BOTH);
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[1,1,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,true,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,true,null,false,false]"));
+  BetweenOptions InclusiveLeft(BetweenOptions::Inclusive::LEFT);
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[1,1,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,false,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,false,null,false,false]"));
+  BetweenOptions InclusiveRight(BetweenOptions::Inclusive::RIGHT);
+  ValidateBetween<TypeParam>(
+      InclusiveRight, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[1,1,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,true,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,true,null,false,false]"));
+  BetweenOptions InclusiveNeither(BetweenOptions::Inclusive::NEITHER);
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[1,1,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,false,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,false,null,false,false]"));
+}
+
+template <typename Type>
+struct BetweenRandomNumeric {
+  static void Test(const std::shared_ptr<DataType>& type) {
+    using ScalarType = typename TypeTraits<Type>::ScalarType;
+    using CType = typename TypeTraits<Type>::CType;
+    auto rand = random::RandomArrayGenerator(0x5416447);
+    const int64_t length = 1000;
+    for (auto null_probability : {0.0, 0.01, 0.1, 0.25, 0.5, 1.0}) {
+      for (auto inclusive :
+           {BetweenOptions::Inclusive::BOTH, BetweenOptions::Inclusive::LEFT,
+            BetweenOptions::Inclusive::RIGHT, BetweenOptions::Inclusive::NEITHER}) {
+        auto data1 =
+            rand.Numeric<typename Type::PhysicalType>(length, 0, 100, null_probability);
+        auto data2 =
+            rand.Numeric<typename Type::PhysicalType>(length, 0, 100, null_probability);
+        auto data3 =
+            rand.Numeric<typename Type::PhysicalType>(length, 0, 100, null_probability);
+
+        // Create view of data as the type (e.g. timestamp)
+        auto array1 = Datum(*data1->View(type));
+        auto array2 = Datum(*data2->View(type));
+        auto array3 = Datum(*data3->View(type));
+        auto scalar1 = Datum(std::make_shared<ScalarType>(CType(10), type));
+        auto scalar2 = Datum(std::make_shared<ScalarType>(CType(30), type));
+        auto scalar3 = Datum(std::make_shared<ScalarType>(CType(50), type));
+        auto options = BetweenOptions(inclusive);
+        ValidateBetween<Type>(options, array1, scalar2, scalar3);
+        ValidateBetween<Type>(options, array1, array2, scalar3);
+        ValidateBetween<Type>(options, array1, array2, array3);
+        ValidateBetween<Type>(options, array1, scalar2, scalar3);
+        ValidateBetween<Type>(options, scalar1, array2, array3);
+        ValidateBetween<Type>(options, scalar1, array2, scalar3);
+        ValidateBetween<Type>(options, scalar1, scalar2, array3);
+        ValidateBetween<Type>(options, array1, scalar2, array3);
+      }
+    }
+  }
+};
+
+TEST(TestNumericBetweenKernel, BetweenPrimitiveRandomTests) {
+  TestRandomPrimitiveCTypes<BetweenRandomNumeric>();
+}
+
+class TestStringBetweenKernel : public ::testing::Test {};
+
+TEST(TestStringBetweenKernel, RandomBetween) {
+  using ScalarType = typename TypeTraits<StringType>::ScalarType;
+
+  auto rand = random::RandomArrayGenerator(0x5416447);
+  for (size_t i = 3; i < 10; i++) {
+    for (auto null_probability : {0.0, 0.01, 0.1, 0.25, 0.5, 1.0}) {
+      for (auto inclusive :
+           {BetweenOptions::Inclusive::BOTH, BetweenOptions::Inclusive::LEFT,
+            BetweenOptions::Inclusive::RIGHT, BetweenOptions::Inclusive::NEITHER}) {
+        const int64_t length = static_cast<int64_t>(1ULL << i);
+        auto array1 = Datum(rand.String(length, 0, 16, null_probability));
+        auto array2 = Datum(rand.String(length, 0, 16, null_probability));
+        auto array3 = Datum(rand.String(length, 0, 16, null_probability));
+        auto scalar1 = Datum(std::make_shared<ScalarType>("fupi"));
+        auto scalar2 = Datum(std::make_shared<ScalarType>("tupu"));
+        auto scalar3 = Datum(std::make_shared<ScalarType>("zito"));
+        auto options = BetweenOptions(inclusive);
+        ValidateBetween<StringType>(options, array1, scalar2, scalar3);
+        ValidateBetween<StringType>(options, scalar1, array2, scalar3);
+        ValidateBetween<StringType>(options, scalar1, scalar2, array3);
+        ValidateBetween<StringType>(options, scalar1, array2, array3);
+        ValidateBetween<StringType>(options, array1, scalar2, array3);
+        ValidateBetween<StringType>(options, array1, array2, scalar3);
+        ValidateBetween<StringType>(options, array1, array2, array3);
+      }
+    }
+  }
+}
+
+TEST(TestStringBetweenKernel, StringArrayScalarScalarTest) {
+  using ScalarType = typename TypeTraits<StringType>::ScalarType;
+  auto l = Datum(std::make_shared<ScalarType>("abc"));
+  auto r = Datum(std::make_shared<ScalarType>("zzz"));
+  BetweenOptions InclusiveOption(BetweenOptions::Inclusive::BOTH);
+  ValidateBetween<StringType>(
+      InclusiveOption, ArrayFromJSON(TypeTraits<StringType>::type_singleton(), "[]"), l,
+      r, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<StringType>(
+      InclusiveOption, ArrayFromJSON(TypeTraits<StringType>::type_singleton(), "[null]"),
+      l, r, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<StringType>(InclusiveOption,
+                              ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                            R"(["aaa", "aaaa", "ccc", "z"])"),
+                              l, r,
+                              ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                                            R"([false, false, true, true])"));
+  ValidateBetween<StringType>(InclusiveOption,
+                              ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                            R"(["a", "aaaa", "c", "z"])"),
+                              l, r,
+                              ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                                            R"([false, false, true, true])"));
+  ValidateBetween<StringType>(InclusiveOption,
+                              ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                            R"(["a", "aaaa", "fff", "zzzz"])"),
+                              l, r,
+                              ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                                            R"([false, false, true, false])"));
+  ValidateBetween<StringType>(InclusiveOption,
+                              ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                            R"(["abc", "baa", "fff", "zzz"])"),
+                              l, r,
+                              ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                                            R"([true, true, true, true])"));
+  ValidateBetween<StringType>(InclusiveOption,
+                              ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                            R"(["abd", null, null, "zzx"])"),
+                              l, r,
+                              ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                                            R"([true, null, null, true])"));
+}
+
+TEST(TestStringBetweenKernel, StringArrayArrayArrayTest) {
+  BetweenOptions InclusiveOption(BetweenOptions::Inclusive::BOTH);
+  ValidateBetween<StringType>(
+      InclusiveOption,
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                    R"(["david","hello","world"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["adam","hi","whirl"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                    R"(["robert","goeiemoreen","whirlwind"])"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[true, false, false]"));
+  ValidateBetween<StringType>(
+      InclusiveOption,
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["x","a","f"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["w","a","e"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["z","a","g"])"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[true, true, true]"));
+  ValidateBetween<StringType>(
+      InclusiveOption,
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                    R"(["block","bit","binary"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                    R"(["bit","nibble","ternary"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["word","d","xyz"])"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[true, false, false]"));
+  ValidateBetween<StringType>(
+      InclusiveOption,
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                    R"(["よしもと","の","ち"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["は","へ","あ"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["な","を","ち"])"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[false, false, true]"));
+  ValidateBetween<StringType>(
+      InclusiveOption,
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["A","ア","王"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["た","あ","歩"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["李","田",null])"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[false, true, null]"));
+  ValidateBetween<StringType>(
+      InclusiveOption,
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["Б",null,"Я"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["А","Ж","Щ"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["Д","Л","Ф"])"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[true, null, false]"));
+}
+
+TEST(TestTimestampsBetweenKernel, TimestampsArrayArrayArrayTest) {
+  const std::string arr_json = R"(["1970-01-01","2000-02-02","1900-02-28"])";
+  const std::string lhs_json = R"(["1970-01-01","2000-02-01","1900-02-28"])";
+  const std::string rhs_json = R"(["1970-01-02","2000-02-02","1900-02-28"])";
+  // Between Options
+  BetweenOptions both(BetweenOptions::Inclusive::BOTH);
+  BetweenOptions left(BetweenOptions::Inclusive::LEFT);
+  BetweenOptions right(BetweenOptions::Inclusive::RIGHT);
+  BetweenOptions neither(BetweenOptions::Inclusive::NEITHER);
+  // Same units should be fine
+  auto arr = ArrayFromJSON(timestamp(TimeUnit::SECOND), arr_json);
+  auto lhs = ArrayFromJSON(timestamp(TimeUnit::SECOND), lhs_json);
+  auto rhs = ArrayFromJSON(timestamp(TimeUnit::SECOND), rhs_json);
+  // BOTH
+  ValidateBetween<TimestampType>(both, arr, lhs, rhs);
+  // LEFT
+  ValidateBetween<TimestampType>(left, arr, lhs, rhs);
+  // RIGHT
+  ValidateBetween<TimestampType>(right, arr, lhs, rhs);
+  // NEITHER
+  ValidateBetween<TimestampType>(neither, arr, lhs, rhs);
+/*  // So are same time zones
+  arr = ArrayFromJSON(timestamp(TimeUnit::SECOND, "America/Chicago"), arr_json);
+  lhs = ArrayFromJSON(timestamp(TimeUnit::SECOND, "America/Chicago"), lhs_json);
+  rhs = ArrayFromJSON(timestamp(TimeUnit::SECOND, "America/Chicago"), rhs_json);
+  // BOTH
+  ValidateBetween<TimestampType>(both, arr, lhs, rhs);
+  // LEFT
+  ValidateBetween<TimestampType>(left, arr, lhs, rhs);
+  // RIGHT
+  ValidateBetween<TimestampType>(right, arr, lhs, rhs);
+  // NEITHER
+  ValidateBetween<TimestampType>(neither, arr, lhs, rhs);
+  // Different units should be fine
+  auto arr = ArrayFromJSON(timestamp(TimeUnit::SECOND), arr_json);
+  auto lhs = ArrayFromJSON(timestamp(TimeUnit::MILLI), lhs_json);
+  auto rhs = ArrayFromJSON(timestamp(TimeUnit::SECOND), rhs_json);
+  // BOTH

Review comment:
       Compare kernels for timestamps have a conversion of the form  `InputType in_type(match::TimestampTypeUnit(ty));` this is probably also helpful for Between.




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

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

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



[GitHub] [arrow] jorisvandenbossche commented on a change in pull request #11882: ARROW-9843: [C++][Python] Implement Between ternary kernel and Python bindings

Posted by GitBox <gi...@apache.org>.
jorisvandenbossche commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r785833236



##########
File path: python/pyarrow/_compute.pyx
##########
@@ -785,6 +785,30 @@ class ElementWiseAggregateOptions(_ElementWiseAggregateOptions):
         self._set_options(skip_nulls)
 
 
+cdef CInclusive unwrap_inclusive(inclusive) except *:
+    if inclusive == "both":
+        return CInclusive_BOTH
+    elif inclusive == "left":
+        return CInclusive_LEFT
+    elif inclusive == "right":
+        return CInclusive_RIGHT
+    elif inclusive == "neither":
+        return CInclusive_NEITHER
+    _raise_invalid_function_option(inclusive, "inclusive")
+
+
+cdef class _BetweenOptions(FunctionOptions):
+    def _set_options(self, inclusive):
+        self.wrapped.reset(
+            new CBetweenOptions(unwrap_inclusive(inclusive))
+        )
+
+
+class BetweenOptions(_BetweenOptions):

Review comment:
       Can you add a docstring to this class? (it's a recent change in master that all classes now have a docstring, see the other classes in this file)




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

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

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



[GitHub] [arrow] bkmgit commented on a change in pull request #11882: ARROW-9843: [C++][Python] Implement Between ternary kernel and Python bindings

Posted by GitBox <gi...@apache.org>.
bkmgit commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r786497997



##########
File path: cpp/src/arrow/compute/kernels/scalar_compare.cc
##########
@@ -156,39 +212,52 @@ struct Maximum {
   }
 };
 
+// Check if timestamp timezones are comparable (either all are empty or none is).
+Status CheckCompareTimestamps(const ExecBatch& batch) {
+  if (batch.num_values() > 0) {
+    for (int i = 0; i < batch.num_values() - 1; ++i) {
+      const auto& tsi = checked_cast<const TimestampType&>(*batch[i].type());
+      for (int j = i + 1; j < batch.num_values(); ++j) {
+        const auto& tsj = checked_cast<const TimestampType&>(*batch[j].type());
+        bool invalid_state = tsi.timezone().empty();
+        invalid_state ^= tsj.timezone().empty();
+        if (invalid_state) {
+          return Status::Invalid(

Review comment:
       Done




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

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

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



[GitHub] [arrow] bkmgit commented on a change in pull request #11882: ARROW-9843: [C++][Python] Implement Between ternary kernel and Python bindings

Posted by GitBox <gi...@apache.org>.
bkmgit commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r786980021



##########
File path: cpp/src/arrow/compute/kernels/scalar_compare.cc
##########
@@ -156,39 +212,52 @@ struct Maximum {
   }
 };
 
+// Check if timestamp timezones are comparable (either all are empty or none is).
+Status CheckCompareTimestamps(const ExecBatch& batch) {
+  if (batch.num_values() > 0) {
+    for (int i = 0; i < batch.num_values() - 1; ++i) {
+      const auto& tsi = checked_cast<const TimestampType&>(*batch[i].type());
+      for (int j = i + 1; j < batch.num_values(); ++j) {

Review comment:
       Checks for the sum.




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

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

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



[GitHub] [arrow] pitrou commented on a change in pull request #11882: ARROW-9843: [C++][Python] Implement Between ternary kernel and Python bindings

Posted by GitBox <gi...@apache.org>.
pitrou commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r786220370



##########
File path: cpp/src/arrow/compute/kernels/scalar_compare.cc
##########
@@ -156,39 +212,52 @@ struct Maximum {
   }
 };
 
+// Check if timestamp timezones are comparable (either all are empty or none is).
+Status CheckCompareTimestamps(const ExecBatch& batch) {
+  if (batch.num_values() > 0) {
+    for (int i = 0; i < batch.num_values() - 1; ++i) {
+      const auto& tsi = checked_cast<const TimestampType&>(*batch[i].type());
+      for (int j = i + 1; j < batch.num_values(); ++j) {

Review comment:
       I don't understand. You're only looking to see if all values either have a timezone, or don't have one, right? Why bother with boolean algebra at all?




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

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

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



[GitHub] [arrow] bkmgit commented on a change in pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
bkmgit commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r767505110



##########
File path: cpp/src/arrow/compute/kernels/scalar_compare_test.cc
##########
@@ -1850,5 +1851,154 @@ TEST(TestMaxElementWiseMinElementWise, CommonTemporal) {
               ResultWith(ScalarFromJSON(date64(), "86400000")));
 }
 
+template <typename ArrowType>
+static void ValidateBetween(const Datum& val, const Datum& lhs, const Datum& rhs,
+                            const Datum& expected) {
+  ASSERT_OK_AND_ASSIGN(Datum result, Between(val, lhs, rhs));
+  AssertArraysEqual(*expected.make_array(), *result.make_array(),
+                    /*verbose=*/true);
+}
+
+template <typename ArrowType>
+static void ValidateBetween(const char* value_str, const Datum& lhs, const Datum& rhs,
+                            const char* expected_str) {
+  auto value = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), value_str);
+  auto expected = ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), expected_str);
+  ValidateBetween<ArrowType>(value, lhs, rhs, expected);
+}
+
+template <>
+void ValidateBetween<StringType>(const char* value_str, const Datum& lhs,
+                                 const Datum& rhs, const char* expected_str) {
+  auto value = ArrayFromJSON(utf8(), value_str);
+  auto expected = ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), expected_str);
+  ValidateBetween<StringType>(value, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+class TestNumericBetweenKernel : public ::testing::Test {};
+
+TYPED_TEST_SUITE(TestNumericBetweenKernel, NumericArrowTypes);
+TYPED_TEST(TestNumericBetweenKernel, SimpleBetweenArrayScalarScalar) {
+  using ScalarType = typename TypeTraits<TypeParam>::ScalarType;
+  using CType = typename TypeTraits<TypeParam>::CType;
+
+  Datum zero(std::make_shared<ScalarType>(CType(0)));
+  Datum four(std::make_shared<ScalarType>(CType(4)));
+  ValidateBetween<TypeParam>("[]", zero, four, "[]");
+  ValidateBetween<TypeParam>("[null]", zero, four, "[null]");
+  ValidateBetween<TypeParam>("[0,0,1,1,2,2]", zero, four, "[0,0,1,1,1,1]");
+  ValidateBetween<TypeParam>("[0,1,2,3,4,5]", zero, four, "[0,1,1,1,0,0]");
+  ValidateBetween<TypeParam>("[5,4,3,2,1,0]", zero, four, "[0,0,1,1,1,0]");
+  ValidateBetween<TypeParam>("[null,0,1,1]", zero, four, "[null,0,1,1]");
+}
+
+TEST(TestSimpleBetweenKernel, SimpleStringTest) {
+  using ScalarType = typename TypeTraits<StringType>::ScalarType;
+  auto l = Datum(std::make_shared<ScalarType>("abc"));
+  auto r = Datum(std::make_shared<ScalarType>("zzz"));
+  ValidateBetween<StringType>("[]", l, r, "[]");
+  ValidateBetween<StringType>("[null]", l, r, "[null]");
+  ValidateBetween<StringType>(R"(["aaa", "aaaa", "ccc", "z"])", l, r,
+                              R"([false, false, true, true])");
+  ValidateBetween<StringType>(R"(["a", "aaaa", "c", "z"])", l, r,
+                              R"([false, false, true, true])");
+  ValidateBetween<StringType>(R"(["a", "aaaa", "fff", "zzzz"])", l, r,
+                              R"([false, false, true, false])");
+  ValidateBetween<StringType>(R"(["abd", null, null, "zzx"])", l, r,
+                              R"([true, null, null, true])");
+}
+
+TEST(TestSimpleBetweenKernel, SimpleTimestampTest) {
+  using ScalarType = typename TypeTraits<TimestampType>::ScalarType;
+  auto checkTimestampArray = [](std::shared_ptr<DataType> type, const char* input_str,
+                                const Datum& lhs, const Datum& rhs,
+                                const char* expected_str) {
+    auto value = ArrayFromJSON(type, input_str);
+    auto expected = ArrayFromJSON(boolean(), expected_str);
+    ValidateBetween<TimestampType>(value, lhs, rhs, expected);
+  };
+  auto unit = TimeUnit::SECOND;
+  auto l = Datum(std::make_shared<ScalarType>(923184000, timestamp(unit)));
+  auto r = Datum(std::make_shared<ScalarType>(1602032602, timestamp(unit)));
+  checkTimestampArray(timestamp(unit), "[]", l, r, "[]");
+  checkTimestampArray(timestamp(unit), "[null]", l, r, "[null]");
+  checkTimestampArray(timestamp(unit), R"(["1970-01-01","2000-02-29","1900-02-28"])", l,
+                      r, "[false,true,false]");
+  checkTimestampArray(timestamp(unit), R"(["1970-01-01","2000-02-29","2004-02-28"])", l,
+                      r, "[false,true,true]");
+  checkTimestampArray(timestamp(unit), R"(["2018-01-01","1999-04-04","1900-02-28"])", l,
+                      r, "[true,false,false]");
+}
+
+TYPED_TEST(TestNumericBetweenKernel, SimpleBetweenArrayArrayArray) {
+  ValidateBetween<TypeParam>(
+      "[]", ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"), "[]");
+  ValidateBetween<TypeParam>(
+      "[null]", ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"), "[null]");
+  ValidateBetween<TypeParam>(
+      "[1,1,2,2,2]",
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      "[true,true,false,false,false]");
+  ValidateBetween<TypeParam>(
+      "[1,1,2,2,2,2]",
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,2,5,5]"),
+      "[true,true,false,null,false,false]");
+}
+
+TEST(TestSimpleBetweenKernel, StringArrayArrayArrayTest) {
+  ValidateBetween<StringType>(
+      R"(["david","hello","world"])",
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["adam","hi","whirl"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                    R"(["robert","goeiemoreen","whirlwind"])"),
+      "[true, false, false]");
+  ValidateBetween<StringType>(
+      R"(["x","a","f"])",
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["w","a","e"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["z","a","g"])"),
+      "[true, false, true]");
+  ValidateBetween<StringType>(
+      R"(["block","bit","binary"])",
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                    R"(["bit","nibble","ternary"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["word","d","xyz"])"),
+      "[true, false, false]");
+  ValidateBetween<StringType>(R"(["Ayumi","アユミ","王梦莹"])",
+                              ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                            R"(["たなか","あゆみ","歩美"])"),
+                              ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                            R"(["李平之","田中","たなか"])"),
+                              "[false, true, false]");

Review comment:
       We can use Arabic and Korean as well. Ordering for Japanese and Mandarin characters in UTF8 is not usually used for many purposes as it is not intuitive without a context as explained in http://www.localizingjapan.com/blog/2011/02/13/ - the test therefore only checks that one gets unicode ordering which may not make sense in many cases. Some Mandarin characters are used in Kanji, but as unicode requires one code point per character, ordering will be problematic. Even languages which use latin alphabet can have different ordering of characters with accents such as Ä, Ö, and Ü. Probably an external function and/or alternative implementation is needed for strings and will need to be something to consider for future versions of the Arrow specification. For Japanese, can also use Kana which has [Gojūon](https://en.wikipedia.org/wiki/Goj%C5%ABon) ordering so that the test makes sense if this is needed. Can leave out recent Unicode extensions described at https://en.wikipedia.org
 /wiki/Kana Can raise a separate issue for the Arrow extension to be able to sort text.




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

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

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



[GitHub] [arrow] bkmgit commented on a change in pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
bkmgit commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r767505110



##########
File path: cpp/src/arrow/compute/kernels/scalar_compare_test.cc
##########
@@ -1850,5 +1851,154 @@ TEST(TestMaxElementWiseMinElementWise, CommonTemporal) {
               ResultWith(ScalarFromJSON(date64(), "86400000")));
 }
 
+template <typename ArrowType>
+static void ValidateBetween(const Datum& val, const Datum& lhs, const Datum& rhs,
+                            const Datum& expected) {
+  ASSERT_OK_AND_ASSIGN(Datum result, Between(val, lhs, rhs));
+  AssertArraysEqual(*expected.make_array(), *result.make_array(),
+                    /*verbose=*/true);
+}
+
+template <typename ArrowType>
+static void ValidateBetween(const char* value_str, const Datum& lhs, const Datum& rhs,
+                            const char* expected_str) {
+  auto value = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), value_str);
+  auto expected = ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), expected_str);
+  ValidateBetween<ArrowType>(value, lhs, rhs, expected);
+}
+
+template <>
+void ValidateBetween<StringType>(const char* value_str, const Datum& lhs,
+                                 const Datum& rhs, const char* expected_str) {
+  auto value = ArrayFromJSON(utf8(), value_str);
+  auto expected = ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), expected_str);
+  ValidateBetween<StringType>(value, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+class TestNumericBetweenKernel : public ::testing::Test {};
+
+TYPED_TEST_SUITE(TestNumericBetweenKernel, NumericArrowTypes);
+TYPED_TEST(TestNumericBetweenKernel, SimpleBetweenArrayScalarScalar) {
+  using ScalarType = typename TypeTraits<TypeParam>::ScalarType;
+  using CType = typename TypeTraits<TypeParam>::CType;
+
+  Datum zero(std::make_shared<ScalarType>(CType(0)));
+  Datum four(std::make_shared<ScalarType>(CType(4)));
+  ValidateBetween<TypeParam>("[]", zero, four, "[]");
+  ValidateBetween<TypeParam>("[null]", zero, four, "[null]");
+  ValidateBetween<TypeParam>("[0,0,1,1,2,2]", zero, four, "[0,0,1,1,1,1]");
+  ValidateBetween<TypeParam>("[0,1,2,3,4,5]", zero, four, "[0,1,1,1,0,0]");
+  ValidateBetween<TypeParam>("[5,4,3,2,1,0]", zero, four, "[0,0,1,1,1,0]");
+  ValidateBetween<TypeParam>("[null,0,1,1]", zero, four, "[null,0,1,1]");
+}
+
+TEST(TestSimpleBetweenKernel, SimpleStringTest) {
+  using ScalarType = typename TypeTraits<StringType>::ScalarType;
+  auto l = Datum(std::make_shared<ScalarType>("abc"));
+  auto r = Datum(std::make_shared<ScalarType>("zzz"));
+  ValidateBetween<StringType>("[]", l, r, "[]");
+  ValidateBetween<StringType>("[null]", l, r, "[null]");
+  ValidateBetween<StringType>(R"(["aaa", "aaaa", "ccc", "z"])", l, r,
+                              R"([false, false, true, true])");
+  ValidateBetween<StringType>(R"(["a", "aaaa", "c", "z"])", l, r,
+                              R"([false, false, true, true])");
+  ValidateBetween<StringType>(R"(["a", "aaaa", "fff", "zzzz"])", l, r,
+                              R"([false, false, true, false])");
+  ValidateBetween<StringType>(R"(["abd", null, null, "zzx"])", l, r,
+                              R"([true, null, null, true])");
+}
+
+TEST(TestSimpleBetweenKernel, SimpleTimestampTest) {
+  using ScalarType = typename TypeTraits<TimestampType>::ScalarType;
+  auto checkTimestampArray = [](std::shared_ptr<DataType> type, const char* input_str,
+                                const Datum& lhs, const Datum& rhs,
+                                const char* expected_str) {
+    auto value = ArrayFromJSON(type, input_str);
+    auto expected = ArrayFromJSON(boolean(), expected_str);
+    ValidateBetween<TimestampType>(value, lhs, rhs, expected);
+  };
+  auto unit = TimeUnit::SECOND;
+  auto l = Datum(std::make_shared<ScalarType>(923184000, timestamp(unit)));
+  auto r = Datum(std::make_shared<ScalarType>(1602032602, timestamp(unit)));
+  checkTimestampArray(timestamp(unit), "[]", l, r, "[]");
+  checkTimestampArray(timestamp(unit), "[null]", l, r, "[null]");
+  checkTimestampArray(timestamp(unit), R"(["1970-01-01","2000-02-29","1900-02-28"])", l,
+                      r, "[false,true,false]");
+  checkTimestampArray(timestamp(unit), R"(["1970-01-01","2000-02-29","2004-02-28"])", l,
+                      r, "[false,true,true]");
+  checkTimestampArray(timestamp(unit), R"(["2018-01-01","1999-04-04","1900-02-28"])", l,
+                      r, "[true,false,false]");
+}
+
+TYPED_TEST(TestNumericBetweenKernel, SimpleBetweenArrayArrayArray) {
+  ValidateBetween<TypeParam>(
+      "[]", ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"), "[]");
+  ValidateBetween<TypeParam>(
+      "[null]", ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"), "[null]");
+  ValidateBetween<TypeParam>(
+      "[1,1,2,2,2]",
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      "[true,true,false,false,false]");
+  ValidateBetween<TypeParam>(
+      "[1,1,2,2,2,2]",
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,2,5,5]"),
+      "[true,true,false,null,false,false]");
+}
+
+TEST(TestSimpleBetweenKernel, StringArrayArrayArrayTest) {
+  ValidateBetween<StringType>(
+      R"(["david","hello","world"])",
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["adam","hi","whirl"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                    R"(["robert","goeiemoreen","whirlwind"])"),
+      "[true, false, false]");
+  ValidateBetween<StringType>(
+      R"(["x","a","f"])",
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["w","a","e"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["z","a","g"])"),
+      "[true, false, true]");
+  ValidateBetween<StringType>(
+      R"(["block","bit","binary"])",
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                    R"(["bit","nibble","ternary"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["word","d","xyz"])"),
+      "[true, false, false]");
+  ValidateBetween<StringType>(R"(["Ayumi","アユミ","王梦莹"])",
+                              ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                            R"(["たなか","あゆみ","歩美"])"),
+                              ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                            R"(["李平之","田中","たなか"])"),
+                              "[false, true, false]");

Review comment:
       We can use Arabic and Korean as well. Ordering for Japanese and Mandarin characters in UTF8 is not usually used for many purposes as it is not intuitive without a context as explained in http://www.localizingjapan.com/blog/2011/02/13/ - the test therefore only checks that one gets unicode ordering which may not make sense in many cases. Some Mandarin characters are used in Kanji, but as unicode requires one code point per character, ordering will be problematic. Even languages which use latin alphabet can have different ordering of characters with accents such as Ä, Ö, and Ü. Probably an external function and/or alternative implementation is needed for strings and will need to be something to consider for future versions of the Arrow specification. For Japanese, can also use Kana which has [Gojūon](https://en.wikipedia.org/wiki/Goj%C5%ABon) ordering so that e test makes sense if this is needed. Can leave out recent Unicode extensions described at https://en.wikipedia.org/w
 iki/Kana Can raise a separate issue for the Arrow extension to be able to sort text.




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

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

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



[GitHub] [arrow] bkmgit commented on a change in pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
bkmgit commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r767459520



##########
File path: cpp/src/arrow/compute/kernels/scalar_compare_test.cc
##########
@@ -1850,5 +1851,154 @@ TEST(TestMaxElementWiseMinElementWise, CommonTemporal) {
               ResultWith(ScalarFromJSON(date64(), "86400000")));
 }
 
+template <typename ArrowType>
+static void ValidateBetween(const Datum& val, const Datum& lhs, const Datum& rhs,
+                            const Datum& expected) {
+  ASSERT_OK_AND_ASSIGN(Datum result, Between(val, lhs, rhs));
+  AssertArraysEqual(*expected.make_array(), *result.make_array(),
+                    /*verbose=*/true);
+}
+
+template <typename ArrowType>
+static void ValidateBetween(const char* value_str, const Datum& lhs, const Datum& rhs,
+                            const char* expected_str) {
+  auto value = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), value_str);
+  auto expected = ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), expected_str);
+  ValidateBetween<ArrowType>(value, lhs, rhs, expected);
+}
+
+template <>
+void ValidateBetween<StringType>(const char* value_str, const Datum& lhs,
+                                 const Datum& rhs, const char* expected_str) {
+  auto value = ArrayFromJSON(utf8(), value_str);
+  auto expected = ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), expected_str);
+  ValidateBetween<StringType>(value, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+class TestNumericBetweenKernel : public ::testing::Test {};
+
+TYPED_TEST_SUITE(TestNumericBetweenKernel, NumericArrowTypes);
+TYPED_TEST(TestNumericBetweenKernel, SimpleBetweenArrayScalarScalar) {
+  using ScalarType = typename TypeTraits<TypeParam>::ScalarType;
+  using CType = typename TypeTraits<TypeParam>::CType;
+
+  Datum zero(std::make_shared<ScalarType>(CType(0)));
+  Datum four(std::make_shared<ScalarType>(CType(4)));
+  ValidateBetween<TypeParam>("[]", zero, four, "[]");
+  ValidateBetween<TypeParam>("[null]", zero, four, "[null]");
+  ValidateBetween<TypeParam>("[0,0,1,1,2,2]", zero, four, "[0,0,1,1,1,1]");
+  ValidateBetween<TypeParam>("[0,1,2,3,4,5]", zero, four, "[0,1,1,1,0,0]");
+  ValidateBetween<TypeParam>("[5,4,3,2,1,0]", zero, four, "[0,0,1,1,1,0]");
+  ValidateBetween<TypeParam>("[null,0,1,1]", zero, four, "[null,0,1,1]");

Review comment:
       Thanks for the suggestion.




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

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

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



[GitHub] [arrow] lidavidm commented on pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
lidavidm commented on pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#issuecomment-997924475


   I think Unicode sorting is fine for strings, I don't think we even implement any other comparisons on strings at the moment? As you say, we can defer it for another issue. It would be good to define what other kinds of comparisons we need to support in the first place.


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

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

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



[GitHub] [arrow] bkmgit commented on a change in pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
bkmgit commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r772830961



##########
File path: cpp/src/arrow/compute/kernels/scalar_compare.cc
##########
@@ -68,6 +72,50 @@ struct GreaterEqual {
   }
 };
 
+struct BetweenLessEqualLessEqual {
+  template <typename T, typename Arg0, typename Arg1, typename Arg2>
+  static constexpr T Call(KernelContext*, const Arg0& middle, const Arg1& left,
+                          const Arg2& right, Status*) {
+    static_assert(std::is_same<T, bool>::value && std::is_same<Arg0, Arg1>::value &&
+                      std::is_same<Arg1, Arg2>::value,
+                  "");
+    return (left <= middle) && (middle <= right);
+  }
+};
+
+struct BetweenLessThanLessEqual {
+  template <typename T, typename Arg0, typename Arg1, typename Arg2>
+  static constexpr T Call(KernelContext*, const Arg0& middle, const Arg1& left,
+                          const Arg2& right, Status*) {
+    static_assert(std::is_same<T, bool>::value && std::is_same<Arg0, Arg1>::value &&
+                      std::is_same<Arg1, Arg2>::value,
+                  "");
+    return (left < middle) && (middle <= right);
+  }
+};
+
+struct BetweenLessEqualLessThan {
+  template <typename T, typename Arg0, typename Arg1, typename Arg2>
+  static constexpr T Call(KernelContext*, const Arg0& middle, const Arg1& left,
+                          const Arg2& right, Status*) {
+    static_assert(std::is_same<T, bool>::value && std::is_same<Arg0, Arg1>::value &&
+                      std::is_same<Arg1, Arg2>::value,
+                  "");
+    return (left <= middle) && (middle < right);
+  }
+};
+
+struct BetweenLessThanLessThan {
+  template <typename T, typename Arg0, typename Arg1, typename Arg2>
+  static constexpr T Call(KernelContext*, const Arg0& middle, const Arg1& left,
+                          const Arg2& right, Status*) {
+    static_assert(std::is_same<T, bool>::value && std::is_same<Arg0, Arg1>::value &&
+                      std::is_same<Arg1, Arg2>::value,
+                  "");
+    return (left < middle) && (middle < right);
+  }
+};

Review comment:
       Thanks.




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

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

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



[GitHub] [arrow] bkmgit commented on a change in pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
bkmgit commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r775902821



##########
File path: cpp/src/arrow/compute/kernels/scalar_compare.cc
##########
@@ -68,6 +72,50 @@ struct GreaterEqual {
   }
 };
 
+struct BetweenLessEqualLessEqual {
+  template <typename T, typename Arg0, typename Arg1, typename Arg2>
+  static constexpr T Call(KernelContext*, const Arg0& middle, const Arg1& left,
+                          const Arg2& right, Status*) {
+    static_assert(std::is_same<T, bool>::value && std::is_same<Arg0, Arg1>::value &&
+                      std::is_same<Arg1, Arg2>::value,
+                  "");
+    return (left <= middle) && (middle <= right);
+  }
+};
+
+struct BetweenLessThanLessEqual {
+  template <typename T, typename Arg0, typename Arg1, typename Arg2>
+  static constexpr T Call(KernelContext*, const Arg0& middle, const Arg1& left,
+                          const Arg2& right, Status*) {
+    static_assert(std::is_same<T, bool>::value && std::is_same<Arg0, Arg1>::value &&
+                      std::is_same<Arg1, Arg2>::value,
+                  "");
+    return (left < middle) && (middle <= right);
+  }
+};
+
+struct BetweenLessEqualLessThan {
+  template <typename T, typename Arg0, typename Arg1, typename Arg2>
+  static constexpr T Call(KernelContext*, const Arg0& middle, const Arg1& left,
+                          const Arg2& right, Status*) {
+    static_assert(std::is_same<T, bool>::value && std::is_same<Arg0, Arg1>::value &&
+                      std::is_same<Arg1, Arg2>::value,
+                  "");
+    return (left <= middle) && (middle < right);
+  }
+};
+
+struct BetweenLessThanLessThan {
+  template <typename T, typename Arg0, typename Arg1, typename Arg2>
+  static constexpr T Call(KernelContext*, const Arg0& middle, const Arg1& left,
+                          const Arg2& right, Status*) {
+    static_assert(std::is_same<T, bool>::value && std::is_same<Arg0, Arg1>::value &&
+                      std::is_same<Arg1, Arg2>::value,
+                  "");
+    return (left < middle) && (middle < right);
+  }
+};

Review comment:
       What was the reason for deprecating options from the compare function? Will the same apply to between?
   It seems possible to call other functions with variants, for example log10 and log10_checked. A smaller API is nice since it means easier binding creation and maintenance in other programming languages, though still want user friendly interfaces.




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

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

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



[GitHub] [arrow] bkmgit commented on a change in pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
bkmgit commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r776772489



##########
File path: cpp/src/arrow/compute/api_scalar.h
##########
@@ -316,6 +316,21 @@ struct ARROW_EXPORT CompareOptions {
   enum CompareOperator op;
 };
 
+enum class BetweenMode : int8_t {
+  LESS_EQUAL_LESS_EQUAL,
+  LESS_EQUAL_LESS,
+  LESS_LESS_EQUAL,
+  LESS_LESS,
+};

Review comment:
       Done.




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

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

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



[GitHub] [arrow] bkmgit commented on a change in pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
bkmgit commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r776851473



##########
File path: cpp/src/arrow/compute/kernels/scalar_compare.cc
##########
@@ -786,6 +932,34 @@ const FunctionDoc max_element_wise_doc{
     {"*args"},
     "ElementWiseAggregateOptions"};
 
+const FunctionDoc between_doc{"Check if values are in a range, val betwen a and b",

Review comment:
       Needed due to chosen interface.




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

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

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



[GitHub] [arrow] bkmgit commented on a change in pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
bkmgit commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r780780727



##########
File path: cpp/src/arrow/compute/kernels/scalar_compare.cc
##########
@@ -746,6 +787,111 @@ std::shared_ptr<ScalarFunction> MakeScalarMinMax(std::string name,
   return func;
 }
 
+template <typename Op>
+void AddIntegerBetween(const std::shared_ptr<DataType>& ty, ScalarFunction* func) {
+  auto exec = GeneratePhysicalInteger<ScalarTernaryEqualTypes, BooleanType, Op>(*ty);
+  DCHECK_OK(func->AddKernel({ty, ty, ty}, boolean(), std::move(exec)));
+}
+
+template <typename InType, typename Op>
+void AddGenericBetween(const std::shared_ptr<DataType>& ty, ScalarFunction* func) {
+  DCHECK_OK(func->AddKernel({ty, ty, ty}, boolean(),
+                            ScalarTernaryEqualTypes<BooleanType, InType, Op>::Exec));
+}

Review comment:
       No longer relevant.




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

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

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



[GitHub] [arrow] bkmgit commented on a change in pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
bkmgit commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r767603547



##########
File path: cpp/src/arrow/compute/kernels/scalar_compare.cc
##########
@@ -68,6 +72,32 @@ struct GreaterEqual {
   }
 };
 
+struct Between {
+  template <typename T, typename Arg0, typename Arg1, typename Arg2>
+  static constexpr T Call(KernelContext*, const Arg0& middle, const Arg1& left,
+                          const Arg2& right, Status*) {
+    static_assert(std::is_same<T, bool>::value && std::is_same<Arg0, Arg1>::value &&
+                      std::is_same<Arg1, Arg2>::value,
+                  "");
+    return (left < middle) && (middle < right);
+  }
+};
+
+template <typename T>
+using is_unsigned_integer = std::integral_constant<bool, std::is_integral<T>::value &&
+                                                             std::is_unsigned<T>::value>;

Review comment:
       Deleted these




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

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

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



[GitHub] [arrow] bkmgit commented on a change in pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
bkmgit commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r770340885



##########
File path: cpp/src/arrow/compute/kernels/scalar_compare_test.cc
##########
@@ -1850,5 +1851,154 @@ TEST(TestMaxElementWiseMinElementWise, CommonTemporal) {
               ResultWith(ScalarFromJSON(date64(), "86400000")));
 }
 
+template <typename ArrowType>
+static void ValidateBetween(const Datum& val, const Datum& lhs, const Datum& rhs,
+                            const Datum& expected) {
+  ASSERT_OK_AND_ASSIGN(Datum result, Between(val, lhs, rhs));
+  AssertArraysEqual(*expected.make_array(), *result.make_array(),
+                    /*verbose=*/true);
+}
+
+template <typename ArrowType>
+static void ValidateBetween(const char* value_str, const Datum& lhs, const Datum& rhs,
+                            const char* expected_str) {
+  auto value = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), value_str);
+  auto expected = ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), expected_str);
+  ValidateBetween<ArrowType>(value, lhs, rhs, expected);
+}
+
+template <>
+void ValidateBetween<StringType>(const char* value_str, const Datum& lhs,
+                                 const Datum& rhs, const char* expected_str) {
+  auto value = ArrayFromJSON(utf8(), value_str);
+  auto expected = ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), expected_str);
+  ValidateBetween<StringType>(value, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+class TestNumericBetweenKernel : public ::testing::Test {};
+
+TYPED_TEST_SUITE(TestNumericBetweenKernel, NumericArrowTypes);
+TYPED_TEST(TestNumericBetweenKernel, SimpleBetweenArrayScalarScalar) {

Review comment:
       Done.




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

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

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



[GitHub] [arrow] bkmgit commented on a change in pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
bkmgit commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r775938819



##########
File path: cpp/src/arrow/compute/kernels/scalar_compare.cc
##########
@@ -68,6 +72,50 @@ struct GreaterEqual {
   }
 };
 
+struct BetweenLessEqualLessEqual {
+  template <typename T, typename Arg0, typename Arg1, typename Arg2>
+  static constexpr T Call(KernelContext*, const Arg0& middle, const Arg1& left,
+                          const Arg2& right, Status*) {
+    static_assert(std::is_same<T, bool>::value && std::is_same<Arg0, Arg1>::value &&
+                      std::is_same<Arg1, Arg2>::value,
+                  "");
+    return (left <= middle) && (middle <= right);
+  }
+};
+
+struct BetweenLessThanLessEqual {
+  template <typename T, typename Arg0, typename Arg1, typename Arg2>
+  static constexpr T Call(KernelContext*, const Arg0& middle, const Arg1& left,
+                          const Arg2& right, Status*) {
+    static_assert(std::is_same<T, bool>::value && std::is_same<Arg0, Arg1>::value &&
+                      std::is_same<Arg1, Arg2>::value,
+                  "");
+    return (left < middle) && (middle <= right);
+  }
+};
+
+struct BetweenLessEqualLessThan {
+  template <typename T, typename Arg0, typename Arg1, typename Arg2>
+  static constexpr T Call(KernelContext*, const Arg0& middle, const Arg1& left,
+                          const Arg2& right, Status*) {
+    static_assert(std::is_same<T, bool>::value && std::is_same<Arg0, Arg1>::value &&
+                      std::is_same<Arg1, Arg2>::value,
+                  "");
+    return (left <= middle) && (middle < right);
+  }
+};
+
+struct BetweenLessThanLessThan {
+  template <typename T, typename Arg0, typename Arg1, typename Arg2>
+  static constexpr T Call(KernelContext*, const Arg0& middle, const Arg1& left,
+                          const Arg2& right, Status*) {
+    static_assert(std::is_same<T, bool>::value && std::is_same<Arg0, Arg1>::value &&
+                      std::is_same<Arg1, Arg2>::value,
+                  "");
+    return (left < middle) && (middle < right);
+  }
+};

Review comment:
       Strings are sorted by iso codes. The codes may not be the appropriate ordering in all cases, see for example, 
   
   - https://en.wikipedia.org/wiki/Alphabetical_order#Language-specific_conventions
   - https://www.postgresql.org/docs/current/collation.html
   - https://www.rdocumentation.org/packages/base/versions/3.6.2/topics/xtfrm




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

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

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



[GitHub] [arrow] bkmgit commented on a change in pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
bkmgit commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r775818526



##########
File path: cpp/src/arrow/compute/kernels/scalar_compare.cc
##########
@@ -68,6 +72,50 @@ struct GreaterEqual {
   }
 };
 
+struct BetweenLessEqualLessEqual {
+  template <typename T, typename Arg0, typename Arg1, typename Arg2>
+  static constexpr T Call(KernelContext*, const Arg0& middle, const Arg1& left,
+                          const Arg2& right, Status*) {
+    static_assert(std::is_same<T, bool>::value && std::is_same<Arg0, Arg1>::value &&
+                      std::is_same<Arg1, Arg2>::value,
+                  "");
+    return (left <= middle) && (middle <= right);
+  }
+};
+
+struct BetweenLessThanLessEqual {
+  template <typename T, typename Arg0, typename Arg1, typename Arg2>
+  static constexpr T Call(KernelContext*, const Arg0& middle, const Arg1& left,
+                          const Arg2& right, Status*) {
+    static_assert(std::is_same<T, bool>::value && std::is_same<Arg0, Arg1>::value &&
+                      std::is_same<Arg1, Arg2>::value,
+                  "");
+    return (left < middle) && (middle <= right);
+  }
+};
+
+struct BetweenLessEqualLessThan {
+  template <typename T, typename Arg0, typename Arg1, typename Arg2>
+  static constexpr T Call(KernelContext*, const Arg0& middle, const Arg1& left,
+                          const Arg2& right, Status*) {
+    static_assert(std::is_same<T, bool>::value && std::is_same<Arg0, Arg1>::value &&
+                      std::is_same<Arg1, Arg2>::value,
+                  "");
+    return (left <= middle) && (middle < right);
+  }
+};
+
+struct BetweenLessThanLessThan {
+  template <typename T, typename Arg0, typename Arg1, typename Arg2>
+  static constexpr T Call(KernelContext*, const Arg0& middle, const Arg1& left,
+                          const Arg2& right, Status*) {
+    static_assert(std::is_same<T, bool>::value && std::is_same<Arg0, Arg1>::value &&
+                      std::is_same<Arg1, Arg2>::value,
+                  "");
+    return (left < middle) && (middle < right);
+  }
+};

Review comment:
       Having between as a kind of compare function makes sense as they have similar code structures. Giving the option to access between or the between functions directly seems nice if one wants to keep the same structure as the compare functions.




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

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

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



[GitHub] [arrow] bkmgit commented on a change in pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
bkmgit commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r775909659



##########
File path: cpp/src/arrow/compute/kernels/scalar_compare.cc
##########
@@ -68,6 +72,50 @@ struct GreaterEqual {
   }
 };
 
+struct BetweenLessEqualLessEqual {
+  template <typename T, typename Arg0, typename Arg1, typename Arg2>
+  static constexpr T Call(KernelContext*, const Arg0& middle, const Arg1& left,
+                          const Arg2& right, Status*) {
+    static_assert(std::is_same<T, bool>::value && std::is_same<Arg0, Arg1>::value &&
+                      std::is_same<Arg1, Arg2>::value,
+                  "");
+    return (left <= middle) && (middle <= right);
+  }
+};
+
+struct BetweenLessThanLessEqual {
+  template <typename T, typename Arg0, typename Arg1, typename Arg2>
+  static constexpr T Call(KernelContext*, const Arg0& middle, const Arg1& left,
+                          const Arg2& right, Status*) {
+    static_assert(std::is_same<T, bool>::value && std::is_same<Arg0, Arg1>::value &&
+                      std::is_same<Arg1, Arg2>::value,
+                  "");
+    return (left < middle) && (middle <= right);
+  }
+};
+
+struct BetweenLessEqualLessThan {
+  template <typename T, typename Arg0, typename Arg1, typename Arg2>
+  static constexpr T Call(KernelContext*, const Arg0& middle, const Arg1& left,
+                          const Arg2& right, Status*) {
+    static_assert(std::is_same<T, bool>::value && std::is_same<Arg0, Arg1>::value &&
+                      std::is_same<Arg1, Arg2>::value,
+                  "");
+    return (left <= middle) && (middle < right);
+  }
+};
+
+struct BetweenLessThanLessThan {
+  template <typename T, typename Arg0, typename Arg1, typename Arg2>
+  static constexpr T Call(KernelContext*, const Arg0& middle, const Arg1& left,
+                          const Arg2& right, Status*) {
+    static_assert(std::is_same<T, bool>::value && std::is_same<Arg0, Arg1>::value &&
+                      std::is_same<Arg1, Arg2>::value,
+                  "");
+    return (left < middle) && (middle < right);
+  }
+};

Review comment:
       Ok. Am happy to attempt something similar to the round function, though it would be nice if it is for both between and compare functions since that would make maintenance and code reuse easier.
   
   For string comparison, some options that give a comparison key similar to a sort key will likely be needed.
   
   @edponce Light review would be helpful.




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

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

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



[GitHub] [arrow] edponce commented on a change in pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
edponce commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r776638260



##########
File path: cpp/src/arrow/util/bit_block_counter.h
##########
@@ -87,6 +87,147 @@ struct BitBlockOrNot<bool> {
   static bool Call(bool left, bool right) { return left || !right; }
 };
 
+// Three Arguments
+template <typename T>
+struct BitBlockAndAnd {
+  static T Call(T left, T mid, T right) { return left & mid & right; }
+};
+
+template <>
+struct BitBlockAndAnd<bool> {
+  static bool Call(bool left, bool mid, bool right) { return left && mid && right; }

Review comment:
       There is no need to specialize all these bitwise operations for `bool` as the language guarantees that for boolean types: `(bool)(a & b) == a && b`. Note the explicit cast which corresponds to the return type `T` in these functions. This simplification also applies to the existing binary bitwise operations, we can resolve this ARROW-15220.




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

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

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



[GitHub] [arrow] bkmgit commented on pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
bkmgit commented on pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#issuecomment-1003037121


   For 1 there would be separate functions
   - less(a,b)
   - less_equal(a,b)
   - between(val,a,b,"inclusive_both")
   - between(val,a,b,"inclusive_left")
   
   For 2b) Compare would include Between type functions as well. So you could have
   - compare("compare_less",a,b)
   - compare("between_inclusive_both",val,a,b)
   or using options
   - compare("compare",a,b,"less")
   - compare("between",val,a,b,"inclusive_both")
   


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

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

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



[GitHub] [arrow] bkmgit commented on a change in pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
bkmgit commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r782760664



##########
File path: python/pyarrow/tests/test_compute.py
##########
@@ -1262,6 +1263,300 @@ def test_filter_null_type():
     assert len(table.filter(mask).column(0)) == 5
 
 
+@pytest.mark.parametrize("ty", ["inclusive"])
+def test_between_array_array_array(ty):
+    BetweenOptions = partial(pc.BetweenOptions)
+
+    val = pa.array([1, 1, 4, 3, 2, 6])
+    arr1 = pa.array([1, 1, 3, 4, None, 5])
+    arr2 = pa.array([1, 2, 4, None, 4, 7])
+
+    inclusive_and_expected = {
+        "both": [True, True, True, None, None, True],
+        "left": [False, True, False, None, None, True],
+        "right": [False, False, True, None, None, True],
+        "neither": [False, False, False, None, None, True],
+    }
+
+    for inclusive, expected in inclusive_and_expected.items():
+        options = BetweenOptions(inclusive=inclusive)
+        result = pc.between(val, arr1, arr2, options=options)
+        np.testing.assert_array_equal(result, pa.array(expected))

Review comment:
       Thanks




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

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

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



[GitHub] [arrow] bkmgit commented on a change in pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
bkmgit commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r782790559



##########
File path: cpp/src/arrow/compute/kernels/scalar_compare_benchmark.cc
##########
@@ -77,5 +77,44 @@ BENCHMARK(GreaterArrayScalarInt64)->Apply(RegressionSetArgs);
 BENCHMARK(GreaterArrayArrayString)->Apply(RegressionSetArgs);
 BENCHMARK(GreaterArrayScalarString)->Apply(RegressionSetArgs);
 
+template <typename Type>
+static void BetweenScalarArrayScalar(benchmark::State& state) {
+  RegressionArgs args(state, /*size_is_bytes=*/false);
+  auto ty = TypeTraits<Type>::type_singleton();
+  auto rand = random::RandomArrayGenerator(kSeed);
+  auto array = rand.ArrayOf(ty, args.size, args.null_proportion);
+  auto scalar_left = *rand.ArrayOf(ty, 1, 0)->GetScalar(0);
+  auto scalar_right = *rand.ArrayOf(ty, 1, 0)->GetScalar(0);
+  for (auto _ : state) {
+    ABORT_NOT_OK(
+        CallFunction("between_less_equal_less_equal", {array, scalar_left, scalar_right})
+            .status());
+  }
+}
+
+template <typename Type>
+static void BetweenArrayArrayArray(benchmark::State& state) {
+  RegressionArgs args(state, /*size_is_bytes=*/false);
+  auto ty = TypeTraits<Type>::type_singleton();
+  auto rand = random::RandomArrayGenerator(kSeed);
+  auto lhs = rand.ArrayOf(ty, args.size, args.null_proportion);
+  auto mid = rand.ArrayOf(ty, args.size, args.null_proportion);
+  auto rhs = rand.ArrayOf(ty, args.size, args.null_proportion);
+  for (auto _ : state) {
+    ABORT_NOT_OK(CallFunction("between_less_equal_less_equal", {mid, lhs, rhs}).status());
+  }
+}
+
+// static void BetweenArrayArrayArrayInt64(benchmark::State& state) {
+//   BetweenArrayArrayArray<Int64Type>(state);
+// }
+//
+// static void BetweenScalarArrayScalarInt64(benchmark::State& state) {
+//   BetweenScalarArrayScalar<Int64Type>(state);
+// }
+
+// BENCHMARK(BetweenArrayArrayArrayInt64)->Apply(RegressionSetArgs);
+// BENCHMARK(BetweenScalarArrayScalarInt64)->Apply(RegressionSetArgs);

Review comment:
       Benchmarks added.




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

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

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



[GitHub] [arrow] bkmgit commented on a change in pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
bkmgit commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r782792995



##########
File path: cpp/src/arrow/compute/api_scalar.h
##########
@@ -316,6 +316,21 @@ struct ARROW_EXPORT CompareOptions {
   enum CompareOperator op;
 };
 
+enum class BetweenMode : int8_t {
+  LESS_EQUAL_LESS_EQUAL,
+  LESS_EQUAL_LESS,
+  LESS_LESS_EQUAL,
+  LESS_LESS,
+};

Review comment:
       @lidavidm Thanks for the feedback. Inclusive is in global space so that it can also be used in NotBetween - follow up issue https://issues.apache.org/jira/browse/ARROW-15223 , unless NotBetween should be implemented as an option of Between.




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

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

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



[GitHub] [arrow] bkmgit commented on a change in pull request #11882: ARROW-9843: [C++][Python] Implement Between ternary kernel and Python bindings

Posted by GitBox <gi...@apache.org>.
bkmgit commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r787607671



##########
File path: cpp/src/arrow/compute/kernels/scalar_compare_test.cc
##########
@@ -1865,5 +1867,683 @@ TEST(TestMaxElementWiseMinElementWise, CommonTemporal) {
               ResultWith(ScalarFromJSON(date64(), "86400000")));
 }
 
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const Datum& val, const Datum& lhs,
+                            const Datum& rhs, const Datum& expected) {
+  ASSERT_OK_AND_ASSIGN(Datum result, Between(val, lhs, rhs, options, nullptr));
+  AssertArraysEqual(*expected.make_array(), *result.make_array(),
+                    /*verbose=*/true);
+}
+
+template <typename ArrowType>

Review comment:
       Thanks. Removed.




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

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

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



[GitHub] [arrow] bkmgit commented on a change in pull request #11882: ARROW-9843: [C++][Python] Implement Between ternary kernel and Python bindings

Posted by GitBox <gi...@apache.org>.
bkmgit commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r787760105



##########
File path: cpp/src/arrow/compute/kernels/scalar_compare_test.cc
##########
@@ -1865,5 +1867,683 @@ TEST(TestMaxElementWiseMinElementWise, CommonTemporal) {
               ResultWith(ScalarFromJSON(date64(), "86400000")));
 }
 
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const Datum& val, const Datum& lhs,
+                            const Datum& rhs, const Datum& expected) {
+  ASSERT_OK_AND_ASSIGN(Datum result, Between(val, lhs, rhs, options, nullptr));
+  AssertArraysEqual(*expected.make_array(), *result.make_array(),
+                    /*verbose=*/true);
+}
+
+template <typename ArrowType>
+void ValidateBetween(BetweenOptions options, const Datum& val, const Datum& lhs,
+                     const Datum& rhs) {
+  CompareOperator lhs_val;
+  CompareOperator val_rhs;
+  BetweenOptions::Inclusive include_endpoints = options.inclusive;
+
+  if (include_endpoints == BetweenOptions::Inclusive::NEITHER) {

Review comment:
       Moved option loop here. 




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

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

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



[GitHub] [arrow] bkmgit commented on a change in pull request #11882: ARROW-9843: [C++][Python] Implement Between ternary kernel and Python bindings

Posted by GitBox <gi...@apache.org>.
bkmgit commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r789123726



##########
File path: cpp/src/arrow/compute/api_scalar.h
##########
@@ -1350,5 +1371,24 @@ ARROW_EXPORT Result<Datum> AssumeTimezone(const Datum& values,
                                           AssumeTimezoneOptions options,
                                           ExecContext* ctx = NULLPTR);
 
+/// \brief Between compares each element in `values`
+/// with `left` as a lower bound and 'right' as an upper bound
+///
+/// \param[in] values input to compare between left and right
+/// \param[in] left used as the lower bound for comparison
+/// \param[in] right used as the upper bound for comparison
+/// \param[in] options for bounds, default is inclusive of both, other
+///  endpoints, other choices are left (exclude left endpoint), right
+///  (exclude right endpoint) and both (exclude both endpoints), optional
+/// \param[in] ctx the function execution context, optional
+///
+/// \return the resulting datum

Review comment:
       Updated to 8.0.0




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

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

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



[GitHub] [arrow] bkmgit commented on a change in pull request #11882: ARROW-9843: [C++][Python] Implement Between ternary kernel and Python bindings

Posted by GitBox <gi...@apache.org>.
bkmgit commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r787035464



##########
File path: cpp/src/arrow/compute/kernels/scalar_compare_test.cc
##########
@@ -1865,5 +1867,683 @@ TEST(TestMaxElementWiseMinElementWise, CommonTemporal) {
               ResultWith(ScalarFromJSON(date64(), "86400000")));
 }
 
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const Datum& val, const Datum& lhs,
+                            const Datum& rhs, const Datum& expected) {
+  ASSERT_OK_AND_ASSIGN(Datum result, Between(val, lhs, rhs, options, nullptr));
+  AssertArraysEqual(*expected.make_array(), *result.make_array(),
+                    /*verbose=*/true);
+}
+
+template <typename ArrowType>
+void ValidateBetween(BetweenOptions options, const Datum& val, const Datum& lhs,
+                     const Datum& rhs) {
+  CompareOperator lhs_val;
+  CompareOperator val_rhs;
+  BetweenOptions::Inclusive include_endpoints = options.inclusive;
+
+  if (include_endpoints == BetweenOptions::Inclusive::NEITHER) {
+    lhs_val = LESS;
+    val_rhs = LESS;
+  } else if (include_endpoints == BetweenOptions::Inclusive::LEFT) {
+    lhs_val = LESS_EQUAL;
+    val_rhs = LESS;
+  } else if (include_endpoints == BetweenOptions::Inclusive::RIGHT) {
+    lhs_val = LESS;
+    val_rhs = LESS_EQUAL;
+  } else {
+    lhs_val = LESS_EQUAL;
+    val_rhs = LESS_EQUAL;
+  }
+
+  ASSERT_OK_AND_ASSIGN(Datum resultl,
+                       CallFunction(CompareOperatorToFunctionName(lhs_val), {lhs, val}));
+  ASSERT_OK_AND_ASSIGN(Datum resultr,
+                       CallFunction(CompareOperatorToFunctionName(val_rhs), {val, rhs}));
+  ASSERT_OK_AND_ASSIGN(Datum expected, CallFunction("and", {resultl, resultr}));
+
+  ValidateBetween<ArrowType>(options, val, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+class TestNumericBetweenKernel : public ::testing::Test {};
+
+TYPED_TEST_SUITE(TestNumericBetweenKernel, NumericArrowTypes);
+TYPED_TEST(TestNumericBetweenKernel, SimpleBetweenArrayScalarScalar) {
+  using ScalarType = typename TypeTraits<TypeParam>::ScalarType;
+  using CType = typename TypeTraits<TypeParam>::CType;
+
+  Datum zero(std::make_shared<ScalarType>(CType(0)));
+  Datum four(std::make_shared<ScalarType>(CType(4)));
+  Datum null(std::make_shared<ScalarType>());
+  BetweenOptions InclusiveBoth(BetweenOptions::Inclusive::BOTH);
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"), zero,
+      four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      zero, four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,1,2,2]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[1,1,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,3,4,5]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[1,1,1,1,1,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,1,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null,0,1,1]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), null, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, null,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  BetweenOptions InclusiveLeft(BetweenOptions::Inclusive::LEFT);
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"), zero,
+      four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      zero, four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,1,2,2]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[1,1,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,3,4,5]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[1,1,1,1,0,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,0,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null,0,1,1]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), null, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, null,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  BetweenOptions InclusiveRight(BetweenOptions::Inclusive::RIGHT);
+  ValidateBetween<TypeParam>(
+      InclusiveRight, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"), zero,
+      four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      zero, four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,1,2,2]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,0,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,3,4,5]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,1,1,1,1,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,1,1,1,1,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null,0,1,1]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null,0,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), null, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, null,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  BetweenOptions InclusiveNeither(BetweenOptions::Inclusive::NEITHER);
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      zero, four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      zero, four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,1,2,2]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,0,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,3,4,5]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,1,1,1,0,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,0,1,1,1,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null,0,1,1]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null,0,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), null, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, null,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+}
+
+TYPED_TEST(TestNumericBetweenKernel, SimpleBetweenScalarArrayArray) {
+  using ScalarType = typename TypeTraits<TypeParam>::ScalarType;
+  using CType = typename TypeTraits<TypeParam>::CType;
+
+  Datum one(std::make_shared<ScalarType>(CType(1)));
+
+  BetweenOptions InclusiveBoth(BetweenOptions::Inclusive::BOTH);
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, one, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,true,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,true,null,false,false]"));
+  BetweenOptions InclusiveLeft(BetweenOptions::Inclusive::LEFT);
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, one, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,true,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,true,null,false,false]"));
+  BetweenOptions InclusiveRight(BetweenOptions::Inclusive::RIGHT);
+  ValidateBetween<TypeParam>(
+      InclusiveRight, one, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,false,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,false,null,false,false]"));
+  BetweenOptions InclusiveNeither(BetweenOptions::Inclusive::NEITHER);
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, one, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,false,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,false,null,false,false]"));
+}
+
+TYPED_TEST(TestNumericBetweenKernel, SimpleBetweenArrayArrayArray) {
+  BetweenOptions InclusiveBoth(BetweenOptions::Inclusive::BOTH);
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[1,1,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,true,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,true,null,false,false]"));
+  BetweenOptions InclusiveLeft(BetweenOptions::Inclusive::LEFT);
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[1,1,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,false,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,false,null,false,false]"));
+  BetweenOptions InclusiveRight(BetweenOptions::Inclusive::RIGHT);
+  ValidateBetween<TypeParam>(
+      InclusiveRight, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[1,1,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,true,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,true,null,false,false]"));
+  BetweenOptions InclusiveNeither(BetweenOptions::Inclusive::NEITHER);
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[1,1,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,false,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,false,null,false,false]"));
+}

Review comment:
       No, will add the other variants, they were tested in the Python code.




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

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

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



[GitHub] [arrow] bkmgit commented on a change in pull request #11882: ARROW-9843: [C++][Python] Implement Between ternary kernel and Python bindings

Posted by GitBox <gi...@apache.org>.
bkmgit commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r789010333



##########
File path: cpp/src/arrow/compute/kernels/scalar_compare_test.cc
##########
@@ -1865,5 +1867,683 @@ TEST(TestMaxElementWiseMinElementWise, CommonTemporal) {
               ResultWith(ScalarFromJSON(date64(), "86400000")));
 }
 
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const Datum& val, const Datum& lhs,
+                            const Datum& rhs, const Datum& expected) {
+  ASSERT_OK_AND_ASSIGN(Datum result, Between(val, lhs, rhs, options, nullptr));
+  AssertArraysEqual(*expected.make_array(), *result.make_array(),
+                    /*verbose=*/true);
+}
+
+template <typename ArrowType>
+void ValidateBetween(BetweenOptions options, const Datum& val, const Datum& lhs,
+                     const Datum& rhs) {
+  CompareOperator lhs_val;
+  CompareOperator val_rhs;
+  BetweenOptions::Inclusive include_endpoints = options.inclusive;
+
+  if (include_endpoints == BetweenOptions::Inclusive::NEITHER) {
+    lhs_val = LESS;
+    val_rhs = LESS;
+  } else if (include_endpoints == BetweenOptions::Inclusive::LEFT) {
+    lhs_val = LESS_EQUAL;
+    val_rhs = LESS;
+  } else if (include_endpoints == BetweenOptions::Inclusive::RIGHT) {
+    lhs_val = LESS;
+    val_rhs = LESS_EQUAL;
+  } else {
+    lhs_val = LESS_EQUAL;
+    val_rhs = LESS_EQUAL;
+  }
+
+  ASSERT_OK_AND_ASSIGN(Datum resultl,
+                       CallFunction(CompareOperatorToFunctionName(lhs_val), {lhs, val}));
+  ASSERT_OK_AND_ASSIGN(Datum resultr,
+                       CallFunction(CompareOperatorToFunctionName(val_rhs), {val, rhs}));
+  ASSERT_OK_AND_ASSIGN(Datum expected, CallFunction("and", {resultl, resultr}));
+
+  ValidateBetween<ArrowType>(options, val, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+class TestNumericBetweenKernel : public ::testing::Test {};
+
+TYPED_TEST_SUITE(TestNumericBetweenKernel, NumericArrowTypes);
+TYPED_TEST(TestNumericBetweenKernel, SimpleBetweenArrayScalarScalar) {
+  using ScalarType = typename TypeTraits<TypeParam>::ScalarType;
+  using CType = typename TypeTraits<TypeParam>::CType;
+
+  Datum zero(std::make_shared<ScalarType>(CType(0)));
+  Datum four(std::make_shared<ScalarType>(CType(4)));
+  Datum null(std::make_shared<ScalarType>());
+  BetweenOptions InclusiveBoth(BetweenOptions::Inclusive::BOTH);
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"), zero,
+      four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      zero, four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,1,2,2]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[1,1,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,3,4,5]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[1,1,1,1,1,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,1,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null,0,1,1]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), null, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, null,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  BetweenOptions InclusiveLeft(BetweenOptions::Inclusive::LEFT);
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"), zero,
+      four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      zero, four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,1,2,2]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[1,1,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,3,4,5]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[1,1,1,1,0,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,0,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null,0,1,1]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), null, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, null,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  BetweenOptions InclusiveRight(BetweenOptions::Inclusive::RIGHT);
+  ValidateBetween<TypeParam>(
+      InclusiveRight, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"), zero,
+      four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      zero, four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,1,2,2]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,0,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,3,4,5]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,1,1,1,1,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,1,1,1,1,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null,0,1,1]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null,0,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), null, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, null,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  BetweenOptions InclusiveNeither(BetweenOptions::Inclusive::NEITHER);
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      zero, four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      zero, four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,1,2,2]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,0,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,3,4,5]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,1,1,1,0,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,0,1,1,1,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null,0,1,1]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null,0,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), null, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, null,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+}
+
+TYPED_TEST(TestNumericBetweenKernel, SimpleBetweenScalarArrayArray) {
+  using ScalarType = typename TypeTraits<TypeParam>::ScalarType;
+  using CType = typename TypeTraits<TypeParam>::CType;
+
+  Datum one(std::make_shared<ScalarType>(CType(1)));
+
+  BetweenOptions InclusiveBoth(BetweenOptions::Inclusive::BOTH);
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, one, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,true,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,true,null,false,false]"));
+  BetweenOptions InclusiveLeft(BetweenOptions::Inclusive::LEFT);
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, one, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,true,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,true,null,false,false]"));
+  BetweenOptions InclusiveRight(BetweenOptions::Inclusive::RIGHT);
+  ValidateBetween<TypeParam>(
+      InclusiveRight, one, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,false,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,false,null,false,false]"));
+  BetweenOptions InclusiveNeither(BetweenOptions::Inclusive::NEITHER);
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, one, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,false,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,false,null,false,false]"));
+}
+
+TYPED_TEST(TestNumericBetweenKernel, SimpleBetweenArrayArrayArray) {
+  BetweenOptions InclusiveBoth(BetweenOptions::Inclusive::BOTH);
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[1,1,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,true,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,true,null,false,false]"));
+  BetweenOptions InclusiveLeft(BetweenOptions::Inclusive::LEFT);
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[1,1,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,false,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,false,null,false,false]"));
+  BetweenOptions InclusiveRight(BetweenOptions::Inclusive::RIGHT);
+  ValidateBetween<TypeParam>(
+      InclusiveRight, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[1,1,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,true,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,true,null,false,false]"));
+  BetweenOptions InclusiveNeither(BetweenOptions::Inclusive::NEITHER);
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[1,1,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,false,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,false,null,false,false]"));
+}

Review comment:
       scalar-scalar-scalar does not work in the current implementation for between nor does scalar-scalar work for the other comparison functions. A possible fix is to check if all data elements are scalars, and if so convert one of the scalars to an array with a single entry. Other suggestions welcome.




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

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

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



[GitHub] [arrow] bkmgit commented on a change in pull request #11882: ARROW-9843: [C++][Python] Implement Between ternary kernel and Python bindings

Posted by GitBox <gi...@apache.org>.
bkmgit commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r787391420



##########
File path: python/pyarrow/tests/test_compute.py
##########
@@ -1262,6 +1263,300 @@ def test_filter_null_type():
     assert len(table.filter(mask).column(0)) == 5
 
 
+@pytest.mark.parametrize("ty", ["inclusive"])
+def test_between_array_array_array(ty):
+    BetweenOptions = partial(pc.BetweenOptions)
+
+    val = pa.array([1, 1, 4, 3, 2, 6])
+    arr1 = pa.array([1, 1, 3, 4, None, 5])
+    arr2 = pa.array([1, 2, 4, None, 4, 7])
+
+    inclusive_and_expected = {
+        "both": [True, True, True, None, None, True],
+        "left": [False, True, False, None, None, True],
+        "right": [False, False, True, None, None, True],
+        "neither": [False, False, False, None, None, True],
+    }
+
+    for inclusive, expected in inclusive_and_expected.items():
+        options = BetweenOptions(inclusive=inclusive)
+        result = pc.between(val, arr1, arr2, options=options)
+        np.testing.assert_array_equal(result, pa.array(expected))

Review comment:
       Just one set of Python tests is used to check that the interface works.




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

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

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



[GitHub] [arrow] edponce commented on a change in pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
edponce commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r783115242



##########
File path: cpp/src/arrow/compute/api_scalar.h
##########
@@ -316,6 +316,21 @@ struct ARROW_EXPORT CompareOptions {
   enum CompareOperator op;
 };
 
+enum class BetweenMode : int8_t {
+  LESS_EQUAL_LESS_EQUAL,
+  LESS_EQUAL_LESS,
+  LESS_LESS_EQUAL,
+  LESS_LESS,
+};

Review comment:
       @lidavidm is correct. There should only be one:
   ```c++
   class BetweenOptions {
    public:
     enum Inclusive {
       BOTH,
       // ...
     };
     // ...
   };
   ```
   in global space, so that it can be used  by both `between` and `not_between`. Something important to note is that[ there is a discussion in JIRA on whether "not_between" should exist](https://issues.apache.org/jira/browse/ARROW-15223).




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

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

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



[GitHub] [arrow] edponce commented on a change in pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
edponce commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r784341390



##########
File path: cpp/src/arrow/compute/kernels/scalar_compare_test.cc
##########
@@ -1865,5 +1861,636 @@ TEST(TestMaxElementWiseMinElementWise, CommonTemporal) {
               ResultWith(ScalarFromJSON(date64(), "86400000")));
 }
 
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const Datum& val, const Datum& lhs,
+                            const Datum& rhs, const Datum& expected) {
+  ASSERT_OK_AND_ASSIGN(Datum result, Between(val, lhs, rhs, options, nullptr));
+  AssertArraysEqual(*expected.make_array(), *result.make_array(),
+                    /*verbose=*/true);
+}
+
+template <typename ArrowType>
+void ValidateBetween(BetweenOptions options, const Datum& val, const Datum& lhs,
+                     const Datum& rhs) {
+  CompareOperator lhs_val;
+  CompareOperator val_rhs;
+  BetweenOptions::Inclusive include_endpoints = options.inclusive;
+
+  if (include_endpoints == BetweenOptions::Inclusive::NEITHER) {
+    lhs_val = LESS;
+    val_rhs = LESS;
+  } else if (include_endpoints == BetweenOptions::Inclusive::LEFT) {
+    lhs_val = LESS_EQUAL;
+    val_rhs = LESS;
+  } else if (include_endpoints == BetweenOptions::Inclusive::RIGHT) {
+    lhs_val = LESS;
+    val_rhs = LESS_EQUAL;
+  } else {
+    lhs_val = LESS_EQUAL;
+    val_rhs = LESS_EQUAL;
+  }
+
+  ASSERT_OK_AND_ASSIGN(Datum resultl,
+                       CallFunction(CompareOperatorToFunctionName(lhs_val), {lhs, val}));
+  ASSERT_OK_AND_ASSIGN(Datum resultr,
+                       CallFunction(CompareOperatorToFunctionName(val_rhs), {val, rhs}));
+  ASSERT_OK_AND_ASSIGN(Datum expected, CallFunction("and", {resultl, resultr}));
+
+  ValidateBetween<ArrowType>(options, val, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+class TestNumericBetweenKernel : public ::testing::Test {};
+
+TYPED_TEST_SUITE(TestNumericBetweenKernel, NumericArrowTypes);
+TYPED_TEST(TestNumericBetweenKernel, SimpleBetweenArrayScalarScalar) {
+  using ScalarType = typename TypeTraits<TypeParam>::ScalarType;
+  using CType = typename TypeTraits<TypeParam>::CType;
+
+  Datum zero(std::make_shared<ScalarType>(CType(0)));
+  Datum four(std::make_shared<ScalarType>(CType(4)));
+  Datum null(std::make_shared<ScalarType>());
+  BetweenOptions InclusiveBoth(BetweenOptions::Inclusive::BOTH);
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"), zero,
+      four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      zero, four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,1,2,2]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[1,1,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,3,4,5]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[1,1,1,1,1,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,1,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null,0,1,1]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), null, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, null,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  BetweenOptions InclusiveLeft(BetweenOptions::Inclusive::LEFT);
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"), zero,
+      four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      zero, four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,1,2,2]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[1,1,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,3,4,5]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[1,1,1,1,0,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,0,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null,0,1,1]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), null, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, null,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  BetweenOptions InclusiveRight(BetweenOptions::Inclusive::RIGHT);
+  ValidateBetween<TypeParam>(
+      InclusiveRight, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"), zero,
+      four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      zero, four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,1,2,2]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,0,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,3,4,5]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,1,1,1,1,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,1,1,1,1,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null,0,1,1]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null,0,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), null, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, null,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  BetweenOptions InclusiveNeither(BetweenOptions::Inclusive::NEITHER);
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      zero, four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      zero, four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,1,2,2]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,0,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,3,4,5]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,1,1,1,0,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,0,1,1,1,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null,0,1,1]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null,0,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), null, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, null,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+}
+
+TYPED_TEST(TestNumericBetweenKernel, SimpleBetweenScalarArrayArray) {
+  using ScalarType = typename TypeTraits<TypeParam>::ScalarType;
+  using CType = typename TypeTraits<TypeParam>::CType;
+
+  Datum one(std::make_shared<ScalarType>(CType(1)));
+
+  BetweenOptions InclusiveBoth(BetweenOptions::Inclusive::BOTH);
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, one, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,true,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,true,null,false,false]"));
+  BetweenOptions InclusiveLeft(BetweenOptions::Inclusive::LEFT);
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, one, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,true,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,true,null,false,false]"));
+  BetweenOptions InclusiveRight(BetweenOptions::Inclusive::RIGHT);
+  ValidateBetween<TypeParam>(
+      InclusiveRight, one, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,false,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,false,null,false,false]"));
+  BetweenOptions InclusiveNeither(BetweenOptions::Inclusive::NEITHER);
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, one, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,false,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,false,null,false,false]"));
+}
+
+TYPED_TEST(TestNumericBetweenKernel, SimpleBetweenArrayArrayArray) {
+  BetweenOptions InclusiveBoth(BetweenOptions::Inclusive::BOTH);
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[1,1,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,true,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,true,null,false,false]"));
+  BetweenOptions InclusiveLeft(BetweenOptions::Inclusive::LEFT);
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[1,1,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,false,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,false,null,false,false]"));
+  BetweenOptions InclusiveRight(BetweenOptions::Inclusive::RIGHT);
+  ValidateBetween<TypeParam>(
+      InclusiveRight, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[1,1,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,true,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,true,null,false,false]"));
+  BetweenOptions InclusiveNeither(BetweenOptions::Inclusive::NEITHER);
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[1,1,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,false,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,false,null,false,false]"));
+}
+
+template <typename Type>
+struct BetweenRandomNumeric {
+  static void Test(const std::shared_ptr<DataType>& type) {
+    using ScalarType = typename TypeTraits<Type>::ScalarType;
+    using CType = typename TypeTraits<Type>::CType;
+    auto rand = random::RandomArrayGenerator(0x5416447);
+    const int64_t length = 1000;
+    for (auto null_probability : {0.0, 0.01, 0.1, 0.25, 0.5, 1.0}) {
+      for (auto inclusive :
+           {BetweenOptions::Inclusive::BOTH, BetweenOptions::Inclusive::LEFT,
+            BetweenOptions::Inclusive::RIGHT, BetweenOptions::Inclusive::NEITHER}) {
+        auto data1 =
+            rand.Numeric<typename Type::PhysicalType>(length, 0, 100, null_probability);
+        auto data2 =
+            rand.Numeric<typename Type::PhysicalType>(length, 0, 100, null_probability);
+        auto data3 =
+            rand.Numeric<typename Type::PhysicalType>(length, 0, 100, null_probability);
+
+        // Create view of data as the type (e.g. timestamp)
+        auto array1 = Datum(*data1->View(type));
+        auto array2 = Datum(*data2->View(type));
+        auto array3 = Datum(*data3->View(type));
+        auto scalar1 = Datum(std::make_shared<ScalarType>(CType(10), type));
+        auto scalar2 = Datum(std::make_shared<ScalarType>(CType(30), type));
+        auto scalar3 = Datum(std::make_shared<ScalarType>(CType(50), type));
+        auto options = BetweenOptions(inclusive);
+        ValidateBetween<Type>(options, array1, scalar2, scalar3);
+        ValidateBetween<Type>(options, array1, array2, scalar3);
+        ValidateBetween<Type>(options, array1, array2, array3);
+        ValidateBetween<Type>(options, array1, scalar2, scalar3);
+        ValidateBetween<Type>(options, scalar1, array2, array3);
+        ValidateBetween<Type>(options, scalar1, array2, scalar3);
+        ValidateBetween<Type>(options, scalar1, scalar2, array3);
+        ValidateBetween<Type>(options, array1, scalar2, array3);
+      }
+    }
+  }
+};
+
+TEST(TestNumericBetweenKernel, BetweenPrimitiveRandomTests) {
+  TestRandomPrimitiveCTypes<BetweenRandomNumeric>();
+}
+
+class TestStringBetweenKernel : public ::testing::Test {};
+
+TEST(TestStringBetweenKernel, RandomBetween) {
+  using ScalarType = typename TypeTraits<StringType>::ScalarType;
+
+  auto rand = random::RandomArrayGenerator(0x5416447);
+  for (size_t i = 3; i < 10; i++) {
+    for (auto null_probability : {0.0, 0.01, 0.1, 0.25, 0.5, 1.0}) {
+      for (auto inclusive :
+           {BetweenOptions::Inclusive::BOTH, BetweenOptions::Inclusive::LEFT,
+            BetweenOptions::Inclusive::RIGHT, BetweenOptions::Inclusive::NEITHER}) {
+        const int64_t length = static_cast<int64_t>(1ULL << i);
+        auto array1 = Datum(rand.String(length, 0, 16, null_probability));
+        auto array2 = Datum(rand.String(length, 0, 16, null_probability));
+        auto array3 = Datum(rand.String(length, 0, 16, null_probability));
+        auto scalar1 = Datum(std::make_shared<ScalarType>("fupi"));
+        auto scalar2 = Datum(std::make_shared<ScalarType>("tupu"));
+        auto scalar3 = Datum(std::make_shared<ScalarType>("zito"));
+        auto options = BetweenOptions(inclusive);
+        ValidateBetween<StringType>(options, array1, scalar2, scalar3);
+        ValidateBetween<StringType>(options, scalar1, array2, scalar3);
+        ValidateBetween<StringType>(options, scalar1, scalar2, array3);
+        ValidateBetween<StringType>(options, scalar1, array2, array3);
+        ValidateBetween<StringType>(options, array1, scalar2, array3);
+        ValidateBetween<StringType>(options, array1, array2, scalar3);
+        ValidateBetween<StringType>(options, array1, array2, array3);
+      }
+    }
+  }
+}
+
+TEST(TestStringBetweenKernel, StringArrayScalarScalarTest) {
+  using ScalarType = typename TypeTraits<StringType>::ScalarType;
+  auto l = Datum(std::make_shared<ScalarType>("abc"));
+  auto r = Datum(std::make_shared<ScalarType>("zzz"));
+  BetweenOptions InclusiveOption(BetweenOptions::Inclusive::BOTH);
+  ValidateBetween<StringType>(
+      InclusiveOption, ArrayFromJSON(TypeTraits<StringType>::type_singleton(), "[]"), l,
+      r, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<StringType>(
+      InclusiveOption, ArrayFromJSON(TypeTraits<StringType>::type_singleton(), "[null]"),
+      l, r, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<StringType>(InclusiveOption,
+                              ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                            R"(["aaa", "aaaa", "ccc", "z"])"),
+                              l, r,
+                              ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                                            R"([false, false, true, true])"));
+  ValidateBetween<StringType>(InclusiveOption,
+                              ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                            R"(["a", "aaaa", "c", "z"])"),
+                              l, r,
+                              ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                                            R"([false, false, true, true])"));
+  ValidateBetween<StringType>(InclusiveOption,
+                              ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                            R"(["a", "aaaa", "fff", "zzzz"])"),
+                              l, r,
+                              ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                                            R"([false, false, true, false])"));
+  ValidateBetween<StringType>(InclusiveOption,
+                              ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                            R"(["abc", "baa", "fff", "zzz"])"),
+                              l, r,
+                              ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                                            R"([true, true, true, true])"));
+  ValidateBetween<StringType>(InclusiveOption,
+                              ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                            R"(["abd", null, null, "zzx"])"),
+                              l, r,
+                              ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                                            R"([true, null, null, true])"));
+}
+
+TEST(TestStringBetweenKernel, StringArrayArrayArrayTest) {
+  BetweenOptions InclusiveOption(BetweenOptions::Inclusive::BOTH);
+  ValidateBetween<StringType>(
+      InclusiveOption,
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                    R"(["david","hello","world"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["adam","hi","whirl"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                    R"(["robert","goeiemoreen","whirlwind"])"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[true, false, false]"));
+  ValidateBetween<StringType>(
+      InclusiveOption,
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["x","a","f"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["w","a","e"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["z","a","g"])"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[true, true, true]"));
+  ValidateBetween<StringType>(
+      InclusiveOption,
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                    R"(["block","bit","binary"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                    R"(["bit","nibble","ternary"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["word","d","xyz"])"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[true, false, false]"));
+  ValidateBetween<StringType>(
+      InclusiveOption,
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                    R"(["よしもと","の","ち"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["は","へ","あ"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["な","を","ち"])"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[false, false, true]"));
+  ValidateBetween<StringType>(
+      InclusiveOption,
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["A","ア","王"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["た","あ","歩"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["李","田",null])"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[false, true, null]"));
+  ValidateBetween<StringType>(
+      InclusiveOption,
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["Б",null,"Я"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["А","Ж","Щ"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["Д","Л","Ф"])"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[true, null, false]"));
+}
+
+TEST(TestTimestampsBetweenKernel, TimestampsArrayArrayArrayTest) {
+  const std::string arr_json = R"(["1970-01-01","2000-02-02","1900-02-28"])";
+  const std::string lhs_json = R"(["1970-01-01","2000-02-01","1900-02-28"])";
+  const std::string rhs_json = R"(["1970-01-02","2000-02-02","1900-02-28"])";
+  // Between Options
+  BetweenOptions both(BetweenOptions::Inclusive::BOTH);
+  BetweenOptions left(BetweenOptions::Inclusive::LEFT);
+  BetweenOptions right(BetweenOptions::Inclusive::RIGHT);
+  BetweenOptions neither(BetweenOptions::Inclusive::NEITHER);
+  // Same units should be fine
+  auto arr = ArrayFromJSON(timestamp(TimeUnit::SECOND), arr_json);
+  auto lhs = ArrayFromJSON(timestamp(TimeUnit::SECOND), lhs_json);
+  auto rhs = ArrayFromJSON(timestamp(TimeUnit::SECOND), rhs_json);
+  // BOTH
+  ValidateBetween<TimestampType>(both, arr, lhs, rhs);
+  // LEFT
+  ValidateBetween<TimestampType>(left, arr, lhs, rhs);
+  // RIGHT
+  ValidateBetween<TimestampType>(right, arr, lhs, rhs);
+  // NEITHER
+  ValidateBetween<TimestampType>(neither, arr, lhs, rhs);
+/*  // So are same time zones
+  arr = ArrayFromJSON(timestamp(TimeUnit::SECOND, "America/Chicago"), arr_json);
+  lhs = ArrayFromJSON(timestamp(TimeUnit::SECOND, "America/Chicago"), lhs_json);
+  rhs = ArrayFromJSON(timestamp(TimeUnit::SECOND, "America/Chicago"), rhs_json);
+  // BOTH
+  ValidateBetween<TimestampType>(both, arr, lhs, rhs);
+  // LEFT
+  ValidateBetween<TimestampType>(left, arr, lhs, rhs);
+  // RIGHT
+  ValidateBetween<TimestampType>(right, arr, lhs, rhs);
+  // NEITHER
+  ValidateBetween<TimestampType>(neither, arr, lhs, rhs);
+  // Different units should be fine
+  auto arr = ArrayFromJSON(timestamp(TimeUnit::SECOND), arr_json);
+  auto lhs = ArrayFromJSON(timestamp(TimeUnit::MILLI), lhs_json);
+  auto rhs = ArrayFromJSON(timestamp(TimeUnit::SECOND), rhs_json);
+  // BOTH

Review comment:
       The error here is because you are redeclaring `arr, lhs, rhs` which were already declared in the first case of this test. Simply remove the `auto` keyword from the above 3 statements and it should work correctly. I tested it locally. 😁 




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

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

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



[GitHub] [arrow] bkmgit commented on a change in pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
bkmgit commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r785343075



##########
File path: go/arrow/memory/Makefile
##########
@@ -56,11 +46,9 @@ _lib/memory_avx2.s: _lib/memory.c
 _lib/memory_sse4.s: _lib/memory.c
 	$(CC) -S $(C_FLAGS) $(ASM_FLAGS_SSE4) $^ -o $@ ; $(PERL_FIXUP_ROTATE) $@
 
-_lib/memory_neon.s: _lib/memory.c
-	$(CC) -S $(C_FLAGS_NEON) $^ -o $@ ; $(PERL_FIXUP_ROTATE) $@
-
 memory_avx2_amd64.s: _lib/memory_avx2.s

Review comment:
       Fixed poor rebase




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

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

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



[GitHub] [arrow] lidavidm commented on pull request #11882: ARROW-9843: [C++][Python] Implement Between ternary kernel and Python bindings

Posted by GitBox <gi...@apache.org>.
lidavidm commented on pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#issuecomment-1014555006


   @bkmgit as Eduardo mentioned, would you like to write a short description for the PR? (It's the first post at top.) This gets used in the commit message.


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

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

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



[GitHub] [arrow] bkmgit commented on pull request #11882: ARROW-9843: [C++][Python] Implement Between ternary kernel and Python bindings

Posted by GitBox <gi...@apache.org>.
bkmgit commented on pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#issuecomment-1014568870


   @lidavidm Hopefully ok now. Thanks for the explanation.


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

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

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



[GitHub] [arrow] bkmgit commented on a change in pull request #11882: ARROW-9843: [C++][Python] Implement Between ternary kernel and Python bindings

Posted by GitBox <gi...@apache.org>.
bkmgit commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r786079703



##########
File path: cpp/src/arrow/compute/kernels/scalar_compare.cc
##########
@@ -772,6 +1107,16 @@ const FunctionDoc less_equal_doc{
     ("A null on either side emits a null comparison result."),
     {"x", "y"}};
 
+const FunctionDoc between_doc{
+    "Check if values are in the given range, val between a and b",

Review comment:
       Ok.




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

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

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



[GitHub] [arrow] bkmgit commented on pull request #11882: ARROW-9843: [C++][Python] Implement Between ternary kernel and Python bindings

Posted by GitBox <gi...@apache.org>.
bkmgit commented on pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#issuecomment-1020947191






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

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

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



[GitHub] [arrow] bkmgit commented on a change in pull request #11882: ARROW-9843: [C++][Python] Implement Between ternary kernel and Python bindings

Posted by GitBox <gi...@apache.org>.
bkmgit commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r791361654



##########
File path: cpp/src/arrow/compute/kernels/scalar_compare_test.cc
##########
@@ -1850,5 +1851,154 @@ TEST(TestMaxElementWiseMinElementWise, CommonTemporal) {
               ResultWith(ScalarFromJSON(date64(), "86400000")));
 }
 
+template <typename ArrowType>
+static void ValidateBetween(const Datum& val, const Datum& lhs, const Datum& rhs,
+                            const Datum& expected) {
+  ASSERT_OK_AND_ASSIGN(Datum result, Between(val, lhs, rhs));
+  AssertArraysEqual(*expected.make_array(), *result.make_array(),
+                    /*verbose=*/true);
+}
+
+template <typename ArrowType>
+static void ValidateBetween(const char* value_str, const Datum& lhs, const Datum& rhs,
+                            const char* expected_str) {
+  auto value = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), value_str);
+  auto expected = ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), expected_str);
+  ValidateBetween<ArrowType>(value, lhs, rhs, expected);
+}
+
+template <>
+void ValidateBetween<StringType>(const char* value_str, const Datum& lhs,
+                                 const Datum& rhs, const char* expected_str) {
+  auto value = ArrayFromJSON(utf8(), value_str);
+  auto expected = ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), expected_str);
+  ValidateBetween<StringType>(value, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+class TestNumericBetweenKernel : public ::testing::Test {};
+
+TYPED_TEST_SUITE(TestNumericBetweenKernel, NumericArrowTypes);
+TYPED_TEST(TestNumericBetweenKernel, SimpleBetweenArrayScalarScalar) {
+  using ScalarType = typename TypeTraits<TypeParam>::ScalarType;
+  using CType = typename TypeTraits<TypeParam>::CType;
+
+  Datum zero(std::make_shared<ScalarType>(CType(0)));
+  Datum four(std::make_shared<ScalarType>(CType(4)));
+  ValidateBetween<TypeParam>("[]", zero, four, "[]");
+  ValidateBetween<TypeParam>("[null]", zero, four, "[null]");
+  ValidateBetween<TypeParam>("[0,0,1,1,2,2]", zero, four, "[0,0,1,1,1,1]");
+  ValidateBetween<TypeParam>("[0,1,2,3,4,5]", zero, four, "[0,1,1,1,0,0]");
+  ValidateBetween<TypeParam>("[5,4,3,2,1,0]", zero, four, "[0,0,1,1,1,0]");
+  ValidateBetween<TypeParam>("[null,0,1,1]", zero, four, "[null,0,1,1]");
+}
+
+TEST(TestSimpleBetweenKernel, SimpleStringTest) {

Review comment:
       Random tests are helpful, but some fixed tests which differentiate between < and <= are useful since random tests will not do this often. Also good to have some utf8 characters other than ASCII.

##########
File path: cpp/src/arrow/compute/kernels/scalar_compare_test.cc
##########
@@ -1850,5 +1851,154 @@ TEST(TestMaxElementWiseMinElementWise, CommonTemporal) {
               ResultWith(ScalarFromJSON(date64(), "86400000")));
 }
 
+template <typename ArrowType>
+static void ValidateBetween(const Datum& val, const Datum& lhs, const Datum& rhs,
+                            const Datum& expected) {
+  ASSERT_OK_AND_ASSIGN(Datum result, Between(val, lhs, rhs));
+  AssertArraysEqual(*expected.make_array(), *result.make_array(),
+                    /*verbose=*/true);
+}
+
+template <typename ArrowType>
+static void ValidateBetween(const char* value_str, const Datum& lhs, const Datum& rhs,
+                            const char* expected_str) {
+  auto value = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), value_str);
+  auto expected = ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), expected_str);
+  ValidateBetween<ArrowType>(value, lhs, rhs, expected);
+}
+
+template <>
+void ValidateBetween<StringType>(const char* value_str, const Datum& lhs,
+                                 const Datum& rhs, const char* expected_str) {
+  auto value = ArrayFromJSON(utf8(), value_str);
+  auto expected = ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), expected_str);
+  ValidateBetween<StringType>(value, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+class TestNumericBetweenKernel : public ::testing::Test {};
+
+TYPED_TEST_SUITE(TestNumericBetweenKernel, NumericArrowTypes);
+TYPED_TEST(TestNumericBetweenKernel, SimpleBetweenArrayScalarScalar) {
+  using ScalarType = typename TypeTraits<TypeParam>::ScalarType;
+  using CType = typename TypeTraits<TypeParam>::CType;
+
+  Datum zero(std::make_shared<ScalarType>(CType(0)));
+  Datum four(std::make_shared<ScalarType>(CType(4)));
+  ValidateBetween<TypeParam>("[]", zero, four, "[]");
+  ValidateBetween<TypeParam>("[null]", zero, four, "[null]");
+  ValidateBetween<TypeParam>("[0,0,1,1,2,2]", zero, four, "[0,0,1,1,1,1]");
+  ValidateBetween<TypeParam>("[0,1,2,3,4,5]", zero, four, "[0,1,1,1,0,0]");
+  ValidateBetween<TypeParam>("[5,4,3,2,1,0]", zero, four, "[0,0,1,1,1,0]");
+  ValidateBetween<TypeParam>("[null,0,1,1]", zero, four, "[null,0,1,1]");
+}
+
+TEST(TestSimpleBetweenKernel, SimpleStringTest) {
+  using ScalarType = typename TypeTraits<StringType>::ScalarType;
+  auto l = Datum(std::make_shared<ScalarType>("abc"));
+  auto r = Datum(std::make_shared<ScalarType>("zzz"));
+  ValidateBetween<StringType>("[]", l, r, "[]");
+  ValidateBetween<StringType>("[null]", l, r, "[null]");
+  ValidateBetween<StringType>(R"(["aaa", "aaaa", "ccc", "z"])", l, r,
+                              R"([false, false, true, true])");
+  ValidateBetween<StringType>(R"(["a", "aaaa", "c", "z"])", l, r,
+                              R"([false, false, true, true])");
+  ValidateBetween<StringType>(R"(["a", "aaaa", "fff", "zzzz"])", l, r,
+                              R"([false, false, true, false])");
+  ValidateBetween<StringType>(R"(["abd", null, null, "zzx"])", l, r,
+                              R"([true, null, null, true])");
+}
+
+TEST(TestSimpleBetweenKernel, SimpleTimestampTest) {
+  using ScalarType = typename TypeTraits<TimestampType>::ScalarType;
+  auto checkTimestampArray = [](std::shared_ptr<DataType> type, const char* input_str,
+                                const Datum& lhs, const Datum& rhs,
+                                const char* expected_str) {
+    auto value = ArrayFromJSON(type, input_str);
+    auto expected = ArrayFromJSON(boolean(), expected_str);
+    ValidateBetween<TimestampType>(value, lhs, rhs, expected);
+  };
+  auto unit = TimeUnit::SECOND;
+  auto l = Datum(std::make_shared<ScalarType>(923184000, timestamp(unit)));
+  auto r = Datum(std::make_shared<ScalarType>(1602032602, timestamp(unit)));
+  checkTimestampArray(timestamp(unit), "[]", l, r, "[]");
+  checkTimestampArray(timestamp(unit), "[null]", l, r, "[null]");
+  checkTimestampArray(timestamp(unit), R"(["1970-01-01","2000-02-29","1900-02-28"])", l,
+                      r, "[false,true,false]");
+  checkTimestampArray(timestamp(unit), R"(["1970-01-01","2000-02-29","2004-02-28"])", l,
+                      r, "[false,true,true]");
+  checkTimestampArray(timestamp(unit), R"(["2018-01-01","1999-04-04","1900-02-28"])", l,
+                      r, "[true,false,false]");
+}
+
+TYPED_TEST(TestNumericBetweenKernel, SimpleBetweenArrayArrayArray) {
+  ValidateBetween<TypeParam>(
+      "[]", ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"), "[]");
+  ValidateBetween<TypeParam>(
+      "[null]", ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"), "[null]");
+  ValidateBetween<TypeParam>(
+      "[1,1,2,2,2]",
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      "[true,true,false,false,false]");
+  ValidateBetween<TypeParam>(
+      "[1,1,2,2,2,2]",
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,2,5,5]"),
+      "[true,true,false,null,false,false]");
+}
+
+TEST(TestSimpleBetweenKernel, StringArrayArrayArrayTest) {
+  ValidateBetween<StringType>(
+      R"(["david","hello","world"])",
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["adam","hi","whirl"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                    R"(["robert","goeiemoreen","whirlwind"])"),
+      "[true, false, false]");
+  ValidateBetween<StringType>(
+      R"(["x","a","f"])",
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["w","a","e"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["z","a","g"])"),
+      "[true, false, true]");
+  ValidateBetween<StringType>(
+      R"(["block","bit","binary"])",
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                    R"(["bit","nibble","ternary"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["word","d","xyz"])"),
+      "[true, false, false]");
+  ValidateBetween<StringType>(R"(["Ayumi","アユミ","王梦莹"])",
+                              ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                            R"(["たなか","あゆみ","歩美"])"),
+                              ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                            R"(["李平之","田中","たなか"])"),
+                              "[false, true, false]");
+}
+
+TEST(TestSimpleBetweenKernel, TimestampArrayArrayArrayTest) {

Review comment:
       Added different time units.




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

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

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



[GitHub] [arrow] edponce commented on a change in pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
edponce commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r782572785



##########
File path: cpp/src/arrow/compute/api_scalar.h
##########
@@ -316,6 +316,21 @@ struct ARROW_EXPORT CompareOptions {
   enum CompareOperator op;
 };
 
+enum class BetweenMode : int8_t {
+  LESS_EQUAL_LESS_EQUAL,
+  LESS_EQUAL_LESS,
+  LESS_LESS_EQUAL,
+  LESS_LESS,
+};

Review comment:
       Yes, I agree with @lidavidm on how the BetweenOptions should be organized.
   Sorry for suggesting a similar approach as in `RoundOptions` but that was a special case where different but similar kernels with function-specific options needed to share an enum.

##########
File path: cpp/src/arrow/compute/kernels/scalar_compare_benchmark.cc
##########
@@ -77,5 +77,44 @@ BENCHMARK(GreaterArrayScalarInt64)->Apply(RegressionSetArgs);
 BENCHMARK(GreaterArrayArrayString)->Apply(RegressionSetArgs);
 BENCHMARK(GreaterArrayScalarString)->Apply(RegressionSetArgs);
 
+template <typename Type>
+static void BetweenScalarArrayScalar(benchmark::State& state) {
+  RegressionArgs args(state, /*size_is_bytes=*/false);
+  auto ty = TypeTraits<Type>::type_singleton();
+  auto rand = random::RandomArrayGenerator(kSeed);
+  auto array = rand.ArrayOf(ty, args.size, args.null_proportion);
+  auto scalar_left = *rand.ArrayOf(ty, 1, 0)->GetScalar(0);
+  auto scalar_right = *rand.ArrayOf(ty, 1, 0)->GetScalar(0);
+  for (auto _ : state) {
+    ABORT_NOT_OK(
+        CallFunction("between_less_equal_less_equal", {array, scalar_left, scalar_right})
+            .status());

Review comment:
       Function is not called "between"




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

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

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



[GitHub] [arrow] bkmgit commented on a change in pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
bkmgit commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r784195693



##########
File path: cpp/src/arrow/compute/kernels/scalar_compare_test.cc
##########
@@ -1865,5 +1861,636 @@ TEST(TestMaxElementWiseMinElementWise, CommonTemporal) {
               ResultWith(ScalarFromJSON(date64(), "86400000")));
 }
 
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const Datum& val, const Datum& lhs,
+                            const Datum& rhs, const Datum& expected) {
+  ASSERT_OK_AND_ASSIGN(Datum result, Between(val, lhs, rhs, options, nullptr));
+  AssertArraysEqual(*expected.make_array(), *result.make_array(),
+                    /*verbose=*/true);
+}
+
+template <typename ArrowType>
+void ValidateBetween(BetweenOptions options, const Datum& val, const Datum& lhs,
+                     const Datum& rhs) {
+  CompareOperator lhs_val;
+  CompareOperator val_rhs;
+  BetweenOptions::Inclusive include_endpoints = options.inclusive;
+
+  if (include_endpoints == BetweenOptions::Inclusive::NEITHER) {
+    lhs_val = LESS;
+    val_rhs = LESS;
+  } else if (include_endpoints == BetweenOptions::Inclusive::LEFT) {
+    lhs_val = LESS_EQUAL;
+    val_rhs = LESS;
+  } else if (include_endpoints == BetweenOptions::Inclusive::RIGHT) {
+    lhs_val = LESS;
+    val_rhs = LESS_EQUAL;
+  } else {
+    lhs_val = LESS_EQUAL;
+    val_rhs = LESS_EQUAL;
+  }
+
+  ASSERT_OK_AND_ASSIGN(Datum resultl,
+                       CallFunction(CompareOperatorToFunctionName(lhs_val), {lhs, val}));
+  ASSERT_OK_AND_ASSIGN(Datum resultr,
+                       CallFunction(CompareOperatorToFunctionName(val_rhs), {val, rhs}));
+  ASSERT_OK_AND_ASSIGN(Datum expected, CallFunction("and", {resultl, resultr}));
+
+  ValidateBetween<ArrowType>(options, val, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+class TestNumericBetweenKernel : public ::testing::Test {};
+
+TYPED_TEST_SUITE(TestNumericBetweenKernel, NumericArrowTypes);
+TYPED_TEST(TestNumericBetweenKernel, SimpleBetweenArrayScalarScalar) {
+  using ScalarType = typename TypeTraits<TypeParam>::ScalarType;
+  using CType = typename TypeTraits<TypeParam>::CType;
+
+  Datum zero(std::make_shared<ScalarType>(CType(0)));
+  Datum four(std::make_shared<ScalarType>(CType(4)));
+  Datum null(std::make_shared<ScalarType>());
+  BetweenOptions InclusiveBoth(BetweenOptions::Inclusive::BOTH);
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"), zero,
+      four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      zero, four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,1,2,2]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[1,1,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,3,4,5]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[1,1,1,1,1,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,1,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null,0,1,1]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), null, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, null,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  BetweenOptions InclusiveLeft(BetweenOptions::Inclusive::LEFT);
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"), zero,
+      four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      zero, four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,1,2,2]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[1,1,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,3,4,5]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[1,1,1,1,0,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,0,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null,0,1,1]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), null, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, null,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  BetweenOptions InclusiveRight(BetweenOptions::Inclusive::RIGHT);
+  ValidateBetween<TypeParam>(
+      InclusiveRight, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"), zero,
+      four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      zero, four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,1,2,2]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,0,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,3,4,5]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,1,1,1,1,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,1,1,1,1,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null,0,1,1]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null,0,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), null, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, null,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  BetweenOptions InclusiveNeither(BetweenOptions::Inclusive::NEITHER);
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      zero, four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      zero, four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,1,2,2]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,0,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,3,4,5]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,1,1,1,0,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,0,1,1,1,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null,0,1,1]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null,0,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), null, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, null,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+}
+
+TYPED_TEST(TestNumericBetweenKernel, SimpleBetweenScalarArrayArray) {
+  using ScalarType = typename TypeTraits<TypeParam>::ScalarType;
+  using CType = typename TypeTraits<TypeParam>::CType;
+
+  Datum one(std::make_shared<ScalarType>(CType(1)));
+
+  BetweenOptions InclusiveBoth(BetweenOptions::Inclusive::BOTH);
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, one, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,true,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,true,null,false,false]"));
+  BetweenOptions InclusiveLeft(BetweenOptions::Inclusive::LEFT);
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, one, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,true,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,true,null,false,false]"));
+  BetweenOptions InclusiveRight(BetweenOptions::Inclusive::RIGHT);
+  ValidateBetween<TypeParam>(
+      InclusiveRight, one, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,false,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,false,null,false,false]"));
+  BetweenOptions InclusiveNeither(BetweenOptions::Inclusive::NEITHER);
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, one, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,false,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,false,null,false,false]"));
+}
+
+TYPED_TEST(TestNumericBetweenKernel, SimpleBetweenArrayArrayArray) {
+  BetweenOptions InclusiveBoth(BetweenOptions::Inclusive::BOTH);
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[1,1,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,true,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,true,null,false,false]"));
+  BetweenOptions InclusiveLeft(BetweenOptions::Inclusive::LEFT);
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[1,1,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,false,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,false,null,false,false]"));
+  BetweenOptions InclusiveRight(BetweenOptions::Inclusive::RIGHT);
+  ValidateBetween<TypeParam>(
+      InclusiveRight, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[1,1,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,true,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,true,null,false,false]"));
+  BetweenOptions InclusiveNeither(BetweenOptions::Inclusive::NEITHER);
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[1,1,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,false,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,false,null,false,false]"));
+}
+
+template <typename Type>
+struct BetweenRandomNumeric {
+  static void Test(const std::shared_ptr<DataType>& type) {
+    using ScalarType = typename TypeTraits<Type>::ScalarType;
+    using CType = typename TypeTraits<Type>::CType;
+    auto rand = random::RandomArrayGenerator(0x5416447);
+    const int64_t length = 1000;
+    for (auto null_probability : {0.0, 0.01, 0.1, 0.25, 0.5, 1.0}) {
+      for (auto inclusive :
+           {BetweenOptions::Inclusive::BOTH, BetweenOptions::Inclusive::LEFT,
+            BetweenOptions::Inclusive::RIGHT, BetweenOptions::Inclusive::NEITHER}) {
+        auto data1 =
+            rand.Numeric<typename Type::PhysicalType>(length, 0, 100, null_probability);
+        auto data2 =
+            rand.Numeric<typename Type::PhysicalType>(length, 0, 100, null_probability);
+        auto data3 =
+            rand.Numeric<typename Type::PhysicalType>(length, 0, 100, null_probability);
+
+        // Create view of data as the type (e.g. timestamp)
+        auto array1 = Datum(*data1->View(type));
+        auto array2 = Datum(*data2->View(type));
+        auto array3 = Datum(*data3->View(type));
+        auto scalar1 = Datum(std::make_shared<ScalarType>(CType(10), type));
+        auto scalar2 = Datum(std::make_shared<ScalarType>(CType(30), type));
+        auto scalar3 = Datum(std::make_shared<ScalarType>(CType(50), type));
+        auto options = BetweenOptions(inclusive);
+        ValidateBetween<Type>(options, array1, scalar2, scalar3);
+        ValidateBetween<Type>(options, array1, array2, scalar3);
+        ValidateBetween<Type>(options, array1, array2, array3);
+        ValidateBetween<Type>(options, array1, scalar2, scalar3);
+        ValidateBetween<Type>(options, scalar1, array2, array3);
+        ValidateBetween<Type>(options, scalar1, array2, scalar3);
+        ValidateBetween<Type>(options, scalar1, scalar2, array3);
+        ValidateBetween<Type>(options, array1, scalar2, array3);
+      }
+    }
+  }
+};
+
+TEST(TestNumericBetweenKernel, BetweenPrimitiveRandomTests) {
+  TestRandomPrimitiveCTypes<BetweenRandomNumeric>();
+}
+
+class TestStringBetweenKernel : public ::testing::Test {};
+
+TEST(TestStringBetweenKernel, RandomBetween) {
+  using ScalarType = typename TypeTraits<StringType>::ScalarType;
+
+  auto rand = random::RandomArrayGenerator(0x5416447);
+  for (size_t i = 3; i < 10; i++) {
+    for (auto null_probability : {0.0, 0.01, 0.1, 0.25, 0.5, 1.0}) {
+      for (auto inclusive :
+           {BetweenOptions::Inclusive::BOTH, BetweenOptions::Inclusive::LEFT,
+            BetweenOptions::Inclusive::RIGHT, BetweenOptions::Inclusive::NEITHER}) {
+        const int64_t length = static_cast<int64_t>(1ULL << i);
+        auto array1 = Datum(rand.String(length, 0, 16, null_probability));
+        auto array2 = Datum(rand.String(length, 0, 16, null_probability));
+        auto array3 = Datum(rand.String(length, 0, 16, null_probability));
+        auto scalar1 = Datum(std::make_shared<ScalarType>("fupi"));
+        auto scalar2 = Datum(std::make_shared<ScalarType>("tupu"));
+        auto scalar3 = Datum(std::make_shared<ScalarType>("zito"));
+        auto options = BetweenOptions(inclusive);
+        ValidateBetween<StringType>(options, array1, scalar2, scalar3);
+        ValidateBetween<StringType>(options, scalar1, array2, scalar3);
+        ValidateBetween<StringType>(options, scalar1, scalar2, array3);
+        ValidateBetween<StringType>(options, scalar1, array2, array3);
+        ValidateBetween<StringType>(options, array1, scalar2, array3);
+        ValidateBetween<StringType>(options, array1, array2, scalar3);
+        ValidateBetween<StringType>(options, array1, array2, array3);
+      }
+    }
+  }
+}
+
+TEST(TestStringBetweenKernel, StringArrayScalarScalarTest) {
+  using ScalarType = typename TypeTraits<StringType>::ScalarType;
+  auto l = Datum(std::make_shared<ScalarType>("abc"));
+  auto r = Datum(std::make_shared<ScalarType>("zzz"));
+  BetweenOptions InclusiveOption(BetweenOptions::Inclusive::BOTH);
+  ValidateBetween<StringType>(
+      InclusiveOption, ArrayFromJSON(TypeTraits<StringType>::type_singleton(), "[]"), l,
+      r, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<StringType>(
+      InclusiveOption, ArrayFromJSON(TypeTraits<StringType>::type_singleton(), "[null]"),
+      l, r, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<StringType>(InclusiveOption,
+                              ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                            R"(["aaa", "aaaa", "ccc", "z"])"),
+                              l, r,
+                              ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                                            R"([false, false, true, true])"));
+  ValidateBetween<StringType>(InclusiveOption,
+                              ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                            R"(["a", "aaaa", "c", "z"])"),
+                              l, r,
+                              ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                                            R"([false, false, true, true])"));
+  ValidateBetween<StringType>(InclusiveOption,
+                              ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                            R"(["a", "aaaa", "fff", "zzzz"])"),
+                              l, r,
+                              ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                                            R"([false, false, true, false])"));
+  ValidateBetween<StringType>(InclusiveOption,
+                              ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                            R"(["abc", "baa", "fff", "zzz"])"),
+                              l, r,
+                              ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                                            R"([true, true, true, true])"));
+  ValidateBetween<StringType>(InclusiveOption,
+                              ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                            R"(["abd", null, null, "zzx"])"),
+                              l, r,
+                              ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                                            R"([true, null, null, true])"));
+}
+
+TEST(TestStringBetweenKernel, StringArrayArrayArrayTest) {
+  BetweenOptions InclusiveOption(BetweenOptions::Inclusive::BOTH);
+  ValidateBetween<StringType>(
+      InclusiveOption,
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                    R"(["david","hello","world"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["adam","hi","whirl"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                    R"(["robert","goeiemoreen","whirlwind"])"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[true, false, false]"));
+  ValidateBetween<StringType>(
+      InclusiveOption,
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["x","a","f"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["w","a","e"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["z","a","g"])"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[true, true, true]"));
+  ValidateBetween<StringType>(
+      InclusiveOption,
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                    R"(["block","bit","binary"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                    R"(["bit","nibble","ternary"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["word","d","xyz"])"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[true, false, false]"));
+  ValidateBetween<StringType>(
+      InclusiveOption,
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                    R"(["よしもと","の","ち"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["は","へ","あ"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["な","を","ち"])"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[false, false, true]"));
+  ValidateBetween<StringType>(
+      InclusiveOption,
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["A","ア","王"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["た","あ","歩"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["李","田",null])"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[false, true, null]"));
+  ValidateBetween<StringType>(
+      InclusiveOption,
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["Б",null,"Я"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["А","Ж","Щ"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["Д","Л","Ф"])"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[true, null, false]"));
+}
+
+TEST(TestTimestampsBetweenKernel, TimestampsArrayArrayArrayTest) {
+  const std::string arr_json = R"(["1970-01-01","2000-02-02","1900-02-28"])";
+  const std::string lhs_json = R"(["1970-01-01","2000-02-01","1900-02-28"])";
+  const std::string rhs_json = R"(["1970-01-02","2000-02-02","1900-02-28"])";
+  // Between Options
+  BetweenOptions both(BetweenOptions::Inclusive::BOTH);
+  BetweenOptions left(BetweenOptions::Inclusive::LEFT);
+  BetweenOptions right(BetweenOptions::Inclusive::RIGHT);
+  BetweenOptions neither(BetweenOptions::Inclusive::NEITHER);
+  // Same units should be fine
+  auto arr = ArrayFromJSON(timestamp(TimeUnit::SECOND), arr_json);
+  auto lhs = ArrayFromJSON(timestamp(TimeUnit::SECOND), lhs_json);
+  auto rhs = ArrayFromJSON(timestamp(TimeUnit::SECOND), rhs_json);
+  // BOTH
+  ValidateBetween<TimestampType>(both, arr, lhs, rhs);
+  // LEFT
+  ValidateBetween<TimestampType>(left, arr, lhs, rhs);
+  // RIGHT
+  ValidateBetween<TimestampType>(right, arr, lhs, rhs);
+  // NEITHER
+  ValidateBetween<TimestampType>(neither, arr, lhs, rhs);
+/*  // So are same time zones
+  arr = ArrayFromJSON(timestamp(TimeUnit::SECOND, "America/Chicago"), arr_json);
+  lhs = ArrayFromJSON(timestamp(TimeUnit::SECOND, "America/Chicago"), lhs_json);
+  rhs = ArrayFromJSON(timestamp(TimeUnit::SECOND, "America/Chicago"), rhs_json);
+  // BOTH
+  ValidateBetween<TimestampType>(both, arr, lhs, rhs);

Review comment:
       They seem to give errors at the moment. Trying to determine why.




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

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

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



[GitHub] [arrow] bkmgit commented on a change in pull request #11882: ARROW-9843: [C++][Python] Implement Between ternary kernel and Python bindings

Posted by GitBox <gi...@apache.org>.
bkmgit commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r793731000



##########
File path: cpp/src/arrow/compute/kernels/scalar_compare.cc
##########
@@ -156,39 +212,49 @@ struct Maximum {
   }
 };
 
+// Check if timestamp timezones are comparable (either all are empty or none is).
+Status CheckCompareTimestamps(const ExecBatch& batch) {
+  if (batch.num_values() > 0) {
+    int invalid_states = 0;
+    for (int i = 0; i < batch.num_values(); i++) {
+      const auto& tsi = checked_cast<const TimestampType&>(*batch[i].type());
+      invalid_states += int(tsi.timezone().empty());
+    }
+    if (invalid_states * (invalid_states - batch.num_values()) != 0) {
+      return Status::TypeError(
+          "Cannot compare timestamp with timezone to timestamp without timezone");
+    }

Review comment:
       Ok.




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

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

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



[GitHub] [arrow] bkmgit commented on a change in pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
bkmgit commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r770238553



##########
File path: cpp/src/arrow/compute/kernels/scalar_compare.cc
##########
@@ -786,6 +878,10 @@ const FunctionDoc max_element_wise_doc{
     {"*args"},
     "ElementWiseAggregateOptions"};
 
+const FunctionDoc between_doc{"Check if values are in a range x <= y <= z",

Review comment:
       Added options to allow selection of 
   - `x <= y <= z`
   - `x < y <= z`
   - `x <= y < z`
   - `x < y < z`




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

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

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



[GitHub] [arrow] bkmgit commented on a change in pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
bkmgit commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r784537983



##########
File path: cpp/src/arrow/compute/kernels/scalar_compare_test.cc
##########
@@ -1865,5 +1861,636 @@ TEST(TestMaxElementWiseMinElementWise, CommonTemporal) {
               ResultWith(ScalarFromJSON(date64(), "86400000")));
 }
 
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const Datum& val, const Datum& lhs,
+                            const Datum& rhs, const Datum& expected) {
+  ASSERT_OK_AND_ASSIGN(Datum result, Between(val, lhs, rhs, options, nullptr));
+  AssertArraysEqual(*expected.make_array(), *result.make_array(),
+                    /*verbose=*/true);
+}
+
+template <typename ArrowType>
+void ValidateBetween(BetweenOptions options, const Datum& val, const Datum& lhs,
+                     const Datum& rhs) {
+  CompareOperator lhs_val;
+  CompareOperator val_rhs;
+  BetweenOptions::Inclusive include_endpoints = options.inclusive;
+
+  if (include_endpoints == BetweenOptions::Inclusive::NEITHER) {
+    lhs_val = LESS;
+    val_rhs = LESS;
+  } else if (include_endpoints == BetweenOptions::Inclusive::LEFT) {
+    lhs_val = LESS_EQUAL;
+    val_rhs = LESS;
+  } else if (include_endpoints == BetweenOptions::Inclusive::RIGHT) {
+    lhs_val = LESS;
+    val_rhs = LESS_EQUAL;
+  } else {
+    lhs_val = LESS_EQUAL;
+    val_rhs = LESS_EQUAL;
+  }
+
+  ASSERT_OK_AND_ASSIGN(Datum resultl,
+                       CallFunction(CompareOperatorToFunctionName(lhs_val), {lhs, val}));
+  ASSERT_OK_AND_ASSIGN(Datum resultr,
+                       CallFunction(CompareOperatorToFunctionName(val_rhs), {val, rhs}));
+  ASSERT_OK_AND_ASSIGN(Datum expected, CallFunction("and", {resultl, resultr}));
+
+  ValidateBetween<ArrowType>(options, val, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+class TestNumericBetweenKernel : public ::testing::Test {};
+
+TYPED_TEST_SUITE(TestNumericBetweenKernel, NumericArrowTypes);
+TYPED_TEST(TestNumericBetweenKernel, SimpleBetweenArrayScalarScalar) {
+  using ScalarType = typename TypeTraits<TypeParam>::ScalarType;
+  using CType = typename TypeTraits<TypeParam>::CType;
+
+  Datum zero(std::make_shared<ScalarType>(CType(0)));
+  Datum four(std::make_shared<ScalarType>(CType(4)));
+  Datum null(std::make_shared<ScalarType>());
+  BetweenOptions InclusiveBoth(BetweenOptions::Inclusive::BOTH);
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"), zero,
+      four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      zero, four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,1,2,2]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[1,1,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,3,4,5]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[1,1,1,1,1,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,1,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null,0,1,1]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), null, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, null,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  BetweenOptions InclusiveLeft(BetweenOptions::Inclusive::LEFT);
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"), zero,
+      four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      zero, four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,1,2,2]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[1,1,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,3,4,5]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[1,1,1,1,0,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,0,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null,0,1,1]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), null, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, null,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  BetweenOptions InclusiveRight(BetweenOptions::Inclusive::RIGHT);
+  ValidateBetween<TypeParam>(
+      InclusiveRight, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"), zero,
+      four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      zero, four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,1,2,2]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,0,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,3,4,5]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,1,1,1,1,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,1,1,1,1,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null,0,1,1]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null,0,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), null, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, null,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  BetweenOptions InclusiveNeither(BetweenOptions::Inclusive::NEITHER);
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      zero, four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      zero, four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,1,2,2]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,0,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,3,4,5]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,1,1,1,0,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,0,1,1,1,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null,0,1,1]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null,0,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), null, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, null,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+}
+
+TYPED_TEST(TestNumericBetweenKernel, SimpleBetweenScalarArrayArray) {
+  using ScalarType = typename TypeTraits<TypeParam>::ScalarType;
+  using CType = typename TypeTraits<TypeParam>::CType;
+
+  Datum one(std::make_shared<ScalarType>(CType(1)));
+
+  BetweenOptions InclusiveBoth(BetweenOptions::Inclusive::BOTH);
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, one, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,true,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,true,null,false,false]"));
+  BetweenOptions InclusiveLeft(BetweenOptions::Inclusive::LEFT);
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, one, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,true,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,true,null,false,false]"));
+  BetweenOptions InclusiveRight(BetweenOptions::Inclusive::RIGHT);
+  ValidateBetween<TypeParam>(
+      InclusiveRight, one, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,false,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,false,null,false,false]"));
+  BetweenOptions InclusiveNeither(BetweenOptions::Inclusive::NEITHER);
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, one, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,false,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,false,null,false,false]"));
+}
+
+TYPED_TEST(TestNumericBetweenKernel, SimpleBetweenArrayArrayArray) {
+  BetweenOptions InclusiveBoth(BetweenOptions::Inclusive::BOTH);
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[1,1,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,true,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,true,null,false,false]"));
+  BetweenOptions InclusiveLeft(BetweenOptions::Inclusive::LEFT);
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[1,1,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,false,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,false,null,false,false]"));
+  BetweenOptions InclusiveRight(BetweenOptions::Inclusive::RIGHT);
+  ValidateBetween<TypeParam>(
+      InclusiveRight, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[1,1,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,true,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,true,null,false,false]"));
+  BetweenOptions InclusiveNeither(BetweenOptions::Inclusive::NEITHER);
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[1,1,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,false,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,false,null,false,false]"));
+}
+
+template <typename Type>
+struct BetweenRandomNumeric {
+  static void Test(const std::shared_ptr<DataType>& type) {
+    using ScalarType = typename TypeTraits<Type>::ScalarType;
+    using CType = typename TypeTraits<Type>::CType;
+    auto rand = random::RandomArrayGenerator(0x5416447);
+    const int64_t length = 1000;
+    for (auto null_probability : {0.0, 0.01, 0.1, 0.25, 0.5, 1.0}) {
+      for (auto inclusive :
+           {BetweenOptions::Inclusive::BOTH, BetweenOptions::Inclusive::LEFT,
+            BetweenOptions::Inclusive::RIGHT, BetweenOptions::Inclusive::NEITHER}) {
+        auto data1 =
+            rand.Numeric<typename Type::PhysicalType>(length, 0, 100, null_probability);
+        auto data2 =
+            rand.Numeric<typename Type::PhysicalType>(length, 0, 100, null_probability);
+        auto data3 =
+            rand.Numeric<typename Type::PhysicalType>(length, 0, 100, null_probability);
+
+        // Create view of data as the type (e.g. timestamp)
+        auto array1 = Datum(*data1->View(type));
+        auto array2 = Datum(*data2->View(type));
+        auto array3 = Datum(*data3->View(type));
+        auto scalar1 = Datum(std::make_shared<ScalarType>(CType(10), type));
+        auto scalar2 = Datum(std::make_shared<ScalarType>(CType(30), type));
+        auto scalar3 = Datum(std::make_shared<ScalarType>(CType(50), type));
+        auto options = BetweenOptions(inclusive);
+        ValidateBetween<Type>(options, array1, scalar2, scalar3);
+        ValidateBetween<Type>(options, array1, array2, scalar3);
+        ValidateBetween<Type>(options, array1, array2, array3);
+        ValidateBetween<Type>(options, array1, scalar2, scalar3);
+        ValidateBetween<Type>(options, scalar1, array2, array3);
+        ValidateBetween<Type>(options, scalar1, array2, scalar3);
+        ValidateBetween<Type>(options, scalar1, scalar2, array3);
+        ValidateBetween<Type>(options, array1, scalar2, array3);
+      }
+    }
+  }
+};
+
+TEST(TestNumericBetweenKernel, BetweenPrimitiveRandomTests) {
+  TestRandomPrimitiveCTypes<BetweenRandomNumeric>();
+}
+
+class TestStringBetweenKernel : public ::testing::Test {};
+
+TEST(TestStringBetweenKernel, RandomBetween) {
+  using ScalarType = typename TypeTraits<StringType>::ScalarType;
+
+  auto rand = random::RandomArrayGenerator(0x5416447);
+  for (size_t i = 3; i < 10; i++) {
+    for (auto null_probability : {0.0, 0.01, 0.1, 0.25, 0.5, 1.0}) {
+      for (auto inclusive :
+           {BetweenOptions::Inclusive::BOTH, BetweenOptions::Inclusive::LEFT,
+            BetweenOptions::Inclusive::RIGHT, BetweenOptions::Inclusive::NEITHER}) {
+        const int64_t length = static_cast<int64_t>(1ULL << i);
+        auto array1 = Datum(rand.String(length, 0, 16, null_probability));
+        auto array2 = Datum(rand.String(length, 0, 16, null_probability));
+        auto array3 = Datum(rand.String(length, 0, 16, null_probability));
+        auto scalar1 = Datum(std::make_shared<ScalarType>("fupi"));
+        auto scalar2 = Datum(std::make_shared<ScalarType>("tupu"));
+        auto scalar3 = Datum(std::make_shared<ScalarType>("zito"));
+        auto options = BetweenOptions(inclusive);
+        ValidateBetween<StringType>(options, array1, scalar2, scalar3);
+        ValidateBetween<StringType>(options, scalar1, array2, scalar3);
+        ValidateBetween<StringType>(options, scalar1, scalar2, array3);
+        ValidateBetween<StringType>(options, scalar1, array2, array3);
+        ValidateBetween<StringType>(options, array1, scalar2, array3);
+        ValidateBetween<StringType>(options, array1, array2, scalar3);
+        ValidateBetween<StringType>(options, array1, array2, array3);
+      }
+    }
+  }
+}
+
+TEST(TestStringBetweenKernel, StringArrayScalarScalarTest) {
+  using ScalarType = typename TypeTraits<StringType>::ScalarType;
+  auto l = Datum(std::make_shared<ScalarType>("abc"));
+  auto r = Datum(std::make_shared<ScalarType>("zzz"));
+  BetweenOptions InclusiveOption(BetweenOptions::Inclusive::BOTH);
+  ValidateBetween<StringType>(
+      InclusiveOption, ArrayFromJSON(TypeTraits<StringType>::type_singleton(), "[]"), l,
+      r, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<StringType>(
+      InclusiveOption, ArrayFromJSON(TypeTraits<StringType>::type_singleton(), "[null]"),
+      l, r, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<StringType>(InclusiveOption,
+                              ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                            R"(["aaa", "aaaa", "ccc", "z"])"),
+                              l, r,
+                              ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                                            R"([false, false, true, true])"));
+  ValidateBetween<StringType>(InclusiveOption,
+                              ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                            R"(["a", "aaaa", "c", "z"])"),
+                              l, r,
+                              ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                                            R"([false, false, true, true])"));
+  ValidateBetween<StringType>(InclusiveOption,
+                              ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                            R"(["a", "aaaa", "fff", "zzzz"])"),
+                              l, r,
+                              ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                                            R"([false, false, true, false])"));
+  ValidateBetween<StringType>(InclusiveOption,
+                              ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                            R"(["abc", "baa", "fff", "zzz"])"),
+                              l, r,
+                              ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                                            R"([true, true, true, true])"));
+  ValidateBetween<StringType>(InclusiveOption,
+                              ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                            R"(["abd", null, null, "zzx"])"),
+                              l, r,
+                              ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                                            R"([true, null, null, true])"));
+}
+
+TEST(TestStringBetweenKernel, StringArrayArrayArrayTest) {
+  BetweenOptions InclusiveOption(BetweenOptions::Inclusive::BOTH);
+  ValidateBetween<StringType>(
+      InclusiveOption,
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                    R"(["david","hello","world"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["adam","hi","whirl"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                    R"(["robert","goeiemoreen","whirlwind"])"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[true, false, false]"));
+  ValidateBetween<StringType>(
+      InclusiveOption,
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["x","a","f"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["w","a","e"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["z","a","g"])"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[true, true, true]"));
+  ValidateBetween<StringType>(
+      InclusiveOption,
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                    R"(["block","bit","binary"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                    R"(["bit","nibble","ternary"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["word","d","xyz"])"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[true, false, false]"));
+  ValidateBetween<StringType>(
+      InclusiveOption,
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                    R"(["よしもと","の","ち"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["は","へ","あ"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["な","を","ち"])"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[false, false, true]"));
+  ValidateBetween<StringType>(
+      InclusiveOption,
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["A","ア","王"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["た","あ","歩"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["李","田",null])"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[false, true, null]"));
+  ValidateBetween<StringType>(
+      InclusiveOption,
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["Б",null,"Я"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["А","Ж","Щ"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["Д","Л","Ф"])"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[true, null, false]"));
+}
+
+TEST(TestTimestampsBetweenKernel, TimestampsArrayArrayArrayTest) {
+  const std::string arr_json = R"(["1970-01-01","2000-02-02","1900-02-28"])";
+  const std::string lhs_json = R"(["1970-01-01","2000-02-01","1900-02-28"])";
+  const std::string rhs_json = R"(["1970-01-02","2000-02-02","1900-02-28"])";
+  // Between Options
+  BetweenOptions both(BetweenOptions::Inclusive::BOTH);
+  BetweenOptions left(BetweenOptions::Inclusive::LEFT);
+  BetweenOptions right(BetweenOptions::Inclusive::RIGHT);
+  BetweenOptions neither(BetweenOptions::Inclusive::NEITHER);
+  // Same units should be fine
+  auto arr = ArrayFromJSON(timestamp(TimeUnit::SECOND), arr_json);
+  auto lhs = ArrayFromJSON(timestamp(TimeUnit::SECOND), lhs_json);
+  auto rhs = ArrayFromJSON(timestamp(TimeUnit::SECOND), rhs_json);
+  // BOTH
+  ValidateBetween<TimestampType>(both, arr, lhs, rhs);
+  // LEFT
+  ValidateBetween<TimestampType>(left, arr, lhs, rhs);
+  // RIGHT
+  ValidateBetween<TimestampType>(right, arr, lhs, rhs);
+  // NEITHER
+  ValidateBetween<TimestampType>(neither, arr, lhs, rhs);

Review comment:
       Removed.




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

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

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



[GitHub] [arrow] bkmgit commented on a change in pull request #11882: ARROW-9843: [C++][Python] Implement Between ternary kernel and Python bindings

Posted by GitBox <gi...@apache.org>.
bkmgit commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r789004670



##########
File path: cpp/src/arrow/compute/kernels/scalar_compare_test.cc
##########
@@ -1865,5 +1867,683 @@ TEST(TestMaxElementWiseMinElementWise, CommonTemporal) {
               ResultWith(ScalarFromJSON(date64(), "86400000")));
 }
 
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const Datum& val, const Datum& lhs,
+                            const Datum& rhs, const Datum& expected) {
+  ASSERT_OK_AND_ASSIGN(Datum result, Between(val, lhs, rhs, options, nullptr));
+  AssertArraysEqual(*expected.make_array(), *result.make_array(),
+                    /*verbose=*/true);
+}
+
+template <typename ArrowType>
+void ValidateBetween(BetweenOptions options, const Datum& val, const Datum& lhs,
+                     const Datum& rhs) {
+  CompareOperator lhs_val;
+  CompareOperator val_rhs;
+  BetweenOptions::Inclusive include_endpoints = options.inclusive;
+
+  if (include_endpoints == BetweenOptions::Inclusive::NEITHER) {

Review comment:
       Added `ARROW_SCOPED_TRACE`




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

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

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



[GitHub] [arrow] bkmgit commented on a change in pull request #11882: ARROW-9843: [C++][Python] Implement Between ternary kernel and Python bindings

Posted by GitBox <gi...@apache.org>.
bkmgit commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r786216564



##########
File path: cpp/src/arrow/compute/kernels/scalar_compare.cc
##########
@@ -156,39 +212,52 @@ struct Maximum {
   }
 };
 
+// Check if timestamp timezones are comparable (either all are empty or none is).
+Status CheckCompareTimestamps(const ExecBatch& batch) {
+  if (batch.num_values() > 0) {
+    for (int i = 0; i < batch.num_values() - 1; ++i) {
+      const auto& tsi = checked_cast<const TimestampType&>(*batch[i].type());
+      for (int j = i + 1; j < batch.num_values(); ++j) {

Review comment:
       The comparison is done using [Boolean algebra](https://en.wikipedia.org/wiki/Boolean_algebra#Basic_operations) which has the truth table:
   
   |  x  | y  |  x^y |
   |---  |---  |---    |
   |  1  | 1  |   1   |
   |  1  | 0  |   0   |
   |  0  | 1  |   0   |
   |  0  |  0 |   1   |
   
   Thus ( 1^1) ^ 1 = 0, ( 1 ^ 0 ) ^ 1 = 0 and ( 1 ^ 1 ) ^ 0 = 1 which are not the result we want.  To keep the code general and extensible, a double loop that allows for pairwise comparisons will give the correct results. 
   
   Another option could be to cast to integers, sum the results and check if the sum is either 0 or batch.numvalues()




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

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

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



[GitHub] [arrow] bkmgit commented on pull request #11882: ARROW-9843: [C++][Python] Implement Between ternary kernel and Python bindings

Posted by GitBox <gi...@apache.org>.
bkmgit commented on pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#issuecomment-1017900746


   > Thanks for doing this. I'm afraid the tests would deserve a lot of cleanup and simplification. Also, the ternary bit-block-counting seems entirely unused, so probably shouldn't be here.
   > 
   > Can you take a look at the comments below?
   
   @pitrou Thanks for the feedback. Have cleaned up the tests and removed ternary bit block count.
   - To avoid templates, [created a small routine to return the type](https://github.com/apache/arrow/pull/11882#discussion_r789005846) Is there a better way to do this?
   - Still need to refactor [PhysicalNumeric](https://github.com/apache/arrow/pull/11882#discussion_r786034907), but let me know if any other changes are needed.


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

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

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



[GitHub] [arrow] bkmgit commented on a change in pull request #11882: ARROW-9843: [C++][Python] Implement Between ternary kernel and Python bindings

Posted by GitBox <gi...@apache.org>.
bkmgit commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r787607909



##########
File path: cpp/src/arrow/compute/kernels/scalar_compare_test.cc
##########
@@ -1865,5 +1867,683 @@ TEST(TestMaxElementWiseMinElementWise, CommonTemporal) {
               ResultWith(ScalarFromJSON(date64(), "86400000")));
 }
 
+template <typename ArrowType>

Review comment:
       Thanks. Removed.




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

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

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



[GitHub] [arrow] bkmgit commented on a change in pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
bkmgit commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r785336284



##########
File path: cpp/src/arrow/util/bit_block_counter.h
##########
@@ -46,7 +46,7 @@ inline uint64_t ShiftWord(uint64_t current, uint64_t next, int64_t shift) {
 }
 
 // These templates are here to help with unit tests
-
+// Two Arguments
 template <typename T>
 constexpr T BitNot(T x) {

Review comment:
       Ok Done.




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

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

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



[GitHub] [arrow] bkmgit commented on pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
bkmgit commented on pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#issuecomment-1013724118


   Made some errors on rebase.


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

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

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



[GitHub] [arrow] bkmgit commented on pull request #11882: ARROW-9843: [C++][Python] Implement Between ternary kernel and Python bindings

Posted by GitBox <gi...@apache.org>.
bkmgit commented on pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#issuecomment-1072055711


   Waiting for review from @pitrou 


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

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

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



[GitHub] [arrow] lidavidm commented on a change in pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
lidavidm commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r772363973



##########
File path: cpp/src/arrow/compute/api_scalar.h
##########
@@ -316,6 +316,19 @@ struct ARROW_EXPORT CompareOptions {
   enum CompareOperator op;
 };
 
+enum BetweenOperator : int8_t {
+  BETWEEN_LESS_EQUAL_LESS_EQUAL,
+  BETWEEN_LESS_EQUAL_LESS_THAN,
+  BETWEEN_LESS_THAN_LESS_EQUAL,
+  BETWEEN_LESS_THAN_LESS_THAN,
+};
+
+struct ARROW_EXPORT BetweenOptions {
+  explicit BetweenOptions(BetweenOperator op) : op(op) {}
+  BetweenOptions() : BetweenOptions(BetweenOperator::BETWEEN_LESS_EQUAL_LESS_EQUAL) {}
+  enum BetweenOperator op;

Review comment:
       1) Follow the pattern of other options and nest the enum in the options. (Otherwise, use an `enum class` to get namespacing.)
   2) The options should be a `class` not a `struct` and should inherit from `FunctionOptions` and set up all the other necessary reflection magic (needed to get ToString, Serialize, etc. for the options). 
   3) Would it make more sense to have an enum for open/closed interval and two options for the lower/upper bound?

##########
File path: cpp/src/arrow/compute/api_scalar.h
##########
@@ -1240,5 +1253,21 @@ ARROW_EXPORT Result<Datum> AssumeTimezone(const Datum& values,
                                           AssumeTimezoneOptions options,
                                           ExecContext* ctx = NULLPTR);
 
+/// \brief Between compares each element in `values`
+/// with `left` as a lower bound and 'right' as an upperbound
+///
+/// \param[in] values input to compare between left and right
+/// \param[in] left used as the lower bound for comparison
+/// \param[in] right used as the upper bound for comparison
+/// \param[in] ctx the function execution context, optional
+///
+/// \return the resulting datum
+///
+/// \note Bounds are not inclusive

Review comment:
       This note doesn't match the implementation.

##########
File path: cpp/src/arrow/util/bit_block_counter.h
##########
@@ -424,6 +565,240 @@ class ARROW_EXPORT OptionalBinaryBitBlockCounter {
   }
 };
 
+/// \brief A class that computes popcounts on the result of bitwise operations
+/// between three bitmaps, 64 bits at a time. A 64-bit word is loaded from each
+/// bitmap, then the popcount is computed on e.g. the bitwise-and of the three
+/// words.
+class ARROW_EXPORT TernaryBitBlockCounter {
+ public:
+  TernaryBitBlockCounter(const uint8_t* left_bitmap, int64_t left_offset,
+                         const uint8_t* mid_bitmap, int64_t mid_offset,
+                         const uint8_t* right_bitmap, int64_t right_offset,
+                         int64_t length)
+      : left_bitmap_(util::MakeNonNull(left_bitmap) + left_offset / 8),
+        left_offset_(left_offset % 8),
+        mid_bitmap_(util::MakeNonNull(mid_bitmap) + mid_offset / 8),
+        mid_offset_(mid_offset % 8),
+        right_bitmap_(util::MakeNonNull(right_bitmap) + right_offset / 8),
+        right_offset_(right_offset % 8),
+        bits_remaining_(length) {}
+
+  /// \brief Return the popcount of the bitwise-and of the next run of
+  /// available bits, up to 64. The returned pair contains the size of run and
+  /// the number of true values. The last block will have a length less than 64
+  /// if the bitmap length is not a multiple of 64, and will return 0-length
+  /// blocks in subsequent invocations.
+  BitBlockCount NextAndAndWord() { return NextWord<detail::BitBlockAndAnd>(); }
+
+  /// \brief Computes "x & ~y & ~z" block for each available run of bits.
+  BitBlockCount NextAndNotAndNotWord() {
+    return NextWord<detail::BitBlockAndNotAndNot>();
+  }
+
+  /// \brief Computes "~x & y & ~z" block for each available run of bits.
+  BitBlockCount NextNotAndAndNotWord() {
+    return NextWord<detail::BitBlockNotAndAndNot>();
+  }
+
+  /// \brief Computes "~x & ~y & z" block for each available run of bits.
+  BitBlockCount NextNotAndNotAndWord() {
+    return NextWord<detail::BitBlockNotAndNotAnd>();
+  }
+
+  /// \brief Computes "~x & y & z" block for each available run of bits.
+  BitBlockCount NextNotAndAndWord() { return NextWord<detail::BitBlockNotAndAnd>(); }
+
+  /// \brief Computes "x & ~y & z" block for each available run of bits.
+  BitBlockCount NextAndNotAndWord() { return NextWord<detail::BitBlockAndNotAnd>(); }
+
+  /// \brief Computes "x & y & ~z" block for each available run of bits.
+  BitBlockCount NextAndAndNotWord() { return NextWord<detail::BitBlockNotAndNotAnd>(); }
+
+  /// \brief Computes "x | y | z" block for each available run of bits.
+  BitBlockCount NextOrOrWord() { return NextWord<detail::BitBlockOrOr>(); }
+
+  /// \brief Computes "x | ~y | z" block for each available run of bits.
+  BitBlockCount NextOrNotOrWord() { return NextWord<detail::BitBlockOrNotOr>(); }
+
+  /// \brief Computes "~x | y | z" block for each available run of bits.
+  BitBlockCount NextNotOrOrWord() { return NextWord<detail::BitBlockNotOrOr>(); }
+
+  /// \brief Computes "x | y | ~z" block for each available run of bits.
+  BitBlockCount NextOrOrNotWord() { return NextWord<detail::BitBlockOrOrNot>(); }
+
+  /// \brief Computes "~x | y | ~z" block for each available run of bits.
+  BitBlockCount NextNotOrOrNotWord() { return NextWord<detail::BitBlockNotOrOrNot>(); }
+
+  /// \brief Computes "x | ~y | ~z" block for each available run of bits.
+  BitBlockCount NextOrNotOrNotWord() { return NextWord<detail::BitBlockOrNotOrNot>(); }
+
+  /// \brief Computes "~x | ~y | z" block for each available run of bits.
+  BitBlockCount NextNotOrNotOrWord() { return NextWord<detail::BitBlockNotOrNotOr>(); }
+
+ private:
+  template <template <typename T> class Op>
+  BitBlockCount NextWord() {
+    using detail::LoadWord;
+    using detail::ShiftWord;
+
+    if (!bits_remaining_) {
+      return {0, 0};
+    }
+    // When the offset is > 0, we need there to be a word beyond the last aligned
+    // word in the bitmap for the bit shifting logic.
+    constexpr int64_t kWordBits = BitBlockCounter::kWordBits;
+    const int64_t bits_required_to_use_words =
+        std::max({left_offset_ == 0 ? 64 : 64 + (64 - left_offset_),
+                  mid_offset_ == 0 ? 64 : 64 + (64 - mid_offset_),
+                  right_offset_ == 0 ? 64 : 64 + (64 - right_offset_)});
+    if (bits_remaining_ < bits_required_to_use_words) {
+      const int16_t run_length =
+          static_cast<int16_t>(std::min(bits_remaining_, kWordBits));
+      int16_t popcount = 0;
+      for (int64_t i = 0; i < run_length; ++i) {
+        if (Op<bool>::Call(bit_util::GetBit(left_bitmap_, left_offset_ + i),
+                           bit_util::GetBit(mid_bitmap_, mid_offset_ + i),
+                           bit_util::GetBit(right_bitmap_, right_offset_ + i))) {
+          ++popcount;
+        }
+      }
+      // This code path should trigger _at most_ 2 times. In the "two times"
+      // case, the first time the run length will be a multiple of 8.
+      left_bitmap_ += run_length / 8;
+      mid_bitmap_ += run_length / 8;
+      right_bitmap_ += run_length / 8;
+      bits_remaining_ -= run_length;
+      return {run_length, popcount};
+    }
+
+    int64_t popcount = 0;
+    if (left_offset_ == 0 && right_offset_ == 0 && mid_offset_ == 0) {
+      popcount = bit_util::PopCount(Op<uint64_t>::Call(
+          LoadWord(left_bitmap_), LoadWord(mid_bitmap_), LoadWord(right_bitmap_)));
+    } else {
+      auto left_word =
+          ShiftWord(LoadWord(left_bitmap_), LoadWord(left_bitmap_ + 8), left_offset_);
+      auto mid_word =
+          ShiftWord(LoadWord(mid_bitmap_), LoadWord(mid_bitmap_ + 8), mid_offset_);
+      auto right_word =
+          ShiftWord(LoadWord(right_bitmap_), LoadWord(right_bitmap_ + 8), right_offset_);
+      popcount = bit_util::PopCount(Op<uint64_t>::Call(left_word, mid_word, right_word));
+    }
+    left_bitmap_ += kWordBits / 8;
+    mid_bitmap_ += kWordBits / 8;
+    right_bitmap_ += kWordBits / 8;
+    bits_remaining_ -= kWordBits;
+    return {64, static_cast<int16_t>(popcount)};
+  }
+
+  const uint8_t* left_bitmap_;
+  int64_t left_offset_;
+  const uint8_t* mid_bitmap_;
+  int64_t mid_offset_;
+  const uint8_t* right_bitmap_;
+  int64_t right_offset_;
+  int64_t bits_remaining_;
+};
+
+class ARROW_EXPORT OptionalTernaryBitBlockCounter {
+ public:
+  // Any bitmap may be NULLPTR
+  OptionalTernaryBitBlockCounter(const uint8_t* left_bitmap, int64_t left_offset,

Review comment:
       Is this actually implemented? I don't see a corresponding diff in the .cc file.

##########
File path: cpp/src/arrow/compute/kernels/scalar_compare_test.cc
##########
@@ -1850,5 +1846,439 @@ TEST(TestMaxElementWiseMinElementWise, CommonTemporal) {
               ResultWith(ScalarFromJSON(date64(), "86400000")));
 }
 
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const Datum& val, const Datum& lhs,
+                            const Datum& rhs, const Datum& expected) {
+  ASSERT_OK_AND_ASSIGN(
+      Datum result,
+      CallFunction(BetweenOperatorToFunctionName(options.op), {val, lhs, rhs}));
+  AssertArraysEqual(*expected.make_array(), *result.make_array(),
+                    /*verbose=*/true);
+}
+
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const Datum& val, const Datum& lhs,
+                            const Datum& rhs, const char* expected_str) {
+  auto expected = ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), expected_str);
+  ValidateBetween<ArrowType>(options, val, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const Datum& val, const Datum& lhs,
+                            const char* rhs_str, const Datum& expected) {
+  auto rhs = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), rhs_str);
+  ValidateBetween<ArrowType>(options, val, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const Datum& val, const char* lhs_str,
+                            const Datum& rhs, const Datum& expected) {
+  auto lhs = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), lhs_str);
+  ValidateBetween<ArrowType>(options, val, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const char* val_str, const Datum& lhs,
+                            const Datum& rhs, const Datum& expected) {
+  auto val = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), val_str);
+  ValidateBetween<ArrowType>(options, val, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const Datum& val, const Datum& lhs,
+                            const char* rhs_str, const char* expected_str) {
+  auto rhs = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), rhs_str);
+  auto expected = ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), expected_str);
+  ValidateBetween<ArrowType>(options, val, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const Datum& val, const char* lhs_str,
+                            const Datum& rhs, const char* expected_str) {
+  auto lhs = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), lhs_str);
+  auto expected = ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), expected_str);
+  ValidateBetween<ArrowType>(options, val, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const Datum& val, const char* lhs_str,
+                            const char* rhs_str, const Datum& expected) {
+  auto lhs = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), lhs_str);
+  auto rhs = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), rhs_str);
+  ValidateBetween<ArrowType>(options, val, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const char* val_str, const Datum& lhs,
+                            const Datum& rhs, const char* expected_str) {
+  auto val = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), val_str);
+  auto expected = ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), expected_str);
+  ValidateBetween<ArrowType>(options, val, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const char* val_str,
+                            const char* lhs_str, const Datum& rhs,
+                            const Datum& expected) {
+  auto val = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), val_str);
+  auto lhs = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), lhs_str);
+  ValidateBetween<ArrowType>(options, val, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const char* val_str, const Datum& lhs,
+                            const char* rhs_str, const Datum& expected) {
+  auto val = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), val_str);
+  auto rhs = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), rhs_str);
+  ValidateBetween<ArrowType>(options, val, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const Datum& val, const char* lhs_str,
+                            const char* rhs_str, const char* expected_str) {
+  auto lhs = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), lhs_str);
+  auto rhs = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), rhs_str);
+  auto expected = ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), expected_str);
+  ValidateBetween<ArrowType>(options, val, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const char* val_str, const Datum& lhs,
+                            const char* rhs_str, const char* expected_str) {
+  auto val = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), val_str);
+  auto rhs = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), rhs_str);
+  auto expected = ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), expected_str);
+  ValidateBetween<ArrowType>(options, val, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const char* val_str,
+                            const char* lhs_str, const Datum& rhs,
+                            const char* expected_str) {
+  auto val = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), val_str);
+  auto lhs = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), lhs_str);
+  auto expected = ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), expected_str);
+  ValidateBetween<ArrowType>(options, val, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const char* val_str,
+                            const char* lhs_str, const char* rhs_str,
+                            const Datum& expected) {
+  auto val = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), val_str);
+  auto lhs = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), lhs_str);
+  auto rhs = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), rhs_str);
+  ValidateBetween<ArrowType>(options, val, lhs, rhs, expected);
+}
+
+template <typename T>
+static inline bool SlowBetween(BetweenOperator op, const T& val, const T& lhs,
+                               const T& rhs) {
+  switch (op) {
+    case BETWEEN_LESS_EQUAL_LESS_EQUAL:
+      return ((lhs <= val) && (val <= rhs));
+    case BETWEEN_LESS_EQUAL_LESS_THAN:
+      return ((lhs <= val) && (val < rhs));
+    case BETWEEN_LESS_THAN_LESS_EQUAL:
+      return ((lhs < val) && (val <= rhs));
+    case BETWEEN_LESS_THAN_LESS_THAN:
+      return ((lhs < val) && (val < rhs));
+    default:
+      return false;
+  }
+}
+
+template <typename ArrayType>
+std::vector<bool> NullBitmapFromThreeArrays(const ArrayType& val, const ArrayType& lhs,
+                                            const ArrayType& rhs) {
+  auto value_lambda = [&val](int64_t i) {
+    return val.null_count() == 0 ? true : val.IsValid(i);
+  };
+
+  auto left_lambda = [&lhs](int64_t i) {
+    return lhs.null_count() == 0 ? true : lhs.IsValid(i);
+  };
+
+  auto right_lambda = [&rhs](int64_t i) {
+    return rhs.null_count() == 0 ? true : rhs.IsValid(i);
+  };
+
+  const int64_t length = lhs.length();
+  std::vector<bool> null_bitmap(length);
+
+  for (int64_t i = 0; i < length; i++) {
+    null_bitmap[i] = value_lambda(i) && left_lambda(i) && right_lambda(i);
+  }
+
+  return null_bitmap;
+}
+
+template <typename ArrowType>
+Datum SimpleArrayScalarScalarBetween(BetweenOptions options, const Datum& val,

Review comment:
       Can we implement this as `and(less(), less())`, i.e. can't we just reuse existing comparison functions?

##########
File path: cpp/src/arrow/compute/api_scalar.h
##########
@@ -316,6 +316,19 @@ struct ARROW_EXPORT CompareOptions {
   enum CompareOperator op;
 };
 
+enum BetweenOperator : int8_t {
+  BETWEEN_LESS_EQUAL_LESS_EQUAL,
+  BETWEEN_LESS_EQUAL_LESS_THAN,
+  BETWEEN_LESS_THAN_LESS_EQUAL,
+  BETWEEN_LESS_THAN_LESS_THAN,
+};
+
+struct ARROW_EXPORT BetweenOptions {
+  explicit BetweenOptions(BetweenOperator op) : op(op) {}
+  BetweenOptions() : BetweenOptions(BetweenOperator::BETWEEN_LESS_EQUAL_LESS_EQUAL) {}
+  enum BetweenOperator op;

Review comment:
       Ah, I see we followed how the comparison kernels appear to work, but I don't think we need to replicate that for this kernel. We can do the dispatch inside the kernel implementation itself as Eduardo suggests.

##########
File path: cpp/src/arrow/util/bit_block_counter.h
##########
@@ -424,6 +565,240 @@ class ARROW_EXPORT OptionalBinaryBitBlockCounter {
   }
 };
 
+/// \brief A class that computes popcounts on the result of bitwise operations
+/// between three bitmaps, 64 bits at a time. A 64-bit word is loaded from each
+/// bitmap, then the popcount is computed on e.g. the bitwise-and of the three
+/// words.
+class ARROW_EXPORT TernaryBitBlockCounter {
+ public:
+  TernaryBitBlockCounter(const uint8_t* left_bitmap, int64_t left_offset,
+                         const uint8_t* mid_bitmap, int64_t mid_offset,
+                         const uint8_t* right_bitmap, int64_t right_offset,
+                         int64_t length)
+      : left_bitmap_(util::MakeNonNull(left_bitmap) + left_offset / 8),
+        left_offset_(left_offset % 8),
+        mid_bitmap_(util::MakeNonNull(mid_bitmap) + mid_offset / 8),
+        mid_offset_(mid_offset % 8),
+        right_bitmap_(util::MakeNonNull(right_bitmap) + right_offset / 8),
+        right_offset_(right_offset % 8),
+        bits_remaining_(length) {}
+
+  /// \brief Return the popcount of the bitwise-and of the next run of
+  /// available bits, up to 64. The returned pair contains the size of run and
+  /// the number of true values. The last block will have a length less than 64
+  /// if the bitmap length is not a multiple of 64, and will return 0-length
+  /// blocks in subsequent invocations.
+  BitBlockCount NextAndAndWord() { return NextWord<detail::BitBlockAndAnd>(); }
+
+  /// \brief Computes "x & ~y & ~z" block for each available run of bits.
+  BitBlockCount NextAndNotAndNotWord() {
+    return NextWord<detail::BitBlockAndNotAndNot>();
+  }
+
+  /// \brief Computes "~x & y & ~z" block for each available run of bits.
+  BitBlockCount NextNotAndAndNotWord() {
+    return NextWord<detail::BitBlockNotAndAndNot>();
+  }
+
+  /// \brief Computes "~x & ~y & z" block for each available run of bits.
+  BitBlockCount NextNotAndNotAndWord() {
+    return NextWord<detail::BitBlockNotAndNotAnd>();
+  }
+
+  /// \brief Computes "~x & y & z" block for each available run of bits.
+  BitBlockCount NextNotAndAndWord() { return NextWord<detail::BitBlockNotAndAnd>(); }
+
+  /// \brief Computes "x & ~y & z" block for each available run of bits.
+  BitBlockCount NextAndNotAndWord() { return NextWord<detail::BitBlockAndNotAnd>(); }
+
+  /// \brief Computes "x & y & ~z" block for each available run of bits.
+  BitBlockCount NextAndAndNotWord() { return NextWord<detail::BitBlockNotAndNotAnd>(); }
+
+  /// \brief Computes "x | y | z" block for each available run of bits.
+  BitBlockCount NextOrOrWord() { return NextWord<detail::BitBlockOrOr>(); }
+
+  /// \brief Computes "x | ~y | z" block for each available run of bits.
+  BitBlockCount NextOrNotOrWord() { return NextWord<detail::BitBlockOrNotOr>(); }
+
+  /// \brief Computes "~x | y | z" block for each available run of bits.
+  BitBlockCount NextNotOrOrWord() { return NextWord<detail::BitBlockNotOrOr>(); }
+
+  /// \brief Computes "x | y | ~z" block for each available run of bits.
+  BitBlockCount NextOrOrNotWord() { return NextWord<detail::BitBlockOrOrNot>(); }
+
+  /// \brief Computes "~x | y | ~z" block for each available run of bits.
+  BitBlockCount NextNotOrOrNotWord() { return NextWord<detail::BitBlockNotOrOrNot>(); }
+
+  /// \brief Computes "x | ~y | ~z" block for each available run of bits.
+  BitBlockCount NextOrNotOrNotWord() { return NextWord<detail::BitBlockOrNotOrNot>(); }
+
+  /// \brief Computes "~x | ~y | z" block for each available run of bits.
+  BitBlockCount NextNotOrNotOrWord() { return NextWord<detail::BitBlockNotOrNotOr>(); }
+
+ private:
+  template <template <typename T> class Op>
+  BitBlockCount NextWord() {
+    using detail::LoadWord;
+    using detail::ShiftWord;
+
+    if (!bits_remaining_) {
+      return {0, 0};
+    }
+    // When the offset is > 0, we need there to be a word beyond the last aligned
+    // word in the bitmap for the bit shifting logic.
+    constexpr int64_t kWordBits = BitBlockCounter::kWordBits;
+    const int64_t bits_required_to_use_words =
+        std::max({left_offset_ == 0 ? 64 : 64 + (64 - left_offset_),
+                  mid_offset_ == 0 ? 64 : 64 + (64 - mid_offset_),
+                  right_offset_ == 0 ? 64 : 64 + (64 - right_offset_)});
+    if (bits_remaining_ < bits_required_to_use_words) {
+      const int16_t run_length =
+          static_cast<int16_t>(std::min(bits_remaining_, kWordBits));
+      int16_t popcount = 0;
+      for (int64_t i = 0; i < run_length; ++i) {
+        if (Op<bool>::Call(bit_util::GetBit(left_bitmap_, left_offset_ + i),
+                           bit_util::GetBit(mid_bitmap_, mid_offset_ + i),
+                           bit_util::GetBit(right_bitmap_, right_offset_ + i))) {
+          ++popcount;
+        }
+      }
+      // This code path should trigger _at most_ 2 times. In the "two times"
+      // case, the first time the run length will be a multiple of 8.
+      left_bitmap_ += run_length / 8;
+      mid_bitmap_ += run_length / 8;
+      right_bitmap_ += run_length / 8;
+      bits_remaining_ -= run_length;
+      return {run_length, popcount};
+    }
+
+    int64_t popcount = 0;
+    if (left_offset_ == 0 && right_offset_ == 0 && mid_offset_ == 0) {
+      popcount = bit_util::PopCount(Op<uint64_t>::Call(
+          LoadWord(left_bitmap_), LoadWord(mid_bitmap_), LoadWord(right_bitmap_)));
+    } else {
+      auto left_word =
+          ShiftWord(LoadWord(left_bitmap_), LoadWord(left_bitmap_ + 8), left_offset_);
+      auto mid_word =
+          ShiftWord(LoadWord(mid_bitmap_), LoadWord(mid_bitmap_ + 8), mid_offset_);
+      auto right_word =
+          ShiftWord(LoadWord(right_bitmap_), LoadWord(right_bitmap_ + 8), right_offset_);
+      popcount = bit_util::PopCount(Op<uint64_t>::Call(left_word, mid_word, right_word));
+    }
+    left_bitmap_ += kWordBits / 8;
+    mid_bitmap_ += kWordBits / 8;
+    right_bitmap_ += kWordBits / 8;
+    bits_remaining_ -= kWordBits;
+    return {64, static_cast<int16_t>(popcount)};
+  }
+
+  const uint8_t* left_bitmap_;
+  int64_t left_offset_;
+  const uint8_t* mid_bitmap_;
+  int64_t mid_offset_;
+  const uint8_t* right_bitmap_;
+  int64_t right_offset_;
+  int64_t bits_remaining_;
+};
+
+class ARROW_EXPORT OptionalTernaryBitBlockCounter {
+ public:
+  // Any bitmap may be NULLPTR
+  OptionalTernaryBitBlockCounter(const uint8_t* left_bitmap, int64_t left_offset,

Review comment:
       Is it even used?

##########
File path: cpp/src/arrow/compute/kernels/scalar_compare_test.cc
##########
@@ -1850,5 +1846,439 @@ TEST(TestMaxElementWiseMinElementWise, CommonTemporal) {
               ResultWith(ScalarFromJSON(date64(), "86400000")));
 }
 
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const Datum& val, const Datum& lhs,
+                            const Datum& rhs, const Datum& expected) {
+  ASSERT_OK_AND_ASSIGN(
+      Datum result,
+      CallFunction(BetweenOperatorToFunctionName(options.op), {val, lhs, rhs}));
+  AssertArraysEqual(*expected.make_array(), *result.make_array(),
+                    /*verbose=*/true);
+}
+
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const Datum& val, const Datum& lhs,
+                            const Datum& rhs, const char* expected_str) {
+  auto expected = ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), expected_str);
+  ValidateBetween<ArrowType>(options, val, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const Datum& val, const Datum& lhs,
+                            const char* rhs_str, const Datum& expected) {
+  auto rhs = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), rhs_str);
+  ValidateBetween<ArrowType>(options, val, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const Datum& val, const char* lhs_str,
+                            const Datum& rhs, const Datum& expected) {
+  auto lhs = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), lhs_str);
+  ValidateBetween<ArrowType>(options, val, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const char* val_str, const Datum& lhs,
+                            const Datum& rhs, const Datum& expected) {
+  auto val = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), val_str);
+  ValidateBetween<ArrowType>(options, val, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const Datum& val, const Datum& lhs,
+                            const char* rhs_str, const char* expected_str) {
+  auto rhs = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), rhs_str);
+  auto expected = ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), expected_str);
+  ValidateBetween<ArrowType>(options, val, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const Datum& val, const char* lhs_str,
+                            const Datum& rhs, const char* expected_str) {
+  auto lhs = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), lhs_str);
+  auto expected = ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), expected_str);
+  ValidateBetween<ArrowType>(options, val, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const Datum& val, const char* lhs_str,
+                            const char* rhs_str, const Datum& expected) {
+  auto lhs = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), lhs_str);
+  auto rhs = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), rhs_str);
+  ValidateBetween<ArrowType>(options, val, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const char* val_str, const Datum& lhs,
+                            const Datum& rhs, const char* expected_str) {
+  auto val = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), val_str);
+  auto expected = ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), expected_str);
+  ValidateBetween<ArrowType>(options, val, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const char* val_str,
+                            const char* lhs_str, const Datum& rhs,
+                            const Datum& expected) {
+  auto val = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), val_str);
+  auto lhs = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), lhs_str);
+  ValidateBetween<ArrowType>(options, val, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const char* val_str, const Datum& lhs,
+                            const char* rhs_str, const Datum& expected) {
+  auto val = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), val_str);
+  auto rhs = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), rhs_str);
+  ValidateBetween<ArrowType>(options, val, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const Datum& val, const char* lhs_str,
+                            const char* rhs_str, const char* expected_str) {
+  auto lhs = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), lhs_str);
+  auto rhs = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), rhs_str);
+  auto expected = ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), expected_str);
+  ValidateBetween<ArrowType>(options, val, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const char* val_str, const Datum& lhs,
+                            const char* rhs_str, const char* expected_str) {
+  auto val = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), val_str);
+  auto rhs = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), rhs_str);
+  auto expected = ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), expected_str);
+  ValidateBetween<ArrowType>(options, val, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const char* val_str,
+                            const char* lhs_str, const Datum& rhs,
+                            const char* expected_str) {
+  auto val = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), val_str);
+  auto lhs = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), lhs_str);
+  auto expected = ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), expected_str);
+  ValidateBetween<ArrowType>(options, val, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const char* val_str,
+                            const char* lhs_str, const char* rhs_str,
+                            const Datum& expected) {
+  auto val = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), val_str);
+  auto lhs = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), lhs_str);
+  auto rhs = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), rhs_str);
+  ValidateBetween<ArrowType>(options, val, lhs, rhs, expected);
+}

Review comment:
       Is it actually useful to have all these permutations? We should just call `ArrayFromJSON` at the call site.




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

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

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



[GitHub] [arrow] bkmgit commented on pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
bkmgit commented on pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#issuecomment-997640514


   @cyb70289 @lidavidm A light review of this pull request would be helpful. Would like to add 
   - further tests 
   - function options to choose comparison mode
   Good comparison of strings is out of scope, but would be good to add in future, currently utf8 codepoint is used for string comparison which is ok for ascii only text, but poor for most other texts.


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

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

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



[GitHub] [arrow] bkmgit commented on a change in pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
bkmgit commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r772827256



##########
File path: cpp/src/arrow/compute/kernels/scalar_compare_test.cc
##########
@@ -1850,5 +1846,439 @@ TEST(TestMaxElementWiseMinElementWise, CommonTemporal) {
               ResultWith(ScalarFromJSON(date64(), "86400000")));
 }
 
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const Datum& val, const Datum& lhs,
+                            const Datum& rhs, const Datum& expected) {
+  ASSERT_OK_AND_ASSIGN(
+      Datum result,
+      CallFunction(BetweenOperatorToFunctionName(options.op), {val, lhs, rhs}));
+  AssertArraysEqual(*expected.make_array(), *result.make_array(),
+                    /*verbose=*/true);
+}
+
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const Datum& val, const Datum& lhs,
+                            const Datum& rhs, const char* expected_str) {
+  auto expected = ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), expected_str);
+  ValidateBetween<ArrowType>(options, val, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const Datum& val, const Datum& lhs,
+                            const char* rhs_str, const Datum& expected) {
+  auto rhs = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), rhs_str);
+  ValidateBetween<ArrowType>(options, val, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const Datum& val, const char* lhs_str,
+                            const Datum& rhs, const Datum& expected) {
+  auto lhs = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), lhs_str);
+  ValidateBetween<ArrowType>(options, val, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const char* val_str, const Datum& lhs,
+                            const Datum& rhs, const Datum& expected) {
+  auto val = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), val_str);
+  ValidateBetween<ArrowType>(options, val, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const Datum& val, const Datum& lhs,
+                            const char* rhs_str, const char* expected_str) {
+  auto rhs = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), rhs_str);
+  auto expected = ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), expected_str);
+  ValidateBetween<ArrowType>(options, val, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const Datum& val, const char* lhs_str,
+                            const Datum& rhs, const char* expected_str) {
+  auto lhs = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), lhs_str);
+  auto expected = ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), expected_str);
+  ValidateBetween<ArrowType>(options, val, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const Datum& val, const char* lhs_str,
+                            const char* rhs_str, const Datum& expected) {
+  auto lhs = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), lhs_str);
+  auto rhs = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), rhs_str);
+  ValidateBetween<ArrowType>(options, val, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const char* val_str, const Datum& lhs,
+                            const Datum& rhs, const char* expected_str) {
+  auto val = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), val_str);
+  auto expected = ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), expected_str);
+  ValidateBetween<ArrowType>(options, val, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const char* val_str,
+                            const char* lhs_str, const Datum& rhs,
+                            const Datum& expected) {
+  auto val = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), val_str);
+  auto lhs = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), lhs_str);
+  ValidateBetween<ArrowType>(options, val, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const char* val_str, const Datum& lhs,
+                            const char* rhs_str, const Datum& expected) {
+  auto val = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), val_str);
+  auto rhs = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), rhs_str);
+  ValidateBetween<ArrowType>(options, val, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const Datum& val, const char* lhs_str,
+                            const char* rhs_str, const char* expected_str) {
+  auto lhs = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), lhs_str);
+  auto rhs = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), rhs_str);
+  auto expected = ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), expected_str);
+  ValidateBetween<ArrowType>(options, val, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const char* val_str, const Datum& lhs,
+                            const char* rhs_str, const char* expected_str) {
+  auto val = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), val_str);
+  auto rhs = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), rhs_str);
+  auto expected = ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), expected_str);
+  ValidateBetween<ArrowType>(options, val, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const char* val_str,
+                            const char* lhs_str, const Datum& rhs,
+                            const char* expected_str) {
+  auto val = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), val_str);
+  auto lhs = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), lhs_str);
+  auto expected = ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), expected_str);
+  ValidateBetween<ArrowType>(options, val, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const char* val_str,
+                            const char* lhs_str, const char* rhs_str,
+                            const Datum& expected) {
+  auto val = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), val_str);
+  auto lhs = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), lhs_str);
+  auto rhs = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), rhs_str);
+  ValidateBetween<ArrowType>(options, val, lhs, rhs, expected);
+}
+
+template <typename T>
+static inline bool SlowBetween(BetweenOperator op, const T& val, const T& lhs,
+                               const T& rhs) {
+  switch (op) {
+    case BETWEEN_LESS_EQUAL_LESS_EQUAL:
+      return ((lhs <= val) && (val <= rhs));
+    case BETWEEN_LESS_EQUAL_LESS_THAN:
+      return ((lhs <= val) && (val < rhs));
+    case BETWEEN_LESS_THAN_LESS_EQUAL:
+      return ((lhs < val) && (val <= rhs));
+    case BETWEEN_LESS_THAN_LESS_THAN:
+      return ((lhs < val) && (val < rhs));
+    default:
+      return false;
+  }
+}
+
+template <typename ArrayType>
+std::vector<bool> NullBitmapFromThreeArrays(const ArrayType& val, const ArrayType& lhs,
+                                            const ArrayType& rhs) {
+  auto value_lambda = [&val](int64_t i) {
+    return val.null_count() == 0 ? true : val.IsValid(i);
+  };
+
+  auto left_lambda = [&lhs](int64_t i) {
+    return lhs.null_count() == 0 ? true : lhs.IsValid(i);
+  };
+
+  auto right_lambda = [&rhs](int64_t i) {
+    return rhs.null_count() == 0 ? true : rhs.IsValid(i);
+  };
+
+  const int64_t length = lhs.length();
+  std::vector<bool> null_bitmap(length);
+
+  for (int64_t i = 0; i < length; i++) {
+    null_bitmap[i] = value_lambda(i) && left_lambda(i) && right_lambda(i);
+  }
+
+  return null_bitmap;
+}
+
+template <typename ArrowType>
+Datum SimpleArrayScalarScalarBetween(BetweenOptions options, const Datum& val,

Review comment:
       My understanding is that would have a performance overhead as this was not the suggested method to use in the original issue https://issues.apache.org/jira/browse/ARROW-9843




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

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

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



[GitHub] [arrow] edponce commented on a change in pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
edponce commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r777666713



##########
File path: cpp/src/arrow/compute/kernels/scalar_compare.cc
##########
@@ -156,39 +210,50 @@ struct Maximum {
   }
 };
 
+// Check if timestamp timezones are comparable (either all are empty or none is).
+Status CheckCompareTimestamps(const ExecBatch& batch) {
+  if (batch.num_values() > 0) {
+    const auto& ts0 = checked_cast<const TimestampType&>(*batch[0].type());
+    bool invalid_state = ts0.timezone().empty();
+    for (int i = 1; i < batch.num_values(); ++i) {
+      const auto& ts = checked_cast<const TimestampType&>(*batch[i].type());
+      invalid_state ^= ts.timezone().empty();
+    }
+    if (invalid_state) {
+      return Status::Invalid(
+          "Cannot compare timestamp with timezone to timestamp without timezone");
+    }

Review comment:
       Oops! I left a bug here. You need to move the check`if (invalid_state) {...}` inside the `for-loop`.
   As it is right now, it will only work for binary cases not ternary or higher because `1^0^1` --> `0`.




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

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

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



[GitHub] [arrow] bkmgit commented on a change in pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
bkmgit commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r775475123



##########
File path: cpp/src/arrow/compute/kernels/scalar_compare.cc
##########
@@ -68,6 +72,50 @@ struct GreaterEqual {
   }
 };
 
+struct BetweenLessEqualLessEqual {
+  template <typename T, typename Arg0, typename Arg1, typename Arg2>
+  static constexpr T Call(KernelContext*, const Arg0& middle, const Arg1& left,
+                          const Arg2& right, Status*) {
+    static_assert(std::is_same<T, bool>::value && std::is_same<Arg0, Arg1>::value &&
+                      std::is_same<Arg1, Arg2>::value,
+                  "");
+    return (left <= middle) && (middle <= right);
+  }
+};
+
+struct BetweenLessThanLessEqual {
+  template <typename T, typename Arg0, typename Arg1, typename Arg2>
+  static constexpr T Call(KernelContext*, const Arg0& middle, const Arg1& left,
+                          const Arg2& right, Status*) {
+    static_assert(std::is_same<T, bool>::value && std::is_same<Arg0, Arg1>::value &&
+                      std::is_same<Arg1, Arg2>::value,
+                  "");
+    return (left < middle) && (middle <= right);
+  }
+};
+
+struct BetweenLessEqualLessThan {
+  template <typename T, typename Arg0, typename Arg1, typename Arg2>
+  static constexpr T Call(KernelContext*, const Arg0& middle, const Arg1& left,
+                          const Arg2& right, Status*) {
+    static_assert(std::is_same<T, bool>::value && std::is_same<Arg0, Arg1>::value &&
+                      std::is_same<Arg1, Arg2>::value,
+                  "");
+    return (left <= middle) && (middle < right);
+  }
+};
+
+struct BetweenLessThanLessThan {
+  template <typename T, typename Arg0, typename Arg1, typename Arg2>
+  static constexpr T Call(KernelContext*, const Arg0& middle, const Arg1& left,
+                          const Arg2& right, Status*) {
+    static_assert(std::is_same<T, bool>::value && std::is_same<Arg0, Arg1>::value &&
+                      std::is_same<Arg1, Arg2>::value,
+                  "");
+    return (left < middle) && (middle < right);
+  }
+};

Review comment:
       Both Compare and Between will need a further option for strings similar to [SortKeys](https://arrow.apache.org/docs/cpp/api/compute.html#_CPPv4N5arrow7compute11SortOptionsE)




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

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

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



[GitHub] [arrow] bkmgit commented on a change in pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
bkmgit commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r775512462



##########
File path: cpp/src/arrow/compute/kernels/scalar_compare_test.cc
##########
@@ -1850,5 +1846,439 @@ TEST(TestMaxElementWiseMinElementWise, CommonTemporal) {
               ResultWith(ScalarFromJSON(date64(), "86400000")));
 }
 
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const Datum& val, const Datum& lhs,
+                            const Datum& rhs, const Datum& expected) {
+  ASSERT_OK_AND_ASSIGN(
+      Datum result,
+      CallFunction(BetweenOperatorToFunctionName(options.op), {val, lhs, rhs}));
+  AssertArraysEqual(*expected.make_array(), *result.make_array(),
+                    /*verbose=*/true);
+}
+
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const Datum& val, const Datum& lhs,
+                            const Datum& rhs, const char* expected_str) {
+  auto expected = ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), expected_str);
+  ValidateBetween<ArrowType>(options, val, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const Datum& val, const Datum& lhs,
+                            const char* rhs_str, const Datum& expected) {
+  auto rhs = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), rhs_str);
+  ValidateBetween<ArrowType>(options, val, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const Datum& val, const char* lhs_str,
+                            const Datum& rhs, const Datum& expected) {
+  auto lhs = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), lhs_str);
+  ValidateBetween<ArrowType>(options, val, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const char* val_str, const Datum& lhs,
+                            const Datum& rhs, const Datum& expected) {
+  auto val = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), val_str);
+  ValidateBetween<ArrowType>(options, val, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const Datum& val, const Datum& lhs,
+                            const char* rhs_str, const char* expected_str) {
+  auto rhs = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), rhs_str);
+  auto expected = ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), expected_str);
+  ValidateBetween<ArrowType>(options, val, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const Datum& val, const char* lhs_str,
+                            const Datum& rhs, const char* expected_str) {
+  auto lhs = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), lhs_str);
+  auto expected = ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), expected_str);
+  ValidateBetween<ArrowType>(options, val, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const Datum& val, const char* lhs_str,
+                            const char* rhs_str, const Datum& expected) {
+  auto lhs = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), lhs_str);
+  auto rhs = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), rhs_str);
+  ValidateBetween<ArrowType>(options, val, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const char* val_str, const Datum& lhs,
+                            const Datum& rhs, const char* expected_str) {
+  auto val = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), val_str);
+  auto expected = ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), expected_str);
+  ValidateBetween<ArrowType>(options, val, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const char* val_str,
+                            const char* lhs_str, const Datum& rhs,
+                            const Datum& expected) {
+  auto val = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), val_str);
+  auto lhs = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), lhs_str);
+  ValidateBetween<ArrowType>(options, val, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const char* val_str, const Datum& lhs,
+                            const char* rhs_str, const Datum& expected) {
+  auto val = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), val_str);
+  auto rhs = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), rhs_str);
+  ValidateBetween<ArrowType>(options, val, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const Datum& val, const char* lhs_str,
+                            const char* rhs_str, const char* expected_str) {
+  auto lhs = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), lhs_str);
+  auto rhs = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), rhs_str);
+  auto expected = ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), expected_str);
+  ValidateBetween<ArrowType>(options, val, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const char* val_str, const Datum& lhs,
+                            const char* rhs_str, const char* expected_str) {
+  auto val = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), val_str);
+  auto rhs = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), rhs_str);
+  auto expected = ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), expected_str);
+  ValidateBetween<ArrowType>(options, val, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const char* val_str,
+                            const char* lhs_str, const Datum& rhs,
+                            const char* expected_str) {
+  auto val = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), val_str);
+  auto lhs = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), lhs_str);
+  auto expected = ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), expected_str);
+  ValidateBetween<ArrowType>(options, val, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const char* val_str,
+                            const char* lhs_str, const char* rhs_str,
+                            const Datum& expected) {
+  auto val = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), val_str);
+  auto lhs = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), lhs_str);
+  auto rhs = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), rhs_str);
+  ValidateBetween<ArrowType>(options, val, lhs, rhs, expected);
+}

Review comment:
       Ok. Done.




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

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

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



[GitHub] [arrow] bkmgit commented on a change in pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
bkmgit commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r772108918



##########
File path: cpp/src/arrow/compute/kernels/scalar_compare.cc
##########
@@ -68,6 +72,50 @@ struct GreaterEqual {
   }
 };
 
+struct BetweenLessEqualLessEqual {
+  template <typename T, typename Arg0, typename Arg1, typename Arg2>
+  static constexpr T Call(KernelContext*, const Arg0& middle, const Arg1& left,
+                          const Arg2& right, Status*) {
+    static_assert(std::is_same<T, bool>::value && std::is_same<Arg0, Arg1>::value &&
+                      std::is_same<Arg1, Arg2>::value,
+                  "");
+    return (left <= middle) && (middle <= right);
+  }
+};
+
+struct BetweenLessThanLessEqual {
+  template <typename T, typename Arg0, typename Arg1, typename Arg2>
+  static constexpr T Call(KernelContext*, const Arg0& middle, const Arg1& left,
+                          const Arg2& right, Status*) {
+    static_assert(std::is_same<T, bool>::value && std::is_same<Arg0, Arg1>::value &&
+                      std::is_same<Arg1, Arg2>::value,
+                  "");
+    return (left < middle) && (middle <= right);
+  }
+};
+
+struct BetweenLessEqualLessThan {
+  template <typename T, typename Arg0, typename Arg1, typename Arg2>
+  static constexpr T Call(KernelContext*, const Arg0& middle, const Arg1& left,
+                          const Arg2& right, Status*) {
+    static_assert(std::is_same<T, bool>::value && std::is_same<Arg0, Arg1>::value &&
+                      std::is_same<Arg1, Arg2>::value,
+                  "");
+    return (left <= middle) && (middle < right);
+  }
+};
+
+struct BetweenLessThanLessThan {
+  template <typename T, typename Arg0, typename Arg1, typename Arg2>
+  static constexpr T Call(KernelContext*, const Arg0& middle, const Arg1& left,
+                          const Arg2& right, Status*) {
+    static_assert(std::is_same<T, bool>::value && std::is_same<Arg0, Arg1>::value &&
+                      std::is_same<Arg1, Arg2>::value,
+                  "");
+    return (left < middle) && (middle < right);
+  }
+};

Review comment:
       @edponce Would function options also be better to use for the compare function?




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

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

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



[GitHub] [arrow] bkmgit commented on a change in pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
bkmgit commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r776929946



##########
File path: cpp/src/arrow/util/bit_block_counter.h
##########
@@ -424,6 +565,141 @@ class ARROW_EXPORT OptionalBinaryBitBlockCounter {
   }
 };
 
+/// \brief A class that computes popcounts on the result of bitwise operations
+/// between three bitmaps, 64 bits at a time. A 64-bit word is loaded from each
+/// bitmap, then the popcount is computed on e.g. the bitwise-and of the three
+/// words.
+class ARROW_EXPORT TernaryBitBlockCounter {
+ public:
+  TernaryBitBlockCounter(const uint8_t* left_bitmap, int64_t left_offset,
+                         const uint8_t* mid_bitmap, int64_t mid_offset,
+                         const uint8_t* right_bitmap, int64_t right_offset,
+                         int64_t length)
+      : left_bitmap_(util::MakeNonNull(left_bitmap) + left_offset / 8),
+        left_offset_(left_offset % 8),
+        mid_bitmap_(util::MakeNonNull(mid_bitmap) + mid_offset / 8),
+        mid_offset_(mid_offset % 8),
+        right_bitmap_(util::MakeNonNull(right_bitmap) + right_offset / 8),
+        right_offset_(right_offset % 8),
+        bits_remaining_(length) {}
+
+  /// \brief Return the popcount of the bitwise-and of the next run of
+  /// available bits, up to 64. The returned pair contains the size of run and
+  /// the number of true values. The last block will have a length less than 64
+  /// if the bitmap length is not a multiple of 64, and will return 0-length
+  /// blocks in subsequent invocations.
+  BitBlockCount NextAndAndWord() { return NextWord<detail::BitBlockAndAnd>(); }

Review comment:
       Ok




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

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

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



[GitHub] [arrow] bkmgit commented on a change in pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
bkmgit commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r776929904



##########
File path: cpp/src/arrow/compute/api_scalar.h
##########
@@ -316,6 +316,18 @@ struct ARROW_EXPORT CompareOptions {
   enum CompareOperator op;
 };
 
+class ARROW_EXPORT BetweenOptions : public FunctionOptions {
+ public:
+  enum Inclusiveness { BOTH, LEFT, RIGHT, NEITHER };

Review comment:
       ok




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

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

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



[GitHub] [arrow] edponce commented on pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
edponce commented on pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#issuecomment-1002959199


   @bkmgit As this PR stands, the `BetweenOptions` are not actually used internally (ie., passed to `CallFunction`) but rather an artifact that only exists if directly using the public scalar API `Between()` which is not the common case. If this PR will not use `BetweenOptions` internally, I would argue for them to not be included nor the public scalar API. These comments depend on the resolution of the previous observations I made wrt to the possible solutions for the `between` API.
   
   Please take all these comments as simple observations for discussion, as I had more time to think about this and did not thought about them earlier.


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

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

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



[GitHub] [arrow] edponce edited a comment on pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
edponce edited a comment on pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#issuecomment-1003110634


   From the previous discussion on compute function's API, recall that the main entry point for invoking a function is `CallFunction(name, args, options, ctx)`. This allows passing function-specific `options`. If functions were to be grouped and the `options` parameter used to select a particular function, then expressing function-specific `options` would be difficult.
   
   For this particular kernel, `between`, I recommend to have a single `between` function that uses the `options` from `CallFunction` to select the desired kernel implementation. This approach will provide the structure for the `not_between` function as well.
   
   @bkmgit I created ARROW-15223 as a follow-up JIRA for adding the `not_between` compute function. Thanks!


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

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

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



[GitHub] [arrow] bkmgit commented on a change in pull request #11882: ARROW-9843: [C++][Python] Implement Between ternary kernel and Python bindings

Posted by GitBox <gi...@apache.org>.
bkmgit commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r787038522



##########
File path: cpp/src/arrow/util/bit_block_counter.h
##########
@@ -422,6 +426,96 @@ class ARROW_EXPORT OptionalBinaryBitBlockCounter {
   }
 };
 
+/// \brief A class that computes popcounts on the result of bitwise operations
+/// between three bitmaps, 64 bits at a time. A 64-bit word is loaded from each
+/// bitmap, then the popcount is computed on e.g. the bitwise-and of the three
+/// words.
+class ARROW_EXPORT TernaryBitBlockCounter {

Review comment:
       Removed.




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

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

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



[GitHub] [arrow] bkmgit commented on a change in pull request #11882: ARROW-9843: [C++][Python] Implement Between ternary kernel and Python bindings

Posted by GitBox <gi...@apache.org>.
bkmgit commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r788996919



##########
File path: cpp/src/arrow/compute/kernels/scalar_compare_test.cc
##########
@@ -1865,5 +1867,683 @@ TEST(TestMaxElementWiseMinElementWise, CommonTemporal) {
               ResultWith(ScalarFromJSON(date64(), "86400000")));
 }
 
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const Datum& val, const Datum& lhs,
+                            const Datum& rhs, const Datum& expected) {
+  ASSERT_OK_AND_ASSIGN(Datum result, Between(val, lhs, rhs, options, nullptr));
+  AssertArraysEqual(*expected.make_array(), *result.make_array(),
+                    /*verbose=*/true);
+}
+
+template <typename ArrowType>
+void ValidateBetween(BetweenOptions options, const Datum& val, const Datum& lhs,
+                     const Datum& rhs) {
+  CompareOperator lhs_val;
+  CompareOperator val_rhs;
+  BetweenOptions::Inclusive include_endpoints = options.inclusive;
+
+  if (include_endpoints == BetweenOptions::Inclusive::NEITHER) {
+    lhs_val = LESS;
+    val_rhs = LESS;
+  } else if (include_endpoints == BetweenOptions::Inclusive::LEFT) {
+    lhs_val = LESS_EQUAL;
+    val_rhs = LESS;
+  } else if (include_endpoints == BetweenOptions::Inclusive::RIGHT) {
+    lhs_val = LESS;
+    val_rhs = LESS_EQUAL;
+  } else {
+    lhs_val = LESS_EQUAL;
+    val_rhs = LESS_EQUAL;
+  }
+
+  ASSERT_OK_AND_ASSIGN(Datum resultl,
+                       CallFunction(CompareOperatorToFunctionName(lhs_val), {lhs, val}));
+  ASSERT_OK_AND_ASSIGN(Datum resultr,
+                       CallFunction(CompareOperatorToFunctionName(val_rhs), {val, rhs}));
+  ASSERT_OK_AND_ASSIGN(Datum expected, CallFunction("and", {resultl, resultr}));
+
+  ValidateBetween<ArrowType>(options, val, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+class TestNumericBetweenKernel : public ::testing::Test {};
+
+TYPED_TEST_SUITE(TestNumericBetweenKernel, NumericArrowTypes);
+TYPED_TEST(TestNumericBetweenKernel, SimpleBetweenArrayScalarScalar) {
+  using ScalarType = typename TypeTraits<TypeParam>::ScalarType;
+  using CType = typename TypeTraits<TypeParam>::CType;
+
+  Datum zero(std::make_shared<ScalarType>(CType(0)));
+  Datum four(std::make_shared<ScalarType>(CType(4)));
+  Datum null(std::make_shared<ScalarType>());
+  BetweenOptions InclusiveBoth(BetweenOptions::Inclusive::BOTH);
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"), zero,
+      four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      zero, four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,1,2,2]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[1,1,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,3,4,5]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[1,1,1,1,1,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,1,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null,0,1,1]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), null, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, null,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  BetweenOptions InclusiveLeft(BetweenOptions::Inclusive::LEFT);
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"), zero,
+      four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      zero, four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,1,2,2]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[1,1,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,3,4,5]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[1,1,1,1,0,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,0,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null,0,1,1]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), null, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, null,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  BetweenOptions InclusiveRight(BetweenOptions::Inclusive::RIGHT);
+  ValidateBetween<TypeParam>(
+      InclusiveRight, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"), zero,
+      four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      zero, four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,1,2,2]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,0,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,3,4,5]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,1,1,1,1,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,1,1,1,1,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null,0,1,1]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null,0,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), null, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, null,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  BetweenOptions InclusiveNeither(BetweenOptions::Inclusive::NEITHER);
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      zero, four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      zero, four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,1,2,2]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,0,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,3,4,5]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,1,1,1,0,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,0,1,1,1,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null,0,1,1]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null,0,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), null, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, null,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+}
+
+TYPED_TEST(TestNumericBetweenKernel, SimpleBetweenScalarArrayArray) {
+  using ScalarType = typename TypeTraits<TypeParam>::ScalarType;
+  using CType = typename TypeTraits<TypeParam>::CType;
+
+  Datum one(std::make_shared<ScalarType>(CType(1)));
+
+  BetweenOptions InclusiveBoth(BetweenOptions::Inclusive::BOTH);
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, one, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,true,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,true,null,false,false]"));
+  BetweenOptions InclusiveLeft(BetweenOptions::Inclusive::LEFT);
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, one, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,true,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,true,null,false,false]"));
+  BetweenOptions InclusiveRight(BetweenOptions::Inclusive::RIGHT);
+  ValidateBetween<TypeParam>(
+      InclusiveRight, one, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,false,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,false,null,false,false]"));
+  BetweenOptions InclusiveNeither(BetweenOptions::Inclusive::NEITHER);
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, one, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,false,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,false,null,false,false]"));
+}
+
+TYPED_TEST(TestNumericBetweenKernel, SimpleBetweenArrayArrayArray) {
+  BetweenOptions InclusiveBoth(BetweenOptions::Inclusive::BOTH);
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[1,1,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,true,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,true,null,false,false]"));
+  BetweenOptions InclusiveLeft(BetweenOptions::Inclusive::LEFT);
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[1,1,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,false,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,false,null,false,false]"));
+  BetweenOptions InclusiveRight(BetweenOptions::Inclusive::RIGHT);
+  ValidateBetween<TypeParam>(
+      InclusiveRight, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[1,1,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,true,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,true,null,false,false]"));
+  BetweenOptions InclusiveNeither(BetweenOptions::Inclusive::NEITHER);
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[1,1,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,false,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,false,null,false,false]"));
+}
+
+template <typename Type>
+struct BetweenRandomNumeric {

Review comment:
       Done for Between. Issue created for other comparison functions. https://issues.apache.org/jira/browse/ARROW-15396




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

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

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



[GitHub] [arrow] pitrou commented on a change in pull request #11882: ARROW-9843: [C++][Python] Implement Between ternary kernel and Python bindings

Posted by GitBox <gi...@apache.org>.
pitrou commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r786076989



##########
File path: cpp/src/arrow/compute/kernels/scalar_compare.cc
##########
@@ -156,39 +212,52 @@ struct Maximum {
   }
 };
 
+// Check if timestamp timezones are comparable (either all are empty or none is).
+Status CheckCompareTimestamps(const ExecBatch& batch) {
+  if (batch.num_values() > 0) {
+    for (int i = 0; i < batch.num_values() - 1; ++i) {
+      const auto& tsi = checked_cast<const TimestampType&>(*batch[i].type());
+      for (int j = i + 1; j < batch.num_values(); ++j) {

Review comment:
       What becomes 1?




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

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

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



[GitHub] [arrow] bkmgit commented on a change in pull request #11882: ARROW-9843: [C++][Python] Implement Between ternary kernel and Python bindings

Posted by GitBox <gi...@apache.org>.
bkmgit commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r787610184



##########
File path: cpp/src/arrow/compute/kernels/scalar_compare_test.cc
##########
@@ -1865,5 +1867,683 @@ TEST(TestMaxElementWiseMinElementWise, CommonTemporal) {
               ResultWith(ScalarFromJSON(date64(), "86400000")));
 }
 
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const Datum& val, const Datum& lhs,
+                            const Datum& rhs, const Datum& expected) {
+  ASSERT_OK_AND_ASSIGN(Datum result, Between(val, lhs, rhs, options, nullptr));
+  AssertArraysEqual(*expected.make_array(), *result.make_array(),
+                    /*verbose=*/true);
+}
+
+template <typename ArrowType>
+void ValidateBetween(BetweenOptions options, const Datum& val, const Datum& lhs,
+                     const Datum& rhs) {
+  CompareOperator lhs_val;
+  CompareOperator val_rhs;
+  BetweenOptions::Inclusive include_endpoints = options.inclusive;
+
+  if (include_endpoints == BetweenOptions::Inclusive::NEITHER) {
+    lhs_val = LESS;
+    val_rhs = LESS;
+  } else if (include_endpoints == BetweenOptions::Inclusive::LEFT) {
+    lhs_val = LESS_EQUAL;
+    val_rhs = LESS;
+  } else if (include_endpoints == BetweenOptions::Inclusive::RIGHT) {
+    lhs_val = LESS;
+    val_rhs = LESS_EQUAL;
+  } else {
+    lhs_val = LESS_EQUAL;
+    val_rhs = LESS_EQUAL;
+  }
+
+  ASSERT_OK_AND_ASSIGN(Datum resultl,
+                       CallFunction(CompareOperatorToFunctionName(lhs_val), {lhs, val}));
+  ASSERT_OK_AND_ASSIGN(Datum resultr,
+                       CallFunction(CompareOperatorToFunctionName(val_rhs), {val, rhs}));
+  ASSERT_OK_AND_ASSIGN(Datum expected, CallFunction("and", {resultl, resultr}));
+
+  ValidateBetween<ArrowType>(options, val, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+class TestNumericBetweenKernel : public ::testing::Test {};
+
+TYPED_TEST_SUITE(TestNumericBetweenKernel, NumericArrowTypes);
+TYPED_TEST(TestNumericBetweenKernel, SimpleBetweenArrayScalarScalar) {
+  using ScalarType = typename TypeTraits<TypeParam>::ScalarType;
+  using CType = typename TypeTraits<TypeParam>::CType;
+
+  Datum zero(std::make_shared<ScalarType>(CType(0)));
+  Datum four(std::make_shared<ScalarType>(CType(4)));
+  Datum null(std::make_shared<ScalarType>());
+  BetweenOptions InclusiveBoth(BetweenOptions::Inclusive::BOTH);
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"), zero,
+      four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      zero, four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,1,2,2]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[1,1,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,3,4,5]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[1,1,1,1,1,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,1,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null,0,1,1]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), null, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, null,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  BetweenOptions InclusiveLeft(BetweenOptions::Inclusive::LEFT);
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"), zero,
+      four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      zero, four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,1,2,2]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[1,1,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,3,4,5]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[1,1,1,1,0,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,0,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null,0,1,1]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), null, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, null,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  BetweenOptions InclusiveRight(BetweenOptions::Inclusive::RIGHT);
+  ValidateBetween<TypeParam>(
+      InclusiveRight, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"), zero,
+      four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      zero, four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,1,2,2]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,0,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,3,4,5]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,1,1,1,1,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,1,1,1,1,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null,0,1,1]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null,0,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), null, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, null,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  BetweenOptions InclusiveNeither(BetweenOptions::Inclusive::NEITHER);
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      zero, four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      zero, four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,1,2,2]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,0,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,3,4,5]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,1,1,1,0,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,0,1,1,1,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null,0,1,1]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null,0,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), null, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, null,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+}
+
+TYPED_TEST(TestNumericBetweenKernel, SimpleBetweenScalarArrayArray) {
+  using ScalarType = typename TypeTraits<TypeParam>::ScalarType;
+  using CType = typename TypeTraits<TypeParam>::CType;
+
+  Datum one(std::make_shared<ScalarType>(CType(1)));
+
+  BetweenOptions InclusiveBoth(BetweenOptions::Inclusive::BOTH);
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, one, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,true,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,true,null,false,false]"));
+  BetweenOptions InclusiveLeft(BetweenOptions::Inclusive::LEFT);
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, one, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,true,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,true,null,false,false]"));
+  BetweenOptions InclusiveRight(BetweenOptions::Inclusive::RIGHT);
+  ValidateBetween<TypeParam>(
+      InclusiveRight, one, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,false,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,false,null,false,false]"));
+  BetweenOptions InclusiveNeither(BetweenOptions::Inclusive::NEITHER);
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, one, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,false,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,false,null,false,false]"));
+}
+
+TYPED_TEST(TestNumericBetweenKernel, SimpleBetweenArrayArrayArray) {
+  BetweenOptions InclusiveBoth(BetweenOptions::Inclusive::BOTH);
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[1,1,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,true,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,true,null,false,false]"));
+  BetweenOptions InclusiveLeft(BetweenOptions::Inclusive::LEFT);
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[1,1,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,false,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,false,null,false,false]"));
+  BetweenOptions InclusiveRight(BetweenOptions::Inclusive::RIGHT);
+  ValidateBetween<TypeParam>(
+      InclusiveRight, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[1,1,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,true,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,true,null,false,false]"));
+  BetweenOptions InclusiveNeither(BetweenOptions::Inclusive::NEITHER);
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[1,1,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,false,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,false,null,false,false]"));
+}

Review comment:
       Added scalar-scalar-scalar.




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

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

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



[GitHub] [arrow] bkmgit commented on pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
bkmgit commented on pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#issuecomment-1013734860


   @github-actions  autotune


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

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

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



[GitHub] [arrow] bkmgit commented on a change in pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
bkmgit commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r767459839



##########
File path: cpp/src/arrow/compute/kernels/scalar_compare.cc
##########
@@ -786,6 +878,10 @@ const FunctionDoc max_element_wise_doc{
     {"*args"},
     "ElementWiseAggregateOptions"};
 
+const FunctionDoc between_doc{"Check if values are in a range x <= y <= z",

Review comment:
       Can change this to `x <= y <= z`.




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

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

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



[GitHub] [arrow] bkmgit commented on a change in pull request #11882: ARROW-9843: [C++][Python] Implement Between ternary kernel and Python bindings

Posted by GitBox <gi...@apache.org>.
bkmgit commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r791361654



##########
File path: cpp/src/arrow/compute/kernels/scalar_compare_test.cc
##########
@@ -1850,5 +1851,154 @@ TEST(TestMaxElementWiseMinElementWise, CommonTemporal) {
               ResultWith(ScalarFromJSON(date64(), "86400000")));
 }
 
+template <typename ArrowType>
+static void ValidateBetween(const Datum& val, const Datum& lhs, const Datum& rhs,
+                            const Datum& expected) {
+  ASSERT_OK_AND_ASSIGN(Datum result, Between(val, lhs, rhs));
+  AssertArraysEqual(*expected.make_array(), *result.make_array(),
+                    /*verbose=*/true);
+}
+
+template <typename ArrowType>
+static void ValidateBetween(const char* value_str, const Datum& lhs, const Datum& rhs,
+                            const char* expected_str) {
+  auto value = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), value_str);
+  auto expected = ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), expected_str);
+  ValidateBetween<ArrowType>(value, lhs, rhs, expected);
+}
+
+template <>
+void ValidateBetween<StringType>(const char* value_str, const Datum& lhs,
+                                 const Datum& rhs, const char* expected_str) {
+  auto value = ArrayFromJSON(utf8(), value_str);
+  auto expected = ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), expected_str);
+  ValidateBetween<StringType>(value, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+class TestNumericBetweenKernel : public ::testing::Test {};
+
+TYPED_TEST_SUITE(TestNumericBetweenKernel, NumericArrowTypes);
+TYPED_TEST(TestNumericBetweenKernel, SimpleBetweenArrayScalarScalar) {
+  using ScalarType = typename TypeTraits<TypeParam>::ScalarType;
+  using CType = typename TypeTraits<TypeParam>::CType;
+
+  Datum zero(std::make_shared<ScalarType>(CType(0)));
+  Datum four(std::make_shared<ScalarType>(CType(4)));
+  ValidateBetween<TypeParam>("[]", zero, four, "[]");
+  ValidateBetween<TypeParam>("[null]", zero, four, "[null]");
+  ValidateBetween<TypeParam>("[0,0,1,1,2,2]", zero, four, "[0,0,1,1,1,1]");
+  ValidateBetween<TypeParam>("[0,1,2,3,4,5]", zero, four, "[0,1,1,1,0,0]");
+  ValidateBetween<TypeParam>("[5,4,3,2,1,0]", zero, four, "[0,0,1,1,1,0]");
+  ValidateBetween<TypeParam>("[null,0,1,1]", zero, four, "[null,0,1,1]");
+}
+
+TEST(TestSimpleBetweenKernel, SimpleStringTest) {

Review comment:
       Random tests are helpful, but some fixed tests which differentiate between < and <= are useful since random tests will not do this often. Also good to have some utf8 characters other than ASCII.




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

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

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



[GitHub] [arrow] bkmgit commented on a change in pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
bkmgit commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r782684990



##########
File path: cpp/src/arrow/compute/api_scalar.h
##########
@@ -1350,5 +1371,24 @@ ARROW_EXPORT Result<Datum> AssumeTimezone(const Datum& values,
                                           AssumeTimezoneOptions options,
                                           ExecContext* ctx = NULLPTR);
 
+/// \brief Between compares each element in `values`
+/// with `left` as a lower bound and 'right' as an upper bound
+///
+/// \param[in] values input to compare between left and right
+/// \param[in] left used as the lower bound for comparison
+/// \param[in] right used as the upper bound for comparison
+/// \param[in] options for bounds, default is inclusive of both, other

Review comment:
       Thanks, fixed.




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

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

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



[GitHub] [arrow] bkmgit commented on a change in pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
bkmgit commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r782791264



##########
File path: cpp/src/arrow/util/bit_block_counter.h
##########
@@ -87,6 +87,147 @@ struct BitBlockOrNot<bool> {
   static bool Call(bool left, bool right) { return left || !right; }
 };
 
+// Three Arguments
+template <typename T>
+struct BitBlockAndAnd {
+  static T Call(T left, T mid, T right) { return left & mid & right; }
+};
+
+template <>
+struct BitBlockAndAnd<bool> {
+  static bool Call(bool left, bool mid, bool right) { return left && mid && right; }

Review comment:
       Done.




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

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

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



[GitHub] [arrow] bkmgit commented on a change in pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
bkmgit commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r782966789



##########
File path: cpp/src/arrow/compute/kernels/scalar_compare_test.cc
##########
@@ -1850,5 +1851,154 @@ TEST(TestMaxElementWiseMinElementWise, CommonTemporal) {
               ResultWith(ScalarFromJSON(date64(), "86400000")));
 }
 
+template <typename ArrowType>
+static void ValidateBetween(const Datum& val, const Datum& lhs, const Datum& rhs,
+                            const Datum& expected) {
+  ASSERT_OK_AND_ASSIGN(Datum result, Between(val, lhs, rhs));
+  AssertArraysEqual(*expected.make_array(), *result.make_array(),
+                    /*verbose=*/true);
+}
+
+template <typename ArrowType>
+static void ValidateBetween(const char* value_str, const Datum& lhs, const Datum& rhs,
+                            const char* expected_str) {
+  auto value = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), value_str);
+  auto expected = ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), expected_str);
+  ValidateBetween<ArrowType>(value, lhs, rhs, expected);
+}
+
+template <>
+void ValidateBetween<StringType>(const char* value_str, const Datum& lhs,
+                                 const Datum& rhs, const char* expected_str) {
+  auto value = ArrayFromJSON(utf8(), value_str);
+  auto expected = ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), expected_str);
+  ValidateBetween<StringType>(value, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+class TestNumericBetweenKernel : public ::testing::Test {};
+
+TYPED_TEST_SUITE(TestNumericBetweenKernel, NumericArrowTypes);
+TYPED_TEST(TestNumericBetweenKernel, SimpleBetweenArrayScalarScalar) {
+  using ScalarType = typename TypeTraits<TypeParam>::ScalarType;
+  using CType = typename TypeTraits<TypeParam>::CType;
+
+  Datum zero(std::make_shared<ScalarType>(CType(0)));
+  Datum four(std::make_shared<ScalarType>(CType(4)));
+  ValidateBetween<TypeParam>("[]", zero, four, "[]");
+  ValidateBetween<TypeParam>("[null]", zero, four, "[null]");
+  ValidateBetween<TypeParam>("[0,0,1,1,2,2]", zero, four, "[0,0,1,1,1,1]");
+  ValidateBetween<TypeParam>("[0,1,2,3,4,5]", zero, four, "[0,1,1,1,0,0]");
+  ValidateBetween<TypeParam>("[5,4,3,2,1,0]", zero, four, "[0,0,1,1,1,0]");
+  ValidateBetween<TypeParam>("[null,0,1,1]", zero, four, "[null,0,1,1]");
+}
+
+TEST(TestSimpleBetweenKernel, SimpleStringTest) {
+  using ScalarType = typename TypeTraits<StringType>::ScalarType;
+  auto l = Datum(std::make_shared<ScalarType>("abc"));
+  auto r = Datum(std::make_shared<ScalarType>("zzz"));
+  ValidateBetween<StringType>("[]", l, r, "[]");
+  ValidateBetween<StringType>("[null]", l, r, "[null]");
+  ValidateBetween<StringType>(R"(["aaa", "aaaa", "ccc", "z"])", l, r,
+                              R"([false, false, true, true])");
+  ValidateBetween<StringType>(R"(["a", "aaaa", "c", "z"])", l, r,
+                              R"([false, false, true, true])");
+  ValidateBetween<StringType>(R"(["a", "aaaa", "fff", "zzzz"])", l, r,
+                              R"([false, false, true, false])");
+  ValidateBetween<StringType>(R"(["abd", null, null, "zzx"])", l, r,
+                              R"([true, null, null, true])");
+}
+
+TEST(TestSimpleBetweenKernel, SimpleTimestampTest) {
+  using ScalarType = typename TypeTraits<TimestampType>::ScalarType;
+  auto checkTimestampArray = [](std::shared_ptr<DataType> type, const char* input_str,
+                                const Datum& lhs, const Datum& rhs,
+                                const char* expected_str) {
+    auto value = ArrayFromJSON(type, input_str);
+    auto expected = ArrayFromJSON(boolean(), expected_str);
+    ValidateBetween<TimestampType>(value, lhs, rhs, expected);
+  };
+  auto unit = TimeUnit::SECOND;
+  auto l = Datum(std::make_shared<ScalarType>(923184000, timestamp(unit)));
+  auto r = Datum(std::make_shared<ScalarType>(1602032602, timestamp(unit)));
+  checkTimestampArray(timestamp(unit), "[]", l, r, "[]");
+  checkTimestampArray(timestamp(unit), "[null]", l, r, "[null]");
+  checkTimestampArray(timestamp(unit), R"(["1970-01-01","2000-02-29","1900-02-28"])", l,
+                      r, "[false,true,false]");
+  checkTimestampArray(timestamp(unit), R"(["1970-01-01","2000-02-29","2004-02-28"])", l,
+                      r, "[false,true,true]");
+  checkTimestampArray(timestamp(unit), R"(["2018-01-01","1999-04-04","1900-02-28"])", l,
+                      r, "[true,false,false]");
+}
+
+TYPED_TEST(TestNumericBetweenKernel, SimpleBetweenArrayArrayArray) {
+  ValidateBetween<TypeParam>(
+      "[]", ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"), "[]");
+  ValidateBetween<TypeParam>(
+      "[null]", ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"), "[null]");
+  ValidateBetween<TypeParam>(
+      "[1,1,2,2,2]",
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      "[true,true,false,false,false]");
+  ValidateBetween<TypeParam>(
+      "[1,1,2,2,2,2]",
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,2,5,5]"),
+      "[true,true,false,null,false,false]");
+}
+
+TEST(TestSimpleBetweenKernel, StringArrayArrayArrayTest) {
+  ValidateBetween<StringType>(
+      R"(["david","hello","world"])",
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["adam","hi","whirl"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                    R"(["robert","goeiemoreen","whirlwind"])"),
+      "[true, false, false]");
+  ValidateBetween<StringType>(
+      R"(["x","a","f"])",
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["w","a","e"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["z","a","g"])"),
+      "[true, false, true]");
+  ValidateBetween<StringType>(
+      R"(["block","bit","binary"])",
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                    R"(["bit","nibble","ternary"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["word","d","xyz"])"),
+      "[true, false, false]");
+  ValidateBetween<StringType>(R"(["Ayumi","アユミ","王梦莹"])",
+                              ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                            R"(["たなか","あゆみ","歩美"])"),
+                              ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                            R"(["李平之","田中","たなか"])"),
+                              "[false, true, false]");
+}
+
+TEST(TestSimpleBetweenKernel, TimestampArrayArrayArrayTest) {

Review comment:
       @kou @jorisvandenbossche Raised an issue to use a tool to measure testing coverage https://issues.apache.org/jira/browse/ARROW-15308 - some of the Python and other language binding tests will duplicate C++ tests. Possibly some test rationalization is required since one wants to also minimize testing time.




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

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

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



[GitHub] [arrow] kou commented on a change in pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
kou commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r776868111



##########
File path: cpp/src/arrow/compute/kernels/scalar_compare.cc
##########
@@ -68,6 +72,50 @@ struct GreaterEqual {
   }
 };
 
+struct BetweenLessEqualLessEqual {
+  template <typename T, typename Arg0, typename Arg1, typename Arg2>
+  static constexpr T Call(KernelContext*, const Arg0& middle, const Arg1& left,
+                          const Arg2& right, Status*) {
+    static_assert(std::is_same<T, bool>::value && std::is_same<Arg0, Arg1>::value &&
+                      std::is_same<Arg1, Arg2>::value,
+                  "");
+    return (left <= middle) && (middle <= right);
+  }
+};
+
+struct BetweenLessThanLessEqual {
+  template <typename T, typename Arg0, typename Arg1, typename Arg2>
+  static constexpr T Call(KernelContext*, const Arg0& middle, const Arg1& left,
+                          const Arg2& right, Status*) {
+    static_assert(std::is_same<T, bool>::value && std::is_same<Arg0, Arg1>::value &&
+                      std::is_same<Arg1, Arg2>::value,
+                  "");
+    return (left < middle) && (middle <= right);
+  }
+};
+
+struct BetweenLessEqualLessThan {
+  template <typename T, typename Arg0, typename Arg1, typename Arg2>
+  static constexpr T Call(KernelContext*, const Arg0& middle, const Arg1& left,
+                          const Arg2& right, Status*) {
+    static_assert(std::is_same<T, bool>::value && std::is_same<Arg0, Arg1>::value &&
+                      std::is_same<Arg1, Arg2>::value,
+                  "");
+    return (left <= middle) && (middle < right);
+  }
+};
+
+struct BetweenLessThanLessThan {
+  template <typename T, typename Arg0, typename Arg1, typename Arg2>
+  static constexpr T Call(KernelContext*, const Arg0& middle, const Arg1& left,
+                          const Arg2& right, Status*) {
+    static_assert(std::is_same<T, bool>::value && std::is_same<Arg0, Arg1>::value &&
+                      std::is_same<Arg1, Arg2>::value,
+                  "");
+    return (left < middle) && (middle < right);
+  }
+};

Review comment:
       I also like the option (1) in https://github.com/apache/arrow/pull/11882#issuecomment-1002956781 . (One between function with options.)




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

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

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



[GitHub] [arrow] bkmgit commented on pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
bkmgit commented on pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#issuecomment-1003175374


   > If functions were to be grouped and the `options` parameter used to select a particular function, then expressing function-specific `options` would be difficult.
   
   It is actually not too bad. [ScalarAggregationOptions](https://arrow.apache.org/docs/cpp/api/compute.html#_CPPv4N5arrow7compute22ScalarAggregateOptionsE) has two parameters, `skip_nulls` and `min_count`. Adding options with a default that preserves existing behavior makes extending kernels and updating language bindings easier.


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

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

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



[GitHub] [arrow] edponce commented on a change in pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
edponce commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r776811241



##########
File path: cpp/src/arrow/compute/api_scalar.h
##########
@@ -316,6 +316,18 @@ struct ARROW_EXPORT CompareOptions {
   enum CompareOperator op;
 };
 
+class ARROW_EXPORT BetweenOptions : public FunctionOptions {
+ public:
+  enum Inclusiveness { BOTH, LEFT, RIGHT, NEITHER };

Review comment:
       Note: We will need to move this enum to global space as a scoped enum for `not_between` so that they both use the same options. This can be done in other PR.

##########
File path: cpp/src/arrow/compute/api_scalar.cc
##########
@@ -99,6 +99,30 @@ struct EnumTraits<compute::CompareOperator>
     return "<INVALID>";
   }
 };
+
+template <>
+struct EnumTraits<compute::BetweenOptions::Inclusiveness>
+    : BasicEnumTraits<compute::BetweenOptions::Inclusiveness,
+                      compute::BetweenOptions::Inclusiveness::BOTH,
+                      compute::BetweenOptions::Inclusiveness::LEFT,
+                      compute::BetweenOptions::Inclusiveness::RIGHT,
+                      compute::BetweenOptions::Inclusiveness::NEITHER> {
+  static std::string name() { return "BetweenOptions::Inclusiveness"; }
+  static std::string value_name(compute::BetweenOptions::Inclusiveness value) {
+    switch (value) {
+      case compute::BetweenOptions::Inclusiveness::BOTH:
+        return "BOTH";
+      case compute::BetweenOptions::Inclusiveness::LEFT:
+        return "LEFT";
+      case compute::BetweenOptions::Inclusiveness::RIGHT:
+        return "RIGHT";
+      case compute::BetweenOptions::Inclusiveness::NEITHER:

Review comment:
       `Inclusiveness` --> `Inclusive` so that it matches Pandas and is less verbose.

##########
File path: cpp/src/arrow/util/bit_block_counter.h
##########
@@ -424,6 +565,141 @@ class ARROW_EXPORT OptionalBinaryBitBlockCounter {
   }
 };
 
+/// \brief A class that computes popcounts on the result of bitwise operations
+/// between three bitmaps, 64 bits at a time. A 64-bit word is loaded from each
+/// bitmap, then the popcount is computed on e.g. the bitwise-and of the three
+/// words.
+class ARROW_EXPORT TernaryBitBlockCounter {
+ public:
+  TernaryBitBlockCounter(const uint8_t* left_bitmap, int64_t left_offset,
+                         const uint8_t* mid_bitmap, int64_t mid_offset,
+                         const uint8_t* right_bitmap, int64_t right_offset,
+                         int64_t length)
+      : left_bitmap_(util::MakeNonNull(left_bitmap) + left_offset / 8),
+        left_offset_(left_offset % 8),
+        mid_bitmap_(util::MakeNonNull(mid_bitmap) + mid_offset / 8),
+        mid_offset_(mid_offset % 8),
+        right_bitmap_(util::MakeNonNull(right_bitmap) + right_offset / 8),
+        right_offset_(right_offset % 8),
+        bits_remaining_(length) {}
+
+  /// \brief Return the popcount of the bitwise-and of the next run of
+  /// available bits, up to 64. The returned pair contains the size of run and
+  /// the number of true values. The last block will have a length less than 64
+  /// if the bitmap length is not a multiple of 64, and will return 0-length
+  /// blocks in subsequent invocations.
+  BitBlockCount NextAndAndWord() { return NextWord<detail::BitBlockAndAnd>(); }

Review comment:
       The only version used is `NextAndAndWord`, so you can remove the other ones.

##########
File path: cpp/src/arrow/compute/kernels/scalar_compare.cc
##########
@@ -746,6 +787,155 @@ std::shared_ptr<ScalarFunction> MakeScalarMinMax(std::string name,
   return func;
 }
 
+template <typename Op>
+void AddIntegerBetween(const std::shared_ptr<DataType>& ty, ScalarFunction* func) {
+  auto exec = GeneratePhysicalInteger<ScalarTernaryEqualTypes, BooleanType, Op>(*ty);
+  DCHECK_OK(func->AddKernel({ty, ty, ty}, boolean(), std::move(exec)));
+}
+
+template <typename InType, typename Op>
+void AddGenericBetween(const std::shared_ptr<DataType>& ty, ScalarFunction* func) {
+  DCHECK_OK(func->AddKernel({ty, ty, ty}, boolean(),
+                            ScalarTernaryEqualTypes<BooleanType, InType, Op>::Exec));
+}
+
+template <typename OutType, typename ArgType, typename Op>
+struct BetweenTimestamps : public ScalarTernaryEqualTypes<OutType, ArgType, Op> {
+  using Base = ScalarTernaryEqualTypes<OutType, ArgType, Op>;
+
+  static Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    const auto& var = checked_cast<const TimestampType&>(*batch[0].type());
+    const auto& lhs = checked_cast<const TimestampType&>(*batch[1].type());
+    const auto& rhs = checked_cast<const TimestampType&>(*batch[2].type());
+    if ((var.timezone().empty() != lhs.timezone().empty()) ||
+        (var.timezone().empty() != rhs.timezone().empty()) ||
+        (lhs.timezone().empty() != rhs.timezone().empty())) {
+      return Status::Invalid("Cannot compare timestamps with and without timezones.");
+    }
+    return Base::Exec(ctx, batch, out);
+  }
+};
+
+template <typename Op>
+std::shared_ptr<ScalarFunction> MakeBetweenFunction(std::string name,
+                                                    const FunctionDoc* doc) {
+  auto func = std::make_shared<CompareFunction>(name, Arity::Ternary(), doc);
+
+  DCHECK_OK(func->AddKernel(
+      {boolean(), boolean(), boolean()}, boolean(),
+      ScalarTernary<BooleanType, BooleanType, BooleanType, BooleanType, Op>::Exec));
+
+  for (const std::shared_ptr<DataType>& ty : IntTypes()) {
+    AddIntegerBetween<Op>(ty, func.get());
+  }
+
+  AddIntegerBetween<Op>(date32(), func.get());
+  AddIntegerBetween<Op>(date64(), func.get());
+
+  AddGenericBetween<FloatType, Op>(float32(), func.get());
+  AddGenericBetween<DoubleType, Op>(float64(), func.get());
+
+  // Add timestamp kernels
+  for (auto unit : TimeUnit::values()) {
+    InputType in_type(match::TimestampTypeUnit(unit));
+    DCHECK_OK(func->AddKernel({in_type, in_type, in_type}, boolean(),
+                              BetweenTimestamps<BooleanType, TimestampType, Op>::Exec));
+  }
+
+  // Duration
+  for (auto unit : TimeUnit::values()) {
+    InputType in_type(match::DurationTypeUnit(unit));
+    auto exec =
+        GeneratePhysicalInteger<ScalarTernaryEqualTypes, BooleanType, Op>(int64());
+    DCHECK_OK(func->AddKernel({in_type, in_type, in_type}, boolean(), std::move(exec)));
+  }
+
+  // Time32 and Time64
+  for (auto unit : {TimeUnit::SECOND, TimeUnit::MILLI}) {
+    InputType in_type(match::Time32TypeUnit(unit));
+    auto exec =
+        GeneratePhysicalInteger<ScalarTernaryEqualTypes, BooleanType, Op>(int32());
+    DCHECK_OK(func->AddKernel({in_type, in_type, in_type}, boolean(), std::move(exec)));
+  }
+  for (auto unit : {TimeUnit::MICRO, TimeUnit::NANO}) {
+    InputType in_type(match::Time64TypeUnit(unit));
+    auto exec =
+        GeneratePhysicalInteger<ScalarTernaryEqualTypes, BooleanType, Op>(int64());
+    DCHECK_OK(func->AddKernel({in_type, in_type, in_type}, boolean(), std::move(exec)));
+  }
+
+  for (const std::shared_ptr<DataType>& ty : BaseBinaryTypes()) {
+    auto exec = GenerateVarBinaryBase<ScalarTernaryEqualTypes, BooleanType, Op>(*ty);
+    DCHECK_OK(func->AddKernel(
+        {InputType::Array(ty), InputType::Scalar(ty), InputType::Scalar(ty)}, boolean(),
+        exec));
+    DCHECK_OK(func->AddKernel(
+        {InputType::Array(ty), InputType::Array(ty), InputType::Array(ty)}, boolean(),
+        std::move(exec)));
+  }
+
+  for (const auto id : {Type::DECIMAL128, Type::DECIMAL256}) {
+    auto exec = GenerateDecimal<ScalarTernaryEqualTypes, BooleanType, Op>(id);
+    DCHECK_OK(func->AddKernel({InputType(id), InputType(id), InputType(id)}, boolean(),
+                              std::move(exec)));
+  }
+
+  {
+    auto exec = ScalarTernaryEqualTypes<BooleanType, FixedSizeBinaryType, Op>::Exec;
+    auto ty = InputType(Type::FIXED_SIZE_BINARY);
+    DCHECK_OK(func->AddKernel({ty, ty, ty}, boolean(), std::move(exec)));
+  }
+
+  return func;
+}
+
+const BetweenOptions* GetDefaultBetweenOptions() {
+  static const auto kBetweenOptions = BetweenOptions::Defaults();
+  return &kBetweenOptions;
+}
+
+const FunctionDoc between_doc{"Check if values are in a range, val betwen a and b",
+                              ("A null on either side emits a null comparison result.\n"
+                               "options are used to specify if the endpoints are\n"
+                               "inclusive, possible values are NEITHER (a<val<b),\n"
+                               "LEFT (a<=val<b), RIGHT (a<val<=b), and the default\n"
+                               "if not specified BOTH (a<=val<=b)"),
+                              {"val", "a", "b"}};
+
+class BetweenMetaFunction : public MetaFunction {
+ public:
+  BetweenMetaFunction()
+      : MetaFunction("between", Arity::Ternary(), &between_doc,
+                     GetDefaultBetweenOptions()) {}
+
+  Result<Datum> ExecuteImpl(const std::vector<Datum>& args,
+                            const FunctionOptions* options,
+                            ExecContext* ctx) const override {
+    const BetweenOptions& between_options = static_cast<const BetweenOptions&>(*options);
+    Datum result;

Review comment:
       I disagree with this approach. It defeats the purpose of having the `BetweenOptions` because they are being consumed by `CallFunction`.
   
   I will take a stab at supporting internal dispatching based on `BetweenOptions` and submit a PR to your branch. Up to now this is looking good but needs a bit of tweaking.

##########
File path: cpp/src/arrow/util/bit_block_counter.h
##########
@@ -87,6 +87,147 @@ struct BitBlockOrNot<bool> {
   static bool Call(bool left, bool right) { return left || !right; }
 };
 
+// Three Arguments
+template <typename T>
+struct BitBlockAndAnd {
+  static T Call(T left, T mid, T right) { return left & mid & right; }
+};
+
+template <>
+struct BitBlockAndAnd<bool> {
+  static bool Call(bool left, bool mid, bool right) { return left && mid && right; }

Review comment:
       Actually, the only combination used in this PR is `BitBlockAndAnd`, so you can remove all the other ones. There is no reason to have dead code. Also, ARROW-15220 got merged so you can see the new form for these operators. Note: It will change [their use as template parameters](https://github.com/apache/arrow/blob/master/cpp/src/arrow/util/bit_block_counter.h#L287-L288) and how [`Call` is invoked](https://github.com/apache/arrow/blob/master/cpp/src/arrow/util/bit_block_counter.h#L322).




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

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

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



[GitHub] [arrow] bkmgit commented on a change in pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
bkmgit commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r776936853



##########
File path: cpp/src/arrow/compute/kernels/scalar_compare.cc
##########
@@ -746,6 +787,155 @@ std::shared_ptr<ScalarFunction> MakeScalarMinMax(std::string name,
   return func;
 }
 
+template <typename Op>
+void AddIntegerBetween(const std::shared_ptr<DataType>& ty, ScalarFunction* func) {
+  auto exec = GeneratePhysicalInteger<ScalarTernaryEqualTypes, BooleanType, Op>(*ty);
+  DCHECK_OK(func->AddKernel({ty, ty, ty}, boolean(), std::move(exec)));
+}
+
+template <typename InType, typename Op>
+void AddGenericBetween(const std::shared_ptr<DataType>& ty, ScalarFunction* func) {
+  DCHECK_OK(func->AddKernel({ty, ty, ty}, boolean(),
+                            ScalarTernaryEqualTypes<BooleanType, InType, Op>::Exec));
+}
+
+template <typename OutType, typename ArgType, typename Op>
+struct BetweenTimestamps : public ScalarTernaryEqualTypes<OutType, ArgType, Op> {
+  using Base = ScalarTernaryEqualTypes<OutType, ArgType, Op>;
+
+  static Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    const auto& var = checked_cast<const TimestampType&>(*batch[0].type());
+    const auto& lhs = checked_cast<const TimestampType&>(*batch[1].type());
+    const auto& rhs = checked_cast<const TimestampType&>(*batch[2].type());
+    if ((var.timezone().empty() != lhs.timezone().empty()) ||
+        (var.timezone().empty() != rhs.timezone().empty()) ||
+        (lhs.timezone().empty() != rhs.timezone().empty())) {
+      return Status::Invalid("Cannot compare timestamps with and without timezones.");
+    }
+    return Base::Exec(ctx, batch, out);
+  }
+};
+
+template <typename Op>
+std::shared_ptr<ScalarFunction> MakeBetweenFunction(std::string name,
+                                                    const FunctionDoc* doc) {
+  auto func = std::make_shared<CompareFunction>(name, Arity::Ternary(), doc);
+
+  DCHECK_OK(func->AddKernel(
+      {boolean(), boolean(), boolean()}, boolean(),
+      ScalarTernary<BooleanType, BooleanType, BooleanType, BooleanType, Op>::Exec));
+
+  for (const std::shared_ptr<DataType>& ty : IntTypes()) {
+    AddIntegerBetween<Op>(ty, func.get());
+  }
+
+  AddIntegerBetween<Op>(date32(), func.get());
+  AddIntegerBetween<Op>(date64(), func.get());
+
+  AddGenericBetween<FloatType, Op>(float32(), func.get());
+  AddGenericBetween<DoubleType, Op>(float64(), func.get());
+
+  // Add timestamp kernels
+  for (auto unit : TimeUnit::values()) {
+    InputType in_type(match::TimestampTypeUnit(unit));
+    DCHECK_OK(func->AddKernel({in_type, in_type, in_type}, boolean(),
+                              BetweenTimestamps<BooleanType, TimestampType, Op>::Exec));
+  }
+
+  // Duration
+  for (auto unit : TimeUnit::values()) {
+    InputType in_type(match::DurationTypeUnit(unit));
+    auto exec =
+        GeneratePhysicalInteger<ScalarTernaryEqualTypes, BooleanType, Op>(int64());
+    DCHECK_OK(func->AddKernel({in_type, in_type, in_type}, boolean(), std::move(exec)));
+  }
+
+  // Time32 and Time64
+  for (auto unit : {TimeUnit::SECOND, TimeUnit::MILLI}) {
+    InputType in_type(match::Time32TypeUnit(unit));
+    auto exec =
+        GeneratePhysicalInteger<ScalarTernaryEqualTypes, BooleanType, Op>(int32());
+    DCHECK_OK(func->AddKernel({in_type, in_type, in_type}, boolean(), std::move(exec)));
+  }
+  for (auto unit : {TimeUnit::MICRO, TimeUnit::NANO}) {
+    InputType in_type(match::Time64TypeUnit(unit));
+    auto exec =
+        GeneratePhysicalInteger<ScalarTernaryEqualTypes, BooleanType, Op>(int64());
+    DCHECK_OK(func->AddKernel({in_type, in_type, in_type}, boolean(), std::move(exec)));
+  }
+
+  for (const std::shared_ptr<DataType>& ty : BaseBinaryTypes()) {
+    auto exec = GenerateVarBinaryBase<ScalarTernaryEqualTypes, BooleanType, Op>(*ty);
+    DCHECK_OK(func->AddKernel(
+        {InputType::Array(ty), InputType::Scalar(ty), InputType::Scalar(ty)}, boolean(),
+        exec));
+    DCHECK_OK(func->AddKernel(
+        {InputType::Array(ty), InputType::Array(ty), InputType::Array(ty)}, boolean(),
+        std::move(exec)));
+  }
+
+  for (const auto id : {Type::DECIMAL128, Type::DECIMAL256}) {
+    auto exec = GenerateDecimal<ScalarTernaryEqualTypes, BooleanType, Op>(id);
+    DCHECK_OK(func->AddKernel({InputType(id), InputType(id), InputType(id)}, boolean(),
+                              std::move(exec)));
+  }
+
+  {
+    auto exec = ScalarTernaryEqualTypes<BooleanType, FixedSizeBinaryType, Op>::Exec;
+    auto ty = InputType(Type::FIXED_SIZE_BINARY);
+    DCHECK_OK(func->AddKernel({ty, ty, ty}, boolean(), std::move(exec)));
+  }
+
+  return func;
+}
+
+const BetweenOptions* GetDefaultBetweenOptions() {
+  static const auto kBetweenOptions = BetweenOptions::Defaults();
+  return &kBetweenOptions;
+}
+
+const FunctionDoc between_doc{"Check if values are in a range, val betwen a and b",
+                              ("A null on either side emits a null comparison result.\n"
+                               "options are used to specify if the endpoints are\n"
+                               "inclusive, possible values are NEITHER (a<val<b),\n"
+                               "LEFT (a<=val<b), RIGHT (a<val<=b), and the default\n"
+                               "if not specified BOTH (a<=val<=b)"),
+                              {"val", "a", "b"}};
+
+class BetweenMetaFunction : public MetaFunction {
+ public:
+  BetweenMetaFunction()
+      : MetaFunction("between", Arity::Ternary(), &between_doc,
+                     GetDefaultBetweenOptions()) {}
+
+  Result<Datum> ExecuteImpl(const std::vector<Datum>& args,
+                            const FunctionOptions* options,
+                            ExecContext* ctx) const override {
+    const BetweenOptions& between_options = static_cast<const BetweenOptions&>(*options);
+    Datum result;

Review comment:
       Thanks for your feedback. Ok, on moving the functions to within the dispatch for between.  Underlying APIs should support natural integration into other programming languages and not force Pandas/Python approach, but also be easy to maintain and at least consistent for related functions. Would it make sense to then also use one compare function, but have functions on top of this for `greater`, `less` etc.? The `not_between` function could then be implemented by calling `between` and reversing the result.
   
   I understand that Pandas has lot of influence. In addition to databases, there are other libraries such as
   - [Ruby Daru](https://github.com/ankane/rover)
   - [R Dplyr](https://dplyr.tidyverse.org/reference/between.html)
   - [Hive Between](https://sqlandhadoop.com/hive-between/)
   - [Ruby Rover](https://github.com/ankane/rover)
   - [Ruby CSV table](https://ruby-doc.org/stdlib-3.1.0/libdoc/csv/rdoc/CSV/Table.html)
   - [Link compare](https://docs.microsoft.com/en-us/dotnet/visual-basic/language-reference/operators/comparison-operators)
   - [Apache Calcite Algebra](https://calcite.apache.org/docs/algebra.html)
   
   For many of these, chained operations is the way to implement between. It is unclear if one can write a Kernel generator that would minimize loading data from memory for chained operations - Calcite approach seems useful, but need to understand it better. Will ask on mailing list for thoughts on these kind of approaches since they would be something new.
   
   Finally, I have not done much performance measurement and optimization of these kernels, but this can probably be done once tests and relevant functionality are working.




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

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

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



[GitHub] [arrow] lidavidm commented on a change in pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
lidavidm commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r775913947



##########
File path: cpp/src/arrow/compute/kernels/scalar_compare.cc
##########
@@ -68,6 +72,50 @@ struct GreaterEqual {
   }
 };
 
+struct BetweenLessEqualLessEqual {
+  template <typename T, typename Arg0, typename Arg1, typename Arg2>
+  static constexpr T Call(KernelContext*, const Arg0& middle, const Arg1& left,
+                          const Arg2& right, Status*) {
+    static_assert(std::is_same<T, bool>::value && std::is_same<Arg0, Arg1>::value &&
+                      std::is_same<Arg1, Arg2>::value,
+                  "");
+    return (left <= middle) && (middle <= right);
+  }
+};
+
+struct BetweenLessThanLessEqual {
+  template <typename T, typename Arg0, typename Arg1, typename Arg2>
+  static constexpr T Call(KernelContext*, const Arg0& middle, const Arg1& left,
+                          const Arg2& right, Status*) {
+    static_assert(std::is_same<T, bool>::value && std::is_same<Arg0, Arg1>::value &&
+                      std::is_same<Arg1, Arg2>::value,
+                  "");
+    return (left < middle) && (middle <= right);
+  }
+};
+
+struct BetweenLessEqualLessThan {
+  template <typename T, typename Arg0, typename Arg1, typename Arg2>
+  static constexpr T Call(KernelContext*, const Arg0& middle, const Arg1& left,
+                          const Arg2& right, Status*) {
+    static_assert(std::is_same<T, bool>::value && std::is_same<Arg0, Arg1>::value &&
+                      std::is_same<Arg1, Arg2>::value,
+                  "");
+    return (left <= middle) && (middle < right);
+  }
+};
+
+struct BetweenLessThanLessThan {
+  template <typename T, typename Arg0, typename Arg1, typename Arg2>
+  static constexpr T Call(KernelContext*, const Arg0& middle, const Arg1& left,
+                          const Arg2& right, Status*) {
+    static_assert(std::is_same<T, bool>::value && std::is_same<Arg0, Arg1>::value &&
+                      std::is_same<Arg1, Arg2>::value,
+                  "");
+    return (left < middle) && (middle < right);
+  }
+};

Review comment:
       Sorry - why do we need a sort key for string comparisons? 




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

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

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



[GitHub] [arrow] edponce commented on a change in pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
edponce commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r770954431



##########
File path: cpp/src/arrow/compute/kernels/scalar_compare.cc
##########
@@ -68,6 +72,50 @@ struct GreaterEqual {
   }
 };
 
+struct BetweenLessEqualLessEqual {
+  template <typename T, typename Arg0, typename Arg1, typename Arg2>
+  static constexpr T Call(KernelContext*, const Arg0& middle, const Arg1& left,
+                          const Arg2& right, Status*) {
+    static_assert(std::is_same<T, bool>::value && std::is_same<Arg0, Arg1>::value &&
+                      std::is_same<Arg1, Arg2>::value,
+                  "");
+    return (left <= middle) && (middle <= right);
+  }
+};
+
+struct BetweenLessThanLessEqual {
+  template <typename T, typename Arg0, typename Arg1, typename Arg2>
+  static constexpr T Call(KernelContext*, const Arg0& middle, const Arg1& left,
+                          const Arg2& right, Status*) {
+    static_assert(std::is_same<T, bool>::value && std::is_same<Arg0, Arg1>::value &&
+                      std::is_same<Arg1, Arg2>::value,
+                  "");
+    return (left < middle) && (middle <= right);
+  }
+};
+
+struct BetweenLessEqualLessThan {
+  template <typename T, typename Arg0, typename Arg1, typename Arg2>
+  static constexpr T Call(KernelContext*, const Arg0& middle, const Arg1& left,
+                          const Arg2& right, Status*) {
+    static_assert(std::is_same<T, bool>::value && std::is_same<Arg0, Arg1>::value &&
+                      std::is_same<Arg1, Arg2>::value,
+                  "");
+    return (left <= middle) && (middle < right);
+  }
+};
+
+struct BetweenLessThanLessThan {
+  template <typename T, typename Arg0, typename Arg1, typename Arg2>
+  static constexpr T Call(KernelContext*, const Arg0& middle, const Arg1& left,
+                          const Arg2& right, Status*) {
+    static_assert(std::is_same<T, bool>::value && std::is_same<Arg0, Arg1>::value &&
+                      std::is_same<Arg1, Arg2>::value,
+                  "");
+    return (left < middle) && (middle < right);
+  }
+};

Review comment:
       I suggest to have a single `between` function that uses `FunctionOptions` to control its behavior. These different comparisons can be represented using an enum, which can then be used as a template parameter for compiler to dispatch accordingly.  See for example how [`round` function](https://github.com/apache/arrow/blob/master/cpp/src/arrow/compute/kernels/scalar_arithmetic.cc#L901) is implemented.




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

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

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



[GitHub] [arrow] lidavidm commented on a change in pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
lidavidm commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r782543512



##########
File path: cpp/src/arrow/compute/api_scalar.h
##########
@@ -1350,5 +1371,24 @@ ARROW_EXPORT Result<Datum> AssumeTimezone(const Datum& values,
                                           AssumeTimezoneOptions options,
                                           ExecContext* ctx = NULLPTR);
 
+/// \brief Between compares each element in `values`
+/// with `left` as a lower bound and 'right' as an upper bound
+///
+/// \param[in] values input to compare between left and right
+/// \param[in] left used as the lower bound for comparison
+/// \param[in] right used as the upper bound for comparison
+/// \param[in] options for bounds, default is inclusive of both, other

Review comment:
       nit: was this meant to say something like `inclusive of both endpoints`?

##########
File path: python/pyarrow/tests/test_compute.py
##########
@@ -1262,6 +1263,300 @@ def test_filter_null_type():
     assert len(table.filter(mask).column(0)) == 5
 
 
+@pytest.mark.parametrize("ty", ["inclusive"])
+def test_between_array_array_array(ty):
+    BetweenOptions = partial(pc.BetweenOptions)
+
+    val = pa.array([1, 1, 4, 3, 2, 6])
+    arr1 = pa.array([1, 1, 3, 4, None, 5])
+    arr2 = pa.array([1, 2, 4, None, 4, 7])
+
+    inclusive_and_expected = {
+        "both": [True, True, True, None, None, True],
+        "left": [False, True, False, None, None, True],
+        "right": [False, False, True, None, None, True],
+        "neither": [False, False, False, None, None, True],
+    }
+
+    for inclusive, expected in inclusive_and_expected.items():
+        options = BetweenOptions(inclusive=inclusive)
+        result = pc.between(val, arr1, arr2, options=options)
+        np.testing.assert_array_equal(result, pa.array(expected))

Review comment:
       Is there perhaps a way to leverage Pandas or NumPy as a reference point to combine some of these test cases?

##########
File path: docs/source/python/api/compute.rst
##########
@@ -158,6 +158,7 @@ they return ``null``.
    less
    less_equal
    not_equal
+   between

Review comment:
       nit: keep in sorted order?

##########
File path: cpp/src/arrow/compute/kernels/scalar_compare_benchmark.cc
##########
@@ -77,5 +77,44 @@ BENCHMARK(GreaterArrayScalarInt64)->Apply(RegressionSetArgs);
 BENCHMARK(GreaterArrayArrayString)->Apply(RegressionSetArgs);
 BENCHMARK(GreaterArrayScalarString)->Apply(RegressionSetArgs);
 
+template <typename Type>
+static void BetweenScalarArrayScalar(benchmark::State& state) {
+  RegressionArgs args(state, /*size_is_bytes=*/false);
+  auto ty = TypeTraits<Type>::type_singleton();
+  auto rand = random::RandomArrayGenerator(kSeed);
+  auto array = rand.ArrayOf(ty, args.size, args.null_proportion);
+  auto scalar_left = *rand.ArrayOf(ty, 1, 0)->GetScalar(0);
+  auto scalar_right = *rand.ArrayOf(ty, 1, 0)->GetScalar(0);
+  for (auto _ : state) {
+    ABORT_NOT_OK(
+        CallFunction("between_less_equal_less_equal", {array, scalar_left, scalar_right})
+            .status());
+  }
+}
+
+template <typename Type>
+static void BetweenArrayArrayArray(benchmark::State& state) {
+  RegressionArgs args(state, /*size_is_bytes=*/false);
+  auto ty = TypeTraits<Type>::type_singleton();
+  auto rand = random::RandomArrayGenerator(kSeed);
+  auto lhs = rand.ArrayOf(ty, args.size, args.null_proportion);
+  auto mid = rand.ArrayOf(ty, args.size, args.null_proportion);
+  auto rhs = rand.ArrayOf(ty, args.size, args.null_proportion);
+  for (auto _ : state) {
+    ABORT_NOT_OK(CallFunction("between_less_equal_less_equal", {mid, lhs, rhs}).status());
+  }
+}
+
+// static void BetweenArrayArrayArrayInt64(benchmark::State& state) {
+//   BetweenArrayArrayArray<Int64Type>(state);
+// }
+//
+// static void BetweenScalarArrayScalarInt64(benchmark::State& state) {
+//   BetweenScalarArrayScalar<Int64Type>(state);
+// }
+
+// BENCHMARK(BetweenArrayArrayArrayInt64)->Apply(RegressionSetArgs);
+// BENCHMARK(BetweenScalarArrayScalarInt64)->Apply(RegressionSetArgs);

Review comment:
       nit: commented-out code, was this meant to be removed or expanded into a full benchmark?

##########
File path: cpp/src/arrow/compute/kernels/scalar_compare_benchmark.cc
##########
@@ -77,5 +77,44 @@ BENCHMARK(GreaterArrayScalarInt64)->Apply(RegressionSetArgs);
 BENCHMARK(GreaterArrayArrayString)->Apply(RegressionSetArgs);
 BENCHMARK(GreaterArrayScalarString)->Apply(RegressionSetArgs);
 
+template <typename Type>
+static void BetweenScalarArrayScalar(benchmark::State& state) {
+  RegressionArgs args(state, /*size_is_bytes=*/false);
+  auto ty = TypeTraits<Type>::type_singleton();
+  auto rand = random::RandomArrayGenerator(kSeed);
+  auto array = rand.ArrayOf(ty, args.size, args.null_proportion);
+  auto scalar_left = *rand.ArrayOf(ty, 1, 0)->GetScalar(0);
+  auto scalar_right = *rand.ArrayOf(ty, 1, 0)->GetScalar(0);
+  for (auto _ : state) {
+    ABORT_NOT_OK(
+        CallFunction("between_less_equal_less_equal", {array, scalar_left, scalar_right})
+            .status());
+  }
+}
+
+template <typename Type>
+static void BetweenArrayArrayArray(benchmark::State& state) {
+  RegressionArgs args(state, /*size_is_bytes=*/false);
+  auto ty = TypeTraits<Type>::type_singleton();
+  auto rand = random::RandomArrayGenerator(kSeed);
+  auto lhs = rand.ArrayOf(ty, args.size, args.null_proportion);
+  auto mid = rand.ArrayOf(ty, args.size, args.null_proportion);
+  auto rhs = rand.ArrayOf(ty, args.size, args.null_proportion);
+  for (auto _ : state) {
+    ABORT_NOT_OK(CallFunction("between_less_equal_less_equal", {mid, lhs, rhs}).status());
+  }
+}
+
+// static void BetweenArrayArrayArrayInt64(benchmark::State& state) {
+//   BetweenArrayArrayArray<Int64Type>(state);
+// }
+//
+// static void BetweenScalarArrayScalarInt64(benchmark::State& state) {
+//   BetweenScalarArrayScalar<Int64Type>(state);
+// }
+
+// BENCHMARK(BetweenArrayArrayArrayInt64)->Apply(RegressionSetArgs);
+// BENCHMARK(BetweenScalarArrayScalarInt64)->Apply(RegressionSetArgs);

Review comment:
       It seems the benchmark definitions above are otherwise unused.

##########
File path: docs/source/cpp/compute.rst
##########
@@ -664,35 +664,43 @@ Decimal values are accepted, but are cast to Float64 first.
 Comparisons
 ~~~~~~~~~~~
 
-These functions expect two inputs of numeric type (in which case they will be
+These functions expect two or three inputs of numeric type (in which case they will be
 cast to the :ref:`common numeric type <common-numeric-type>` before comparison),
-or two inputs of Binary- or String-like types, or two inputs of Temporal types.
-If any input is dictionary encoded it will be expanded for the purposes of
-comparison. If any of the input elements in a pair is null, the corresponding
+or two or three inputs of Binary- or String-like types, or two or three inputs of Temporal 
+types. If any input is dictionary encoded it will be expanded for the purposes of
+comparison. If any of the input elements in a pair or triple is null, the corresponding
 output element is null. Decimal arguments will be promoted in the same way as
 for ``add`` and ``subtract``.
 
-+----------------+------------+---------------------------------------------+---------------------+
-| Function names | Arity      | Input types                                 | Output type         |
-+================+============+=============================================+=====================+
-| equal          | Binary     | Numeric, Temporal, Binary- and String-like  | Boolean             |
-+----------------+------------+---------------------------------------------+---------------------+
-| greater        | Binary     | Numeric, Temporal, Binary- and String-like  | Boolean             |
-+----------------+------------+---------------------------------------------+---------------------+
-| greater_equal  | Binary     | Numeric, Temporal, Binary- and String-like  | Boolean             |
-+----------------+------------+---------------------------------------------+---------------------+
-| less           | Binary     | Numeric, Temporal, Binary- and String-like  | Boolean             |
-+----------------+------------+---------------------------------------------+---------------------+
-| less_equal     | Binary     | Numeric, Temporal, Binary- and String-like  | Boolean             |
-+----------------+------------+---------------------------------------------+---------------------+
-| not_equal      | Binary     | Numeric, Temporal, Binary- and String-like  | Boolean             |
-+----------------+------------+---------------------------------------------+---------------------+
++----------------+---------+---------------------------------------------+-------------+--------------------------+-------+
+| Function names | Arity   | Input types                                 | Output type | Options Class            | Notes |
++================+=========+=============================================+=============+==========================+=======+
+| equal          | Binary  | Numeric, Temporal, Binary- and String-like  | Boolean     |                          |       |
++----------------+---------+---------------------------------------------+-------------+--------------------------+-------+
+| greater        | Binary  | Numeric, Temporal, Binary- and String-like  | Boolean     |                          |       |
++----------------+---------+---------------------------------------------+-------------+--------------------------+-------+
+| greater_equal  | Binary  | Numeric, Temporal, Binary- and String-like  | Boolean     |                          |       |
++----------------+---------+---------------------------------------------+-------------+--------------------------+-------+
+| less           | Binary  | Numeric, Temporal, Binary- and String-like  | Boolean     |                          |       |
++----------------+---------+---------------------------------------------+-------------+--------------------------+-------+
+| less_equal     | Binary  | Numeric, Temporal, Binary- and String-like  | Boolean     |                          |       |
++----------------+---------+---------------------------------------------+-------------+--------------------------+-------+
+| not_equal      | Binary  | Numeric, Temporal, Binary- and String-like  | Boolean     |                          |       |
++----------------+---------+---------------------------------------------+-------------+--------------------------+-------+
+| between        | Ternary | Numeric, Temporal, Binary- and String-like  | Boolean     | :struct:`BetweenOptions` | \(1)  |

Review comment:
       nit: keep in sorted order?

##########
File path: cpp/src/arrow/compute/api_scalar.h
##########
@@ -1350,5 +1371,24 @@ ARROW_EXPORT Result<Datum> AssumeTimezone(const Datum& values,
                                           AssumeTimezoneOptions options,
                                           ExecContext* ctx = NULLPTR);
 
+/// \brief Between compares each element in `values`
+/// with `left` as a lower bound and 'right' as an upper bound
+///
+/// \param[in] values input to compare between left and right
+/// \param[in] left used as the lower bound for comparison
+/// \param[in] right used as the upper bound for comparison
+/// \param[in] options for bounds, default is inclusive of both, other
+///  endpoints, other choices are left (exclude left endpoint), right
+///  (exclude right endpoint) and both (exclude both endpoints), optional
+/// \param[in] ctx the function execution context, optional
+///
+/// \return the resulting datum

Review comment:
       Also `\since 7.0.0` (optimistically :slightly_smiling_face:)?

##########
File path: cpp/src/arrow/compute/api_scalar.h
##########
@@ -316,6 +316,21 @@ struct ARROW_EXPORT CompareOptions {
   enum CompareOperator op;
 };
 
+enum class BetweenMode : int8_t {
+  LESS_EQUAL_LESS_EQUAL,
+  LESS_EQUAL_LESS,
+  LESS_LESS_EQUAL,
+  LESS_LESS,
+};

Review comment:
       Hmm, @edponce is this structure what you expected? I think for consistency with other options it would look like this:
   
   ```cpp
   class BetweenOptions {
    public:
     enum Inclusive {
       BOTH,
       // ...
     };
     // ...
   };
   ```
   
   ...where `enum` is deliberately used over `enum class` so that the enum members can be accessed as `BetweenOptions::BOTH`.




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

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

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



[GitHub] [arrow] bkmgit commented on pull request #11882: ARROW-9843: [C++][Python] Implement Between ternary kernel and python bindings

Posted by GitBox <gi...@apache.org>.
bkmgit commented on pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#issuecomment-1013887736


   Not able to remove/add labels


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

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

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



[GitHub] [arrow] pitrou commented on a change in pull request #11882: ARROW-9843: [C++][Python] Implement Between ternary kernel and Python bindings

Posted by GitBox <gi...@apache.org>.
pitrou commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r786042927



##########
File path: cpp/src/arrow/compute/kernels/codegen_internal.h
##########
@@ -445,6 +445,29 @@ static void VisitTwoArrayValuesInline(const ArrayData& arr0, const ArrayData& ar
                         arr0.length, std::move(visit_valid), std::move(visit_null));
 }
 
+template <typename Arg0Type, typename Arg1Type, typename Arg2Type, typename VisitFunc,
+          typename NullFunc>
+static void VisitThreeArrayValuesInline(const ArrayData& arr0, const ArrayData& arr1,

Review comment:
       This doesn't seem used at all in this PR, why is it here?




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

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

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



[GitHub] [arrow] bkmgit commented on a change in pull request #11882: ARROW-9843: [C++][Python] Implement Between ternary kernel and Python bindings

Posted by GitBox <gi...@apache.org>.
bkmgit commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r787031774



##########
File path: cpp/src/arrow/compute/kernels/codegen_internal.h
##########
@@ -445,6 +445,29 @@ static void VisitTwoArrayValuesInline(const ArrayData& arr0, const ArrayData& ar
                         arr0.length, std::move(visit_valid), std::move(visit_null));
 }
 
+template <typename Arg0Type, typename Arg1Type, typename Arg2Type, typename VisitFunc,
+          typename NullFunc>
+static void VisitThreeArrayValuesInline(const ArrayData& arr0, const ArrayData& arr1,

Review comment:
       Removed.




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

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

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



[GitHub] [arrow] bkmgit commented on a change in pull request #11882: ARROW-9843: [C++][Python] Implement Between ternary kernel and Python bindings

Posted by GitBox <gi...@apache.org>.
bkmgit commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r793735145



##########
File path: cpp/src/arrow/compute/kernels/scalar_compare_test.cc
##########
@@ -1865,5 +1928,654 @@ TEST(TestMaxElementWiseMinElementWise, CommonTemporal) {
               ResultWith(ScalarFromJSON(date64(), "86400000")));
 }
 
+static void ValidateBetween(BetweenOptions options, const Datum& val, const Datum& lhs,
+                            const Datum& rhs, const Datum& expected) {
+  ASSERT_OK_AND_ASSIGN(Datum result, Between(val, lhs, rhs, options, nullptr));
+  if ((val.is_scalar()) && (lhs.is_scalar()) && (rhs.is_scalar())) {
+    AssertScalarsEqual(*expected.scalar(), *result.scalar(), /*verbose=*/true);
+  } else {
+    AssertArraysEqual(*expected.make_array(), *result.make_array(),
+                      /*verbose=*/true);
+  }
+}
+
+void ValidateBetween(const Datum& val, const Datum& lhs, const Datum& rhs) {
+  CompareOperator lhs_val;
+  CompareOperator val_rhs;
+  for (auto inclusive :
+       {BetweenOptions::Inclusive::BOTH, BetweenOptions::Inclusive::LEFT,
+        BetweenOptions::Inclusive::RIGHT, BetweenOptions::Inclusive::NEITHER}) {
+    auto options = BetweenOptions(inclusive);
+    if (inclusive == BetweenOptions::Inclusive::NEITHER) {
+      lhs_val = LESS;
+      val_rhs = LESS;
+    } else if (inclusive == BetweenOptions::Inclusive::LEFT) {
+      lhs_val = LESS_EQUAL;
+      val_rhs = LESS;
+    } else if (inclusive == BetweenOptions::Inclusive::RIGHT) {
+      lhs_val = LESS;
+      val_rhs = LESS_EQUAL;
+    } else {
+      lhs_val = LESS_EQUAL;
+      val_rhs = LESS_EQUAL;
+    }
+
+    ASSERT_OK_AND_ASSIGN(
+        Datum resultl, CallFunction(CompareOperatorToFunctionName(lhs_val), {lhs, val}));
+    ASSERT_OK_AND_ASSIGN(
+        Datum resultr, CallFunction(CompareOperatorToFunctionName(val_rhs), {val, rhs}));
+    ASSERT_OK_AND_ASSIGN(Datum expected, CallFunction("and", {resultl, resultr}));
+
+    ValidateBetween(options, val, lhs, rhs, expected);
+  }
+}
+
+class TestNumericBetweenKernel : public ::testing::Test {};
+
+std::shared_ptr<DataType> GetType(std::shared_ptr<DataType> type) {
+  auto type_string = type->ToString();
+  if (type_string == "duration[s]") {
+    return int64();
+  } else if (type_string == "duration[ms]") {
+    return int64();
+  } else if (type_string == "duration[ns]") {
+    return int64();
+  } else if (type_string == "uint8") {
+    return uint8();
+  } else if (type_string == "uint16") {
+    return uint16();
+  } else if (type_string == "uint32") {
+    return uint32();
+  } else if (type_string == "uint64") {
+    return uint64();
+  } else if (type_string == "int8") {
+    return int8();
+  } else if (type_string == "int16") {
+    return int16();
+  } else if (type_string == "int32") {
+    return int32();
+  } else if (type_string == "int64") {
+    return int64();
+  } else if (type_string == "float") {
+    return float32();
+  } else if (type_string == "double") {
+    return float64();
+  } else {
+    return int64();
+  }
+}
+
+TEST(TestNumericBetweenKernel, 3Scalars) {
+  for (const auto& types : {DurationTypes(), NumericTypes()}) {
+    for (const std::shared_ptr<DataType>& ty : types) {
+      ARROW_SCOPED_TRACE("type = ", ty->ToString());
+      auto tt = GetType(ty);
+      auto zero = Datum(ScalarFromJSON(tt, "0"));
+      auto two = Datum(ScalarFromJSON(tt, "2"));
+      auto four = Datum(ScalarFromJSON(tt, "4"));
+      auto null = Datum(ScalarFromJSON(tt, "null"));
+      ValidateBetween(zero, two, four);
+      ValidateBetween(two, zero, four);
+      ValidateBetween(two, two, four);
+      ValidateBetween(four, two, four);
+      ValidateBetween(null, two, four);
+      ValidateBetween(two, null, four);
+      ValidateBetween(two, zero, null);
+    }
+  }
+}
+
+TEST(TestNumericBetweenKernel, 1Array2Scalars) {
+  for (const auto& types : {DurationTypes(), NumericTypes()}) {
+    for (const std::shared_ptr<DataType>& ty : types) {
+      ARROW_SCOPED_TRACE("type = ", ty->ToString());
+      auto tt = GetType(ty);
+      auto zero = Datum(ScalarFromJSON(tt, "0"));
+      auto four = Datum(ScalarFromJSON(tt, "4"));
+      auto null = Datum(ScalarFromJSON(tt, "null"));
+      ValidateBetween(Datum(ArrayFromJSON(tt, "[]")), zero, four);
+      ValidateBetween(Datum(ArrayFromJSON(tt, "[null]")), zero, four);
+      ValidateBetween(Datum(ArrayFromJSON(tt, "[0,1,2,3,4,5]")), zero, four);
+      ValidateBetween(Datum(ArrayFromJSON(tt, "[null,0,1,1]")), zero, four);
+      ValidateBetween(Datum(ArrayFromJSON(tt, "[5,4,3,2,1,0]")), null, four);
+      ValidateBetween(Datum(ArrayFromJSON(tt, "[5,4,3,2,1,0]")), zero, null);
+      ValidateBetween(zero, Datum(ArrayFromJSON(tt, "[]")), four);
+      ValidateBetween(zero, Datum(ArrayFromJSON(tt, "[null]")), four);
+      ValidateBetween(zero, Datum(ArrayFromJSON(tt, "[0,1,2,3,4,5]")), four);
+      ValidateBetween(zero, Datum(ArrayFromJSON(tt, "[null,0,1,1]")), four);
+      ValidateBetween(null, Datum(ArrayFromJSON(tt, "[5,4,3,2,1,0]")), four);
+      ValidateBetween(zero, Datum(ArrayFromJSON(tt, "[5,4,3,2,1,0]")), null);
+      ValidateBetween(zero, four, Datum(ArrayFromJSON(tt, "[]")));
+      ValidateBetween(zero, four, Datum(ArrayFromJSON(tt, "[null]")));
+      ValidateBetween(zero, four, Datum(ArrayFromJSON(tt, "[0,1,2,3,4,5]")));
+      ValidateBetween(zero, four, Datum(ArrayFromJSON(tt, "[null,0,1,1]")));
+      ValidateBetween(null, four, Datum(ArrayFromJSON(tt, "[5,4,3,2,1,0]")));
+      ValidateBetween(zero, null, Datum(ArrayFromJSON(tt, "[5,4,3,2,1,0]")));
+    }
+  }
+}
+
+TEST(TestNumericBetweenKernel, 2Arrays1Scalar) {
+  for (const auto& types : {DurationTypes(), NumericTypes()}) {
+    for (const std::shared_ptr<DataType>& ty : types) {
+      ARROW_SCOPED_TRACE("type = ", ty->ToString());
+      auto tt = GetType(ty);
+      auto one = Datum(ScalarFromJSON(tt, "1"));
+      ValidateBetween(one, Datum(ArrayFromJSON(tt, "[]")),
+                      Datum(ArrayFromJSON(tt, "[]")));
+      ValidateBetween(one, Datum(ArrayFromJSON(tt, "[null]")),
+                      Datum(ArrayFromJSON(tt, "[null]")));
+      ValidateBetween(one, Datum(ArrayFromJSON(tt, "[0,0,1,3,3]")),
+                      Datum(ArrayFromJSON(tt, "[10,10,2,5,5]")));
+      ValidateBetween(one, Datum(ArrayFromJSON(tt, "[0,0,1,null,3,3]")),
+                      Datum(ArrayFromJSON(tt, "[0,10,2,2,5,5]")));
+      ValidateBetween(Datum(ArrayFromJSON(tt, "[]")), one,
+                      Datum(ArrayFromJSON(tt, "[]")));
+      ValidateBetween(Datum(ArrayFromJSON(tt, "[null]")), one,
+                      Datum(ArrayFromJSON(tt, "[null]")));
+      ValidateBetween(Datum(ArrayFromJSON(tt, "[0,0,1,3,3]")), one,
+                      Datum(ArrayFromJSON(tt, "[10,10,2,5,5]")));
+      ValidateBetween(Datum(ArrayFromJSON(tt, "[0,0,1,null,3,3]")), one,
+                      Datum(ArrayFromJSON(tt, "[0,10,2,2,5,5]")));
+      ValidateBetween(Datum(ArrayFromJSON(tt, "[]")), Datum(ArrayFromJSON(tt, "[]")),
+                      one);
+      ValidateBetween(Datum(ArrayFromJSON(tt, "[null]")),
+                      Datum(ArrayFromJSON(tt, "[null]")), one);
+      ValidateBetween(Datum(ArrayFromJSON(tt, "[0,0,1,3,3]")),
+                      Datum(ArrayFromJSON(tt, "[10,10,2,5,5]")), one);
+      ValidateBetween(Datum(ArrayFromJSON(tt, "[0,0,1,null,3,3]")),
+                      Datum(ArrayFromJSON(tt, "[0,10,2,2,5,5]")), one);
+    }
+  }
+}
+
+TEST(TestNumericBetweenKernel, 3Arrays) {
+  for (const auto& types : {DurationTypes(), NumericTypes()}) {
+    for (const std::shared_ptr<DataType>& ty : types) {
+      ARROW_SCOPED_TRACE("type = ", ty->ToString());
+      ValidateBetween(Datum(ArrayFromJSON(ty, "[]")), Datum(ArrayFromJSON(ty, "[]")),
+                      Datum(ArrayFromJSON(ty, "[]")));
+      ValidateBetween(Datum(ArrayFromJSON(ty, "[null]")),
+                      Datum(ArrayFromJSON(ty, "[null]")),
+                      Datum(ArrayFromJSON(ty, "[null]")));
+      ValidateBetween(Datum(ArrayFromJSON(ty, "[1,1,2,2,2]")),
+                      Datum(ArrayFromJSON(ty, "[0,0,1,3,3]")),
+                      Datum(ArrayFromJSON(ty, "[10,10,2,5,5]")));
+      ValidateBetween(Datum(ArrayFromJSON(ty, "[0,1,2,2,2,2]")),
+                      Datum(ArrayFromJSON(ty, "[0,0,1,null,3,3]")),
+                      Datum(ArrayFromJSON(ty, "[0,10,2,2,5,5]")));
+    }
+  }
+}
+
+TEST(TestNumericBetweenKernel, Random) {
+  for (const auto& types : {DurationTypes(), NumericTypes()}) {
+    for (const std::shared_ptr<DataType>& ty : types) {
+      auto rand = random::RandomArrayGenerator(0x5416447);
+      const int64_t length = 100;
+      for (auto null_probability : {0.0, 0.01, 0.1, 0.25, 0.5, 1.0}) {
+        ARROW_SCOPED_TRACE("type = ", ty->ToString());
+        auto tt = GetType(ty);
+        auto metadata =
+            key_value_metadata({"null_probability"}, {std::to_string(null_probability)});
+        auto field = ::arrow::field("[0,100]", std::move(ty), std::move(metadata));
+        auto data1 = rand.ArrayOf(*field, length);
+        auto data2 = rand.ArrayOf(*field, length);
+        auto data3 = rand.ArrayOf(*field, length);
+
+        // Create view of data as the type (e.g. float64)
+        auto array1 = Datum(*data1->View(tt));
+        auto array2 = Datum(*data2->View(tt));
+        auto array3 = Datum(*data3->View(tt));
+        auto scalar1 = Datum(ScalarFromJSON(tt, "10"));
+        auto scalar2 = Datum(ScalarFromJSON(tt, "30"));
+        auto scalar3 = Datum(ScalarFromJSON(tt, "50"));
+        ValidateBetween(scalar1, scalar2, scalar3);
+	ValidateBetween(array1, scalar2, scalar3);
+        ValidateBetween(array1, array2, scalar3);
+        ValidateBetween(array1, array2, array3);
+        ValidateBetween(array1, scalar2, scalar3);
+        ValidateBetween(scalar1, array2, array3);
+        ValidateBetween(scalar1, array2, scalar3);
+        ValidateBetween(scalar1, scalar2, array3);
+        ValidateBetween(array1, scalar2, array3);
+      }
+    }
+  }
+}
+
+class TestStringBetweenKernel : public ::testing::Test {};
+
+TEST(TestStringBetweenKernel, Random) {
+  using ScalarType = typename TypeTraits<StringType>::ScalarType;
+
+  auto rand = random::RandomArrayGenerator(0x5416447);
+  for (size_t i = 3; i < 10; i++) {
+    for (auto null_probability : {0.0, 0.01, 0.1, 0.25, 0.5, 1.0}) {
+      const int64_t length = static_cast<int64_t>(1ULL << i);
+      auto array1 = Datum(rand.String(length, 0, 16, null_probability));
+      auto array2 = Datum(rand.String(length, 0, 16, null_probability));
+      auto array3 = Datum(rand.String(length, 0, 16, null_probability));
+      auto scalar1 = Datum(std::make_shared<ScalarType>("fupi"));
+      auto scalar2 = Datum(std::make_shared<ScalarType>("tupu"));
+      auto scalar3 = Datum(std::make_shared<ScalarType>("zito"));
+      ValidateBetween(scalar1, scalar2, scalar3);
+      ValidateBetween(array1, scalar2, scalar3);
+      ValidateBetween(scalar1, array2, scalar3);
+      ValidateBetween(scalar1, scalar2, array3);
+      ValidateBetween(scalar1, array2, array3);
+      ValidateBetween(array1, scalar2, array3);
+      ValidateBetween(array1, array2, scalar3);
+      ValidateBetween(array1, array2, array3);
+    }
+  }
+}
+
+TEST(TestStringBetweenKernel, 1Array2Scalars) {
+  using ScalarType = typename TypeTraits<StringType>::ScalarType;
+  auto l = Datum(std::make_shared<ScalarType>("abc"));
+  auto r = Datum(std::make_shared<ScalarType>("zzz"));
+  ValidateBetween(Datum(ArrayFromJSON(TypeTraits<StringType>::type_singleton(), "[]")), l,
+                  r);
+  ValidateBetween(
+      Datum(ArrayFromJSON(TypeTraits<StringType>::type_singleton(), "[null]")), l, r);
+  ValidateBetween(Datum(ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                      R"(["aaa", "aaaa", "ccc", "z"])")),
+                  l, r);
+  ValidateBetween(Datum(ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                      R"(["abc", "baa", "fff", "zzz"])")),
+                  l, r);
+  ValidateBetween(Datum(ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                      R"(["abd", null, null, "zzx"])")),
+                  l, r);
+  ValidateBetween(l, Datum(ArrayFromJSON(TypeTraits<StringType>::type_singleton(), "[]")),
+                  r);
+  ValidateBetween(l,
+      Datum(ArrayFromJSON(TypeTraits<StringType>::type_singleton(), "[null]")), r);
+  ValidateBetween(l, Datum(ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                      R"(["aaa", "aaaa", "ccc", "z"])")),
+                  r);
+  ValidateBetween(l, Datum(ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                      R"(["abc", "baa", "fff", "zzz"])")),
+                  r);
+  ValidateBetween(l, Datum(ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                      R"(["abd", null, null, "zzx"])")),
+                  r);
+  ValidateBetween(l, r,
+                  Datum(ArrayFromJSON(TypeTraits<StringType>::type_singleton(), "[]")));
+  ValidateBetween(l, r,
+      Datum(ArrayFromJSON(TypeTraits<StringType>::type_singleton(), "[null]")));
+  ValidateBetween(l, r, Datum(ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                       R"(["aaa", "aaaa", "ccc", "z"])")));
+  ValidateBetween(l, r, Datum(ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                      R"(["abc", "baa", "fff", "zzz"])")));
+  ValidateBetween(l, r, Datum(ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                      R"(["abd", null, null, "zzx"])")));
+}
+
+TEST(TestStringBetweenKernel, 2Arrays1Scalar) {
+  using ScalarType = typename TypeTraits<StringType>::ScalarType;
+  auto r = Datum(std::make_shared<ScalarType>("zzz"));
+  ValidateBetween(r, Datum(ArrayFromJSON(TypeTraits<StringType>::type_singleton(), "[]")),
+                  Datum(ArrayFromJSON(TypeTraits<StringType>::type_singleton(), "[]")));
+  ValidateBetween(r, Datum(ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                   R"(["aaa", "aaaa", "ccc", "z"])")),
+                  Datum(ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                R"(["abc", "baa", "fff", "zzz"])")));
+  ValidateBetween(r, Datum(ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                   R"(["abc", "baa", "fff", "zzz"])")),
+                  Datum(ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                R"(["abd", null, null, "zzx"])")));
+  ValidateBetween(Datum(ArrayFromJSON(TypeTraits<StringType>::type_singleton(), "[]")), r,
+                  Datum(ArrayFromJSON(TypeTraits<StringType>::type_singleton(), "[]")));
+  ValidateBetween(Datum(ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                R"(["aaa", "aaaa", "ccc", "z"])")), r,
+                  Datum(ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                R"(["abc", "baa", "fff", "zzz"])")));
+  ValidateBetween(Datum(ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                R"(["abc", "baa", "fff", "zzz"])")), r,
+                  Datum(ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                R"(["abd", null, null, "zzx"])")));
+  ValidateBetween(Datum(ArrayFromJSON(TypeTraits<StringType>::type_singleton(), "[]")),
+                  Datum(ArrayFromJSON(TypeTraits<StringType>::type_singleton(), "[]")),
+		  r);
+  ValidateBetween(Datum(ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                R"(["aaa", "aaaa", "ccc", "z"])")),
+		  Datum(ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                R"(["abc", "baa", "fff", "zzz"])")),
+		  r);
+  ValidateBetween(Datum(ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                R"(["abc", "baa", "fff", "zzz"])")),
+                  Datum(ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                R"(["abd", null, null, "zzx"])")),
+		  r);
+}
+
+TEST(TestStringBetweenKernel, 3Arrays) {
+  ValidateBetween(Datum(ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                      R"(["david","hello","world"])")),
+                  Datum(ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                      R"(["adam","hi","whirl"])")),
+                  Datum(ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                      R"(["robert","goeiemoreen","whirlwind"])")));
+  ValidateBetween(
+      Datum(ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["x","a","f"])")),
+      Datum(ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["w","a","e"])")),
+      Datum(ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["z","a","g"])")));
+  ValidateBetween(Datum(ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                      R"(["block","bit","binary"])")),
+                  Datum(ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                      R"(["bit","nibble","ternary"])")),
+                  Datum(ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                      R"(["word","d","xyz"])")));
+  ValidateBetween(Datum(ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                      R"(["よしもと","の","ち"])")),
+                  Datum(ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                      R"(["は","へ","あ"])")),
+                  Datum(ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                      R"(["な","を","ち"])")));
+  ValidateBetween(Datum(ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                      R"(["A","ア","王"])")),
+                  Datum(ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                      R"(["た","あ","歩"])")),
+                  Datum(ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                      R"(["李","田",null])")));
+  ValidateBetween(
+      Datum(ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["Б",null,"Я"])")),
+      Datum(ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["А","Ж","Щ"])")),
+      Datum(ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["Д","Л","Ф"])")));
+}
+
+TEST(TestTimestampsBetweenKernel, 1Array2Scalars) {
+  const std::string scalar1_json = R"("1980-02-02")";
+  const std::string scalar2_json = R"("1970-01-01")";
+  const std::string array_json = R"(["1970-01-02","1980-02-02","1970-02-28"])";
+  // Same units should be fine
+  ValidateBetween(Datum(ArrayFromJSON(timestamp(TimeUnit::SECOND), array_json)),
+                  Datum(ScalarFromJSON(timestamp(TimeUnit::SECOND), scalar1_json)),
+                  Datum(ScalarFromJSON(timestamp(TimeUnit::SECOND), scalar2_json)));
+  ValidateBetween(Datum(ScalarFromJSON(timestamp(TimeUnit::SECOND), scalar1_json)),
+                  Datum(ArrayFromJSON(timestamp(TimeUnit::SECOND), array_json)),
+                  Datum(ScalarFromJSON(timestamp(TimeUnit::SECOND), scalar2_json)));
+  ValidateBetween(Datum(ScalarFromJSON(timestamp(TimeUnit::SECOND), scalar1_json)),
+                  Datum(ScalarFromJSON(timestamp(TimeUnit::SECOND), scalar2_json)),
+                  Datum(ArrayFromJSON(timestamp(TimeUnit::SECOND), array_json)));
+  // Different timezones should be fine
+  ValidateBetween(
+      Datum(ArrayFromJSON(timestamp(TimeUnit::SECOND, "Africa/Cairo"), array_json)),
+      Datum(ScalarFromJSON(timestamp(TimeUnit::SECOND, "America/Chicago"), scalar1_json)),
+      Datum(ScalarFromJSON(timestamp(TimeUnit::SECOND, "Asia/Beijing"), scalar2_json)));
+  ValidateBetween(
+      Datum(ScalarFromJSON(timestamp(TimeUnit::NANO, "Europe/Berlin"), scalar1_json)),
+      Datum(ArrayFromJSON(timestamp(TimeUnit::NANO, "America/Phoenix"), array_json)),
+      Datum(ScalarFromJSON(timestamp(TimeUnit::NANO, "Africa/Nairobi"), scalar2_json)));
+  ValidateBetween(
+      Datum(ScalarFromJSON(timestamp(TimeUnit::NANO, "Europe/Berlin"), scalar1_json)),
+      Datum(ScalarFromJSON(timestamp(TimeUnit::NANO, "Asia/Tokyo"), scalar2_json)),
+      Datum(ArrayFromJSON(timestamp(TimeUnit::NANO, "Africa/Nairobi"), array_json)));
+  // Different units should be fine
+  ValidateBetween(Datum(ArrayFromJSON(timestamp(TimeUnit::SECOND), array_json)),
+                  Datum(ScalarFromJSON(timestamp(TimeUnit::MILLI), scalar1_json)),
+                  Datum(ScalarFromJSON(timestamp(TimeUnit::SECOND), scalar2_json)));
+  ValidateBetween(Datum(ScalarFromJSON(timestamp(TimeUnit::SECOND), scalar1_json)),
+                  Datum(ArrayFromJSON(timestamp(TimeUnit::NANO), array_json)),
+                  Datum(ScalarFromJSON(timestamp(TimeUnit::SECOND), scalar2_json)));
+  ValidateBetween(Datum(ScalarFromJSON(timestamp(TimeUnit::SECOND), scalar1_json)),
+                  Datum(ScalarFromJSON(timestamp(TimeUnit::NANO), scalar2_json)),
+                  Datum(ArrayFromJSON(timestamp(TimeUnit::SECOND), array_json)));
+  // But comparing naive to zoned is not OK
+  for (auto inclusive :
+       {BetweenOptions::Inclusive::BOTH, BetweenOptions::Inclusive::LEFT,
+        BetweenOptions::Inclusive::RIGHT, BetweenOptions::Inclusive::NEITHER}) {
+    auto options = BetweenOptions(inclusive);
+    EXPECT_RAISES_WITH_MESSAGE_THAT(
+        TypeError,
+        ::testing::HasSubstr(
+            "Cannot compare timestamp with timezone to timestamp without timezone"),
+        Between(ArrayFromJSON(timestamp(TimeUnit::SECOND), array_json),
+                ScalarFromJSON(timestamp(TimeUnit::SECOND), scalar1_json),
+                ScalarFromJSON(timestamp(TimeUnit::SECOND, "Asia/Tokyo"), scalar2_json),
+                options, nullptr));
+    EXPECT_RAISES_WITH_MESSAGE_THAT(
+        TypeError,
+        ::testing::HasSubstr(
+            "Cannot compare timestamp with timezone to timestamp without timezone"),
+        Between(ScalarFromJSON(timestamp(TimeUnit::SECOND), scalar1_json),
+                ArrayFromJSON(timestamp(TimeUnit::SECOND, "America/New_York"), array_json),
+                ScalarFromJSON(timestamp(TimeUnit::SECOND, "Europe/Berlin"), scalar2_json),
+                options, nullptr));
+    EXPECT_RAISES_WITH_MESSAGE_THAT(
+        TypeError,
+        ::testing::HasSubstr(
+            "Cannot compare timestamp with timezone to timestamp without timezone"),
+        Between(ScalarFromJSON(timestamp(TimeUnit::SECOND, "Africa/Nairobi"), scalar1_json),
+                ArrayFromJSON(timestamp(TimeUnit::SECOND), array_json),
+                ScalarFromJSON(timestamp(TimeUnit::SECOND), scalar2_json), options, nullptr));
+  }
+}
+
+TEST(TestTimestampsBetweenKernel, 2Arrays1Scalar) {
+  const std::string scalar_json = R"("1980-02-02")";
+  const std::string array1_json = R"(["1970-01-01","1980-02-01","1970-02-28"])";
+  const std::string array2_json = R"(["1970-01-02","1980-02-02","1970-02-28"])";
+  // Same units should be fine
+  ValidateBetween(Datum(ScalarFromJSON(timestamp(TimeUnit::SECOND), scalar_json)),
+                  Datum(ArrayFromJSON(timestamp(TimeUnit::SECOND), array1_json)),
+                  Datum(ArrayFromJSON(timestamp(TimeUnit::SECOND), array2_json)));
+  ValidateBetween(Datum(ArrayFromJSON(timestamp(TimeUnit::SECOND), array1_json)),
+                  Datum(ScalarFromJSON(timestamp(TimeUnit::SECOND), scalar_json)),
+                  Datum(ArrayFromJSON(timestamp(TimeUnit::SECOND), array2_json)));
+  ValidateBetween(Datum(ArrayFromJSON(timestamp(TimeUnit::SECOND), array1_json)),
+                  Datum(ArrayFromJSON(timestamp(TimeUnit::SECOND), array2_json)),
+                  Datum(ScalarFromJSON(timestamp(TimeUnit::SECOND), scalar_json)));
+  // Different timezones should be fine
+  ValidateBetween(
+      Datum(ScalarFromJSON(timestamp(TimeUnit::SECOND, "Africa/Cairo"), scalar_json)),
+      Datum(ArrayFromJSON(timestamp(TimeUnit::SECOND, "America/Chicago"), array1_json)),
+      Datum(ArrayFromJSON(timestamp(TimeUnit::SECOND, "Asia/Beijing"), array2_json)));
+  ValidateBetween(
+      Datum(ArrayFromJSON(timestamp(TimeUnit::NANO, "Europe/Berlin"), array1_json)),
+      Datum(ScalarFromJSON(timestamp(TimeUnit::NANO, "America/Phoenix"), scalar_json)),
+      Datum(ArrayFromJSON(timestamp(TimeUnit::NANO, "Africa/Nairobi"), array2_json)));
+  ValidateBetween(
+      Datum(ArrayFromJSON(timestamp(TimeUnit::NANO, "Europe/Berlin"), array1_json)),
+      Datum(ArrayFromJSON(timestamp(TimeUnit::NANO, "Asia/Tokyo"), array2_json)),
+      Datum(ScalarFromJSON(timestamp(TimeUnit::NANO, "Africa/Nairobi"), scalar_json)));
+  // Different units should be fine
+  ValidateBetween(Datum(ScalarFromJSON(timestamp(TimeUnit::SECOND), scalar_json)),
+                  Datum(ArrayFromJSON(timestamp(TimeUnit::MILLI), array1_json)),
+                  Datum(ArrayFromJSON(timestamp(TimeUnit::SECOND), array2_json)));
+  ValidateBetween(Datum(ArrayFromJSON(timestamp(TimeUnit::SECOND), array1_json)),
+                  Datum(ScalarFromJSON(timestamp(TimeUnit::NANO), scalar_json)),
+                  Datum(ArrayFromJSON(timestamp(TimeUnit::SECOND), array2_json)));
+  ValidateBetween(Datum(ArrayFromJSON(timestamp(TimeUnit::SECOND), array1_json)),
+                  Datum(ArrayFromJSON(timestamp(TimeUnit::NANO), array2_json)),
+                  Datum(ScalarFromJSON(timestamp(TimeUnit::SECOND), scalar_json)));
+  // But comparing naive to zoned is not OK
+  for (auto inclusive :
+       {BetweenOptions::Inclusive::BOTH, BetweenOptions::Inclusive::LEFT,
+        BetweenOptions::Inclusive::RIGHT, BetweenOptions::Inclusive::NEITHER}) {
+    auto options = BetweenOptions(inclusive);
+    EXPECT_RAISES_WITH_MESSAGE_THAT(
+        TypeError,
+        ::testing::HasSubstr(
+            "Cannot compare timestamp with timezone to timestamp without timezone"),
+        Between(ScalarFromJSON(timestamp(TimeUnit::SECOND), scalar_json),
+                ArrayFromJSON(timestamp(TimeUnit::SECOND), array1_json),
+                ArrayFromJSON(timestamp(TimeUnit::SECOND, "Asia/Tokyo"), array2_json),
+                options, nullptr));
+    EXPECT_RAISES_WITH_MESSAGE_THAT(
+        TypeError,
+        ::testing::HasSubstr(
+            "Cannot compare timestamp with timezone to timestamp without timezone"),
+        Between(ArrayFromJSON(timestamp(TimeUnit::SECOND), array1_json),
+                ScalarFromJSON(timestamp(TimeUnit::SECOND, "America/New_York"), scalar_json),
+                ArrayFromJSON(timestamp(TimeUnit::SECOND, "Europe/Berlin"), array2_json),
+                options, nullptr));
+    EXPECT_RAISES_WITH_MESSAGE_THAT(
+        TypeError,
+        ::testing::HasSubstr(
+            "Cannot compare timestamp with timezone to timestamp without timezone"),
+        Between(ArrayFromJSON(timestamp(TimeUnit::SECOND, "Africa/Nairobi"), array1_json),
+                ScalarFromJSON(timestamp(TimeUnit::SECOND), scalar_json),
+                ArrayFromJSON(timestamp(TimeUnit::SECOND), array2_json), options, nullptr));
+  }
+}
+
+TEST(TestTimestampsBetweenKernel, 3Arrays) {
+  const std::string arr_json = R"(["1970-01-01","1980-02-02","1970-02-28"])";
+  const std::string lhs_json = R"(["1970-01-01","1980-02-01","1970-02-28"])";
+  const std::string rhs_json = R"(["1970-01-02","1980-02-02","1970-02-28"])";
+  // Same units should be fine
+  ValidateBetween(Datum(ArrayFromJSON(timestamp(TimeUnit::SECOND), arr_json)),
+                  Datum(ArrayFromJSON(timestamp(TimeUnit::SECOND), lhs_json)),
+                  Datum(ArrayFromJSON(timestamp(TimeUnit::SECOND), rhs_json)));
+  // Different timezones should be fine
+  ValidateBetween(
+      Datum(ArrayFromJSON(timestamp(TimeUnit::SECOND, "Africa/Cairo"), arr_json)),
+      Datum(ArrayFromJSON(timestamp(TimeUnit::SECOND, "America/Chicago"), lhs_json)),
+      Datum(ArrayFromJSON(timestamp(TimeUnit::SECOND, "Asia/Beijing"), rhs_json)));
+  ValidateBetween(
+      Datum(ArrayFromJSON(timestamp(TimeUnit::NANO, "Europe/Berlin"), arr_json)),
+      Datum(ArrayFromJSON(timestamp(TimeUnit::NANO, "America/Phoenix"), lhs_json)),
+      Datum(ArrayFromJSON(timestamp(TimeUnit::NANO, "Africa/Nairobi"), rhs_json)));
+  // Different units should be fine
+  ValidateBetween(Datum(ArrayFromJSON(timestamp(TimeUnit::SECOND), arr_json)),
+                  Datum(ArrayFromJSON(timestamp(TimeUnit::MILLI), lhs_json)),
+                  Datum(ArrayFromJSON(timestamp(TimeUnit::SECOND), rhs_json)));
+  ValidateBetween(Datum(ArrayFromJSON(timestamp(TimeUnit::SECOND), arr_json)),
+                  Datum(ArrayFromJSON(timestamp(TimeUnit::NANO), lhs_json)),
+                  Datum(ArrayFromJSON(timestamp(TimeUnit::SECOND), rhs_json)));
+  // But comparing naive to zoned is not OK
+  for (auto inclusive :
+       {BetweenOptions::Inclusive::BOTH, BetweenOptions::Inclusive::LEFT,
+        BetweenOptions::Inclusive::RIGHT, BetweenOptions::Inclusive::NEITHER}) {
+    auto options = BetweenOptions(inclusive);
+    EXPECT_RAISES_WITH_MESSAGE_THAT(
+        TypeError,
+        ::testing::HasSubstr(
+            "Cannot compare timestamp with timezone to timestamp without timezone"),
+        Between(ArrayFromJSON(timestamp(TimeUnit::SECOND), arr_json),
+                ArrayFromJSON(timestamp(TimeUnit::SECOND), lhs_json),
+                ArrayFromJSON(timestamp(TimeUnit::SECOND, "Asia/Tokyo"), rhs_json),
+                options, nullptr));
+    EXPECT_RAISES_WITH_MESSAGE_THAT(
+        TypeError,
+        ::testing::HasSubstr(
+            "Cannot compare timestamp with timezone to timestamp without timezone"),
+        Between(ArrayFromJSON(timestamp(TimeUnit::SECOND), arr_json),
+                ArrayFromJSON(timestamp(TimeUnit::SECOND, "America/New_York"), lhs_json),
+                ArrayFromJSON(timestamp(TimeUnit::SECOND, "Europe/Berlin"), rhs_json),
+                options, nullptr));
+    EXPECT_RAISES_WITH_MESSAGE_THAT(
+        TypeError,
+        ::testing::HasSubstr(
+            "Cannot compare timestamp with timezone to timestamp without timezone"),
+        Between(ArrayFromJSON(timestamp(TimeUnit::SECOND, "Africa/Nairobi"), arr_json),
+                ArrayFromJSON(timestamp(TimeUnit::SECOND), lhs_json),
+                ArrayFromJSON(timestamp(TimeUnit::SECOND), rhs_json), options, nullptr));
+  }
+}
+
+template <typename ArrowType>
+class TestBetweenDecimal : public ::testing::Test {};
+TYPED_TEST_SUITE(TestBetweenDecimal, DecimalArrowTypes);
+
+TYPED_TEST(TestBetweenDecimal, 3Arrays) {
+  auto ty = std::make_shared<TypeParam>(3, 2);

Review comment:
       Thanks for the suggestion. Was not sure how to get the decimal types in ty.




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

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

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



[GitHub] [arrow] bkmgit commented on a change in pull request #11882: ARROW-9843: [C++][Python] Implement Between ternary kernel and Python bindings

Posted by GitBox <gi...@apache.org>.
bkmgit commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r787015085



##########
File path: cpp/src/arrow/compute/kernels/scalar_compare.cc
##########
@@ -772,6 +1107,16 @@ const FunctionDoc less_equal_doc{
     ("A null on either side emits a null comparison result."),
     {"x", "y"}};
 
+const FunctionDoc between_doc{
+    "Check if values are in the given range, val between a and b",
+    ("A null on either side emits a null comparison result.\n"
+     "options are used to specify if the endpoints are\n"
+     "inclusive, possible values are NEITHER (a < val < b),\n"
+     "LEFT (a <= val < b), RIGHT (a < val <= b), and \n"
+     "BOTH (a <= val <= b). Default is BOTH."),
+    {"val", "a", "b"},

Review comment:
       Done




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

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

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



[GitHub] [arrow] bkmgit commented on a change in pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
bkmgit commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r769436558



##########
File path: cpp/src/arrow/compute/kernels/scalar_compare_test.cc
##########
@@ -1850,5 +1851,154 @@ TEST(TestMaxElementWiseMinElementWise, CommonTemporal) {
               ResultWith(ScalarFromJSON(date64(), "86400000")));
 }
 
+template <typename ArrowType>
+static void ValidateBetween(const Datum& val, const Datum& lhs, const Datum& rhs,
+                            const Datum& expected) {
+  ASSERT_OK_AND_ASSIGN(Datum result, Between(val, lhs, rhs));
+  AssertArraysEqual(*expected.make_array(), *result.make_array(),
+                    /*verbose=*/true);
+}
+
+template <typename ArrowType>
+static void ValidateBetween(const char* value_str, const Datum& lhs, const Datum& rhs,
+                            const char* expected_str) {
+  auto value = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), value_str);
+  auto expected = ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), expected_str);
+  ValidateBetween<ArrowType>(value, lhs, rhs, expected);
+}
+
+template <>
+void ValidateBetween<StringType>(const char* value_str, const Datum& lhs,
+                                 const Datum& rhs, const char* expected_str) {
+  auto value = ArrayFromJSON(utf8(), value_str);
+  auto expected = ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), expected_str);
+  ValidateBetween<StringType>(value, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+class TestNumericBetweenKernel : public ::testing::Test {};
+
+TYPED_TEST_SUITE(TestNumericBetweenKernel, NumericArrowTypes);
+TYPED_TEST(TestNumericBetweenKernel, SimpleBetweenArrayScalarScalar) {
+  using ScalarType = typename TypeTraits<TypeParam>::ScalarType;
+  using CType = typename TypeTraits<TypeParam>::CType;
+
+  Datum zero(std::make_shared<ScalarType>(CType(0)));
+  Datum four(std::make_shared<ScalarType>(CType(4)));
+  ValidateBetween<TypeParam>("[]", zero, four, "[]");
+  ValidateBetween<TypeParam>("[null]", zero, four, "[null]");
+  ValidateBetween<TypeParam>("[0,0,1,1,2,2]", zero, four, "[0,0,1,1,1,1]");
+  ValidateBetween<TypeParam>("[0,1,2,3,4,5]", zero, four, "[0,1,1,1,0,0]");
+  ValidateBetween<TypeParam>("[5,4,3,2,1,0]", zero, four, "[0,0,1,1,1,0]");
+  ValidateBetween<TypeParam>("[null,0,1,1]", zero, four, "[null,0,1,1]");
+}
+
+TEST(TestSimpleBetweenKernel, SimpleStringTest) {
+  using ScalarType = typename TypeTraits<StringType>::ScalarType;
+  auto l = Datum(std::make_shared<ScalarType>("abc"));
+  auto r = Datum(std::make_shared<ScalarType>("zzz"));
+  ValidateBetween<StringType>("[]", l, r, "[]");
+  ValidateBetween<StringType>("[null]", l, r, "[null]");
+  ValidateBetween<StringType>(R"(["aaa", "aaaa", "ccc", "z"])", l, r,
+                              R"([false, false, true, true])");
+  ValidateBetween<StringType>(R"(["a", "aaaa", "c", "z"])", l, r,
+                              R"([false, false, true, true])");
+  ValidateBetween<StringType>(R"(["a", "aaaa", "fff", "zzzz"])", l, r,
+                              R"([false, false, true, false])");
+  ValidateBetween<StringType>(R"(["abd", null, null, "zzx"])", l, r,
+                              R"([true, null, null, true])");
+}
+
+TEST(TestSimpleBetweenKernel, SimpleTimestampTest) {
+  using ScalarType = typename TypeTraits<TimestampType>::ScalarType;
+  auto checkTimestampArray = [](std::shared_ptr<DataType> type, const char* input_str,
+                                const Datum& lhs, const Datum& rhs,
+                                const char* expected_str) {
+    auto value = ArrayFromJSON(type, input_str);
+    auto expected = ArrayFromJSON(boolean(), expected_str);
+    ValidateBetween<TimestampType>(value, lhs, rhs, expected);
+  };
+  auto unit = TimeUnit::SECOND;
+  auto l = Datum(std::make_shared<ScalarType>(923184000, timestamp(unit)));
+  auto r = Datum(std::make_shared<ScalarType>(1602032602, timestamp(unit)));
+  checkTimestampArray(timestamp(unit), "[]", l, r, "[]");
+  checkTimestampArray(timestamp(unit), "[null]", l, r, "[null]");
+  checkTimestampArray(timestamp(unit), R"(["1970-01-01","2000-02-29","1900-02-28"])", l,
+                      r, "[false,true,false]");
+  checkTimestampArray(timestamp(unit), R"(["1970-01-01","2000-02-29","2004-02-28"])", l,
+                      r, "[false,true,true]");
+  checkTimestampArray(timestamp(unit), R"(["2018-01-01","1999-04-04","1900-02-28"])", l,
+                      r, "[true,false,false]");
+}
+
+TYPED_TEST(TestNumericBetweenKernel, SimpleBetweenArrayArrayArray) {
+  ValidateBetween<TypeParam>(
+      "[]", ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"), "[]");
+  ValidateBetween<TypeParam>(
+      "[null]", ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"), "[null]");
+  ValidateBetween<TypeParam>(
+      "[1,1,2,2,2]",
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      "[true,true,false,false,false]");
+  ValidateBetween<TypeParam>(
+      "[1,1,2,2,2,2]",
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,2,5,5]"),
+      "[true,true,false,null,false,false]");
+}
+
+TEST(TestSimpleBetweenKernel, StringArrayArrayArrayTest) {
+  ValidateBetween<StringType>(
+      R"(["david","hello","world"])",
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["adam","hi","whirl"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                    R"(["robert","goeiemoreen","whirlwind"])"),
+      "[true, false, false]");
+  ValidateBetween<StringType>(
+      R"(["x","a","f"])",
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["w","a","e"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["z","a","g"])"),
+      "[true, false, true]");
+  ValidateBetween<StringType>(
+      R"(["block","bit","binary"])",
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                    R"(["bit","nibble","ternary"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["word","d","xyz"])"),
+      "[true, false, false]");
+  ValidateBetween<StringType>(R"(["Ayumi","アユミ","王梦莹"])",
+                              ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                            R"(["たなか","あゆみ","歩美"])"),
+                              ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                            R"(["李平之","田中","たなか"])"),
+                              "[false, true, false]");

Review comment:
       Added comments in Jira




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

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

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



[GitHub] [arrow] bkmgit commented on a change in pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
bkmgit commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r768542510



##########
File path: cpp/src/arrow/compute/kernels/scalar_compare.cc
##########
@@ -746,6 +768,76 @@ std::shared_ptr<ScalarFunction> MakeScalarMinMax(std::string name,
   return func;
 }
 
+template <typename Op>
+void AddIntegerBetween(const std::shared_ptr<DataType>& ty, ScalarFunction* func) {
+  auto exec = GeneratePhysicalInteger<ScalarTernaryEqualTypes, BooleanType, Op>(*ty);
+  DCHECK_OK(func->AddKernel({ty, ty, ty}, boolean(), std::move(exec)));
+}
+
+template <typename InType, typename Op>
+void AddGenericBetween(const std::shared_ptr<DataType>& ty, ScalarFunction* func) {
+  DCHECK_OK(func->AddKernel({ty, ty, ty}, boolean(),
+                            ScalarTernaryEqualTypes<BooleanType, InType, Op>::Exec));
+}
+
+template <typename Op>
+std::shared_ptr<ScalarFunction> MakeBetweenFunction(std::string name,
+                                                    const FunctionDoc* doc) {
+  auto func = std::make_shared<CompareFunction>(name, Arity::Ternary(), doc);
+
+  for (const std::shared_ptr<DataType>& ty : IntTypes()) {
+    AddIntegerBetween<Op>(ty, func.get());
+  }
+
+  AddIntegerBetween<Op>(date32(), func.get());
+  AddIntegerBetween<Op>(date64(), func.get());
+
+  AddGenericBetween<FloatType, Op>(float32(), func.get());
+  AddGenericBetween<DoubleType, Op>(float64(), func.get());
+
+  // Add timestamp kernels
+  for (auto unit : TimeUnit::values()) {
+    InputType in_type(match::TimestampTypeUnit(unit));
+    auto exec =
+        GeneratePhysicalInteger<ScalarTernaryEqualTypes, BooleanType, Op>(int64());
+    DCHECK_OK(func->AddKernel({in_type, in_type, in_type}, boolean(), std::move(exec)));

Review comment:
       Created separate issue https://issues.apache.org/jira/browse/ARROW-15093 removed from this request.




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

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

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



[GitHub] [arrow] bkmgit commented on a change in pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
bkmgit commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r768853049



##########
File path: cpp/src/arrow/compute/kernels/scalar_compare_test.cc
##########
@@ -1850,5 +1851,154 @@ TEST(TestMaxElementWiseMinElementWise, CommonTemporal) {
               ResultWith(ScalarFromJSON(date64(), "86400000")));
 }
 
+template <typename ArrowType>
+static void ValidateBetween(const Datum& val, const Datum& lhs, const Datum& rhs,
+                            const Datum& expected) {
+  ASSERT_OK_AND_ASSIGN(Datum result, Between(val, lhs, rhs));
+  AssertArraysEqual(*expected.make_array(), *result.make_array(),
+                    /*verbose=*/true);
+}
+
+template <typename ArrowType>
+static void ValidateBetween(const char* value_str, const Datum& lhs, const Datum& rhs,
+                            const char* expected_str) {
+  auto value = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), value_str);
+  auto expected = ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), expected_str);
+  ValidateBetween<ArrowType>(value, lhs, rhs, expected);
+}
+
+template <>
+void ValidateBetween<StringType>(const char* value_str, const Datum& lhs,
+                                 const Datum& rhs, const char* expected_str) {
+  auto value = ArrayFromJSON(utf8(), value_str);
+  auto expected = ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), expected_str);
+  ValidateBetween<StringType>(value, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+class TestNumericBetweenKernel : public ::testing::Test {};
+
+TYPED_TEST_SUITE(TestNumericBetweenKernel, NumericArrowTypes);
+TYPED_TEST(TestNumericBetweenKernel, SimpleBetweenArrayScalarScalar) {
+  using ScalarType = typename TypeTraits<TypeParam>::ScalarType;
+  using CType = typename TypeTraits<TypeParam>::CType;
+
+  Datum zero(std::make_shared<ScalarType>(CType(0)));
+  Datum four(std::make_shared<ScalarType>(CType(4)));
+  ValidateBetween<TypeParam>("[]", zero, four, "[]");
+  ValidateBetween<TypeParam>("[null]", zero, four, "[null]");
+  ValidateBetween<TypeParam>("[0,0,1,1,2,2]", zero, four, "[0,0,1,1,1,1]");
+  ValidateBetween<TypeParam>("[0,1,2,3,4,5]", zero, four, "[0,1,1,1,0,0]");
+  ValidateBetween<TypeParam>("[5,4,3,2,1,0]", zero, four, "[0,0,1,1,1,0]");
+  ValidateBetween<TypeParam>("[null,0,1,1]", zero, four, "[null,0,1,1]");
+}
+
+TEST(TestSimpleBetweenKernel, SimpleStringTest) {
+  using ScalarType = typename TypeTraits<StringType>::ScalarType;
+  auto l = Datum(std::make_shared<ScalarType>("abc"));
+  auto r = Datum(std::make_shared<ScalarType>("zzz"));
+  ValidateBetween<StringType>("[]", l, r, "[]");
+  ValidateBetween<StringType>("[null]", l, r, "[null]");
+  ValidateBetween<StringType>(R"(["aaa", "aaaa", "ccc", "z"])", l, r,
+                              R"([false, false, true, true])");
+  ValidateBetween<StringType>(R"(["a", "aaaa", "c", "z"])", l, r,
+                              R"([false, false, true, true])");
+  ValidateBetween<StringType>(R"(["a", "aaaa", "fff", "zzzz"])", l, r,
+                              R"([false, false, true, false])");
+  ValidateBetween<StringType>(R"(["abd", null, null, "zzx"])", l, r,
+                              R"([true, null, null, true])");
+}
+
+TEST(TestSimpleBetweenKernel, SimpleTimestampTest) {

Review comment:
       Will leave this out for now, Timestamps can be done in a follow up.




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

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

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



[GitHub] [arrow] bkmgit commented on a change in pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
bkmgit commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r767453486



##########
File path: cpp/src/arrow/compute/kernels/scalar_compare_test.cc
##########
@@ -1850,5 +1851,154 @@ TEST(TestMaxElementWiseMinElementWise, CommonTemporal) {
               ResultWith(ScalarFromJSON(date64(), "86400000")));
 }
 
+template <typename ArrowType>
+static void ValidateBetween(const Datum& val, const Datum& lhs, const Datum& rhs,
+                            const Datum& expected) {
+  ASSERT_OK_AND_ASSIGN(Datum result, Between(val, lhs, rhs));
+  AssertArraysEqual(*expected.make_array(), *result.make_array(),
+                    /*verbose=*/true);
+}
+
+template <typename ArrowType>
+static void ValidateBetween(const char* value_str, const Datum& lhs, const Datum& rhs,
+                            const char* expected_str) {
+  auto value = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), value_str);
+  auto expected = ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), expected_str);
+  ValidateBetween<ArrowType>(value, lhs, rhs, expected);
+}
+
+template <>
+void ValidateBetween<StringType>(const char* value_str, const Datum& lhs,
+                                 const Datum& rhs, const char* expected_str) {
+  auto value = ArrayFromJSON(utf8(), value_str);
+  auto expected = ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), expected_str);
+  ValidateBetween<StringType>(value, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+class TestNumericBetweenKernel : public ::testing::Test {};
+
+TYPED_TEST_SUITE(TestNumericBetweenKernel, NumericArrowTypes);
+TYPED_TEST(TestNumericBetweenKernel, SimpleBetweenArrayScalarScalar) {
+  using ScalarType = typename TypeTraits<TypeParam>::ScalarType;
+  using CType = typename TypeTraits<TypeParam>::CType;
+
+  Datum zero(std::make_shared<ScalarType>(CType(0)));
+  Datum four(std::make_shared<ScalarType>(CType(4)));
+  ValidateBetween<TypeParam>("[]", zero, four, "[]");
+  ValidateBetween<TypeParam>("[null]", zero, four, "[null]");
+  ValidateBetween<TypeParam>("[0,0,1,1,2,2]", zero, four, "[0,0,1,1,1,1]");
+  ValidateBetween<TypeParam>("[0,1,2,3,4,5]", zero, four, "[0,1,1,1,0,0]");
+  ValidateBetween<TypeParam>("[5,4,3,2,1,0]", zero, four, "[0,0,1,1,1,0]");
+  ValidateBetween<TypeParam>("[null,0,1,1]", zero, four, "[null,0,1,1]");
+}
+
+TEST(TestSimpleBetweenKernel, SimpleStringTest) {
+  using ScalarType = typename TypeTraits<StringType>::ScalarType;
+  auto l = Datum(std::make_shared<ScalarType>("abc"));
+  auto r = Datum(std::make_shared<ScalarType>("zzz"));
+  ValidateBetween<StringType>("[]", l, r, "[]");
+  ValidateBetween<StringType>("[null]", l, r, "[null]");
+  ValidateBetween<StringType>(R"(["aaa", "aaaa", "ccc", "z"])", l, r,
+                              R"([false, false, true, true])");
+  ValidateBetween<StringType>(R"(["a", "aaaa", "c", "z"])", l, r,
+                              R"([false, false, true, true])");
+  ValidateBetween<StringType>(R"(["a", "aaaa", "fff", "zzzz"])", l, r,
+                              R"([false, false, true, false])");
+  ValidateBetween<StringType>(R"(["abd", null, null, "zzx"])", l, r,
+                              R"([true, null, null, true])");
+}
+
+TEST(TestSimpleBetweenKernel, SimpleTimestampTest) {
+  using ScalarType = typename TypeTraits<TimestampType>::ScalarType;
+  auto checkTimestampArray = [](std::shared_ptr<DataType> type, const char* input_str,
+                                const Datum& lhs, const Datum& rhs,
+                                const char* expected_str) {
+    auto value = ArrayFromJSON(type, input_str);
+    auto expected = ArrayFromJSON(boolean(), expected_str);
+    ValidateBetween<TimestampType>(value, lhs, rhs, expected);
+  };
+  auto unit = TimeUnit::SECOND;
+  auto l = Datum(std::make_shared<ScalarType>(923184000, timestamp(unit)));
+  auto r = Datum(std::make_shared<ScalarType>(1602032602, timestamp(unit)));
+  checkTimestampArray(timestamp(unit), "[]", l, r, "[]");
+  checkTimestampArray(timestamp(unit), "[null]", l, r, "[null]");
+  checkTimestampArray(timestamp(unit), R"(["1970-01-01","2000-02-29","1900-02-28"])", l,
+                      r, "[false,true,false]");
+  checkTimestampArray(timestamp(unit), R"(["1970-01-01","2000-02-29","2004-02-28"])", l,
+                      r, "[false,true,true]");
+  checkTimestampArray(timestamp(unit), R"(["2018-01-01","1999-04-04","1900-02-28"])", l,
+                      r, "[true,false,false]");
+}
+
+TYPED_TEST(TestNumericBetweenKernel, SimpleBetweenArrayArrayArray) {
+  ValidateBetween<TypeParam>(
+      "[]", ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"), "[]");
+  ValidateBetween<TypeParam>(
+      "[null]", ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"), "[null]");
+  ValidateBetween<TypeParam>(
+      "[1,1,2,2,2]",
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      "[true,true,false,false,false]");
+  ValidateBetween<TypeParam>(
+      "[1,1,2,2,2,2]",
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,2,5,5]"),
+      "[true,true,false,null,false,false]");
+}
+
+TEST(TestSimpleBetweenKernel, StringArrayArrayArrayTest) {
+  ValidateBetween<StringType>(
+      R"(["david","hello","world"])",
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["adam","hi","whirl"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                    R"(["robert","goeiemoreen","whirlwind"])"),
+      "[true, false, false]");
+  ValidateBetween<StringType>(
+      R"(["x","a","f"])",
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["w","a","e"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["z","a","g"])"),
+      "[true, false, true]");
+  ValidateBetween<StringType>(
+      R"(["block","bit","binary"])",
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                    R"(["bit","nibble","ternary"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["word","d","xyz"])"),
+      "[true, false, false]");
+  ValidateBetween<StringType>(R"(["Ayumi","アユミ","王梦莹"])",
+                              ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                            R"(["たなか","あゆみ","歩美"])"),
+                              ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                            R"(["李平之","田中","たなか"])"),
+                              "[false, true, false]");

Review comment:
       Yes. It can be removed later if a different implementation is done with different orderings, but non ASCII characters in UTF8 are likely to occur.




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

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

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



[GitHub] [arrow] bkmgit commented on a change in pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
bkmgit commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r771991996



##########
File path: cpp/src/arrow/compute/kernels/scalar_compare.cc
##########
@@ -746,6 +768,76 @@ std::shared_ptr<ScalarFunction> MakeScalarMinMax(std::string name,
   return func;
 }
 
+template <typename Op>
+void AddIntegerBetween(const std::shared_ptr<DataType>& ty, ScalarFunction* func) {
+  auto exec = GeneratePhysicalInteger<ScalarTernaryEqualTypes, BooleanType, Op>(*ty);
+  DCHECK_OK(func->AddKernel({ty, ty, ty}, boolean(), std::move(exec)));
+}
+
+template <typename InType, typename Op>
+void AddGenericBetween(const std::shared_ptr<DataType>& ty, ScalarFunction* func) {
+  DCHECK_OK(func->AddKernel({ty, ty, ty}, boolean(),
+                            ScalarTernaryEqualTypes<BooleanType, InType, Op>::Exec));
+}
+
+template <typename Op>
+std::shared_ptr<ScalarFunction> MakeBetweenFunction(std::string name,
+                                                    const FunctionDoc* doc) {
+  auto func = std::make_shared<CompareFunction>(name, Arity::Ternary(), doc);
+
+  for (const std::shared_ptr<DataType>& ty : IntTypes()) {
+    AddIntegerBetween<Op>(ty, func.get());
+  }
+
+  AddIntegerBetween<Op>(date32(), func.get());
+  AddIntegerBetween<Op>(date64(), func.get());
+
+  AddGenericBetween<FloatType, Op>(float32(), func.get());
+  AddGenericBetween<DoubleType, Op>(float64(), func.get());
+
+  // Add timestamp kernels
+  for (auto unit : TimeUnit::values()) {
+    InputType in_type(match::TimestampTypeUnit(unit));
+    auto exec =
+        GeneratePhysicalInteger<ScalarTernaryEqualTypes, BooleanType, Op>(int64());
+    DCHECK_OK(func->AddKernel({in_type, in_type, in_type}, boolean(), std::move(exec)));

Review comment:
       Have implemented this here.




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

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

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



[GitHub] [arrow] bkmgit commented on a change in pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
bkmgit commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r775463859



##########
File path: cpp/src/arrow/util/bit_block_counter.h
##########
@@ -424,6 +565,240 @@ class ARROW_EXPORT OptionalBinaryBitBlockCounter {
   }
 };
 
+/// \brief A class that computes popcounts on the result of bitwise operations
+/// between three bitmaps, 64 bits at a time. A 64-bit word is loaded from each
+/// bitmap, then the popcount is computed on e.g. the bitwise-and of the three
+/// words.
+class ARROW_EXPORT TernaryBitBlockCounter {
+ public:
+  TernaryBitBlockCounter(const uint8_t* left_bitmap, int64_t left_offset,
+                         const uint8_t* mid_bitmap, int64_t mid_offset,
+                         const uint8_t* right_bitmap, int64_t right_offset,
+                         int64_t length)
+      : left_bitmap_(util::MakeNonNull(left_bitmap) + left_offset / 8),
+        left_offset_(left_offset % 8),
+        mid_bitmap_(util::MakeNonNull(mid_bitmap) + mid_offset / 8),
+        mid_offset_(mid_offset % 8),
+        right_bitmap_(util::MakeNonNull(right_bitmap) + right_offset / 8),
+        right_offset_(right_offset % 8),
+        bits_remaining_(length) {}
+
+  /// \brief Return the popcount of the bitwise-and of the next run of
+  /// available bits, up to 64. The returned pair contains the size of run and
+  /// the number of true values. The last block will have a length less than 64
+  /// if the bitmap length is not a multiple of 64, and will return 0-length
+  /// blocks in subsequent invocations.
+  BitBlockCount NextAndAndWord() { return NextWord<detail::BitBlockAndAnd>(); }
+
+  /// \brief Computes "x & ~y & ~z" block for each available run of bits.
+  BitBlockCount NextAndNotAndNotWord() {
+    return NextWord<detail::BitBlockAndNotAndNot>();
+  }
+
+  /// \brief Computes "~x & y & ~z" block for each available run of bits.
+  BitBlockCount NextNotAndAndNotWord() {
+    return NextWord<detail::BitBlockNotAndAndNot>();
+  }
+
+  /// \brief Computes "~x & ~y & z" block for each available run of bits.
+  BitBlockCount NextNotAndNotAndWord() {
+    return NextWord<detail::BitBlockNotAndNotAnd>();
+  }
+
+  /// \brief Computes "~x & y & z" block for each available run of bits.
+  BitBlockCount NextNotAndAndWord() { return NextWord<detail::BitBlockNotAndAnd>(); }
+
+  /// \brief Computes "x & ~y & z" block for each available run of bits.
+  BitBlockCount NextAndNotAndWord() { return NextWord<detail::BitBlockAndNotAnd>(); }
+
+  /// \brief Computes "x & y & ~z" block for each available run of bits.
+  BitBlockCount NextAndAndNotWord() { return NextWord<detail::BitBlockNotAndNotAnd>(); }
+
+  /// \brief Computes "x | y | z" block for each available run of bits.
+  BitBlockCount NextOrOrWord() { return NextWord<detail::BitBlockOrOr>(); }
+
+  /// \brief Computes "x | ~y | z" block for each available run of bits.
+  BitBlockCount NextOrNotOrWord() { return NextWord<detail::BitBlockOrNotOr>(); }
+
+  /// \brief Computes "~x | y | z" block for each available run of bits.
+  BitBlockCount NextNotOrOrWord() { return NextWord<detail::BitBlockNotOrOr>(); }
+
+  /// \brief Computes "x | y | ~z" block for each available run of bits.
+  BitBlockCount NextOrOrNotWord() { return NextWord<detail::BitBlockOrOrNot>(); }
+
+  /// \brief Computes "~x | y | ~z" block for each available run of bits.
+  BitBlockCount NextNotOrOrNotWord() { return NextWord<detail::BitBlockNotOrOrNot>(); }
+
+  /// \brief Computes "x | ~y | ~z" block for each available run of bits.
+  BitBlockCount NextOrNotOrNotWord() { return NextWord<detail::BitBlockOrNotOrNot>(); }
+
+  /// \brief Computes "~x | ~y | z" block for each available run of bits.
+  BitBlockCount NextNotOrNotOrWord() { return NextWord<detail::BitBlockNotOrNotOr>(); }
+
+ private:
+  template <template <typename T> class Op>
+  BitBlockCount NextWord() {
+    using detail::LoadWord;
+    using detail::ShiftWord;
+
+    if (!bits_remaining_) {
+      return {0, 0};
+    }
+    // When the offset is > 0, we need there to be a word beyond the last aligned
+    // word in the bitmap for the bit shifting logic.
+    constexpr int64_t kWordBits = BitBlockCounter::kWordBits;
+    const int64_t bits_required_to_use_words =
+        std::max({left_offset_ == 0 ? 64 : 64 + (64 - left_offset_),
+                  mid_offset_ == 0 ? 64 : 64 + (64 - mid_offset_),
+                  right_offset_ == 0 ? 64 : 64 + (64 - right_offset_)});
+    if (bits_remaining_ < bits_required_to_use_words) {
+      const int16_t run_length =
+          static_cast<int16_t>(std::min(bits_remaining_, kWordBits));
+      int16_t popcount = 0;
+      for (int64_t i = 0; i < run_length; ++i) {
+        if (Op<bool>::Call(bit_util::GetBit(left_bitmap_, left_offset_ + i),
+                           bit_util::GetBit(mid_bitmap_, mid_offset_ + i),
+                           bit_util::GetBit(right_bitmap_, right_offset_ + i))) {
+          ++popcount;
+        }
+      }
+      // This code path should trigger _at most_ 2 times. In the "two times"
+      // case, the first time the run length will be a multiple of 8.
+      left_bitmap_ += run_length / 8;
+      mid_bitmap_ += run_length / 8;
+      right_bitmap_ += run_length / 8;
+      bits_remaining_ -= run_length;
+      return {run_length, popcount};
+    }
+
+    int64_t popcount = 0;
+    if (left_offset_ == 0 && right_offset_ == 0 && mid_offset_ == 0) {
+      popcount = bit_util::PopCount(Op<uint64_t>::Call(
+          LoadWord(left_bitmap_), LoadWord(mid_bitmap_), LoadWord(right_bitmap_)));
+    } else {
+      auto left_word =
+          ShiftWord(LoadWord(left_bitmap_), LoadWord(left_bitmap_ + 8), left_offset_);
+      auto mid_word =
+          ShiftWord(LoadWord(mid_bitmap_), LoadWord(mid_bitmap_ + 8), mid_offset_);
+      auto right_word =
+          ShiftWord(LoadWord(right_bitmap_), LoadWord(right_bitmap_ + 8), right_offset_);
+      popcount = bit_util::PopCount(Op<uint64_t>::Call(left_word, mid_word, right_word));
+    }
+    left_bitmap_ += kWordBits / 8;
+    mid_bitmap_ += kWordBits / 8;
+    right_bitmap_ += kWordBits / 8;
+    bits_remaining_ -= kWordBits;
+    return {64, static_cast<int16_t>(popcount)};
+  }
+
+  const uint8_t* left_bitmap_;
+  int64_t left_offset_;
+  const uint8_t* mid_bitmap_;
+  int64_t mid_offset_;
+  const uint8_t* right_bitmap_;
+  int64_t right_offset_;
+  int64_t bits_remaining_;
+};
+
+class ARROW_EXPORT OptionalTernaryBitBlockCounter {
+ public:
+  // Any bitmap may be NULLPTR
+  OptionalTernaryBitBlockCounter(const uint8_t* left_bitmap, int64_t left_offset,

Review comment:
       Removed since at present not used.




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

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

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



[GitHub] [arrow] edponce commented on a change in pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
edponce commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r776638260



##########
File path: cpp/src/arrow/util/bit_block_counter.h
##########
@@ -87,6 +87,147 @@ struct BitBlockOrNot<bool> {
   static bool Call(bool left, bool right) { return left || !right; }
 };
 
+// Three Arguments
+template <typename T>
+struct BitBlockAndAnd {
+  static T Call(T left, T mid, T right) { return left & mid & right; }
+};
+
+template <>
+struct BitBlockAndAnd<bool> {
+  static bool Call(bool left, bool mid, bool right) { return left && mid && right; }

Review comment:
       There is no need to specialize all these bitwise operations for `bool` as the language guarantees that for boolean types: `(bool)(a & b) == a && b`. Note the explicit cast which corresponds to the return type `T` in these functions. This simplification also applies to the existing binary bitwise operations, so we can resolve this in a follow-up PR (ARROW-15220).




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

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

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



[GitHub] [arrow] bkmgit commented on a change in pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
bkmgit commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r776652189



##########
File path: cpp/src/arrow/compute/kernels/scalar_compare.cc
##########
@@ -746,6 +787,111 @@ std::shared_ptr<ScalarFunction> MakeScalarMinMax(std::string name,
   return func;
 }
 
+template <typename Op>
+void AddIntegerBetween(const std::shared_ptr<DataType>& ty, ScalarFunction* func) {
+  auto exec = GeneratePhysicalInteger<ScalarTernaryEqualTypes, BooleanType, Op>(*ty);
+  DCHECK_OK(func->AddKernel({ty, ty, ty}, boolean(), std::move(exec)));
+}
+
+template <typename InType, typename Op>
+void AddGenericBetween(const std::shared_ptr<DataType>& ty, ScalarFunction* func) {
+  DCHECK_OK(func->AddKernel({ty, ty, ty}, boolean(),
+                            ScalarTernaryEqualTypes<BooleanType, InType, Op>::Exec));
+}
+
+template <typename OutType, typename ArgType, typename Op>
+struct BetweenTimestamps : public ScalarTernaryEqualTypes<OutType, ArgType, Op> {
+  using Base = ScalarTernaryEqualTypes<OutType, ArgType, Op>;
+
+  static Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    const auto& var = checked_cast<const TimestampType&>(*batch[0].type());
+    const auto& lhs = checked_cast<const TimestampType&>(*batch[1].type());
+    const auto& rhs = checked_cast<const TimestampType&>(*batch[2].type());
+    if ((var.timezone().empty() != lhs.timezone().empty()) ||
+        (var.timezone().empty() != rhs.timezone().empty()) ||
+        (lhs.timezone().empty() != rhs.timezone().empty())) {
+      return Status::Invalid(
+          "Cannot use timestamps with timezone and timestamps without timezones, got: ",
+          var.timezone().empty(), " ", lhs.timezone().empty(), " and ",

Review comment:
       Done.




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

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

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



[GitHub] [arrow] bkmgit commented on a change in pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
bkmgit commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r776649821



##########
File path: cpp/src/arrow/compute/api_scalar.h
##########
@@ -1240,5 +1255,22 @@ ARROW_EXPORT Result<Datum> AssumeTimezone(const Datum& values,
                                           AssumeTimezoneOptions options,
                                           ExecContext* ctx = NULLPTR);
 
+/// \brief Between compares each element in `values`
+/// with `left` as a lower bound and 'right' as an upperbound

Review comment:
       Done thanks.




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

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

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



[GitHub] [arrow] edponce edited a comment on pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
edponce edited a comment on pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#issuecomment-1002956781


   I have the following observations to make wrt to the API for this `between` compute function. The purpose of `between` is to support range comparisons similar to [SQL `BETWEEN`](https://www.w3schools.com/sql/sql_between.asp) and [pandas `between`](https://pandas.pydata.org/docs/reference/api/pandas.Series.between.html#pandas-series-between). These APIs are different from this PR's implementation, but the behavior is equivalent. Nevertheless, I suggest either of the following:
   1. Implement `between` compute function not as a _logical compare_ function but as an independent function that takes 3 inputs and an option for declaring bounds-inclusiveness (BOTH, NONE, LEFT, RIGHT). Similar to the Pandas API.
   2. Implement `between` as an extension to the _logical compare_ functions. Now there are two ways to go about this:
     a. As this PR does, implement 4 new compare functions `less_less`, `less_equal_less`, `less_less_equal`, and `less_equal_less_equal`.
     b. Have a single variadic `compare` function that uses options to select the type of comparison (e.g., LESS, GREATER_EQUAL, LESS_LESS)
   
   I recommend to follow option (1), for the following reasons. It is a common API and the function name clearly states its operation. The issue with (2a) is that its function names are not common-case, and if we make a single `between` with options for LESS_LESS, etc. then I would argue that the other logical comparisons need to be merged into a single compute function for symmetry/consistency purposes. This last point touches on option (2b) which I do not think is a reasonable solution because of its complexity and one could argue that an analogous case would be to have a single `arithmetic` function where the operation performed is specified via options. Obviously, this is not desired.


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

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

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



[GitHub] [arrow] bkmgit commented on a change in pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
bkmgit commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r776929889



##########
File path: cpp/src/arrow/compute/api_scalar.cc
##########
@@ -99,6 +99,30 @@ struct EnumTraits<compute::CompareOperator>
     return "<INVALID>";
   }
 };
+
+template <>
+struct EnumTraits<compute::BetweenOptions::Inclusiveness>
+    : BasicEnumTraits<compute::BetweenOptions::Inclusiveness,
+                      compute::BetweenOptions::Inclusiveness::BOTH,
+                      compute::BetweenOptions::Inclusiveness::LEFT,
+                      compute::BetweenOptions::Inclusiveness::RIGHT,
+                      compute::BetweenOptions::Inclusiveness::NEITHER> {
+  static std::string name() { return "BetweenOptions::Inclusiveness"; }
+  static std::string value_name(compute::BetweenOptions::Inclusiveness value) {
+    switch (value) {
+      case compute::BetweenOptions::Inclusiveness::BOTH:
+        return "BOTH";
+      case compute::BetweenOptions::Inclusiveness::LEFT:
+        return "LEFT";
+      case compute::BetweenOptions::Inclusiveness::RIGHT:
+        return "RIGHT";
+      case compute::BetweenOptions::Inclusiveness::NEITHER:

Review comment:
       Ok




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

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

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



[GitHub] [arrow] bkmgit commented on a change in pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
bkmgit commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r776589669



##########
File path: cpp/src/arrow/compute/kernels/scalar_compare.cc
##########
@@ -68,6 +72,50 @@ struct GreaterEqual {
   }
 };
 
+struct BetweenLessEqualLessEqual {
+  template <typename T, typename Arg0, typename Arg1, typename Arg2>
+  static constexpr T Call(KernelContext*, const Arg0& middle, const Arg1& left,
+                          const Arg2& right, Status*) {
+    static_assert(std::is_same<T, bool>::value && std::is_same<Arg0, Arg1>::value &&
+                      std::is_same<Arg1, Arg2>::value,
+                  "");
+    return (left <= middle) && (middle <= right);
+  }
+};
+
+struct BetweenLessThanLessEqual {
+  template <typename T, typename Arg0, typename Arg1, typename Arg2>
+  static constexpr T Call(KernelContext*, const Arg0& middle, const Arg1& left,
+                          const Arg2& right, Status*) {
+    static_assert(std::is_same<T, bool>::value && std::is_same<Arg0, Arg1>::value &&
+                      std::is_same<Arg1, Arg2>::value,
+                  "");
+    return (left < middle) && (middle <= right);
+  }
+};
+
+struct BetweenLessEqualLessThan {
+  template <typename T, typename Arg0, typename Arg1, typename Arg2>
+  static constexpr T Call(KernelContext*, const Arg0& middle, const Arg1& left,
+                          const Arg2& right, Status*) {
+    static_assert(std::is_same<T, bool>::value && std::is_same<Arg0, Arg1>::value &&
+                      std::is_same<Arg1, Arg2>::value,
+                  "");
+    return (left <= middle) && (middle < right);
+  }
+};
+
+struct BetweenLessThanLessThan {
+  template <typename T, typename Arg0, typename Arg1, typename Arg2>
+  static constexpr T Call(KernelContext*, const Arg0& middle, const Arg1& left,
+                          const Arg2& right, Status*) {
+    static_assert(std::is_same<T, bool>::value && std::is_same<Arg0, Arg1>::value &&
+                      std::is_same<Arg1, Arg2>::value,
+                  "");
+    return (left < middle) && (middle < right);
+  }
+};

Review comment:
       Ok. Even with UTF-8, sorting and comparison is not trivial, but a FunctionOption for that makes sense. This can be implemented later. 
   
   @kou @pitrou At present having four separate between functions seems reasonable. Will add Python bindings. Ideally, this could end up in 7.0.0 - then can request feedback from users and either keep the current structure for Between and Compare or use correctly implemented FunctionOptions for both Between and Compare, rather than having a wrapper - this would end up in 8.0.0.




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

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

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



[GitHub] [arrow] edponce edited a comment on pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
edponce edited a comment on pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#issuecomment-1002956781


   I have the following observations to make wrt to the API for this `between` compute function. The purpose of `between` is to support range comparisons similar to [SQL `BETWEEN`](https://www.w3schools.com/sql/sql_between.asp) and [pandas `between`](https://pandas.pydata.org/docs/reference/api/pandas.Series.between.html#pandas-series-between). These APIs are different from this PR's implementation, but the behavior is equivalent. Nevertheless, I suggest either of the following:
   1. Implement `between` compute function not as a _logical compare_ function but as an independent range-based function that takes 3 inputs and an option for declaring bounds-inclusiveness (BOTH, NONE, LEFT, RIGHT). Similar to the Pandas API.
   2. Implement `between` as an extension to the _logical compare_ functions. Now there are two ways to go about this:
     a. As this PR does, implement 4 new compare functions `less_less`, `less_equal_less`, `less_less_equal`, and `less_equal_less_equal`.
     b. Have a single variadic `compare` function that uses options to select the type of comparison (e.g., LESS, GREATER_EQUAL, LESS_LESS)
   
   I recommend to follow option (1) because it is a common API and the function name clearly states its operation. The issue with (2a) is that its function names are not common-case, and if we make a single `between` with options for LESS_LESS, etc. then I would argue that the other logical comparisons need to be merged into a single compute function for symmetry/consistency purposes. This last point touches on option (2b) which I do not think is a reasonable solution because of its complexity and one could argue that an analogous case would be to have a single `arithmetic` function where the operation performed is specified via options. Obviously, this is not desired.


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

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

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



[GitHub] [arrow] edponce commented on pull request #11882: ARROW-9843: [C++][Python] Implement Between ternary kernel and Python bindings

Posted by GitBox <gi...@apache.org>.
edponce commented on pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#issuecomment-1072048776


   Is there anything left to do in this PR?


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

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

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



[GitHub] [arrow] lidavidm commented on pull request #11882: ARROW-9843: [C++][Python] Implement Between ternary kernel and Python bindings

Posted by GitBox <gi...@apache.org>.
lidavidm commented on pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#issuecomment-1072372569


   It also needs to be rebased now.


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

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

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



[GitHub] [arrow] bkmgit commented on pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
bkmgit commented on pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#issuecomment-1008333615


   @edponce Thanks for making it work. Other than formatting, tests seem to pass for commit https://github.com/apache/arrow/pull/11882/commits/b84ca3b63fcbf13b03e13c40ada6a7e93b141811 . Need to add further tests for
   
   - TimeStamps
   - ARRAY_SCALAR_ARRAY and related variations
   
   Automated random tests work for these, but they typically do not catch corner cases.
   
   Started on Python bindings, but still figuring these out.


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

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

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



[GitHub] [arrow] bkmgit commented on a change in pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
bkmgit commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r782685081



##########
File path: cpp/src/arrow/compute/api_scalar.h
##########
@@ -1350,5 +1371,24 @@ ARROW_EXPORT Result<Datum> AssumeTimezone(const Datum& values,
                                           AssumeTimezoneOptions options,
                                           ExecContext* ctx = NULLPTR);
 
+/// \brief Between compares each element in `values`
+/// with `left` as a lower bound and 'right' as an upper bound
+///
+/// \param[in] values input to compare between left and right
+/// \param[in] left used as the lower bound for comparison
+/// \param[in] right used as the upper bound for comparison
+/// \param[in] options for bounds, default is inclusive of both, other
+///  endpoints, other choices are left (exclude left endpoint), right
+///  (exclude right endpoint) and both (exclude both endpoints), optional
+/// \param[in] ctx the function execution context, optional
+///
+/// \return the resulting datum

Review comment:
       Hopefully, may need to update.




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

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

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



[GitHub] [arrow] edponce commented on a change in pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
edponce commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r785333007



##########
File path: cpp/src/arrow/util/bit_block_counter.h
##########
@@ -46,7 +46,7 @@ inline uint64_t ShiftWord(uint64_t current, uint64_t next, int64_t shift) {
 }
 
 // These templates are here to help with unit tests
-
+// Two Arguments
 template <typename T>
 constexpr T BitNot(T x) {

Review comment:
       This comment only applies to `BitBlockAnd` and `BitBlockAndNot`. IMO, I would remove these comments because the function's signature if clear enough to identify the number of arguments.

##########
File path: go/arrow/memory/_lib/arch.h
##########
@@ -22,8 +22,6 @@
     #define FULL_NAME(x) x##_sse4
 #elif __SSE3__ == 1
     #define FULL_NAME(x) x##_sse3
-#elif defined(__ARM_NEON) || defined(__ARM_NEON__)
-    #define FULL_NAME(x) x##_neon
 #else

Review comment:
       Why this change?

##########
File path: dev/tasks/java-jars/README.md
##########
@@ -16,7 +16,7 @@ See the License for the specific language governing permissions and
 limitations under the License.
 -->
 
-# Java Jars Task
+# Jars.

Review comment:
       Remove the period?

##########
File path: go/arrow/memory/_lib/arch.h
##########
@@ -22,8 +22,6 @@
     #define FULL_NAME(x) x##_sse4
 #elif __SSE3__ == 1
     #define FULL_NAME(x) x##_sse3
-#elif defined(__ARM_NEON) || defined(__ARM_NEON__)
-    #define FULL_NAME(x) x##_neon
 #else
     #define FULL_NAME(x) x##_x86
 #endif

Review comment:
       Missing EOF

##########
File path: go/arrow/memory/Makefile
##########
@@ -56,11 +46,9 @@ _lib/memory_avx2.s: _lib/memory.c
 _lib/memory_sse4.s: _lib/memory.c
 	$(CC) -S $(C_FLAGS) $(ASM_FLAGS_SSE4) $^ -o $@ ; $(PERL_FIXUP_ROTATE) $@
 
-_lib/memory_neon.s: _lib/memory.c
-	$(CC) -S $(C_FLAGS_NEON) $^ -o $@ ; $(PERL_FIXUP_ROTATE) $@
-
 memory_avx2_amd64.s: _lib/memory_avx2.s

Review comment:
       Why all these changes to this Makefile?

##########
File path: ci/docker/java-jni-manylinux-201x.dockerfile
##########
@@ -30,8 +30,7 @@ RUN vcpkg install --clean-after-build \
         boost-regex \
         boost-system \
         boost-variant \
-        # Use enable rtti to avoid link problems in Gandiva
-        llvm[clang,default-options,default-targets,lld,tools,enable-rtti]
+        llvm

Review comment:
       Why this change?




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

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

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



[GitHub] [arrow] bkmgit commented on a change in pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
bkmgit commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r785334190



##########
File path: ci/docker/java-jni-manylinux-201x.dockerfile
##########
@@ -30,8 +30,7 @@ RUN vcpkg install --clean-after-build \
         boost-regex \
         boost-system \
         boost-variant \
-        # Use enable rtti to avoid link problems in Gandiva
-        llvm[clang,default-options,default-targets,lld,tools,enable-rtti]
+        llvm

Review comment:
       From rebase, see https://github.com/apache/arrow/blob/master/ci/docker/java-jni-manylinux-201x.dockerfile#L33-L34




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

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

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



[GitHub] [arrow] bkmgit commented on a change in pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
bkmgit commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r785343003



##########
File path: go/arrow/memory/_lib/arch.h
##########
@@ -22,8 +22,6 @@
     #define FULL_NAME(x) x##_sse4
 #elif __SSE3__ == 1
     #define FULL_NAME(x) x##_sse3
-#elif defined(__ARM_NEON) || defined(__ARM_NEON__)
-    #define FULL_NAME(x) x##_neon
 #else
     #define FULL_NAME(x) x##_x86
 #endif

Review comment:
       Fixed.




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

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

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



[GitHub] [arrow] bkmgit commented on a change in pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
bkmgit commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r785343038



##########
File path: go/arrow/memory/_lib/arch.h
##########
@@ -22,8 +22,6 @@
     #define FULL_NAME(x) x##_sse4
 #elif __SSE3__ == 1
     #define FULL_NAME(x) x##_sse3
-#elif defined(__ARM_NEON) || defined(__ARM_NEON__)
-    #define FULL_NAME(x) x##_neon
 #else

Review comment:
       Fixed poor rebase




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

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

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



[GitHub] [arrow] edponce commented on a change in pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
edponce commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r776597556



##########
File path: cpp/src/arrow/compute/api_scalar.h
##########
@@ -1240,5 +1255,22 @@ ARROW_EXPORT Result<Datum> AssumeTimezone(const Datum& values,
                                           AssumeTimezoneOptions options,
                                           ExecContext* ctx = NULLPTR);
 
+/// \brief Between compares each element in `values`
+/// with `left` as a lower bound and 'right' as an upperbound

Review comment:
       `upperbound` --> `upper bound`

##########
File path: cpp/src/arrow/compute/api_scalar.h
##########
@@ -1240,5 +1255,22 @@ ARROW_EXPORT Result<Datum> AssumeTimezone(const Datum& values,
                                           AssumeTimezoneOptions options,
                                           ExecContext* ctx = NULLPTR);
 
+/// \brief Between compares each element in `values`
+/// with `left` as a lower bound and 'right' as an upperbound
+///
+/// \param[in] values input to compare between left and right
+/// \param[in] left used as the lower bound for comparison
+/// \param[in] right used as the upper bound for comparison
+//  \param[in] options between options, optional
+/// \param[in] ctx the function execution context, optional
+///
+/// \return the resulting datum
+///
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Between(const Datum& values, const Datum& left, const Datum& right,
+                      BetweenOptions options = BetweenOptions(),

Review comment:
       `BetweenOptions()` --> `BetweenOptions::Defaults()`

##########
File path: cpp/src/arrow/compute/api_scalar.cc
##########
@@ -506,6 +537,12 @@ void RegisterScalarOptions(FunctionRegistry* registry) {
     return CallFunction(REGISTRY_NAME, {left, right}, ctx);                     \
   }
 
+#define SCALAR_EAGER_TERNARY(NAME, REGISTRY_NAME)                               \
+  Result<Datum> NAME(const Datum& value, const Datum& left, const Datum& right, \
+                     ExecContext* ctx) {                                        \
+    return CallFunction(REGISTRY_NAME, {value, left, right}, ctx);              \
+  }
+

Review comment:
       Remove this macro as it is not used and at the moment there are no other ternary functions for a general macro to be too useful.

##########
File path: cpp/src/arrow/compute/api_scalar.h
##########
@@ -316,6 +316,21 @@ struct ARROW_EXPORT CompareOptions {
   enum CompareOperator op;
 };
 
+enum class BetweenMode : int8_t {
+  LESS_EQUAL_LESS_EQUAL,
+  LESS_EQUAL_LESS,
+  LESS_LESS_EQUAL,
+  LESS_LESS,
+};

Review comment:
       I suggest to place this `enum` inside `BetweenOptions` as it is not necessary to have it in global space. Also rename to `Operator`, so that it would be invoked as `BetweenOptions::Operator::LESS_LESS` or `BetweenOptions::LESS_LESS`.
   
   I recognize there are some inconsistencies wrt to the visibility and scopeness of enums, and it is out-of-scope for this PR to discuss such details. I will ask to get feedback on whether we should revise the conventions for `FunctionOptions`.

##########
File path: cpp/src/arrow/compute/kernels/scalar_compare.cc
##########
@@ -786,6 +932,34 @@ const FunctionDoc max_element_wise_doc{
     {"*args"},
     "ElementWiseAggregateOptions"};
 
+const FunctionDoc between_doc{"Check if values are in a range, val betwen a and b",

Review comment:
       Remove `between_doc` because there is no `CallFunction("between", ...)`.

##########
File path: cpp/src/arrow/util/bit_block_counter.h
##########
@@ -87,6 +87,147 @@ struct BitBlockOrNot<bool> {
   static bool Call(bool left, bool right) { return left || !right; }
 };
 
+// Three Arguments
+template <typename T>
+struct BitBlockAndAnd {
+  static T Call(T left, T mid, T right) { return left & mid & right; }
+};
+
+template <>
+struct BitBlockAndAnd<bool> {
+  static bool Call(bool left, bool mid, bool right) { return left && mid && right; }

Review comment:
       There is no need to specialize all these bitwise operations for `bool` as the language guarantees that for boolean types: `(bool)(a & b) == a && b`. Note the explicit cast which corresponds to the return type `T` in these functions. This simplification also applies to the existing binary bitwise operations, so we can resolve this in a follow-up PR.

##########
File path: cpp/src/arrow/compute/kernels/scalar_compare.cc
##########
@@ -746,6 +787,111 @@ std::shared_ptr<ScalarFunction> MakeScalarMinMax(std::string name,
   return func;
 }
 
+template <typename Op>
+void AddIntegerBetween(const std::shared_ptr<DataType>& ty, ScalarFunction* func) {
+  auto exec = GeneratePhysicalInteger<ScalarTernaryEqualTypes, BooleanType, Op>(*ty);
+  DCHECK_OK(func->AddKernel({ty, ty, ty}, boolean(), std::move(exec)));
+}
+
+template <typename InType, typename Op>
+void AddGenericBetween(const std::shared_ptr<DataType>& ty, ScalarFunction* func) {
+  DCHECK_OK(func->AddKernel({ty, ty, ty}, boolean(),
+                            ScalarTernaryEqualTypes<BooleanType, InType, Op>::Exec));
+}
+
+template <typename OutType, typename ArgType, typename Op>
+struct BetweenTimestamps : public ScalarTernaryEqualTypes<OutType, ArgType, Op> {
+  using Base = ScalarTernaryEqualTypes<OutType, ArgType, Op>;
+
+  static Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    const auto& var = checked_cast<const TimestampType&>(*batch[0].type());
+    const auto& lhs = checked_cast<const TimestampType&>(*batch[1].type());
+    const auto& rhs = checked_cast<const TimestampType&>(*batch[2].type());
+    if ((var.timezone().empty() != lhs.timezone().empty()) ||
+        (var.timezone().empty() != rhs.timezone().empty()) ||
+        (lhs.timezone().empty() != rhs.timezone().empty())) {
+      return Status::Invalid(
+          "Cannot use timestamps with timezone and timestamps without timezones, got: ",
+          var.timezone().empty(), " ", lhs.timezone().empty(), " and ",

Review comment:
       I would remove printing the values of `XXX.empty()` as they will output boolean values which are not that meaningful with the current wording. I suggest: `Cannot compare timestamps with and without timezones`.

##########
File path: cpp/src/arrow/compute/kernels/scalar_compare.cc
##########
@@ -746,6 +787,111 @@ std::shared_ptr<ScalarFunction> MakeScalarMinMax(std::string name,
   return func;
 }
 
+template <typename Op>
+void AddIntegerBetween(const std::shared_ptr<DataType>& ty, ScalarFunction* func) {
+  auto exec = GeneratePhysicalInteger<ScalarTernaryEqualTypes, BooleanType, Op>(*ty);
+  DCHECK_OK(func->AddKernel({ty, ty, ty}, boolean(), std::move(exec)));
+}
+
+template <typename InType, typename Op>
+void AddGenericBetween(const std::shared_ptr<DataType>& ty, ScalarFunction* func) {
+  DCHECK_OK(func->AddKernel({ty, ty, ty}, boolean(),
+                            ScalarTernaryEqualTypes<BooleanType, InType, Op>::Exec));
+}
+
+template <typename OutType, typename ArgType, typename Op>
+struct BetweenTimestamps : public ScalarTernaryEqualTypes<OutType, ArgType, Op> {
+  using Base = ScalarTernaryEqualTypes<OutType, ArgType, Op>;
+
+  static Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    const auto& var = checked_cast<const TimestampType&>(*batch[0].type());
+    const auto& lhs = checked_cast<const TimestampType&>(*batch[1].type());
+    const auto& rhs = checked_cast<const TimestampType&>(*batch[2].type());
+    if ((var.timezone().empty() != lhs.timezone().empty()) ||
+        (var.timezone().empty() != rhs.timezone().empty()) ||
+        (lhs.timezone().empty() != rhs.timezone().empty())) {
+      return Status::Invalid(
+          "Cannot use timestamps with timezone and timestamps without timezones, got: ",
+          var.timezone().empty(), " ", lhs.timezone().empty(), " and ",
+          rhs.timezone().empty());
+    }
+    return Base::Exec(ctx, batch, out);
+  }
+};
+
+template <typename Op>
+std::shared_ptr<ScalarFunction> MakeBetweenFunction(std::string name,
+                                                    const FunctionDoc* doc) {
+  auto func = std::make_shared<CompareFunction>(name, Arity::Ternary(), doc);
+
+  DCHECK_OK(func->AddKernel(
+      {boolean(), boolean(), boolean()}, boolean(),
+      ScalarTernary<BooleanType, BooleanType, BooleanType, BooleanType, Op>::Exec));
+
+  for (const std::shared_ptr<DataType>& ty : IntTypes()) {
+    AddIntegerBetween<Op>(ty, func.get());
+  }
+
+  AddIntegerBetween<Op>(date32(), func.get());
+  AddIntegerBetween<Op>(date64(), func.get());
+
+  AddGenericBetween<FloatType, Op>(float32(), func.get());
+  AddGenericBetween<DoubleType, Op>(float64(), func.get());
+
+  // Add timestamp kernels
+  for (auto unit : TimeUnit::values()) {
+    InputType in_type(match::TimestampTypeUnit(unit));
+    DCHECK_OK(func->AddKernel({in_type, in_type, in_type}, boolean(),
+                              BetweenTimestamps<BooleanType, TimestampType, Op>::Exec));
+  }
+
+  // Duration
+  for (auto unit : TimeUnit::values()) {
+    InputType in_type(match::DurationTypeUnit(unit));
+    auto exec =
+        GeneratePhysicalInteger<ScalarTernaryEqualTypes, BooleanType, Op>(int64());
+    DCHECK_OK(func->AddKernel({in_type, in_type, in_type}, boolean(), std::move(exec)));
+  }
+
+  // Time32 and Time64
+  for (auto unit : {TimeUnit::SECOND, TimeUnit::MILLI}) {
+    InputType in_type(match::Time32TypeUnit(unit));
+    auto exec =
+        GeneratePhysicalInteger<ScalarTernaryEqualTypes, BooleanType, Op>(int32());
+    DCHECK_OK(func->AddKernel({in_type, in_type, in_type}, boolean(), std::move(exec)));
+  }
+  for (auto unit : {TimeUnit::MICRO, TimeUnit::NANO}) {
+    InputType in_type(match::Time64TypeUnit(unit));
+    auto exec =
+        GeneratePhysicalInteger<ScalarTernaryEqualTypes, BooleanType, Op>(int64());
+    DCHECK_OK(func->AddKernel({in_type, in_type, in_type}, boolean(), std::move(exec)));
+  }
+
+  for (const std::shared_ptr<DataType>& ty : BaseBinaryTypes()) {
+    auto exec = GenerateVarBinaryBase<ScalarTernaryEqualTypes, BooleanType, Op>(*ty);

Review comment:
       For consistency, do not dereference `ty`.

##########
File path: cpp/src/arrow/compute/kernels/scalar_compare.cc
##########
@@ -746,6 +787,111 @@ std::shared_ptr<ScalarFunction> MakeScalarMinMax(std::string name,
   return func;
 }
 
+template <typename Op>
+void AddIntegerBetween(const std::shared_ptr<DataType>& ty, ScalarFunction* func) {
+  auto exec = GeneratePhysicalInteger<ScalarTernaryEqualTypes, BooleanType, Op>(*ty);
+  DCHECK_OK(func->AddKernel({ty, ty, ty}, boolean(), std::move(exec)));
+}
+
+template <typename InType, typename Op>
+void AddGenericBetween(const std::shared_ptr<DataType>& ty, ScalarFunction* func) {
+  DCHECK_OK(func->AddKernel({ty, ty, ty}, boolean(),
+                            ScalarTernaryEqualTypes<BooleanType, InType, Op>::Exec));
+}

Review comment:
       Maybe inline `AddIntegerBetween` and `AddGenericBetween`, and remove these functions.
   There are already several explicit `GeneratePhysicalInteger` in `MakeBetweenFunction()`.
   You can use local lambdas too.

##########
File path: cpp/src/arrow/compute/kernels/scalar_compare.cc
##########
@@ -746,6 +787,111 @@ std::shared_ptr<ScalarFunction> MakeScalarMinMax(std::string name,
   return func;
 }
 
+template <typename Op>
+void AddIntegerBetween(const std::shared_ptr<DataType>& ty, ScalarFunction* func) {
+  auto exec = GeneratePhysicalInteger<ScalarTernaryEqualTypes, BooleanType, Op>(*ty);
+  DCHECK_OK(func->AddKernel({ty, ty, ty}, boolean(), std::move(exec)));
+}
+
+template <typename InType, typename Op>
+void AddGenericBetween(const std::shared_ptr<DataType>& ty, ScalarFunction* func) {
+  DCHECK_OK(func->AddKernel({ty, ty, ty}, boolean(),
+                            ScalarTernaryEqualTypes<BooleanType, InType, Op>::Exec));
+}
+
+template <typename OutType, typename ArgType, typename Op>
+struct BetweenTimestamps : public ScalarTernaryEqualTypes<OutType, ArgType, Op> {
+  using Base = ScalarTernaryEqualTypes<OutType, ArgType, Op>;
+
+  static Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    const auto& var = checked_cast<const TimestampType&>(*batch[0].type());
+    const auto& lhs = checked_cast<const TimestampType&>(*batch[1].type());
+    const auto& rhs = checked_cast<const TimestampType&>(*batch[2].type());
+    if ((var.timezone().empty() != lhs.timezone().empty()) ||
+        (var.timezone().empty() != rhs.timezone().empty()) ||
+        (lhs.timezone().empty() != rhs.timezone().empty())) {
+      return Status::Invalid(
+          "Cannot use timestamps with timezone and timestamps without timezones, got: ",
+          var.timezone().empty(), " ", lhs.timezone().empty(), " and ",
+          rhs.timezone().empty());
+    }
+    return Base::Exec(ctx, batch, out);
+  }
+};
+
+template <typename Op>
+std::shared_ptr<ScalarFunction> MakeBetweenFunction(std::string name,
+                                                    const FunctionDoc* doc) {
+  auto func = std::make_shared<CompareFunction>(name, Arity::Ternary(), doc);
+
+  DCHECK_OK(func->AddKernel(
+      {boolean(), boolean(), boolean()}, boolean(),
+      ScalarTernary<BooleanType, BooleanType, BooleanType, BooleanType, Op>::Exec));
+
+  for (const std::shared_ptr<DataType>& ty : IntTypes()) {
+    AddIntegerBetween<Op>(ty, func.get());
+  }
+
+  AddIntegerBetween<Op>(date32(), func.get());
+  AddIntegerBetween<Op>(date64(), func.get());
+
+  AddGenericBetween<FloatType, Op>(float32(), func.get());
+  AddGenericBetween<DoubleType, Op>(float64(), func.get());
+
+  // Add timestamp kernels
+  for (auto unit : TimeUnit::values()) {
+    InputType in_type(match::TimestampTypeUnit(unit));
+    DCHECK_OK(func->AddKernel({in_type, in_type, in_type}, boolean(),
+                              BetweenTimestamps<BooleanType, TimestampType, Op>::Exec));
+  }
+
+  // Duration
+  for (auto unit : TimeUnit::values()) {
+    InputType in_type(match::DurationTypeUnit(unit));
+    auto exec =
+        GeneratePhysicalInteger<ScalarTernaryEqualTypes, BooleanType, Op>(int64());
+    DCHECK_OK(func->AddKernel({in_type, in_type, in_type}, boolean(), std::move(exec)));
+  }
+
+  // Time32 and Time64
+  for (auto unit : {TimeUnit::SECOND, TimeUnit::MILLI}) {
+    InputType in_type(match::Time32TypeUnit(unit));
+    auto exec =
+        GeneratePhysicalInteger<ScalarTernaryEqualTypes, BooleanType, Op>(int32());
+    DCHECK_OK(func->AddKernel({in_type, in_type, in_type}, boolean(), std::move(exec)));
+  }
+  for (auto unit : {TimeUnit::MICRO, TimeUnit::NANO}) {
+    InputType in_type(match::Time64TypeUnit(unit));
+    auto exec =
+        GeneratePhysicalInteger<ScalarTernaryEqualTypes, BooleanType, Op>(int64());
+    DCHECK_OK(func->AddKernel({in_type, in_type, in_type}, boolean(), std::move(exec)));
+  }
+
+  for (const std::shared_ptr<DataType>& ty : BaseBinaryTypes()) {
+    auto exec = GenerateVarBinaryBase<ScalarTernaryEqualTypes, BooleanType, Op>(*ty);
+    DCHECK_OK(func->AddKernel(
+        {InputType::Array(ty), InputType::Scalar(ty), InputType::Scalar(ty)}, boolean(),
+        exec));
+    DCHECK_OK(func->AddKernel(
+        {InputType::Array(ty), InputType::Array(ty), InputType::Array(ty)}, boolean(),
+        std::move(exec)));
+  }

Review comment:
       I am confused here. Why only `BaseBinaryTypes` have `Array-Scalar-Scalar` and `Array-Array-Array`, and others types do not? Also, what about mixed shapes `XXX-Scalar-Array` and `XXX-Array-Scalar?




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

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

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



[GitHub] [arrow] edponce commented on pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
edponce commented on pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#issuecomment-1003181836


   @bkmgit The issue with options is not the number of distinct parameters. `FunctionOptions` can have arbitrary number of parameters. The issue is more general than that. If compute functions were to be consolidated, then `FunctionOptions` would need to have a hierarchical structure as well. This results in more complexity and less flexibility on the core C++ implementation. In fact, it would make adding functions more difficult. Also, I would argue that for other language bindings their is no inherent benefit of one versus the other. You will still need a binding per function-options combination.
   
   If you want to discuss the structure of compute functions further, I suggest you to propose an approach in ML. This would allow more visibility in the discussion and gather more opinions.


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

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

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



[GitHub] [arrow] lidavidm commented on a change in pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
lidavidm commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r775548721



##########
File path: cpp/src/arrow/compute/api_scalar.h
##########
@@ -316,6 +316,19 @@ struct ARROW_EXPORT CompareOptions {
   enum CompareOperator op;
 };
 
+enum BetweenOperator : int8_t {
+  BETWEEN_LESS_EQUAL_LESS_EQUAL,
+  BETWEEN_LESS_EQUAL_LESS_THAN,
+  BETWEEN_LESS_THAN_LESS_EQUAL,
+  BETWEEN_LESS_THAN_LESS_THAN,
+};
+
+struct ARROW_EXPORT BetweenOptions {
+  explicit BetweenOptions(BetweenOperator op) : op(op) {}
+  BetweenOptions() : BetweenOptions(BetweenOperator::BETWEEN_LESS_EQUAL_LESS_EQUAL) {}
+  enum BetweenOperator op;

Review comment:
       Either use enum class, or do this:
   
   ```
   class BetweenOptions {
       enum BetweenOperator {...};
   };
   ```
   
   That way the variants can be accessed as `BetweenOptions::LESS_LESS`. However, we shouldn't use a plain enum at the top level since the enum values won't be namespaced.




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

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

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



[GitHub] [arrow] lidavidm commented on a change in pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
lidavidm commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r775939523



##########
File path: cpp/src/arrow/compute/kernels/scalar_compare.cc
##########
@@ -68,6 +72,50 @@ struct GreaterEqual {
   }
 };
 
+struct BetweenLessEqualLessEqual {
+  template <typename T, typename Arg0, typename Arg1, typename Arg2>
+  static constexpr T Call(KernelContext*, const Arg0& middle, const Arg1& left,
+                          const Arg2& right, Status*) {
+    static_assert(std::is_same<T, bool>::value && std::is_same<Arg0, Arg1>::value &&
+                      std::is_same<Arg1, Arg2>::value,
+                  "");
+    return (left <= middle) && (middle <= right);
+  }
+};
+
+struct BetweenLessThanLessEqual {
+  template <typename T, typename Arg0, typename Arg1, typename Arg2>
+  static constexpr T Call(KernelContext*, const Arg0& middle, const Arg1& left,
+                          const Arg2& right, Status*) {
+    static_assert(std::is_same<T, bool>::value && std::is_same<Arg0, Arg1>::value &&
+                      std::is_same<Arg1, Arg2>::value,
+                  "");
+    return (left < middle) && (middle <= right);
+  }
+};
+
+struct BetweenLessEqualLessThan {
+  template <typename T, typename Arg0, typename Arg1, typename Arg2>
+  static constexpr T Call(KernelContext*, const Arg0& middle, const Arg1& left,
+                          const Arg2& right, Status*) {
+    static_assert(std::is_same<T, bool>::value && std::is_same<Arg0, Arg1>::value &&
+                      std::is_same<Arg1, Arg2>::value,
+                  "");
+    return (left <= middle) && (middle < right);
+  }
+};
+
+struct BetweenLessThanLessThan {
+  template <typename T, typename Arg0, typename Arg1, typename Arg2>
+  static constexpr T Call(KernelContext*, const Arg0& middle, const Arg1& left,
+                          const Arg2& right, Status*) {
+    static_assert(std::is_same<T, bool>::value && std::is_same<Arg0, Arg1>::value &&
+                      std::is_same<Arg1, Arg2>::value,
+                  "");
+    return (left < middle) && (middle < right);
+  }
+};

Review comment:
       Ah, ok. That can be filed in a separate issue. We currently only sort by Unicode code point and I'm not sure we necessarily want to implement locale-specific sorts ourselves (I think this has come up before), it may be worth discussing on the mailing list.




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

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

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



[GitHub] [arrow] bkmgit commented on a change in pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
bkmgit commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r775815638



##########
File path: cpp/src/arrow/compute/api_scalar.h
##########
@@ -316,6 +316,19 @@ struct ARROW_EXPORT CompareOptions {
   enum CompareOperator op;
 };
 
+enum BetweenOperator : int8_t {
+  BETWEEN_LESS_EQUAL_LESS_EQUAL,
+  BETWEEN_LESS_EQUAL_LESS_THAN,
+  BETWEEN_LESS_THAN_LESS_EQUAL,
+  BETWEEN_LESS_THAN_LESS_THAN,
+};
+
+struct ARROW_EXPORT BetweenOptions {
+  explicit BetweenOptions(BetweenOperator op) : op(op) {}
+  BetweenOptions() : BetweenOptions(BetweenOperator::BETWEEN_LESS_EQUAL_LESS_EQUAL) {}
+  enum BetweenOperator op;

Review comment:
       Function options implemented.




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

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

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



[GitHub] [arrow] lidavidm commented on a change in pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
lidavidm commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r775899788



##########
File path: cpp/src/arrow/compute/kernels/scalar_compare.cc
##########
@@ -68,6 +72,50 @@ struct GreaterEqual {
   }
 };
 
+struct BetweenLessEqualLessEqual {
+  template <typename T, typename Arg0, typename Arg1, typename Arg2>
+  static constexpr T Call(KernelContext*, const Arg0& middle, const Arg1& left,
+                          const Arg2& right, Status*) {
+    static_assert(std::is_same<T, bool>::value && std::is_same<Arg0, Arg1>::value &&
+                      std::is_same<Arg1, Arg2>::value,
+                  "");
+    return (left <= middle) && (middle <= right);
+  }
+};
+
+struct BetweenLessThanLessEqual {
+  template <typename T, typename Arg0, typename Arg1, typename Arg2>
+  static constexpr T Call(KernelContext*, const Arg0& middle, const Arg1& left,
+                          const Arg2& right, Status*) {
+    static_assert(std::is_same<T, bool>::value && std::is_same<Arg0, Arg1>::value &&
+                      std::is_same<Arg1, Arg2>::value,
+                  "");
+    return (left < middle) && (middle <= right);
+  }
+};
+
+struct BetweenLessEqualLessThan {
+  template <typename T, typename Arg0, typename Arg1, typename Arg2>
+  static constexpr T Call(KernelContext*, const Arg0& middle, const Arg1& left,
+                          const Arg2& right, Status*) {
+    static_assert(std::is_same<T, bool>::value && std::is_same<Arg0, Arg1>::value &&
+                      std::is_same<Arg1, Arg2>::value,
+                  "");
+    return (left <= middle) && (middle < right);
+  }
+};
+
+struct BetweenLessThanLessThan {
+  template <typename T, typename Arg0, typename Arg1, typename Arg2>
+  static constexpr T Call(KernelContext*, const Arg0& middle, const Arg1& left,
+                          const Arg2& right, Status*) {
+    static_assert(std::is_same<T, bool>::value && std::is_same<Arg0, Arg1>::value &&
+                      std::is_same<Arg1, Arg2>::value,
+                  "");
+    return (left < middle) && (middle < right);
+  }
+};

Review comment:
       Also because the current structure gets exposed in Python oddly (as four separate functions).
   
   I think either we should have one function "between" with BetweenOptions, or we should have four separate functions with no options, but the current state where we have both options and four separate functions doesn't make sense.




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

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

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



[GitHub] [arrow] bkmgit commented on pull request #11882: ARROW-9843: [C++][Python] Implement Between ternary kernel and Python bindings

Posted by GitBox <gi...@apache.org>.
bkmgit commented on pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#issuecomment-1072055711


   Waiting for review from @pitrou 


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

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

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



[GitHub] [arrow] edponce commented on pull request #11882: ARROW-9843: [C++][Python] Implement Between ternary kernel and Python bindings

Posted by GitBox <gi...@apache.org>.
edponce commented on pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#issuecomment-1072048776


   Is there anything left to do in this PR?


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

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

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



[GitHub] [arrow] bkmgit commented on pull request #11882: ARROW-9843: [C++][Python] Implement Between ternary kernel and Python bindings

Posted by GitBox <gi...@apache.org>.
bkmgit commented on pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#issuecomment-1018186232


   @githubactions autotune


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

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

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



[GitHub] [arrow] bkmgit commented on a change in pull request #11882: ARROW-9843: [C++][Python] Implement Between ternary kernel and Python bindings

Posted by GitBox <gi...@apache.org>.
bkmgit commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r790283940



##########
File path: cpp/src/arrow/compute/kernels/scalar_compare.cc
##########
@@ -746,6 +808,279 @@ std::shared_ptr<ScalarFunction> MakeScalarMinMax(std::string name,
   return func;
 }
 
+template <template <BetweenOptions::Inclusive> class Op>
+std::shared_ptr<ScalarFunction> MakeBetweenFunction(std::string name,
+                                                    const FunctionDoc* doc) {
+  using BetweenState = OptionsWrapper<BetweenOptions>;
+  static const auto kDefaultOptions = BetweenOptions::Defaults();
+  auto func =
+      std::make_shared<CompareFunction>(name, Arity::Ternary(), doc, &kDefaultOptions);
+
+  // Add kernels for physical numeric types, temporal types done separately
+  for (const auto& types : {DurationTypes(), IntervalTypes(), NumericTypes()}) {
+    for (const auto& ty : types) {
+      auto type_id = ty->id();
+      auto exec = [type_id](KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+        // Resolve generator based on options
+        const auto& state = static_cast<const BetweenState&>(*ctx->state());
+        switch (state.options.inclusive) {
+          case BetweenOptions::Inclusive::BOTH:
+            return GeneratePhysicalNumeric<ScalarTernaryEqualTypes, BooleanType,
+                                           Op<BetweenOptions::Inclusive::BOTH>>(type_id)(
+                ctx, batch, out);
+          case BetweenOptions::Inclusive::LEFT:
+            return GeneratePhysicalNumeric<ScalarTernaryEqualTypes, BooleanType,
+                                           Op<BetweenOptions::Inclusive::LEFT>>(type_id)(
+                ctx, batch, out);
+          case BetweenOptions::Inclusive::RIGHT:
+            return GeneratePhysicalNumeric<ScalarTernaryEqualTypes, BooleanType,
+                                           Op<BetweenOptions::Inclusive::RIGHT>>(type_id)(
+                ctx, batch, out);
+          case BetweenOptions::Inclusive::NEITHER:
+            return GeneratePhysicalNumeric<ScalarTernaryEqualTypes, BooleanType,
+                                           Op<BetweenOptions::Inclusive::NEITHER>>(
+                type_id)(ctx, batch, out);
+          default:
+            return Status::NotImplemented("between inclusiveness not implemented: ",
+                                          state.options.ToString());
+        }
+      };
+      DCHECK_OK(func->AddKernel({ty, ty, ty}, boolean(), exec, BetweenState::Init));
+    }
+  }
+
+  // Add timestamp kernels
+  for (const auto unit : TimeUnit::values()) {
+    InputType in_type(match::TimestampTypeUnit(unit));
+    auto type_id = Type::TIMESTAMP;
+    auto exec = [type_id](KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+      // Validate timezones in all entries or in none
+      RETURN_NOT_OK(CheckCompareTimestamps(batch));
+      // Resolve generator based on options
+      const auto& state = static_cast<const BetweenState&>(*ctx->state());
+      switch (state.options.inclusive) {
+        case BetweenOptions::Inclusive::BOTH:
+          return GeneratePhysicalNumeric<ScalarTernaryEqualTypes, BooleanType,
+                                         Op<BetweenOptions::Inclusive::BOTH>>(type_id)(
+              ctx, batch, out);
+        case BetweenOptions::Inclusive::LEFT:
+          return GeneratePhysicalNumeric<ScalarTernaryEqualTypes, BooleanType,
+                                         Op<BetweenOptions::Inclusive::LEFT>>(type_id)(
+              ctx, batch, out);
+        case BetweenOptions::Inclusive::RIGHT:
+          return GeneratePhysicalNumeric<ScalarTernaryEqualTypes, BooleanType,
+                                         Op<BetweenOptions::Inclusive::RIGHT>>(type_id)(
+              ctx, batch, out);
+        case BetweenOptions::Inclusive::NEITHER:
+          return GeneratePhysicalNumeric<ScalarTernaryEqualTypes, BooleanType,
+                                         Op<BetweenOptions::Inclusive::NEITHER>>(type_id)(
+              ctx, batch, out);
+        default:
+          return Status::NotImplemented("between inclusiveness not implemented: ",
+                                        state.options.ToString());
+      }
+    };
+    DCHECK_OK(func->AddKernel({in_type, in_type, in_type}, boolean(), exec,
+                              BetweenState::Init));
+  }
+
+  // Add time kernels
+  for (const auto unit : {TimeUnit::SECOND, TimeUnit::MILLI}) {
+    auto type_id = Type::TIME32;
+    auto exec = [type_id](KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+      // Resolve generator based on options
+      const auto& state = static_cast<const BetweenState&>(*ctx->state());
+      switch (state.options.inclusive) {
+        case BetweenOptions::Inclusive::BOTH:
+          return GeneratePhysicalNumeric<ScalarTernaryEqualTypes, BooleanType,
+                                         Op<BetweenOptions::Inclusive::BOTH>>(type_id)(
+              ctx, batch, out);
+        case BetweenOptions::Inclusive::LEFT:
+          return GeneratePhysicalNumeric<ScalarTernaryEqualTypes, BooleanType,
+                                         Op<BetweenOptions::Inclusive::LEFT>>(type_id)(
+              ctx, batch, out);
+        case BetweenOptions::Inclusive::RIGHT:
+          return GeneratePhysicalNumeric<ScalarTernaryEqualTypes, BooleanType,
+                                         Op<BetweenOptions::Inclusive::RIGHT>>(type_id)(
+              ctx, batch, out);
+        case BetweenOptions::Inclusive::NEITHER:
+          return GeneratePhysicalNumeric<ScalarTernaryEqualTypes, BooleanType,
+                                         Op<BetweenOptions::Inclusive::NEITHER>>(type_id)(
+              ctx, batch, out);
+        default:
+          return Status::NotImplemented("between inclusiveness not implemented: ",
+                                        state.options.ToString());
+      }
+    };
+    InputType in_type(match::Time32TypeUnit(unit));
+    DCHECK_OK(func->AddKernel({in_type, in_type, in_type}, boolean(), exec,
+                              BetweenState::Init));
+  }
+
+  for (const auto& unit : {TimeUnit::MICRO, TimeUnit::NANO}) {
+    auto type_id = Type::TIME64;
+    auto exec = [type_id](KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+      // Resolve generator based on options
+      const auto& state = static_cast<const BetweenState&>(*ctx->state());
+      switch (state.options.inclusive) {
+        case BetweenOptions::Inclusive::BOTH:
+          return GeneratePhysicalNumeric<ScalarTernaryEqualTypes, BooleanType,
+                                         Op<BetweenOptions::Inclusive::BOTH>>(type_id)(
+              ctx, batch, out);
+        case BetweenOptions::Inclusive::LEFT:
+          return GeneratePhysicalNumeric<ScalarTernaryEqualTypes, BooleanType,
+                                         Op<BetweenOptions::Inclusive::LEFT>>(type_id)(
+              ctx, batch, out);
+        case BetweenOptions::Inclusive::RIGHT:
+          return GeneratePhysicalNumeric<ScalarTernaryEqualTypes, BooleanType,
+                                         Op<BetweenOptions::Inclusive::RIGHT>>(type_id)(
+              ctx, batch, out);
+        case BetweenOptions::Inclusive::NEITHER:
+          return GeneratePhysicalNumeric<ScalarTernaryEqualTypes, BooleanType,
+                                         Op<BetweenOptions::Inclusive::NEITHER>>(type_id)(
+              ctx, batch, out);
+        default:
+          return Status::NotImplemented("between inclusiveness not implemented: ",
+                                        state.options.ToString());
+      }
+    };
+    InputType in_type(match::Time64TypeUnit(unit));
+    DCHECK_OK(func->AddKernel({in_type, in_type, in_type}, boolean(), exec,
+                              BetweenState::Init));
+  }
+
+  // Add kernels for base binary types
+  for (const auto& ty : BaseBinaryTypes()) {
+    auto type_id = ty->id();
+    auto exec = [type_id](KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+      // Resolve generator based on options
+      const auto& state = static_cast<const BetweenState&>(*ctx->state());
+      switch (state.options.inclusive) {
+        case BetweenOptions::Inclusive::BOTH:
+          return GenerateVarBinaryBase<ScalarTernaryEqualTypes, BooleanType,
+                                       Op<BetweenOptions::Inclusive::BOTH>>(type_id)(
+              ctx, batch, out);
+        case BetweenOptions::Inclusive::LEFT:
+          return GenerateVarBinaryBase<ScalarTernaryEqualTypes, BooleanType,
+                                       Op<BetweenOptions::Inclusive::LEFT>>(type_id)(
+              ctx, batch, out);
+        case BetweenOptions::Inclusive::RIGHT:
+          return GenerateVarBinaryBase<ScalarTernaryEqualTypes, BooleanType,
+                                       Op<BetweenOptions::Inclusive::RIGHT>>(type_id)(
+              ctx, batch, out);
+        case BetweenOptions::Inclusive::NEITHER:
+          return GenerateVarBinaryBase<ScalarTernaryEqualTypes, BooleanType,
+                                       Op<BetweenOptions::Inclusive::NEITHER>>(type_id)(
+              ctx, batch, out);
+        default:
+          return Status::NotImplemented("between inclusiveness not implemented: ",
+                                        state.options.ToString());
+      }
+    };
+    DCHECK_OK(func->AddKernel({ty, ty, ty}, boolean(), exec, BetweenState::Init));
+  }
+
+  // Add kernels for decimal types
+  for (const auto type_id : {Type::DECIMAL128, Type::DECIMAL256}) {
+    auto exec = [type_id](KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+      // Resolve generator based on options
+      const auto& state = static_cast<const BetweenState&>(*ctx->state());
+      switch (state.options.inclusive) {
+        case BetweenOptions::Inclusive::BOTH:
+          return GenerateDecimal<ScalarTernaryEqualTypes, BooleanType,
+                                 Op<BetweenOptions::Inclusive::BOTH>>(type_id)(ctx, batch,
+                                                                               out);
+        case BetweenOptions::Inclusive::LEFT:
+          return GenerateDecimal<ScalarTernaryEqualTypes, BooleanType,
+                                 Op<BetweenOptions::Inclusive::LEFT>>(type_id)(ctx, batch,
+                                                                               out);
+        case BetweenOptions::Inclusive::RIGHT:
+          return GenerateDecimal<ScalarTernaryEqualTypes, BooleanType,
+                                 Op<BetweenOptions::Inclusive::RIGHT>>(type_id)(
+              ctx, batch, out);
+        case BetweenOptions::Inclusive::NEITHER:
+          return GenerateDecimal<ScalarTernaryEqualTypes, BooleanType,
+                                 Op<BetweenOptions::Inclusive::NEITHER>>(type_id)(
+              ctx, batch, out);
+        default:
+          return Status::NotImplemented("between inclusiveness not implemented: ",
+                                        state.options.ToString());
+      }
+    };
+    InputType ty(type_id);
+    DCHECK_OK(func->AddKernel({ty, ty, ty}, boolean(), exec, BetweenState::Init));
+  }
+
+  // Add kernels for date types
+  for (const auto type_id : {Type::DATE32, Type::DATE64}) {
+    auto exec = [type_id](KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+      // Resolve generator based on options
+      const auto& state = static_cast<const BetweenState&>(*ctx->state());
+      switch (state.options.inclusive) {
+        case BetweenOptions::Inclusive::BOTH:
+          return GeneratePhysicalNumeric<ScalarTernaryEqualTypes, BooleanType,
+                                         Op<BetweenOptions::Inclusive::BOTH>>(type_id)(
+              ctx, batch, out);
+        case BetweenOptions::Inclusive::LEFT:
+          return GeneratePhysicalNumeric<ScalarTernaryEqualTypes, BooleanType,
+                                         Op<BetweenOptions::Inclusive::LEFT>>(type_id)(
+              ctx, batch, out);
+        case BetweenOptions::Inclusive::RIGHT:
+          return GeneratePhysicalNumeric<ScalarTernaryEqualTypes, BooleanType,
+                                         Op<BetweenOptions::Inclusive::RIGHT>>(type_id)(
+              ctx, batch, out);
+        case BetweenOptions::Inclusive::NEITHER:
+          return GeneratePhysicalNumeric<ScalarTernaryEqualTypes, BooleanType,
+                                         Op<BetweenOptions::Inclusive::NEITHER>>(type_id)(
+              ctx, batch, out);
+        default:
+          return Status::NotImplemented("between inclusiveness not implemented: ",
+                                        state.options.ToString());
+      }
+    };

Review comment:
       Created a routine to minimize duplication within scalar_compare.cc. It uses only one template at the moment for between options and uses an if statement to choose the appropriate routine. Many more changes would be needed to use a template since the routines take different arguments, perhaps such changes are best done in another issue? One of the few routines that uses two templates is [Memoize](https://github.com/apache/arrow/blob/master/cpp/src/arrow/util/cache_internal.h#L175).




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

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

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



[GitHub] [arrow] edponce commented on pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
edponce commented on pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#issuecomment-1013764157


   LGTM! Thanks @bkmgit for working on this. My last comments are:
   * Update the description message of this PR
   * Nit: Remove GH labels not related


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

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

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



[GitHub] [arrow] bkmgit commented on a change in pull request #11882: ARROW-9843: [C++][Python] Implement Between ternary kernel and Python bindings

Posted by GitBox <gi...@apache.org>.
bkmgit commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r786000919



##########
File path: python/pyarrow/_compute.pyx
##########
@@ -785,6 +785,30 @@ class ElementWiseAggregateOptions(_ElementWiseAggregateOptions):
         self._set_options(skip_nulls)
 
 
+cdef CInclusive unwrap_inclusive(inclusive) except *:
+    if inclusive == "both":
+        return CInclusive_BOTH
+    elif inclusive == "left":
+        return CInclusive_LEFT
+    elif inclusive == "right":
+        return CInclusive_RIGHT
+    elif inclusive == "neither":
+        return CInclusive_NEITHER
+    _raise_invalid_function_option(inclusive, "inclusive")
+
+
+cdef class _BetweenOptions(FunctionOptions):
+    def _set_options(self, inclusive):
+        self.wrapped.reset(
+            new CBetweenOptions(unwrap_inclusive(inclusive))
+        )
+
+
+class BetweenOptions(_BetweenOptions):

Review comment:
       Done




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

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

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



[GitHub] [arrow] bkmgit commented on a change in pull request #11882: ARROW-9843: [C++][Python] Implement Between ternary kernel and Python bindings

Posted by GitBox <gi...@apache.org>.
bkmgit commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r786222578



##########
File path: cpp/src/arrow/compute/kernels/scalar_compare.cc
##########
@@ -746,6 +808,279 @@ std::shared_ptr<ScalarFunction> MakeScalarMinMax(std::string name,
   return func;
 }
 
+template <template <BetweenOptions::Inclusive> class Op>
+std::shared_ptr<ScalarFunction> MakeBetweenFunction(std::string name,
+                                                    const FunctionDoc* doc) {
+  using BetweenState = OptionsWrapper<BetweenOptions>;
+  static const auto kDefaultOptions = BetweenOptions::Defaults();
+  auto func =
+      std::make_shared<CompareFunction>(name, Arity::Ternary(), doc, &kDefaultOptions);
+
+  // Add kernels for physical numeric types, temporal types done separately
+  for (const auto& types : {DurationTypes(), IntervalTypes(), NumericTypes()}) {
+    for (const auto& ty : types) {
+      auto type_id = ty->id();
+      auto exec = [type_id](KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+        // Resolve generator based on options
+        const auto& state = static_cast<const BetweenState&>(*ctx->state());
+        switch (state.options.inclusive) {
+          case BetweenOptions::Inclusive::BOTH:
+            return GeneratePhysicalNumeric<ScalarTernaryEqualTypes, BooleanType,
+                                           Op<BetweenOptions::Inclusive::BOTH>>(type_id)(
+                ctx, batch, out);
+          case BetweenOptions::Inclusive::LEFT:
+            return GeneratePhysicalNumeric<ScalarTernaryEqualTypes, BooleanType,
+                                           Op<BetweenOptions::Inclusive::LEFT>>(type_id)(
+                ctx, batch, out);
+          case BetweenOptions::Inclusive::RIGHT:
+            return GeneratePhysicalNumeric<ScalarTernaryEqualTypes, BooleanType,
+                                           Op<BetweenOptions::Inclusive::RIGHT>>(type_id)(
+                ctx, batch, out);
+          case BetweenOptions::Inclusive::NEITHER:
+            return GeneratePhysicalNumeric<ScalarTernaryEqualTypes, BooleanType,
+                                           Op<BetweenOptions::Inclusive::NEITHER>>(
+                type_id)(ctx, batch, out);
+          default:
+            return Status::NotImplemented("between inclusiveness not implemented: ",
+                                          state.options.ToString());
+        }
+      };
+      DCHECK_OK(func->AddKernel({ty, ty, ty}, boolean(), exec, BetweenState::Init));
+    }
+  }
+
+  // Add timestamp kernels
+  for (const auto unit : TimeUnit::values()) {
+    InputType in_type(match::TimestampTypeUnit(unit));
+    auto type_id = Type::TIMESTAMP;
+    auto exec = [type_id](KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+      // Validate timezones in all entries or in none
+      RETURN_NOT_OK(CheckCompareTimestamps(batch));
+      // Resolve generator based on options
+      const auto& state = static_cast<const BetweenState&>(*ctx->state());
+      switch (state.options.inclusive) {
+        case BetweenOptions::Inclusive::BOTH:
+          return GeneratePhysicalNumeric<ScalarTernaryEqualTypes, BooleanType,
+                                         Op<BetweenOptions::Inclusive::BOTH>>(type_id)(
+              ctx, batch, out);
+        case BetweenOptions::Inclusive::LEFT:
+          return GeneratePhysicalNumeric<ScalarTernaryEqualTypes, BooleanType,
+                                         Op<BetweenOptions::Inclusive::LEFT>>(type_id)(
+              ctx, batch, out);
+        case BetweenOptions::Inclusive::RIGHT:
+          return GeneratePhysicalNumeric<ScalarTernaryEqualTypes, BooleanType,
+                                         Op<BetweenOptions::Inclusive::RIGHT>>(type_id)(
+              ctx, batch, out);
+        case BetweenOptions::Inclusive::NEITHER:
+          return GeneratePhysicalNumeric<ScalarTernaryEqualTypes, BooleanType,
+                                         Op<BetweenOptions::Inclusive::NEITHER>>(type_id)(
+              ctx, batch, out);
+        default:
+          return Status::NotImplemented("between inclusiveness not implemented: ",
+                                        state.options.ToString());
+      }
+    };
+    DCHECK_OK(func->AddKernel({in_type, in_type, in_type}, boolean(), exec,
+                              BetweenState::Init));
+  }
+
+  // Add time kernels
+  for (const auto unit : {TimeUnit::SECOND, TimeUnit::MILLI}) {
+    auto type_id = Type::TIME32;
+    auto exec = [type_id](KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+      // Resolve generator based on options
+      const auto& state = static_cast<const BetweenState&>(*ctx->state());
+      switch (state.options.inclusive) {
+        case BetweenOptions::Inclusive::BOTH:
+          return GeneratePhysicalNumeric<ScalarTernaryEqualTypes, BooleanType,
+                                         Op<BetweenOptions::Inclusive::BOTH>>(type_id)(
+              ctx, batch, out);
+        case BetweenOptions::Inclusive::LEFT:
+          return GeneratePhysicalNumeric<ScalarTernaryEqualTypes, BooleanType,
+                                         Op<BetweenOptions::Inclusive::LEFT>>(type_id)(
+              ctx, batch, out);
+        case BetweenOptions::Inclusive::RIGHT:
+          return GeneratePhysicalNumeric<ScalarTernaryEqualTypes, BooleanType,
+                                         Op<BetweenOptions::Inclusive::RIGHT>>(type_id)(
+              ctx, batch, out);
+        case BetweenOptions::Inclusive::NEITHER:
+          return GeneratePhysicalNumeric<ScalarTernaryEqualTypes, BooleanType,
+                                         Op<BetweenOptions::Inclusive::NEITHER>>(type_id)(
+              ctx, batch, out);
+        default:
+          return Status::NotImplemented("between inclusiveness not implemented: ",
+                                        state.options.ToString());
+      }
+    };
+    InputType in_type(match::Time32TypeUnit(unit));
+    DCHECK_OK(func->AddKernel({in_type, in_type, in_type}, boolean(), exec,
+                              BetweenState::Init));
+  }
+
+  for (const auto& unit : {TimeUnit::MICRO, TimeUnit::NANO}) {
+    auto type_id = Type::TIME64;
+    auto exec = [type_id](KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+      // Resolve generator based on options
+      const auto& state = static_cast<const BetweenState&>(*ctx->state());
+      switch (state.options.inclusive) {
+        case BetweenOptions::Inclusive::BOTH:
+          return GeneratePhysicalNumeric<ScalarTernaryEqualTypes, BooleanType,
+                                         Op<BetweenOptions::Inclusive::BOTH>>(type_id)(
+              ctx, batch, out);
+        case BetweenOptions::Inclusive::LEFT:
+          return GeneratePhysicalNumeric<ScalarTernaryEqualTypes, BooleanType,
+                                         Op<BetweenOptions::Inclusive::LEFT>>(type_id)(
+              ctx, batch, out);
+        case BetweenOptions::Inclusive::RIGHT:
+          return GeneratePhysicalNumeric<ScalarTernaryEqualTypes, BooleanType,
+                                         Op<BetweenOptions::Inclusive::RIGHT>>(type_id)(
+              ctx, batch, out);
+        case BetweenOptions::Inclusive::NEITHER:
+          return GeneratePhysicalNumeric<ScalarTernaryEqualTypes, BooleanType,
+                                         Op<BetweenOptions::Inclusive::NEITHER>>(type_id)(
+              ctx, batch, out);
+        default:
+          return Status::NotImplemented("between inclusiveness not implemented: ",
+                                        state.options.ToString());
+      }
+    };
+    InputType in_type(match::Time64TypeUnit(unit));
+    DCHECK_OK(func->AddKernel({in_type, in_type, in_type}, boolean(), exec,
+                              BetweenState::Init));
+  }
+
+  // Add kernels for base binary types
+  for (const auto& ty : BaseBinaryTypes()) {
+    auto type_id = ty->id();
+    auto exec = [type_id](KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+      // Resolve generator based on options
+      const auto& state = static_cast<const BetweenState&>(*ctx->state());
+      switch (state.options.inclusive) {
+        case BetweenOptions::Inclusive::BOTH:
+          return GenerateVarBinaryBase<ScalarTernaryEqualTypes, BooleanType,
+                                       Op<BetweenOptions::Inclusive::BOTH>>(type_id)(
+              ctx, batch, out);
+        case BetweenOptions::Inclusive::LEFT:
+          return GenerateVarBinaryBase<ScalarTernaryEqualTypes, BooleanType,
+                                       Op<BetweenOptions::Inclusive::LEFT>>(type_id)(
+              ctx, batch, out);
+        case BetweenOptions::Inclusive::RIGHT:
+          return GenerateVarBinaryBase<ScalarTernaryEqualTypes, BooleanType,
+                                       Op<BetweenOptions::Inclusive::RIGHT>>(type_id)(
+              ctx, batch, out);
+        case BetweenOptions::Inclusive::NEITHER:
+          return GenerateVarBinaryBase<ScalarTernaryEqualTypes, BooleanType,
+                                       Op<BetweenOptions::Inclusive::NEITHER>>(type_id)(
+              ctx, batch, out);
+        default:
+          return Status::NotImplemented("between inclusiveness not implemented: ",
+                                        state.options.ToString());
+      }
+    };
+    DCHECK_OK(func->AddKernel({ty, ty, ty}, boolean(), exec, BetweenState::Init));
+  }
+
+  // Add kernels for decimal types
+  for (const auto type_id : {Type::DECIMAL128, Type::DECIMAL256}) {
+    auto exec = [type_id](KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+      // Resolve generator based on options
+      const auto& state = static_cast<const BetweenState&>(*ctx->state());
+      switch (state.options.inclusive) {
+        case BetweenOptions::Inclusive::BOTH:
+          return GenerateDecimal<ScalarTernaryEqualTypes, BooleanType,
+                                 Op<BetweenOptions::Inclusive::BOTH>>(type_id)(ctx, batch,
+                                                                               out);
+        case BetweenOptions::Inclusive::LEFT:
+          return GenerateDecimal<ScalarTernaryEqualTypes, BooleanType,
+                                 Op<BetweenOptions::Inclusive::LEFT>>(type_id)(ctx, batch,
+                                                                               out);
+        case BetweenOptions::Inclusive::RIGHT:
+          return GenerateDecimal<ScalarTernaryEqualTypes, BooleanType,
+                                 Op<BetweenOptions::Inclusive::RIGHT>>(type_id)(
+              ctx, batch, out);
+        case BetweenOptions::Inclusive::NEITHER:
+          return GenerateDecimal<ScalarTernaryEqualTypes, BooleanType,
+                                 Op<BetweenOptions::Inclusive::NEITHER>>(type_id)(
+              ctx, batch, out);
+        default:
+          return Status::NotImplemented("between inclusiveness not implemented: ",
+                                        state.options.ToString());
+      }
+    };
+    InputType ty(type_id);
+    DCHECK_OK(func->AddKernel({ty, ty, ty}, boolean(), exec, BetweenState::Init));
+  }
+
+  // Add kernels for date types
+  for (const auto type_id : {Type::DATE32, Type::DATE64}) {
+    auto exec = [type_id](KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+      // Resolve generator based on options
+      const auto& state = static_cast<const BetweenState&>(*ctx->state());
+      switch (state.options.inclusive) {
+        case BetweenOptions::Inclusive::BOTH:
+          return GeneratePhysicalNumeric<ScalarTernaryEqualTypes, BooleanType,
+                                         Op<BetweenOptions::Inclusive::BOTH>>(type_id)(
+              ctx, batch, out);
+        case BetweenOptions::Inclusive::LEFT:
+          return GeneratePhysicalNumeric<ScalarTernaryEqualTypes, BooleanType,
+                                         Op<BetweenOptions::Inclusive::LEFT>>(type_id)(
+              ctx, batch, out);
+        case BetweenOptions::Inclusive::RIGHT:
+          return GeneratePhysicalNumeric<ScalarTernaryEqualTypes, BooleanType,
+                                         Op<BetweenOptions::Inclusive::RIGHT>>(type_id)(
+              ctx, batch, out);
+        case BetweenOptions::Inclusive::NEITHER:
+          return GeneratePhysicalNumeric<ScalarTernaryEqualTypes, BooleanType,
+                                         Op<BetweenOptions::Inclusive::NEITHER>>(type_id)(
+              ctx, batch, out);
+        default:
+          return Status::NotImplemented("between inclusiveness not implemented: ",
+                                        state.options.ToString());
+      }
+    };

Review comment:
       Ok.




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

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

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



[GitHub] [arrow] edponce commented on a change in pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
edponce commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r782572785



##########
File path: cpp/src/arrow/compute/api_scalar.h
##########
@@ -316,6 +316,21 @@ struct ARROW_EXPORT CompareOptions {
   enum CompareOperator op;
 };
 
+enum class BetweenMode : int8_t {
+  LESS_EQUAL_LESS_EQUAL,
+  LESS_EQUAL_LESS,
+  LESS_LESS_EQUAL,
+  LESS_LESS,
+};

Review comment:
       Yes, I agree with @lidavidm on how the BetweenOptions should be organized.




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

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

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



[GitHub] [arrow] bkmgit commented on a change in pull request #11882: ARROW-9843: [C++][Python] Implement Between ternary kernel and Python bindings

Posted by GitBox <gi...@apache.org>.
bkmgit commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r786982220



##########
File path: python/pyarrow/tests/test_compute.py
##########
@@ -1360,6 +1361,153 @@ def test_filter_null_type():
     assert len(table.filter(mask).column(0)) == 5
 
 
+@pytest.mark.parametrize("ty", ["inclusive"])
+def test_between_array_array_array(ty):
+    BetweenOptions = partial(pc.BetweenOptions)
+
+    val = pa.array([1, 1, 4, 3, 2, 6])
+    arr1 = pa.array([1, 1, 3, 4, None, 5])
+    arr2 = pa.array([1, 2, 4, None, 4, 7])
+
+    inclusive_and_expected = {
+        "both": [True, True, True, None, None, True],
+        "left": [False, True, False, None, None, True],
+        "right": [False, False, True, None, None, True],
+        "neither": [False, False, False, None, None, True],
+    }
+
+    for inclusive, expected in inclusive_and_expected.items():
+        options = BetweenOptions(inclusive=inclusive)
+        result = pc.between(val, arr1, arr2, options=options)
+        assert result.equals(pa.array(expected))
+
+
+@pytest.mark.parametrize("ty", ["inclusive"])
+def test_between_array_array_scalar(ty):

Review comment:
       Ok.




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

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

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



[GitHub] [arrow] bkmgit commented on a change in pull request #11882: ARROW-9843: [C++][Python] Implement Between ternary kernel and Python bindings

Posted by GitBox <gi...@apache.org>.
bkmgit commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r787037948



##########
File path: cpp/src/arrow/compute/kernels/scalar_compare_test.cc
##########
@@ -1865,5 +1867,683 @@ TEST(TestMaxElementWiseMinElementWise, CommonTemporal) {
               ResultWith(ScalarFromJSON(date64(), "86400000")));
 }
 
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const Datum& val, const Datum& lhs,
+                            const Datum& rhs, const Datum& expected) {
+  ASSERT_OK_AND_ASSIGN(Datum result, Between(val, lhs, rhs, options, nullptr));
+  AssertArraysEqual(*expected.make_array(), *result.make_array(),
+                    /*verbose=*/true);
+}
+
+template <typename ArrowType>
+void ValidateBetween(BetweenOptions options, const Datum& val, const Datum& lhs,
+                     const Datum& rhs) {
+  CompareOperator lhs_val;
+  CompareOperator val_rhs;
+  BetweenOptions::Inclusive include_endpoints = options.inclusive;
+
+  if (include_endpoints == BetweenOptions::Inclusive::NEITHER) {
+    lhs_val = LESS;
+    val_rhs = LESS;
+  } else if (include_endpoints == BetweenOptions::Inclusive::LEFT) {
+    lhs_val = LESS_EQUAL;
+    val_rhs = LESS;
+  } else if (include_endpoints == BetweenOptions::Inclusive::RIGHT) {
+    lhs_val = LESS;
+    val_rhs = LESS_EQUAL;
+  } else {
+    lhs_val = LESS_EQUAL;
+    val_rhs = LESS_EQUAL;
+  }
+
+  ASSERT_OK_AND_ASSIGN(Datum resultl,
+                       CallFunction(CompareOperatorToFunctionName(lhs_val), {lhs, val}));
+  ASSERT_OK_AND_ASSIGN(Datum resultr,
+                       CallFunction(CompareOperatorToFunctionName(val_rhs), {val, rhs}));
+  ASSERT_OK_AND_ASSIGN(Datum expected, CallFunction("and", {resultl, resultr}));
+
+  ValidateBetween<ArrowType>(options, val, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+class TestNumericBetweenKernel : public ::testing::Test {};
+
+TYPED_TEST_SUITE(TestNumericBetweenKernel, NumericArrowTypes);
+TYPED_TEST(TestNumericBetweenKernel, SimpleBetweenArrayScalarScalar) {
+  using ScalarType = typename TypeTraits<TypeParam>::ScalarType;
+  using CType = typename TypeTraits<TypeParam>::CType;
+
+  Datum zero(std::make_shared<ScalarType>(CType(0)));
+  Datum four(std::make_shared<ScalarType>(CType(4)));
+  Datum null(std::make_shared<ScalarType>());
+  BetweenOptions InclusiveBoth(BetweenOptions::Inclusive::BOTH);
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"), zero,
+      four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      zero, four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,1,2,2]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[1,1,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,3,4,5]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[1,1,1,1,1,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,1,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null,0,1,1]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), null, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, null,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  BetweenOptions InclusiveLeft(BetweenOptions::Inclusive::LEFT);
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"), zero,
+      four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      zero, four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,1,2,2]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[1,1,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,3,4,5]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[1,1,1,1,0,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,0,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null,0,1,1]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), null, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, null,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  BetweenOptions InclusiveRight(BetweenOptions::Inclusive::RIGHT);
+  ValidateBetween<TypeParam>(
+      InclusiveRight, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"), zero,
+      four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      zero, four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,1,2,2]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,0,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,3,4,5]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,1,1,1,1,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,1,1,1,1,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null,0,1,1]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null,0,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), null, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, null,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  BetweenOptions InclusiveNeither(BetweenOptions::Inclusive::NEITHER);
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      zero, four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      zero, four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,1,2,2]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,0,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,3,4,5]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,1,1,1,0,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,0,1,1,1,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null,0,1,1]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null,0,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), null, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, null,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+}
+
+TYPED_TEST(TestNumericBetweenKernel, SimpleBetweenScalarArrayArray) {
+  using ScalarType = typename TypeTraits<TypeParam>::ScalarType;
+  using CType = typename TypeTraits<TypeParam>::CType;
+
+  Datum one(std::make_shared<ScalarType>(CType(1)));
+
+  BetweenOptions InclusiveBoth(BetweenOptions::Inclusive::BOTH);
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, one, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,true,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,true,null,false,false]"));
+  BetweenOptions InclusiveLeft(BetweenOptions::Inclusive::LEFT);
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, one, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,true,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,true,null,false,false]"));
+  BetweenOptions InclusiveRight(BetweenOptions::Inclusive::RIGHT);
+  ValidateBetween<TypeParam>(
+      InclusiveRight, one, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,false,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,false,null,false,false]"));
+  BetweenOptions InclusiveNeither(BetweenOptions::Inclusive::NEITHER);
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, one, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,false,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,false,null,false,false]"));
+}
+
+TYPED_TEST(TestNumericBetweenKernel, SimpleBetweenArrayArrayArray) {
+  BetweenOptions InclusiveBoth(BetweenOptions::Inclusive::BOTH);
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[1,1,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,true,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,true,null,false,false]"));
+  BetweenOptions InclusiveLeft(BetweenOptions::Inclusive::LEFT);
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[1,1,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,false,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,false,null,false,false]"));
+  BetweenOptions InclusiveRight(BetweenOptions::Inclusive::RIGHT);
+  ValidateBetween<TypeParam>(
+      InclusiveRight, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[1,1,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,true,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,true,null,false,false]"));
+  BetweenOptions InclusiveNeither(BetweenOptions::Inclusive::NEITHER);
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[1,1,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,false,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,false,null,false,false]"));
+}
+
+template <typename Type>
+struct BetweenRandomNumeric {

Review comment:
       The structure is the same as CompareRandomNumeric, can change both.




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

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

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



[GitHub] [arrow] bkmgit commented on a change in pull request #11882: ARROW-9843: [C++][Python] Implement Between ternary kernel and Python bindings

Posted by GitBox <gi...@apache.org>.
bkmgit commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r787595498



##########
File path: cpp/src/arrow/compute/kernels/scalar_compare_test.cc
##########
@@ -1865,5 +1867,683 @@ TEST(TestMaxElementWiseMinElementWise, CommonTemporal) {
               ResultWith(ScalarFromJSON(date64(), "86400000")));
 }
 
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const Datum& val, const Datum& lhs,
+                            const Datum& rhs, const Datum& expected) {
+  ASSERT_OK_AND_ASSIGN(Datum result, Between(val, lhs, rhs, options, nullptr));
+  AssertArraysEqual(*expected.make_array(), *result.make_array(),
+                    /*verbose=*/true);
+}
+
+template <typename ArrowType>
+void ValidateBetween(BetweenOptions options, const Datum& val, const Datum& lhs,
+                     const Datum& rhs) {
+  CompareOperator lhs_val;
+  CompareOperator val_rhs;
+  BetweenOptions::Inclusive include_endpoints = options.inclusive;
+
+  if (include_endpoints == BetweenOptions::Inclusive::NEITHER) {
+    lhs_val = LESS;
+    val_rhs = LESS;
+  } else if (include_endpoints == BetweenOptions::Inclusive::LEFT) {
+    lhs_val = LESS_EQUAL;
+    val_rhs = LESS;
+  } else if (include_endpoints == BetweenOptions::Inclusive::RIGHT) {
+    lhs_val = LESS;
+    val_rhs = LESS_EQUAL;
+  } else {
+    lhs_val = LESS_EQUAL;
+    val_rhs = LESS_EQUAL;
+  }
+
+  ASSERT_OK_AND_ASSIGN(Datum resultl,
+                       CallFunction(CompareOperatorToFunctionName(lhs_val), {lhs, val}));
+  ASSERT_OK_AND_ASSIGN(Datum resultr,
+                       CallFunction(CompareOperatorToFunctionName(val_rhs), {val, rhs}));
+  ASSERT_OK_AND_ASSIGN(Datum expected, CallFunction("and", {resultl, resultr}));
+
+  ValidateBetween<ArrowType>(options, val, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+class TestNumericBetweenKernel : public ::testing::Test {};
+
+TYPED_TEST_SUITE(TestNumericBetweenKernel, NumericArrowTypes);
+TYPED_TEST(TestNumericBetweenKernel, SimpleBetweenArrayScalarScalar) {
+  using ScalarType = typename TypeTraits<TypeParam>::ScalarType;
+  using CType = typename TypeTraits<TypeParam>::CType;
+
+  Datum zero(std::make_shared<ScalarType>(CType(0)));
+  Datum four(std::make_shared<ScalarType>(CType(4)));
+  Datum null(std::make_shared<ScalarType>());
+  BetweenOptions InclusiveBoth(BetweenOptions::Inclusive::BOTH);
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"), zero,
+      four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      zero, four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,1,2,2]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[1,1,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,3,4,5]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[1,1,1,1,1,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,1,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null,0,1,1]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), null, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, null,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  BetweenOptions InclusiveLeft(BetweenOptions::Inclusive::LEFT);
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"), zero,
+      four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      zero, four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,1,2,2]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[1,1,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,3,4,5]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[1,1,1,1,0,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,0,1,1,1,1]"));

Review comment:
       Put in a loop over the option.




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

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

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



[GitHub] [arrow] edponce commented on pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
edponce commented on pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#issuecomment-1002956781


   I have the following observation to make wrt to the API for this `between` compute function. The purpose of `between` is to support range comparisons similar to [SQL `BETWEEN`](https://www.w3schools.com/sql/sql_between.asp) and [pandas `between`](https://pandas.pydata.org/docs/reference/api/pandas.Series.between.html#pandas-series-between). These APIs are different from this PR's implementation, but the behavior is equivalent. Nevertheless, I suggest either of the following:
   1. Implement `between` compute function not as a _logical compare_ function but as an independent function that takes 3 inputs and an option for declaring bounds-inclusiveness. Similar to the Pandas API.
   2. Implement `between` as an extension to the _logical compare_ functions. Now there are two ways to go about this:
     a. As this PR does, implement 4 new compare functions `less_less`, `less_equal_less`, `less_less_equal`, and `less_equal_less_equal`.
     b. Have a single variadic `compare` function that uses options to select the type of comparison (e.g., LESS, GREATER_EQUAL, LESS_LESS)
   
   I recommend to follow option (1), for the following reasons. It is a common API and the function name clearly states its operation. The issue with (2a) is that its function names are not common-case, and if we make a single `between` with options for LESS_LESS, etc. then I would argue that the other logical comparisons need to be merged into a single compute function for symmetry/consistency purposes. This last point touches on option (2b) which I do not think is a reasonable solution because of its complexity and one could argue that an analogous case would be to have a single `arithmetic` function where the operation performed is specified via options. Obviously, this is not desired.


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

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

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



[GitHub] [arrow] edponce commented on a change in pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
edponce commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r776864703



##########
File path: cpp/src/arrow/compute/kernels/scalar_compare.cc
##########
@@ -746,6 +787,155 @@ std::shared_ptr<ScalarFunction> MakeScalarMinMax(std::string name,
   return func;
 }
 
+template <typename Op>
+void AddIntegerBetween(const std::shared_ptr<DataType>& ty, ScalarFunction* func) {
+  auto exec = GeneratePhysicalInteger<ScalarTernaryEqualTypes, BooleanType, Op>(*ty);
+  DCHECK_OK(func->AddKernel({ty, ty, ty}, boolean(), std::move(exec)));
+}
+
+template <typename InType, typename Op>
+void AddGenericBetween(const std::shared_ptr<DataType>& ty, ScalarFunction* func) {
+  DCHECK_OK(func->AddKernel({ty, ty, ty}, boolean(),
+                            ScalarTernaryEqualTypes<BooleanType, InType, Op>::Exec));
+}
+
+template <typename OutType, typename ArgType, typename Op>
+struct BetweenTimestamps : public ScalarTernaryEqualTypes<OutType, ArgType, Op> {
+  using Base = ScalarTernaryEqualTypes<OutType, ArgType, Op>;
+
+  static Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    const auto& var = checked_cast<const TimestampType&>(*batch[0].type());
+    const auto& lhs = checked_cast<const TimestampType&>(*batch[1].type());
+    const auto& rhs = checked_cast<const TimestampType&>(*batch[2].type());
+    if ((var.timezone().empty() != lhs.timezone().empty()) ||
+        (var.timezone().empty() != rhs.timezone().empty()) ||
+        (lhs.timezone().empty() != rhs.timezone().empty())) {
+      return Status::Invalid("Cannot compare timestamps with and without timezones.");
+    }
+    return Base::Exec(ctx, batch, out);
+  }
+};
+
+template <typename Op>
+std::shared_ptr<ScalarFunction> MakeBetweenFunction(std::string name,
+                                                    const FunctionDoc* doc) {
+  auto func = std::make_shared<CompareFunction>(name, Arity::Ternary(), doc);
+
+  DCHECK_OK(func->AddKernel(
+      {boolean(), boolean(), boolean()}, boolean(),
+      ScalarTernary<BooleanType, BooleanType, BooleanType, BooleanType, Op>::Exec));
+
+  for (const std::shared_ptr<DataType>& ty : IntTypes()) {
+    AddIntegerBetween<Op>(ty, func.get());
+  }
+
+  AddIntegerBetween<Op>(date32(), func.get());
+  AddIntegerBetween<Op>(date64(), func.get());
+
+  AddGenericBetween<FloatType, Op>(float32(), func.get());
+  AddGenericBetween<DoubleType, Op>(float64(), func.get());
+
+  // Add timestamp kernels
+  for (auto unit : TimeUnit::values()) {
+    InputType in_type(match::TimestampTypeUnit(unit));
+    DCHECK_OK(func->AddKernel({in_type, in_type, in_type}, boolean(),
+                              BetweenTimestamps<BooleanType, TimestampType, Op>::Exec));
+  }
+
+  // Duration
+  for (auto unit : TimeUnit::values()) {
+    InputType in_type(match::DurationTypeUnit(unit));
+    auto exec =
+        GeneratePhysicalInteger<ScalarTernaryEqualTypes, BooleanType, Op>(int64());
+    DCHECK_OK(func->AddKernel({in_type, in_type, in_type}, boolean(), std::move(exec)));
+  }
+
+  // Time32 and Time64
+  for (auto unit : {TimeUnit::SECOND, TimeUnit::MILLI}) {
+    InputType in_type(match::Time32TypeUnit(unit));
+    auto exec =
+        GeneratePhysicalInteger<ScalarTernaryEqualTypes, BooleanType, Op>(int32());
+    DCHECK_OK(func->AddKernel({in_type, in_type, in_type}, boolean(), std::move(exec)));
+  }
+  for (auto unit : {TimeUnit::MICRO, TimeUnit::NANO}) {
+    InputType in_type(match::Time64TypeUnit(unit));
+    auto exec =
+        GeneratePhysicalInteger<ScalarTernaryEqualTypes, BooleanType, Op>(int64());
+    DCHECK_OK(func->AddKernel({in_type, in_type, in_type}, boolean(), std::move(exec)));
+  }
+
+  for (const std::shared_ptr<DataType>& ty : BaseBinaryTypes()) {
+    auto exec = GenerateVarBinaryBase<ScalarTernaryEqualTypes, BooleanType, Op>(*ty);
+    DCHECK_OK(func->AddKernel(
+        {InputType::Array(ty), InputType::Scalar(ty), InputType::Scalar(ty)}, boolean(),
+        exec));
+    DCHECK_OK(func->AddKernel(
+        {InputType::Array(ty), InputType::Array(ty), InputType::Array(ty)}, boolean(),
+        std::move(exec)));
+  }
+
+  for (const auto id : {Type::DECIMAL128, Type::DECIMAL256}) {
+    auto exec = GenerateDecimal<ScalarTernaryEqualTypes, BooleanType, Op>(id);
+    DCHECK_OK(func->AddKernel({InputType(id), InputType(id), InputType(id)}, boolean(),
+                              std::move(exec)));
+  }
+
+  {
+    auto exec = ScalarTernaryEqualTypes<BooleanType, FixedSizeBinaryType, Op>::Exec;
+    auto ty = InputType(Type::FIXED_SIZE_BINARY);
+    DCHECK_OK(func->AddKernel({ty, ty, ty}, boolean(), std::move(exec)));
+  }
+
+  return func;
+}
+
+const BetweenOptions* GetDefaultBetweenOptions() {
+  static const auto kBetweenOptions = BetweenOptions::Defaults();
+  return &kBetweenOptions;
+}
+
+const FunctionDoc between_doc{"Check if values are in a range, val betwen a and b",
+                              ("A null on either side emits a null comparison result.\n"
+                               "options are used to specify if the endpoints are\n"
+                               "inclusive, possible values are NEITHER (a<val<b),\n"
+                               "LEFT (a<=val<b), RIGHT (a<val<=b), and the default\n"
+                               "if not specified BOTH (a<=val<=b)"),
+                              {"val", "a", "b"}};
+
+class BetweenMetaFunction : public MetaFunction {
+ public:
+  BetweenMetaFunction()
+      : MetaFunction("between", Arity::Ternary(), &between_doc,
+                     GetDefaultBetweenOptions()) {}
+
+  Result<Datum> ExecuteImpl(const std::vector<Datum>& args,
+                            const FunctionOptions* options,
+                            ExecContext* ctx) const override {
+    const BetweenOptions& between_options = static_cast<const BetweenOptions&>(*options);
+    Datum result;

Review comment:
       I disagree with this approach. It defeats the purpose of having the `BetweenOptions` because they are not being consumed by `CallFunction`.
   
   I will take a stab at supporting internal dispatching based on `BetweenOptions` and submit a PR to your branch. Up to now this is looking good but needs a bit of tweaking.




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

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

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



[GitHub] [arrow] bkmgit commented on pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
bkmgit commented on pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#issuecomment-1002987357


   These are good suggestions, thanks for your feedback. Can change BETWEEN to use options as the default. It seems we may also need NOT BETWEEN but will leave this for later if it is desired as it is in SQL. Language specific bindings may want to have additional features to make transitioning users easier.
   
   That being said, a single arithmetic function is quite desirable. One could do
   ARITHMETIC("add", a, b)
   ARITHMETIC("multiply",a,b)
   ARITHMETIC("modulo",a,b)
   
   At the moment there is:
   
   ARROW_ASSIGN_OR_RAISE(incremented_datum,
                         arrow::compute::CallFunction("add", {numbers_array, increment}));
   and the internal API which can be directly called
   ARROW_ASSIGN_OR_RAISE(incremented_datum,
                         arrow::compute::Add(numbers_array, increment));
   Internally, it would be nicer to have
   ARROW_ASSIGN_OR_RAISE(incremented_datum,
                         arrow::compute::Arithmetic::Add(numbers_array, increment));
   
   Templating could generate the related kernels efficiently, which would be useful for porting to GPUs and for kernel optimization. At the moment, related operations are put in the same file, but there is no other hierarchy for compute functions.
   
   Similarly, one would have
   COMPARE(a,b,"gt")
   COMPARE(a,b,"ge")
   COMPARE(a,b,"lt")
   COMPARE(a,b,"le")
   BETWEEN(val,a,b,"both")
   BETWEEN(val,a,b,"left")
   BETWEEN(val,a,b,"right")
   BETWEEN(val,a,b,"neither")
   NOT_BETWEEN(val,a,b,"both")
   NOT_BETWEEN(val,a,b,"left")
   NOT_BETWEEN(val,a,b,"right")
   NOT_BETWEEN(val,a,b,"neither")
   
   These would all be within scalar_compare.cc  If the BETWEEN function with options works okay, maybe we can transition GREATER, GREATER_EQUAL etc. to have a common COMPARE function with an option, as well as alternative bindings?


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

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

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



[GitHub] [arrow] bkmgit commented on pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
bkmgit commented on pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#issuecomment-1003163731


   @edponce `between` with the options `both`, `left`, `right` and `neither` is there now, though some cleanup and refactoring is needed as well as addition of variants other than ArrayArrayArray and ArrayScalarScalar. Used a MetaFunction rather than choosing the kernel during dispatch.  This gives language bindings some freedom in how they use these functions. It is also something that can be done for the compare function rather than entirely deprecating the convenient compare interface.


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

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

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



[GitHub] [arrow] bkmgit commented on a change in pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
bkmgit commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r767473378



##########
File path: cpp/src/arrow/compute/kernels/scalar_compare_test.cc
##########
@@ -1850,5 +1851,154 @@ TEST(TestMaxElementWiseMinElementWise, CommonTemporal) {
               ResultWith(ScalarFromJSON(date64(), "86400000")));
 }
 
+template <typename ArrowType>
+static void ValidateBetween(const Datum& val, const Datum& lhs, const Datum& rhs,
+                            const Datum& expected) {
+  ASSERT_OK_AND_ASSIGN(Datum result, Between(val, lhs, rhs));
+  AssertArraysEqual(*expected.make_array(), *result.make_array(),
+                    /*verbose=*/true);
+}
+
+template <typename ArrowType>
+static void ValidateBetween(const char* value_str, const Datum& lhs, const Datum& rhs,
+                            const char* expected_str) {
+  auto value = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), value_str);
+  auto expected = ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), expected_str);
+  ValidateBetween<ArrowType>(value, lhs, rhs, expected);
+}
+
+template <>
+void ValidateBetween<StringType>(const char* value_str, const Datum& lhs,
+                                 const Datum& rhs, const char* expected_str) {
+  auto value = ArrayFromJSON(utf8(), value_str);
+  auto expected = ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), expected_str);
+  ValidateBetween<StringType>(value, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+class TestNumericBetweenKernel : public ::testing::Test {};
+
+TYPED_TEST_SUITE(TestNumericBetweenKernel, NumericArrowTypes);
+TYPED_TEST(TestNumericBetweenKernel, SimpleBetweenArrayScalarScalar) {

Review comment:
       Ok




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

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

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



[GitHub] [arrow] kou commented on a change in pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
kou commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r768118485



##########
File path: cpp/src/arrow/compute/kernels/scalar_compare_test.cc
##########
@@ -1850,5 +1851,154 @@ TEST(TestMaxElementWiseMinElementWise, CommonTemporal) {
               ResultWith(ScalarFromJSON(date64(), "86400000")));
 }
 
+template <typename ArrowType>
+static void ValidateBetween(const Datum& val, const Datum& lhs, const Datum& rhs,
+                            const Datum& expected) {
+  ASSERT_OK_AND_ASSIGN(Datum result, Between(val, lhs, rhs));
+  AssertArraysEqual(*expected.make_array(), *result.make_array(),
+                    /*verbose=*/true);
+}
+
+template <typename ArrowType>
+static void ValidateBetween(const char* value_str, const Datum& lhs, const Datum& rhs,
+                            const char* expected_str) {
+  auto value = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), value_str);
+  auto expected = ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), expected_str);
+  ValidateBetween<ArrowType>(value, lhs, rhs, expected);
+}
+
+template <>
+void ValidateBetween<StringType>(const char* value_str, const Datum& lhs,
+                                 const Datum& rhs, const char* expected_str) {
+  auto value = ArrayFromJSON(utf8(), value_str);
+  auto expected = ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), expected_str);
+  ValidateBetween<StringType>(value, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+class TestNumericBetweenKernel : public ::testing::Test {};
+
+TYPED_TEST_SUITE(TestNumericBetweenKernel, NumericArrowTypes);
+TYPED_TEST(TestNumericBetweenKernel, SimpleBetweenArrayScalarScalar) {
+  using ScalarType = typename TypeTraits<TypeParam>::ScalarType;
+  using CType = typename TypeTraits<TypeParam>::CType;
+
+  Datum zero(std::make_shared<ScalarType>(CType(0)));
+  Datum four(std::make_shared<ScalarType>(CType(4)));
+  ValidateBetween<TypeParam>("[]", zero, four, "[]");
+  ValidateBetween<TypeParam>("[null]", zero, four, "[null]");
+  ValidateBetween<TypeParam>("[0,0,1,1,2,2]", zero, four, "[0,0,1,1,1,1]");
+  ValidateBetween<TypeParam>("[0,1,2,3,4,5]", zero, four, "[0,1,1,1,0,0]");
+  ValidateBetween<TypeParam>("[5,4,3,2,1,0]", zero, four, "[0,0,1,1,1,0]");
+  ValidateBetween<TypeParam>("[null,0,1,1]", zero, four, "[null,0,1,1]");
+}
+
+TEST(TestSimpleBetweenKernel, SimpleStringTest) {
+  using ScalarType = typename TypeTraits<StringType>::ScalarType;
+  auto l = Datum(std::make_shared<ScalarType>("abc"));
+  auto r = Datum(std::make_shared<ScalarType>("zzz"));
+  ValidateBetween<StringType>("[]", l, r, "[]");
+  ValidateBetween<StringType>("[null]", l, r, "[null]");
+  ValidateBetween<StringType>(R"(["aaa", "aaaa", "ccc", "z"])", l, r,
+                              R"([false, false, true, true])");
+  ValidateBetween<StringType>(R"(["a", "aaaa", "c", "z"])", l, r,
+                              R"([false, false, true, true])");
+  ValidateBetween<StringType>(R"(["a", "aaaa", "fff", "zzzz"])", l, r,
+                              R"([false, false, true, false])");
+  ValidateBetween<StringType>(R"(["abd", null, null, "zzx"])", l, r,
+                              R"([true, null, null, true])");
+}
+
+TEST(TestSimpleBetweenKernel, SimpleTimestampTest) {
+  using ScalarType = typename TypeTraits<TimestampType>::ScalarType;
+  auto checkTimestampArray = [](std::shared_ptr<DataType> type, const char* input_str,
+                                const Datum& lhs, const Datum& rhs,
+                                const char* expected_str) {
+    auto value = ArrayFromJSON(type, input_str);
+    auto expected = ArrayFromJSON(boolean(), expected_str);
+    ValidateBetween<TimestampType>(value, lhs, rhs, expected);
+  };
+  auto unit = TimeUnit::SECOND;
+  auto l = Datum(std::make_shared<ScalarType>(923184000, timestamp(unit)));
+  auto r = Datum(std::make_shared<ScalarType>(1602032602, timestamp(unit)));
+  checkTimestampArray(timestamp(unit), "[]", l, r, "[]");
+  checkTimestampArray(timestamp(unit), "[null]", l, r, "[null]");
+  checkTimestampArray(timestamp(unit), R"(["1970-01-01","2000-02-29","1900-02-28"])", l,
+                      r, "[false,true,false]");
+  checkTimestampArray(timestamp(unit), R"(["1970-01-01","2000-02-29","2004-02-28"])", l,
+                      r, "[false,true,true]");
+  checkTimestampArray(timestamp(unit), R"(["2018-01-01","1999-04-04","1900-02-28"])", l,
+                      r, "[true,false,false]");
+}
+
+TYPED_TEST(TestNumericBetweenKernel, SimpleBetweenArrayArrayArray) {
+  ValidateBetween<TypeParam>(
+      "[]", ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"), "[]");
+  ValidateBetween<TypeParam>(
+      "[null]", ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"), "[null]");
+  ValidateBetween<TypeParam>(
+      "[1,1,2,2,2]",
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      "[true,true,false,false,false]");
+  ValidateBetween<TypeParam>(
+      "[1,1,2,2,2,2]",
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,2,5,5]"),
+      "[true,true,false,null,false,false]");
+}
+
+TEST(TestSimpleBetweenKernel, StringArrayArrayArrayTest) {
+  ValidateBetween<StringType>(
+      R"(["david","hello","world"])",
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["adam","hi","whirl"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                    R"(["robert","goeiemoreen","whirlwind"])"),
+      "[true, false, false]");
+  ValidateBetween<StringType>(
+      R"(["x","a","f"])",
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["w","a","e"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["z","a","g"])"),
+      "[true, false, true]");
+  ValidateBetween<StringType>(
+      R"(["block","bit","binary"])",
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                    R"(["bit","nibble","ternary"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["word","d","xyz"])"),
+      "[true, false, false]");
+  ValidateBetween<StringType>(R"(["Ayumi","アユミ","王梦莹"])",
+                              ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                            R"(["たなか","あゆみ","歩美"])"),
+                              ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                            R"(["李平之","田中","たなか"])"),
+                              "[false, true, false]");

Review comment:
       Using Japanese for test data is OK but test data should be simple as much as possible for readability/maintenability.
   Could you note the issue as comment?




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

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

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



[GitHub] [arrow] kou commented on a change in pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
kou commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r767455395



##########
File path: cpp/src/arrow/compute/kernels/scalar_compare_test.cc
##########
@@ -1850,5 +1851,154 @@ TEST(TestMaxElementWiseMinElementWise, CommonTemporal) {
               ResultWith(ScalarFromJSON(date64(), "86400000")));
 }
 
+template <typename ArrowType>
+static void ValidateBetween(const Datum& val, const Datum& lhs, const Datum& rhs,
+                            const Datum& expected) {
+  ASSERT_OK_AND_ASSIGN(Datum result, Between(val, lhs, rhs));
+  AssertArraysEqual(*expected.make_array(), *result.make_array(),
+                    /*verbose=*/true);
+}
+
+template <typename ArrowType>
+static void ValidateBetween(const char* value_str, const Datum& lhs, const Datum& rhs,
+                            const char* expected_str) {
+  auto value = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), value_str);
+  auto expected = ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), expected_str);
+  ValidateBetween<ArrowType>(value, lhs, rhs, expected);
+}
+
+template <>
+void ValidateBetween<StringType>(const char* value_str, const Datum& lhs,
+                                 const Datum& rhs, const char* expected_str) {
+  auto value = ArrayFromJSON(utf8(), value_str);
+  auto expected = ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), expected_str);
+  ValidateBetween<StringType>(value, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+class TestNumericBetweenKernel : public ::testing::Test {};
+
+TYPED_TEST_SUITE(TestNumericBetweenKernel, NumericArrowTypes);
+TYPED_TEST(TestNumericBetweenKernel, SimpleBetweenArrayScalarScalar) {
+  using ScalarType = typename TypeTraits<TypeParam>::ScalarType;
+  using CType = typename TypeTraits<TypeParam>::CType;
+
+  Datum zero(std::make_shared<ScalarType>(CType(0)));
+  Datum four(std::make_shared<ScalarType>(CType(4)));
+  ValidateBetween<TypeParam>("[]", zero, four, "[]");
+  ValidateBetween<TypeParam>("[null]", zero, four, "[null]");
+  ValidateBetween<TypeParam>("[0,0,1,1,2,2]", zero, four, "[0,0,1,1,1,1]");
+  ValidateBetween<TypeParam>("[0,1,2,3,4,5]", zero, four, "[0,1,1,1,0,0]");
+  ValidateBetween<TypeParam>("[5,4,3,2,1,0]", zero, four, "[0,0,1,1,1,0]");
+  ValidateBetween<TypeParam>("[null,0,1,1]", zero, four, "[null,0,1,1]");
+}
+
+TEST(TestSimpleBetweenKernel, SimpleStringTest) {
+  using ScalarType = typename TypeTraits<StringType>::ScalarType;
+  auto l = Datum(std::make_shared<ScalarType>("abc"));
+  auto r = Datum(std::make_shared<ScalarType>("zzz"));
+  ValidateBetween<StringType>("[]", l, r, "[]");
+  ValidateBetween<StringType>("[null]", l, r, "[null]");
+  ValidateBetween<StringType>(R"(["aaa", "aaaa", "ccc", "z"])", l, r,
+                              R"([false, false, true, true])");
+  ValidateBetween<StringType>(R"(["a", "aaaa", "c", "z"])", l, r,
+                              R"([false, false, true, true])");
+  ValidateBetween<StringType>(R"(["a", "aaaa", "fff", "zzzz"])", l, r,
+                              R"([false, false, true, false])");
+  ValidateBetween<StringType>(R"(["abd", null, null, "zzx"])", l, r,
+                              R"([true, null, null, true])");
+}
+
+TEST(TestSimpleBetweenKernel, SimpleTimestampTest) {
+  using ScalarType = typename TypeTraits<TimestampType>::ScalarType;
+  auto checkTimestampArray = [](std::shared_ptr<DataType> type, const char* input_str,
+                                const Datum& lhs, const Datum& rhs,
+                                const char* expected_str) {
+    auto value = ArrayFromJSON(type, input_str);
+    auto expected = ArrayFromJSON(boolean(), expected_str);
+    ValidateBetween<TimestampType>(value, lhs, rhs, expected);
+  };
+  auto unit = TimeUnit::SECOND;
+  auto l = Datum(std::make_shared<ScalarType>(923184000, timestamp(unit)));
+  auto r = Datum(std::make_shared<ScalarType>(1602032602, timestamp(unit)));
+  checkTimestampArray(timestamp(unit), "[]", l, r, "[]");
+  checkTimestampArray(timestamp(unit), "[null]", l, r, "[null]");
+  checkTimestampArray(timestamp(unit), R"(["1970-01-01","2000-02-29","1900-02-28"])", l,
+                      r, "[false,true,false]");
+  checkTimestampArray(timestamp(unit), R"(["1970-01-01","2000-02-29","2004-02-28"])", l,
+                      r, "[false,true,true]");
+  checkTimestampArray(timestamp(unit), R"(["2018-01-01","1999-04-04","1900-02-28"])", l,
+                      r, "[true,false,false]");
+}
+
+TYPED_TEST(TestNumericBetweenKernel, SimpleBetweenArrayArrayArray) {
+  ValidateBetween<TypeParam>(
+      "[]", ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"), "[]");
+  ValidateBetween<TypeParam>(
+      "[null]", ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"), "[null]");
+  ValidateBetween<TypeParam>(
+      "[1,1,2,2,2]",
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      "[true,true,false,false,false]");
+  ValidateBetween<TypeParam>(
+      "[1,1,2,2,2,2]",
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,2,5,5]"),
+      "[true,true,false,null,false,false]");
+}
+
+TEST(TestSimpleBetweenKernel, StringArrayArrayArrayTest) {
+  ValidateBetween<StringType>(
+      R"(["david","hello","world"])",
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["adam","hi","whirl"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                    R"(["robert","goeiemoreen","whirlwind"])"),
+      "[true, false, false]");
+  ValidateBetween<StringType>(
+      R"(["x","a","f"])",
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["w","a","e"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["z","a","g"])"),
+      "[true, false, true]");
+  ValidateBetween<StringType>(
+      R"(["block","bit","binary"])",
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                    R"(["bit","nibble","ternary"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["word","d","xyz"])"),
+      "[true, false, false]");
+  ValidateBetween<StringType>(R"(["Ayumi","アユミ","王梦莹"])",
+                              ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                            R"(["たなか","あゆみ","歩美"])"),
+                              ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                            R"(["李平之","田中","たなか"])"),
+                              "[false, true, false]");

Review comment:
       I think that this is not a suitable test case for non ASCII characters.
   It uses Japanese but it's difficult to understand expected order even for Japanese (me).
   `"あいう"` ("abc" for English) or something will be better than `"あゆみ"`, `"アユミ"` and so on. 




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

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

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



[GitHub] [arrow] lidavidm commented on a change in pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
lidavidm commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r773114262



##########
File path: cpp/src/arrow/compute/kernels/scalar_compare_test.cc
##########
@@ -1850,5 +1846,439 @@ TEST(TestMaxElementWiseMinElementWise, CommonTemporal) {
               ResultWith(ScalarFromJSON(date64(), "86400000")));
 }
 
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const Datum& val, const Datum& lhs,
+                            const Datum& rhs, const Datum& expected) {
+  ASSERT_OK_AND_ASSIGN(
+      Datum result,
+      CallFunction(BetweenOperatorToFunctionName(options.op), {val, lhs, rhs}));
+  AssertArraysEqual(*expected.make_array(), *result.make_array(),
+                    /*verbose=*/true);
+}
+
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const Datum& val, const Datum& lhs,
+                            const Datum& rhs, const char* expected_str) {
+  auto expected = ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), expected_str);
+  ValidateBetween<ArrowType>(options, val, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const Datum& val, const Datum& lhs,
+                            const char* rhs_str, const Datum& expected) {
+  auto rhs = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), rhs_str);
+  ValidateBetween<ArrowType>(options, val, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const Datum& val, const char* lhs_str,
+                            const Datum& rhs, const Datum& expected) {
+  auto lhs = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), lhs_str);
+  ValidateBetween<ArrowType>(options, val, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const char* val_str, const Datum& lhs,
+                            const Datum& rhs, const Datum& expected) {
+  auto val = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), val_str);
+  ValidateBetween<ArrowType>(options, val, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const Datum& val, const Datum& lhs,
+                            const char* rhs_str, const char* expected_str) {
+  auto rhs = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), rhs_str);
+  auto expected = ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), expected_str);
+  ValidateBetween<ArrowType>(options, val, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const Datum& val, const char* lhs_str,
+                            const Datum& rhs, const char* expected_str) {
+  auto lhs = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), lhs_str);
+  auto expected = ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), expected_str);
+  ValidateBetween<ArrowType>(options, val, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const Datum& val, const char* lhs_str,
+                            const char* rhs_str, const Datum& expected) {
+  auto lhs = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), lhs_str);
+  auto rhs = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), rhs_str);
+  ValidateBetween<ArrowType>(options, val, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const char* val_str, const Datum& lhs,
+                            const Datum& rhs, const char* expected_str) {
+  auto val = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), val_str);
+  auto expected = ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), expected_str);
+  ValidateBetween<ArrowType>(options, val, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const char* val_str,
+                            const char* lhs_str, const Datum& rhs,
+                            const Datum& expected) {
+  auto val = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), val_str);
+  auto lhs = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), lhs_str);
+  ValidateBetween<ArrowType>(options, val, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const char* val_str, const Datum& lhs,
+                            const char* rhs_str, const Datum& expected) {
+  auto val = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), val_str);
+  auto rhs = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), rhs_str);
+  ValidateBetween<ArrowType>(options, val, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const Datum& val, const char* lhs_str,
+                            const char* rhs_str, const char* expected_str) {
+  auto lhs = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), lhs_str);
+  auto rhs = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), rhs_str);
+  auto expected = ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), expected_str);
+  ValidateBetween<ArrowType>(options, val, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const char* val_str, const Datum& lhs,
+                            const char* rhs_str, const char* expected_str) {
+  auto val = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), val_str);
+  auto rhs = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), rhs_str);
+  auto expected = ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), expected_str);
+  ValidateBetween<ArrowType>(options, val, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const char* val_str,
+                            const char* lhs_str, const Datum& rhs,
+                            const char* expected_str) {
+  auto val = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), val_str);
+  auto lhs = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), lhs_str);
+  auto expected = ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), expected_str);
+  ValidateBetween<ArrowType>(options, val, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const char* val_str,
+                            const char* lhs_str, const char* rhs_str,
+                            const Datum& expected) {
+  auto val = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), val_str);
+  auto lhs = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), lhs_str);
+  auto rhs = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), rhs_str);
+  ValidateBetween<ArrowType>(options, val, lhs, rhs, expected);
+}
+
+template <typename T>
+static inline bool SlowBetween(BetweenOperator op, const T& val, const T& lhs,
+                               const T& rhs) {
+  switch (op) {
+    case BETWEEN_LESS_EQUAL_LESS_EQUAL:
+      return ((lhs <= val) && (val <= rhs));
+    case BETWEEN_LESS_EQUAL_LESS_THAN:
+      return ((lhs <= val) && (val < rhs));
+    case BETWEEN_LESS_THAN_LESS_EQUAL:
+      return ((lhs < val) && (val <= rhs));
+    case BETWEEN_LESS_THAN_LESS_THAN:
+      return ((lhs < val) && (val < rhs));
+    default:
+      return false;
+  }
+}
+
+template <typename ArrayType>
+std::vector<bool> NullBitmapFromThreeArrays(const ArrayType& val, const ArrayType& lhs,
+                                            const ArrayType& rhs) {
+  auto value_lambda = [&val](int64_t i) {
+    return val.null_count() == 0 ? true : val.IsValid(i);
+  };
+
+  auto left_lambda = [&lhs](int64_t i) {
+    return lhs.null_count() == 0 ? true : lhs.IsValid(i);
+  };
+
+  auto right_lambda = [&rhs](int64_t i) {
+    return rhs.null_count() == 0 ? true : rhs.IsValid(i);
+  };
+
+  const int64_t length = lhs.length();
+  std::vector<bool> null_bitmap(length);
+
+  for (int64_t i = 0; i < length; i++) {
+    null_bitmap[i] = value_lambda(i) && left_lambda(i) && right_lambda(i);
+  }
+
+  return null_bitmap;
+}
+
+template <typename ArrowType>
+Datum SimpleArrayScalarScalarBetween(BetweenOptions options, const Datum& val,

Review comment:
       Sorry, I meant these test functions. Not the actual kernel.




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

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

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



[GitHub] [arrow] bkmgit commented on a change in pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
bkmgit commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r772831263



##########
File path: cpp/src/arrow/compute/api_scalar.h
##########
@@ -1240,5 +1253,21 @@ ARROW_EXPORT Result<Datum> AssumeTimezone(const Datum& values,
                                           AssumeTimezoneOptions options,
                                           ExecContext* ctx = NULLPTR);
 
+/// \brief Between compares each element in `values`
+/// with `left` as a lower bound and 'right' as an upperbound
+///
+/// \param[in] values input to compare between left and right
+/// \param[in] left used as the lower bound for comparison
+/// \param[in] right used as the upper bound for comparison
+/// \param[in] ctx the function execution context, optional
+///
+/// \return the resulting datum
+///
+/// \note Bounds are not inclusive

Review comment:
       Thanks for catching that.




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

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

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



[GitHub] [arrow] bkmgit commented on a change in pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
bkmgit commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r771099492



##########
File path: cpp/src/arrow/compute/kernels/scalar_compare.cc
##########
@@ -68,6 +72,50 @@ struct GreaterEqual {
   }
 };
 
+struct BetweenLessEqualLessEqual {
+  template <typename T, typename Arg0, typename Arg1, typename Arg2>
+  static constexpr T Call(KernelContext*, const Arg0& middle, const Arg1& left,
+                          const Arg2& right, Status*) {
+    static_assert(std::is_same<T, bool>::value && std::is_same<Arg0, Arg1>::value &&
+                      std::is_same<Arg1, Arg2>::value,
+                  "");
+    return (left <= middle) && (middle <= right);
+  }
+};
+
+struct BetweenLessThanLessEqual {
+  template <typename T, typename Arg0, typename Arg1, typename Arg2>
+  static constexpr T Call(KernelContext*, const Arg0& middle, const Arg1& left,
+                          const Arg2& right, Status*) {
+    static_assert(std::is_same<T, bool>::value && std::is_same<Arg0, Arg1>::value &&
+                      std::is_same<Arg1, Arg2>::value,
+                  "");
+    return (left < middle) && (middle <= right);
+  }
+};
+
+struct BetweenLessEqualLessThan {
+  template <typename T, typename Arg0, typename Arg1, typename Arg2>
+  static constexpr T Call(KernelContext*, const Arg0& middle, const Arg1& left,
+                          const Arg2& right, Status*) {
+    static_assert(std::is_same<T, bool>::value && std::is_same<Arg0, Arg1>::value &&
+                      std::is_same<Arg1, Arg2>::value,
+                  "");
+    return (left <= middle) && (middle < right);
+  }
+};
+
+struct BetweenLessThanLessThan {
+  template <typename T, typename Arg0, typename Arg1, typename Arg2>
+  static constexpr T Call(KernelContext*, const Arg0& middle, const Arg1& left,
+                          const Arg2& right, Status*) {
+    static_assert(std::is_same<T, bool>::value && std::is_same<Arg0, Arg1>::value &&
+                      std::is_same<Arg1, Arg2>::value,
+                  "");
+    return (left < middle) && (middle < right);
+  }
+};

Review comment:
       @edponce Thanks for the suggestion.




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

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

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



[GitHub] [arrow] bkmgit commented on a change in pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
bkmgit commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r775353155



##########
File path: cpp/src/arrow/compute/api_scalar.h
##########
@@ -316,6 +316,19 @@ struct ARROW_EXPORT CompareOptions {
   enum CompareOperator op;
 };
 
+enum BetweenOperator : int8_t {
+  BETWEEN_LESS_EQUAL_LESS_EQUAL,
+  BETWEEN_LESS_EQUAL_LESS_THAN,
+  BETWEEN_LESS_THAN_LESS_EQUAL,
+  BETWEEN_LESS_THAN_LESS_THAN,
+};
+
+struct ARROW_EXPORT BetweenOptions {
+  explicit BetweenOptions(BetweenOperator op) : op(op) {}
+  BetweenOptions() : BetweenOptions(BetweenOperator::BETWEEN_LESS_EQUAL_LESS_EQUAL) {}
+  enum BetweenOperator op;

Review comment:
       Four options seem to make sense, nesting further seems to complicate things unnecessarily
   
   1. LESS_LESS
   2. LESS_EQUAL_LESS
   3. LESS_LESS_EQUAL
   4. LESS_EQUAL_LESS_EQUAL




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

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

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



[GitHub] [arrow] bkmgit commented on a change in pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
bkmgit commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r780780395



##########
File path: cpp/src/arrow/util/bit_block_counter.h
##########
@@ -424,6 +565,141 @@ class ARROW_EXPORT OptionalBinaryBitBlockCounter {
   }
 };
 
+/// \brief A class that computes popcounts on the result of bitwise operations
+/// between three bitmaps, 64 bits at a time. A 64-bit word is loaded from each
+/// bitmap, then the popcount is computed on e.g. the bitwise-and of the three
+/// words.
+class ARROW_EXPORT TernaryBitBlockCounter {
+ public:
+  TernaryBitBlockCounter(const uint8_t* left_bitmap, int64_t left_offset,
+                         const uint8_t* mid_bitmap, int64_t mid_offset,
+                         const uint8_t* right_bitmap, int64_t right_offset,
+                         int64_t length)
+      : left_bitmap_(util::MakeNonNull(left_bitmap) + left_offset / 8),
+        left_offset_(left_offset % 8),
+        mid_bitmap_(util::MakeNonNull(mid_bitmap) + mid_offset / 8),
+        mid_offset_(mid_offset % 8),
+        right_bitmap_(util::MakeNonNull(right_bitmap) + right_offset / 8),
+        right_offset_(right_offset % 8),
+        bits_remaining_(length) {}
+
+  /// \brief Return the popcount of the bitwise-and of the next run of
+  /// available bits, up to 64. The returned pair contains the size of run and
+  /// the number of true values. The last block will have a length less than 64
+  /// if the bitmap length is not a multiple of 64, and will return 0-length
+  /// blocks in subsequent invocations.
+  BitBlockCount NextAndAndWord() { return NextWord<detail::BitBlockAndAnd>(); }

Review comment:
       Done.




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

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

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



[GitHub] [arrow] bkmgit commented on a change in pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
bkmgit commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r782685962



##########
File path: docs/source/cpp/compute.rst
##########
@@ -664,35 +664,43 @@ Decimal values are accepted, but are cast to Float64 first.
 Comparisons
 ~~~~~~~~~~~
 
-These functions expect two inputs of numeric type (in which case they will be
+These functions expect two or three inputs of numeric type (in which case they will be
 cast to the :ref:`common numeric type <common-numeric-type>` before comparison),
-or two inputs of Binary- or String-like types, or two inputs of Temporal types.
-If any input is dictionary encoded it will be expanded for the purposes of
-comparison. If any of the input elements in a pair is null, the corresponding
+or two or three inputs of Binary- or String-like types, or two or three inputs of Temporal 
+types. If any input is dictionary encoded it will be expanded for the purposes of
+comparison. If any of the input elements in a pair or triple is null, the corresponding
 output element is null. Decimal arguments will be promoted in the same way as
 for ``add`` and ``subtract``.
 
-+----------------+------------+---------------------------------------------+---------------------+
-| Function names | Arity      | Input types                                 | Output type         |
-+================+============+=============================================+=====================+
-| equal          | Binary     | Numeric, Temporal, Binary- and String-like  | Boolean             |
-+----------------+------------+---------------------------------------------+---------------------+
-| greater        | Binary     | Numeric, Temporal, Binary- and String-like  | Boolean             |
-+----------------+------------+---------------------------------------------+---------------------+
-| greater_equal  | Binary     | Numeric, Temporal, Binary- and String-like  | Boolean             |
-+----------------+------------+---------------------------------------------+---------------------+
-| less           | Binary     | Numeric, Temporal, Binary- and String-like  | Boolean             |
-+----------------+------------+---------------------------------------------+---------------------+
-| less_equal     | Binary     | Numeric, Temporal, Binary- and String-like  | Boolean             |
-+----------------+------------+---------------------------------------------+---------------------+
-| not_equal      | Binary     | Numeric, Temporal, Binary- and String-like  | Boolean             |
-+----------------+------------+---------------------------------------------+---------------------+
++----------------+---------+---------------------------------------------+-------------+--------------------------+-------+
+| Function names | Arity   | Input types                                 | Output type | Options Class            | Notes |
++================+=========+=============================================+=============+==========================+=======+
+| equal          | Binary  | Numeric, Temporal, Binary- and String-like  | Boolean     |                          |       |
++----------------+---------+---------------------------------------------+-------------+--------------------------+-------+
+| greater        | Binary  | Numeric, Temporal, Binary- and String-like  | Boolean     |                          |       |
++----------------+---------+---------------------------------------------+-------------+--------------------------+-------+
+| greater_equal  | Binary  | Numeric, Temporal, Binary- and String-like  | Boolean     |                          |       |
++----------------+---------+---------------------------------------------+-------------+--------------------------+-------+
+| less           | Binary  | Numeric, Temporal, Binary- and String-like  | Boolean     |                          |       |
++----------------+---------+---------------------------------------------+-------------+--------------------------+-------+
+| less_equal     | Binary  | Numeric, Temporal, Binary- and String-like  | Boolean     |                          |       |
++----------------+---------+---------------------------------------------+-------------+--------------------------+-------+
+| not_equal      | Binary  | Numeric, Temporal, Binary- and String-like  | Boolean     |                          |       |
++----------------+---------+---------------------------------------------+-------------+--------------------------+-------+
+| between        | Ternary | Numeric, Temporal, Binary- and String-like  | Boolean     | :struct:`BetweenOptions` | \(1)  |

Review comment:
       Done.




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

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

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



[GitHub] [arrow] bkmgit commented on a change in pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
bkmgit commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r784973501



##########
File path: cpp/src/arrow/compute/kernels/scalar_compare_test.cc
##########
@@ -1865,5 +1861,636 @@ TEST(TestMaxElementWiseMinElementWise, CommonTemporal) {
               ResultWith(ScalarFromJSON(date64(), "86400000")));
 }
 
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const Datum& val, const Datum& lhs,
+                            const Datum& rhs, const Datum& expected) {
+  ASSERT_OK_AND_ASSIGN(Datum result, Between(val, lhs, rhs, options, nullptr));
+  AssertArraysEqual(*expected.make_array(), *result.make_array(),
+                    /*verbose=*/true);
+}
+
+template <typename ArrowType>
+void ValidateBetween(BetweenOptions options, const Datum& val, const Datum& lhs,
+                     const Datum& rhs) {
+  CompareOperator lhs_val;
+  CompareOperator val_rhs;
+  BetweenOptions::Inclusive include_endpoints = options.inclusive;
+
+  if (include_endpoints == BetweenOptions::Inclusive::NEITHER) {
+    lhs_val = LESS;
+    val_rhs = LESS;
+  } else if (include_endpoints == BetweenOptions::Inclusive::LEFT) {
+    lhs_val = LESS_EQUAL;
+    val_rhs = LESS;
+  } else if (include_endpoints == BetweenOptions::Inclusive::RIGHT) {
+    lhs_val = LESS;
+    val_rhs = LESS_EQUAL;
+  } else {
+    lhs_val = LESS_EQUAL;
+    val_rhs = LESS_EQUAL;
+  }
+
+  ASSERT_OK_AND_ASSIGN(Datum resultl,
+                       CallFunction(CompareOperatorToFunctionName(lhs_val), {lhs, val}));
+  ASSERT_OK_AND_ASSIGN(Datum resultr,
+                       CallFunction(CompareOperatorToFunctionName(val_rhs), {val, rhs}));
+  ASSERT_OK_AND_ASSIGN(Datum expected, CallFunction("and", {resultl, resultr}));
+
+  ValidateBetween<ArrowType>(options, val, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+class TestNumericBetweenKernel : public ::testing::Test {};
+
+TYPED_TEST_SUITE(TestNumericBetweenKernel, NumericArrowTypes);
+TYPED_TEST(TestNumericBetweenKernel, SimpleBetweenArrayScalarScalar) {
+  using ScalarType = typename TypeTraits<TypeParam>::ScalarType;
+  using CType = typename TypeTraits<TypeParam>::CType;
+
+  Datum zero(std::make_shared<ScalarType>(CType(0)));
+  Datum four(std::make_shared<ScalarType>(CType(4)));
+  Datum null(std::make_shared<ScalarType>());
+  BetweenOptions InclusiveBoth(BetweenOptions::Inclusive::BOTH);
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"), zero,
+      four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      zero, four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,1,2,2]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[1,1,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,3,4,5]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[1,1,1,1,1,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,1,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null,0,1,1]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), null, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, null,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  BetweenOptions InclusiveLeft(BetweenOptions::Inclusive::LEFT);
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"), zero,
+      four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      zero, four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,1,2,2]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[1,1,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,3,4,5]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[1,1,1,1,0,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,0,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null,0,1,1]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), null, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, null,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  BetweenOptions InclusiveRight(BetweenOptions::Inclusive::RIGHT);
+  ValidateBetween<TypeParam>(
+      InclusiveRight, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"), zero,
+      four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      zero, four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,1,2,2]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,0,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,3,4,5]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,1,1,1,1,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,1,1,1,1,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null,0,1,1]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null,0,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), null, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, null,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  BetweenOptions InclusiveNeither(BetweenOptions::Inclusive::NEITHER);
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      zero, four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      zero, four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,1,2,2]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,0,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,3,4,5]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,1,1,1,0,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,0,1,1,1,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null,0,1,1]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null,0,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), null, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, null,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+}
+
+TYPED_TEST(TestNumericBetweenKernel, SimpleBetweenScalarArrayArray) {
+  using ScalarType = typename TypeTraits<TypeParam>::ScalarType;
+  using CType = typename TypeTraits<TypeParam>::CType;
+
+  Datum one(std::make_shared<ScalarType>(CType(1)));
+
+  BetweenOptions InclusiveBoth(BetweenOptions::Inclusive::BOTH);
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, one, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,true,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,true,null,false,false]"));
+  BetweenOptions InclusiveLeft(BetweenOptions::Inclusive::LEFT);
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, one, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,true,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,true,null,false,false]"));
+  BetweenOptions InclusiveRight(BetweenOptions::Inclusive::RIGHT);
+  ValidateBetween<TypeParam>(
+      InclusiveRight, one, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,false,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,false,null,false,false]"));
+  BetweenOptions InclusiveNeither(BetweenOptions::Inclusive::NEITHER);
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, one, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,false,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,false,null,false,false]"));
+}
+
+TYPED_TEST(TestNumericBetweenKernel, SimpleBetweenArrayArrayArray) {
+  BetweenOptions InclusiveBoth(BetweenOptions::Inclusive::BOTH);
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[1,1,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,true,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,true,null,false,false]"));
+  BetweenOptions InclusiveLeft(BetweenOptions::Inclusive::LEFT);
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[1,1,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,false,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,false,null,false,false]"));
+  BetweenOptions InclusiveRight(BetweenOptions::Inclusive::RIGHT);
+  ValidateBetween<TypeParam>(
+      InclusiveRight, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[1,1,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,true,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,true,null,false,false]"));
+  BetweenOptions InclusiveNeither(BetweenOptions::Inclusive::NEITHER);
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[1,1,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,false,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,false,null,false,false]"));
+}
+
+template <typename Type>
+struct BetweenRandomNumeric {
+  static void Test(const std::shared_ptr<DataType>& type) {
+    using ScalarType = typename TypeTraits<Type>::ScalarType;
+    using CType = typename TypeTraits<Type>::CType;
+    auto rand = random::RandomArrayGenerator(0x5416447);
+    const int64_t length = 1000;
+    for (auto null_probability : {0.0, 0.01, 0.1, 0.25, 0.5, 1.0}) {
+      for (auto inclusive :
+           {BetweenOptions::Inclusive::BOTH, BetweenOptions::Inclusive::LEFT,
+            BetweenOptions::Inclusive::RIGHT, BetweenOptions::Inclusive::NEITHER}) {
+        auto data1 =
+            rand.Numeric<typename Type::PhysicalType>(length, 0, 100, null_probability);
+        auto data2 =
+            rand.Numeric<typename Type::PhysicalType>(length, 0, 100, null_probability);
+        auto data3 =
+            rand.Numeric<typename Type::PhysicalType>(length, 0, 100, null_probability);
+
+        // Create view of data as the type (e.g. timestamp)
+        auto array1 = Datum(*data1->View(type));
+        auto array2 = Datum(*data2->View(type));
+        auto array3 = Datum(*data3->View(type));
+        auto scalar1 = Datum(std::make_shared<ScalarType>(CType(10), type));
+        auto scalar2 = Datum(std::make_shared<ScalarType>(CType(30), type));
+        auto scalar3 = Datum(std::make_shared<ScalarType>(CType(50), type));
+        auto options = BetweenOptions(inclusive);
+        ValidateBetween<Type>(options, array1, scalar2, scalar3);
+        ValidateBetween<Type>(options, array1, array2, scalar3);
+        ValidateBetween<Type>(options, array1, array2, array3);
+        ValidateBetween<Type>(options, array1, scalar2, scalar3);
+        ValidateBetween<Type>(options, scalar1, array2, array3);
+        ValidateBetween<Type>(options, scalar1, array2, scalar3);
+        ValidateBetween<Type>(options, scalar1, scalar2, array3);
+        ValidateBetween<Type>(options, array1, scalar2, array3);
+      }
+    }
+  }
+};
+
+TEST(TestNumericBetweenKernel, BetweenPrimitiveRandomTests) {
+  TestRandomPrimitiveCTypes<BetweenRandomNumeric>();
+}
+
+class TestStringBetweenKernel : public ::testing::Test {};
+
+TEST(TestStringBetweenKernel, RandomBetween) {
+  using ScalarType = typename TypeTraits<StringType>::ScalarType;
+
+  auto rand = random::RandomArrayGenerator(0x5416447);
+  for (size_t i = 3; i < 10; i++) {
+    for (auto null_probability : {0.0, 0.01, 0.1, 0.25, 0.5, 1.0}) {
+      for (auto inclusive :
+           {BetweenOptions::Inclusive::BOTH, BetweenOptions::Inclusive::LEFT,
+            BetweenOptions::Inclusive::RIGHT, BetweenOptions::Inclusive::NEITHER}) {
+        const int64_t length = static_cast<int64_t>(1ULL << i);
+        auto array1 = Datum(rand.String(length, 0, 16, null_probability));
+        auto array2 = Datum(rand.String(length, 0, 16, null_probability));
+        auto array3 = Datum(rand.String(length, 0, 16, null_probability));
+        auto scalar1 = Datum(std::make_shared<ScalarType>("fupi"));
+        auto scalar2 = Datum(std::make_shared<ScalarType>("tupu"));
+        auto scalar3 = Datum(std::make_shared<ScalarType>("zito"));
+        auto options = BetweenOptions(inclusive);
+        ValidateBetween<StringType>(options, array1, scalar2, scalar3);
+        ValidateBetween<StringType>(options, scalar1, array2, scalar3);
+        ValidateBetween<StringType>(options, scalar1, scalar2, array3);
+        ValidateBetween<StringType>(options, scalar1, array2, array3);
+        ValidateBetween<StringType>(options, array1, scalar2, array3);
+        ValidateBetween<StringType>(options, array1, array2, scalar3);
+        ValidateBetween<StringType>(options, array1, array2, array3);
+      }
+    }
+  }
+}
+
+TEST(TestStringBetweenKernel, StringArrayScalarScalarTest) {
+  using ScalarType = typename TypeTraits<StringType>::ScalarType;
+  auto l = Datum(std::make_shared<ScalarType>("abc"));
+  auto r = Datum(std::make_shared<ScalarType>("zzz"));
+  BetweenOptions InclusiveOption(BetweenOptions::Inclusive::BOTH);
+  ValidateBetween<StringType>(
+      InclusiveOption, ArrayFromJSON(TypeTraits<StringType>::type_singleton(), "[]"), l,
+      r, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<StringType>(
+      InclusiveOption, ArrayFromJSON(TypeTraits<StringType>::type_singleton(), "[null]"),
+      l, r, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<StringType>(InclusiveOption,
+                              ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                            R"(["aaa", "aaaa", "ccc", "z"])"),
+                              l, r,
+                              ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                                            R"([false, false, true, true])"));
+  ValidateBetween<StringType>(InclusiveOption,
+                              ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                            R"(["a", "aaaa", "c", "z"])"),
+                              l, r,
+                              ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                                            R"([false, false, true, true])"));
+  ValidateBetween<StringType>(InclusiveOption,
+                              ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                            R"(["a", "aaaa", "fff", "zzzz"])"),
+                              l, r,
+                              ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                                            R"([false, false, true, false])"));
+  ValidateBetween<StringType>(InclusiveOption,
+                              ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                            R"(["abc", "baa", "fff", "zzz"])"),
+                              l, r,
+                              ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                                            R"([true, true, true, true])"));
+  ValidateBetween<StringType>(InclusiveOption,
+                              ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                            R"(["abd", null, null, "zzx"])"),
+                              l, r,
+                              ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                                            R"([true, null, null, true])"));
+}
+
+TEST(TestStringBetweenKernel, StringArrayArrayArrayTest) {
+  BetweenOptions InclusiveOption(BetweenOptions::Inclusive::BOTH);
+  ValidateBetween<StringType>(
+      InclusiveOption,
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                    R"(["david","hello","world"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["adam","hi","whirl"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                    R"(["robert","goeiemoreen","whirlwind"])"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[true, false, false]"));
+  ValidateBetween<StringType>(
+      InclusiveOption,
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["x","a","f"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["w","a","e"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["z","a","g"])"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[true, true, true]"));
+  ValidateBetween<StringType>(
+      InclusiveOption,
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                    R"(["block","bit","binary"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                    R"(["bit","nibble","ternary"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["word","d","xyz"])"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[true, false, false]"));
+  ValidateBetween<StringType>(
+      InclusiveOption,
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                    R"(["よしもと","の","ち"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["は","へ","あ"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["な","を","ち"])"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[false, false, true]"));
+  ValidateBetween<StringType>(
+      InclusiveOption,
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["A","ア","王"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["た","あ","歩"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["李","田",null])"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[false, true, null]"));
+  ValidateBetween<StringType>(
+      InclusiveOption,
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["Б",null,"Я"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["А","Ж","Щ"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["Д","Л","Ф"])"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[true, null, false]"));
+}
+
+TEST(TestTimestampsBetweenKernel, TimestampsArrayArrayArrayTest) {
+  const std::string arr_json = R"(["1970-01-01","2000-02-02","1900-02-28"])";
+  const std::string lhs_json = R"(["1970-01-01","2000-02-01","1900-02-28"])";
+  const std::string rhs_json = R"(["1970-01-02","2000-02-02","1900-02-28"])";
+  // Between Options
+  BetweenOptions both(BetweenOptions::Inclusive::BOTH);
+  BetweenOptions left(BetweenOptions::Inclusive::LEFT);
+  BetweenOptions right(BetweenOptions::Inclusive::RIGHT);
+  BetweenOptions neither(BetweenOptions::Inclusive::NEITHER);
+  // Same units should be fine
+  auto arr = ArrayFromJSON(timestamp(TimeUnit::SECOND), arr_json);
+  auto lhs = ArrayFromJSON(timestamp(TimeUnit::SECOND), lhs_json);
+  auto rhs = ArrayFromJSON(timestamp(TimeUnit::SECOND), rhs_json);
+  // BOTH
+  ValidateBetween<TimestampType>(both, arr, lhs, rhs);
+  // LEFT
+  ValidateBetween<TimestampType>(left, arr, lhs, rhs);
+  // RIGHT
+  ValidateBetween<TimestampType>(right, arr, lhs, rhs);
+  // NEITHER
+  ValidateBetween<TimestampType>(neither, arr, lhs, rhs);
+/*  // So are same time zones
+  arr = ArrayFromJSON(timestamp(TimeUnit::SECOND, "America/Chicago"), arr_json);
+  lhs = ArrayFromJSON(timestamp(TimeUnit::SECOND, "America/Chicago"), lhs_json);
+  rhs = ArrayFromJSON(timestamp(TimeUnit::SECOND, "America/Chicago"), rhs_json);
+  // BOTH
+  ValidateBetween<TimestampType>(both, arr, lhs, rhs);
+  // LEFT
+  ValidateBetween<TimestampType>(left, arr, lhs, rhs);
+  // RIGHT
+  ValidateBetween<TimestampType>(right, arr, lhs, rhs);
+  // NEITHER
+  ValidateBetween<TimestampType>(neither, arr, lhs, rhs);
+  // Different units should be fine
+  auto arr = ArrayFromJSON(timestamp(TimeUnit::SECOND), arr_json);
+  auto lhs = ArrayFromJSON(timestamp(TimeUnit::MILLI), lhs_json);
+  auto rhs = ArrayFromJSON(timestamp(TimeUnit::SECOND), rhs_json);
+  // BOTH

Review comment:
       Updating it




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

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

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



[GitHub] [arrow] bkmgit commented on pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
bkmgit commented on pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#issuecomment-1013735121


   @github-actions crossbow submit autotune


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

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

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



[GitHub] [arrow] bkmgit commented on a change in pull request #11882: ARROW-9843: [C++][Python] Implement Between ternary kernel and Python bindings

Posted by GitBox <gi...@apache.org>.
bkmgit commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r787036818



##########
File path: cpp/src/arrow/compute/kernels/scalar_compare_test.cc
##########
@@ -1865,5 +1867,683 @@ TEST(TestMaxElementWiseMinElementWise, CommonTemporal) {
               ResultWith(ScalarFromJSON(date64(), "86400000")));
 }
 
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const Datum& val, const Datum& lhs,
+                            const Datum& rhs, const Datum& expected) {
+  ASSERT_OK_AND_ASSIGN(Datum result, Between(val, lhs, rhs, options, nullptr));
+  AssertArraysEqual(*expected.make_array(), *result.make_array(),
+                    /*verbose=*/true);
+}
+
+template <typename ArrowType>
+void ValidateBetween(BetweenOptions options, const Datum& val, const Datum& lhs,
+                     const Datum& rhs) {
+  CompareOperator lhs_val;
+  CompareOperator val_rhs;
+  BetweenOptions::Inclusive include_endpoints = options.inclusive;
+
+  if (include_endpoints == BetweenOptions::Inclusive::NEITHER) {
+    lhs_val = LESS;
+    val_rhs = LESS;
+  } else if (include_endpoints == BetweenOptions::Inclusive::LEFT) {
+    lhs_val = LESS_EQUAL;
+    val_rhs = LESS;
+  } else if (include_endpoints == BetweenOptions::Inclusive::RIGHT) {
+    lhs_val = LESS;
+    val_rhs = LESS_EQUAL;
+  } else {
+    lhs_val = LESS_EQUAL;
+    val_rhs = LESS_EQUAL;
+  }
+
+  ASSERT_OK_AND_ASSIGN(Datum resultl,
+                       CallFunction(CompareOperatorToFunctionName(lhs_val), {lhs, val}));
+  ASSERT_OK_AND_ASSIGN(Datum resultr,
+                       CallFunction(CompareOperatorToFunctionName(val_rhs), {val, rhs}));
+  ASSERT_OK_AND_ASSIGN(Datum expected, CallFunction("and", {resultl, resultr}));
+
+  ValidateBetween<ArrowType>(options, val, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+class TestNumericBetweenKernel : public ::testing::Test {};
+
+TYPED_TEST_SUITE(TestNumericBetweenKernel, NumericArrowTypes);
+TYPED_TEST(TestNumericBetweenKernel, SimpleBetweenArrayScalarScalar) {
+  using ScalarType = typename TypeTraits<TypeParam>::ScalarType;
+  using CType = typename TypeTraits<TypeParam>::CType;
+
+  Datum zero(std::make_shared<ScalarType>(CType(0)));
+  Datum four(std::make_shared<ScalarType>(CType(4)));
+  Datum null(std::make_shared<ScalarType>());
+  BetweenOptions InclusiveBoth(BetweenOptions::Inclusive::BOTH);
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"), zero,
+      four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      zero, four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,1,2,2]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[1,1,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,3,4,5]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[1,1,1,1,1,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,1,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null,0,1,1]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), null, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, null,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  BetweenOptions InclusiveLeft(BetweenOptions::Inclusive::LEFT);
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"), zero,
+      four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      zero, four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,1,2,2]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[1,1,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,3,4,5]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[1,1,1,1,0,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,0,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null,0,1,1]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), null, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, null,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  BetweenOptions InclusiveRight(BetweenOptions::Inclusive::RIGHT);
+  ValidateBetween<TypeParam>(
+      InclusiveRight, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"), zero,
+      four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      zero, four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,1,2,2]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,0,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,3,4,5]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,1,1,1,1,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,1,1,1,1,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null,0,1,1]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null,0,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), null, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, null,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  BetweenOptions InclusiveNeither(BetweenOptions::Inclusive::NEITHER);
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      zero, four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      zero, four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,1,2,2]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,0,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,3,4,5]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,1,1,1,0,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,0,1,1,1,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null,0,1,1]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null,0,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), null, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, null,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+}
+
+TYPED_TEST(TestNumericBetweenKernel, SimpleBetweenScalarArrayArray) {
+  using ScalarType = typename TypeTraits<TypeParam>::ScalarType;
+  using CType = typename TypeTraits<TypeParam>::CType;
+
+  Datum one(std::make_shared<ScalarType>(CType(1)));
+
+  BetweenOptions InclusiveBoth(BetweenOptions::Inclusive::BOTH);
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, one, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,true,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,true,null,false,false]"));
+  BetweenOptions InclusiveLeft(BetweenOptions::Inclusive::LEFT);
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, one, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,true,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,true,null,false,false]"));
+  BetweenOptions InclusiveRight(BetweenOptions::Inclusive::RIGHT);
+  ValidateBetween<TypeParam>(
+      InclusiveRight, one, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,false,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,false,null,false,false]"));
+  BetweenOptions InclusiveNeither(BetweenOptions::Inclusive::NEITHER);
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, one, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,false,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,false,null,false,false]"));
+}
+
+TYPED_TEST(TestNumericBetweenKernel, SimpleBetweenArrayArrayArray) {
+  BetweenOptions InclusiveBoth(BetweenOptions::Inclusive::BOTH);
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[1,1,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,true,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,true,null,false,false]"));
+  BetweenOptions InclusiveLeft(BetweenOptions::Inclusive::LEFT);
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[1,1,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,false,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,false,null,false,false]"));
+  BetweenOptions InclusiveRight(BetweenOptions::Inclusive::RIGHT);
+  ValidateBetween<TypeParam>(
+      InclusiveRight, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[1,1,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,true,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,true,null,false,false]"));
+  BetweenOptions InclusiveNeither(BetweenOptions::Inclusive::NEITHER);
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[1,1,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,false,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,false,null,false,false]"));
+}
+
+template <typename Type>
+struct BetweenRandomNumeric {
+  static void Test(const std::shared_ptr<DataType>& type) {
+    using ScalarType = typename TypeTraits<Type>::ScalarType;
+    using CType = typename TypeTraits<Type>::CType;
+    auto rand = random::RandomArrayGenerator(0x5416447);
+    const int64_t length = 1000;

Review comment:
       Ok. Also changed CompareRandomNumeric to 1000.




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

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

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



[GitHub] [arrow] pitrou commented on a change in pull request #11882: ARROW-9843: [C++][Python] Implement Between ternary kernel and Python bindings

Posted by GitBox <gi...@apache.org>.
pitrou commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r786018651



##########
File path: cpp/src/arrow/compute/api_scalar.h
##########
@@ -342,6 +342,17 @@ struct ARROW_EXPORT CompareOptions {
   enum CompareOperator op;
 };
 
+class ARROW_EXPORT BetweenOptions : public FunctionOptions {
+ public:
+  enum Inclusive { BOTH, LEFT, RIGHT, NEITHER };
+  explicit BetweenOptions(Inclusive inclusive = BOTH);
+  static BetweenOptions Defaults() { return BetweenOptions(); }
+  constexpr static char const kTypeName[] = "BetweenOptions";
+
+  /// Inclusive option to apply

Review comment:
       Can you make this more explanatory?

##########
File path: cpp/src/arrow/compute/kernels/scalar_compare.cc
##########
@@ -156,39 +212,52 @@ struct Maximum {
   }
 };
 
+// Check if timestamp timezones are comparable (either all are empty or none is).
+Status CheckCompareTimestamps(const ExecBatch& batch) {
+  if (batch.num_values() > 0) {
+    for (int i = 0; i < batch.num_values() - 1; ++i) {
+      const auto& tsi = checked_cast<const TimestampType&>(*batch[i].type());
+      for (int j = i + 1; j < batch.num_values(); ++j) {

Review comment:
       Why are there two nested loops? This shouldn't be necessary, e.g.:
   ```c++
     if (batch.num_values() > 0) {
       const bool has_no_timezone = checked_cast<const TimestampType&>(*batch[0].type()).timezone().empty();
       for (int i = 1; i < batch.num_values(); ++i) {
         if (has_no_timezone != checked_cast<const TimestampType&>(*batch[i].type()).timezone().empty()) {
           ...
   ```

##########
File path: cpp/src/arrow/compute/kernels/scalar_compare.cc
##########
@@ -772,6 +1107,16 @@ const FunctionDoc less_equal_doc{
     ("A null on either side emits a null comparison result."),
     {"x", "y"}};
 
+const FunctionDoc between_doc{
+    "Check if values are in the given range, val between a and b",

Review comment:
       This seems to say the same thing in two different ways.
   Perhaps simply "Check if values are in the given range"?

##########
File path: cpp/src/arrow/compute/kernels/scalar_compare.cc
##########
@@ -746,6 +808,279 @@ std::shared_ptr<ScalarFunction> MakeScalarMinMax(std::string name,
   return func;
 }
 
+template <template <BetweenOptions::Inclusive> class Op>
+std::shared_ptr<ScalarFunction> MakeBetweenFunction(std::string name,
+                                                    const FunctionDoc* doc) {
+  using BetweenState = OptionsWrapper<BetweenOptions>;
+  static const auto kDefaultOptions = BetweenOptions::Defaults();
+  auto func =
+      std::make_shared<CompareFunction>(name, Arity::Ternary(), doc, &kDefaultOptions);
+
+  // Add kernels for physical numeric types, temporal types done separately
+  for (const auto& types : {DurationTypes(), IntervalTypes(), NumericTypes()}) {
+    for (const auto& ty : types) {
+      auto type_id = ty->id();
+      auto exec = [type_id](KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+        // Resolve generator based on options
+        const auto& state = static_cast<const BetweenState&>(*ctx->state());
+        switch (state.options.inclusive) {
+          case BetweenOptions::Inclusive::BOTH:
+            return GeneratePhysicalNumeric<ScalarTernaryEqualTypes, BooleanType,
+                                           Op<BetweenOptions::Inclusive::BOTH>>(type_id)(
+                ctx, batch, out);
+          case BetweenOptions::Inclusive::LEFT:
+            return GeneratePhysicalNumeric<ScalarTernaryEqualTypes, BooleanType,
+                                           Op<BetweenOptions::Inclusive::LEFT>>(type_id)(
+                ctx, batch, out);
+          case BetweenOptions::Inclusive::RIGHT:
+            return GeneratePhysicalNumeric<ScalarTernaryEqualTypes, BooleanType,
+                                           Op<BetweenOptions::Inclusive::RIGHT>>(type_id)(
+                ctx, batch, out);
+          case BetweenOptions::Inclusive::NEITHER:
+            return GeneratePhysicalNumeric<ScalarTernaryEqualTypes, BooleanType,
+                                           Op<BetweenOptions::Inclusive::NEITHER>>(
+                type_id)(ctx, batch, out);
+          default:
+            return Status::NotImplemented("between inclusiveness not implemented: ",
+                                          state.options.ToString());
+        }
+      };
+      DCHECK_OK(func->AddKernel({ty, ty, ty}, boolean(), exec, BetweenState::Init));
+    }
+  }
+
+  // Add timestamp kernels
+  for (const auto unit : TimeUnit::values()) {
+    InputType in_type(match::TimestampTypeUnit(unit));
+    auto type_id = Type::TIMESTAMP;
+    auto exec = [type_id](KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+      // Validate timezones in all entries or in none
+      RETURN_NOT_OK(CheckCompareTimestamps(batch));
+      // Resolve generator based on options
+      const auto& state = static_cast<const BetweenState&>(*ctx->state());
+      switch (state.options.inclusive) {
+        case BetweenOptions::Inclusive::BOTH:
+          return GeneratePhysicalNumeric<ScalarTernaryEqualTypes, BooleanType,
+                                         Op<BetweenOptions::Inclusive::BOTH>>(type_id)(
+              ctx, batch, out);
+        case BetweenOptions::Inclusive::LEFT:
+          return GeneratePhysicalNumeric<ScalarTernaryEqualTypes, BooleanType,
+                                         Op<BetweenOptions::Inclusive::LEFT>>(type_id)(
+              ctx, batch, out);
+        case BetweenOptions::Inclusive::RIGHT:
+          return GeneratePhysicalNumeric<ScalarTernaryEqualTypes, BooleanType,
+                                         Op<BetweenOptions::Inclusive::RIGHT>>(type_id)(
+              ctx, batch, out);
+        case BetweenOptions::Inclusive::NEITHER:
+          return GeneratePhysicalNumeric<ScalarTernaryEqualTypes, BooleanType,
+                                         Op<BetweenOptions::Inclusive::NEITHER>>(type_id)(
+              ctx, batch, out);
+        default:
+          return Status::NotImplemented("between inclusiveness not implemented: ",
+                                        state.options.ToString());
+      }
+    };
+    DCHECK_OK(func->AddKernel({in_type, in_type, in_type}, boolean(), exec,
+                              BetweenState::Init));
+  }
+
+  // Add time kernels
+  for (const auto unit : {TimeUnit::SECOND, TimeUnit::MILLI}) {
+    auto type_id = Type::TIME32;
+    auto exec = [type_id](KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+      // Resolve generator based on options
+      const auto& state = static_cast<const BetweenState&>(*ctx->state());
+      switch (state.options.inclusive) {
+        case BetweenOptions::Inclusive::BOTH:
+          return GeneratePhysicalNumeric<ScalarTernaryEqualTypes, BooleanType,
+                                         Op<BetweenOptions::Inclusive::BOTH>>(type_id)(
+              ctx, batch, out);
+        case BetweenOptions::Inclusive::LEFT:
+          return GeneratePhysicalNumeric<ScalarTernaryEqualTypes, BooleanType,
+                                         Op<BetweenOptions::Inclusive::LEFT>>(type_id)(
+              ctx, batch, out);
+        case BetweenOptions::Inclusive::RIGHT:
+          return GeneratePhysicalNumeric<ScalarTernaryEqualTypes, BooleanType,
+                                         Op<BetweenOptions::Inclusive::RIGHT>>(type_id)(
+              ctx, batch, out);
+        case BetweenOptions::Inclusive::NEITHER:
+          return GeneratePhysicalNumeric<ScalarTernaryEqualTypes, BooleanType,
+                                         Op<BetweenOptions::Inclusive::NEITHER>>(type_id)(
+              ctx, batch, out);
+        default:
+          return Status::NotImplemented("between inclusiveness not implemented: ",
+                                        state.options.ToString());
+      }
+    };
+    InputType in_type(match::Time32TypeUnit(unit));
+    DCHECK_OK(func->AddKernel({in_type, in_type, in_type}, boolean(), exec,
+                              BetweenState::Init));
+  }
+
+  for (const auto& unit : {TimeUnit::MICRO, TimeUnit::NANO}) {
+    auto type_id = Type::TIME64;
+    auto exec = [type_id](KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+      // Resolve generator based on options
+      const auto& state = static_cast<const BetweenState&>(*ctx->state());
+      switch (state.options.inclusive) {
+        case BetweenOptions::Inclusive::BOTH:
+          return GeneratePhysicalNumeric<ScalarTernaryEqualTypes, BooleanType,
+                                         Op<BetweenOptions::Inclusive::BOTH>>(type_id)(
+              ctx, batch, out);
+        case BetweenOptions::Inclusive::LEFT:
+          return GeneratePhysicalNumeric<ScalarTernaryEqualTypes, BooleanType,
+                                         Op<BetweenOptions::Inclusive::LEFT>>(type_id)(
+              ctx, batch, out);
+        case BetweenOptions::Inclusive::RIGHT:
+          return GeneratePhysicalNumeric<ScalarTernaryEqualTypes, BooleanType,
+                                         Op<BetweenOptions::Inclusive::RIGHT>>(type_id)(
+              ctx, batch, out);
+        case BetweenOptions::Inclusive::NEITHER:
+          return GeneratePhysicalNumeric<ScalarTernaryEqualTypes, BooleanType,
+                                         Op<BetweenOptions::Inclusive::NEITHER>>(type_id)(
+              ctx, batch, out);
+        default:
+          return Status::NotImplemented("between inclusiveness not implemented: ",
+                                        state.options.ToString());
+      }
+    };
+    InputType in_type(match::Time64TypeUnit(unit));
+    DCHECK_OK(func->AddKernel({in_type, in_type, in_type}, boolean(), exec,
+                              BetweenState::Init));
+  }
+
+  // Add kernels for base binary types
+  for (const auto& ty : BaseBinaryTypes()) {
+    auto type_id = ty->id();
+    auto exec = [type_id](KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+      // Resolve generator based on options
+      const auto& state = static_cast<const BetweenState&>(*ctx->state());
+      switch (state.options.inclusive) {
+        case BetweenOptions::Inclusive::BOTH:
+          return GenerateVarBinaryBase<ScalarTernaryEqualTypes, BooleanType,
+                                       Op<BetweenOptions::Inclusive::BOTH>>(type_id)(
+              ctx, batch, out);
+        case BetweenOptions::Inclusive::LEFT:
+          return GenerateVarBinaryBase<ScalarTernaryEqualTypes, BooleanType,
+                                       Op<BetweenOptions::Inclusive::LEFT>>(type_id)(
+              ctx, batch, out);
+        case BetweenOptions::Inclusive::RIGHT:
+          return GenerateVarBinaryBase<ScalarTernaryEqualTypes, BooleanType,
+                                       Op<BetweenOptions::Inclusive::RIGHT>>(type_id)(
+              ctx, batch, out);
+        case BetweenOptions::Inclusive::NEITHER:
+          return GenerateVarBinaryBase<ScalarTernaryEqualTypes, BooleanType,
+                                       Op<BetweenOptions::Inclusive::NEITHER>>(type_id)(
+              ctx, batch, out);
+        default:
+          return Status::NotImplemented("between inclusiveness not implemented: ",
+                                        state.options.ToString());
+      }
+    };
+    DCHECK_OK(func->AddKernel({ty, ty, ty}, boolean(), exec, BetweenState::Init));
+  }
+
+  // Add kernels for decimal types
+  for (const auto type_id : {Type::DECIMAL128, Type::DECIMAL256}) {
+    auto exec = [type_id](KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+      // Resolve generator based on options
+      const auto& state = static_cast<const BetweenState&>(*ctx->state());
+      switch (state.options.inclusive) {
+        case BetweenOptions::Inclusive::BOTH:
+          return GenerateDecimal<ScalarTernaryEqualTypes, BooleanType,
+                                 Op<BetweenOptions::Inclusive::BOTH>>(type_id)(ctx, batch,
+                                                                               out);
+        case BetweenOptions::Inclusive::LEFT:
+          return GenerateDecimal<ScalarTernaryEqualTypes, BooleanType,
+                                 Op<BetweenOptions::Inclusive::LEFT>>(type_id)(ctx, batch,
+                                                                               out);
+        case BetweenOptions::Inclusive::RIGHT:
+          return GenerateDecimal<ScalarTernaryEqualTypes, BooleanType,
+                                 Op<BetweenOptions::Inclusive::RIGHT>>(type_id)(
+              ctx, batch, out);
+        case BetweenOptions::Inclusive::NEITHER:
+          return GenerateDecimal<ScalarTernaryEqualTypes, BooleanType,
+                                 Op<BetweenOptions::Inclusive::NEITHER>>(type_id)(
+              ctx, batch, out);
+        default:
+          return Status::NotImplemented("between inclusiveness not implemented: ",
+                                        state.options.ToString());
+      }
+    };
+    InputType ty(type_id);
+    DCHECK_OK(func->AddKernel({ty, ty, ty}, boolean(), exec, BetweenState::Init));
+  }
+
+  // Add kernels for date types
+  for (const auto type_id : {Type::DATE32, Type::DATE64}) {
+    auto exec = [type_id](KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+      // Resolve generator based on options
+      const auto& state = static_cast<const BetweenState&>(*ctx->state());
+      switch (state.options.inclusive) {
+        case BetweenOptions::Inclusive::BOTH:
+          return GeneratePhysicalNumeric<ScalarTernaryEqualTypes, BooleanType,
+                                         Op<BetweenOptions::Inclusive::BOTH>>(type_id)(
+              ctx, batch, out);
+        case BetweenOptions::Inclusive::LEFT:
+          return GeneratePhysicalNumeric<ScalarTernaryEqualTypes, BooleanType,
+                                         Op<BetweenOptions::Inclusive::LEFT>>(type_id)(
+              ctx, batch, out);
+        case BetweenOptions::Inclusive::RIGHT:
+          return GeneratePhysicalNumeric<ScalarTernaryEqualTypes, BooleanType,
+                                         Op<BetweenOptions::Inclusive::RIGHT>>(type_id)(
+              ctx, batch, out);
+        case BetweenOptions::Inclusive::NEITHER:
+          return GeneratePhysicalNumeric<ScalarTernaryEqualTypes, BooleanType,
+                                         Op<BetweenOptions::Inclusive::NEITHER>>(type_id)(
+              ctx, batch, out);
+        default:
+          return Status::NotImplemented("between inclusiveness not implemented: ",
+                                        state.options.ToString());
+      }
+    };

Review comment:
       There is a lot of repetition with all those `exec` lambdas, is it possible to factor them out somewhat?

##########
File path: cpp/src/arrow/compute/kernels/scalar_compare.cc
##########
@@ -156,39 +212,52 @@ struct Maximum {
   }
 };
 
+// Check if timestamp timezones are comparable (either all are empty or none is).
+Status CheckCompareTimestamps(const ExecBatch& batch) {
+  if (batch.num_values() > 0) {
+    for (int i = 0; i < batch.num_values() - 1; ++i) {
+      const auto& tsi = checked_cast<const TimestampType&>(*batch[i].type());
+      for (int j = i + 1; j < batch.num_values(); ++j) {
+        const auto& tsj = checked_cast<const TimestampType&>(*batch[j].type());
+        bool invalid_state = tsi.timezone().empty();
+        invalid_state ^= tsj.timezone().empty();
+        if (invalid_state) {
+          return Status::Invalid(

Review comment:
       Please make this a `TypeError`.

##########
File path: cpp/src/arrow/compute/kernels/scalar_compare.cc
##########
@@ -772,6 +1107,16 @@ const FunctionDoc less_equal_doc{
     ("A null on either side emits a null comparison result."),
     {"x", "y"}};
 
+const FunctionDoc between_doc{
+    "Check if values are in the given range, val between a and b",
+    ("A null on either side emits a null comparison result.\n"
+     "options are used to specify if the endpoints are\n"

Review comment:
       ```suggestion
        "Options are used to specify if the left and right endpoints are\n"
   ```

##########
File path: cpp/src/arrow/compute/kernels/scalar_compare.cc
##########
@@ -772,6 +1107,16 @@ const FunctionDoc less_equal_doc{
     ("A null on either side emits a null comparison result."),
     {"x", "y"}};
 
+const FunctionDoc between_doc{
+    "Check if values are in the given range, val between a and b",
+    ("A null on either side emits a null comparison result.\n"
+     "options are used to specify if the endpoints are\n"
+     "inclusive, possible values are NEITHER (a < val < b),\n"
+     "LEFT (a <= val < b), RIGHT (a < val <= b), and \n"
+     "BOTH (a <= val <= b). Default is BOTH."),
+    {"val", "a", "b"},

Review comment:
       Better: `{"values", "left", "right"}`

##########
File path: cpp/src/arrow/compute/kernels/codegen_internal.h
##########
@@ -1134,8 +1340,43 @@ ArrayKernelExec GeneratePhysicalInteger(detail::GetTypeId get_id) {
   }
 }
 
-template <template <typename... Args> class Generator, typename... Args>
+template <template <typename... Args> class Generator, typename Type0, typename... Args>
 ArrayKernelExec GeneratePhysicalNumeric(detail::GetTypeId get_id) {

Review comment:
       Can you add comments to these two functions, to explain how they differ?

##########
File path: cpp/src/arrow/compute/kernels/codegen_internal.h
##########
@@ -445,6 +445,29 @@ static void VisitTwoArrayValuesInline(const ArrayData& arr0, const ArrayData& ar
                         arr0.length, std::move(visit_valid), std::move(visit_null));
 }
 
+template <typename Arg0Type, typename Arg1Type, typename Arg2Type, typename VisitFunc,
+          typename NullFunc>
+static void VisitThreeArrayValuesInline(const ArrayData& arr0, const ArrayData& arr1,

Review comment:
       This doesn't seem used at all in this PR, which is it here?

##########
File path: cpp/src/arrow/compute/kernels/scalar_compare_test.cc
##########
@@ -1865,5 +1867,683 @@ TEST(TestMaxElementWiseMinElementWise, CommonTemporal) {
               ResultWith(ScalarFromJSON(date64(), "86400000")));
 }
 
+template <typename ArrowType>

Review comment:
       What is the template parameter for?

##########
File path: cpp/src/arrow/compute/kernels/scalar_compare_test.cc
##########
@@ -1865,5 +1867,683 @@ TEST(TestMaxElementWiseMinElementWise, CommonTemporal) {
               ResultWith(ScalarFromJSON(date64(), "86400000")));
 }
 
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const Datum& val, const Datum& lhs,
+                            const Datum& rhs, const Datum& expected) {
+  ASSERT_OK_AND_ASSIGN(Datum result, Between(val, lhs, rhs, options, nullptr));
+  AssertArraysEqual(*expected.make_array(), *result.make_array(),
+                    /*verbose=*/true);
+}
+
+template <typename ArrowType>

Review comment:
       Same here.

##########
File path: cpp/src/arrow/compute/kernels/scalar_compare_test.cc
##########
@@ -1865,5 +1867,683 @@ TEST(TestMaxElementWiseMinElementWise, CommonTemporal) {
               ResultWith(ScalarFromJSON(date64(), "86400000")));
 }
 
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const Datum& val, const Datum& lhs,
+                            const Datum& rhs, const Datum& expected) {
+  ASSERT_OK_AND_ASSIGN(Datum result, Between(val, lhs, rhs, options, nullptr));
+  AssertArraysEqual(*expected.make_array(), *result.make_array(),
+                    /*verbose=*/true);
+}
+
+template <typename ArrowType>
+void ValidateBetween(BetweenOptions options, const Datum& val, const Datum& lhs,
+                     const Datum& rhs) {
+  CompareOperator lhs_val;
+  CompareOperator val_rhs;
+  BetweenOptions::Inclusive include_endpoints = options.inclusive;
+
+  if (include_endpoints == BetweenOptions::Inclusive::NEITHER) {
+    lhs_val = LESS;
+    val_rhs = LESS;
+  } else if (include_endpoints == BetweenOptions::Inclusive::LEFT) {
+    lhs_val = LESS_EQUAL;
+    val_rhs = LESS;
+  } else if (include_endpoints == BetweenOptions::Inclusive::RIGHT) {
+    lhs_val = LESS;
+    val_rhs = LESS_EQUAL;
+  } else {
+    lhs_val = LESS_EQUAL;
+    val_rhs = LESS_EQUAL;
+  }
+
+  ASSERT_OK_AND_ASSIGN(Datum resultl,
+                       CallFunction(CompareOperatorToFunctionName(lhs_val), {lhs, val}));
+  ASSERT_OK_AND_ASSIGN(Datum resultr,
+                       CallFunction(CompareOperatorToFunctionName(val_rhs), {val, rhs}));
+  ASSERT_OK_AND_ASSIGN(Datum expected, CallFunction("and", {resultl, resultr}));
+
+  ValidateBetween<ArrowType>(options, val, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+class TestNumericBetweenKernel : public ::testing::Test {};
+
+TYPED_TEST_SUITE(TestNumericBetweenKernel, NumericArrowTypes);
+TYPED_TEST(TestNumericBetweenKernel, SimpleBetweenArrayScalarScalar) {

Review comment:
       Is type parametrization required here? The only place where it seems used is to instantiate the scalars, but you could use `ScalarFromJSON`. Removing type parametrization would help keep compile times under control.

##########
File path: cpp/src/arrow/compute/kernels/scalar_compare_test.cc
##########
@@ -1865,5 +1867,683 @@ TEST(TestMaxElementWiseMinElementWise, CommonTemporal) {
               ResultWith(ScalarFromJSON(date64(), "86400000")));
 }
 
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const Datum& val, const Datum& lhs,
+                            const Datum& rhs, const Datum& expected) {
+  ASSERT_OK_AND_ASSIGN(Datum result, Between(val, lhs, rhs, options, nullptr));
+  AssertArraysEqual(*expected.make_array(), *result.make_array(),
+                    /*verbose=*/true);
+}
+
+template <typename ArrowType>
+void ValidateBetween(BetweenOptions options, const Datum& val, const Datum& lhs,
+                     const Datum& rhs) {
+  CompareOperator lhs_val;
+  CompareOperator val_rhs;
+  BetweenOptions::Inclusive include_endpoints = options.inclusive;
+
+  if (include_endpoints == BetweenOptions::Inclusive::NEITHER) {
+    lhs_val = LESS;
+    val_rhs = LESS;
+  } else if (include_endpoints == BetweenOptions::Inclusive::LEFT) {
+    lhs_val = LESS_EQUAL;
+    val_rhs = LESS;
+  } else if (include_endpoints == BetweenOptions::Inclusive::RIGHT) {
+    lhs_val = LESS;
+    val_rhs = LESS_EQUAL;
+  } else {
+    lhs_val = LESS_EQUAL;
+    val_rhs = LESS_EQUAL;
+  }
+
+  ASSERT_OK_AND_ASSIGN(Datum resultl,
+                       CallFunction(CompareOperatorToFunctionName(lhs_val), {lhs, val}));
+  ASSERT_OK_AND_ASSIGN(Datum resultr,
+                       CallFunction(CompareOperatorToFunctionName(val_rhs), {val, rhs}));
+  ASSERT_OK_AND_ASSIGN(Datum expected, CallFunction("and", {resultl, resultr}));
+
+  ValidateBetween<ArrowType>(options, val, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+class TestNumericBetweenKernel : public ::testing::Test {};
+
+TYPED_TEST_SUITE(TestNumericBetweenKernel, NumericArrowTypes);
+TYPED_TEST(TestNumericBetweenKernel, SimpleBetweenArrayScalarScalar) {
+  using ScalarType = typename TypeTraits<TypeParam>::ScalarType;
+  using CType = typename TypeTraits<TypeParam>::CType;
+
+  Datum zero(std::make_shared<ScalarType>(CType(0)));
+  Datum four(std::make_shared<ScalarType>(CType(4)));
+  Datum null(std::make_shared<ScalarType>());
+  BetweenOptions InclusiveBoth(BetweenOptions::Inclusive::BOTH);
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"), zero,
+      four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      zero, four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,1,2,2]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[1,1,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,3,4,5]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[1,1,1,1,1,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,1,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null,0,1,1]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), null, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, null,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  BetweenOptions InclusiveLeft(BetweenOptions::Inclusive::LEFT);
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"), zero,
+      four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      zero, four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,1,2,2]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[1,1,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,3,4,5]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[1,1,1,1,0,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,0,1,1,1,1]"));

Review comment:
       Same question here: why write all those tests out by hand?

##########
File path: cpp/src/arrow/compute/kernels/scalar_compare_test.cc
##########
@@ -1865,5 +1867,683 @@ TEST(TestMaxElementWiseMinElementWise, CommonTemporal) {
               ResultWith(ScalarFromJSON(date64(), "86400000")));
 }
 
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const Datum& val, const Datum& lhs,
+                            const Datum& rhs, const Datum& expected) {
+  ASSERT_OK_AND_ASSIGN(Datum result, Between(val, lhs, rhs, options, nullptr));
+  AssertArraysEqual(*expected.make_array(), *result.make_array(),
+                    /*verbose=*/true);
+}
+
+template <typename ArrowType>
+void ValidateBetween(BetweenOptions options, const Datum& val, const Datum& lhs,
+                     const Datum& rhs) {
+  CompareOperator lhs_val;
+  CompareOperator val_rhs;
+  BetweenOptions::Inclusive include_endpoints = options.inclusive;
+
+  if (include_endpoints == BetweenOptions::Inclusive::NEITHER) {

Review comment:
       You could even validate all `Inclusive` values at once in this function, no? And you can use `ARROW_SCOPED_TRACE` to add context to error messages.

##########
File path: cpp/src/arrow/compute/kernels/scalar_compare_test.cc
##########
@@ -1865,5 +1867,683 @@ TEST(TestMaxElementWiseMinElementWise, CommonTemporal) {
               ResultWith(ScalarFromJSON(date64(), "86400000")));
 }
 
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const Datum& val, const Datum& lhs,
+                            const Datum& rhs, const Datum& expected) {
+  ASSERT_OK_AND_ASSIGN(Datum result, Between(val, lhs, rhs, options, nullptr));
+  AssertArraysEqual(*expected.make_array(), *result.make_array(),
+                    /*verbose=*/true);
+}
+
+template <typename ArrowType>
+void ValidateBetween(BetweenOptions options, const Datum& val, const Datum& lhs,
+                     const Datum& rhs) {
+  CompareOperator lhs_val;
+  CompareOperator val_rhs;
+  BetweenOptions::Inclusive include_endpoints = options.inclusive;
+
+  if (include_endpoints == BetweenOptions::Inclusive::NEITHER) {
+    lhs_val = LESS;
+    val_rhs = LESS;
+  } else if (include_endpoints == BetweenOptions::Inclusive::LEFT) {
+    lhs_val = LESS_EQUAL;
+    val_rhs = LESS;
+  } else if (include_endpoints == BetweenOptions::Inclusive::RIGHT) {
+    lhs_val = LESS;
+    val_rhs = LESS_EQUAL;
+  } else {
+    lhs_val = LESS_EQUAL;
+    val_rhs = LESS_EQUAL;
+  }
+
+  ASSERT_OK_AND_ASSIGN(Datum resultl,
+                       CallFunction(CompareOperatorToFunctionName(lhs_val), {lhs, val}));
+  ASSERT_OK_AND_ASSIGN(Datum resultr,
+                       CallFunction(CompareOperatorToFunctionName(val_rhs), {val, rhs}));
+  ASSERT_OK_AND_ASSIGN(Datum expected, CallFunction("and", {resultl, resultr}));
+
+  ValidateBetween<ArrowType>(options, val, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+class TestNumericBetweenKernel : public ::testing::Test {};
+
+TYPED_TEST_SUITE(TestNumericBetweenKernel, NumericArrowTypes);
+TYPED_TEST(TestNumericBetweenKernel, SimpleBetweenArrayScalarScalar) {
+  using ScalarType = typename TypeTraits<TypeParam>::ScalarType;
+  using CType = typename TypeTraits<TypeParam>::CType;
+
+  Datum zero(std::make_shared<ScalarType>(CType(0)));
+  Datum four(std::make_shared<ScalarType>(CType(4)));
+  Datum null(std::make_shared<ScalarType>());
+  BetweenOptions InclusiveBoth(BetweenOptions::Inclusive::BOTH);
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"), zero,
+      four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      zero, four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,1,2,2]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[1,1,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,3,4,5]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[1,1,1,1,1,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,1,1,1,1,1]"));

Review comment:
       There doesn't seem to be any point in writing those three tests out. One of them should suffice.

##########
File path: cpp/src/arrow/util/bit_block_counter.h
##########
@@ -422,6 +426,96 @@ class ARROW_EXPORT OptionalBinaryBitBlockCounter {
   }
 };
 
+/// \brief A class that computes popcounts on the result of bitwise operations
+/// between three bitmaps, 64 bits at a time. A 64-bit word is loaded from each
+/// bitmap, then the popcount is computed on e.g. the bitwise-and of the three
+/// words.
+class ARROW_EXPORT TernaryBitBlockCounter {

Review comment:
       This isn't used in the PR, is it?

##########
File path: docs/source/cpp/compute.rst
##########
@@ -669,35 +669,43 @@ Decimal values are accepted, but are cast to Float64 first.
 Comparisons
 ~~~~~~~~~~~
 
-These functions expect two inputs of numeric type (in which case they will be
+These functions expect two or three inputs of numeric type (in which case they will be
 cast to the :ref:`common numeric type <common-numeric-type>` before comparison),
-or two inputs of Binary- or String-like types, or two inputs of Temporal types.
-If any input is dictionary encoded it will be expanded for the purposes of
-comparison. If any of the input elements in a pair is null, the corresponding
+or two or three inputs of Binary- or String-like types, or two or three inputs of Temporal 
+types. If any input is dictionary encoded it will be expanded for the purposes of
+comparison. If any of the input elements in a pair or triple is null, the corresponding
 output element is null. Decimal arguments will be promoted in the same way as
 for ``add`` and ``subtract``.
 
-+----------------+------------+---------------------------------------------+---------------------+
-| Function names | Arity      | Input types                                 | Output type         |
-+================+============+=============================================+=====================+
-| equal          | Binary     | Numeric, Temporal, Binary- and String-like  | Boolean             |
-+----------------+------------+---------------------------------------------+---------------------+
-| greater        | Binary     | Numeric, Temporal, Binary- and String-like  | Boolean             |
-+----------------+------------+---------------------------------------------+---------------------+
-| greater_equal  | Binary     | Numeric, Temporal, Binary- and String-like  | Boolean             |
-+----------------+------------+---------------------------------------------+---------------------+
-| less           | Binary     | Numeric, Temporal, Binary- and String-like  | Boolean             |
-+----------------+------------+---------------------------------------------+---------------------+
-| less_equal     | Binary     | Numeric, Temporal, Binary- and String-like  | Boolean             |
-+----------------+------------+---------------------------------------------+---------------------+
-| not_equal      | Binary     | Numeric, Temporal, Binary- and String-like  | Boolean             |
-+----------------+------------+---------------------------------------------+---------------------+
++----------------+---------+---------------------------------------------+-------------+--------------------------+-------+
+| Function names | Arity   | Input types                                 | Output type | Options Class            | Notes |
++================+=========+=============================================+=============+==========================+=======+
+| between        | Ternary | Numeric, Temporal, Binary- and String-like  | Boolean     | :struct:`BetweenOptions` | \(1)  |
++----------------+---------+---------------------------------------------+-------------+--------------------------+-------+
+| equal          | Binary  | Numeric, Temporal, Binary- and String-like  | Boolean     |                          |       |
++----------------+---------+---------------------------------------------+-------------+--------------------------+-------+
+| greater        | Binary  | Numeric, Temporal, Binary- and String-like  | Boolean     |                          |       |
++----------------+---------+---------------------------------------------+-------------+--------------------------+-------+
+| greater_equal  | Binary  | Numeric, Temporal, Binary- and String-like  | Boolean     |                          |       |
++----------------+---------+---------------------------------------------+-------------+--------------------------+-------+
+| less           | Binary  | Numeric, Temporal, Binary- and String-like  | Boolean     |                          |       |
++----------------+---------+---------------------------------------------+-------------+--------------------------+-------+
+| less_equal     | Binary  | Numeric, Temporal, Binary- and String-like  | Boolean     |                          |       |
++----------------+---------+---------------------------------------------+-------------+--------------------------+-------+
+| not_equal      | Binary  | Numeric, Temporal, Binary- and String-like  | Boolean     |                          |       |
++----------------+---------+---------------------------------------------+-------------+--------------------------+-------+
 
 These functions take any number of inputs of numeric type (in which case they
 will be cast to the :ref:`common numeric type <common-numeric-type>` before
 comparison) or of temporal types. If any input is dictionary encoded it will be
 expanded for the purposes of comparison.
 
+* \(1) The default is `BetweenOptions::Inclusive::BOTH`, other options are
+  `BetweenOptions::Inclusive::LEFT`, `BetweenOptions::Inclusive::RIGHT` and
+  `BetweenOptions::Inlcusive::NEITHER` corresponding to a <= value <= b,
+  a <= value < b, a < value <= b and a < value < b.  Strings are presently
+  compared by their UTF8 codepoint values.

Review comment:
       ```suggestion
   * \(1) Options are used to control whether either comparison endpoint is inclusive.
     The default is ``BetweenOptions::Inclusive::BOTH``, other possible value are
     ``BetweenOptions::Inclusive::LEFT``, ``BetweenOptions::Inclusive::RIGHT`` and
     ``BetweenOptions::Inlcusive::NEITHER`` corresponding to ``a <= value <= b``,
     ``a <= value < b``, ``a < value <= b`` and ``a < value < b`` (respectively).
     Strings are presently compared by their UTF8 codepoint values.
   ```

##########
File path: docs/source/cpp/compute.rst
##########
@@ -669,35 +669,43 @@ Decimal values are accepted, but are cast to Float64 first.
 Comparisons
 ~~~~~~~~~~~
 
-These functions expect two inputs of numeric type (in which case they will be
+These functions expect two or three inputs of numeric type (in which case they will be
 cast to the :ref:`common numeric type <common-numeric-type>` before comparison),
-or two inputs of Binary- or String-like types, or two inputs of Temporal types.
-If any input is dictionary encoded it will be expanded for the purposes of
-comparison. If any of the input elements in a pair is null, the corresponding
+or two or three inputs of Binary- or String-like types, or two or three inputs of Temporal 
+types. If any input is dictionary encoded it will be expanded for the purposes of
+comparison. If any of the input elements in a pair or triple is null, the corresponding
 output element is null. Decimal arguments will be promoted in the same way as
 for ``add`` and ``subtract``.
 
-+----------------+------------+---------------------------------------------+---------------------+
-| Function names | Arity      | Input types                                 | Output type         |
-+================+============+=============================================+=====================+
-| equal          | Binary     | Numeric, Temporal, Binary- and String-like  | Boolean             |
-+----------------+------------+---------------------------------------------+---------------------+
-| greater        | Binary     | Numeric, Temporal, Binary- and String-like  | Boolean             |
-+----------------+------------+---------------------------------------------+---------------------+
-| greater_equal  | Binary     | Numeric, Temporal, Binary- and String-like  | Boolean             |
-+----------------+------------+---------------------------------------------+---------------------+
-| less           | Binary     | Numeric, Temporal, Binary- and String-like  | Boolean             |
-+----------------+------------+---------------------------------------------+---------------------+
-| less_equal     | Binary     | Numeric, Temporal, Binary- and String-like  | Boolean             |
-+----------------+------------+---------------------------------------------+---------------------+
-| not_equal      | Binary     | Numeric, Temporal, Binary- and String-like  | Boolean             |
-+----------------+------------+---------------------------------------------+---------------------+
++----------------+---------+---------------------------------------------+-------------+--------------------------+-------+
+| Function names | Arity   | Input types                                 | Output type | Options Class            | Notes |

Review comment:
       ```suggestion
   | Function names | Arity   | Input types                                 | Output type | Options class            | Notes |
   ```

##########
File path: python/pyarrow/_compute.pyx
##########
@@ -785,6 +785,40 @@ class ElementWiseAggregateOptions(_ElementWiseAggregateOptions):
         self._set_options(skip_nulls)
 
 
+cdef CInclusive unwrap_inclusive(inclusive) except *:
+    if inclusive == "both":
+        return CInclusive_BOTH
+    elif inclusive == "left":
+        return CInclusive_LEFT
+    elif inclusive == "right":
+        return CInclusive_RIGHT
+    elif inclusive == "neither":
+        return CInclusive_NEITHER
+    _raise_invalid_function_option(inclusive, "inclusive")
+
+
+cdef class _BetweenOptions(FunctionOptions):
+    def _set_options(self, inclusive):
+        self.wrapped.reset(
+            new CBetweenOptions(unwrap_inclusive(inclusive))
+        )
+
+
+class BetweenOptions(_BetweenOptions):
+    """
+    Option to include endpoints

Review comment:
       ```suggestion
       Options for the `between` function.
   ```

##########
File path: cpp/src/arrow/compute/kernels/scalar_compare_test.cc
##########
@@ -1865,5 +1867,683 @@ TEST(TestMaxElementWiseMinElementWise, CommonTemporal) {
               ResultWith(ScalarFromJSON(date64(), "86400000")));
 }
 
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const Datum& val, const Datum& lhs,
+                            const Datum& rhs, const Datum& expected) {
+  ASSERT_OK_AND_ASSIGN(Datum result, Between(val, lhs, rhs, options, nullptr));
+  AssertArraysEqual(*expected.make_array(), *result.make_array(),
+                    /*verbose=*/true);
+}
+
+template <typename ArrowType>
+void ValidateBetween(BetweenOptions options, const Datum& val, const Datum& lhs,
+                     const Datum& rhs) {
+  CompareOperator lhs_val;
+  CompareOperator val_rhs;
+  BetweenOptions::Inclusive include_endpoints = options.inclusive;
+
+  if (include_endpoints == BetweenOptions::Inclusive::NEITHER) {
+    lhs_val = LESS;
+    val_rhs = LESS;
+  } else if (include_endpoints == BetweenOptions::Inclusive::LEFT) {
+    lhs_val = LESS_EQUAL;
+    val_rhs = LESS;
+  } else if (include_endpoints == BetweenOptions::Inclusive::RIGHT) {
+    lhs_val = LESS;
+    val_rhs = LESS_EQUAL;
+  } else {
+    lhs_val = LESS_EQUAL;
+    val_rhs = LESS_EQUAL;
+  }
+
+  ASSERT_OK_AND_ASSIGN(Datum resultl,
+                       CallFunction(CompareOperatorToFunctionName(lhs_val), {lhs, val}));
+  ASSERT_OK_AND_ASSIGN(Datum resultr,
+                       CallFunction(CompareOperatorToFunctionName(val_rhs), {val, rhs}));
+  ASSERT_OK_AND_ASSIGN(Datum expected, CallFunction("and", {resultl, resultr}));
+
+  ValidateBetween<ArrowType>(options, val, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+class TestNumericBetweenKernel : public ::testing::Test {};
+
+TYPED_TEST_SUITE(TestNumericBetweenKernel, NumericArrowTypes);
+TYPED_TEST(TestNumericBetweenKernel, SimpleBetweenArrayScalarScalar) {
+  using ScalarType = typename TypeTraits<TypeParam>::ScalarType;
+  using CType = typename TypeTraits<TypeParam>::CType;
+
+  Datum zero(std::make_shared<ScalarType>(CType(0)));
+  Datum four(std::make_shared<ScalarType>(CType(4)));
+  Datum null(std::make_shared<ScalarType>());
+  BetweenOptions InclusiveBoth(BetweenOptions::Inclusive::BOTH);
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"), zero,
+      four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      zero, four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,1,2,2]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[1,1,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,3,4,5]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[1,1,1,1,1,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,1,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null,0,1,1]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), null, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, null,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  BetweenOptions InclusiveLeft(BetweenOptions::Inclusive::LEFT);
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"), zero,
+      four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      zero, four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,1,2,2]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[1,1,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,3,4,5]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[1,1,1,1,0,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,0,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null,0,1,1]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), null, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, null,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  BetweenOptions InclusiveRight(BetweenOptions::Inclusive::RIGHT);
+  ValidateBetween<TypeParam>(
+      InclusiveRight, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"), zero,
+      four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      zero, four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,1,2,2]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,0,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,3,4,5]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,1,1,1,1,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,1,1,1,1,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null,0,1,1]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null,0,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), null, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, null,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  BetweenOptions InclusiveNeither(BetweenOptions::Inclusive::NEITHER);
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      zero, four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      zero, four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,1,2,2]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,0,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,3,4,5]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,1,1,1,0,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,0,1,1,1,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null,0,1,1]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null,0,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), null, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, null,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+}
+
+TYPED_TEST(TestNumericBetweenKernel, SimpleBetweenScalarArrayArray) {
+  using ScalarType = typename TypeTraits<TypeParam>::ScalarType;
+  using CType = typename TypeTraits<TypeParam>::CType;
+
+  Datum one(std::make_shared<ScalarType>(CType(1)));
+
+  BetweenOptions InclusiveBoth(BetweenOptions::Inclusive::BOTH);
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, one, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,true,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,true,null,false,false]"));
+  BetweenOptions InclusiveLeft(BetweenOptions::Inclusive::LEFT);
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, one, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,true,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,true,null,false,false]"));
+  BetweenOptions InclusiveRight(BetweenOptions::Inclusive::RIGHT);
+  ValidateBetween<TypeParam>(
+      InclusiveRight, one, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,false,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,false,null,false,false]"));
+  BetweenOptions InclusiveNeither(BetweenOptions::Inclusive::NEITHER);
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, one, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,false,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,false,null,false,false]"));
+}
+
+TYPED_TEST(TestNumericBetweenKernel, SimpleBetweenArrayArrayArray) {
+  BetweenOptions InclusiveBoth(BetweenOptions::Inclusive::BOTH);
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[1,1,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,true,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,true,null,false,false]"));
+  BetweenOptions InclusiveLeft(BetweenOptions::Inclusive::LEFT);
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[1,1,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,false,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,false,null,false,false]"));
+  BetweenOptions InclusiveRight(BetweenOptions::Inclusive::RIGHT);
+  ValidateBetween<TypeParam>(
+      InclusiveRight, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[1,1,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,true,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,true,null,false,false]"));
+  BetweenOptions InclusiveNeither(BetweenOptions::Inclusive::NEITHER);
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[1,1,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,false,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,false,null,false,false]"));
+}
+
+template <typename Type>
+struct BetweenRandomNumeric {
+  static void Test(const std::shared_ptr<DataType>& type) {
+    using ScalarType = typename TypeTraits<Type>::ScalarType;
+    using CType = typename TypeTraits<Type>::CType;
+    auto rand = random::RandomArrayGenerator(0x5416447);
+    const int64_t length = 1000;

Review comment:
       This seems slightly overkill, let's do `100`?

##########
File path: cpp/src/arrow/compute/kernels/scalar_compare_test.cc
##########
@@ -1865,5 +1867,683 @@ TEST(TestMaxElementWiseMinElementWise, CommonTemporal) {
               ResultWith(ScalarFromJSON(date64(), "86400000")));
 }
 
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const Datum& val, const Datum& lhs,
+                            const Datum& rhs, const Datum& expected) {
+  ASSERT_OK_AND_ASSIGN(Datum result, Between(val, lhs, rhs, options, nullptr));
+  AssertArraysEqual(*expected.make_array(), *result.make_array(),
+                    /*verbose=*/true);
+}
+
+template <typename ArrowType>
+void ValidateBetween(BetweenOptions options, const Datum& val, const Datum& lhs,
+                     const Datum& rhs) {
+  CompareOperator lhs_val;
+  CompareOperator val_rhs;
+  BetweenOptions::Inclusive include_endpoints = options.inclusive;
+
+  if (include_endpoints == BetweenOptions::Inclusive::NEITHER) {
+    lhs_val = LESS;
+    val_rhs = LESS;
+  } else if (include_endpoints == BetweenOptions::Inclusive::LEFT) {
+    lhs_val = LESS_EQUAL;
+    val_rhs = LESS;
+  } else if (include_endpoints == BetweenOptions::Inclusive::RIGHT) {
+    lhs_val = LESS;
+    val_rhs = LESS_EQUAL;
+  } else {
+    lhs_val = LESS_EQUAL;
+    val_rhs = LESS_EQUAL;
+  }
+
+  ASSERT_OK_AND_ASSIGN(Datum resultl,
+                       CallFunction(CompareOperatorToFunctionName(lhs_val), {lhs, val}));
+  ASSERT_OK_AND_ASSIGN(Datum resultr,
+                       CallFunction(CompareOperatorToFunctionName(val_rhs), {val, rhs}));
+  ASSERT_OK_AND_ASSIGN(Datum expected, CallFunction("and", {resultl, resultr}));
+
+  ValidateBetween<ArrowType>(options, val, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+class TestNumericBetweenKernel : public ::testing::Test {};
+
+TYPED_TEST_SUITE(TestNumericBetweenKernel, NumericArrowTypes);
+TYPED_TEST(TestNumericBetweenKernel, SimpleBetweenArrayScalarScalar) {
+  using ScalarType = typename TypeTraits<TypeParam>::ScalarType;
+  using CType = typename TypeTraits<TypeParam>::CType;
+
+  Datum zero(std::make_shared<ScalarType>(CType(0)));
+  Datum four(std::make_shared<ScalarType>(CType(4)));
+  Datum null(std::make_shared<ScalarType>());
+  BetweenOptions InclusiveBoth(BetweenOptions::Inclusive::BOTH);
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"), zero,
+      four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      zero, four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,1,2,2]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[1,1,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,3,4,5]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[1,1,1,1,1,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,1,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null,0,1,1]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), null, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, null,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  BetweenOptions InclusiveLeft(BetweenOptions::Inclusive::LEFT);
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"), zero,
+      four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      zero, four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,1,2,2]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[1,1,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,3,4,5]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[1,1,1,1,0,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,0,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null,0,1,1]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), null, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, null,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  BetweenOptions InclusiveRight(BetweenOptions::Inclusive::RIGHT);
+  ValidateBetween<TypeParam>(
+      InclusiveRight, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"), zero,
+      four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      zero, four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,1,2,2]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,0,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,3,4,5]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,1,1,1,1,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,1,1,1,1,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null,0,1,1]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null,0,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), null, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, null,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  BetweenOptions InclusiveNeither(BetweenOptions::Inclusive::NEITHER);
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      zero, four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      zero, four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,1,2,2]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,0,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,3,4,5]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,1,1,1,0,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,0,1,1,1,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null,0,1,1]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null,0,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), null, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, null,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+}
+
+TYPED_TEST(TestNumericBetweenKernel, SimpleBetweenScalarArrayArray) {
+  using ScalarType = typename TypeTraits<TypeParam>::ScalarType;
+  using CType = typename TypeTraits<TypeParam>::CType;
+
+  Datum one(std::make_shared<ScalarType>(CType(1)));
+
+  BetweenOptions InclusiveBoth(BetweenOptions::Inclusive::BOTH);
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, one, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,true,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,true,null,false,false]"));
+  BetweenOptions InclusiveLeft(BetweenOptions::Inclusive::LEFT);
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, one, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,true,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,true,null,false,false]"));
+  BetweenOptions InclusiveRight(BetweenOptions::Inclusive::RIGHT);
+  ValidateBetween<TypeParam>(
+      InclusiveRight, one, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,false,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,false,null,false,false]"));
+  BetweenOptions InclusiveNeither(BetweenOptions::Inclusive::NEITHER);
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, one, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,false,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,false,null,false,false]"));
+}
+
+TYPED_TEST(TestNumericBetweenKernel, SimpleBetweenArrayArrayArray) {
+  BetweenOptions InclusiveBoth(BetweenOptions::Inclusive::BOTH);
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[1,1,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,true,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,true,null,false,false]"));
+  BetweenOptions InclusiveLeft(BetweenOptions::Inclusive::LEFT);
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[1,1,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,false,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,false,null,false,false]"));
+  BetweenOptions InclusiveRight(BetweenOptions::Inclusive::RIGHT);
+  ValidateBetween<TypeParam>(
+      InclusiveRight, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[1,1,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,true,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,true,null,false,false]"));
+  BetweenOptions InclusiveNeither(BetweenOptions::Inclusive::NEITHER);
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[1,1,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,false,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,false,null,false,false]"));
+}

Review comment:
       Is the scalar-scalar-scalar variant tested somewhere?

##########
File path: docs/source/python/api/compute.rst
##########
@@ -146,12 +146,13 @@ variants which detect domain errors where appropriate.
 Comparisons
 -----------
 
-These functions expect two inputs of the same type. If one of the inputs is `null`
-they return ``null``.
+These functions expect two or three inputs of the same type. If one of the inputs is 
+`null` they return ``null``.

Review comment:
       ```suggestion
   ``null`` they return ``null``.
   ```

##########
File path: python/pyarrow/_compute.pyx
##########
@@ -785,6 +785,40 @@ class ElementWiseAggregateOptions(_ElementWiseAggregateOptions):
         self._set_options(skip_nulls)
 
 
+cdef CInclusive unwrap_inclusive(inclusive) except *:
+    if inclusive == "both":
+        return CInclusive_BOTH
+    elif inclusive == "left":
+        return CInclusive_LEFT
+    elif inclusive == "right":
+        return CInclusive_RIGHT
+    elif inclusive == "neither":
+        return CInclusive_NEITHER
+    _raise_invalid_function_option(inclusive, "inclusive")
+
+
+cdef class _BetweenOptions(FunctionOptions):
+    def _set_options(self, inclusive):
+        self.wrapped.reset(
+            new CBetweenOptions(unwrap_inclusive(inclusive))
+        )
+
+
+class BetweenOptions(_BetweenOptions):
+    """
+    Option to include endpoints
+
+    Parameters
+    ----------
+    inclusive : str, default "both"
+        Choices are, "both" ( a <= val <= b), "left" ( a <= value < b ),
+        "right" ( a < value <= b ), and "neither" ( a < value < b )
+    """
+
+    def __init__(self, inclusive="both"):

Review comment:
       Let's make this keyword-only:
   
   ```suggestion
       def __init__(self, *, inclusive="both"):
   ```

##########
File path: cpp/src/arrow/compute/kernels/scalar_compare_test.cc
##########
@@ -1865,5 +1867,683 @@ TEST(TestMaxElementWiseMinElementWise, CommonTemporal) {
               ResultWith(ScalarFromJSON(date64(), "86400000")));
 }
 
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const Datum& val, const Datum& lhs,
+                            const Datum& rhs, const Datum& expected) {
+  ASSERT_OK_AND_ASSIGN(Datum result, Between(val, lhs, rhs, options, nullptr));
+  AssertArraysEqual(*expected.make_array(), *result.make_array(),
+                    /*verbose=*/true);
+}
+
+template <typename ArrowType>
+void ValidateBetween(BetweenOptions options, const Datum& val, const Datum& lhs,
+                     const Datum& rhs) {
+  CompareOperator lhs_val;
+  CompareOperator val_rhs;
+  BetweenOptions::Inclusive include_endpoints = options.inclusive;
+
+  if (include_endpoints == BetweenOptions::Inclusive::NEITHER) {
+    lhs_val = LESS;
+    val_rhs = LESS;
+  } else if (include_endpoints == BetweenOptions::Inclusive::LEFT) {
+    lhs_val = LESS_EQUAL;
+    val_rhs = LESS;
+  } else if (include_endpoints == BetweenOptions::Inclusive::RIGHT) {
+    lhs_val = LESS;
+    val_rhs = LESS_EQUAL;
+  } else {
+    lhs_val = LESS_EQUAL;
+    val_rhs = LESS_EQUAL;
+  }
+
+  ASSERT_OK_AND_ASSIGN(Datum resultl,
+                       CallFunction(CompareOperatorToFunctionName(lhs_val), {lhs, val}));
+  ASSERT_OK_AND_ASSIGN(Datum resultr,
+                       CallFunction(CompareOperatorToFunctionName(val_rhs), {val, rhs}));
+  ASSERT_OK_AND_ASSIGN(Datum expected, CallFunction("and", {resultl, resultr}));
+
+  ValidateBetween<ArrowType>(options, val, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+class TestNumericBetweenKernel : public ::testing::Test {};
+
+TYPED_TEST_SUITE(TestNumericBetweenKernel, NumericArrowTypes);
+TYPED_TEST(TestNumericBetweenKernel, SimpleBetweenArrayScalarScalar) {
+  using ScalarType = typename TypeTraits<TypeParam>::ScalarType;
+  using CType = typename TypeTraits<TypeParam>::CType;
+
+  Datum zero(std::make_shared<ScalarType>(CType(0)));
+  Datum four(std::make_shared<ScalarType>(CType(4)));
+  Datum null(std::make_shared<ScalarType>());
+  BetweenOptions InclusiveBoth(BetweenOptions::Inclusive::BOTH);
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"), zero,
+      four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      zero, four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,1,2,2]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[1,1,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,3,4,5]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[1,1,1,1,1,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,1,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null,0,1,1]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), null, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, null,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  BetweenOptions InclusiveLeft(BetweenOptions::Inclusive::LEFT);
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"), zero,
+      four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      zero, four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,1,2,2]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[1,1,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,3,4,5]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[1,1,1,1,0,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,0,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null,0,1,1]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), null, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, null,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  BetweenOptions InclusiveRight(BetweenOptions::Inclusive::RIGHT);
+  ValidateBetween<TypeParam>(
+      InclusiveRight, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"), zero,
+      four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      zero, four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,1,2,2]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,0,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,3,4,5]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,1,1,1,1,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,1,1,1,1,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null,0,1,1]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null,0,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), null, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, null,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  BetweenOptions InclusiveNeither(BetweenOptions::Inclusive::NEITHER);
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      zero, four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      zero, four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,1,2,2]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,0,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,3,4,5]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,1,1,1,0,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,0,1,1,1,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null,0,1,1]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null,0,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), null, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, null,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+}
+
+TYPED_TEST(TestNumericBetweenKernel, SimpleBetweenScalarArrayArray) {
+  using ScalarType = typename TypeTraits<TypeParam>::ScalarType;
+  using CType = typename TypeTraits<TypeParam>::CType;
+
+  Datum one(std::make_shared<ScalarType>(CType(1)));
+
+  BetweenOptions InclusiveBoth(BetweenOptions::Inclusive::BOTH);
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, one, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,true,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,true,null,false,false]"));
+  BetweenOptions InclusiveLeft(BetweenOptions::Inclusive::LEFT);
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, one, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,true,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,true,null,false,false]"));
+  BetweenOptions InclusiveRight(BetweenOptions::Inclusive::RIGHT);
+  ValidateBetween<TypeParam>(
+      InclusiveRight, one, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,false,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,false,null,false,false]"));
+  BetweenOptions InclusiveNeither(BetweenOptions::Inclusive::NEITHER);
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, one, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,false,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,false,null,false,false]"));
+}
+
+TYPED_TEST(TestNumericBetweenKernel, SimpleBetweenArrayArrayArray) {
+  BetweenOptions InclusiveBoth(BetweenOptions::Inclusive::BOTH);
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[1,1,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,true,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,true,null,false,false]"));
+  BetweenOptions InclusiveLeft(BetweenOptions::Inclusive::LEFT);
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[1,1,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,false,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,false,null,false,false]"));
+  BetweenOptions InclusiveRight(BetweenOptions::Inclusive::RIGHT);
+  ValidateBetween<TypeParam>(
+      InclusiveRight, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[1,1,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,true,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,true,null,false,false]"));
+  BetweenOptions InclusiveNeither(BetweenOptions::Inclusive::NEITHER);
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[1,1,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,false,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,false,null,false,false]"));
+}
+
+template <typename Type>
+struct BetweenRandomNumeric {
+  static void Test(const std::shared_ptr<DataType>& type) {
+    using ScalarType = typename TypeTraits<Type>::ScalarType;
+    using CType = typename TypeTraits<Type>::CType;
+    auto rand = random::RandomArrayGenerator(0x5416447);
+    const int64_t length = 1000;
+    for (auto null_probability : {0.0, 0.01, 0.1, 0.25, 0.5, 1.0}) {
+      for (auto inclusive :
+           {BetweenOptions::Inclusive::BOTH, BetweenOptions::Inclusive::LEFT,
+            BetweenOptions::Inclusive::RIGHT, BetweenOptions::Inclusive::NEITHER}) {
+        auto data1 =
+            rand.Numeric<typename Type::PhysicalType>(length, 0, 100, null_probability);
+        auto data2 =
+            rand.Numeric<typename Type::PhysicalType>(length, 0, 100, null_probability);
+        auto data3 =
+            rand.Numeric<typename Type::PhysicalType>(length, 0, 100, null_probability);
+
+        // Create view of data as the type (e.g. timestamp)
+        auto array1 = Datum(*data1->View(type));
+        auto array2 = Datum(*data2->View(type));
+        auto array3 = Datum(*data3->View(type));
+        auto scalar1 = Datum(std::make_shared<ScalarType>(CType(10), type));
+        auto scalar2 = Datum(std::make_shared<ScalarType>(CType(30), type));
+        auto scalar3 = Datum(std::make_shared<ScalarType>(CType(50), type));
+        auto options = BetweenOptions(inclusive);
+        ValidateBetween<Type>(options, array1, scalar2, scalar3);
+        ValidateBetween<Type>(options, array1, array2, scalar3);
+        ValidateBetween<Type>(options, array1, array2, array3);
+        ValidateBetween<Type>(options, array1, scalar2, scalar3);
+        ValidateBetween<Type>(options, scalar1, array2, array3);
+        ValidateBetween<Type>(options, scalar1, array2, scalar3);
+        ValidateBetween<Type>(options, scalar1, scalar2, array3);
+        ValidateBetween<Type>(options, array1, scalar2, array3);
+      }
+    }
+  }
+};
+
+TEST(TestNumericBetweenKernel, BetweenPrimitiveRandomTests) {
+  TestRandomPrimitiveCTypes<BetweenRandomNumeric>();
+}
+
+class TestStringBetweenKernel : public ::testing::Test {};
+
+TEST(TestStringBetweenKernel, RandomBetween) {
+  using ScalarType = typename TypeTraits<StringType>::ScalarType;
+
+  auto rand = random::RandomArrayGenerator(0x5416447);
+  for (size_t i = 3; i < 10; i++) {
+    for (auto null_probability : {0.0, 0.01, 0.1, 0.25, 0.5, 1.0}) {
+      for (auto inclusive :
+           {BetweenOptions::Inclusive::BOTH, BetweenOptions::Inclusive::LEFT,
+            BetweenOptions::Inclusive::RIGHT, BetweenOptions::Inclusive::NEITHER}) {
+        const int64_t length = static_cast<int64_t>(1ULL << i);
+        auto array1 = Datum(rand.String(length, 0, 16, null_probability));
+        auto array2 = Datum(rand.String(length, 0, 16, null_probability));
+        auto array3 = Datum(rand.String(length, 0, 16, null_probability));
+        auto scalar1 = Datum(std::make_shared<ScalarType>("fupi"));
+        auto scalar2 = Datum(std::make_shared<ScalarType>("tupu"));
+        auto scalar3 = Datum(std::make_shared<ScalarType>("zito"));
+        auto options = BetweenOptions(inclusive);
+        ValidateBetween<StringType>(options, array1, scalar2, scalar3);
+        ValidateBetween<StringType>(options, scalar1, array2, scalar3);
+        ValidateBetween<StringType>(options, scalar1, scalar2, array3);
+        ValidateBetween<StringType>(options, scalar1, array2, array3);
+        ValidateBetween<StringType>(options, array1, scalar2, array3);
+        ValidateBetween<StringType>(options, array1, array2, scalar3);
+        ValidateBetween<StringType>(options, array1, array2, array3);
+      }
+    }
+  }
+}
+
+TEST(TestStringBetweenKernel, StringArrayScalarScalarTest) {
+  using ScalarType = typename TypeTraits<StringType>::ScalarType;
+  auto l = Datum(std::make_shared<ScalarType>("abc"));
+  auto r = Datum(std::make_shared<ScalarType>("zzz"));
+  BetweenOptions InclusiveOption(BetweenOptions::Inclusive::BOTH);
+  ValidateBetween<StringType>(
+      InclusiveOption, ArrayFromJSON(TypeTraits<StringType>::type_singleton(), "[]"), l,
+      r, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<StringType>(
+      InclusiveOption, ArrayFromJSON(TypeTraits<StringType>::type_singleton(), "[null]"),
+      l, r, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<StringType>(InclusiveOption,
+                              ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                            R"(["aaa", "aaaa", "ccc", "z"])"),
+                              l, r,
+                              ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                                            R"([false, false, true, true])"));
+  ValidateBetween<StringType>(InclusiveOption,
+                              ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                            R"(["a", "aaaa", "c", "z"])"),
+                              l, r,
+                              ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                                            R"([false, false, true, true])"));
+  ValidateBetween<StringType>(InclusiveOption,
+                              ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                            R"(["a", "aaaa", "fff", "zzzz"])"),
+                              l, r,
+                              ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                                            R"([false, false, true, false])"));
+  ValidateBetween<StringType>(InclusiveOption,
+                              ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                            R"(["abc", "baa", "fff", "zzz"])"),
+                              l, r,
+                              ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                                            R"([true, true, true, true])"));
+  ValidateBetween<StringType>(InclusiveOption,
+                              ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                            R"(["abd", null, null, "zzx"])"),
+                              l, r,
+                              ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                                            R"([true, null, null, true])"));
+}
+
+TEST(TestStringBetweenKernel, StringArrayArrayArrayTest) {
+  BetweenOptions InclusiveOption(BetweenOptions::Inclusive::BOTH);
+  ValidateBetween<StringType>(
+      InclusiveOption,
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                    R"(["david","hello","world"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["adam","hi","whirl"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                    R"(["robert","goeiemoreen","whirlwind"])"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[true, false, false]"));
+  ValidateBetween<StringType>(
+      InclusiveOption,
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["x","a","f"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["w","a","e"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["z","a","g"])"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[true, true, true]"));
+  ValidateBetween<StringType>(
+      InclusiveOption,
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                    R"(["block","bit","binary"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                    R"(["bit","nibble","ternary"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["word","d","xyz"])"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[true, false, false]"));
+  ValidateBetween<StringType>(
+      InclusiveOption,
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                    R"(["よしもと","の","ち"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["は","へ","あ"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["な","を","ち"])"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[false, false, true]"));
+  ValidateBetween<StringType>(
+      InclusiveOption,
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["A","ア","王"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["た","あ","歩"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["李","田",null])"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[false, true, null]"));
+  ValidateBetween<StringType>(
+      InclusiveOption,
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["Б",null,"Я"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["А","Ж","Щ"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["Д","Л","Ф"])"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[true, null, false]"));
+}
+
+TEST(TestTimestampsBetweenKernel, TimestampsArrayArrayArrayTest) {
+  const std::string arr_json = R"(["1970-01-01","1980-02-02","1970-02-28"])";
+  const std::string lhs_json = R"(["1970-01-01","1980-02-01","1970-02-28"])";
+  const std::string rhs_json = R"(["1970-01-02","1980-02-02","1970-02-28"])";
+  // Between Options
+  BetweenOptions both(BetweenOptions::Inclusive::BOTH);
+  BetweenOptions left(BetweenOptions::Inclusive::LEFT);
+  BetweenOptions right(BetweenOptions::Inclusive::RIGHT);
+  BetweenOptions neither(BetweenOptions::Inclusive::NEITHER);
+  // Same units should be fine
+  ValidateBetween<TimestampType>(both,
+                                 ArrayFromJSON(timestamp(TimeUnit::SECOND), arr_json),
+                                 ArrayFromJSON(timestamp(TimeUnit::SECOND), lhs_json),
+                                 ArrayFromJSON(timestamp(TimeUnit::SECOND), rhs_json));
+  ValidateBetween<TimestampType>(left,
+                                 ArrayFromJSON(timestamp(TimeUnit::SECOND), arr_json),
+                                 ArrayFromJSON(timestamp(TimeUnit::SECOND), lhs_json),
+                                 ArrayFromJSON(timestamp(TimeUnit::SECOND), rhs_json));
+  ValidateBetween<TimestampType>(right,
+                                 ArrayFromJSON(timestamp(TimeUnit::SECOND), arr_json),
+                                 ArrayFromJSON(timestamp(TimeUnit::SECOND), lhs_json),
+                                 ArrayFromJSON(timestamp(TimeUnit::SECOND), rhs_json));
+  ValidateBetween<TimestampType>(neither,
+                                 ArrayFromJSON(timestamp(TimeUnit::SECOND), arr_json),
+                                 ArrayFromJSON(timestamp(TimeUnit::SECOND), lhs_json),
+                                 ArrayFromJSON(timestamp(TimeUnit::SECOND), rhs_json));
+  ValidateBetween<TimestampType>(both,
+                                 ArrayFromJSON(timestamp(TimeUnit::MILLI), arr_json),
+                                 ArrayFromJSON(timestamp(TimeUnit::MILLI), lhs_json),
+                                 ArrayFromJSON(timestamp(TimeUnit::MILLI), rhs_json));
+  ValidateBetween<TimestampType>(left,
+                                 ArrayFromJSON(timestamp(TimeUnit::MILLI), arr_json),
+                                 ArrayFromJSON(timestamp(TimeUnit::MILLI), lhs_json),
+                                 ArrayFromJSON(timestamp(TimeUnit::MILLI), rhs_json));
+  ValidateBetween<TimestampType>(right,
+                                 ArrayFromJSON(timestamp(TimeUnit::MILLI), arr_json),
+                                 ArrayFromJSON(timestamp(TimeUnit::MILLI), lhs_json),
+                                 ArrayFromJSON(timestamp(TimeUnit::MILLI), rhs_json));
+  ValidateBetween<TimestampType>(neither,
+                                 ArrayFromJSON(timestamp(TimeUnit::MILLI), arr_json),
+                                 ArrayFromJSON(timestamp(TimeUnit::MILLI), lhs_json),
+                                 ArrayFromJSON(timestamp(TimeUnit::MILLI), rhs_json));

Review comment:
       This would deserve a bit more thought and care to avoid endless repetitions of nearly the same code :-)

##########
File path: cpp/src/arrow/compute/kernels/scalar_compare_test.cc
##########
@@ -1865,5 +1867,683 @@ TEST(TestMaxElementWiseMinElementWise, CommonTemporal) {
               ResultWith(ScalarFromJSON(date64(), "86400000")));
 }
 
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const Datum& val, const Datum& lhs,
+                            const Datum& rhs, const Datum& expected) {
+  ASSERT_OK_AND_ASSIGN(Datum result, Between(val, lhs, rhs, options, nullptr));
+  AssertArraysEqual(*expected.make_array(), *result.make_array(),
+                    /*verbose=*/true);
+}
+
+template <typename ArrowType>
+void ValidateBetween(BetweenOptions options, const Datum& val, const Datum& lhs,
+                     const Datum& rhs) {
+  CompareOperator lhs_val;
+  CompareOperator val_rhs;
+  BetweenOptions::Inclusive include_endpoints = options.inclusive;
+
+  if (include_endpoints == BetweenOptions::Inclusive::NEITHER) {
+    lhs_val = LESS;
+    val_rhs = LESS;
+  } else if (include_endpoints == BetweenOptions::Inclusive::LEFT) {
+    lhs_val = LESS_EQUAL;
+    val_rhs = LESS;
+  } else if (include_endpoints == BetweenOptions::Inclusive::RIGHT) {
+    lhs_val = LESS;
+    val_rhs = LESS_EQUAL;
+  } else {
+    lhs_val = LESS_EQUAL;
+    val_rhs = LESS_EQUAL;
+  }
+
+  ASSERT_OK_AND_ASSIGN(Datum resultl,
+                       CallFunction(CompareOperatorToFunctionName(lhs_val), {lhs, val}));
+  ASSERT_OK_AND_ASSIGN(Datum resultr,
+                       CallFunction(CompareOperatorToFunctionName(val_rhs), {val, rhs}));
+  ASSERT_OK_AND_ASSIGN(Datum expected, CallFunction("and", {resultl, resultr}));
+
+  ValidateBetween<ArrowType>(options, val, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+class TestNumericBetweenKernel : public ::testing::Test {};
+
+TYPED_TEST_SUITE(TestNumericBetweenKernel, NumericArrowTypes);
+TYPED_TEST(TestNumericBetweenKernel, SimpleBetweenArrayScalarScalar) {
+  using ScalarType = typename TypeTraits<TypeParam>::ScalarType;
+  using CType = typename TypeTraits<TypeParam>::CType;
+
+  Datum zero(std::make_shared<ScalarType>(CType(0)));
+  Datum four(std::make_shared<ScalarType>(CType(4)));
+  Datum null(std::make_shared<ScalarType>());
+  BetweenOptions InclusiveBoth(BetweenOptions::Inclusive::BOTH);
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"), zero,
+      four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      zero, four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,1,2,2]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[1,1,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,3,4,5]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[1,1,1,1,1,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,1,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null,0,1,1]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), null, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, null,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  BetweenOptions InclusiveLeft(BetweenOptions::Inclusive::LEFT);
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"), zero,
+      four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      zero, four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,1,2,2]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[1,1,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,3,4,5]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[1,1,1,1,0,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,0,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null,0,1,1]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), null, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, null,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  BetweenOptions InclusiveRight(BetweenOptions::Inclusive::RIGHT);
+  ValidateBetween<TypeParam>(
+      InclusiveRight, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"), zero,
+      four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      zero, four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,1,2,2]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,0,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,3,4,5]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,1,1,1,1,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,1,1,1,1,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null,0,1,1]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null,0,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), null, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, null,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  BetweenOptions InclusiveNeither(BetweenOptions::Inclusive::NEITHER);
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      zero, four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      zero, four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,1,2,2]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,0,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,3,4,5]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,1,1,1,0,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,0,1,1,1,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null,0,1,1]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null,0,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), null, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, null,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+}
+
+TYPED_TEST(TestNumericBetweenKernel, SimpleBetweenScalarArrayArray) {
+  using ScalarType = typename TypeTraits<TypeParam>::ScalarType;
+  using CType = typename TypeTraits<TypeParam>::CType;
+
+  Datum one(std::make_shared<ScalarType>(CType(1)));
+
+  BetweenOptions InclusiveBoth(BetweenOptions::Inclusive::BOTH);
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, one, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,true,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,true,null,false,false]"));
+  BetweenOptions InclusiveLeft(BetweenOptions::Inclusive::LEFT);
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, one, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,true,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,true,null,false,false]"));
+  BetweenOptions InclusiveRight(BetweenOptions::Inclusive::RIGHT);
+  ValidateBetween<TypeParam>(
+      InclusiveRight, one, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,false,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,false,null,false,false]"));
+  BetweenOptions InclusiveNeither(BetweenOptions::Inclusive::NEITHER);
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, one, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,false,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,false,null,false,false]"));
+}
+
+TYPED_TEST(TestNumericBetweenKernel, SimpleBetweenArrayArrayArray) {
+  BetweenOptions InclusiveBoth(BetweenOptions::Inclusive::BOTH);
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[1,1,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,true,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,true,null,false,false]"));
+  BetweenOptions InclusiveLeft(BetweenOptions::Inclusive::LEFT);
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[1,1,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,false,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,false,null,false,false]"));
+  BetweenOptions InclusiveRight(BetweenOptions::Inclusive::RIGHT);
+  ValidateBetween<TypeParam>(
+      InclusiveRight, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[1,1,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,true,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,true,null,false,false]"));
+  BetweenOptions InclusiveNeither(BetweenOptions::Inclusive::NEITHER);
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[1,1,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,false,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,false,null,false,false]"));
+}
+
+template <typename Type>
+struct BetweenRandomNumeric {

Review comment:
       I'm not sure this needs to be type-parametrized either. You can use `RandomArrayGenerator.ArrayOf`.

##########
File path: python/pyarrow/_compute.pyx
##########
@@ -785,6 +785,40 @@ class ElementWiseAggregateOptions(_ElementWiseAggregateOptions):
         self._set_options(skip_nulls)
 
 
+cdef CInclusive unwrap_inclusive(inclusive) except *:
+    if inclusive == "both":
+        return CInclusive_BOTH
+    elif inclusive == "left":
+        return CInclusive_LEFT
+    elif inclusive == "right":
+        return CInclusive_RIGHT
+    elif inclusive == "neither":
+        return CInclusive_NEITHER
+    _raise_invalid_function_option(inclusive, "inclusive")
+
+
+cdef class _BetweenOptions(FunctionOptions):
+    def _set_options(self, inclusive):
+        self.wrapped.reset(
+            new CBetweenOptions(unwrap_inclusive(inclusive))
+        )
+
+
+class BetweenOptions(_BetweenOptions):
+    """
+    Option to include endpoints
+
+    Parameters
+    ----------
+    inclusive : str, default "both"
+        Choices are, "both" ( a <= val <= b), "left" ( a <= value < b ),
+        "right" ( a < value <= b ), and "neither" ( a < value < b )

Review comment:
       ```suggestion
           Choices are "both" ( a <= val <= b), "left" ( a <= value < b ),
           "right" ( a < value <= b ) and "neither" ( a < value < b ).
   ```

##########
File path: cpp/src/arrow/compute/kernels/scalar_compare_test.cc
##########
@@ -1865,5 +1867,683 @@ TEST(TestMaxElementWiseMinElementWise, CommonTemporal) {
               ResultWith(ScalarFromJSON(date64(), "86400000")));
 }
 
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const Datum& val, const Datum& lhs,
+                            const Datum& rhs, const Datum& expected) {
+  ASSERT_OK_AND_ASSIGN(Datum result, Between(val, lhs, rhs, options, nullptr));
+  AssertArraysEqual(*expected.make_array(), *result.make_array(),
+                    /*verbose=*/true);
+}
+
+template <typename ArrowType>
+void ValidateBetween(BetweenOptions options, const Datum& val, const Datum& lhs,
+                     const Datum& rhs) {
+  CompareOperator lhs_val;
+  CompareOperator val_rhs;
+  BetweenOptions::Inclusive include_endpoints = options.inclusive;
+
+  if (include_endpoints == BetweenOptions::Inclusive::NEITHER) {
+    lhs_val = LESS;
+    val_rhs = LESS;
+  } else if (include_endpoints == BetweenOptions::Inclusive::LEFT) {
+    lhs_val = LESS_EQUAL;
+    val_rhs = LESS;
+  } else if (include_endpoints == BetweenOptions::Inclusive::RIGHT) {
+    lhs_val = LESS;
+    val_rhs = LESS_EQUAL;
+  } else {
+    lhs_val = LESS_EQUAL;
+    val_rhs = LESS_EQUAL;
+  }
+
+  ASSERT_OK_AND_ASSIGN(Datum resultl,
+                       CallFunction(CompareOperatorToFunctionName(lhs_val), {lhs, val}));
+  ASSERT_OK_AND_ASSIGN(Datum resultr,
+                       CallFunction(CompareOperatorToFunctionName(val_rhs), {val, rhs}));
+  ASSERT_OK_AND_ASSIGN(Datum expected, CallFunction("and", {resultl, resultr}));
+
+  ValidateBetween<ArrowType>(options, val, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+class TestNumericBetweenKernel : public ::testing::Test {};
+
+TYPED_TEST_SUITE(TestNumericBetweenKernel, NumericArrowTypes);
+TYPED_TEST(TestNumericBetweenKernel, SimpleBetweenArrayScalarScalar) {
+  using ScalarType = typename TypeTraits<TypeParam>::ScalarType;
+  using CType = typename TypeTraits<TypeParam>::CType;
+
+  Datum zero(std::make_shared<ScalarType>(CType(0)));
+  Datum four(std::make_shared<ScalarType>(CType(4)));
+  Datum null(std::make_shared<ScalarType>());
+  BetweenOptions InclusiveBoth(BetweenOptions::Inclusive::BOTH);
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"), zero,
+      four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      zero, four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,1,2,2]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[1,1,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,3,4,5]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[1,1,1,1,1,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,1,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null,0,1,1]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), null, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, null,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  BetweenOptions InclusiveLeft(BetweenOptions::Inclusive::LEFT);
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"), zero,
+      four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      zero, four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,1,2,2]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[1,1,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,3,4,5]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[1,1,1,1,0,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,0,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null,0,1,1]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), null, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, null,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  BetweenOptions InclusiveRight(BetweenOptions::Inclusive::RIGHT);
+  ValidateBetween<TypeParam>(
+      InclusiveRight, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"), zero,
+      four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      zero, four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,1,2,2]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,0,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,3,4,5]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,1,1,1,1,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,1,1,1,1,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null,0,1,1]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null,0,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), null, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, null,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  BetweenOptions InclusiveNeither(BetweenOptions::Inclusive::NEITHER);
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      zero, four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      zero, four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,1,2,2]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,0,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,3,4,5]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,1,1,1,0,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,0,1,1,1,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null,0,1,1]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null,0,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), null, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, null,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+}
+
+TYPED_TEST(TestNumericBetweenKernel, SimpleBetweenScalarArrayArray) {
+  using ScalarType = typename TypeTraits<TypeParam>::ScalarType;
+  using CType = typename TypeTraits<TypeParam>::CType;
+
+  Datum one(std::make_shared<ScalarType>(CType(1)));
+
+  BetweenOptions InclusiveBoth(BetweenOptions::Inclusive::BOTH);
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, one, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,true,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,true,null,false,false]"));
+  BetweenOptions InclusiveLeft(BetweenOptions::Inclusive::LEFT);
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, one, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,true,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,true,null,false,false]"));
+  BetweenOptions InclusiveRight(BetweenOptions::Inclusive::RIGHT);
+  ValidateBetween<TypeParam>(
+      InclusiveRight, one, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,false,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,false,null,false,false]"));
+  BetweenOptions InclusiveNeither(BetweenOptions::Inclusive::NEITHER);
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, one, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,false,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,false,null,false,false]"));
+}
+
+TYPED_TEST(TestNumericBetweenKernel, SimpleBetweenArrayArrayArray) {
+  BetweenOptions InclusiveBoth(BetweenOptions::Inclusive::BOTH);
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[1,1,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,true,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,true,null,false,false]"));
+  BetweenOptions InclusiveLeft(BetweenOptions::Inclusive::LEFT);
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[1,1,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,false,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,false,null,false,false]"));
+  BetweenOptions InclusiveRight(BetweenOptions::Inclusive::RIGHT);
+  ValidateBetween<TypeParam>(
+      InclusiveRight, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[1,1,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,true,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,true,null,false,false]"));
+  BetweenOptions InclusiveNeither(BetweenOptions::Inclusive::NEITHER);
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[1,1,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,false,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,false,null,false,false]"));
+}
+
+template <typename Type>
+struct BetweenRandomNumeric {
+  static void Test(const std::shared_ptr<DataType>& type) {
+    using ScalarType = typename TypeTraits<Type>::ScalarType;
+    using CType = typename TypeTraits<Type>::CType;
+    auto rand = random::RandomArrayGenerator(0x5416447);
+    const int64_t length = 1000;
+    for (auto null_probability : {0.0, 0.01, 0.1, 0.25, 0.5, 1.0}) {
+      for (auto inclusive :
+           {BetweenOptions::Inclusive::BOTH, BetweenOptions::Inclusive::LEFT,
+            BetweenOptions::Inclusive::RIGHT, BetweenOptions::Inclusive::NEITHER}) {
+        auto data1 =
+            rand.Numeric<typename Type::PhysicalType>(length, 0, 100, null_probability);
+        auto data2 =
+            rand.Numeric<typename Type::PhysicalType>(length, 0, 100, null_probability);
+        auto data3 =
+            rand.Numeric<typename Type::PhysicalType>(length, 0, 100, null_probability);
+
+        // Create view of data as the type (e.g. timestamp)
+        auto array1 = Datum(*data1->View(type));
+        auto array2 = Datum(*data2->View(type));
+        auto array3 = Datum(*data3->View(type));
+        auto scalar1 = Datum(std::make_shared<ScalarType>(CType(10), type));
+        auto scalar2 = Datum(std::make_shared<ScalarType>(CType(30), type));
+        auto scalar3 = Datum(std::make_shared<ScalarType>(CType(50), type));
+        auto options = BetweenOptions(inclusive);
+        ValidateBetween<Type>(options, array1, scalar2, scalar3);
+        ValidateBetween<Type>(options, array1, array2, scalar3);
+        ValidateBetween<Type>(options, array1, array2, array3);
+        ValidateBetween<Type>(options, array1, scalar2, scalar3);
+        ValidateBetween<Type>(options, scalar1, array2, array3);
+        ValidateBetween<Type>(options, scalar1, array2, scalar3);
+        ValidateBetween<Type>(options, scalar1, scalar2, array3);
+        ValidateBetween<Type>(options, array1, scalar2, array3);
+      }
+    }
+  }
+};
+
+TEST(TestNumericBetweenKernel, BetweenPrimitiveRandomTests) {
+  TestRandomPrimitiveCTypes<BetweenRandomNumeric>();
+}
+
+class TestStringBetweenKernel : public ::testing::Test {};
+
+TEST(TestStringBetweenKernel, RandomBetween) {
+  using ScalarType = typename TypeTraits<StringType>::ScalarType;
+
+  auto rand = random::RandomArrayGenerator(0x5416447);
+  for (size_t i = 3; i < 10; i++) {
+    for (auto null_probability : {0.0, 0.01, 0.1, 0.25, 0.5, 1.0}) {
+      for (auto inclusive :
+           {BetweenOptions::Inclusive::BOTH, BetweenOptions::Inclusive::LEFT,
+            BetweenOptions::Inclusive::RIGHT, BetweenOptions::Inclusive::NEITHER}) {
+        const int64_t length = static_cast<int64_t>(1ULL << i);
+        auto array1 = Datum(rand.String(length, 0, 16, null_probability));
+        auto array2 = Datum(rand.String(length, 0, 16, null_probability));
+        auto array3 = Datum(rand.String(length, 0, 16, null_probability));
+        auto scalar1 = Datum(std::make_shared<ScalarType>("fupi"));
+        auto scalar2 = Datum(std::make_shared<ScalarType>("tupu"));
+        auto scalar3 = Datum(std::make_shared<ScalarType>("zito"));
+        auto options = BetweenOptions(inclusive);
+        ValidateBetween<StringType>(options, array1, scalar2, scalar3);
+        ValidateBetween<StringType>(options, scalar1, array2, scalar3);
+        ValidateBetween<StringType>(options, scalar1, scalar2, array3);
+        ValidateBetween<StringType>(options, scalar1, array2, array3);
+        ValidateBetween<StringType>(options, array1, scalar2, array3);
+        ValidateBetween<StringType>(options, array1, array2, scalar3);
+        ValidateBetween<StringType>(options, array1, array2, array3);
+      }
+    }
+  }
+}
+
+TEST(TestStringBetweenKernel, StringArrayScalarScalarTest) {
+  using ScalarType = typename TypeTraits<StringType>::ScalarType;
+  auto l = Datum(std::make_shared<ScalarType>("abc"));
+  auto r = Datum(std::make_shared<ScalarType>("zzz"));
+  BetweenOptions InclusiveOption(BetweenOptions::Inclusive::BOTH);
+  ValidateBetween<StringType>(
+      InclusiveOption, ArrayFromJSON(TypeTraits<StringType>::type_singleton(), "[]"), l,
+      r, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<StringType>(
+      InclusiveOption, ArrayFromJSON(TypeTraits<StringType>::type_singleton(), "[null]"),
+      l, r, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<StringType>(InclusiveOption,
+                              ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                            R"(["aaa", "aaaa", "ccc", "z"])"),
+                              l, r,
+                              ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                                            R"([false, false, true, true])"));
+  ValidateBetween<StringType>(InclusiveOption,
+                              ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                            R"(["a", "aaaa", "c", "z"])"),
+                              l, r,
+                              ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                                            R"([false, false, true, true])"));
+  ValidateBetween<StringType>(InclusiveOption,
+                              ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                            R"(["a", "aaaa", "fff", "zzzz"])"),
+                              l, r,
+                              ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                                            R"([false, false, true, false])"));
+  ValidateBetween<StringType>(InclusiveOption,
+                              ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                            R"(["abc", "baa", "fff", "zzz"])"),
+                              l, r,
+                              ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                                            R"([true, true, true, true])"));
+  ValidateBetween<StringType>(InclusiveOption,
+                              ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                            R"(["abd", null, null, "zzx"])"),
+                              l, r,
+                              ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                                            R"([true, null, null, true])"));
+}
+
+TEST(TestStringBetweenKernel, StringArrayArrayArrayTest) {
+  BetweenOptions InclusiveOption(BetweenOptions::Inclusive::BOTH);
+  ValidateBetween<StringType>(
+      InclusiveOption,
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                    R"(["david","hello","world"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["adam","hi","whirl"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                    R"(["robert","goeiemoreen","whirlwind"])"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[true, false, false]"));
+  ValidateBetween<StringType>(
+      InclusiveOption,
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["x","a","f"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["w","a","e"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["z","a","g"])"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[true, true, true]"));
+  ValidateBetween<StringType>(
+      InclusiveOption,
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                    R"(["block","bit","binary"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                    R"(["bit","nibble","ternary"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["word","d","xyz"])"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[true, false, false]"));
+  ValidateBetween<StringType>(
+      InclusiveOption,
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                    R"(["よしもと","の","ち"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["は","へ","あ"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["な","を","ち"])"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[false, false, true]"));
+  ValidateBetween<StringType>(
+      InclusiveOption,
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["A","ア","王"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["た","あ","歩"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["李","田",null])"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[false, true, null]"));
+  ValidateBetween<StringType>(
+      InclusiveOption,
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["Б",null,"Я"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["А","Ж","Щ"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["Д","Л","Ф"])"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[true, null, false]"));

Review comment:
       Again, does it make sense to write down all these tests by hand? Are we afraid that the compiled code may work on some values but not others?

##########
File path: python/pyarrow/tests/test_compute.py
##########
@@ -1360,6 +1361,153 @@ def test_filter_null_type():
     assert len(table.filter(mask).column(0)) == 5
 
 
+@pytest.mark.parametrize("ty", ["inclusive"])
+def test_between_array_array_array(ty):
+    BetweenOptions = partial(pc.BetweenOptions)
+
+    val = pa.array([1, 1, 4, 3, 2, 6])
+    arr1 = pa.array([1, 1, 3, 4, None, 5])
+    arr2 = pa.array([1, 2, 4, None, 4, 7])
+
+    inclusive_and_expected = {
+        "both": [True, True, True, None, None, True],
+        "left": [False, True, False, None, None, True],
+        "right": [False, False, True, None, None, True],
+        "neither": [False, False, False, None, None, True],
+    }
+
+    for inclusive, expected in inclusive_and_expected.items():
+        options = BetweenOptions(inclusive=inclusive)
+        result = pc.between(val, arr1, arr2, options=options)
+        assert result.equals(pa.array(expected))
+
+
+@pytest.mark.parametrize("ty", ["inclusive"])
+def test_between_array_array_scalar(ty):

Review comment:
       There's no need to rewrite in Python all the tests that already exist in C++! Can you just keep a basic test (such as the above) to check that the API is wired correctly?

##########
File path: cpp/src/arrow/compute/kernels/scalar_compare.cc
##########
@@ -746,6 +808,279 @@ std::shared_ptr<ScalarFunction> MakeScalarMinMax(std::string name,
   return func;
 }
 
+template <template <BetweenOptions::Inclusive> class Op>
+std::shared_ptr<ScalarFunction> MakeBetweenFunction(std::string name,
+                                                    const FunctionDoc* doc) {
+  using BetweenState = OptionsWrapper<BetweenOptions>;
+  static const auto kDefaultOptions = BetweenOptions::Defaults();
+  auto func =
+      std::make_shared<CompareFunction>(name, Arity::Ternary(), doc, &kDefaultOptions);
+
+  // Add kernels for physical numeric types, temporal types done separately
+  for (const auto& types : {DurationTypes(), IntervalTypes(), NumericTypes()}) {
+    for (const auto& ty : types) {
+      auto type_id = ty->id();
+      auto exec = [type_id](KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+        // Resolve generator based on options
+        const auto& state = static_cast<const BetweenState&>(*ctx->state());
+        switch (state.options.inclusive) {
+          case BetweenOptions::Inclusive::BOTH:
+            return GeneratePhysicalNumeric<ScalarTernaryEqualTypes, BooleanType,
+                                           Op<BetweenOptions::Inclusive::BOTH>>(type_id)(
+                ctx, batch, out);
+          case BetweenOptions::Inclusive::LEFT:
+            return GeneratePhysicalNumeric<ScalarTernaryEqualTypes, BooleanType,
+                                           Op<BetweenOptions::Inclusive::LEFT>>(type_id)(
+                ctx, batch, out);
+          case BetweenOptions::Inclusive::RIGHT:
+            return GeneratePhysicalNumeric<ScalarTernaryEqualTypes, BooleanType,
+                                           Op<BetweenOptions::Inclusive::RIGHT>>(type_id)(
+                ctx, batch, out);
+          case BetweenOptions::Inclusive::NEITHER:
+            return GeneratePhysicalNumeric<ScalarTernaryEqualTypes, BooleanType,
+                                           Op<BetweenOptions::Inclusive::NEITHER>>(
+                type_id)(ctx, batch, out);
+          default:
+            return Status::NotImplemented("between inclusiveness not implemented: ",
+                                          state.options.ToString());
+        }
+      };
+      DCHECK_OK(func->AddKernel({ty, ty, ty}, boolean(), exec, BetweenState::Init));
+    }
+  }
+
+  // Add timestamp kernels
+  for (const auto unit : TimeUnit::values()) {
+    InputType in_type(match::TimestampTypeUnit(unit));
+    auto type_id = Type::TIMESTAMP;
+    auto exec = [type_id](KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+      // Validate timezones in all entries or in none
+      RETURN_NOT_OK(CheckCompareTimestamps(batch));
+      // Resolve generator based on options
+      const auto& state = static_cast<const BetweenState&>(*ctx->state());
+      switch (state.options.inclusive) {
+        case BetweenOptions::Inclusive::BOTH:
+          return GeneratePhysicalNumeric<ScalarTernaryEqualTypes, BooleanType,
+                                         Op<BetweenOptions::Inclusive::BOTH>>(type_id)(
+              ctx, batch, out);
+        case BetweenOptions::Inclusive::LEFT:
+          return GeneratePhysicalNumeric<ScalarTernaryEqualTypes, BooleanType,
+                                         Op<BetweenOptions::Inclusive::LEFT>>(type_id)(
+              ctx, batch, out);
+        case BetweenOptions::Inclusive::RIGHT:
+          return GeneratePhysicalNumeric<ScalarTernaryEqualTypes, BooleanType,
+                                         Op<BetweenOptions::Inclusive::RIGHT>>(type_id)(
+              ctx, batch, out);
+        case BetweenOptions::Inclusive::NEITHER:
+          return GeneratePhysicalNumeric<ScalarTernaryEqualTypes, BooleanType,
+                                         Op<BetweenOptions::Inclusive::NEITHER>>(type_id)(
+              ctx, batch, out);
+        default:
+          return Status::NotImplemented("between inclusiveness not implemented: ",
+                                        state.options.ToString());
+      }
+    };
+    DCHECK_OK(func->AddKernel({in_type, in_type, in_type}, boolean(), exec,
+                              BetweenState::Init));
+  }
+
+  // Add time kernels
+  for (const auto unit : {TimeUnit::SECOND, TimeUnit::MILLI}) {
+    auto type_id = Type::TIME32;
+    auto exec = [type_id](KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+      // Resolve generator based on options
+      const auto& state = static_cast<const BetweenState&>(*ctx->state());
+      switch (state.options.inclusive) {
+        case BetweenOptions::Inclusive::BOTH:
+          return GeneratePhysicalNumeric<ScalarTernaryEqualTypes, BooleanType,
+                                         Op<BetweenOptions::Inclusive::BOTH>>(type_id)(
+              ctx, batch, out);
+        case BetweenOptions::Inclusive::LEFT:
+          return GeneratePhysicalNumeric<ScalarTernaryEqualTypes, BooleanType,
+                                         Op<BetweenOptions::Inclusive::LEFT>>(type_id)(
+              ctx, batch, out);
+        case BetweenOptions::Inclusive::RIGHT:
+          return GeneratePhysicalNumeric<ScalarTernaryEqualTypes, BooleanType,
+                                         Op<BetweenOptions::Inclusive::RIGHT>>(type_id)(
+              ctx, batch, out);
+        case BetweenOptions::Inclusive::NEITHER:
+          return GeneratePhysicalNumeric<ScalarTernaryEqualTypes, BooleanType,
+                                         Op<BetweenOptions::Inclusive::NEITHER>>(type_id)(
+              ctx, batch, out);
+        default:
+          return Status::NotImplemented("between inclusiveness not implemented: ",
+                                        state.options.ToString());
+      }
+    };
+    InputType in_type(match::Time32TypeUnit(unit));
+    DCHECK_OK(func->AddKernel({in_type, in_type, in_type}, boolean(), exec,
+                              BetweenState::Init));
+  }
+
+  for (const auto& unit : {TimeUnit::MICRO, TimeUnit::NANO}) {
+    auto type_id = Type::TIME64;
+    auto exec = [type_id](KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+      // Resolve generator based on options
+      const auto& state = static_cast<const BetweenState&>(*ctx->state());
+      switch (state.options.inclusive) {
+        case BetweenOptions::Inclusive::BOTH:
+          return GeneratePhysicalNumeric<ScalarTernaryEqualTypes, BooleanType,
+                                         Op<BetweenOptions::Inclusive::BOTH>>(type_id)(
+              ctx, batch, out);
+        case BetweenOptions::Inclusive::LEFT:
+          return GeneratePhysicalNumeric<ScalarTernaryEqualTypes, BooleanType,
+                                         Op<BetweenOptions::Inclusive::LEFT>>(type_id)(
+              ctx, batch, out);
+        case BetweenOptions::Inclusive::RIGHT:
+          return GeneratePhysicalNumeric<ScalarTernaryEqualTypes, BooleanType,
+                                         Op<BetweenOptions::Inclusive::RIGHT>>(type_id)(
+              ctx, batch, out);
+        case BetweenOptions::Inclusive::NEITHER:
+          return GeneratePhysicalNumeric<ScalarTernaryEqualTypes, BooleanType,
+                                         Op<BetweenOptions::Inclusive::NEITHER>>(type_id)(
+              ctx, batch, out);
+        default:
+          return Status::NotImplemented("between inclusiveness not implemented: ",
+                                        state.options.ToString());
+      }
+    };
+    InputType in_type(match::Time64TypeUnit(unit));
+    DCHECK_OK(func->AddKernel({in_type, in_type, in_type}, boolean(), exec,
+                              BetweenState::Init));
+  }
+
+  // Add kernels for base binary types
+  for (const auto& ty : BaseBinaryTypes()) {
+    auto type_id = ty->id();
+    auto exec = [type_id](KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+      // Resolve generator based on options
+      const auto& state = static_cast<const BetweenState&>(*ctx->state());
+      switch (state.options.inclusive) {
+        case BetweenOptions::Inclusive::BOTH:
+          return GenerateVarBinaryBase<ScalarTernaryEqualTypes, BooleanType,
+                                       Op<BetweenOptions::Inclusive::BOTH>>(type_id)(
+              ctx, batch, out);
+        case BetweenOptions::Inclusive::LEFT:
+          return GenerateVarBinaryBase<ScalarTernaryEqualTypes, BooleanType,
+                                       Op<BetweenOptions::Inclusive::LEFT>>(type_id)(
+              ctx, batch, out);
+        case BetweenOptions::Inclusive::RIGHT:
+          return GenerateVarBinaryBase<ScalarTernaryEqualTypes, BooleanType,
+                                       Op<BetweenOptions::Inclusive::RIGHT>>(type_id)(
+              ctx, batch, out);
+        case BetweenOptions::Inclusive::NEITHER:
+          return GenerateVarBinaryBase<ScalarTernaryEqualTypes, BooleanType,
+                                       Op<BetweenOptions::Inclusive::NEITHER>>(type_id)(
+              ctx, batch, out);
+        default:
+          return Status::NotImplemented("between inclusiveness not implemented: ",
+                                        state.options.ToString());
+      }
+    };
+    DCHECK_OK(func->AddKernel({ty, ty, ty}, boolean(), exec, BetweenState::Init));
+  }
+
+  // Add kernels for decimal types
+  for (const auto type_id : {Type::DECIMAL128, Type::DECIMAL256}) {
+    auto exec = [type_id](KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+      // Resolve generator based on options
+      const auto& state = static_cast<const BetweenState&>(*ctx->state());
+      switch (state.options.inclusive) {
+        case BetweenOptions::Inclusive::BOTH:
+          return GenerateDecimal<ScalarTernaryEqualTypes, BooleanType,
+                                 Op<BetweenOptions::Inclusive::BOTH>>(type_id)(ctx, batch,
+                                                                               out);
+        case BetweenOptions::Inclusive::LEFT:
+          return GenerateDecimal<ScalarTernaryEqualTypes, BooleanType,
+                                 Op<BetweenOptions::Inclusive::LEFT>>(type_id)(ctx, batch,
+                                                                               out);
+        case BetweenOptions::Inclusive::RIGHT:
+          return GenerateDecimal<ScalarTernaryEqualTypes, BooleanType,
+                                 Op<BetweenOptions::Inclusive::RIGHT>>(type_id)(
+              ctx, batch, out);
+        case BetweenOptions::Inclusive::NEITHER:
+          return GenerateDecimal<ScalarTernaryEqualTypes, BooleanType,
+                                 Op<BetweenOptions::Inclusive::NEITHER>>(type_id)(
+              ctx, batch, out);
+        default:
+          return Status::NotImplemented("between inclusiveness not implemented: ",
+                                        state.options.ToString());
+      }
+    };
+    InputType ty(type_id);
+    DCHECK_OK(func->AddKernel({ty, ty, ty}, boolean(), exec, BetweenState::Init));
+  }
+
+  // Add kernels for date types
+  for (const auto type_id : {Type::DATE32, Type::DATE64}) {
+    auto exec = [type_id](KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+      // Resolve generator based on options
+      const auto& state = static_cast<const BetweenState&>(*ctx->state());
+      switch (state.options.inclusive) {
+        case BetweenOptions::Inclusive::BOTH:
+          return GeneratePhysicalNumeric<ScalarTernaryEqualTypes, BooleanType,
+                                         Op<BetweenOptions::Inclusive::BOTH>>(type_id)(
+              ctx, batch, out);
+        case BetweenOptions::Inclusive::LEFT:
+          return GeneratePhysicalNumeric<ScalarTernaryEqualTypes, BooleanType,
+                                         Op<BetweenOptions::Inclusive::LEFT>>(type_id)(
+              ctx, batch, out);
+        case BetweenOptions::Inclusive::RIGHT:
+          return GeneratePhysicalNumeric<ScalarTernaryEqualTypes, BooleanType,
+                                         Op<BetweenOptions::Inclusive::RIGHT>>(type_id)(
+              ctx, batch, out);
+        case BetweenOptions::Inclusive::NEITHER:
+          return GeneratePhysicalNumeric<ScalarTernaryEqualTypes, BooleanType,
+                                         Op<BetweenOptions::Inclusive::NEITHER>>(type_id)(
+              ctx, batch, out);
+        default:
+          return Status::NotImplemented("between inclusiveness not implemented: ",
+                                        state.options.ToString());
+      }
+    };

Review comment:
       Something like this wouldn't work as-is, but it would if we refactored `GeneratePhysicalNumeric` etc. as functor classes:
   ```c++
   template <template <typename...> class GenerateFunc>
   ArrayKernelExec MakeBetweenArrayExec(Type::type type_id) {
     auto exec = [type_id](KernelContext* ctx, const ExecBatch& batch, Datum* out) {
       // Resolve generator based on options
       const auto& state = static_cast<const BetweenState&>(*ctx->state());
       switch (state.options.inclusive) {
         case BetweenOptions::Inclusive::BOTH:
           return GenerateFunc<ScalarTernaryEqualTypes, BooleanType,
                               Op<BetweenOptions::Inclusive::BOTH>>(type_id)(
               ctx, batch, out);
         case BetweenOptions::Inclusive::LEFT:
           return GenerateFunc<ScalarTernaryEqualTypes, BooleanType,
                               Op<BetweenOptions::Inclusive::LEFT>>(type_id)(
               ctx, batch, out);
         case BetweenOptions::Inclusive::RIGHT:
           return GenerateFunc<ScalarTernaryEqualTypes, BooleanType,
                               Op<BetweenOptions::Inclusive::RIGHT>>(type_id)(
               ctx, batch, out);
         case BetweenOptions::Inclusive::NEITHER:
           return GenerateFunc<ScalarTernaryEqualTypes, BooleanType,
                               Op<BetweenOptions::Inclusive::NEITHER>>(type_id)(
               ctx, batch, out);
         default:
           return Status::NotImplemented("between inclusiveness not implemented: ",
                                         state.options.ToString());
       }
     };
   }
   ```




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

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

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



[GitHub] [arrow] edponce commented on pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
edponce commented on pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#issuecomment-1013718034


   @bkmgit I am curious why there are many changes to Java, Go, R made alongside this Between PR?


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

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

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



[GitHub] [arrow] bkmgit commented on a change in pull request #11882: ARROW-9843: [C++][Python] Implement Between ternary kernel and Python bindings

Posted by GitBox <gi...@apache.org>.
bkmgit commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r787593339



##########
File path: cpp/src/arrow/compute/kernels/scalar_compare_test.cc
##########
@@ -1865,5 +1867,683 @@ TEST(TestMaxElementWiseMinElementWise, CommonTemporal) {
               ResultWith(ScalarFromJSON(date64(), "86400000")));
 }
 
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const Datum& val, const Datum& lhs,
+                            const Datum& rhs, const Datum& expected) {
+  ASSERT_OK_AND_ASSIGN(Datum result, Between(val, lhs, rhs, options, nullptr));
+  AssertArraysEqual(*expected.make_array(), *result.make_array(),
+                    /*verbose=*/true);
+}
+
+template <typename ArrowType>
+void ValidateBetween(BetweenOptions options, const Datum& val, const Datum& lhs,
+                     const Datum& rhs) {
+  CompareOperator lhs_val;
+  CompareOperator val_rhs;
+  BetweenOptions::Inclusive include_endpoints = options.inclusive;
+
+  if (include_endpoints == BetweenOptions::Inclusive::NEITHER) {
+    lhs_val = LESS;
+    val_rhs = LESS;
+  } else if (include_endpoints == BetweenOptions::Inclusive::LEFT) {
+    lhs_val = LESS_EQUAL;
+    val_rhs = LESS;
+  } else if (include_endpoints == BetweenOptions::Inclusive::RIGHT) {
+    lhs_val = LESS;
+    val_rhs = LESS_EQUAL;
+  } else {
+    lhs_val = LESS_EQUAL;
+    val_rhs = LESS_EQUAL;
+  }
+
+  ASSERT_OK_AND_ASSIGN(Datum resultl,
+                       CallFunction(CompareOperatorToFunctionName(lhs_val), {lhs, val}));
+  ASSERT_OK_AND_ASSIGN(Datum resultr,
+                       CallFunction(CompareOperatorToFunctionName(val_rhs), {val, rhs}));
+  ASSERT_OK_AND_ASSIGN(Datum expected, CallFunction("and", {resultl, resultr}));
+
+  ValidateBetween<ArrowType>(options, val, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+class TestNumericBetweenKernel : public ::testing::Test {};
+
+TYPED_TEST_SUITE(TestNumericBetweenKernel, NumericArrowTypes);
+TYPED_TEST(TestNumericBetweenKernel, SimpleBetweenArrayScalarScalar) {
+  using ScalarType = typename TypeTraits<TypeParam>::ScalarType;
+  using CType = typename TypeTraits<TypeParam>::CType;
+
+  Datum zero(std::make_shared<ScalarType>(CType(0)));
+  Datum four(std::make_shared<ScalarType>(CType(4)));
+  Datum null(std::make_shared<ScalarType>());
+  BetweenOptions InclusiveBoth(BetweenOptions::Inclusive::BOTH);
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"), zero,
+      four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      zero, four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,1,2,2]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[1,1,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,3,4,5]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[1,1,1,1,1,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,1,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null,0,1,1]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), null, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, null,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  BetweenOptions InclusiveLeft(BetweenOptions::Inclusive::LEFT);
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"), zero,
+      four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      zero, four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,1,2,2]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[1,1,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,3,4,5]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[1,1,1,1,0,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,0,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null,0,1,1]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), null, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, null,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  BetweenOptions InclusiveRight(BetweenOptions::Inclusive::RIGHT);
+  ValidateBetween<TypeParam>(
+      InclusiveRight, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"), zero,
+      four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      zero, four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,1,2,2]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,0,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,3,4,5]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,1,1,1,1,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,1,1,1,1,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null,0,1,1]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null,0,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), null, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, null,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  BetweenOptions InclusiveNeither(BetweenOptions::Inclusive::NEITHER);
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      zero, four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      zero, four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,1,2,2]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,0,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,3,4,5]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,1,1,1,0,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,0,1,1,1,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null,0,1,1]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null,0,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), null, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, null,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+}
+
+TYPED_TEST(TestNumericBetweenKernel, SimpleBetweenScalarArrayArray) {
+  using ScalarType = typename TypeTraits<TypeParam>::ScalarType;
+  using CType = typename TypeTraits<TypeParam>::CType;
+
+  Datum one(std::make_shared<ScalarType>(CType(1)));
+
+  BetweenOptions InclusiveBoth(BetweenOptions::Inclusive::BOTH);
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, one, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,true,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,true,null,false,false]"));
+  BetweenOptions InclusiveLeft(BetweenOptions::Inclusive::LEFT);
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, one, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,true,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,true,null,false,false]"));
+  BetweenOptions InclusiveRight(BetweenOptions::Inclusive::RIGHT);
+  ValidateBetween<TypeParam>(
+      InclusiveRight, one, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,false,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,false,null,false,false]"));
+  BetweenOptions InclusiveNeither(BetweenOptions::Inclusive::NEITHER);
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, one, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,false,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,false,null,false,false]"));
+}
+
+TYPED_TEST(TestNumericBetweenKernel, SimpleBetweenArrayArrayArray) {
+  BetweenOptions InclusiveBoth(BetweenOptions::Inclusive::BOTH);
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[1,1,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,true,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,true,null,false,false]"));
+  BetweenOptions InclusiveLeft(BetweenOptions::Inclusive::LEFT);
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[1,1,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,false,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,false,null,false,false]"));
+  BetweenOptions InclusiveRight(BetweenOptions::Inclusive::RIGHT);
+  ValidateBetween<TypeParam>(
+      InclusiveRight, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[1,1,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,true,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,true,null,false,false]"));
+  BetweenOptions InclusiveNeither(BetweenOptions::Inclusive::NEITHER);
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[1,1,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,false,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,false,null,false,false]"));
+}
+
+template <typename Type>
+struct BetweenRandomNumeric {
+  static void Test(const std::shared_ptr<DataType>& type) {
+    using ScalarType = typename TypeTraits<Type>::ScalarType;
+    using CType = typename TypeTraits<Type>::CType;
+    auto rand = random::RandomArrayGenerator(0x5416447);
+    const int64_t length = 1000;
+    for (auto null_probability : {0.0, 0.01, 0.1, 0.25, 0.5, 1.0}) {
+      for (auto inclusive :
+           {BetweenOptions::Inclusive::BOTH, BetweenOptions::Inclusive::LEFT,
+            BetweenOptions::Inclusive::RIGHT, BetweenOptions::Inclusive::NEITHER}) {
+        auto data1 =
+            rand.Numeric<typename Type::PhysicalType>(length, 0, 100, null_probability);
+        auto data2 =
+            rand.Numeric<typename Type::PhysicalType>(length, 0, 100, null_probability);
+        auto data3 =
+            rand.Numeric<typename Type::PhysicalType>(length, 0, 100, null_probability);
+
+        // Create view of data as the type (e.g. timestamp)
+        auto array1 = Datum(*data1->View(type));
+        auto array2 = Datum(*data2->View(type));
+        auto array3 = Datum(*data3->View(type));
+        auto scalar1 = Datum(std::make_shared<ScalarType>(CType(10), type));
+        auto scalar2 = Datum(std::make_shared<ScalarType>(CType(30), type));
+        auto scalar3 = Datum(std::make_shared<ScalarType>(CType(50), type));
+        auto options = BetweenOptions(inclusive);
+        ValidateBetween<Type>(options, array1, scalar2, scalar3);
+        ValidateBetween<Type>(options, array1, array2, scalar3);
+        ValidateBetween<Type>(options, array1, array2, array3);
+        ValidateBetween<Type>(options, array1, scalar2, scalar3);
+        ValidateBetween<Type>(options, scalar1, array2, array3);
+        ValidateBetween<Type>(options, scalar1, array2, scalar3);
+        ValidateBetween<Type>(options, scalar1, scalar2, array3);
+        ValidateBetween<Type>(options, array1, scalar2, array3);
+      }
+    }
+  }
+};
+
+TEST(TestNumericBetweenKernel, BetweenPrimitiveRandomTests) {
+  TestRandomPrimitiveCTypes<BetweenRandomNumeric>();
+}
+
+class TestStringBetweenKernel : public ::testing::Test {};
+
+TEST(TestStringBetweenKernel, RandomBetween) {
+  using ScalarType = typename TypeTraits<StringType>::ScalarType;
+
+  auto rand = random::RandomArrayGenerator(0x5416447);
+  for (size_t i = 3; i < 10; i++) {
+    for (auto null_probability : {0.0, 0.01, 0.1, 0.25, 0.5, 1.0}) {
+      for (auto inclusive :
+           {BetweenOptions::Inclusive::BOTH, BetweenOptions::Inclusive::LEFT,
+            BetweenOptions::Inclusive::RIGHT, BetweenOptions::Inclusive::NEITHER}) {
+        const int64_t length = static_cast<int64_t>(1ULL << i);
+        auto array1 = Datum(rand.String(length, 0, 16, null_probability));
+        auto array2 = Datum(rand.String(length, 0, 16, null_probability));
+        auto array3 = Datum(rand.String(length, 0, 16, null_probability));
+        auto scalar1 = Datum(std::make_shared<ScalarType>("fupi"));
+        auto scalar2 = Datum(std::make_shared<ScalarType>("tupu"));
+        auto scalar3 = Datum(std::make_shared<ScalarType>("zito"));
+        auto options = BetweenOptions(inclusive);
+        ValidateBetween<StringType>(options, array1, scalar2, scalar3);
+        ValidateBetween<StringType>(options, scalar1, array2, scalar3);
+        ValidateBetween<StringType>(options, scalar1, scalar2, array3);
+        ValidateBetween<StringType>(options, scalar1, array2, array3);
+        ValidateBetween<StringType>(options, array1, scalar2, array3);
+        ValidateBetween<StringType>(options, array1, array2, scalar3);
+        ValidateBetween<StringType>(options, array1, array2, array3);
+      }
+    }
+  }
+}
+
+TEST(TestStringBetweenKernel, StringArrayScalarScalarTest) {
+  using ScalarType = typename TypeTraits<StringType>::ScalarType;
+  auto l = Datum(std::make_shared<ScalarType>("abc"));
+  auto r = Datum(std::make_shared<ScalarType>("zzz"));
+  BetweenOptions InclusiveOption(BetweenOptions::Inclusive::BOTH);
+  ValidateBetween<StringType>(
+      InclusiveOption, ArrayFromJSON(TypeTraits<StringType>::type_singleton(), "[]"), l,
+      r, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<StringType>(
+      InclusiveOption, ArrayFromJSON(TypeTraits<StringType>::type_singleton(), "[null]"),
+      l, r, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<StringType>(InclusiveOption,
+                              ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                            R"(["aaa", "aaaa", "ccc", "z"])"),
+                              l, r,
+                              ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                                            R"([false, false, true, true])"));
+  ValidateBetween<StringType>(InclusiveOption,
+                              ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                            R"(["a", "aaaa", "c", "z"])"),
+                              l, r,
+                              ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                                            R"([false, false, true, true])"));
+  ValidateBetween<StringType>(InclusiveOption,
+                              ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                            R"(["a", "aaaa", "fff", "zzzz"])"),
+                              l, r,
+                              ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                                            R"([false, false, true, false])"));
+  ValidateBetween<StringType>(InclusiveOption,
+                              ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                            R"(["abc", "baa", "fff", "zzz"])"),
+                              l, r,
+                              ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                                            R"([true, true, true, true])"));
+  ValidateBetween<StringType>(InclusiveOption,
+                              ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                            R"(["abd", null, null, "zzx"])"),
+                              l, r,
+                              ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                                            R"([true, null, null, true])"));
+}
+
+TEST(TestStringBetweenKernel, StringArrayArrayArrayTest) {
+  BetweenOptions InclusiveOption(BetweenOptions::Inclusive::BOTH);
+  ValidateBetween<StringType>(
+      InclusiveOption,
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                    R"(["david","hello","world"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["adam","hi","whirl"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                    R"(["robert","goeiemoreen","whirlwind"])"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[true, false, false]"));
+  ValidateBetween<StringType>(
+      InclusiveOption,
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["x","a","f"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["w","a","e"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["z","a","g"])"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[true, true, true]"));
+  ValidateBetween<StringType>(
+      InclusiveOption,
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                    R"(["block","bit","binary"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                    R"(["bit","nibble","ternary"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["word","d","xyz"])"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[true, false, false]"));
+  ValidateBetween<StringType>(
+      InclusiveOption,
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                    R"(["よしもと","の","ち"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["は","へ","あ"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["な","を","ち"])"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[false, false, true]"));
+  ValidateBetween<StringType>(
+      InclusiveOption,
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["A","ア","王"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["た","あ","歩"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["李","田",null])"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[false, true, null]"));
+  ValidateBetween<StringType>(
+      InclusiveOption,
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["Б",null,"Я"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["А","Ж","Щ"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["Д","Л","Ф"])"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[true, null, false]"));
+}
+
+TEST(TestTimestampsBetweenKernel, TimestampsArrayArrayArrayTest) {
+  const std::string arr_json = R"(["1970-01-01","1980-02-02","1970-02-28"])";
+  const std::string lhs_json = R"(["1970-01-01","1980-02-01","1970-02-28"])";
+  const std::string rhs_json = R"(["1970-01-02","1980-02-02","1970-02-28"])";
+  // Between Options
+  BetweenOptions both(BetweenOptions::Inclusive::BOTH);
+  BetweenOptions left(BetweenOptions::Inclusive::LEFT);
+  BetweenOptions right(BetweenOptions::Inclusive::RIGHT);
+  BetweenOptions neither(BetweenOptions::Inclusive::NEITHER);
+  // Same units should be fine
+  ValidateBetween<TimestampType>(both,
+                                 ArrayFromJSON(timestamp(TimeUnit::SECOND), arr_json),
+                                 ArrayFromJSON(timestamp(TimeUnit::SECOND), lhs_json),
+                                 ArrayFromJSON(timestamp(TimeUnit::SECOND), rhs_json));
+  ValidateBetween<TimestampType>(left,
+                                 ArrayFromJSON(timestamp(TimeUnit::SECOND), arr_json),
+                                 ArrayFromJSON(timestamp(TimeUnit::SECOND), lhs_json),
+                                 ArrayFromJSON(timestamp(TimeUnit::SECOND), rhs_json));
+  ValidateBetween<TimestampType>(right,
+                                 ArrayFromJSON(timestamp(TimeUnit::SECOND), arr_json),
+                                 ArrayFromJSON(timestamp(TimeUnit::SECOND), lhs_json),
+                                 ArrayFromJSON(timestamp(TimeUnit::SECOND), rhs_json));
+  ValidateBetween<TimestampType>(neither,
+                                 ArrayFromJSON(timestamp(TimeUnit::SECOND), arr_json),
+                                 ArrayFromJSON(timestamp(TimeUnit::SECOND), lhs_json),
+                                 ArrayFromJSON(timestamp(TimeUnit::SECOND), rhs_json));
+  ValidateBetween<TimestampType>(both,
+                                 ArrayFromJSON(timestamp(TimeUnit::MILLI), arr_json),
+                                 ArrayFromJSON(timestamp(TimeUnit::MILLI), lhs_json),
+                                 ArrayFromJSON(timestamp(TimeUnit::MILLI), rhs_json));
+  ValidateBetween<TimestampType>(left,
+                                 ArrayFromJSON(timestamp(TimeUnit::MILLI), arr_json),
+                                 ArrayFromJSON(timestamp(TimeUnit::MILLI), lhs_json),
+                                 ArrayFromJSON(timestamp(TimeUnit::MILLI), rhs_json));
+  ValidateBetween<TimestampType>(right,
+                                 ArrayFromJSON(timestamp(TimeUnit::MILLI), arr_json),
+                                 ArrayFromJSON(timestamp(TimeUnit::MILLI), lhs_json),
+                                 ArrayFromJSON(timestamp(TimeUnit::MILLI), rhs_json));
+  ValidateBetween<TimestampType>(neither,
+                                 ArrayFromJSON(timestamp(TimeUnit::MILLI), arr_json),
+                                 ArrayFromJSON(timestamp(TimeUnit::MILLI), lhs_json),
+                                 ArrayFromJSON(timestamp(TimeUnit::MILLI), rhs_json));

Review comment:
       Have put these in a loop over the option.




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

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

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



[GitHub] [arrow] bkmgit commented on a change in pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
bkmgit commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r785186730



##########
File path: cpp/src/arrow/compute/kernels/scalar_compare_test.cc
##########
@@ -1865,5 +1861,636 @@ TEST(TestMaxElementWiseMinElementWise, CommonTemporal) {
               ResultWith(ScalarFromJSON(date64(), "86400000")));
 }
 
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const Datum& val, const Datum& lhs,
+                            const Datum& rhs, const Datum& expected) {
+  ASSERT_OK_AND_ASSIGN(Datum result, Between(val, lhs, rhs, options, nullptr));
+  AssertArraysEqual(*expected.make_array(), *result.make_array(),
+                    /*verbose=*/true);
+}
+
+template <typename ArrowType>
+void ValidateBetween(BetweenOptions options, const Datum& val, const Datum& lhs,
+                     const Datum& rhs) {
+  CompareOperator lhs_val;
+  CompareOperator val_rhs;
+  BetweenOptions::Inclusive include_endpoints = options.inclusive;
+
+  if (include_endpoints == BetweenOptions::Inclusive::NEITHER) {
+    lhs_val = LESS;
+    val_rhs = LESS;
+  } else if (include_endpoints == BetweenOptions::Inclusive::LEFT) {
+    lhs_val = LESS_EQUAL;
+    val_rhs = LESS;
+  } else if (include_endpoints == BetweenOptions::Inclusive::RIGHT) {
+    lhs_val = LESS;
+    val_rhs = LESS_EQUAL;
+  } else {
+    lhs_val = LESS_EQUAL;
+    val_rhs = LESS_EQUAL;
+  }
+
+  ASSERT_OK_AND_ASSIGN(Datum resultl,
+                       CallFunction(CompareOperatorToFunctionName(lhs_val), {lhs, val}));
+  ASSERT_OK_AND_ASSIGN(Datum resultr,
+                       CallFunction(CompareOperatorToFunctionName(val_rhs), {val, rhs}));
+  ASSERT_OK_AND_ASSIGN(Datum expected, CallFunction("and", {resultl, resultr}));
+
+  ValidateBetween<ArrowType>(options, val, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+class TestNumericBetweenKernel : public ::testing::Test {};
+
+TYPED_TEST_SUITE(TestNumericBetweenKernel, NumericArrowTypes);
+TYPED_TEST(TestNumericBetweenKernel, SimpleBetweenArrayScalarScalar) {
+  using ScalarType = typename TypeTraits<TypeParam>::ScalarType;
+  using CType = typename TypeTraits<TypeParam>::CType;
+
+  Datum zero(std::make_shared<ScalarType>(CType(0)));
+  Datum four(std::make_shared<ScalarType>(CType(4)));
+  Datum null(std::make_shared<ScalarType>());
+  BetweenOptions InclusiveBoth(BetweenOptions::Inclusive::BOTH);
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"), zero,
+      four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      zero, four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,1,2,2]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[1,1,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,3,4,5]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[1,1,1,1,1,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,1,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null,0,1,1]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), null, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, null,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  BetweenOptions InclusiveLeft(BetweenOptions::Inclusive::LEFT);
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"), zero,
+      four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      zero, four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,1,2,2]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[1,1,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,3,4,5]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[1,1,1,1,0,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,0,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null,0,1,1]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), null, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, null,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  BetweenOptions InclusiveRight(BetweenOptions::Inclusive::RIGHT);
+  ValidateBetween<TypeParam>(
+      InclusiveRight, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"), zero,
+      four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      zero, four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,1,2,2]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,0,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,3,4,5]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,1,1,1,1,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,1,1,1,1,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null,0,1,1]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null,0,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), null, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, null,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  BetweenOptions InclusiveNeither(BetweenOptions::Inclusive::NEITHER);
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      zero, four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      zero, four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,1,2,2]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,0,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,3,4,5]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,1,1,1,0,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,0,1,1,1,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null,0,1,1]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null,0,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), null, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, null,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+}
+
+TYPED_TEST(TestNumericBetweenKernel, SimpleBetweenScalarArrayArray) {
+  using ScalarType = typename TypeTraits<TypeParam>::ScalarType;
+  using CType = typename TypeTraits<TypeParam>::CType;
+
+  Datum one(std::make_shared<ScalarType>(CType(1)));
+
+  BetweenOptions InclusiveBoth(BetweenOptions::Inclusive::BOTH);
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, one, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,true,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,true,null,false,false]"));
+  BetweenOptions InclusiveLeft(BetweenOptions::Inclusive::LEFT);
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, one, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,true,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,true,null,false,false]"));
+  BetweenOptions InclusiveRight(BetweenOptions::Inclusive::RIGHT);
+  ValidateBetween<TypeParam>(
+      InclusiveRight, one, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,false,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,false,null,false,false]"));
+  BetweenOptions InclusiveNeither(BetweenOptions::Inclusive::NEITHER);
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, one, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,false,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,false,null,false,false]"));
+}
+
+TYPED_TEST(TestNumericBetweenKernel, SimpleBetweenArrayArrayArray) {
+  BetweenOptions InclusiveBoth(BetweenOptions::Inclusive::BOTH);
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[1,1,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,true,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,true,null,false,false]"));
+  BetweenOptions InclusiveLeft(BetweenOptions::Inclusive::LEFT);
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[1,1,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,false,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,false,null,false,false]"));
+  BetweenOptions InclusiveRight(BetweenOptions::Inclusive::RIGHT);
+  ValidateBetween<TypeParam>(
+      InclusiveRight, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[1,1,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,true,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,true,null,false,false]"));
+  BetweenOptions InclusiveNeither(BetweenOptions::Inclusive::NEITHER);
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[1,1,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,false,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,false,null,false,false]"));
+}
+
+template <typename Type>
+struct BetweenRandomNumeric {
+  static void Test(const std::shared_ptr<DataType>& type) {
+    using ScalarType = typename TypeTraits<Type>::ScalarType;
+    using CType = typename TypeTraits<Type>::CType;
+    auto rand = random::RandomArrayGenerator(0x5416447);
+    const int64_t length = 1000;
+    for (auto null_probability : {0.0, 0.01, 0.1, 0.25, 0.5, 1.0}) {
+      for (auto inclusive :
+           {BetweenOptions::Inclusive::BOTH, BetweenOptions::Inclusive::LEFT,
+            BetweenOptions::Inclusive::RIGHT, BetweenOptions::Inclusive::NEITHER}) {
+        auto data1 =
+            rand.Numeric<typename Type::PhysicalType>(length, 0, 100, null_probability);
+        auto data2 =
+            rand.Numeric<typename Type::PhysicalType>(length, 0, 100, null_probability);
+        auto data3 =
+            rand.Numeric<typename Type::PhysicalType>(length, 0, 100, null_probability);
+
+        // Create view of data as the type (e.g. timestamp)
+        auto array1 = Datum(*data1->View(type));
+        auto array2 = Datum(*data2->View(type));
+        auto array3 = Datum(*data3->View(type));
+        auto scalar1 = Datum(std::make_shared<ScalarType>(CType(10), type));
+        auto scalar2 = Datum(std::make_shared<ScalarType>(CType(30), type));
+        auto scalar3 = Datum(std::make_shared<ScalarType>(CType(50), type));
+        auto options = BetweenOptions(inclusive);
+        ValidateBetween<Type>(options, array1, scalar2, scalar3);
+        ValidateBetween<Type>(options, array1, array2, scalar3);
+        ValidateBetween<Type>(options, array1, array2, array3);
+        ValidateBetween<Type>(options, array1, scalar2, scalar3);
+        ValidateBetween<Type>(options, scalar1, array2, array3);
+        ValidateBetween<Type>(options, scalar1, array2, scalar3);
+        ValidateBetween<Type>(options, scalar1, scalar2, array3);
+        ValidateBetween<Type>(options, array1, scalar2, array3);
+      }
+    }
+  }
+};
+
+TEST(TestNumericBetweenKernel, BetweenPrimitiveRandomTests) {
+  TestRandomPrimitiveCTypes<BetweenRandomNumeric>();
+}
+
+class TestStringBetweenKernel : public ::testing::Test {};
+
+TEST(TestStringBetweenKernel, RandomBetween) {
+  using ScalarType = typename TypeTraits<StringType>::ScalarType;
+
+  auto rand = random::RandomArrayGenerator(0x5416447);
+  for (size_t i = 3; i < 10; i++) {
+    for (auto null_probability : {0.0, 0.01, 0.1, 0.25, 0.5, 1.0}) {
+      for (auto inclusive :
+           {BetweenOptions::Inclusive::BOTH, BetweenOptions::Inclusive::LEFT,
+            BetweenOptions::Inclusive::RIGHT, BetweenOptions::Inclusive::NEITHER}) {
+        const int64_t length = static_cast<int64_t>(1ULL << i);
+        auto array1 = Datum(rand.String(length, 0, 16, null_probability));
+        auto array2 = Datum(rand.String(length, 0, 16, null_probability));
+        auto array3 = Datum(rand.String(length, 0, 16, null_probability));
+        auto scalar1 = Datum(std::make_shared<ScalarType>("fupi"));
+        auto scalar2 = Datum(std::make_shared<ScalarType>("tupu"));
+        auto scalar3 = Datum(std::make_shared<ScalarType>("zito"));
+        auto options = BetweenOptions(inclusive);
+        ValidateBetween<StringType>(options, array1, scalar2, scalar3);
+        ValidateBetween<StringType>(options, scalar1, array2, scalar3);
+        ValidateBetween<StringType>(options, scalar1, scalar2, array3);
+        ValidateBetween<StringType>(options, scalar1, array2, array3);
+        ValidateBetween<StringType>(options, array1, scalar2, array3);
+        ValidateBetween<StringType>(options, array1, array2, scalar3);
+        ValidateBetween<StringType>(options, array1, array2, array3);
+      }
+    }
+  }
+}
+
+TEST(TestStringBetweenKernel, StringArrayScalarScalarTest) {
+  using ScalarType = typename TypeTraits<StringType>::ScalarType;
+  auto l = Datum(std::make_shared<ScalarType>("abc"));
+  auto r = Datum(std::make_shared<ScalarType>("zzz"));
+  BetweenOptions InclusiveOption(BetweenOptions::Inclusive::BOTH);
+  ValidateBetween<StringType>(
+      InclusiveOption, ArrayFromJSON(TypeTraits<StringType>::type_singleton(), "[]"), l,
+      r, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<StringType>(
+      InclusiveOption, ArrayFromJSON(TypeTraits<StringType>::type_singleton(), "[null]"),
+      l, r, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<StringType>(InclusiveOption,
+                              ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                            R"(["aaa", "aaaa", "ccc", "z"])"),
+                              l, r,
+                              ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                                            R"([false, false, true, true])"));
+  ValidateBetween<StringType>(InclusiveOption,
+                              ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                            R"(["a", "aaaa", "c", "z"])"),
+                              l, r,
+                              ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                                            R"([false, false, true, true])"));
+  ValidateBetween<StringType>(InclusiveOption,
+                              ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                            R"(["a", "aaaa", "fff", "zzzz"])"),
+                              l, r,
+                              ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                                            R"([false, false, true, false])"));
+  ValidateBetween<StringType>(InclusiveOption,
+                              ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                            R"(["abc", "baa", "fff", "zzz"])"),
+                              l, r,
+                              ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                                            R"([true, true, true, true])"));
+  ValidateBetween<StringType>(InclusiveOption,
+                              ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                            R"(["abd", null, null, "zzx"])"),
+                              l, r,
+                              ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                                            R"([true, null, null, true])"));
+}
+
+TEST(TestStringBetweenKernel, StringArrayArrayArrayTest) {
+  BetweenOptions InclusiveOption(BetweenOptions::Inclusive::BOTH);
+  ValidateBetween<StringType>(
+      InclusiveOption,
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                    R"(["david","hello","world"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["adam","hi","whirl"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                    R"(["robert","goeiemoreen","whirlwind"])"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[true, false, false]"));
+  ValidateBetween<StringType>(
+      InclusiveOption,
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["x","a","f"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["w","a","e"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["z","a","g"])"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[true, true, true]"));
+  ValidateBetween<StringType>(
+      InclusiveOption,
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                    R"(["block","bit","binary"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                    R"(["bit","nibble","ternary"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["word","d","xyz"])"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[true, false, false]"));
+  ValidateBetween<StringType>(
+      InclusiveOption,
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                    R"(["よしもと","の","ち"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["は","へ","あ"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["な","を","ち"])"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[false, false, true]"));
+  ValidateBetween<StringType>(
+      InclusiveOption,
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["A","ア","王"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["た","あ","歩"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["李","田",null])"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[false, true, null]"));
+  ValidateBetween<StringType>(
+      InclusiveOption,
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["Б",null,"Я"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["А","Ж","Щ"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["Д","Л","Ф"])"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[true, null, false]"));
+}
+
+TEST(TestTimestampsBetweenKernel, TimestampsArrayArrayArrayTest) {
+  const std::string arr_json = R"(["1970-01-01","2000-02-02","1900-02-28"])";
+  const std::string lhs_json = R"(["1970-01-01","2000-02-01","1900-02-28"])";
+  const std::string rhs_json = R"(["1970-01-02","2000-02-02","1900-02-28"])";
+  // Between Options
+  BetweenOptions both(BetweenOptions::Inclusive::BOTH);
+  BetweenOptions left(BetweenOptions::Inclusive::LEFT);
+  BetweenOptions right(BetweenOptions::Inclusive::RIGHT);
+  BetweenOptions neither(BetweenOptions::Inclusive::NEITHER);
+  // Same units should be fine
+  auto arr = ArrayFromJSON(timestamp(TimeUnit::SECOND), arr_json);
+  auto lhs = ArrayFromJSON(timestamp(TimeUnit::SECOND), lhs_json);
+  auto rhs = ArrayFromJSON(timestamp(TimeUnit::SECOND), rhs_json);
+  // BOTH
+  ValidateBetween<TimestampType>(both, arr, lhs, rhs);
+  // LEFT
+  ValidateBetween<TimestampType>(left, arr, lhs, rhs);
+  // RIGHT
+  ValidateBetween<TimestampType>(right, arr, lhs, rhs);
+  // NEITHER
+  ValidateBetween<TimestampType>(neither, arr, lhs, rhs);
+/*  // So are same time zones
+  arr = ArrayFromJSON(timestamp(TimeUnit::SECOND, "America/Chicago"), arr_json);
+  lhs = ArrayFromJSON(timestamp(TimeUnit::SECOND, "America/Chicago"), lhs_json);
+  rhs = ArrayFromJSON(timestamp(TimeUnit::SECOND, "America/Chicago"), rhs_json);
+  // BOTH
+  ValidateBetween<TimestampType>(both, arr, lhs, rhs);
+  // LEFT
+  ValidateBetween<TimestampType>(left, arr, lhs, rhs);
+  // RIGHT
+  ValidateBetween<TimestampType>(right, arr, lhs, rhs);
+  // NEITHER
+  ValidateBetween<TimestampType>(neither, arr, lhs, rhs);
+  // Different units should be fine
+  auto arr = ArrayFromJSON(timestamp(TimeUnit::SECOND), arr_json);
+  auto lhs = ArrayFromJSON(timestamp(TimeUnit::MILLI), lhs_json);
+  auto rhs = ArrayFromJSON(timestamp(TimeUnit::SECOND), rhs_json);
+  // BOTH

Review comment:
       Some fixes are still needed to pass tests,  (timezones are fine, but timestamps with different units are not), otherwise most of the functionality is there now.




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

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

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



[GitHub] [arrow] bkmgit commented on a change in pull request #11882: ARROW-9843: [C++][Python] Implement Between ternary kernel and Python bindings

Posted by GitBox <gi...@apache.org>.
bkmgit commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r793890393



##########
File path: cpp/src/arrow/ipc/json_simple.cc
##########
@@ -983,6 +983,16 @@ Status ScalarFromJSON(const std::shared_ptr<DataType>& type,
   return Status::OK();
 }
 
+Status ScalarFromJSON(const std::shared_ptr<DataType>& type,

Review comment:
       Removed thanks.




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

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

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



[GitHub] [arrow] kou commented on a change in pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
kou commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r767427799



##########
File path: cpp/src/arrow/compute/kernels/scalar_compare.cc
##########
@@ -786,6 +878,10 @@ const FunctionDoc max_element_wise_doc{
     {"*args"},
     "ElementWiseAggregateOptions"};
 
+const FunctionDoc between_doc{"Check if values are in a range x <= y <= z",

Review comment:
       I think that `x <=  y <= z` is better for the default because pandas and SQL's `BETWEEN` use it.
   But the implementation in this pull request uses `x < y < z`.




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

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

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



[GitHub] [arrow] bkmgit commented on a change in pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
bkmgit commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r767650639



##########
File path: cpp/src/arrow/util/bit_block_counter.h
##########
@@ -586,5 +961,144 @@ static void VisitTwoBitBlocksVoid(const std::shared_ptr<Buffer>& left_bitmap_buf
   }
 }
 
+template <typename VisitNotNull, typename VisitNull>
+static void VisitThreeBitBlocksVoid(
+    const std::shared_ptr<Buffer>& left_bitmap_buf, int64_t left_offset,
+    const std::shared_ptr<Buffer>& mid_bitmap_buf, int64_t mid_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) && (mid_bitmap_buf == NULLPTR)) ||
+      ((mid_bitmap_buf == NULLPTR) && (right_bitmap_buf == NULLPTR)) ||
+      ((left_bitmap_buf == NULLPTR) && (right_bitmap_buf == NULLPTR))) {
+    // At most one bitmap is present
+    if ((left_bitmap_buf == NULLPTR) && (mid_bitmap_buf == NULLPTR)) {
+      return VisitBitBlocksVoid(right_bitmap_buf, right_offset, length,
+                                std::forward<VisitNotNull>(visit_not_null),
+                                std::forward<VisitNull>(visit_null));
+    } else if ((mid_bitmap_buf == NULLPTR) && (right_bitmap_buf == NULLPTR)) {
+      return VisitBitBlocksVoid(left_bitmap_buf, left_offset, length,
+                                std::forward<VisitNotNull>(visit_not_null),
+                                std::forward<VisitNull>(visit_null));
+    } else {
+      return VisitBitBlocksVoid(mid_bitmap_buf, mid_offset, length,
+                                std::forward<VisitNotNull>(visit_not_null),
+                                std::forward<VisitNull>(visit_null));
+    }
+  }
+  // Two bitmaps are present
+  if (left_bitmap_buf == NULLPTR) {
+    const uint8_t* mid_bitmap = mid_bitmap_buf->data();

Review comment:
       Thanks for the suggestion.




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

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

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



[GitHub] [arrow] bkmgit commented on a change in pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
bkmgit commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r768777465



##########
File path: cpp/src/arrow/compute/kernels/scalar_compare_test.cc
##########
@@ -1850,5 +1851,154 @@ TEST(TestMaxElementWiseMinElementWise, CommonTemporal) {
               ResultWith(ScalarFromJSON(date64(), "86400000")));
 }
 
+template <typename ArrowType>
+static void ValidateBetween(const Datum& val, const Datum& lhs, const Datum& rhs,
+                            const Datum& expected) {
+  ASSERT_OK_AND_ASSIGN(Datum result, Between(val, lhs, rhs));
+  AssertArraysEqual(*expected.make_array(), *result.make_array(),
+                    /*verbose=*/true);
+}
+
+template <typename ArrowType>
+static void ValidateBetween(const char* value_str, const Datum& lhs, const Datum& rhs,
+                            const char* expected_str) {
+  auto value = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), value_str);
+  auto expected = ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), expected_str);
+  ValidateBetween<ArrowType>(value, lhs, rhs, expected);
+}
+
+template <>
+void ValidateBetween<StringType>(const char* value_str, const Datum& lhs,
+                                 const Datum& rhs, const char* expected_str) {
+  auto value = ArrayFromJSON(utf8(), value_str);
+  auto expected = ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), expected_str);
+  ValidateBetween<StringType>(value, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+class TestNumericBetweenKernel : public ::testing::Test {};
+
+TYPED_TEST_SUITE(TestNumericBetweenKernel, NumericArrowTypes);
+TYPED_TEST(TestNumericBetweenKernel, SimpleBetweenArrayScalarScalar) {
+  using ScalarType = typename TypeTraits<TypeParam>::ScalarType;
+  using CType = typename TypeTraits<TypeParam>::CType;
+
+  Datum zero(std::make_shared<ScalarType>(CType(0)));
+  Datum four(std::make_shared<ScalarType>(CType(4)));
+  ValidateBetween<TypeParam>("[]", zero, four, "[]");
+  ValidateBetween<TypeParam>("[null]", zero, four, "[null]");
+  ValidateBetween<TypeParam>("[0,0,1,1,2,2]", zero, four, "[0,0,1,1,1,1]");
+  ValidateBetween<TypeParam>("[0,1,2,3,4,5]", zero, four, "[0,1,1,1,0,0]");
+  ValidateBetween<TypeParam>("[5,4,3,2,1,0]", zero, four, "[0,0,1,1,1,0]");
+  ValidateBetween<TypeParam>("[null,0,1,1]", zero, four, "[null,0,1,1]");
+}
+
+TEST(TestSimpleBetweenKernel, SimpleStringTest) {
+  using ScalarType = typename TypeTraits<StringType>::ScalarType;
+  auto l = Datum(std::make_shared<ScalarType>("abc"));
+  auto r = Datum(std::make_shared<ScalarType>("zzz"));
+  ValidateBetween<StringType>("[]", l, r, "[]");
+  ValidateBetween<StringType>("[null]", l, r, "[null]");
+  ValidateBetween<StringType>(R"(["aaa", "aaaa", "ccc", "z"])", l, r,
+                              R"([false, false, true, true])");
+  ValidateBetween<StringType>(R"(["a", "aaaa", "c", "z"])", l, r,
+                              R"([false, false, true, true])");
+  ValidateBetween<StringType>(R"(["a", "aaaa", "fff", "zzzz"])", l, r,
+                              R"([false, false, true, false])");
+  ValidateBetween<StringType>(R"(["abd", null, null, "zzx"])", l, r,
+                              R"([true, null, null, true])");
+}
+
+TEST(TestSimpleBetweenKernel, SimpleTimestampTest) {
+  using ScalarType = typename TypeTraits<TimestampType>::ScalarType;
+  auto checkTimestampArray = [](std::shared_ptr<DataType> type, const char* input_str,
+                                const Datum& lhs, const Datum& rhs,
+                                const char* expected_str) {
+    auto value = ArrayFromJSON(type, input_str);
+    auto expected = ArrayFromJSON(boolean(), expected_str);
+    ValidateBetween<TimestampType>(value, lhs, rhs, expected);
+  };
+  auto unit = TimeUnit::SECOND;
+  auto l = Datum(std::make_shared<ScalarType>(923184000, timestamp(unit)));
+  auto r = Datum(std::make_shared<ScalarType>(1602032602, timestamp(unit)));
+  checkTimestampArray(timestamp(unit), "[]", l, r, "[]");
+  checkTimestampArray(timestamp(unit), "[null]", l, r, "[null]");
+  checkTimestampArray(timestamp(unit), R"(["1970-01-01","2000-02-29","1900-02-28"])", l,
+                      r, "[false,true,false]");
+  checkTimestampArray(timestamp(unit), R"(["1970-01-01","2000-02-29","2004-02-28"])", l,
+                      r, "[false,true,true]");
+  checkTimestampArray(timestamp(unit), R"(["2018-01-01","1999-04-04","1900-02-28"])", l,
+                      r, "[true,false,false]");
+}
+
+TYPED_TEST(TestNumericBetweenKernel, SimpleBetweenArrayArrayArray) {

Review comment:
       Ok.




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

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

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



[GitHub] [arrow] lidavidm commented on a change in pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
lidavidm commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r775898924



##########
File path: cpp/src/arrow/compute/kernels/scalar_compare.cc
##########
@@ -68,6 +72,50 @@ struct GreaterEqual {
   }
 };
 
+struct BetweenLessEqualLessEqual {
+  template <typename T, typename Arg0, typename Arg1, typename Arg2>
+  static constexpr T Call(KernelContext*, const Arg0& middle, const Arg1& left,
+                          const Arg2& right, Status*) {
+    static_assert(std::is_same<T, bool>::value && std::is_same<Arg0, Arg1>::value &&
+                      std::is_same<Arg1, Arg2>::value,
+                  "");
+    return (left <= middle) && (middle <= right);
+  }
+};
+
+struct BetweenLessThanLessEqual {
+  template <typename T, typename Arg0, typename Arg1, typename Arg2>
+  static constexpr T Call(KernelContext*, const Arg0& middle, const Arg1& left,
+                          const Arg2& right, Status*) {
+    static_assert(std::is_same<T, bool>::value && std::is_same<Arg0, Arg1>::value &&
+                      std::is_same<Arg1, Arg2>::value,
+                  "");
+    return (left < middle) && (middle <= right);
+  }
+};
+
+struct BetweenLessEqualLessThan {
+  template <typename T, typename Arg0, typename Arg1, typename Arg2>
+  static constexpr T Call(KernelContext*, const Arg0& middle, const Arg1& left,
+                          const Arg2& right, Status*) {
+    static_assert(std::is_same<T, bool>::value && std::is_same<Arg0, Arg1>::value &&
+                      std::is_same<Arg1, Arg2>::value,
+                  "");
+    return (left <= middle) && (middle < right);
+  }
+};
+
+struct BetweenLessThanLessThan {
+  template <typename T, typename Arg0, typename Arg1, typename Arg2>
+  static constexpr T Call(KernelContext*, const Arg0& middle, const Arg1& left,
+                          const Arg2& right, Status*) {
+    static_assert(std::is_same<T, bool>::value && std::is_same<Arg0, Arg1>::value &&
+                      std::is_same<Arg1, Arg2>::value,
+                  "");
+    return (left < middle) && (middle < right);
+  }
+};

Review comment:
       I don't think we want to keep the structure of the compare functions though. CompareOptions is going to be removed (ARROW-13378) and CompareFunction is really only there to make tests more convenient. I don't think we need to make between act like the compare functions, and I still agree with Eduardo that a structure like the round function makes more sense.




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

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

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



[GitHub] [arrow] edponce commented on a change in pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
edponce commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r776495341



##########
File path: cpp/src/arrow/compute/kernels/scalar_compare.cc
##########
@@ -68,6 +72,50 @@ struct GreaterEqual {
   }
 };
 
+struct BetweenLessEqualLessEqual {
+  template <typename T, typename Arg0, typename Arg1, typename Arg2>
+  static constexpr T Call(KernelContext*, const Arg0& middle, const Arg1& left,
+                          const Arg2& right, Status*) {
+    static_assert(std::is_same<T, bool>::value && std::is_same<Arg0, Arg1>::value &&
+                      std::is_same<Arg1, Arg2>::value,
+                  "");
+    return (left <= middle) && (middle <= right);
+  }
+};
+
+struct BetweenLessThanLessEqual {
+  template <typename T, typename Arg0, typename Arg1, typename Arg2>
+  static constexpr T Call(KernelContext*, const Arg0& middle, const Arg1& left,
+                          const Arg2& right, Status*) {
+    static_assert(std::is_same<T, bool>::value && std::is_same<Arg0, Arg1>::value &&
+                      std::is_same<Arg1, Arg2>::value,
+                  "");
+    return (left < middle) && (middle <= right);
+  }
+};
+
+struct BetweenLessEqualLessThan {
+  template <typename T, typename Arg0, typename Arg1, typename Arg2>
+  static constexpr T Call(KernelContext*, const Arg0& middle, const Arg1& left,
+                          const Arg2& right, Status*) {
+    static_assert(std::is_same<T, bool>::value && std::is_same<Arg0, Arg1>::value &&
+                      std::is_same<Arg1, Arg2>::value,
+                  "");
+    return (left <= middle) && (middle < right);
+  }
+};
+
+struct BetweenLessThanLessThan {
+  template <typename T, typename Arg0, typename Arg1, typename Arg2>
+  static constexpr T Call(KernelContext*, const Arg0& middle, const Arg1& left,
+                          const Arg2& right, Status*) {
+    static_assert(std::is_same<T, bool>::value && std::is_same<Arg0, Arg1>::value &&
+                      std::is_same<Arg1, Arg2>::value,
+                  "");
+    return (left < middle) && (middle < right);
+  }
+};

Review comment:
       AFAIK, Arrow string transformations operate on ASCII and UTF-8 encodings. Support for any other encodings would need to be discussed further as its implications are not trivial wrt to implementation.




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

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

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



[GitHub] [arrow] bkmgit commented on a change in pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
bkmgit commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r769434179



##########
File path: cpp/src/arrow/compute/kernels/scalar_compare_test.cc
##########
@@ -1850,5 +1851,154 @@ TEST(TestMaxElementWiseMinElementWise, CommonTemporal) {
               ResultWith(ScalarFromJSON(date64(), "86400000")));
 }
 
+template <typename ArrowType>
+static void ValidateBetween(const Datum& val, const Datum& lhs, const Datum& rhs,
+                            const Datum& expected) {
+  ASSERT_OK_AND_ASSIGN(Datum result, Between(val, lhs, rhs));
+  AssertArraysEqual(*expected.make_array(), *result.make_array(),
+                    /*verbose=*/true);
+}
+
+template <typename ArrowType>
+static void ValidateBetween(const char* value_str, const Datum& lhs, const Datum& rhs,
+                            const char* expected_str) {
+  auto value = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), value_str);
+  auto expected = ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), expected_str);
+  ValidateBetween<ArrowType>(value, lhs, rhs, expected);
+}
+
+template <>
+void ValidateBetween<StringType>(const char* value_str, const Datum& lhs,
+                                 const Datum& rhs, const char* expected_str) {
+  auto value = ArrayFromJSON(utf8(), value_str);
+  auto expected = ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), expected_str);
+  ValidateBetween<StringType>(value, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+class TestNumericBetweenKernel : public ::testing::Test {};
+
+TYPED_TEST_SUITE(TestNumericBetweenKernel, NumericArrowTypes);
+TYPED_TEST(TestNumericBetweenKernel, SimpleBetweenArrayScalarScalar) {
+  using ScalarType = typename TypeTraits<TypeParam>::ScalarType;
+  using CType = typename TypeTraits<TypeParam>::CType;
+
+  Datum zero(std::make_shared<ScalarType>(CType(0)));
+  Datum four(std::make_shared<ScalarType>(CType(4)));
+  ValidateBetween<TypeParam>("[]", zero, four, "[]");
+  ValidateBetween<TypeParam>("[null]", zero, four, "[null]");
+  ValidateBetween<TypeParam>("[0,0,1,1,2,2]", zero, four, "[0,0,1,1,1,1]");
+  ValidateBetween<TypeParam>("[0,1,2,3,4,5]", zero, four, "[0,1,1,1,0,0]");
+  ValidateBetween<TypeParam>("[5,4,3,2,1,0]", zero, four, "[0,0,1,1,1,0]");
+  ValidateBetween<TypeParam>("[null,0,1,1]", zero, four, "[null,0,1,1]");

Review comment:
       Implemented.




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

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

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



[GitHub] [arrow] edponce edited a comment on pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
edponce edited a comment on pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#issuecomment-1002956781


   I have the following observations to make wrt to the API for this `between` compute function. The purpose of `between` is to support range comparisons similar to [SQL `BETWEEN`](https://www.w3schools.com/sql/sql_between.asp) and [pandas `between`](https://pandas.pydata.org/docs/reference/api/pandas.Series.between.html#pandas-series-between). These APIs are different from this PR's implementation, but the behavior is equivalent. Nevertheless, I suggest either of the following:
   1. Implement `between` compute function not as a _logical compare_ function but as an independent function that takes 3 inputs and an option for declaring bounds-inclusiveness. Similar to the Pandas API.
   2. Implement `between` as an extension to the _logical compare_ functions. Now there are two ways to go about this:
     a. As this PR does, implement 4 new compare functions `less_less`, `less_equal_less`, `less_less_equal`, and `less_equal_less_equal`.
     b. Have a single variadic `compare` function that uses options to select the type of comparison (e.g., LESS, GREATER_EQUAL, LESS_LESS)
   
   I recommend to follow option (1), for the following reasons. It is a common API and the function name clearly states its operation. The issue with (2a) is that its function names are not common-case, and if we make a single `between` with options for LESS_LESS, etc. then I would argue that the other logical comparisons need to be merged into a single compute function for symmetry/consistency purposes. This last point touches on option (2b) which I do not think is a reasonable solution because of its complexity and one could argue that an analogous case would be to have a single `arithmetic` function where the operation performed is specified via options. Obviously, this is not desired.


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

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

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



[GitHub] [arrow] bkmgit commented on a change in pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
bkmgit commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r776651384



##########
File path: cpp/src/arrow/compute/api_scalar.cc
##########
@@ -506,6 +537,12 @@ void RegisterScalarOptions(FunctionRegistry* registry) {
     return CallFunction(REGISTRY_NAME, {left, right}, ctx);                     \
   }
 
+#define SCALAR_EAGER_TERNARY(NAME, REGISTRY_NAME)                               \
+  Result<Datum> NAME(const Datum& value, const Datum& left, const Datum& right, \
+                     ExecContext* ctx) {                                        \
+    return CallFunction(REGISTRY_NAME, {value, left, right}, ctx);              \
+  }
+

Review comment:
       Done




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

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

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



[GitHub] [arrow] edponce edited a comment on pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
edponce edited a comment on pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#issuecomment-1002956781


   I have the following observations to make wrt to the API for this `between` compute function. The purpose of `between` is to support range comparisons similar to [SQL `BETWEEN`](https://www.w3schools.com/sql/sql_between.asp) and [pandas `between`](https://pandas.pydata.org/docs/reference/api/pandas.Series.between.html#pandas-series-between). These APIs are different from this PR's implementation, but the behavior is equivalent. Nevertheless, I suggest either of the following:
   1. Implement `between` compute function not as a _logical compare_ function but as an independent range-based function that takes 3 inputs and an option for declaring bounds-inclusiveness (BOTH, NONE, LEFT, RIGHT). Similar to the Pandas API.
   2. Implement `between` as an extension to the _logical compare_ functions. Now there are two ways to go about this:
     a. As this PR does, implement 4 new compare functions `less_less`, `less_equal_less`, `less_less_equal`, and `less_equal_less_equal`.
     b. Have a single variadic `compare` function that uses options to select the type of comparison (e.g., LESS, GREATER_EQUAL, LESS_LESS)
   
   I recommend to follow option (1), for the following reasons. It is a common API and the function name clearly states its operation. The issue with (2a) is that its function names are not common-case, and if we make a single `between` with options for LESS_LESS, etc. then I would argue that the other logical comparisons need to be merged into a single compute function for symmetry/consistency purposes. This last point touches on option (2b) which I do not think is a reasonable solution because of its complexity and one could argue that an analogous case would be to have a single `arithmetic` function where the operation performed is specified via options. Obviously, this is not desired.


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

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

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



[GitHub] [arrow] lidavidm commented on pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
lidavidm commented on pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#issuecomment-1003017231


   IMO it's less ergonomic for functions like add and less to have a consolidated function, it'd be `CallFunction("add", {a, b})` vs 
   
   ```
   ArithmeticOptions options(ArithmeticOptions::ADD);
   CallFunction("arithmetic", {a, b}, &options);
   ```
   
   and note this difference would also be reflected in Expressions and how they get serialized


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

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

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



[GitHub] [arrow] edponce commented on a change in pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
edponce commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r776495341



##########
File path: cpp/src/arrow/compute/kernels/scalar_compare.cc
##########
@@ -68,6 +72,50 @@ struct GreaterEqual {
   }
 };
 
+struct BetweenLessEqualLessEqual {
+  template <typename T, typename Arg0, typename Arg1, typename Arg2>
+  static constexpr T Call(KernelContext*, const Arg0& middle, const Arg1& left,
+                          const Arg2& right, Status*) {
+    static_assert(std::is_same<T, bool>::value && std::is_same<Arg0, Arg1>::value &&
+                      std::is_same<Arg1, Arg2>::value,
+                  "");
+    return (left <= middle) && (middle <= right);
+  }
+};
+
+struct BetweenLessThanLessEqual {
+  template <typename T, typename Arg0, typename Arg1, typename Arg2>
+  static constexpr T Call(KernelContext*, const Arg0& middle, const Arg1& left,
+                          const Arg2& right, Status*) {
+    static_assert(std::is_same<T, bool>::value && std::is_same<Arg0, Arg1>::value &&
+                      std::is_same<Arg1, Arg2>::value,
+                  "");
+    return (left < middle) && (middle <= right);
+  }
+};
+
+struct BetweenLessEqualLessThan {
+  template <typename T, typename Arg0, typename Arg1, typename Arg2>
+  static constexpr T Call(KernelContext*, const Arg0& middle, const Arg1& left,
+                          const Arg2& right, Status*) {
+    static_assert(std::is_same<T, bool>::value && std::is_same<Arg0, Arg1>::value &&
+                      std::is_same<Arg1, Arg2>::value,
+                  "");
+    return (left <= middle) && (middle < right);
+  }
+};
+
+struct BetweenLessThanLessThan {
+  template <typename T, typename Arg0, typename Arg1, typename Arg2>
+  static constexpr T Call(KernelContext*, const Arg0& middle, const Arg1& left,
+                          const Arg2& right, Status*) {
+    static_assert(std::is_same<T, bool>::value && std::is_same<Arg0, Arg1>::value &&
+                      std::is_same<Arg1, Arg2>::value,
+                  "");
+    return (left < middle) && (middle < right);
+  }
+};

Review comment:
       AFAIK, Arrow string transformations operate on ASCII and UTF-8 encodings. Support for any other encodings would need to be discussed further as its implications are not trivial.




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

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

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



[GitHub] [arrow] lidavidm commented on pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
lidavidm commented on pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#issuecomment-1003037496


   Ah, thanks. I would not prefer 2b.


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

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

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



[GitHub] [arrow] bkmgit commented on a change in pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
bkmgit commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r774571193



##########
File path: cpp/src/arrow/compute/kernels/scalar_compare_test.cc
##########
@@ -1850,5 +1846,439 @@ TEST(TestMaxElementWiseMinElementWise, CommonTemporal) {
               ResultWith(ScalarFromJSON(date64(), "86400000")));
 }
 
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const Datum& val, const Datum& lhs,
+                            const Datum& rhs, const Datum& expected) {
+  ASSERT_OK_AND_ASSIGN(
+      Datum result,
+      CallFunction(BetweenOperatorToFunctionName(options.op), {val, lhs, rhs}));
+  AssertArraysEqual(*expected.make_array(), *result.make_array(),
+                    /*verbose=*/true);
+}
+
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const Datum& val, const Datum& lhs,
+                            const Datum& rhs, const char* expected_str) {
+  auto expected = ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), expected_str);
+  ValidateBetween<ArrowType>(options, val, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const Datum& val, const Datum& lhs,
+                            const char* rhs_str, const Datum& expected) {
+  auto rhs = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), rhs_str);
+  ValidateBetween<ArrowType>(options, val, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const Datum& val, const char* lhs_str,
+                            const Datum& rhs, const Datum& expected) {
+  auto lhs = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), lhs_str);
+  ValidateBetween<ArrowType>(options, val, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const char* val_str, const Datum& lhs,
+                            const Datum& rhs, const Datum& expected) {
+  auto val = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), val_str);
+  ValidateBetween<ArrowType>(options, val, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const Datum& val, const Datum& lhs,
+                            const char* rhs_str, const char* expected_str) {
+  auto rhs = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), rhs_str);
+  auto expected = ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), expected_str);
+  ValidateBetween<ArrowType>(options, val, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const Datum& val, const char* lhs_str,
+                            const Datum& rhs, const char* expected_str) {
+  auto lhs = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), lhs_str);
+  auto expected = ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), expected_str);
+  ValidateBetween<ArrowType>(options, val, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const Datum& val, const char* lhs_str,
+                            const char* rhs_str, const Datum& expected) {
+  auto lhs = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), lhs_str);
+  auto rhs = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), rhs_str);
+  ValidateBetween<ArrowType>(options, val, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const char* val_str, const Datum& lhs,
+                            const Datum& rhs, const char* expected_str) {
+  auto val = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), val_str);
+  auto expected = ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), expected_str);
+  ValidateBetween<ArrowType>(options, val, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const char* val_str,
+                            const char* lhs_str, const Datum& rhs,
+                            const Datum& expected) {
+  auto val = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), val_str);
+  auto lhs = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), lhs_str);
+  ValidateBetween<ArrowType>(options, val, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const char* val_str, const Datum& lhs,
+                            const char* rhs_str, const Datum& expected) {
+  auto val = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), val_str);
+  auto rhs = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), rhs_str);
+  ValidateBetween<ArrowType>(options, val, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const Datum& val, const char* lhs_str,
+                            const char* rhs_str, const char* expected_str) {
+  auto lhs = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), lhs_str);
+  auto rhs = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), rhs_str);
+  auto expected = ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), expected_str);
+  ValidateBetween<ArrowType>(options, val, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const char* val_str, const Datum& lhs,
+                            const char* rhs_str, const char* expected_str) {
+  auto val = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), val_str);
+  auto rhs = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), rhs_str);
+  auto expected = ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), expected_str);
+  ValidateBetween<ArrowType>(options, val, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const char* val_str,
+                            const char* lhs_str, const Datum& rhs,
+                            const char* expected_str) {
+  auto val = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), val_str);
+  auto lhs = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), lhs_str);
+  auto expected = ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), expected_str);
+  ValidateBetween<ArrowType>(options, val, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const char* val_str,
+                            const char* lhs_str, const char* rhs_str,
+                            const Datum& expected) {
+  auto val = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), val_str);
+  auto lhs = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), lhs_str);
+  auto rhs = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), rhs_str);
+  ValidateBetween<ArrowType>(options, val, lhs, rhs, expected);
+}
+
+template <typename T>
+static inline bool SlowBetween(BetweenOperator op, const T& val, const T& lhs,
+                               const T& rhs) {
+  switch (op) {
+    case BETWEEN_LESS_EQUAL_LESS_EQUAL:
+      return ((lhs <= val) && (val <= rhs));
+    case BETWEEN_LESS_EQUAL_LESS_THAN:
+      return ((lhs <= val) && (val < rhs));
+    case BETWEEN_LESS_THAN_LESS_EQUAL:
+      return ((lhs < val) && (val <= rhs));
+    case BETWEEN_LESS_THAN_LESS_THAN:
+      return ((lhs < val) && (val < rhs));
+    default:
+      return false;
+  }
+}
+
+template <typename ArrayType>
+std::vector<bool> NullBitmapFromThreeArrays(const ArrayType& val, const ArrayType& lhs,
+                                            const ArrayType& rhs) {
+  auto value_lambda = [&val](int64_t i) {
+    return val.null_count() == 0 ? true : val.IsValid(i);
+  };
+
+  auto left_lambda = [&lhs](int64_t i) {
+    return lhs.null_count() == 0 ? true : lhs.IsValid(i);
+  };
+
+  auto right_lambda = [&rhs](int64_t i) {
+    return rhs.null_count() == 0 ? true : rhs.IsValid(i);
+  };
+
+  const int64_t length = lhs.length();
+  std::vector<bool> null_bitmap(length);
+
+  for (int64_t i = 0; i < length; i++) {
+    null_bitmap[i] = value_lambda(i) && left_lambda(i) && right_lambda(i);
+  }
+
+  return null_bitmap;
+}
+
+template <typename ArrowType>
+Datum SimpleArrayScalarScalarBetween(BetweenOptions options, const Datum& val,

Review comment:
       This cleans up the code a lot. Thanks. Currently one of the random tests fails, which passed with previous implementation,but checking this.




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

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

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



[GitHub] [arrow] bkmgit commented on a change in pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
bkmgit commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r773122843



##########
File path: cpp/src/arrow/compute/kernels/scalar_compare_test.cc
##########
@@ -1850,5 +1846,439 @@ TEST(TestMaxElementWiseMinElementWise, CommonTemporal) {
               ResultWith(ScalarFromJSON(date64(), "86400000")));
 }
 
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const Datum& val, const Datum& lhs,
+                            const Datum& rhs, const Datum& expected) {
+  ASSERT_OK_AND_ASSIGN(
+      Datum result,
+      CallFunction(BetweenOperatorToFunctionName(options.op), {val, lhs, rhs}));
+  AssertArraysEqual(*expected.make_array(), *result.make_array(),
+                    /*verbose=*/true);
+}
+
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const Datum& val, const Datum& lhs,
+                            const Datum& rhs, const char* expected_str) {
+  auto expected = ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), expected_str);
+  ValidateBetween<ArrowType>(options, val, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const Datum& val, const Datum& lhs,
+                            const char* rhs_str, const Datum& expected) {
+  auto rhs = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), rhs_str);
+  ValidateBetween<ArrowType>(options, val, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const Datum& val, const char* lhs_str,
+                            const Datum& rhs, const Datum& expected) {
+  auto lhs = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), lhs_str);
+  ValidateBetween<ArrowType>(options, val, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const char* val_str, const Datum& lhs,
+                            const Datum& rhs, const Datum& expected) {
+  auto val = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), val_str);
+  ValidateBetween<ArrowType>(options, val, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const Datum& val, const Datum& lhs,
+                            const char* rhs_str, const char* expected_str) {
+  auto rhs = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), rhs_str);
+  auto expected = ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), expected_str);
+  ValidateBetween<ArrowType>(options, val, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const Datum& val, const char* lhs_str,
+                            const Datum& rhs, const char* expected_str) {
+  auto lhs = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), lhs_str);
+  auto expected = ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), expected_str);
+  ValidateBetween<ArrowType>(options, val, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const Datum& val, const char* lhs_str,
+                            const char* rhs_str, const Datum& expected) {
+  auto lhs = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), lhs_str);
+  auto rhs = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), rhs_str);
+  ValidateBetween<ArrowType>(options, val, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const char* val_str, const Datum& lhs,
+                            const Datum& rhs, const char* expected_str) {
+  auto val = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), val_str);
+  auto expected = ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), expected_str);
+  ValidateBetween<ArrowType>(options, val, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const char* val_str,
+                            const char* lhs_str, const Datum& rhs,
+                            const Datum& expected) {
+  auto val = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), val_str);
+  auto lhs = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), lhs_str);
+  ValidateBetween<ArrowType>(options, val, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const char* val_str, const Datum& lhs,
+                            const char* rhs_str, const Datum& expected) {
+  auto val = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), val_str);
+  auto rhs = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), rhs_str);
+  ValidateBetween<ArrowType>(options, val, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const Datum& val, const char* lhs_str,
+                            const char* rhs_str, const char* expected_str) {
+  auto lhs = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), lhs_str);
+  auto rhs = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), rhs_str);
+  auto expected = ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), expected_str);
+  ValidateBetween<ArrowType>(options, val, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const char* val_str, const Datum& lhs,
+                            const char* rhs_str, const char* expected_str) {
+  auto val = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), val_str);
+  auto rhs = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), rhs_str);
+  auto expected = ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), expected_str);
+  ValidateBetween<ArrowType>(options, val, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const char* val_str,
+                            const char* lhs_str, const Datum& rhs,
+                            const char* expected_str) {
+  auto val = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), val_str);
+  auto lhs = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), lhs_str);
+  auto expected = ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), expected_str);
+  ValidateBetween<ArrowType>(options, val, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const char* val_str,
+                            const char* lhs_str, const char* rhs_str,
+                            const Datum& expected) {
+  auto val = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), val_str);
+  auto lhs = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), lhs_str);
+  auto rhs = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), rhs_str);
+  ValidateBetween<ArrowType>(options, val, lhs, rhs, expected);
+}
+
+template <typename T>
+static inline bool SlowBetween(BetweenOperator op, const T& val, const T& lhs,
+                               const T& rhs) {
+  switch (op) {
+    case BETWEEN_LESS_EQUAL_LESS_EQUAL:
+      return ((lhs <= val) && (val <= rhs));
+    case BETWEEN_LESS_EQUAL_LESS_THAN:
+      return ((lhs <= val) && (val < rhs));
+    case BETWEEN_LESS_THAN_LESS_EQUAL:
+      return ((lhs < val) && (val <= rhs));
+    case BETWEEN_LESS_THAN_LESS_THAN:
+      return ((lhs < val) && (val < rhs));
+    default:
+      return false;
+  }
+}
+
+template <typename ArrayType>
+std::vector<bool> NullBitmapFromThreeArrays(const ArrayType& val, const ArrayType& lhs,
+                                            const ArrayType& rhs) {
+  auto value_lambda = [&val](int64_t i) {
+    return val.null_count() == 0 ? true : val.IsValid(i);
+  };
+
+  auto left_lambda = [&lhs](int64_t i) {
+    return lhs.null_count() == 0 ? true : lhs.IsValid(i);
+  };
+
+  auto right_lambda = [&rhs](int64_t i) {
+    return rhs.null_count() == 0 ? true : rhs.IsValid(i);
+  };
+
+  const int64_t length = lhs.length();
+  std::vector<bool> null_bitmap(length);
+
+  for (int64_t i = 0; i < length; i++) {
+    null_bitmap[i] = value_lambda(i) && left_lambda(i) && right_lambda(i);
+  }
+
+  return null_bitmap;
+}
+
+template <typename ArrowType>
+Datum SimpleArrayScalarScalarBetween(BetweenOptions options, const Datum& val,

Review comment:
       Ok, thanks.




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

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

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



[GitHub] [arrow] edponce commented on a change in pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
edponce commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r784960871



##########
File path: cpp/src/arrow/compute/kernels/scalar_compare_test.cc
##########
@@ -1865,5 +1861,636 @@ TEST(TestMaxElementWiseMinElementWise, CommonTemporal) {
               ResultWith(ScalarFromJSON(date64(), "86400000")));
 }
 
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const Datum& val, const Datum& lhs,
+                            const Datum& rhs, const Datum& expected) {
+  ASSERT_OK_AND_ASSIGN(Datum result, Between(val, lhs, rhs, options, nullptr));
+  AssertArraysEqual(*expected.make_array(), *result.make_array(),
+                    /*verbose=*/true);
+}
+
+template <typename ArrowType>
+void ValidateBetween(BetweenOptions options, const Datum& val, const Datum& lhs,
+                     const Datum& rhs) {
+  CompareOperator lhs_val;
+  CompareOperator val_rhs;
+  BetweenOptions::Inclusive include_endpoints = options.inclusive;
+
+  if (include_endpoints == BetweenOptions::Inclusive::NEITHER) {
+    lhs_val = LESS;
+    val_rhs = LESS;
+  } else if (include_endpoints == BetweenOptions::Inclusive::LEFT) {
+    lhs_val = LESS_EQUAL;
+    val_rhs = LESS;
+  } else if (include_endpoints == BetweenOptions::Inclusive::RIGHT) {
+    lhs_val = LESS;
+    val_rhs = LESS_EQUAL;
+  } else {
+    lhs_val = LESS_EQUAL;
+    val_rhs = LESS_EQUAL;
+  }
+
+  ASSERT_OK_AND_ASSIGN(Datum resultl,
+                       CallFunction(CompareOperatorToFunctionName(lhs_val), {lhs, val}));
+  ASSERT_OK_AND_ASSIGN(Datum resultr,
+                       CallFunction(CompareOperatorToFunctionName(val_rhs), {val, rhs}));
+  ASSERT_OK_AND_ASSIGN(Datum expected, CallFunction("and", {resultl, resultr}));
+
+  ValidateBetween<ArrowType>(options, val, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+class TestNumericBetweenKernel : public ::testing::Test {};
+
+TYPED_TEST_SUITE(TestNumericBetweenKernel, NumericArrowTypes);
+TYPED_TEST(TestNumericBetweenKernel, SimpleBetweenArrayScalarScalar) {
+  using ScalarType = typename TypeTraits<TypeParam>::ScalarType;
+  using CType = typename TypeTraits<TypeParam>::CType;
+
+  Datum zero(std::make_shared<ScalarType>(CType(0)));
+  Datum four(std::make_shared<ScalarType>(CType(4)));
+  Datum null(std::make_shared<ScalarType>());
+  BetweenOptions InclusiveBoth(BetweenOptions::Inclusive::BOTH);
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"), zero,
+      four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      zero, four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,1,2,2]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[1,1,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,3,4,5]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[1,1,1,1,1,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,1,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null,0,1,1]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), null, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, null,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  BetweenOptions InclusiveLeft(BetweenOptions::Inclusive::LEFT);
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"), zero,
+      four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      zero, four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,1,2,2]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[1,1,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,3,4,5]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[1,1,1,1,0,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,0,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null,0,1,1]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), null, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, null,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  BetweenOptions InclusiveRight(BetweenOptions::Inclusive::RIGHT);
+  ValidateBetween<TypeParam>(
+      InclusiveRight, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"), zero,
+      four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      zero, four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,1,2,2]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,0,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,3,4,5]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,1,1,1,1,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,1,1,1,1,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null,0,1,1]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null,0,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), null, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, null,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  BetweenOptions InclusiveNeither(BetweenOptions::Inclusive::NEITHER);
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      zero, four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      zero, four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,1,2,2]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,0,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,3,4,5]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,1,1,1,0,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,0,1,1,1,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null,0,1,1]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null,0,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), null, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, null,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+}
+
+TYPED_TEST(TestNumericBetweenKernel, SimpleBetweenScalarArrayArray) {
+  using ScalarType = typename TypeTraits<TypeParam>::ScalarType;
+  using CType = typename TypeTraits<TypeParam>::CType;
+
+  Datum one(std::make_shared<ScalarType>(CType(1)));
+
+  BetweenOptions InclusiveBoth(BetweenOptions::Inclusive::BOTH);
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, one, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,true,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,true,null,false,false]"));
+  BetweenOptions InclusiveLeft(BetweenOptions::Inclusive::LEFT);
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, one, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,true,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,true,null,false,false]"));
+  BetweenOptions InclusiveRight(BetweenOptions::Inclusive::RIGHT);
+  ValidateBetween<TypeParam>(
+      InclusiveRight, one, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,false,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,false,null,false,false]"));
+  BetweenOptions InclusiveNeither(BetweenOptions::Inclusive::NEITHER);
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, one, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,false,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,false,null,false,false]"));
+}
+
+TYPED_TEST(TestNumericBetweenKernel, SimpleBetweenArrayArrayArray) {
+  BetweenOptions InclusiveBoth(BetweenOptions::Inclusive::BOTH);
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[1,1,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,true,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,true,null,false,false]"));
+  BetweenOptions InclusiveLeft(BetweenOptions::Inclusive::LEFT);
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[1,1,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,false,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,false,null,false,false]"));
+  BetweenOptions InclusiveRight(BetweenOptions::Inclusive::RIGHT);
+  ValidateBetween<TypeParam>(
+      InclusiveRight, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[1,1,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,true,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,true,null,false,false]"));
+  BetweenOptions InclusiveNeither(BetweenOptions::Inclusive::NEITHER);
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[1,1,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,false,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,false,null,false,false]"));
+}
+
+template <typename Type>
+struct BetweenRandomNumeric {
+  static void Test(const std::shared_ptr<DataType>& type) {
+    using ScalarType = typename TypeTraits<Type>::ScalarType;
+    using CType = typename TypeTraits<Type>::CType;
+    auto rand = random::RandomArrayGenerator(0x5416447);
+    const int64_t length = 1000;
+    for (auto null_probability : {0.0, 0.01, 0.1, 0.25, 0.5, 1.0}) {
+      for (auto inclusive :
+           {BetweenOptions::Inclusive::BOTH, BetweenOptions::Inclusive::LEFT,
+            BetweenOptions::Inclusive::RIGHT, BetweenOptions::Inclusive::NEITHER}) {
+        auto data1 =
+            rand.Numeric<typename Type::PhysicalType>(length, 0, 100, null_probability);
+        auto data2 =
+            rand.Numeric<typename Type::PhysicalType>(length, 0, 100, null_probability);
+        auto data3 =
+            rand.Numeric<typename Type::PhysicalType>(length, 0, 100, null_probability);
+
+        // Create view of data as the type (e.g. timestamp)
+        auto array1 = Datum(*data1->View(type));
+        auto array2 = Datum(*data2->View(type));
+        auto array3 = Datum(*data3->View(type));
+        auto scalar1 = Datum(std::make_shared<ScalarType>(CType(10), type));
+        auto scalar2 = Datum(std::make_shared<ScalarType>(CType(30), type));
+        auto scalar3 = Datum(std::make_shared<ScalarType>(CType(50), type));
+        auto options = BetweenOptions(inclusive);
+        ValidateBetween<Type>(options, array1, scalar2, scalar3);
+        ValidateBetween<Type>(options, array1, array2, scalar3);
+        ValidateBetween<Type>(options, array1, array2, array3);
+        ValidateBetween<Type>(options, array1, scalar2, scalar3);
+        ValidateBetween<Type>(options, scalar1, array2, array3);
+        ValidateBetween<Type>(options, scalar1, array2, scalar3);
+        ValidateBetween<Type>(options, scalar1, scalar2, array3);
+        ValidateBetween<Type>(options, array1, scalar2, array3);
+      }
+    }
+  }
+};
+
+TEST(TestNumericBetweenKernel, BetweenPrimitiveRandomTests) {
+  TestRandomPrimitiveCTypes<BetweenRandomNumeric>();
+}
+
+class TestStringBetweenKernel : public ::testing::Test {};
+
+TEST(TestStringBetweenKernel, RandomBetween) {
+  using ScalarType = typename TypeTraits<StringType>::ScalarType;
+
+  auto rand = random::RandomArrayGenerator(0x5416447);
+  for (size_t i = 3; i < 10; i++) {
+    for (auto null_probability : {0.0, 0.01, 0.1, 0.25, 0.5, 1.0}) {
+      for (auto inclusive :
+           {BetweenOptions::Inclusive::BOTH, BetweenOptions::Inclusive::LEFT,
+            BetweenOptions::Inclusive::RIGHT, BetweenOptions::Inclusive::NEITHER}) {
+        const int64_t length = static_cast<int64_t>(1ULL << i);
+        auto array1 = Datum(rand.String(length, 0, 16, null_probability));
+        auto array2 = Datum(rand.String(length, 0, 16, null_probability));
+        auto array3 = Datum(rand.String(length, 0, 16, null_probability));
+        auto scalar1 = Datum(std::make_shared<ScalarType>("fupi"));
+        auto scalar2 = Datum(std::make_shared<ScalarType>("tupu"));
+        auto scalar3 = Datum(std::make_shared<ScalarType>("zito"));
+        auto options = BetweenOptions(inclusive);
+        ValidateBetween<StringType>(options, array1, scalar2, scalar3);
+        ValidateBetween<StringType>(options, scalar1, array2, scalar3);
+        ValidateBetween<StringType>(options, scalar1, scalar2, array3);
+        ValidateBetween<StringType>(options, scalar1, array2, array3);
+        ValidateBetween<StringType>(options, array1, scalar2, array3);
+        ValidateBetween<StringType>(options, array1, array2, scalar3);
+        ValidateBetween<StringType>(options, array1, array2, array3);
+      }
+    }
+  }
+}
+
+TEST(TestStringBetweenKernel, StringArrayScalarScalarTest) {
+  using ScalarType = typename TypeTraits<StringType>::ScalarType;
+  auto l = Datum(std::make_shared<ScalarType>("abc"));
+  auto r = Datum(std::make_shared<ScalarType>("zzz"));
+  BetweenOptions InclusiveOption(BetweenOptions::Inclusive::BOTH);
+  ValidateBetween<StringType>(
+      InclusiveOption, ArrayFromJSON(TypeTraits<StringType>::type_singleton(), "[]"), l,
+      r, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<StringType>(
+      InclusiveOption, ArrayFromJSON(TypeTraits<StringType>::type_singleton(), "[null]"),
+      l, r, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<StringType>(InclusiveOption,
+                              ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                            R"(["aaa", "aaaa", "ccc", "z"])"),
+                              l, r,
+                              ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                                            R"([false, false, true, true])"));
+  ValidateBetween<StringType>(InclusiveOption,
+                              ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                            R"(["a", "aaaa", "c", "z"])"),
+                              l, r,
+                              ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                                            R"([false, false, true, true])"));
+  ValidateBetween<StringType>(InclusiveOption,
+                              ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                            R"(["a", "aaaa", "fff", "zzzz"])"),
+                              l, r,
+                              ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                                            R"([false, false, true, false])"));
+  ValidateBetween<StringType>(InclusiveOption,
+                              ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                            R"(["abc", "baa", "fff", "zzz"])"),
+                              l, r,
+                              ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                                            R"([true, true, true, true])"));
+  ValidateBetween<StringType>(InclusiveOption,
+                              ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                            R"(["abd", null, null, "zzx"])"),
+                              l, r,
+                              ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                                            R"([true, null, null, true])"));
+}
+
+TEST(TestStringBetweenKernel, StringArrayArrayArrayTest) {
+  BetweenOptions InclusiveOption(BetweenOptions::Inclusive::BOTH);
+  ValidateBetween<StringType>(
+      InclusiveOption,
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                    R"(["david","hello","world"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["adam","hi","whirl"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                    R"(["robert","goeiemoreen","whirlwind"])"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[true, false, false]"));
+  ValidateBetween<StringType>(
+      InclusiveOption,
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["x","a","f"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["w","a","e"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["z","a","g"])"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[true, true, true]"));
+  ValidateBetween<StringType>(
+      InclusiveOption,
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                    R"(["block","bit","binary"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                    R"(["bit","nibble","ternary"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["word","d","xyz"])"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[true, false, false]"));
+  ValidateBetween<StringType>(
+      InclusiveOption,
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                    R"(["よしもと","の","ち"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["は","へ","あ"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["な","を","ち"])"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[false, false, true]"));
+  ValidateBetween<StringType>(
+      InclusiveOption,
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["A","ア","王"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["た","あ","歩"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["李","田",null])"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[false, true, null]"));
+  ValidateBetween<StringType>(
+      InclusiveOption,
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["Б",null,"Я"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["А","Ж","Щ"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["Д","Л","Ф"])"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[true, null, false]"));
+}
+
+TEST(TestTimestampsBetweenKernel, TimestampsArrayArrayArrayTest) {
+  const std::string arr_json = R"(["1970-01-01","2000-02-02","1900-02-28"])";
+  const std::string lhs_json = R"(["1970-01-01","2000-02-01","1900-02-28"])";
+  const std::string rhs_json = R"(["1970-01-02","2000-02-02","1900-02-28"])";
+  // Between Options
+  BetweenOptions both(BetweenOptions::Inclusive::BOTH);
+  BetweenOptions left(BetweenOptions::Inclusive::LEFT);
+  BetweenOptions right(BetweenOptions::Inclusive::RIGHT);
+  BetweenOptions neither(BetweenOptions::Inclusive::NEITHER);
+  // Same units should be fine
+  auto arr = ArrayFromJSON(timestamp(TimeUnit::SECOND), arr_json);
+  auto lhs = ArrayFromJSON(timestamp(TimeUnit::SECOND), lhs_json);
+  auto rhs = ArrayFromJSON(timestamp(TimeUnit::SECOND), rhs_json);
+  // BOTH
+  ValidateBetween<TimestampType>(both, arr, lhs, rhs);
+  // LEFT
+  ValidateBetween<TimestampType>(left, arr, lhs, rhs);
+  // RIGHT
+  ValidateBetween<TimestampType>(right, arr, lhs, rhs);
+  // NEITHER
+  ValidateBetween<TimestampType>(neither, arr, lhs, rhs);
+/*  // So are same time zones
+  arr = ArrayFromJSON(timestamp(TimeUnit::SECOND, "America/Chicago"), arr_json);
+  lhs = ArrayFromJSON(timestamp(TimeUnit::SECOND, "America/Chicago"), lhs_json);
+  rhs = ArrayFromJSON(timestamp(TimeUnit::SECOND, "America/Chicago"), rhs_json);
+  // BOTH
+  ValidateBetween<TimestampType>(both, arr, lhs, rhs);
+  // LEFT
+  ValidateBetween<TimestampType>(left, arr, lhs, rhs);
+  // RIGHT
+  ValidateBetween<TimestampType>(right, arr, lhs, rhs);
+  // NEITHER
+  ValidateBetween<TimestampType>(neither, arr, lhs, rhs);
+  // Different units should be fine
+  auto arr = ArrayFromJSON(timestamp(TimeUnit::SECOND), arr_json);
+  auto lhs = ArrayFromJSON(timestamp(TimeUnit::MILLI), lhs_json);
+  auto rhs = ArrayFromJSON(timestamp(TimeUnit::SECOND), rhs_json);
+  // BOTH

Review comment:
       cc @bkmgit 




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

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

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



[GitHub] [arrow] bkmgit commented on a change in pull request #11882: ARROW-9843: [C++][Python] Implement Between ternary kernel and Python bindings

Posted by GitBox <gi...@apache.org>.
bkmgit commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r786979074



##########
File path: cpp/src/arrow/compute/api_scalar.h
##########
@@ -342,6 +342,17 @@ struct ARROW_EXPORT CompareOptions {
   enum CompareOperator op;
 };
 
+class ARROW_EXPORT BetweenOptions : public FunctionOptions {
+ public:
+  enum Inclusive { BOTH, LEFT, RIGHT, NEITHER };
+  explicit BetweenOptions(Inclusive inclusive = BOTH);
+  static BetweenOptions Defaults() { return BetweenOptions(); }
+  constexpr static char const kTypeName[] = "BetweenOptions";
+
+  /// Inclusive option to apply

Review comment:
       Updated to
   ``` 
     /// Indicate boundary points to include, both (a <= val <= b), 
     /// left ( a <= value < b) , right ( a < value <= b)
     /// or neither ( a < value < b )
   ```




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

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

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



[GitHub] [arrow] bkmgit commented on a change in pull request #11882: ARROW-9843: [C++][Python] Implement Between ternary kernel and Python bindings

Posted by GitBox <gi...@apache.org>.
bkmgit commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r787032953



##########
File path: cpp/src/arrow/compute/kernels/scalar_compare_test.cc
##########
@@ -1850,5 +1851,154 @@ TEST(TestMaxElementWiseMinElementWise, CommonTemporal) {
               ResultWith(ScalarFromJSON(date64(), "86400000")));
 }
 
+template <typename ArrowType>
+static void ValidateBetween(const Datum& val, const Datum& lhs, const Datum& rhs,
+                            const Datum& expected) {
+  ASSERT_OK_AND_ASSIGN(Datum result, Between(val, lhs, rhs));
+  AssertArraysEqual(*expected.make_array(), *result.make_array(),
+                    /*verbose=*/true);
+}
+
+template <typename ArrowType>
+static void ValidateBetween(const char* value_str, const Datum& lhs, const Datum& rhs,
+                            const char* expected_str) {
+  auto value = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), value_str);
+  auto expected = ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), expected_str);
+  ValidateBetween<ArrowType>(value, lhs, rhs, expected);
+}
+
+template <>
+void ValidateBetween<StringType>(const char* value_str, const Datum& lhs,
+                                 const Datum& rhs, const char* expected_str) {
+  auto value = ArrayFromJSON(utf8(), value_str);
+  auto expected = ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), expected_str);
+  ValidateBetween<StringType>(value, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+class TestNumericBetweenKernel : public ::testing::Test {};
+
+TYPED_TEST_SUITE(TestNumericBetweenKernel, NumericArrowTypes);
+TYPED_TEST(TestNumericBetweenKernel, SimpleBetweenArrayScalarScalar) {
+  using ScalarType = typename TypeTraits<TypeParam>::ScalarType;
+  using CType = typename TypeTraits<TypeParam>::CType;
+
+  Datum zero(std::make_shared<ScalarType>(CType(0)));
+  Datum four(std::make_shared<ScalarType>(CType(4)));
+  ValidateBetween<TypeParam>("[]", zero, four, "[]");
+  ValidateBetween<TypeParam>("[null]", zero, four, "[null]");
+  ValidateBetween<TypeParam>("[0,0,1,1,2,2]", zero, four, "[0,0,1,1,1,1]");
+  ValidateBetween<TypeParam>("[0,1,2,3,4,5]", zero, four, "[0,1,1,1,0,0]");
+  ValidateBetween<TypeParam>("[5,4,3,2,1,0]", zero, four, "[0,0,1,1,1,0]");
+  ValidateBetween<TypeParam>("[null,0,1,1]", zero, four, "[null,0,1,1]");
+}
+
+TEST(TestSimpleBetweenKernel, SimpleStringTest) {
+  using ScalarType = typename TypeTraits<StringType>::ScalarType;
+  auto l = Datum(std::make_shared<ScalarType>("abc"));
+  auto r = Datum(std::make_shared<ScalarType>("zzz"));
+  ValidateBetween<StringType>("[]", l, r, "[]");
+  ValidateBetween<StringType>("[null]", l, r, "[null]");
+  ValidateBetween<StringType>(R"(["aaa", "aaaa", "ccc", "z"])", l, r,
+                              R"([false, false, true, true])");
+  ValidateBetween<StringType>(R"(["a", "aaaa", "c", "z"])", l, r,
+                              R"([false, false, true, true])");
+  ValidateBetween<StringType>(R"(["a", "aaaa", "fff", "zzzz"])", l, r,
+                              R"([false, false, true, false])");
+  ValidateBetween<StringType>(R"(["abd", null, null, "zzx"])", l, r,
+                              R"([true, null, null, true])");
+}
+
+TEST(TestSimpleBetweenKernel, SimpleTimestampTest) {

Review comment:
       Timestamp tests have been added.




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

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

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



[GitHub] [arrow] bkmgit commented on a change in pull request #11882: ARROW-9843: [C++][Python] Implement Between ternary kernel and Python bindings

Posted by GitBox <gi...@apache.org>.
bkmgit commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r787594779



##########
File path: cpp/src/arrow/compute/kernels/scalar_compare_test.cc
##########
@@ -1865,5 +1867,683 @@ TEST(TestMaxElementWiseMinElementWise, CommonTemporal) {
               ResultWith(ScalarFromJSON(date64(), "86400000")));
 }
 
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const Datum& val, const Datum& lhs,
+                            const Datum& rhs, const Datum& expected) {
+  ASSERT_OK_AND_ASSIGN(Datum result, Between(val, lhs, rhs, options, nullptr));
+  AssertArraysEqual(*expected.make_array(), *result.make_array(),
+                    /*verbose=*/true);
+}
+
+template <typename ArrowType>
+void ValidateBetween(BetweenOptions options, const Datum& val, const Datum& lhs,
+                     const Datum& rhs) {
+  CompareOperator lhs_val;
+  CompareOperator val_rhs;
+  BetweenOptions::Inclusive include_endpoints = options.inclusive;
+
+  if (include_endpoints == BetweenOptions::Inclusive::NEITHER) {
+    lhs_val = LESS;
+    val_rhs = LESS;
+  } else if (include_endpoints == BetweenOptions::Inclusive::LEFT) {
+    lhs_val = LESS_EQUAL;
+    val_rhs = LESS;
+  } else if (include_endpoints == BetweenOptions::Inclusive::RIGHT) {
+    lhs_val = LESS;
+    val_rhs = LESS_EQUAL;
+  } else {
+    lhs_val = LESS_EQUAL;
+    val_rhs = LESS_EQUAL;
+  }
+
+  ASSERT_OK_AND_ASSIGN(Datum resultl,
+                       CallFunction(CompareOperatorToFunctionName(lhs_val), {lhs, val}));
+  ASSERT_OK_AND_ASSIGN(Datum resultr,
+                       CallFunction(CompareOperatorToFunctionName(val_rhs), {val, rhs}));
+  ASSERT_OK_AND_ASSIGN(Datum expected, CallFunction("and", {resultl, resultr}));
+
+  ValidateBetween<ArrowType>(options, val, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+class TestNumericBetweenKernel : public ::testing::Test {};
+
+TYPED_TEST_SUITE(TestNumericBetweenKernel, NumericArrowTypes);
+TYPED_TEST(TestNumericBetweenKernel, SimpleBetweenArrayScalarScalar) {
+  using ScalarType = typename TypeTraits<TypeParam>::ScalarType;
+  using CType = typename TypeTraits<TypeParam>::CType;
+
+  Datum zero(std::make_shared<ScalarType>(CType(0)));
+  Datum four(std::make_shared<ScalarType>(CType(4)));
+  Datum null(std::make_shared<ScalarType>());
+  BetweenOptions InclusiveBoth(BetweenOptions::Inclusive::BOTH);
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"), zero,
+      four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      zero, four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,1,2,2]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[1,1,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,3,4,5]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[1,1,1,1,1,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,1,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null,0,1,1]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), null, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, null,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  BetweenOptions InclusiveLeft(BetweenOptions::Inclusive::LEFT);
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"), zero,
+      four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      zero, four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,1,2,2]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[1,1,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,3,4,5]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[1,1,1,1,0,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,0,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null,0,1,1]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), null, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, null,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  BetweenOptions InclusiveRight(BetweenOptions::Inclusive::RIGHT);
+  ValidateBetween<TypeParam>(
+      InclusiveRight, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"), zero,
+      four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      zero, four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,1,2,2]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,0,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,3,4,5]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,1,1,1,1,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,1,1,1,1,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null,0,1,1]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null,0,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), null, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, null,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  BetweenOptions InclusiveNeither(BetweenOptions::Inclusive::NEITHER);
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      zero, four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      zero, four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,1,2,2]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,0,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,3,4,5]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,1,1,1,0,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,0,1,1,1,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null,0,1,1]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null,0,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), null, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, null,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+}
+
+TYPED_TEST(TestNumericBetweenKernel, SimpleBetweenScalarArrayArray) {
+  using ScalarType = typename TypeTraits<TypeParam>::ScalarType;
+  using CType = typename TypeTraits<TypeParam>::CType;
+
+  Datum one(std::make_shared<ScalarType>(CType(1)));
+
+  BetweenOptions InclusiveBoth(BetweenOptions::Inclusive::BOTH);
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, one, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,true,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,true,null,false,false]"));
+  BetweenOptions InclusiveLeft(BetweenOptions::Inclusive::LEFT);
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, one, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,true,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,true,null,false,false]"));
+  BetweenOptions InclusiveRight(BetweenOptions::Inclusive::RIGHT);
+  ValidateBetween<TypeParam>(
+      InclusiveRight, one, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,false,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,false,null,false,false]"));
+  BetweenOptions InclusiveNeither(BetweenOptions::Inclusive::NEITHER);
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, one, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,false,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,false,null,false,false]"));
+}
+
+TYPED_TEST(TestNumericBetweenKernel, SimpleBetweenArrayArrayArray) {
+  BetweenOptions InclusiveBoth(BetweenOptions::Inclusive::BOTH);
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[1,1,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,true,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,true,null,false,false]"));
+  BetweenOptions InclusiveLeft(BetweenOptions::Inclusive::LEFT);
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[1,1,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,false,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,false,null,false,false]"));
+  BetweenOptions InclusiveRight(BetweenOptions::Inclusive::RIGHT);
+  ValidateBetween<TypeParam>(
+      InclusiveRight, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[1,1,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,true,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,true,null,false,false]"));
+  BetweenOptions InclusiveNeither(BetweenOptions::Inclusive::NEITHER);
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[1,1,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,false,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,false,null,false,false]"));
+}
+
+template <typename Type>
+struct BetweenRandomNumeric {
+  static void Test(const std::shared_ptr<DataType>& type) {
+    using ScalarType = typename TypeTraits<Type>::ScalarType;
+    using CType = typename TypeTraits<Type>::CType;
+    auto rand = random::RandomArrayGenerator(0x5416447);
+    const int64_t length = 1000;
+    for (auto null_probability : {0.0, 0.01, 0.1, 0.25, 0.5, 1.0}) {
+      for (auto inclusive :
+           {BetweenOptions::Inclusive::BOTH, BetweenOptions::Inclusive::LEFT,
+            BetweenOptions::Inclusive::RIGHT, BetweenOptions::Inclusive::NEITHER}) {
+        auto data1 =
+            rand.Numeric<typename Type::PhysicalType>(length, 0, 100, null_probability);
+        auto data2 =
+            rand.Numeric<typename Type::PhysicalType>(length, 0, 100, null_probability);
+        auto data3 =
+            rand.Numeric<typename Type::PhysicalType>(length, 0, 100, null_probability);
+
+        // Create view of data as the type (e.g. timestamp)
+        auto array1 = Datum(*data1->View(type));
+        auto array2 = Datum(*data2->View(type));
+        auto array3 = Datum(*data3->View(type));
+        auto scalar1 = Datum(std::make_shared<ScalarType>(CType(10), type));
+        auto scalar2 = Datum(std::make_shared<ScalarType>(CType(30), type));
+        auto scalar3 = Datum(std::make_shared<ScalarType>(CType(50), type));
+        auto options = BetweenOptions(inclusive);
+        ValidateBetween<Type>(options, array1, scalar2, scalar3);
+        ValidateBetween<Type>(options, array1, array2, scalar3);
+        ValidateBetween<Type>(options, array1, array2, array3);
+        ValidateBetween<Type>(options, array1, scalar2, scalar3);
+        ValidateBetween<Type>(options, scalar1, array2, array3);
+        ValidateBetween<Type>(options, scalar1, array2, scalar3);
+        ValidateBetween<Type>(options, scalar1, scalar2, array3);
+        ValidateBetween<Type>(options, array1, scalar2, array3);
+      }
+    }
+  }
+};
+
+TEST(TestNumericBetweenKernel, BetweenPrimitiveRandomTests) {
+  TestRandomPrimitiveCTypes<BetweenRandomNumeric>();
+}
+
+class TestStringBetweenKernel : public ::testing::Test {};
+
+TEST(TestStringBetweenKernel, RandomBetween) {
+  using ScalarType = typename TypeTraits<StringType>::ScalarType;
+
+  auto rand = random::RandomArrayGenerator(0x5416447);
+  for (size_t i = 3; i < 10; i++) {
+    for (auto null_probability : {0.0, 0.01, 0.1, 0.25, 0.5, 1.0}) {
+      for (auto inclusive :
+           {BetweenOptions::Inclusive::BOTH, BetweenOptions::Inclusive::LEFT,
+            BetweenOptions::Inclusive::RIGHT, BetweenOptions::Inclusive::NEITHER}) {
+        const int64_t length = static_cast<int64_t>(1ULL << i);
+        auto array1 = Datum(rand.String(length, 0, 16, null_probability));
+        auto array2 = Datum(rand.String(length, 0, 16, null_probability));
+        auto array3 = Datum(rand.String(length, 0, 16, null_probability));
+        auto scalar1 = Datum(std::make_shared<ScalarType>("fupi"));
+        auto scalar2 = Datum(std::make_shared<ScalarType>("tupu"));
+        auto scalar3 = Datum(std::make_shared<ScalarType>("zito"));
+        auto options = BetweenOptions(inclusive);
+        ValidateBetween<StringType>(options, array1, scalar2, scalar3);
+        ValidateBetween<StringType>(options, scalar1, array2, scalar3);
+        ValidateBetween<StringType>(options, scalar1, scalar2, array3);
+        ValidateBetween<StringType>(options, scalar1, array2, array3);
+        ValidateBetween<StringType>(options, array1, scalar2, array3);
+        ValidateBetween<StringType>(options, array1, array2, scalar3);
+        ValidateBetween<StringType>(options, array1, array2, array3);
+      }
+    }
+  }
+}
+
+TEST(TestStringBetweenKernel, StringArrayScalarScalarTest) {
+  using ScalarType = typename TypeTraits<StringType>::ScalarType;
+  auto l = Datum(std::make_shared<ScalarType>("abc"));
+  auto r = Datum(std::make_shared<ScalarType>("zzz"));
+  BetweenOptions InclusiveOption(BetweenOptions::Inclusive::BOTH);
+  ValidateBetween<StringType>(
+      InclusiveOption, ArrayFromJSON(TypeTraits<StringType>::type_singleton(), "[]"), l,
+      r, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<StringType>(
+      InclusiveOption, ArrayFromJSON(TypeTraits<StringType>::type_singleton(), "[null]"),
+      l, r, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<StringType>(InclusiveOption,
+                              ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                            R"(["aaa", "aaaa", "ccc", "z"])"),
+                              l, r,
+                              ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                                            R"([false, false, true, true])"));
+  ValidateBetween<StringType>(InclusiveOption,
+                              ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                            R"(["a", "aaaa", "c", "z"])"),
+                              l, r,
+                              ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                                            R"([false, false, true, true])"));
+  ValidateBetween<StringType>(InclusiveOption,
+                              ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                            R"(["a", "aaaa", "fff", "zzzz"])"),
+                              l, r,
+                              ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                                            R"([false, false, true, false])"));
+  ValidateBetween<StringType>(InclusiveOption,
+                              ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                            R"(["abc", "baa", "fff", "zzz"])"),
+                              l, r,
+                              ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                                            R"([true, true, true, true])"));
+  ValidateBetween<StringType>(InclusiveOption,
+                              ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                            R"(["abd", null, null, "zzx"])"),
+                              l, r,
+                              ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                                            R"([true, null, null, true])"));
+}
+
+TEST(TestStringBetweenKernel, StringArrayArrayArrayTest) {
+  BetweenOptions InclusiveOption(BetweenOptions::Inclusive::BOTH);
+  ValidateBetween<StringType>(
+      InclusiveOption,
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                    R"(["david","hello","world"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["adam","hi","whirl"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                    R"(["robert","goeiemoreen","whirlwind"])"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[true, false, false]"));
+  ValidateBetween<StringType>(
+      InclusiveOption,
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["x","a","f"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["w","a","e"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["z","a","g"])"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[true, true, true]"));
+  ValidateBetween<StringType>(
+      InclusiveOption,
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                    R"(["block","bit","binary"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                    R"(["bit","nibble","ternary"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["word","d","xyz"])"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[true, false, false]"));
+  ValidateBetween<StringType>(
+      InclusiveOption,
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                    R"(["よしもと","の","ち"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["は","へ","あ"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["な","を","ち"])"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[false, false, true]"));
+  ValidateBetween<StringType>(
+      InclusiveOption,
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["A","ア","王"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["た","あ","歩"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["李","田",null])"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[false, true, null]"));
+  ValidateBetween<StringType>(
+      InclusiveOption,
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["Б",null,"Я"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["А","Ж","Щ"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["Д","Л","Ф"])"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[true, null, false]"));

Review comment:
       Put in a loop over the option, so compares to values obtained from a combination of compare functions. Have come across compiler issues before.




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

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

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



[GitHub] [arrow] edponce commented on a change in pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
edponce commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r784191893



##########
File path: cpp/src/arrow/compute/kernels/scalar_compare_test.cc
##########
@@ -1865,5 +1861,636 @@ TEST(TestMaxElementWiseMinElementWise, CommonTemporal) {
               ResultWith(ScalarFromJSON(date64(), "86400000")));
 }
 
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const Datum& val, const Datum& lhs,
+                            const Datum& rhs, const Datum& expected) {
+  ASSERT_OK_AND_ASSIGN(Datum result, Between(val, lhs, rhs, options, nullptr));
+  AssertArraysEqual(*expected.make_array(), *result.make_array(),
+                    /*verbose=*/true);
+}
+
+template <typename ArrowType>
+void ValidateBetween(BetweenOptions options, const Datum& val, const Datum& lhs,
+                     const Datum& rhs) {
+  CompareOperator lhs_val;
+  CompareOperator val_rhs;
+  BetweenOptions::Inclusive include_endpoints = options.inclusive;
+
+  if (include_endpoints == BetweenOptions::Inclusive::NEITHER) {
+    lhs_val = LESS;
+    val_rhs = LESS;
+  } else if (include_endpoints == BetweenOptions::Inclusive::LEFT) {
+    lhs_val = LESS_EQUAL;
+    val_rhs = LESS;
+  } else if (include_endpoints == BetweenOptions::Inclusive::RIGHT) {
+    lhs_val = LESS;
+    val_rhs = LESS_EQUAL;
+  } else {
+    lhs_val = LESS_EQUAL;
+    val_rhs = LESS_EQUAL;
+  }
+
+  ASSERT_OK_AND_ASSIGN(Datum resultl,
+                       CallFunction(CompareOperatorToFunctionName(lhs_val), {lhs, val}));
+  ASSERT_OK_AND_ASSIGN(Datum resultr,
+                       CallFunction(CompareOperatorToFunctionName(val_rhs), {val, rhs}));
+  ASSERT_OK_AND_ASSIGN(Datum expected, CallFunction("and", {resultl, resultr}));
+
+  ValidateBetween<ArrowType>(options, val, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+class TestNumericBetweenKernel : public ::testing::Test {};
+
+TYPED_TEST_SUITE(TestNumericBetweenKernel, NumericArrowTypes);
+TYPED_TEST(TestNumericBetweenKernel, SimpleBetweenArrayScalarScalar) {
+  using ScalarType = typename TypeTraits<TypeParam>::ScalarType;
+  using CType = typename TypeTraits<TypeParam>::CType;
+
+  Datum zero(std::make_shared<ScalarType>(CType(0)));
+  Datum four(std::make_shared<ScalarType>(CType(4)));
+  Datum null(std::make_shared<ScalarType>());
+  BetweenOptions InclusiveBoth(BetweenOptions::Inclusive::BOTH);
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"), zero,
+      four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      zero, four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,1,2,2]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[1,1,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,3,4,5]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[1,1,1,1,1,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,1,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null,0,1,1]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), null, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, null,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  BetweenOptions InclusiveLeft(BetweenOptions::Inclusive::LEFT);
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"), zero,
+      four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      zero, four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,1,2,2]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[1,1,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,3,4,5]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[1,1,1,1,0,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,0,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null,0,1,1]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), null, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, null,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  BetweenOptions InclusiveRight(BetweenOptions::Inclusive::RIGHT);
+  ValidateBetween<TypeParam>(
+      InclusiveRight, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"), zero,
+      four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      zero, four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,1,2,2]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,0,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,3,4,5]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,1,1,1,1,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,1,1,1,1,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null,0,1,1]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null,0,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), null, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, null,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  BetweenOptions InclusiveNeither(BetweenOptions::Inclusive::NEITHER);
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      zero, four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      zero, four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,1,2,2]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,0,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,3,4,5]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,1,1,1,0,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,0,1,1,1,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null,0,1,1]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null,0,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), null, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, null,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+}
+
+TYPED_TEST(TestNumericBetweenKernel, SimpleBetweenScalarArrayArray) {
+  using ScalarType = typename TypeTraits<TypeParam>::ScalarType;
+  using CType = typename TypeTraits<TypeParam>::CType;
+
+  Datum one(std::make_shared<ScalarType>(CType(1)));
+
+  BetweenOptions InclusiveBoth(BetweenOptions::Inclusive::BOTH);
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, one, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,true,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,true,null,false,false]"));
+  BetweenOptions InclusiveLeft(BetweenOptions::Inclusive::LEFT);
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, one, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,true,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,true,null,false,false]"));
+  BetweenOptions InclusiveRight(BetweenOptions::Inclusive::RIGHT);
+  ValidateBetween<TypeParam>(
+      InclusiveRight, one, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,false,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,false,null,false,false]"));
+  BetweenOptions InclusiveNeither(BetweenOptions::Inclusive::NEITHER);
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, one, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,false,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,false,null,false,false]"));
+}
+
+TYPED_TEST(TestNumericBetweenKernel, SimpleBetweenArrayArrayArray) {
+  BetweenOptions InclusiveBoth(BetweenOptions::Inclusive::BOTH);
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[1,1,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,true,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,true,null,false,false]"));
+  BetweenOptions InclusiveLeft(BetweenOptions::Inclusive::LEFT);
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[1,1,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,false,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,false,null,false,false]"));
+  BetweenOptions InclusiveRight(BetweenOptions::Inclusive::RIGHT);
+  ValidateBetween<TypeParam>(
+      InclusiveRight, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[1,1,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,true,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,true,null,false,false]"));
+  BetweenOptions InclusiveNeither(BetweenOptions::Inclusive::NEITHER);
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[1,1,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,false,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,false,null,false,false]"));
+}
+
+template <typename Type>
+struct BetweenRandomNumeric {
+  static void Test(const std::shared_ptr<DataType>& type) {
+    using ScalarType = typename TypeTraits<Type>::ScalarType;
+    using CType = typename TypeTraits<Type>::CType;
+    auto rand = random::RandomArrayGenerator(0x5416447);
+    const int64_t length = 1000;
+    for (auto null_probability : {0.0, 0.01, 0.1, 0.25, 0.5, 1.0}) {
+      for (auto inclusive :
+           {BetweenOptions::Inclusive::BOTH, BetweenOptions::Inclusive::LEFT,
+            BetweenOptions::Inclusive::RIGHT, BetweenOptions::Inclusive::NEITHER}) {
+        auto data1 =
+            rand.Numeric<typename Type::PhysicalType>(length, 0, 100, null_probability);
+        auto data2 =
+            rand.Numeric<typename Type::PhysicalType>(length, 0, 100, null_probability);
+        auto data3 =
+            rand.Numeric<typename Type::PhysicalType>(length, 0, 100, null_probability);
+
+        // Create view of data as the type (e.g. timestamp)
+        auto array1 = Datum(*data1->View(type));
+        auto array2 = Datum(*data2->View(type));
+        auto array3 = Datum(*data3->View(type));
+        auto scalar1 = Datum(std::make_shared<ScalarType>(CType(10), type));
+        auto scalar2 = Datum(std::make_shared<ScalarType>(CType(30), type));
+        auto scalar3 = Datum(std::make_shared<ScalarType>(CType(50), type));
+        auto options = BetweenOptions(inclusive);
+        ValidateBetween<Type>(options, array1, scalar2, scalar3);
+        ValidateBetween<Type>(options, array1, array2, scalar3);
+        ValidateBetween<Type>(options, array1, array2, array3);
+        ValidateBetween<Type>(options, array1, scalar2, scalar3);
+        ValidateBetween<Type>(options, scalar1, array2, array3);
+        ValidateBetween<Type>(options, scalar1, array2, scalar3);
+        ValidateBetween<Type>(options, scalar1, scalar2, array3);
+        ValidateBetween<Type>(options, array1, scalar2, array3);
+      }
+    }
+  }
+};
+
+TEST(TestNumericBetweenKernel, BetweenPrimitiveRandomTests) {
+  TestRandomPrimitiveCTypes<BetweenRandomNumeric>();
+}
+
+class TestStringBetweenKernel : public ::testing::Test {};
+
+TEST(TestStringBetweenKernel, RandomBetween) {
+  using ScalarType = typename TypeTraits<StringType>::ScalarType;
+
+  auto rand = random::RandomArrayGenerator(0x5416447);
+  for (size_t i = 3; i < 10; i++) {
+    for (auto null_probability : {0.0, 0.01, 0.1, 0.25, 0.5, 1.0}) {
+      for (auto inclusive :
+           {BetweenOptions::Inclusive::BOTH, BetweenOptions::Inclusive::LEFT,
+            BetweenOptions::Inclusive::RIGHT, BetweenOptions::Inclusive::NEITHER}) {
+        const int64_t length = static_cast<int64_t>(1ULL << i);
+        auto array1 = Datum(rand.String(length, 0, 16, null_probability));
+        auto array2 = Datum(rand.String(length, 0, 16, null_probability));
+        auto array3 = Datum(rand.String(length, 0, 16, null_probability));
+        auto scalar1 = Datum(std::make_shared<ScalarType>("fupi"));
+        auto scalar2 = Datum(std::make_shared<ScalarType>("tupu"));
+        auto scalar3 = Datum(std::make_shared<ScalarType>("zito"));
+        auto options = BetweenOptions(inclusive);
+        ValidateBetween<StringType>(options, array1, scalar2, scalar3);
+        ValidateBetween<StringType>(options, scalar1, array2, scalar3);
+        ValidateBetween<StringType>(options, scalar1, scalar2, array3);
+        ValidateBetween<StringType>(options, scalar1, array2, array3);
+        ValidateBetween<StringType>(options, array1, scalar2, array3);
+        ValidateBetween<StringType>(options, array1, array2, scalar3);
+        ValidateBetween<StringType>(options, array1, array2, array3);
+      }
+    }
+  }
+}
+
+TEST(TestStringBetweenKernel, StringArrayScalarScalarTest) {
+  using ScalarType = typename TypeTraits<StringType>::ScalarType;
+  auto l = Datum(std::make_shared<ScalarType>("abc"));
+  auto r = Datum(std::make_shared<ScalarType>("zzz"));
+  BetweenOptions InclusiveOption(BetweenOptions::Inclusive::BOTH);
+  ValidateBetween<StringType>(
+      InclusiveOption, ArrayFromJSON(TypeTraits<StringType>::type_singleton(), "[]"), l,
+      r, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<StringType>(
+      InclusiveOption, ArrayFromJSON(TypeTraits<StringType>::type_singleton(), "[null]"),
+      l, r, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<StringType>(InclusiveOption,
+                              ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                            R"(["aaa", "aaaa", "ccc", "z"])"),
+                              l, r,
+                              ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                                            R"([false, false, true, true])"));
+  ValidateBetween<StringType>(InclusiveOption,
+                              ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                            R"(["a", "aaaa", "c", "z"])"),
+                              l, r,
+                              ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                                            R"([false, false, true, true])"));
+  ValidateBetween<StringType>(InclusiveOption,
+                              ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                            R"(["a", "aaaa", "fff", "zzzz"])"),
+                              l, r,
+                              ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                                            R"([false, false, true, false])"));
+  ValidateBetween<StringType>(InclusiveOption,
+                              ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                            R"(["abc", "baa", "fff", "zzz"])"),
+                              l, r,
+                              ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                                            R"([true, true, true, true])"));
+  ValidateBetween<StringType>(InclusiveOption,
+                              ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                            R"(["abd", null, null, "zzx"])"),
+                              l, r,
+                              ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                                            R"([true, null, null, true])"));
+}
+
+TEST(TestStringBetweenKernel, StringArrayArrayArrayTest) {
+  BetweenOptions InclusiveOption(BetweenOptions::Inclusive::BOTH);
+  ValidateBetween<StringType>(
+      InclusiveOption,
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                    R"(["david","hello","world"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["adam","hi","whirl"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                    R"(["robert","goeiemoreen","whirlwind"])"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[true, false, false]"));
+  ValidateBetween<StringType>(
+      InclusiveOption,
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["x","a","f"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["w","a","e"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["z","a","g"])"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[true, true, true]"));
+  ValidateBetween<StringType>(
+      InclusiveOption,
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                    R"(["block","bit","binary"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                    R"(["bit","nibble","ternary"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["word","d","xyz"])"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[true, false, false]"));
+  ValidateBetween<StringType>(
+      InclusiveOption,
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                    R"(["よしもと","の","ち"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["は","へ","あ"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["な","を","ち"])"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[false, false, true]"));
+  ValidateBetween<StringType>(
+      InclusiveOption,
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["A","ア","王"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["た","あ","歩"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["李","田",null])"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[false, true, null]"));
+  ValidateBetween<StringType>(
+      InclusiveOption,
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["Б",null,"Я"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["А","Ж","Щ"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["Д","Л","Ф"])"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[true, null, false]"));
+}
+
+TEST(TestTimestampsBetweenKernel, TimestampsArrayArrayArrayTest) {
+  const std::string arr_json = R"(["1970-01-01","2000-02-02","1900-02-28"])";
+  const std::string lhs_json = R"(["1970-01-01","2000-02-01","1900-02-28"])";
+  const std::string rhs_json = R"(["1970-01-02","2000-02-02","1900-02-28"])";
+  // Between Options
+  BetweenOptions both(BetweenOptions::Inclusive::BOTH);
+  BetweenOptions left(BetweenOptions::Inclusive::LEFT);
+  BetweenOptions right(BetweenOptions::Inclusive::RIGHT);
+  BetweenOptions neither(BetweenOptions::Inclusive::NEITHER);
+  // Same units should be fine
+  auto arr = ArrayFromJSON(timestamp(TimeUnit::SECOND), arr_json);
+  auto lhs = ArrayFromJSON(timestamp(TimeUnit::SECOND), lhs_json);
+  auto rhs = ArrayFromJSON(timestamp(TimeUnit::SECOND), rhs_json);
+  // BOTH
+  ValidateBetween<TimestampType>(both, arr, lhs, rhs);
+  // LEFT
+  ValidateBetween<TimestampType>(left, arr, lhs, rhs);
+  // RIGHT
+  ValidateBetween<TimestampType>(right, arr, lhs, rhs);
+  // NEITHER
+  ValidateBetween<TimestampType>(neither, arr, lhs, rhs);
+/*  // So are same time zones
+  arr = ArrayFromJSON(timestamp(TimeUnit::SECOND, "America/Chicago"), arr_json);
+  lhs = ArrayFromJSON(timestamp(TimeUnit::SECOND, "America/Chicago"), lhs_json);
+  rhs = ArrayFromJSON(timestamp(TimeUnit::SECOND, "America/Chicago"), rhs_json);
+  // BOTH
+  ValidateBetween<TimestampType>(both, arr, lhs, rhs);

Review comment:
       Commented code should be removed. Are these tests not supported?




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

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

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



[GitHub] [arrow] github-actions[bot] commented on pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#issuecomment-1013735250


   ```
   Unable to match any tasks for `autotune`
   The Archery job run can be found at: https://github.com/apache/arrow/actions/runs/1702298478```


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

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

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



[GitHub] [arrow] edponce commented on pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
edponce commented on pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#issuecomment-1013732577


   The are some linter errors, run clang-format.


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

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

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



[GitHub] [arrow] edponce edited a comment on pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
edponce edited a comment on pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#issuecomment-1013718034


   @bkmgit I am curious why there are many changes to Java, Go, R, Makefile files made alongside this Between PR?


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

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

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



[GitHub] [arrow] bkmgit commented on a change in pull request #11882: ARROW-9843: [C++][Python] Implement Between ternary kernel and Python bindings

Posted by GitBox <gi...@apache.org>.
bkmgit commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r786076345



##########
File path: cpp/src/arrow/compute/kernels/scalar_compare.cc
##########
@@ -156,39 +212,52 @@ struct Maximum {
   }
 };
 
+// Check if timestamp timezones are comparable (either all are empty or none is).
+Status CheckCompareTimestamps(const ExecBatch& batch) {
+  if (batch.num_values() > 0) {
+    for (int i = 0; i < batch.num_values() - 1; ++i) {
+      const auto& tsi = checked_cast<const TimestampType&>(*batch[i].type());
+      for (int j = i + 1; j < batch.num_values(); ++j) {

Review comment:
       It is doing pairwise comparisons. With only a single loop, if first two boolean evaluations give 0, then this become 1, and comparing with a third 0 forces an error. Maybe another evaluation mechanism should be used?




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

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

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



[GitHub] [arrow] bkmgit commented on a change in pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
bkmgit commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r780780707



##########
File path: cpp/src/arrow/compute/kernels/scalar_compare.cc
##########
@@ -746,6 +787,111 @@ std::shared_ptr<ScalarFunction> MakeScalarMinMax(std::string name,
   return func;
 }
 
+template <typename Op>
+void AddIntegerBetween(const std::shared_ptr<DataType>& ty, ScalarFunction* func) {
+  auto exec = GeneratePhysicalInteger<ScalarTernaryEqualTypes, BooleanType, Op>(*ty);
+  DCHECK_OK(func->AddKernel({ty, ty, ty}, boolean(), std::move(exec)));
+}
+
+template <typename InType, typename Op>
+void AddGenericBetween(const std::shared_ptr<DataType>& ty, ScalarFunction* func) {
+  DCHECK_OK(func->AddKernel({ty, ty, ty}, boolean(),
+                            ScalarTernaryEqualTypes<BooleanType, InType, Op>::Exec));
+}
+
+template <typename OutType, typename ArgType, typename Op>
+struct BetweenTimestamps : public ScalarTernaryEqualTypes<OutType, ArgType, Op> {
+  using Base = ScalarTernaryEqualTypes<OutType, ArgType, Op>;
+
+  static Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    const auto& var = checked_cast<const TimestampType&>(*batch[0].type());
+    const auto& lhs = checked_cast<const TimestampType&>(*batch[1].type());
+    const auto& rhs = checked_cast<const TimestampType&>(*batch[2].type());
+    if ((var.timezone().empty() != lhs.timezone().empty()) ||
+        (var.timezone().empty() != rhs.timezone().empty()) ||
+        (lhs.timezone().empty() != rhs.timezone().empty())) {
+      return Status::Invalid(
+          "Cannot use timestamps with timezone and timestamps without timezones, got: ",
+          var.timezone().empty(), " ", lhs.timezone().empty(), " and ",
+          rhs.timezone().empty());
+    }
+    return Base::Exec(ctx, batch, out);
+  }
+};
+
+template <typename Op>
+std::shared_ptr<ScalarFunction> MakeBetweenFunction(std::string name,
+                                                    const FunctionDoc* doc) {
+  auto func = std::make_shared<CompareFunction>(name, Arity::Ternary(), doc);
+
+  DCHECK_OK(func->AddKernel(
+      {boolean(), boolean(), boolean()}, boolean(),
+      ScalarTernary<BooleanType, BooleanType, BooleanType, BooleanType, Op>::Exec));
+
+  for (const std::shared_ptr<DataType>& ty : IntTypes()) {
+    AddIntegerBetween<Op>(ty, func.get());
+  }
+
+  AddIntegerBetween<Op>(date32(), func.get());
+  AddIntegerBetween<Op>(date64(), func.get());
+
+  AddGenericBetween<FloatType, Op>(float32(), func.get());
+  AddGenericBetween<DoubleType, Op>(float64(), func.get());
+
+  // Add timestamp kernels
+  for (auto unit : TimeUnit::values()) {
+    InputType in_type(match::TimestampTypeUnit(unit));
+    DCHECK_OK(func->AddKernel({in_type, in_type, in_type}, boolean(),
+                              BetweenTimestamps<BooleanType, TimestampType, Op>::Exec));
+  }
+
+  // Duration
+  for (auto unit : TimeUnit::values()) {
+    InputType in_type(match::DurationTypeUnit(unit));
+    auto exec =
+        GeneratePhysicalInteger<ScalarTernaryEqualTypes, BooleanType, Op>(int64());
+    DCHECK_OK(func->AddKernel({in_type, in_type, in_type}, boolean(), std::move(exec)));
+  }
+
+  // Time32 and Time64
+  for (auto unit : {TimeUnit::SECOND, TimeUnit::MILLI}) {
+    InputType in_type(match::Time32TypeUnit(unit));
+    auto exec =
+        GeneratePhysicalInteger<ScalarTernaryEqualTypes, BooleanType, Op>(int32());
+    DCHECK_OK(func->AddKernel({in_type, in_type, in_type}, boolean(), std::move(exec)));
+  }
+  for (auto unit : {TimeUnit::MICRO, TimeUnit::NANO}) {
+    InputType in_type(match::Time64TypeUnit(unit));
+    auto exec =
+        GeneratePhysicalInteger<ScalarTernaryEqualTypes, BooleanType, Op>(int64());
+    DCHECK_OK(func->AddKernel({in_type, in_type, in_type}, boolean(), std::move(exec)));
+  }
+
+  for (const std::shared_ptr<DataType>& ty : BaseBinaryTypes()) {
+    auto exec = GenerateVarBinaryBase<ScalarTernaryEqualTypes, BooleanType, Op>(*ty);

Review comment:
       No longer relevant.




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

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

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



[GitHub] [arrow] bkmgit commented on a change in pull request #11882: ARROW-9843: [C++][Python] Implement Between ternary kernel and Python bindings

Posted by GitBox <gi...@apache.org>.
bkmgit commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r791363813



##########
File path: cpp/src/arrow/compute/kernels/scalar_compare_test.cc
##########
@@ -1850,5 +1851,154 @@ TEST(TestMaxElementWiseMinElementWise, CommonTemporal) {
               ResultWith(ScalarFromJSON(date64(), "86400000")));
 }
 
+template <typename ArrowType>
+static void ValidateBetween(const Datum& val, const Datum& lhs, const Datum& rhs,
+                            const Datum& expected) {
+  ASSERT_OK_AND_ASSIGN(Datum result, Between(val, lhs, rhs));
+  AssertArraysEqual(*expected.make_array(), *result.make_array(),
+                    /*verbose=*/true);
+}
+
+template <typename ArrowType>
+static void ValidateBetween(const char* value_str, const Datum& lhs, const Datum& rhs,
+                            const char* expected_str) {
+  auto value = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), value_str);
+  auto expected = ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), expected_str);
+  ValidateBetween<ArrowType>(value, lhs, rhs, expected);
+}
+
+template <>
+void ValidateBetween<StringType>(const char* value_str, const Datum& lhs,
+                                 const Datum& rhs, const char* expected_str) {
+  auto value = ArrayFromJSON(utf8(), value_str);
+  auto expected = ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), expected_str);
+  ValidateBetween<StringType>(value, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+class TestNumericBetweenKernel : public ::testing::Test {};
+
+TYPED_TEST_SUITE(TestNumericBetweenKernel, NumericArrowTypes);
+TYPED_TEST(TestNumericBetweenKernel, SimpleBetweenArrayScalarScalar) {
+  using ScalarType = typename TypeTraits<TypeParam>::ScalarType;
+  using CType = typename TypeTraits<TypeParam>::CType;
+
+  Datum zero(std::make_shared<ScalarType>(CType(0)));
+  Datum four(std::make_shared<ScalarType>(CType(4)));
+  ValidateBetween<TypeParam>("[]", zero, four, "[]");
+  ValidateBetween<TypeParam>("[null]", zero, four, "[null]");
+  ValidateBetween<TypeParam>("[0,0,1,1,2,2]", zero, four, "[0,0,1,1,1,1]");
+  ValidateBetween<TypeParam>("[0,1,2,3,4,5]", zero, four, "[0,1,1,1,0,0]");
+  ValidateBetween<TypeParam>("[5,4,3,2,1,0]", zero, four, "[0,0,1,1,1,0]");
+  ValidateBetween<TypeParam>("[null,0,1,1]", zero, four, "[null,0,1,1]");
+}
+
+TEST(TestSimpleBetweenKernel, SimpleStringTest) {
+  using ScalarType = typename TypeTraits<StringType>::ScalarType;
+  auto l = Datum(std::make_shared<ScalarType>("abc"));
+  auto r = Datum(std::make_shared<ScalarType>("zzz"));
+  ValidateBetween<StringType>("[]", l, r, "[]");
+  ValidateBetween<StringType>("[null]", l, r, "[null]");
+  ValidateBetween<StringType>(R"(["aaa", "aaaa", "ccc", "z"])", l, r,
+                              R"([false, false, true, true])");
+  ValidateBetween<StringType>(R"(["a", "aaaa", "c", "z"])", l, r,
+                              R"([false, false, true, true])");
+  ValidateBetween<StringType>(R"(["a", "aaaa", "fff", "zzzz"])", l, r,
+                              R"([false, false, true, false])");
+  ValidateBetween<StringType>(R"(["abd", null, null, "zzx"])", l, r,
+                              R"([true, null, null, true])");
+}
+
+TEST(TestSimpleBetweenKernel, SimpleTimestampTest) {
+  using ScalarType = typename TypeTraits<TimestampType>::ScalarType;
+  auto checkTimestampArray = [](std::shared_ptr<DataType> type, const char* input_str,
+                                const Datum& lhs, const Datum& rhs,
+                                const char* expected_str) {
+    auto value = ArrayFromJSON(type, input_str);
+    auto expected = ArrayFromJSON(boolean(), expected_str);
+    ValidateBetween<TimestampType>(value, lhs, rhs, expected);
+  };
+  auto unit = TimeUnit::SECOND;
+  auto l = Datum(std::make_shared<ScalarType>(923184000, timestamp(unit)));
+  auto r = Datum(std::make_shared<ScalarType>(1602032602, timestamp(unit)));
+  checkTimestampArray(timestamp(unit), "[]", l, r, "[]");
+  checkTimestampArray(timestamp(unit), "[null]", l, r, "[null]");
+  checkTimestampArray(timestamp(unit), R"(["1970-01-01","2000-02-29","1900-02-28"])", l,
+                      r, "[false,true,false]");
+  checkTimestampArray(timestamp(unit), R"(["1970-01-01","2000-02-29","2004-02-28"])", l,
+                      r, "[false,true,true]");
+  checkTimestampArray(timestamp(unit), R"(["2018-01-01","1999-04-04","1900-02-28"])", l,
+                      r, "[true,false,false]");
+}
+
+TYPED_TEST(TestNumericBetweenKernel, SimpleBetweenArrayArrayArray) {
+  ValidateBetween<TypeParam>(
+      "[]", ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"), "[]");
+  ValidateBetween<TypeParam>(
+      "[null]", ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"), "[null]");
+  ValidateBetween<TypeParam>(
+      "[1,1,2,2,2]",
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      "[true,true,false,false,false]");
+  ValidateBetween<TypeParam>(
+      "[1,1,2,2,2,2]",
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,2,5,5]"),
+      "[true,true,false,null,false,false]");
+}
+
+TEST(TestSimpleBetweenKernel, StringArrayArrayArrayTest) {
+  ValidateBetween<StringType>(
+      R"(["david","hello","world"])",
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["adam","hi","whirl"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                    R"(["robert","goeiemoreen","whirlwind"])"),
+      "[true, false, false]");
+  ValidateBetween<StringType>(
+      R"(["x","a","f"])",
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["w","a","e"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["z","a","g"])"),
+      "[true, false, true]");
+  ValidateBetween<StringType>(
+      R"(["block","bit","binary"])",
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                    R"(["bit","nibble","ternary"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["word","d","xyz"])"),
+      "[true, false, false]");
+  ValidateBetween<StringType>(R"(["Ayumi","アユミ","王梦莹"])",
+                              ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                            R"(["たなか","あゆみ","歩美"])"),
+                              ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                            R"(["李平之","田中","たなか"])"),
+                              "[false, true, false]");
+}
+
+TEST(TestSimpleBetweenKernel, TimestampArrayArrayArrayTest) {

Review comment:
       Added different time units.




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

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

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



[GitHub] [arrow] bkmgit commented on pull request #11882: ARROW-9843: [C++][Python] Implement Between ternary kernel and Python bindings

Posted by GitBox <gi...@apache.org>.
bkmgit commented on pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#issuecomment-1021091591


   Implicit casting is not implemented for between, but is available in the other comparison kernels. Created another [issue](https://issues.apache.org/jira/browse/ARROW-15446) for this.


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

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

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



[GitHub] [arrow] pitrou commented on a change in pull request #11882: ARROW-9843: [C++][Python] Implement Between ternary kernel and Python bindings

Posted by GitBox <gi...@apache.org>.
pitrou commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r793690969



##########
File path: cpp/src/arrow/compute/kernels/scalar_compare.cc
##########
@@ -156,39 +212,49 @@ struct Maximum {
   }
 };
 
+// Check if timestamp timezones are comparable (either all are empty or none is).
+Status CheckCompareTimestamps(const ExecBatch& batch) {
+  if (batch.num_values() > 0) {
+    int invalid_states = 0;
+    for (int i = 0; i < batch.num_values(); i++) {
+      const auto& tsi = checked_cast<const TimestampType&>(*batch[i].type());
+      invalid_states += int(tsi.timezone().empty());
+    }
+    if (invalid_states * (invalid_states - batch.num_values()) != 0) {
+      return Status::TypeError(
+          "Cannot compare timestamp with timezone to timestamp without timezone");
+    }

Review comment:
       Ok, the logic here is still gratuitously sophisticated. How about (untested):
   
   ```suggestion
       bool have_no_timezone =
           checked_cast<const TimestampType&>(*batch[0].type()).timezone().empty();
       for (int i = 1; i < batch.num_values(); i++) {
         if (checked_cast<const TimestampType&>(*batch[i].type()).timezone().empty() !=
             have_no_timezone) {
           return Status::TypeError(
               "Cannot compare timestamp with timezone to timestamp without timezone");
         }
       }
   ```

##########
File path: cpp/src/arrow/compute/kernels/scalar_compare.cc
##########
@@ -746,6 +805,199 @@ std::shared_ptr<ScalarFunction> MakeScalarMinMax(std::string name,
   return func;
 }
 
+template <template <BetweenOptions::Inclusive> class Op>
+Status MakeBetweenArrayExec(Type::type type_id, KernelContext* ctx,
+                            const ExecBatch& batch, Datum* out) {
+  using BetweenState = OptionsWrapper<BetweenOptions>;
+  const auto& state = static_cast<const BetweenState&>(*ctx->state());
+  if (type_id == Type::FIXED_SIZE_BINARY) {
+    switch (state.options.inclusive) {
+      case BetweenOptions::Inclusive::BOTH:
+        return ScalarTernaryEqualTypes<BooleanType, FixedSizeBinaryType,
+                                       Op<BetweenOptions::Inclusive::BOTH>>::Exec(ctx,
+                                                                                  batch,
+                                                                                  out);
+      case BetweenOptions::Inclusive::LEFT:
+        return ScalarTernaryEqualTypes<BooleanType, FixedSizeBinaryType,
+                                       Op<BetweenOptions::Inclusive::LEFT>>::Exec(ctx,
+                                                                                  batch,
+                                                                                  out);
+      case BetweenOptions::Inclusive::RIGHT:
+        return ScalarTernaryEqualTypes<BooleanType, FixedSizeBinaryType,
+                                       Op<BetweenOptions::Inclusive::RIGHT>>::Exec(ctx,
+                                                                                   batch,
+                                                                                   out);
+      case BetweenOptions::Inclusive::NEITHER:
+        return ScalarTernaryEqualTypes<
+            BooleanType, FixedSizeBinaryType,
+            Op<BetweenOptions::Inclusive::NEITHER>>::Exec(ctx, batch, out);
+      default:
+        return Status::NotImplemented("between inclusiveness not implemented: ",
+                                      state.options.ToString());
+    }
+  } else if (type_id == Type::DECIMAL128 || type_id == Type::DECIMAL256) {

Review comment:
       (untested, but should probably work)
   
   ```suggestion
     } else if (is_decimal(type_id)) {
   ```

##########
File path: cpp/src/arrow/compute/api_scalar.h
##########
@@ -1350,5 +1363,26 @@ ARROW_EXPORT Result<Datum> AssumeTimezone(const Datum& values,
                                           AssumeTimezoneOptions options,
                                           ExecContext* ctx = NULLPTR);
 
+/// \brief Between compares each element in `values`
+/// with `left` as a lower bound and 'right' as an upper bound

Review comment:
       ```suggestion
   /// \brief Compute whether each element in `values` is between `left` and `right`
   ```

##########
File path: cpp/src/arrow/util/bit_block_counter.h
##########
@@ -422,6 +426,96 @@ class ARROW_EXPORT OptionalBinaryBitBlockCounter {
   }
 };
 
+/// \brief A class that computes popcounts on the result of bitwise operations
+/// between three bitmaps, 64 bits at a time. A 64-bit word is loaded from each
+/// bitmap, then the popcount is computed on e.g. the bitwise-and of the three
+/// words.
+class ARROW_EXPORT TernaryBitBlockCounter {

Review comment:
       Apparently you forgot to remove it :-)

##########
File path: cpp/src/arrow/compute/kernels/scalar_compare.cc
##########
@@ -772,6 +1024,16 @@ const FunctionDoc less_equal_doc{
     ("A null on either side emits a null comparison result."),
     {"x", "y"}};
 
+const FunctionDoc between_doc{
+    "Check if values are in the given range, val between a and b",

Review comment:
       ```suggestion
       "Check if values are in the given range",
   ```

##########
File path: cpp/src/arrow/ipc/json_simple.cc
##########
@@ -983,6 +983,16 @@ Status ScalarFromJSON(const std::shared_ptr<DataType>& type,
   return Status::OK();
 }
 
+Status ScalarFromJSON(const std::shared_ptr<DataType>& type,

Review comment:
       These two overloads are not used anywhere, are they?
   (they are not even exposed in `json_simple.h`)

##########
File path: python/pyarrow/tests/test_compute.py
##########
@@ -1360,6 +1361,27 @@ def test_filter_null_type():
     assert len(table.filter(mask).column(0)) == 5
 
 
+@pytest.mark.parametrize("ty", ["inclusive"])
+def test_between_array_array_scalar(ty):
+    BetweenOptions = partial(pc.BetweenOptions)
+
+    val = pa.array([1, 1, 4, 3, 2, 5])
+    arr1 = pa.array([1, 2, 3, 2, None, 2])
+    scalar2 = pa.scalar(4)
+
+    inclusive_and_expected = {
+        "both": [True, False, True, True, None, False],
+        "left": [True, False, False, True, None, False],
+        "right": [False, False, True, True, None, False],
+        "neither": [False, False, False, True, None, False],
+    }
+
+    for inclusive, expected in inclusive_and_expected.items():
+        options = BetweenOptions(inclusive=inclusive)
+        result = pc.between(val, arr1, scalar2, options=options)

Review comment:
       You might just as well pass the option directly:
   ```suggestion
           result = pc.between(val, arr1, scalar2, inclusive=inclusive)
   ```

##########
File path: cpp/src/arrow/compute/kernels/scalar_compare.cc
##########
@@ -746,6 +805,199 @@ std::shared_ptr<ScalarFunction> MakeScalarMinMax(std::string name,
   return func;
 }
 
+template <template <BetweenOptions::Inclusive> class Op>
+Status MakeBetweenArrayExec(Type::type type_id, KernelContext* ctx,
+                            const ExecBatch& batch, Datum* out) {
+  using BetweenState = OptionsWrapper<BetweenOptions>;
+  const auto& state = static_cast<const BetweenState&>(*ctx->state());
+  if (type_id == Type::FIXED_SIZE_BINARY) {
+    switch (state.options.inclusive) {
+      case BetweenOptions::Inclusive::BOTH:
+        return ScalarTernaryEqualTypes<BooleanType, FixedSizeBinaryType,
+                                       Op<BetweenOptions::Inclusive::BOTH>>::Exec(ctx,
+                                                                                  batch,
+                                                                                  out);
+      case BetweenOptions::Inclusive::LEFT:
+        return ScalarTernaryEqualTypes<BooleanType, FixedSizeBinaryType,
+                                       Op<BetweenOptions::Inclusive::LEFT>>::Exec(ctx,
+                                                                                  batch,
+                                                                                  out);
+      case BetweenOptions::Inclusive::RIGHT:
+        return ScalarTernaryEqualTypes<BooleanType, FixedSizeBinaryType,
+                                       Op<BetweenOptions::Inclusive::RIGHT>>::Exec(ctx,
+                                                                                   batch,
+                                                                                   out);
+      case BetweenOptions::Inclusive::NEITHER:
+        return ScalarTernaryEqualTypes<
+            BooleanType, FixedSizeBinaryType,
+            Op<BetweenOptions::Inclusive::NEITHER>>::Exec(ctx, batch, out);
+      default:
+        return Status::NotImplemented("between inclusiveness not implemented: ",
+                                      state.options.ToString());
+    }
+  } else if (type_id == Type::DECIMAL128 || type_id == Type::DECIMAL256) {
+    switch (state.options.inclusive) {
+      case BetweenOptions::Inclusive::BOTH:
+        return GenerateDecimal<ScalarTernaryEqualTypes, BooleanType,
+                               Op<BetweenOptions::Inclusive::BOTH>>(type_id)(ctx, batch,
+                                                                             out);
+      case BetweenOptions::Inclusive::LEFT:
+        return GenerateDecimal<ScalarTernaryEqualTypes, BooleanType,
+                               Op<BetweenOptions::Inclusive::LEFT>>(type_id)(ctx, batch,
+                                                                             out);
+      case BetweenOptions::Inclusive::RIGHT:
+        return GenerateDecimal<ScalarTernaryEqualTypes, BooleanType,
+                               Op<BetweenOptions::Inclusive::RIGHT>>(type_id)(ctx, batch,
+                                                                              out);
+      case BetweenOptions::Inclusive::NEITHER:
+        return GenerateDecimal<ScalarTernaryEqualTypes, BooleanType,
+                               Op<BetweenOptions::Inclusive::NEITHER>>(type_id)(
+            ctx, batch, out);
+      default:
+        return Status::NotImplemented("between inclusiveness not implemented: ",
+                                      state.options.ToString());
+    }
+  } else if (type_id == Type::BINARY || type_id == Type::STRING ||
+             type_id == Type::LARGE_BINARY || type_id == Type::LARGE_STRING) {

Review comment:
       (untested, but should probably work)
   ```suggestion
     } else if (is_base_binary_like(type_id)) {
   ```

##########
File path: python/pyarrow/tests/test_compute.py
##########
@@ -1360,6 +1361,27 @@ def test_filter_null_type():
     assert len(table.filter(mask).column(0)) == 5
 
 
+@pytest.mark.parametrize("ty", ["inclusive"])
+def test_between_array_array_scalar(ty):
+    BetweenOptions = partial(pc.BetweenOptions)

Review comment:
       This is harmless, but does it actually serve a purpose?

##########
File path: cpp/src/arrow/compute/kernels/scalar_compare_test.cc
##########
@@ -1865,5 +1928,654 @@ TEST(TestMaxElementWiseMinElementWise, CommonTemporal) {
               ResultWith(ScalarFromJSON(date64(), "86400000")));
 }
 
+static void ValidateBetween(BetweenOptions options, const Datum& val, const Datum& lhs,
+                            const Datum& rhs, const Datum& expected) {
+  ASSERT_OK_AND_ASSIGN(Datum result, Between(val, lhs, rhs, options, nullptr));
+  if ((val.is_scalar()) && (lhs.is_scalar()) && (rhs.is_scalar())) {
+    AssertScalarsEqual(*expected.scalar(), *result.scalar(), /*verbose=*/true);
+  } else {
+    AssertArraysEqual(*expected.make_array(), *result.make_array(),
+                      /*verbose=*/true);
+  }
+}
+
+void ValidateBetween(const Datum& val, const Datum& lhs, const Datum& rhs) {
+  CompareOperator lhs_val;
+  CompareOperator val_rhs;
+  for (auto inclusive :
+       {BetweenOptions::Inclusive::BOTH, BetweenOptions::Inclusive::LEFT,
+        BetweenOptions::Inclusive::RIGHT, BetweenOptions::Inclusive::NEITHER}) {
+    auto options = BetweenOptions(inclusive);
+    if (inclusive == BetweenOptions::Inclusive::NEITHER) {
+      lhs_val = LESS;
+      val_rhs = LESS;
+    } else if (inclusive == BetweenOptions::Inclusive::LEFT) {
+      lhs_val = LESS_EQUAL;
+      val_rhs = LESS;
+    } else if (inclusive == BetweenOptions::Inclusive::RIGHT) {
+      lhs_val = LESS;
+      val_rhs = LESS_EQUAL;
+    } else {
+      lhs_val = LESS_EQUAL;
+      val_rhs = LESS_EQUAL;
+    }
+
+    ASSERT_OK_AND_ASSIGN(
+        Datum resultl, CallFunction(CompareOperatorToFunctionName(lhs_val), {lhs, val}));
+    ASSERT_OK_AND_ASSIGN(
+        Datum resultr, CallFunction(CompareOperatorToFunctionName(val_rhs), {val, rhs}));
+    ASSERT_OK_AND_ASSIGN(Datum expected, CallFunction("and", {resultl, resultr}));
+
+    ValidateBetween(options, val, lhs, rhs, expected);
+  }
+}
+
+class TestNumericBetweenKernel : public ::testing::Test {};
+
+std::shared_ptr<DataType> GetType(std::shared_ptr<DataType> type) {
+  auto type_string = type->ToString();
+  if (type_string == "duration[s]") {
+    return int64();
+  } else if (type_string == "duration[ms]") {
+    return int64();
+  } else if (type_string == "duration[ns]") {
+    return int64();
+  } else if (type_string == "uint8") {
+    return uint8();
+  } else if (type_string == "uint16") {
+    return uint16();
+  } else if (type_string == "uint32") {
+    return uint32();
+  } else if (type_string == "uint64") {
+    return uint64();
+  } else if (type_string == "int8") {
+    return int8();
+  } else if (type_string == "int16") {
+    return int16();
+  } else if (type_string == "int32") {
+    return int32();
+  } else if (type_string == "int64") {
+    return int64();
+  } else if (type_string == "float") {
+    return float32();
+  } else if (type_string == "double") {
+    return float64();
+  } else {
+    return int64();
+  }
+}
+
+TEST(TestNumericBetweenKernel, 3Scalars) {
+  for (const auto& types : {DurationTypes(), NumericTypes()}) {
+    for (const std::shared_ptr<DataType>& ty : types) {
+      ARROW_SCOPED_TRACE("type = ", ty->ToString());
+      auto tt = GetType(ty);
+      auto zero = Datum(ScalarFromJSON(tt, "0"));
+      auto two = Datum(ScalarFromJSON(tt, "2"));
+      auto four = Datum(ScalarFromJSON(tt, "4"));
+      auto null = Datum(ScalarFromJSON(tt, "null"));
+      ValidateBetween(zero, two, four);
+      ValidateBetween(two, zero, four);
+      ValidateBetween(two, two, four);
+      ValidateBetween(four, two, four);
+      ValidateBetween(null, two, four);
+      ValidateBetween(two, null, four);
+      ValidateBetween(two, zero, null);
+    }
+  }
+}
+
+TEST(TestNumericBetweenKernel, 1Array2Scalars) {
+  for (const auto& types : {DurationTypes(), NumericTypes()}) {
+    for (const std::shared_ptr<DataType>& ty : types) {
+      ARROW_SCOPED_TRACE("type = ", ty->ToString());
+      auto tt = GetType(ty);
+      auto zero = Datum(ScalarFromJSON(tt, "0"));
+      auto four = Datum(ScalarFromJSON(tt, "4"));
+      auto null = Datum(ScalarFromJSON(tt, "null"));
+      ValidateBetween(Datum(ArrayFromJSON(tt, "[]")), zero, four);
+      ValidateBetween(Datum(ArrayFromJSON(tt, "[null]")), zero, four);
+      ValidateBetween(Datum(ArrayFromJSON(tt, "[0,1,2,3,4,5]")), zero, four);
+      ValidateBetween(Datum(ArrayFromJSON(tt, "[null,0,1,1]")), zero, four);
+      ValidateBetween(Datum(ArrayFromJSON(tt, "[5,4,3,2,1,0]")), null, four);
+      ValidateBetween(Datum(ArrayFromJSON(tt, "[5,4,3,2,1,0]")), zero, null);
+      ValidateBetween(zero, Datum(ArrayFromJSON(tt, "[]")), four);
+      ValidateBetween(zero, Datum(ArrayFromJSON(tt, "[null]")), four);
+      ValidateBetween(zero, Datum(ArrayFromJSON(tt, "[0,1,2,3,4,5]")), four);
+      ValidateBetween(zero, Datum(ArrayFromJSON(tt, "[null,0,1,1]")), four);
+      ValidateBetween(null, Datum(ArrayFromJSON(tt, "[5,4,3,2,1,0]")), four);
+      ValidateBetween(zero, Datum(ArrayFromJSON(tt, "[5,4,3,2,1,0]")), null);
+      ValidateBetween(zero, four, Datum(ArrayFromJSON(tt, "[]")));
+      ValidateBetween(zero, four, Datum(ArrayFromJSON(tt, "[null]")));
+      ValidateBetween(zero, four, Datum(ArrayFromJSON(tt, "[0,1,2,3,4,5]")));
+      ValidateBetween(zero, four, Datum(ArrayFromJSON(tt, "[null,0,1,1]")));
+      ValidateBetween(null, four, Datum(ArrayFromJSON(tt, "[5,4,3,2,1,0]")));
+      ValidateBetween(zero, null, Datum(ArrayFromJSON(tt, "[5,4,3,2,1,0]")));
+    }
+  }
+}
+
+TEST(TestNumericBetweenKernel, 2Arrays1Scalar) {
+  for (const auto& types : {DurationTypes(), NumericTypes()}) {
+    for (const std::shared_ptr<DataType>& ty : types) {
+      ARROW_SCOPED_TRACE("type = ", ty->ToString());
+      auto tt = GetType(ty);
+      auto one = Datum(ScalarFromJSON(tt, "1"));
+      ValidateBetween(one, Datum(ArrayFromJSON(tt, "[]")),
+                      Datum(ArrayFromJSON(tt, "[]")));
+      ValidateBetween(one, Datum(ArrayFromJSON(tt, "[null]")),
+                      Datum(ArrayFromJSON(tt, "[null]")));
+      ValidateBetween(one, Datum(ArrayFromJSON(tt, "[0,0,1,3,3]")),
+                      Datum(ArrayFromJSON(tt, "[10,10,2,5,5]")));
+      ValidateBetween(one, Datum(ArrayFromJSON(tt, "[0,0,1,null,3,3]")),
+                      Datum(ArrayFromJSON(tt, "[0,10,2,2,5,5]")));
+      ValidateBetween(Datum(ArrayFromJSON(tt, "[]")), one,
+                      Datum(ArrayFromJSON(tt, "[]")));
+      ValidateBetween(Datum(ArrayFromJSON(tt, "[null]")), one,
+                      Datum(ArrayFromJSON(tt, "[null]")));
+      ValidateBetween(Datum(ArrayFromJSON(tt, "[0,0,1,3,3]")), one,
+                      Datum(ArrayFromJSON(tt, "[10,10,2,5,5]")));
+      ValidateBetween(Datum(ArrayFromJSON(tt, "[0,0,1,null,3,3]")), one,
+                      Datum(ArrayFromJSON(tt, "[0,10,2,2,5,5]")));
+      ValidateBetween(Datum(ArrayFromJSON(tt, "[]")), Datum(ArrayFromJSON(tt, "[]")),
+                      one);
+      ValidateBetween(Datum(ArrayFromJSON(tt, "[null]")),
+                      Datum(ArrayFromJSON(tt, "[null]")), one);
+      ValidateBetween(Datum(ArrayFromJSON(tt, "[0,0,1,3,3]")),
+                      Datum(ArrayFromJSON(tt, "[10,10,2,5,5]")), one);
+      ValidateBetween(Datum(ArrayFromJSON(tt, "[0,0,1,null,3,3]")),
+                      Datum(ArrayFromJSON(tt, "[0,10,2,2,5,5]")), one);
+    }
+  }
+}
+
+TEST(TestNumericBetweenKernel, 3Arrays) {
+  for (const auto& types : {DurationTypes(), NumericTypes()}) {
+    for (const std::shared_ptr<DataType>& ty : types) {
+      ARROW_SCOPED_TRACE("type = ", ty->ToString());
+      ValidateBetween(Datum(ArrayFromJSON(ty, "[]")), Datum(ArrayFromJSON(ty, "[]")),
+                      Datum(ArrayFromJSON(ty, "[]")));
+      ValidateBetween(Datum(ArrayFromJSON(ty, "[null]")),
+                      Datum(ArrayFromJSON(ty, "[null]")),
+                      Datum(ArrayFromJSON(ty, "[null]")));
+      ValidateBetween(Datum(ArrayFromJSON(ty, "[1,1,2,2,2]")),
+                      Datum(ArrayFromJSON(ty, "[0,0,1,3,3]")),
+                      Datum(ArrayFromJSON(ty, "[10,10,2,5,5]")));
+      ValidateBetween(Datum(ArrayFromJSON(ty, "[0,1,2,2,2,2]")),
+                      Datum(ArrayFromJSON(ty, "[0,0,1,null,3,3]")),
+                      Datum(ArrayFromJSON(ty, "[0,10,2,2,5,5]")));
+    }
+  }
+}
+
+TEST(TestNumericBetweenKernel, Random) {
+  for (const auto& types : {DurationTypes(), NumericTypes()}) {
+    for (const std::shared_ptr<DataType>& ty : types) {
+      auto rand = random::RandomArrayGenerator(0x5416447);
+      const int64_t length = 100;
+      for (auto null_probability : {0.0, 0.01, 0.1, 0.25, 0.5, 1.0}) {
+        ARROW_SCOPED_TRACE("type = ", ty->ToString());
+        auto tt = GetType(ty);
+        auto metadata =
+            key_value_metadata({"null_probability"}, {std::to_string(null_probability)});
+        auto field = ::arrow::field("[0,100]", std::move(ty), std::move(metadata));
+        auto data1 = rand.ArrayOf(*field, length);
+        auto data2 = rand.ArrayOf(*field, length);
+        auto data3 = rand.ArrayOf(*field, length);
+
+        // Create view of data as the type (e.g. float64)
+        auto array1 = Datum(*data1->View(tt));
+        auto array2 = Datum(*data2->View(tt));
+        auto array3 = Datum(*data3->View(tt));
+        auto scalar1 = Datum(ScalarFromJSON(tt, "10"));
+        auto scalar2 = Datum(ScalarFromJSON(tt, "30"));
+        auto scalar3 = Datum(ScalarFromJSON(tt, "50"));
+        ValidateBetween(scalar1, scalar2, scalar3);
+	ValidateBetween(array1, scalar2, scalar3);
+        ValidateBetween(array1, array2, scalar3);
+        ValidateBetween(array1, array2, array3);
+        ValidateBetween(array1, scalar2, scalar3);
+        ValidateBetween(scalar1, array2, array3);
+        ValidateBetween(scalar1, array2, scalar3);
+        ValidateBetween(scalar1, scalar2, array3);
+        ValidateBetween(array1, scalar2, array3);
+      }
+    }
+  }
+}
+
+class TestStringBetweenKernel : public ::testing::Test {};
+
+TEST(TestStringBetweenKernel, Random) {
+  using ScalarType = typename TypeTraits<StringType>::ScalarType;
+
+  auto rand = random::RandomArrayGenerator(0x5416447);
+  for (size_t i = 3; i < 10; i++) {
+    for (auto null_probability : {0.0, 0.01, 0.1, 0.25, 0.5, 1.0}) {
+      const int64_t length = static_cast<int64_t>(1ULL << i);
+      auto array1 = Datum(rand.String(length, 0, 16, null_probability));
+      auto array2 = Datum(rand.String(length, 0, 16, null_probability));
+      auto array3 = Datum(rand.String(length, 0, 16, null_probability));
+      auto scalar1 = Datum(std::make_shared<ScalarType>("fupi"));
+      auto scalar2 = Datum(std::make_shared<ScalarType>("tupu"));
+      auto scalar3 = Datum(std::make_shared<ScalarType>("zito"));
+      ValidateBetween(scalar1, scalar2, scalar3);
+      ValidateBetween(array1, scalar2, scalar3);
+      ValidateBetween(scalar1, array2, scalar3);
+      ValidateBetween(scalar1, scalar2, array3);
+      ValidateBetween(scalar1, array2, array3);
+      ValidateBetween(array1, scalar2, array3);
+      ValidateBetween(array1, array2, scalar3);
+      ValidateBetween(array1, array2, array3);
+    }
+  }
+}
+
+TEST(TestStringBetweenKernel, 1Array2Scalars) {
+  using ScalarType = typename TypeTraits<StringType>::ScalarType;
+  auto l = Datum(std::make_shared<ScalarType>("abc"));
+  auto r = Datum(std::make_shared<ScalarType>("zzz"));
+  ValidateBetween(Datum(ArrayFromJSON(TypeTraits<StringType>::type_singleton(), "[]")), l,
+                  r);
+  ValidateBetween(
+      Datum(ArrayFromJSON(TypeTraits<StringType>::type_singleton(), "[null]")), l, r);
+  ValidateBetween(Datum(ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                      R"(["aaa", "aaaa", "ccc", "z"])")),
+                  l, r);
+  ValidateBetween(Datum(ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                      R"(["abc", "baa", "fff", "zzz"])")),
+                  l, r);
+  ValidateBetween(Datum(ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                      R"(["abd", null, null, "zzx"])")),
+                  l, r);
+  ValidateBetween(l, Datum(ArrayFromJSON(TypeTraits<StringType>::type_singleton(), "[]")),
+                  r);
+  ValidateBetween(l,
+      Datum(ArrayFromJSON(TypeTraits<StringType>::type_singleton(), "[null]")), r);
+  ValidateBetween(l, Datum(ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                      R"(["aaa", "aaaa", "ccc", "z"])")),
+                  r);
+  ValidateBetween(l, Datum(ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                      R"(["abc", "baa", "fff", "zzz"])")),
+                  r);
+  ValidateBetween(l, Datum(ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                      R"(["abd", null, null, "zzx"])")),
+                  r);
+  ValidateBetween(l, r,
+                  Datum(ArrayFromJSON(TypeTraits<StringType>::type_singleton(), "[]")));
+  ValidateBetween(l, r,
+      Datum(ArrayFromJSON(TypeTraits<StringType>::type_singleton(), "[null]")));
+  ValidateBetween(l, r, Datum(ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                       R"(["aaa", "aaaa", "ccc", "z"])")));
+  ValidateBetween(l, r, Datum(ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                      R"(["abc", "baa", "fff", "zzz"])")));
+  ValidateBetween(l, r, Datum(ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                      R"(["abd", null, null, "zzx"])")));
+}
+
+TEST(TestStringBetweenKernel, 2Arrays1Scalar) {
+  using ScalarType = typename TypeTraits<StringType>::ScalarType;
+  auto r = Datum(std::make_shared<ScalarType>("zzz"));
+  ValidateBetween(r, Datum(ArrayFromJSON(TypeTraits<StringType>::type_singleton(), "[]")),
+                  Datum(ArrayFromJSON(TypeTraits<StringType>::type_singleton(), "[]")));
+  ValidateBetween(r, Datum(ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                   R"(["aaa", "aaaa", "ccc", "z"])")),
+                  Datum(ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                R"(["abc", "baa", "fff", "zzz"])")));
+  ValidateBetween(r, Datum(ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                   R"(["abc", "baa", "fff", "zzz"])")),
+                  Datum(ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                R"(["abd", null, null, "zzx"])")));
+  ValidateBetween(Datum(ArrayFromJSON(TypeTraits<StringType>::type_singleton(), "[]")), r,
+                  Datum(ArrayFromJSON(TypeTraits<StringType>::type_singleton(), "[]")));
+  ValidateBetween(Datum(ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                R"(["aaa", "aaaa", "ccc", "z"])")), r,
+                  Datum(ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                R"(["abc", "baa", "fff", "zzz"])")));
+  ValidateBetween(Datum(ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                R"(["abc", "baa", "fff", "zzz"])")), r,
+                  Datum(ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                R"(["abd", null, null, "zzx"])")));
+  ValidateBetween(Datum(ArrayFromJSON(TypeTraits<StringType>::type_singleton(), "[]")),
+                  Datum(ArrayFromJSON(TypeTraits<StringType>::type_singleton(), "[]")),
+		  r);
+  ValidateBetween(Datum(ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                R"(["aaa", "aaaa", "ccc", "z"])")),
+		  Datum(ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                R"(["abc", "baa", "fff", "zzz"])")),
+		  r);
+  ValidateBetween(Datum(ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                R"(["abc", "baa", "fff", "zzz"])")),
+                  Datum(ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                R"(["abd", null, null, "zzx"])")),
+		  r);
+}
+
+TEST(TestStringBetweenKernel, 3Arrays) {
+  ValidateBetween(Datum(ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                      R"(["david","hello","world"])")),
+                  Datum(ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                      R"(["adam","hi","whirl"])")),
+                  Datum(ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                      R"(["robert","goeiemoreen","whirlwind"])")));
+  ValidateBetween(
+      Datum(ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["x","a","f"])")),
+      Datum(ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["w","a","e"])")),
+      Datum(ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["z","a","g"])")));
+  ValidateBetween(Datum(ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                      R"(["block","bit","binary"])")),
+                  Datum(ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                      R"(["bit","nibble","ternary"])")),
+                  Datum(ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                      R"(["word","d","xyz"])")));
+  ValidateBetween(Datum(ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                      R"(["よしもと","の","ち"])")),
+                  Datum(ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                      R"(["は","へ","あ"])")),
+                  Datum(ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                      R"(["な","を","ち"])")));
+  ValidateBetween(Datum(ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                      R"(["A","ア","王"])")),
+                  Datum(ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                      R"(["た","あ","歩"])")),
+                  Datum(ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                      R"(["李","田",null])")));
+  ValidateBetween(
+      Datum(ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["Б",null,"Я"])")),
+      Datum(ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["А","Ж","Щ"])")),
+      Datum(ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["Д","Л","Ф"])")));
+}
+
+TEST(TestTimestampsBetweenKernel, 1Array2Scalars) {
+  const std::string scalar1_json = R"("1980-02-02")";
+  const std::string scalar2_json = R"("1970-01-01")";
+  const std::string array_json = R"(["1970-01-02","1980-02-02","1970-02-28"])";
+  // Same units should be fine
+  ValidateBetween(Datum(ArrayFromJSON(timestamp(TimeUnit::SECOND), array_json)),
+                  Datum(ScalarFromJSON(timestamp(TimeUnit::SECOND), scalar1_json)),
+                  Datum(ScalarFromJSON(timestamp(TimeUnit::SECOND), scalar2_json)));
+  ValidateBetween(Datum(ScalarFromJSON(timestamp(TimeUnit::SECOND), scalar1_json)),
+                  Datum(ArrayFromJSON(timestamp(TimeUnit::SECOND), array_json)),
+                  Datum(ScalarFromJSON(timestamp(TimeUnit::SECOND), scalar2_json)));
+  ValidateBetween(Datum(ScalarFromJSON(timestamp(TimeUnit::SECOND), scalar1_json)),
+                  Datum(ScalarFromJSON(timestamp(TimeUnit::SECOND), scalar2_json)),
+                  Datum(ArrayFromJSON(timestamp(TimeUnit::SECOND), array_json)));
+  // Different timezones should be fine
+  ValidateBetween(
+      Datum(ArrayFromJSON(timestamp(TimeUnit::SECOND, "Africa/Cairo"), array_json)),
+      Datum(ScalarFromJSON(timestamp(TimeUnit::SECOND, "America/Chicago"), scalar1_json)),
+      Datum(ScalarFromJSON(timestamp(TimeUnit::SECOND, "Asia/Beijing"), scalar2_json)));
+  ValidateBetween(
+      Datum(ScalarFromJSON(timestamp(TimeUnit::NANO, "Europe/Berlin"), scalar1_json)),
+      Datum(ArrayFromJSON(timestamp(TimeUnit::NANO, "America/Phoenix"), array_json)),
+      Datum(ScalarFromJSON(timestamp(TimeUnit::NANO, "Africa/Nairobi"), scalar2_json)));
+  ValidateBetween(
+      Datum(ScalarFromJSON(timestamp(TimeUnit::NANO, "Europe/Berlin"), scalar1_json)),
+      Datum(ScalarFromJSON(timestamp(TimeUnit::NANO, "Asia/Tokyo"), scalar2_json)),
+      Datum(ArrayFromJSON(timestamp(TimeUnit::NANO, "Africa/Nairobi"), array_json)));
+  // Different units should be fine
+  ValidateBetween(Datum(ArrayFromJSON(timestamp(TimeUnit::SECOND), array_json)),
+                  Datum(ScalarFromJSON(timestamp(TimeUnit::MILLI), scalar1_json)),
+                  Datum(ScalarFromJSON(timestamp(TimeUnit::SECOND), scalar2_json)));
+  ValidateBetween(Datum(ScalarFromJSON(timestamp(TimeUnit::SECOND), scalar1_json)),
+                  Datum(ArrayFromJSON(timestamp(TimeUnit::NANO), array_json)),
+                  Datum(ScalarFromJSON(timestamp(TimeUnit::SECOND), scalar2_json)));
+  ValidateBetween(Datum(ScalarFromJSON(timestamp(TimeUnit::SECOND), scalar1_json)),
+                  Datum(ScalarFromJSON(timestamp(TimeUnit::NANO), scalar2_json)),
+                  Datum(ArrayFromJSON(timestamp(TimeUnit::SECOND), array_json)));
+  // But comparing naive to zoned is not OK
+  for (auto inclusive :
+       {BetweenOptions::Inclusive::BOTH, BetweenOptions::Inclusive::LEFT,
+        BetweenOptions::Inclusive::RIGHT, BetweenOptions::Inclusive::NEITHER}) {
+    auto options = BetweenOptions(inclusive);
+    EXPECT_RAISES_WITH_MESSAGE_THAT(
+        TypeError,
+        ::testing::HasSubstr(
+            "Cannot compare timestamp with timezone to timestamp without timezone"),
+        Between(ArrayFromJSON(timestamp(TimeUnit::SECOND), array_json),
+                ScalarFromJSON(timestamp(TimeUnit::SECOND), scalar1_json),
+                ScalarFromJSON(timestamp(TimeUnit::SECOND, "Asia/Tokyo"), scalar2_json),
+                options, nullptr));
+    EXPECT_RAISES_WITH_MESSAGE_THAT(
+        TypeError,
+        ::testing::HasSubstr(
+            "Cannot compare timestamp with timezone to timestamp without timezone"),
+        Between(ScalarFromJSON(timestamp(TimeUnit::SECOND), scalar1_json),
+                ArrayFromJSON(timestamp(TimeUnit::SECOND, "America/New_York"), array_json),
+                ScalarFromJSON(timestamp(TimeUnit::SECOND, "Europe/Berlin"), scalar2_json),
+                options, nullptr));
+    EXPECT_RAISES_WITH_MESSAGE_THAT(
+        TypeError,
+        ::testing::HasSubstr(
+            "Cannot compare timestamp with timezone to timestamp without timezone"),
+        Between(ScalarFromJSON(timestamp(TimeUnit::SECOND, "Africa/Nairobi"), scalar1_json),
+                ArrayFromJSON(timestamp(TimeUnit::SECOND), array_json),
+                ScalarFromJSON(timestamp(TimeUnit::SECOND), scalar2_json), options, nullptr));
+  }
+}
+
+TEST(TestTimestampsBetweenKernel, 2Arrays1Scalar) {
+  const std::string scalar_json = R"("1980-02-02")";
+  const std::string array1_json = R"(["1970-01-01","1980-02-01","1970-02-28"])";
+  const std::string array2_json = R"(["1970-01-02","1980-02-02","1970-02-28"])";
+  // Same units should be fine
+  ValidateBetween(Datum(ScalarFromJSON(timestamp(TimeUnit::SECOND), scalar_json)),
+                  Datum(ArrayFromJSON(timestamp(TimeUnit::SECOND), array1_json)),
+                  Datum(ArrayFromJSON(timestamp(TimeUnit::SECOND), array2_json)));
+  ValidateBetween(Datum(ArrayFromJSON(timestamp(TimeUnit::SECOND), array1_json)),
+                  Datum(ScalarFromJSON(timestamp(TimeUnit::SECOND), scalar_json)),
+                  Datum(ArrayFromJSON(timestamp(TimeUnit::SECOND), array2_json)));
+  ValidateBetween(Datum(ArrayFromJSON(timestamp(TimeUnit::SECOND), array1_json)),
+                  Datum(ArrayFromJSON(timestamp(TimeUnit::SECOND), array2_json)),
+                  Datum(ScalarFromJSON(timestamp(TimeUnit::SECOND), scalar_json)));
+  // Different timezones should be fine
+  ValidateBetween(
+      Datum(ScalarFromJSON(timestamp(TimeUnit::SECOND, "Africa/Cairo"), scalar_json)),
+      Datum(ArrayFromJSON(timestamp(TimeUnit::SECOND, "America/Chicago"), array1_json)),
+      Datum(ArrayFromJSON(timestamp(TimeUnit::SECOND, "Asia/Beijing"), array2_json)));
+  ValidateBetween(
+      Datum(ArrayFromJSON(timestamp(TimeUnit::NANO, "Europe/Berlin"), array1_json)),
+      Datum(ScalarFromJSON(timestamp(TimeUnit::NANO, "America/Phoenix"), scalar_json)),
+      Datum(ArrayFromJSON(timestamp(TimeUnit::NANO, "Africa/Nairobi"), array2_json)));
+  ValidateBetween(
+      Datum(ArrayFromJSON(timestamp(TimeUnit::NANO, "Europe/Berlin"), array1_json)),
+      Datum(ArrayFromJSON(timestamp(TimeUnit::NANO, "Asia/Tokyo"), array2_json)),
+      Datum(ScalarFromJSON(timestamp(TimeUnit::NANO, "Africa/Nairobi"), scalar_json)));
+  // Different units should be fine
+  ValidateBetween(Datum(ScalarFromJSON(timestamp(TimeUnit::SECOND), scalar_json)),
+                  Datum(ArrayFromJSON(timestamp(TimeUnit::MILLI), array1_json)),
+                  Datum(ArrayFromJSON(timestamp(TimeUnit::SECOND), array2_json)));
+  ValidateBetween(Datum(ArrayFromJSON(timestamp(TimeUnit::SECOND), array1_json)),
+                  Datum(ScalarFromJSON(timestamp(TimeUnit::NANO), scalar_json)),
+                  Datum(ArrayFromJSON(timestamp(TimeUnit::SECOND), array2_json)));
+  ValidateBetween(Datum(ArrayFromJSON(timestamp(TimeUnit::SECOND), array1_json)),
+                  Datum(ArrayFromJSON(timestamp(TimeUnit::NANO), array2_json)),
+                  Datum(ScalarFromJSON(timestamp(TimeUnit::SECOND), scalar_json)));
+  // But comparing naive to zoned is not OK
+  for (auto inclusive :
+       {BetweenOptions::Inclusive::BOTH, BetweenOptions::Inclusive::LEFT,
+        BetweenOptions::Inclusive::RIGHT, BetweenOptions::Inclusive::NEITHER}) {
+    auto options = BetweenOptions(inclusive);
+    EXPECT_RAISES_WITH_MESSAGE_THAT(
+        TypeError,
+        ::testing::HasSubstr(
+            "Cannot compare timestamp with timezone to timestamp without timezone"),
+        Between(ScalarFromJSON(timestamp(TimeUnit::SECOND), scalar_json),
+                ArrayFromJSON(timestamp(TimeUnit::SECOND), array1_json),
+                ArrayFromJSON(timestamp(TimeUnit::SECOND, "Asia/Tokyo"), array2_json),
+                options, nullptr));
+    EXPECT_RAISES_WITH_MESSAGE_THAT(
+        TypeError,
+        ::testing::HasSubstr(
+            "Cannot compare timestamp with timezone to timestamp without timezone"),
+        Between(ArrayFromJSON(timestamp(TimeUnit::SECOND), array1_json),
+                ScalarFromJSON(timestamp(TimeUnit::SECOND, "America/New_York"), scalar_json),
+                ArrayFromJSON(timestamp(TimeUnit::SECOND, "Europe/Berlin"), array2_json),
+                options, nullptr));
+    EXPECT_RAISES_WITH_MESSAGE_THAT(
+        TypeError,
+        ::testing::HasSubstr(
+            "Cannot compare timestamp with timezone to timestamp without timezone"),
+        Between(ArrayFromJSON(timestamp(TimeUnit::SECOND, "Africa/Nairobi"), array1_json),
+                ScalarFromJSON(timestamp(TimeUnit::SECOND), scalar_json),
+                ArrayFromJSON(timestamp(TimeUnit::SECOND), array2_json), options, nullptr));
+  }
+}
+
+TEST(TestTimestampsBetweenKernel, 3Arrays) {
+  const std::string arr_json = R"(["1970-01-01","1980-02-02","1970-02-28"])";
+  const std::string lhs_json = R"(["1970-01-01","1980-02-01","1970-02-28"])";
+  const std::string rhs_json = R"(["1970-01-02","1980-02-02","1970-02-28"])";
+  // Same units should be fine
+  ValidateBetween(Datum(ArrayFromJSON(timestamp(TimeUnit::SECOND), arr_json)),
+                  Datum(ArrayFromJSON(timestamp(TimeUnit::SECOND), lhs_json)),
+                  Datum(ArrayFromJSON(timestamp(TimeUnit::SECOND), rhs_json)));
+  // Different timezones should be fine
+  ValidateBetween(
+      Datum(ArrayFromJSON(timestamp(TimeUnit::SECOND, "Africa/Cairo"), arr_json)),
+      Datum(ArrayFromJSON(timestamp(TimeUnit::SECOND, "America/Chicago"), lhs_json)),
+      Datum(ArrayFromJSON(timestamp(TimeUnit::SECOND, "Asia/Beijing"), rhs_json)));
+  ValidateBetween(
+      Datum(ArrayFromJSON(timestamp(TimeUnit::NANO, "Europe/Berlin"), arr_json)),
+      Datum(ArrayFromJSON(timestamp(TimeUnit::NANO, "America/Phoenix"), lhs_json)),
+      Datum(ArrayFromJSON(timestamp(TimeUnit::NANO, "Africa/Nairobi"), rhs_json)));
+  // Different units should be fine
+  ValidateBetween(Datum(ArrayFromJSON(timestamp(TimeUnit::SECOND), arr_json)),
+                  Datum(ArrayFromJSON(timestamp(TimeUnit::MILLI), lhs_json)),
+                  Datum(ArrayFromJSON(timestamp(TimeUnit::SECOND), rhs_json)));
+  ValidateBetween(Datum(ArrayFromJSON(timestamp(TimeUnit::SECOND), arr_json)),
+                  Datum(ArrayFromJSON(timestamp(TimeUnit::NANO), lhs_json)),
+                  Datum(ArrayFromJSON(timestamp(TimeUnit::SECOND), rhs_json)));
+  // But comparing naive to zoned is not OK
+  for (auto inclusive :
+       {BetweenOptions::Inclusive::BOTH, BetweenOptions::Inclusive::LEFT,
+        BetweenOptions::Inclusive::RIGHT, BetweenOptions::Inclusive::NEITHER}) {
+    auto options = BetweenOptions(inclusive);
+    EXPECT_RAISES_WITH_MESSAGE_THAT(
+        TypeError,
+        ::testing::HasSubstr(
+            "Cannot compare timestamp with timezone to timestamp without timezone"),
+        Between(ArrayFromJSON(timestamp(TimeUnit::SECOND), arr_json),
+                ArrayFromJSON(timestamp(TimeUnit::SECOND), lhs_json),
+                ArrayFromJSON(timestamp(TimeUnit::SECOND, "Asia/Tokyo"), rhs_json),
+                options, nullptr));
+    EXPECT_RAISES_WITH_MESSAGE_THAT(
+        TypeError,
+        ::testing::HasSubstr(
+            "Cannot compare timestamp with timezone to timestamp without timezone"),
+        Between(ArrayFromJSON(timestamp(TimeUnit::SECOND), arr_json),
+                ArrayFromJSON(timestamp(TimeUnit::SECOND, "America/New_York"), lhs_json),
+                ArrayFromJSON(timestamp(TimeUnit::SECOND, "Europe/Berlin"), rhs_json),
+                options, nullptr));
+    EXPECT_RAISES_WITH_MESSAGE_THAT(
+        TypeError,
+        ::testing::HasSubstr(
+            "Cannot compare timestamp with timezone to timestamp without timezone"),
+        Between(ArrayFromJSON(timestamp(TimeUnit::SECOND, "Africa/Nairobi"), arr_json),
+                ArrayFromJSON(timestamp(TimeUnit::SECOND), lhs_json),
+                ArrayFromJSON(timestamp(TimeUnit::SECOND), rhs_json), options, nullptr));
+  }
+}
+
+template <typename ArrowType>
+class TestBetweenDecimal : public ::testing::Test {};
+TYPED_TEST_SUITE(TestBetweenDecimal, DecimalArrowTypes);
+
+TYPED_TEST(TestBetweenDecimal, 3Arrays) {
+  auto ty = std::make_shared<TypeParam>(3, 2);

Review comment:
       So, for the record, the typed tests compiles a separate test for each type parameter. To minimize compile time expansion, we can instead iterate over types at runtime, for example like this:
   ```c++
     for (auto decimal_factory : {decimal128, decimal256}) {
       ty = decimal_factory(3, 2);
       ARROW_SCOPED_TRACE("Type =", ty->ToString());
       // etc.
     }
   




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

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

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



[GitHub] [arrow] pitrou commented on a change in pull request #11882: ARROW-9843: [C++][Python] Implement Between ternary kernel and Python bindings

Posted by GitBox <gi...@apache.org>.
pitrou commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r793681883



##########
File path: cpp/src/arrow/compute/kernels/scalar_compare.cc
##########
@@ -746,6 +808,279 @@ std::shared_ptr<ScalarFunction> MakeScalarMinMax(std::string name,
   return func;
 }
 
+template <template <BetweenOptions::Inclusive> class Op>
+std::shared_ptr<ScalarFunction> MakeBetweenFunction(std::string name,
+                                                    const FunctionDoc* doc) {
+  using BetweenState = OptionsWrapper<BetweenOptions>;
+  static const auto kDefaultOptions = BetweenOptions::Defaults();
+  auto func =
+      std::make_shared<CompareFunction>(name, Arity::Ternary(), doc, &kDefaultOptions);
+
+  // Add kernels for physical numeric types, temporal types done separately
+  for (const auto& types : {DurationTypes(), IntervalTypes(), NumericTypes()}) {
+    for (const auto& ty : types) {
+      auto type_id = ty->id();
+      auto exec = [type_id](KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+        // Resolve generator based on options
+        const auto& state = static_cast<const BetweenState&>(*ctx->state());
+        switch (state.options.inclusive) {
+          case BetweenOptions::Inclusive::BOTH:
+            return GeneratePhysicalNumeric<ScalarTernaryEqualTypes, BooleanType,
+                                           Op<BetweenOptions::Inclusive::BOTH>>(type_id)(
+                ctx, batch, out);
+          case BetweenOptions::Inclusive::LEFT:
+            return GeneratePhysicalNumeric<ScalarTernaryEqualTypes, BooleanType,
+                                           Op<BetweenOptions::Inclusive::LEFT>>(type_id)(
+                ctx, batch, out);
+          case BetweenOptions::Inclusive::RIGHT:
+            return GeneratePhysicalNumeric<ScalarTernaryEqualTypes, BooleanType,
+                                           Op<BetweenOptions::Inclusive::RIGHT>>(type_id)(
+                ctx, batch, out);
+          case BetweenOptions::Inclusive::NEITHER:
+            return GeneratePhysicalNumeric<ScalarTernaryEqualTypes, BooleanType,
+                                           Op<BetweenOptions::Inclusive::NEITHER>>(
+                type_id)(ctx, batch, out);
+          default:
+            return Status::NotImplemented("between inclusiveness not implemented: ",
+                                          state.options.ToString());
+        }
+      };
+      DCHECK_OK(func->AddKernel({ty, ty, ty}, boolean(), exec, BetweenState::Init));
+    }
+  }
+
+  // Add timestamp kernels
+  for (const auto unit : TimeUnit::values()) {
+    InputType in_type(match::TimestampTypeUnit(unit));
+    auto type_id = Type::TIMESTAMP;
+    auto exec = [type_id](KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+      // Validate timezones in all entries or in none
+      RETURN_NOT_OK(CheckCompareTimestamps(batch));
+      // Resolve generator based on options
+      const auto& state = static_cast<const BetweenState&>(*ctx->state());
+      switch (state.options.inclusive) {
+        case BetweenOptions::Inclusive::BOTH:
+          return GeneratePhysicalNumeric<ScalarTernaryEqualTypes, BooleanType,
+                                         Op<BetweenOptions::Inclusive::BOTH>>(type_id)(
+              ctx, batch, out);
+        case BetweenOptions::Inclusive::LEFT:
+          return GeneratePhysicalNumeric<ScalarTernaryEqualTypes, BooleanType,
+                                         Op<BetweenOptions::Inclusive::LEFT>>(type_id)(
+              ctx, batch, out);
+        case BetweenOptions::Inclusive::RIGHT:
+          return GeneratePhysicalNumeric<ScalarTernaryEqualTypes, BooleanType,
+                                         Op<BetweenOptions::Inclusive::RIGHT>>(type_id)(
+              ctx, batch, out);
+        case BetweenOptions::Inclusive::NEITHER:
+          return GeneratePhysicalNumeric<ScalarTernaryEqualTypes, BooleanType,
+                                         Op<BetweenOptions::Inclusive::NEITHER>>(type_id)(
+              ctx, batch, out);
+        default:
+          return Status::NotImplemented("between inclusiveness not implemented: ",
+                                        state.options.ToString());
+      }
+    };
+    DCHECK_OK(func->AddKernel({in_type, in_type, in_type}, boolean(), exec,
+                              BetweenState::Init));
+  }
+
+  // Add time kernels
+  for (const auto unit : {TimeUnit::SECOND, TimeUnit::MILLI}) {
+    auto type_id = Type::TIME32;
+    auto exec = [type_id](KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+      // Resolve generator based on options
+      const auto& state = static_cast<const BetweenState&>(*ctx->state());
+      switch (state.options.inclusive) {
+        case BetweenOptions::Inclusive::BOTH:
+          return GeneratePhysicalNumeric<ScalarTernaryEqualTypes, BooleanType,
+                                         Op<BetweenOptions::Inclusive::BOTH>>(type_id)(
+              ctx, batch, out);
+        case BetweenOptions::Inclusive::LEFT:
+          return GeneratePhysicalNumeric<ScalarTernaryEqualTypes, BooleanType,
+                                         Op<BetweenOptions::Inclusive::LEFT>>(type_id)(
+              ctx, batch, out);
+        case BetweenOptions::Inclusive::RIGHT:
+          return GeneratePhysicalNumeric<ScalarTernaryEqualTypes, BooleanType,
+                                         Op<BetweenOptions::Inclusive::RIGHT>>(type_id)(
+              ctx, batch, out);
+        case BetweenOptions::Inclusive::NEITHER:
+          return GeneratePhysicalNumeric<ScalarTernaryEqualTypes, BooleanType,
+                                         Op<BetweenOptions::Inclusive::NEITHER>>(type_id)(
+              ctx, batch, out);
+        default:
+          return Status::NotImplemented("between inclusiveness not implemented: ",
+                                        state.options.ToString());
+      }
+    };
+    InputType in_type(match::Time32TypeUnit(unit));
+    DCHECK_OK(func->AddKernel({in_type, in_type, in_type}, boolean(), exec,
+                              BetweenState::Init));
+  }
+
+  for (const auto& unit : {TimeUnit::MICRO, TimeUnit::NANO}) {
+    auto type_id = Type::TIME64;
+    auto exec = [type_id](KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+      // Resolve generator based on options
+      const auto& state = static_cast<const BetweenState&>(*ctx->state());
+      switch (state.options.inclusive) {
+        case BetweenOptions::Inclusive::BOTH:
+          return GeneratePhysicalNumeric<ScalarTernaryEqualTypes, BooleanType,
+                                         Op<BetweenOptions::Inclusive::BOTH>>(type_id)(
+              ctx, batch, out);
+        case BetweenOptions::Inclusive::LEFT:
+          return GeneratePhysicalNumeric<ScalarTernaryEqualTypes, BooleanType,
+                                         Op<BetweenOptions::Inclusive::LEFT>>(type_id)(
+              ctx, batch, out);
+        case BetweenOptions::Inclusive::RIGHT:
+          return GeneratePhysicalNumeric<ScalarTernaryEqualTypes, BooleanType,
+                                         Op<BetweenOptions::Inclusive::RIGHT>>(type_id)(
+              ctx, batch, out);
+        case BetweenOptions::Inclusive::NEITHER:
+          return GeneratePhysicalNumeric<ScalarTernaryEqualTypes, BooleanType,
+                                         Op<BetweenOptions::Inclusive::NEITHER>>(type_id)(
+              ctx, batch, out);
+        default:
+          return Status::NotImplemented("between inclusiveness not implemented: ",
+                                        state.options.ToString());
+      }
+    };
+    InputType in_type(match::Time64TypeUnit(unit));
+    DCHECK_OK(func->AddKernel({in_type, in_type, in_type}, boolean(), exec,
+                              BetweenState::Init));
+  }
+
+  // Add kernels for base binary types
+  for (const auto& ty : BaseBinaryTypes()) {
+    auto type_id = ty->id();
+    auto exec = [type_id](KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+      // Resolve generator based on options
+      const auto& state = static_cast<const BetweenState&>(*ctx->state());
+      switch (state.options.inclusive) {
+        case BetweenOptions::Inclusive::BOTH:
+          return GenerateVarBinaryBase<ScalarTernaryEqualTypes, BooleanType,
+                                       Op<BetweenOptions::Inclusive::BOTH>>(type_id)(
+              ctx, batch, out);
+        case BetweenOptions::Inclusive::LEFT:
+          return GenerateVarBinaryBase<ScalarTernaryEqualTypes, BooleanType,
+                                       Op<BetweenOptions::Inclusive::LEFT>>(type_id)(
+              ctx, batch, out);
+        case BetweenOptions::Inclusive::RIGHT:
+          return GenerateVarBinaryBase<ScalarTernaryEqualTypes, BooleanType,
+                                       Op<BetweenOptions::Inclusive::RIGHT>>(type_id)(
+              ctx, batch, out);
+        case BetweenOptions::Inclusive::NEITHER:
+          return GenerateVarBinaryBase<ScalarTernaryEqualTypes, BooleanType,
+                                       Op<BetweenOptions::Inclusive::NEITHER>>(type_id)(
+              ctx, batch, out);
+        default:
+          return Status::NotImplemented("between inclusiveness not implemented: ",
+                                        state.options.ToString());
+      }
+    };
+    DCHECK_OK(func->AddKernel({ty, ty, ty}, boolean(), exec, BetweenState::Init));
+  }
+
+  // Add kernels for decimal types
+  for (const auto type_id : {Type::DECIMAL128, Type::DECIMAL256}) {
+    auto exec = [type_id](KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+      // Resolve generator based on options
+      const auto& state = static_cast<const BetweenState&>(*ctx->state());
+      switch (state.options.inclusive) {
+        case BetweenOptions::Inclusive::BOTH:
+          return GenerateDecimal<ScalarTernaryEqualTypes, BooleanType,
+                                 Op<BetweenOptions::Inclusive::BOTH>>(type_id)(ctx, batch,
+                                                                               out);
+        case BetweenOptions::Inclusive::LEFT:
+          return GenerateDecimal<ScalarTernaryEqualTypes, BooleanType,
+                                 Op<BetweenOptions::Inclusive::LEFT>>(type_id)(ctx, batch,
+                                                                               out);
+        case BetweenOptions::Inclusive::RIGHT:
+          return GenerateDecimal<ScalarTernaryEqualTypes, BooleanType,
+                                 Op<BetweenOptions::Inclusive::RIGHT>>(type_id)(
+              ctx, batch, out);
+        case BetweenOptions::Inclusive::NEITHER:
+          return GenerateDecimal<ScalarTernaryEqualTypes, BooleanType,
+                                 Op<BetweenOptions::Inclusive::NEITHER>>(type_id)(
+              ctx, batch, out);
+        default:
+          return Status::NotImplemented("between inclusiveness not implemented: ",
+                                        state.options.ToString());
+      }
+    };
+    InputType ty(type_id);
+    DCHECK_OK(func->AddKernel({ty, ty, ty}, boolean(), exec, BetweenState::Init));
+  }
+
+  // Add kernels for date types
+  for (const auto type_id : {Type::DATE32, Type::DATE64}) {
+    auto exec = [type_id](KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+      // Resolve generator based on options
+      const auto& state = static_cast<const BetweenState&>(*ctx->state());
+      switch (state.options.inclusive) {
+        case BetweenOptions::Inclusive::BOTH:
+          return GeneratePhysicalNumeric<ScalarTernaryEqualTypes, BooleanType,
+                                         Op<BetweenOptions::Inclusive::BOTH>>(type_id)(
+              ctx, batch, out);
+        case BetweenOptions::Inclusive::LEFT:
+          return GeneratePhysicalNumeric<ScalarTernaryEqualTypes, BooleanType,
+                                         Op<BetweenOptions::Inclusive::LEFT>>(type_id)(
+              ctx, batch, out);
+        case BetweenOptions::Inclusive::RIGHT:
+          return GeneratePhysicalNumeric<ScalarTernaryEqualTypes, BooleanType,
+                                         Op<BetweenOptions::Inclusive::RIGHT>>(type_id)(
+              ctx, batch, out);
+        case BetweenOptions::Inclusive::NEITHER:
+          return GeneratePhysicalNumeric<ScalarTernaryEqualTypes, BooleanType,
+                                         Op<BetweenOptions::Inclusive::NEITHER>>(type_id)(
+              ctx, batch, out);
+        default:
+          return Status::NotImplemented("between inclusiveness not implemented: ",
+                                        state.options.ToString());
+      }
+    };

Review comment:
       Ok for a follow up PR.




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

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

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



[GitHub] [arrow] bkmgit commented on pull request #11882: ARROW-9843: [C++][Python] Implement Between ternary kernel and Python bindings

Posted by GitBox <gi...@apache.org>.
bkmgit commented on pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#issuecomment-1072517178


   Thanks. Rebased.


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

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

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



[GitHub] [arrow] lidavidm commented on pull request #11882: ARROW-9843: [C++][Python] Implement Between ternary kernel and Python bindings

Posted by GitBox <gi...@apache.org>.
lidavidm commented on pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#issuecomment-1072372569


   It also needs to be rebased now.


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

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

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



[GitHub] [arrow] edponce commented on pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
edponce commented on pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#issuecomment-1008355611


   @bkmgit For Python, you would only need to add bindings for `BetweenOptions` because the compute function is "binded" automatically (`call_function` mechanism). I suggest that you [`grep -R RoundOptions arrow/python/pyarrow/`](https://github.com/apache/arrow/tree/master/python/pyarrow) so you can see what are the things needed in Python to provide full support for `between` function. The following are main actions to complete for Python bindings:
   * [Cython header binding for options `pyarrow/includes/libarrow.pxd`](https://github.com/apache/arrow/blob/master/python/pyarrow/includes/libarrow.pxd#L1866-L1893) - Note that the enum is in global scope because it is used by `Round` and `RoundToMultiple`. In this case, `BetweenOptions` will be used by `Between` and `NotBetween`, so maybe it is wise in C++ to put `BetweenOptions` in global space now instead of in `NotBetween` PR.
   * [Cython/Python definitions for options `pyarrow/_compute.pyx`](https://github.com/apache/arrow/blob/master/python/pyarrow/_compute.pyx#L743-L776)
   * [Python test for `BetweenOptions`](https://github.com/apache/arrow/blob/master/python/pyarrow/tests/test_compute.py#L153)
   * [Python tests for compute function](https://github.com/apache/arrow/blob/master/python/pyarrow/tests/test_compute.py#L1409-L1472)
   * [Add function to Python docs](https://github.com/apache/arrow/blob/master/docs/source/python/api/compute.rst#comparisons)


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

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

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



[GitHub] [arrow] bkmgit commented on a change in pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
bkmgit commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r782761321



##########
File path: cpp/src/arrow/compute/kernels/scalar_compare_benchmark.cc
##########
@@ -77,5 +77,44 @@ BENCHMARK(GreaterArrayScalarInt64)->Apply(RegressionSetArgs);
 BENCHMARK(GreaterArrayArrayString)->Apply(RegressionSetArgs);
 BENCHMARK(GreaterArrayScalarString)->Apply(RegressionSetArgs);
 
+template <typename Type>
+static void BetweenScalarArrayScalar(benchmark::State& state) {
+  RegressionArgs args(state, /*size_is_bytes=*/false);
+  auto ty = TypeTraits<Type>::type_singleton();
+  auto rand = random::RandomArrayGenerator(kSeed);
+  auto array = rand.ArrayOf(ty, args.size, args.null_proportion);
+  auto scalar_left = *rand.ArrayOf(ty, 1, 0)->GetScalar(0);
+  auto scalar_right = *rand.ArrayOf(ty, 1, 0)->GetScalar(0);
+  for (auto _ : state) {
+    ABORT_NOT_OK(
+        CallFunction("between_less_equal_less_equal", {array, scalar_left, scalar_right})
+            .status());

Review comment:
       Fixed.




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

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

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



[GitHub] [arrow] lidavidm commented on a change in pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
lidavidm commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r783092378



##########
File path: cpp/src/arrow/compute/api_scalar.h
##########
@@ -316,6 +316,21 @@ struct ARROW_EXPORT CompareOptions {
   enum CompareOperator op;
 };
 
+enum class BetweenMode : int8_t {
+  LESS_EQUAL_LESS_EQUAL,
+  LESS_EQUAL_LESS,
+  LESS_LESS_EQUAL,
+  LESS_LESS,
+};

Review comment:
       not_between can also use BetweenOptions, it doesn't need its own options class unless I'm mistaken.




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

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

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



[GitHub] [arrow] bkmgit commented on a change in pull request #11882: ARROW-9843: [C++][Python] Implement Between ternary kernel and Python bindings

Posted by GitBox <gi...@apache.org>.
bkmgit commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r789029505



##########
File path: cpp/src/arrow/compute/kernels/scalar_compare_test.cc
##########
@@ -1865,5 +1867,683 @@ TEST(TestMaxElementWiseMinElementWise, CommonTemporal) {
               ResultWith(ScalarFromJSON(date64(), "86400000")));
 }
 
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const Datum& val, const Datum& lhs,
+                            const Datum& rhs, const Datum& expected) {
+  ASSERT_OK_AND_ASSIGN(Datum result, Between(val, lhs, rhs, options, nullptr));
+  AssertArraysEqual(*expected.make_array(), *result.make_array(),
+                    /*verbose=*/true);
+}
+
+template <typename ArrowType>
+void ValidateBetween(BetweenOptions options, const Datum& val, const Datum& lhs,
+                     const Datum& rhs) {
+  CompareOperator lhs_val;
+  CompareOperator val_rhs;
+  BetweenOptions::Inclusive include_endpoints = options.inclusive;
+
+  if (include_endpoints == BetweenOptions::Inclusive::NEITHER) {
+    lhs_val = LESS;
+    val_rhs = LESS;
+  } else if (include_endpoints == BetweenOptions::Inclusive::LEFT) {
+    lhs_val = LESS_EQUAL;
+    val_rhs = LESS;
+  } else if (include_endpoints == BetweenOptions::Inclusive::RIGHT) {
+    lhs_val = LESS;
+    val_rhs = LESS_EQUAL;
+  } else {
+    lhs_val = LESS_EQUAL;
+    val_rhs = LESS_EQUAL;
+  }
+
+  ASSERT_OK_AND_ASSIGN(Datum resultl,
+                       CallFunction(CompareOperatorToFunctionName(lhs_val), {lhs, val}));
+  ASSERT_OK_AND_ASSIGN(Datum resultr,
+                       CallFunction(CompareOperatorToFunctionName(val_rhs), {val, rhs}));
+  ASSERT_OK_AND_ASSIGN(Datum expected, CallFunction("and", {resultl, resultr}));
+
+  ValidateBetween<ArrowType>(options, val, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+class TestNumericBetweenKernel : public ::testing::Test {};
+
+TYPED_TEST_SUITE(TestNumericBetweenKernel, NumericArrowTypes);
+TYPED_TEST(TestNumericBetweenKernel, SimpleBetweenArrayScalarScalar) {
+  using ScalarType = typename TypeTraits<TypeParam>::ScalarType;
+  using CType = typename TypeTraits<TypeParam>::CType;
+
+  Datum zero(std::make_shared<ScalarType>(CType(0)));
+  Datum four(std::make_shared<ScalarType>(CType(4)));
+  Datum null(std::make_shared<ScalarType>());
+  BetweenOptions InclusiveBoth(BetweenOptions::Inclusive::BOTH);
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"), zero,
+      four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      zero, four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,1,2,2]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[1,1,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,3,4,5]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[1,1,1,1,1,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,1,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null,0,1,1]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), null, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, null,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  BetweenOptions InclusiveLeft(BetweenOptions::Inclusive::LEFT);
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"), zero,
+      four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      zero, four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,1,2,2]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[1,1,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,3,4,5]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[1,1,1,1,0,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,0,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null,0,1,1]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), null, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, null,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  BetweenOptions InclusiveRight(BetweenOptions::Inclusive::RIGHT);
+  ValidateBetween<TypeParam>(
+      InclusiveRight, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"), zero,
+      four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      zero, four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,1,2,2]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,0,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,3,4,5]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,1,1,1,1,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,1,1,1,1,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null,0,1,1]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null,0,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), null, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, null,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  BetweenOptions InclusiveNeither(BetweenOptions::Inclusive::NEITHER);
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      zero, four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      zero, four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,1,2,2]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,0,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,3,4,5]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,1,1,1,0,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,0,1,1,1,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null,0,1,1]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null,0,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), null, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, null,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+}
+
+TYPED_TEST(TestNumericBetweenKernel, SimpleBetweenScalarArrayArray) {
+  using ScalarType = typename TypeTraits<TypeParam>::ScalarType;
+  using CType = typename TypeTraits<TypeParam>::CType;
+
+  Datum one(std::make_shared<ScalarType>(CType(1)));
+
+  BetweenOptions InclusiveBoth(BetweenOptions::Inclusive::BOTH);
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, one, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,true,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,true,null,false,false]"));
+  BetweenOptions InclusiveLeft(BetweenOptions::Inclusive::LEFT);
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, one, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,true,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,true,null,false,false]"));
+  BetweenOptions InclusiveRight(BetweenOptions::Inclusive::RIGHT);
+  ValidateBetween<TypeParam>(
+      InclusiveRight, one, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,false,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,false,null,false,false]"));
+  BetweenOptions InclusiveNeither(BetweenOptions::Inclusive::NEITHER);
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, one, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,false,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,false,null,false,false]"));
+}
+
+TYPED_TEST(TestNumericBetweenKernel, SimpleBetweenArrayArrayArray) {
+  BetweenOptions InclusiveBoth(BetweenOptions::Inclusive::BOTH);
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[1,1,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,true,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,true,null,false,false]"));
+  BetweenOptions InclusiveLeft(BetweenOptions::Inclusive::LEFT);
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[1,1,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,false,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,false,null,false,false]"));
+  BetweenOptions InclusiveRight(BetweenOptions::Inclusive::RIGHT);
+  ValidateBetween<TypeParam>(
+      InclusiveRight, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[1,1,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,true,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,true,null,false,false]"));
+  BetweenOptions InclusiveNeither(BetweenOptions::Inclusive::NEITHER);
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[1,1,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,false,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,false,null,false,false]"));
+}

Review comment:
       codegen_internal.h does have ScalarScalar and ScalarScalarScalar structs.




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

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

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



[GitHub] [arrow] bkmgit commented on pull request #11882: ARROW-9843: [C++][Python] Implement Between ternary kernel and Python bindings

Posted by GitBox <gi...@apache.org>.
bkmgit commented on pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#issuecomment-1018189007


   @github-actions autotune


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

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

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



[GitHub] [arrow] bkmgit commented on a change in pull request #11882: ARROW-9843: [C++][Python] Implement Between ternary kernel and Python bindings

Posted by GitBox <gi...@apache.org>.
bkmgit commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r789124521



##########
File path: cpp/src/arrow/compute/kernels/scalar_compare_test.cc
##########
@@ -1865,5 +1867,683 @@ TEST(TestMaxElementWiseMinElementWise, CommonTemporal) {
               ResultWith(ScalarFromJSON(date64(), "86400000")));
 }
 
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const Datum& val, const Datum& lhs,
+                            const Datum& rhs, const Datum& expected) {
+  ASSERT_OK_AND_ASSIGN(Datum result, Between(val, lhs, rhs, options, nullptr));
+  AssertArraysEqual(*expected.make_array(), *result.make_array(),
+                    /*verbose=*/true);
+}
+
+template <typename ArrowType>
+void ValidateBetween(BetweenOptions options, const Datum& val, const Datum& lhs,
+                     const Datum& rhs) {
+  CompareOperator lhs_val;
+  CompareOperator val_rhs;
+  BetweenOptions::Inclusive include_endpoints = options.inclusive;
+
+  if (include_endpoints == BetweenOptions::Inclusive::NEITHER) {
+    lhs_val = LESS;
+    val_rhs = LESS;
+  } else if (include_endpoints == BetweenOptions::Inclusive::LEFT) {
+    lhs_val = LESS_EQUAL;
+    val_rhs = LESS;
+  } else if (include_endpoints == BetweenOptions::Inclusive::RIGHT) {
+    lhs_val = LESS;
+    val_rhs = LESS_EQUAL;
+  } else {
+    lhs_val = LESS_EQUAL;
+    val_rhs = LESS_EQUAL;
+  }
+
+  ASSERT_OK_AND_ASSIGN(Datum resultl,
+                       CallFunction(CompareOperatorToFunctionName(lhs_val), {lhs, val}));
+  ASSERT_OK_AND_ASSIGN(Datum resultr,
+                       CallFunction(CompareOperatorToFunctionName(val_rhs), {val, rhs}));
+  ASSERT_OK_AND_ASSIGN(Datum expected, CallFunction("and", {resultl, resultr}));
+
+  ValidateBetween<ArrowType>(options, val, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+class TestNumericBetweenKernel : public ::testing::Test {};
+
+TYPED_TEST_SUITE(TestNumericBetweenKernel, NumericArrowTypes);
+TYPED_TEST(TestNumericBetweenKernel, SimpleBetweenArrayScalarScalar) {
+  using ScalarType = typename TypeTraits<TypeParam>::ScalarType;
+  using CType = typename TypeTraits<TypeParam>::CType;
+
+  Datum zero(std::make_shared<ScalarType>(CType(0)));
+  Datum four(std::make_shared<ScalarType>(CType(4)));
+  Datum null(std::make_shared<ScalarType>());
+  BetweenOptions InclusiveBoth(BetweenOptions::Inclusive::BOTH);
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"), zero,
+      four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      zero, four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,1,2,2]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[1,1,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,3,4,5]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[1,1,1,1,1,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,1,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null,0,1,1]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), null, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, null,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  BetweenOptions InclusiveLeft(BetweenOptions::Inclusive::LEFT);
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"), zero,
+      four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      zero, four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,1,2,2]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[1,1,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,3,4,5]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[1,1,1,1,0,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,0,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null,0,1,1]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), null, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, null,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  BetweenOptions InclusiveRight(BetweenOptions::Inclusive::RIGHT);
+  ValidateBetween<TypeParam>(
+      InclusiveRight, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"), zero,
+      four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      zero, four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,1,2,2]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,0,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,3,4,5]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,1,1,1,1,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,1,1,1,1,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null,0,1,1]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null,0,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), null, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, null,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  BetweenOptions InclusiveNeither(BetweenOptions::Inclusive::NEITHER);
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      zero, four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      zero, four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,1,2,2]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,0,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,3,4,5]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,1,1,1,0,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,0,1,1,1,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null,0,1,1]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null,0,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), null, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, null,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+}
+
+TYPED_TEST(TestNumericBetweenKernel, SimpleBetweenScalarArrayArray) {
+  using ScalarType = typename TypeTraits<TypeParam>::ScalarType;
+  using CType = typename TypeTraits<TypeParam>::CType;
+
+  Datum one(std::make_shared<ScalarType>(CType(1)));
+
+  BetweenOptions InclusiveBoth(BetweenOptions::Inclusive::BOTH);
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, one, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,true,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,true,null,false,false]"));
+  BetweenOptions InclusiveLeft(BetweenOptions::Inclusive::LEFT);
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, one, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,true,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,true,null,false,false]"));
+  BetweenOptions InclusiveRight(BetweenOptions::Inclusive::RIGHT);
+  ValidateBetween<TypeParam>(
+      InclusiveRight, one, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,false,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,false,null,false,false]"));
+  BetweenOptions InclusiveNeither(BetweenOptions::Inclusive::NEITHER);
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, one, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,false,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,false,null,false,false]"));
+}
+
+TYPED_TEST(TestNumericBetweenKernel, SimpleBetweenArrayArrayArray) {
+  BetweenOptions InclusiveBoth(BetweenOptions::Inclusive::BOTH);
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[1,1,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,true,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,true,null,false,false]"));
+  BetweenOptions InclusiveLeft(BetweenOptions::Inclusive::LEFT);
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[1,1,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,false,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,false,null,false,false]"));
+  BetweenOptions InclusiveRight(BetweenOptions::Inclusive::RIGHT);
+  ValidateBetween<TypeParam>(
+      InclusiveRight, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[1,1,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,true,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,true,null,false,false]"));
+  BetweenOptions InclusiveNeither(BetweenOptions::Inclusive::NEITHER);
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[1,1,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,false,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,false,null,false,false]"));
+}

Review comment:
       Fixed. Works now. Need to output scalars.




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

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

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



[GitHub] [arrow] bkmgit commented on a change in pull request #11882: ARROW-9843: [C++][Python] Implement Between ternary kernel and Python bindings

Posted by GitBox <gi...@apache.org>.
bkmgit commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r789005846



##########
File path: cpp/src/arrow/compute/kernels/scalar_compare_test.cc
##########
@@ -1865,5 +1867,683 @@ TEST(TestMaxElementWiseMinElementWise, CommonTemporal) {
               ResultWith(ScalarFromJSON(date64(), "86400000")));
 }
 
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const Datum& val, const Datum& lhs,
+                            const Datum& rhs, const Datum& expected) {
+  ASSERT_OK_AND_ASSIGN(Datum result, Between(val, lhs, rhs, options, nullptr));
+  AssertArraysEqual(*expected.make_array(), *result.make_array(),
+                    /*verbose=*/true);
+}
+
+template <typename ArrowType>
+void ValidateBetween(BetweenOptions options, const Datum& val, const Datum& lhs,
+                     const Datum& rhs) {
+  CompareOperator lhs_val;
+  CompareOperator val_rhs;
+  BetweenOptions::Inclusive include_endpoints = options.inclusive;
+
+  if (include_endpoints == BetweenOptions::Inclusive::NEITHER) {
+    lhs_val = LESS;
+    val_rhs = LESS;
+  } else if (include_endpoints == BetweenOptions::Inclusive::LEFT) {
+    lhs_val = LESS_EQUAL;
+    val_rhs = LESS;
+  } else if (include_endpoints == BetweenOptions::Inclusive::RIGHT) {
+    lhs_val = LESS;
+    val_rhs = LESS_EQUAL;
+  } else {
+    lhs_val = LESS_EQUAL;
+    val_rhs = LESS_EQUAL;
+  }
+
+  ASSERT_OK_AND_ASSIGN(Datum resultl,
+                       CallFunction(CompareOperatorToFunctionName(lhs_val), {lhs, val}));
+  ASSERT_OK_AND_ASSIGN(Datum resultr,
+                       CallFunction(CompareOperatorToFunctionName(val_rhs), {val, rhs}));
+  ASSERT_OK_AND_ASSIGN(Datum expected, CallFunction("and", {resultl, resultr}));
+
+  ValidateBetween<ArrowType>(options, val, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+class TestNumericBetweenKernel : public ::testing::Test {};
+
+TYPED_TEST_SUITE(TestNumericBetweenKernel, NumericArrowTypes);
+TYPED_TEST(TestNumericBetweenKernel, SimpleBetweenArrayScalarScalar) {

Review comment:
       Removed type parametrization. Needed to introduce a function get the type. This function could possibly be put elsewhere. Maybe there is a better way to get the type? 




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

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

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



[GitHub] [arrow] edponce commented on a change in pull request #11882: ARROW-9843: [C++][Python] Implement Between ternary kernel and Python bindings

Posted by GitBox <gi...@apache.org>.
edponce commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r793127766



##########
File path: cpp/src/arrow/compute/kernels/scalar_compare.cc
##########
@@ -746,6 +808,279 @@ std::shared_ptr<ScalarFunction> MakeScalarMinMax(std::string name,
   return func;
 }
 
+template <template <BetweenOptions::Inclusive> class Op>
+std::shared_ptr<ScalarFunction> MakeBetweenFunction(std::string name,
+                                                    const FunctionDoc* doc) {
+  using BetweenState = OptionsWrapper<BetweenOptions>;
+  static const auto kDefaultOptions = BetweenOptions::Defaults();
+  auto func =
+      std::make_shared<CompareFunction>(name, Arity::Ternary(), doc, &kDefaultOptions);
+
+  // Add kernels for physical numeric types, temporal types done separately
+  for (const auto& types : {DurationTypes(), IntervalTypes(), NumericTypes()}) {
+    for (const auto& ty : types) {
+      auto type_id = ty->id();
+      auto exec = [type_id](KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+        // Resolve generator based on options
+        const auto& state = static_cast<const BetweenState&>(*ctx->state());
+        switch (state.options.inclusive) {
+          case BetweenOptions::Inclusive::BOTH:
+            return GeneratePhysicalNumeric<ScalarTernaryEqualTypes, BooleanType,
+                                           Op<BetweenOptions::Inclusive::BOTH>>(type_id)(
+                ctx, batch, out);
+          case BetweenOptions::Inclusive::LEFT:
+            return GeneratePhysicalNumeric<ScalarTernaryEqualTypes, BooleanType,
+                                           Op<BetweenOptions::Inclusive::LEFT>>(type_id)(
+                ctx, batch, out);
+          case BetweenOptions::Inclusive::RIGHT:
+            return GeneratePhysicalNumeric<ScalarTernaryEqualTypes, BooleanType,
+                                           Op<BetweenOptions::Inclusive::RIGHT>>(type_id)(
+                ctx, batch, out);
+          case BetweenOptions::Inclusive::NEITHER:
+            return GeneratePhysicalNumeric<ScalarTernaryEqualTypes, BooleanType,
+                                           Op<BetweenOptions::Inclusive::NEITHER>>(
+                type_id)(ctx, batch, out);
+          default:
+            return Status::NotImplemented("between inclusiveness not implemented: ",
+                                          state.options.ToString());
+        }
+      };
+      DCHECK_OK(func->AddKernel({ty, ty, ty}, boolean(), exec, BetweenState::Init));
+    }
+  }
+
+  // Add timestamp kernels
+  for (const auto unit : TimeUnit::values()) {
+    InputType in_type(match::TimestampTypeUnit(unit));
+    auto type_id = Type::TIMESTAMP;
+    auto exec = [type_id](KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+      // Validate timezones in all entries or in none
+      RETURN_NOT_OK(CheckCompareTimestamps(batch));
+      // Resolve generator based on options
+      const auto& state = static_cast<const BetweenState&>(*ctx->state());
+      switch (state.options.inclusive) {
+        case BetweenOptions::Inclusive::BOTH:
+          return GeneratePhysicalNumeric<ScalarTernaryEqualTypes, BooleanType,
+                                         Op<BetweenOptions::Inclusive::BOTH>>(type_id)(
+              ctx, batch, out);
+        case BetweenOptions::Inclusive::LEFT:
+          return GeneratePhysicalNumeric<ScalarTernaryEqualTypes, BooleanType,
+                                         Op<BetweenOptions::Inclusive::LEFT>>(type_id)(
+              ctx, batch, out);
+        case BetweenOptions::Inclusive::RIGHT:
+          return GeneratePhysicalNumeric<ScalarTernaryEqualTypes, BooleanType,
+                                         Op<BetweenOptions::Inclusive::RIGHT>>(type_id)(
+              ctx, batch, out);
+        case BetweenOptions::Inclusive::NEITHER:
+          return GeneratePhysicalNumeric<ScalarTernaryEqualTypes, BooleanType,
+                                         Op<BetweenOptions::Inclusive::NEITHER>>(type_id)(
+              ctx, batch, out);
+        default:
+          return Status::NotImplemented("between inclusiveness not implemented: ",
+                                        state.options.ToString());
+      }
+    };
+    DCHECK_OK(func->AddKernel({in_type, in_type, in_type}, boolean(), exec,
+                              BetweenState::Init));
+  }
+
+  // Add time kernels
+  for (const auto unit : {TimeUnit::SECOND, TimeUnit::MILLI}) {
+    auto type_id = Type::TIME32;
+    auto exec = [type_id](KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+      // Resolve generator based on options
+      const auto& state = static_cast<const BetweenState&>(*ctx->state());
+      switch (state.options.inclusive) {
+        case BetweenOptions::Inclusive::BOTH:
+          return GeneratePhysicalNumeric<ScalarTernaryEqualTypes, BooleanType,
+                                         Op<BetweenOptions::Inclusive::BOTH>>(type_id)(
+              ctx, batch, out);
+        case BetweenOptions::Inclusive::LEFT:
+          return GeneratePhysicalNumeric<ScalarTernaryEqualTypes, BooleanType,
+                                         Op<BetweenOptions::Inclusive::LEFT>>(type_id)(
+              ctx, batch, out);
+        case BetweenOptions::Inclusive::RIGHT:
+          return GeneratePhysicalNumeric<ScalarTernaryEqualTypes, BooleanType,
+                                         Op<BetweenOptions::Inclusive::RIGHT>>(type_id)(
+              ctx, batch, out);
+        case BetweenOptions::Inclusive::NEITHER:
+          return GeneratePhysicalNumeric<ScalarTernaryEqualTypes, BooleanType,
+                                         Op<BetweenOptions::Inclusive::NEITHER>>(type_id)(
+              ctx, batch, out);
+        default:
+          return Status::NotImplemented("between inclusiveness not implemented: ",
+                                        state.options.ToString());
+      }
+    };
+    InputType in_type(match::Time32TypeUnit(unit));
+    DCHECK_OK(func->AddKernel({in_type, in_type, in_type}, boolean(), exec,
+                              BetweenState::Init));
+  }
+
+  for (const auto& unit : {TimeUnit::MICRO, TimeUnit::NANO}) {
+    auto type_id = Type::TIME64;
+    auto exec = [type_id](KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+      // Resolve generator based on options
+      const auto& state = static_cast<const BetweenState&>(*ctx->state());
+      switch (state.options.inclusive) {
+        case BetweenOptions::Inclusive::BOTH:
+          return GeneratePhysicalNumeric<ScalarTernaryEqualTypes, BooleanType,
+                                         Op<BetweenOptions::Inclusive::BOTH>>(type_id)(
+              ctx, batch, out);
+        case BetweenOptions::Inclusive::LEFT:
+          return GeneratePhysicalNumeric<ScalarTernaryEqualTypes, BooleanType,
+                                         Op<BetweenOptions::Inclusive::LEFT>>(type_id)(
+              ctx, batch, out);
+        case BetweenOptions::Inclusive::RIGHT:
+          return GeneratePhysicalNumeric<ScalarTernaryEqualTypes, BooleanType,
+                                         Op<BetweenOptions::Inclusive::RIGHT>>(type_id)(
+              ctx, batch, out);
+        case BetweenOptions::Inclusive::NEITHER:
+          return GeneratePhysicalNumeric<ScalarTernaryEqualTypes, BooleanType,
+                                         Op<BetweenOptions::Inclusive::NEITHER>>(type_id)(
+              ctx, batch, out);
+        default:
+          return Status::NotImplemented("between inclusiveness not implemented: ",
+                                        state.options.ToString());
+      }
+    };
+    InputType in_type(match::Time64TypeUnit(unit));
+    DCHECK_OK(func->AddKernel({in_type, in_type, in_type}, boolean(), exec,
+                              BetweenState::Init));
+  }
+
+  // Add kernels for base binary types
+  for (const auto& ty : BaseBinaryTypes()) {
+    auto type_id = ty->id();
+    auto exec = [type_id](KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+      // Resolve generator based on options
+      const auto& state = static_cast<const BetweenState&>(*ctx->state());
+      switch (state.options.inclusive) {
+        case BetweenOptions::Inclusive::BOTH:
+          return GenerateVarBinaryBase<ScalarTernaryEqualTypes, BooleanType,
+                                       Op<BetweenOptions::Inclusive::BOTH>>(type_id)(
+              ctx, batch, out);
+        case BetweenOptions::Inclusive::LEFT:
+          return GenerateVarBinaryBase<ScalarTernaryEqualTypes, BooleanType,
+                                       Op<BetweenOptions::Inclusive::LEFT>>(type_id)(
+              ctx, batch, out);
+        case BetweenOptions::Inclusive::RIGHT:
+          return GenerateVarBinaryBase<ScalarTernaryEqualTypes, BooleanType,
+                                       Op<BetweenOptions::Inclusive::RIGHT>>(type_id)(
+              ctx, batch, out);
+        case BetweenOptions::Inclusive::NEITHER:
+          return GenerateVarBinaryBase<ScalarTernaryEqualTypes, BooleanType,
+                                       Op<BetweenOptions::Inclusive::NEITHER>>(type_id)(
+              ctx, batch, out);
+        default:
+          return Status::NotImplemented("between inclusiveness not implemented: ",
+                                        state.options.ToString());
+      }
+    };
+    DCHECK_OK(func->AddKernel({ty, ty, ty}, boolean(), exec, BetweenState::Init));
+  }
+
+  // Add kernels for decimal types
+  for (const auto type_id : {Type::DECIMAL128, Type::DECIMAL256}) {
+    auto exec = [type_id](KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+      // Resolve generator based on options
+      const auto& state = static_cast<const BetweenState&>(*ctx->state());
+      switch (state.options.inclusive) {
+        case BetweenOptions::Inclusive::BOTH:
+          return GenerateDecimal<ScalarTernaryEqualTypes, BooleanType,
+                                 Op<BetweenOptions::Inclusive::BOTH>>(type_id)(ctx, batch,
+                                                                               out);
+        case BetweenOptions::Inclusive::LEFT:
+          return GenerateDecimal<ScalarTernaryEqualTypes, BooleanType,
+                                 Op<BetweenOptions::Inclusive::LEFT>>(type_id)(ctx, batch,
+                                                                               out);
+        case BetweenOptions::Inclusive::RIGHT:
+          return GenerateDecimal<ScalarTernaryEqualTypes, BooleanType,
+                                 Op<BetweenOptions::Inclusive::RIGHT>>(type_id)(
+              ctx, batch, out);
+        case BetweenOptions::Inclusive::NEITHER:
+          return GenerateDecimal<ScalarTernaryEqualTypes, BooleanType,
+                                 Op<BetweenOptions::Inclusive::NEITHER>>(type_id)(
+              ctx, batch, out);
+        default:
+          return Status::NotImplemented("between inclusiveness not implemented: ",
+                                        state.options.ToString());
+      }
+    };
+    InputType ty(type_id);
+    DCHECK_OK(func->AddKernel({ty, ty, ty}, boolean(), exec, BetweenState::Init));
+  }
+
+  // Add kernels for date types
+  for (const auto type_id : {Type::DATE32, Type::DATE64}) {
+    auto exec = [type_id](KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+      // Resolve generator based on options
+      const auto& state = static_cast<const BetweenState&>(*ctx->state());
+      switch (state.options.inclusive) {
+        case BetweenOptions::Inclusive::BOTH:
+          return GeneratePhysicalNumeric<ScalarTernaryEqualTypes, BooleanType,
+                                         Op<BetweenOptions::Inclusive::BOTH>>(type_id)(
+              ctx, batch, out);
+        case BetweenOptions::Inclusive::LEFT:
+          return GeneratePhysicalNumeric<ScalarTernaryEqualTypes, BooleanType,
+                                         Op<BetweenOptions::Inclusive::LEFT>>(type_id)(
+              ctx, batch, out);
+        case BetweenOptions::Inclusive::RIGHT:
+          return GeneratePhysicalNumeric<ScalarTernaryEqualTypes, BooleanType,
+                                         Op<BetweenOptions::Inclusive::RIGHT>>(type_id)(
+              ctx, batch, out);
+        case BetweenOptions::Inclusive::NEITHER:
+          return GeneratePhysicalNumeric<ScalarTernaryEqualTypes, BooleanType,
+                                         Op<BetweenOptions::Inclusive::NEITHER>>(type_id)(
+              ctx, batch, out);
+        default:
+          return Status::NotImplemented("between inclusiveness not implemented: ",
+                                        state.options.ToString());
+      }
+    };

Review comment:
       To be able to simplify/generalize as @pitrou proposed, we would need to make the following changes:
   * Add [`GenerateFixedSizeBinary()`](https://github.com/apache/arrow/blob/master/cpp/src/arrow/compute/kernels/codegen_internal.h#L1307)
   * Convert `GenerateXXX` functions into functors (so they can be used as template parameters, too)
   * Add `GenerateFunc` template parameter to `MakeBetweenArrayExec`.
   
   I agree that this could be done in a follow-up PR (maybe as part of [ARROW-14134](https://issues.apache.org/jira/browse/ARROW-14134)).




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

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

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



[GitHub] [arrow] bkmgit commented on a change in pull request #11882: ARROW-9843: [C++][Python] Implement Between ternary kernel and Python bindings

Posted by GitBox <gi...@apache.org>.
bkmgit commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r793732155



##########
File path: cpp/src/arrow/compute/kernels/scalar_compare.cc
##########
@@ -746,6 +805,199 @@ std::shared_ptr<ScalarFunction> MakeScalarMinMax(std::string name,
   return func;
 }
 
+template <template <BetweenOptions::Inclusive> class Op>
+Status MakeBetweenArrayExec(Type::type type_id, KernelContext* ctx,
+                            const ExecBatch& batch, Datum* out) {
+  using BetweenState = OptionsWrapper<BetweenOptions>;
+  const auto& state = static_cast<const BetweenState&>(*ctx->state());
+  if (type_id == Type::FIXED_SIZE_BINARY) {
+    switch (state.options.inclusive) {
+      case BetweenOptions::Inclusive::BOTH:
+        return ScalarTernaryEqualTypes<BooleanType, FixedSizeBinaryType,
+                                       Op<BetweenOptions::Inclusive::BOTH>>::Exec(ctx,
+                                                                                  batch,
+                                                                                  out);
+      case BetweenOptions::Inclusive::LEFT:
+        return ScalarTernaryEqualTypes<BooleanType, FixedSizeBinaryType,
+                                       Op<BetweenOptions::Inclusive::LEFT>>::Exec(ctx,
+                                                                                  batch,
+                                                                                  out);
+      case BetweenOptions::Inclusive::RIGHT:
+        return ScalarTernaryEqualTypes<BooleanType, FixedSizeBinaryType,
+                                       Op<BetweenOptions::Inclusive::RIGHT>>::Exec(ctx,
+                                                                                   batch,
+                                                                                   out);
+      case BetweenOptions::Inclusive::NEITHER:
+        return ScalarTernaryEqualTypes<
+            BooleanType, FixedSizeBinaryType,
+            Op<BetweenOptions::Inclusive::NEITHER>>::Exec(ctx, batch, out);
+      default:
+        return Status::NotImplemented("between inclusiveness not implemented: ",
+                                      state.options.ToString());
+    }
+  } else if (type_id == Type::DECIMAL128 || type_id == Type::DECIMAL256) {

Review comment:
       Will test, nice helper function.




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

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

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



[GitHub] [arrow] bkmgit commented on pull request #11882: ARROW-9843: [C++][Python] Implement Between ternary kernel and Python bindings

Posted by GitBox <gi...@apache.org>.
bkmgit commented on pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#issuecomment-1029311842


   @edponce @pitrou @kou Let me know if any further comments, there are a number of follow up issues, but this should allow people to try out the interfaces.


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

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

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



[GitHub] [arrow] bkmgit commented on a change in pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
bkmgit commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r769435933



##########
File path: cpp/src/arrow/compute/kernels/scalar_compare_test.cc
##########
@@ -1850,5 +1851,154 @@ TEST(TestMaxElementWiseMinElementWise, CommonTemporal) {
               ResultWith(ScalarFromJSON(date64(), "86400000")));
 }
 
+template <typename ArrowType>
+static void ValidateBetween(const Datum& val, const Datum& lhs, const Datum& rhs,
+                            const Datum& expected) {
+  ASSERT_OK_AND_ASSIGN(Datum result, Between(val, lhs, rhs));
+  AssertArraysEqual(*expected.make_array(), *result.make_array(),
+                    /*verbose=*/true);
+}
+
+template <typename ArrowType>
+static void ValidateBetween(const char* value_str, const Datum& lhs, const Datum& rhs,
+                            const char* expected_str) {
+  auto value = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), value_str);
+  auto expected = ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), expected_str);
+  ValidateBetween<ArrowType>(value, lhs, rhs, expected);
+}
+
+template <>
+void ValidateBetween<StringType>(const char* value_str, const Datum& lhs,
+                                 const Datum& rhs, const char* expected_str) {
+  auto value = ArrayFromJSON(utf8(), value_str);
+  auto expected = ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), expected_str);
+  ValidateBetween<StringType>(value, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+class TestNumericBetweenKernel : public ::testing::Test {};
+
+TYPED_TEST_SUITE(TestNumericBetweenKernel, NumericArrowTypes);
+TYPED_TEST(TestNumericBetweenKernel, SimpleBetweenArrayScalarScalar) {
+  using ScalarType = typename TypeTraits<TypeParam>::ScalarType;
+  using CType = typename TypeTraits<TypeParam>::CType;
+
+  Datum zero(std::make_shared<ScalarType>(CType(0)));
+  Datum four(std::make_shared<ScalarType>(CType(4)));
+  ValidateBetween<TypeParam>("[]", zero, four, "[]");
+  ValidateBetween<TypeParam>("[null]", zero, four, "[null]");
+  ValidateBetween<TypeParam>("[0,0,1,1,2,2]", zero, four, "[0,0,1,1,1,1]");
+  ValidateBetween<TypeParam>("[0,1,2,3,4,5]", zero, four, "[0,1,1,1,0,0]");
+  ValidateBetween<TypeParam>("[5,4,3,2,1,0]", zero, four, "[0,0,1,1,1,0]");
+  ValidateBetween<TypeParam>("[null,0,1,1]", zero, four, "[null,0,1,1]");
+}
+
+TEST(TestSimpleBetweenKernel, SimpleStringTest) {

Review comment:
       Need to refactor tests to be similar to the compare ones, then add further tests.




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

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

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



[GitHub] [arrow] bkmgit commented on a change in pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
bkmgit commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r767635769



##########
File path: cpp/src/arrow/compute/kernels/codegen_internal.h
##########
@@ -982,6 +1021,407 @@ template <typename OutType, typename ArgType, typename Op>
 using ScalarBinaryNotNullStatefulEqualTypes =
     ScalarBinaryNotNullStateful<OutType, ArgType, ArgType, Op>;
 
+// A kernel exec generator for ternary functions that addresses both array and
+// scalar inputs and dispatches input iteration and output writing to other
+// templates
+//
+// This template executes the operator even on the data behind null values,
+// therefore it is generally only suitable for operators that are safe to apply
+// even on the null slot values.
+//
+// The "Op" functor should have the form
+//
+// struct Op {
+//   template <typename OutValue, typename Arg0Value, typename Arg1Value, typename
+//   Arg2Value> static OutValue Call(KernelContext* ctx, Arg0Value arg0, Arg1Value arg1,
+//   Arg2Value arg2, Status *st) {

Review comment:
       Reformatted.




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

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

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



[GitHub] [arrow] bkmgit commented on a change in pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
bkmgit commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r768855443



##########
File path: cpp/src/arrow/compute/kernels/scalar_compare_test.cc
##########
@@ -1850,5 +1851,154 @@ TEST(TestMaxElementWiseMinElementWise, CommonTemporal) {
               ResultWith(ScalarFromJSON(date64(), "86400000")));
 }
 
+template <typename ArrowType>
+static void ValidateBetween(const Datum& val, const Datum& lhs, const Datum& rhs,
+                            const Datum& expected) {
+  ASSERT_OK_AND_ASSIGN(Datum result, Between(val, lhs, rhs));
+  AssertArraysEqual(*expected.make_array(), *result.make_array(),
+                    /*verbose=*/true);
+}
+
+template <typename ArrowType>
+static void ValidateBetween(const char* value_str, const Datum& lhs, const Datum& rhs,
+                            const char* expected_str) {
+  auto value = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), value_str);
+  auto expected = ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), expected_str);
+  ValidateBetween<ArrowType>(value, lhs, rhs, expected);
+}
+
+template <>
+void ValidateBetween<StringType>(const char* value_str, const Datum& lhs,
+                                 const Datum& rhs, const char* expected_str) {
+  auto value = ArrayFromJSON(utf8(), value_str);
+  auto expected = ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), expected_str);
+  ValidateBetween<StringType>(value, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+class TestNumericBetweenKernel : public ::testing::Test {};
+
+TYPED_TEST_SUITE(TestNumericBetweenKernel, NumericArrowTypes);
+TYPED_TEST(TestNumericBetweenKernel, SimpleBetweenArrayScalarScalar) {
+  using ScalarType = typename TypeTraits<TypeParam>::ScalarType;
+  using CType = typename TypeTraits<TypeParam>::CType;
+
+  Datum zero(std::make_shared<ScalarType>(CType(0)));
+  Datum four(std::make_shared<ScalarType>(CType(4)));
+  ValidateBetween<TypeParam>("[]", zero, four, "[]");
+  ValidateBetween<TypeParam>("[null]", zero, four, "[null]");
+  ValidateBetween<TypeParam>("[0,0,1,1,2,2]", zero, four, "[0,0,1,1,1,1]");
+  ValidateBetween<TypeParam>("[0,1,2,3,4,5]", zero, four, "[0,1,1,1,0,0]");
+  ValidateBetween<TypeParam>("[5,4,3,2,1,0]", zero, four, "[0,0,1,1,1,0]");
+  ValidateBetween<TypeParam>("[null,0,1,1]", zero, four, "[null,0,1,1]");
+}
+
+TEST(TestSimpleBetweenKernel, SimpleStringTest) {
+  using ScalarType = typename TypeTraits<StringType>::ScalarType;
+  auto l = Datum(std::make_shared<ScalarType>("abc"));
+  auto r = Datum(std::make_shared<ScalarType>("zzz"));
+  ValidateBetween<StringType>("[]", l, r, "[]");
+  ValidateBetween<StringType>("[null]", l, r, "[null]");
+  ValidateBetween<StringType>(R"(["aaa", "aaaa", "ccc", "z"])", l, r,
+                              R"([false, false, true, true])");
+  ValidateBetween<StringType>(R"(["a", "aaaa", "c", "z"])", l, r,
+                              R"([false, false, true, true])");
+  ValidateBetween<StringType>(R"(["a", "aaaa", "fff", "zzzz"])", l, r,
+                              R"([false, false, true, false])");
+  ValidateBetween<StringType>(R"(["abd", null, null, "zzx"])", l, r,
+                              R"([true, null, null, true])");
+}
+
+TEST(TestSimpleBetweenKernel, SimpleTimestampTest) {
+  using ScalarType = typename TypeTraits<TimestampType>::ScalarType;
+  auto checkTimestampArray = [](std::shared_ptr<DataType> type, const char* input_str,
+                                const Datum& lhs, const Datum& rhs,
+                                const char* expected_str) {
+    auto value = ArrayFromJSON(type, input_str);
+    auto expected = ArrayFromJSON(boolean(), expected_str);
+    ValidateBetween<TimestampType>(value, lhs, rhs, expected);
+  };
+  auto unit = TimeUnit::SECOND;
+  auto l = Datum(std::make_shared<ScalarType>(923184000, timestamp(unit)));
+  auto r = Datum(std::make_shared<ScalarType>(1602032602, timestamp(unit)));
+  checkTimestampArray(timestamp(unit), "[]", l, r, "[]");
+  checkTimestampArray(timestamp(unit), "[null]", l, r, "[null]");
+  checkTimestampArray(timestamp(unit), R"(["1970-01-01","2000-02-29","1900-02-28"])", l,
+                      r, "[false,true,false]");
+  checkTimestampArray(timestamp(unit), R"(["1970-01-01","2000-02-29","2004-02-28"])", l,
+                      r, "[false,true,true]");
+  checkTimestampArray(timestamp(unit), R"(["2018-01-01","1999-04-04","1900-02-28"])", l,
+                      r, "[true,false,false]");
+}
+
+TYPED_TEST(TestNumericBetweenKernel, SimpleBetweenArrayArrayArray) {
+  ValidateBetween<TypeParam>(
+      "[]", ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"), "[]");
+  ValidateBetween<TypeParam>(
+      "[null]", ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"), "[null]");
+  ValidateBetween<TypeParam>(
+      "[1,1,2,2,2]",
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      "[true,true,false,false,false]");
+  ValidateBetween<TypeParam>(
+      "[1,1,2,2,2,2]",
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,2,5,5]"),
+      "[true,true,false,null,false,false]");
+}
+
+TEST(TestSimpleBetweenKernel, StringArrayArrayArrayTest) {
+  ValidateBetween<StringType>(
+      R"(["david","hello","world"])",
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["adam","hi","whirl"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                    R"(["robert","goeiemoreen","whirlwind"])"),
+      "[true, false, false]");
+  ValidateBetween<StringType>(
+      R"(["x","a","f"])",
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["w","a","e"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["z","a","g"])"),
+      "[true, false, true]");
+  ValidateBetween<StringType>(
+      R"(["block","bit","binary"])",
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                    R"(["bit","nibble","ternary"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["word","d","xyz"])"),
+      "[true, false, false]");
+  ValidateBetween<StringType>(R"(["Ayumi","アユミ","王梦莹"])",
+                              ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                            R"(["たなか","あゆみ","歩美"])"),
+                              ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                            R"(["李平之","田中","たなか"])"),
+                              "[false, true, false]");
+}
+
+TEST(TestSimpleBetweenKernel, TimestampArrayArrayArrayTest) {

Review comment:
       Will add time tests




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

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

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



[GitHub] [arrow] bkmgit commented on a change in pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
bkmgit commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r767709026



##########
File path: cpp/src/arrow/compute/kernels/scalar_compare_test.cc
##########
@@ -1850,5 +1851,154 @@ TEST(TestMaxElementWiseMinElementWise, CommonTemporal) {
               ResultWith(ScalarFromJSON(date64(), "86400000")));
 }
 
+template <typename ArrowType>
+static void ValidateBetween(const Datum& val, const Datum& lhs, const Datum& rhs,
+                            const Datum& expected) {
+  ASSERT_OK_AND_ASSIGN(Datum result, Between(val, lhs, rhs));
+  AssertArraysEqual(*expected.make_array(), *result.make_array(),
+                    /*verbose=*/true);
+}
+
+template <typename ArrowType>
+static void ValidateBetween(const char* value_str, const Datum& lhs, const Datum& rhs,
+                            const char* expected_str) {
+  auto value = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), value_str);
+  auto expected = ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), expected_str);
+  ValidateBetween<ArrowType>(value, lhs, rhs, expected);
+}
+
+template <>
+void ValidateBetween<StringType>(const char* value_str, const Datum& lhs,
+                                 const Datum& rhs, const char* expected_str) {
+  auto value = ArrayFromJSON(utf8(), value_str);
+  auto expected = ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), expected_str);
+  ValidateBetween<StringType>(value, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+class TestNumericBetweenKernel : public ::testing::Test {};
+
+TYPED_TEST_SUITE(TestNumericBetweenKernel, NumericArrowTypes);
+TYPED_TEST(TestNumericBetweenKernel, SimpleBetweenArrayScalarScalar) {
+  using ScalarType = typename TypeTraits<TypeParam>::ScalarType;
+  using CType = typename TypeTraits<TypeParam>::CType;
+
+  Datum zero(std::make_shared<ScalarType>(CType(0)));
+  Datum four(std::make_shared<ScalarType>(CType(4)));
+  ValidateBetween<TypeParam>("[]", zero, four, "[]");
+  ValidateBetween<TypeParam>("[null]", zero, four, "[null]");
+  ValidateBetween<TypeParam>("[0,0,1,1,2,2]", zero, four, "[0,0,1,1,1,1]");
+  ValidateBetween<TypeParam>("[0,1,2,3,4,5]", zero, four, "[0,1,1,1,0,0]");
+  ValidateBetween<TypeParam>("[5,4,3,2,1,0]", zero, four, "[0,0,1,1,1,0]");
+  ValidateBetween<TypeParam>("[null,0,1,1]", zero, four, "[null,0,1,1]");
+}
+
+TEST(TestSimpleBetweenKernel, SimpleStringTest) {
+  using ScalarType = typename TypeTraits<StringType>::ScalarType;
+  auto l = Datum(std::make_shared<ScalarType>("abc"));
+  auto r = Datum(std::make_shared<ScalarType>("zzz"));
+  ValidateBetween<StringType>("[]", l, r, "[]");
+  ValidateBetween<StringType>("[null]", l, r, "[null]");
+  ValidateBetween<StringType>(R"(["aaa", "aaaa", "ccc", "z"])", l, r,
+                              R"([false, false, true, true])");
+  ValidateBetween<StringType>(R"(["a", "aaaa", "c", "z"])", l, r,
+                              R"([false, false, true, true])");
+  ValidateBetween<StringType>(R"(["a", "aaaa", "fff", "zzzz"])", l, r,
+                              R"([false, false, true, false])");
+  ValidateBetween<StringType>(R"(["abd", null, null, "zzx"])", l, r,
+                              R"([true, null, null, true])");
+}
+
+TEST(TestSimpleBetweenKernel, SimpleTimestampTest) {
+  using ScalarType = typename TypeTraits<TimestampType>::ScalarType;
+  auto checkTimestampArray = [](std::shared_ptr<DataType> type, const char* input_str,
+                                const Datum& lhs, const Datum& rhs,
+                                const char* expected_str) {
+    auto value = ArrayFromJSON(type, input_str);
+    auto expected = ArrayFromJSON(boolean(), expected_str);
+    ValidateBetween<TimestampType>(value, lhs, rhs, expected);
+  };
+  auto unit = TimeUnit::SECOND;
+  auto l = Datum(std::make_shared<ScalarType>(923184000, timestamp(unit)));
+  auto r = Datum(std::make_shared<ScalarType>(1602032602, timestamp(unit)));
+  checkTimestampArray(timestamp(unit), "[]", l, r, "[]");
+  checkTimestampArray(timestamp(unit), "[null]", l, r, "[null]");
+  checkTimestampArray(timestamp(unit), R"(["1970-01-01","2000-02-29","1900-02-28"])", l,
+                      r, "[false,true,false]");
+  checkTimestampArray(timestamp(unit), R"(["1970-01-01","2000-02-29","2004-02-28"])", l,
+                      r, "[false,true,true]");
+  checkTimestampArray(timestamp(unit), R"(["2018-01-01","1999-04-04","1900-02-28"])", l,
+                      r, "[true,false,false]");
+}
+
+TYPED_TEST(TestNumericBetweenKernel, SimpleBetweenArrayArrayArray) {
+  ValidateBetween<TypeParam>(
+      "[]", ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"), "[]");
+  ValidateBetween<TypeParam>(
+      "[null]", ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"), "[null]");
+  ValidateBetween<TypeParam>(
+      "[1,1,2,2,2]",
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      "[true,true,false,false,false]");
+  ValidateBetween<TypeParam>(
+      "[1,1,2,2,2,2]",
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,2,5,5]"),
+      "[true,true,false,null,false,false]");
+}
+
+TEST(TestSimpleBetweenKernel, StringArrayArrayArrayTest) {
+  ValidateBetween<StringType>(
+      R"(["david","hello","world"])",
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["adam","hi","whirl"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                    R"(["robert","goeiemoreen","whirlwind"])"),
+      "[true, false, false]");
+  ValidateBetween<StringType>(
+      R"(["x","a","f"])",
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["w","a","e"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["z","a","g"])"),
+      "[true, false, true]");
+  ValidateBetween<StringType>(
+      R"(["block","bit","binary"])",
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                    R"(["bit","nibble","ternary"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["word","d","xyz"])"),
+      "[true, false, false]");
+  ValidateBetween<StringType>(R"(["Ayumi","アユミ","王梦莹"])",
+                              ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                            R"(["たなか","あゆみ","歩美"])"),
+                              ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                            R"(["李平之","田中","たなか"])"),
+                              "[false, true, false]");

Review comment:
       There is a related issue for sorting https://issues.apache.org/jira/browse/ARROW-12046




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

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

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



[GitHub] [arrow] bkmgit commented on a change in pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
bkmgit commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r767593940



##########
File path: cpp/src/arrow/compute/kernels/codegen_internal.h
##########
@@ -583,7 +606,23 @@ struct OutputAdapter<Type, enable_if_base_binary<Type>> {
     return Status::NotImplemented("NYI");
   }
 };
+/*
+template <typename Type>
+struct OutputAdapter<Type, enable_if_base_binary<Type>> {
+  using T = typename TypeTraits<Type>::ScalarType::ValueType;
 
+  template <typename Generator>
+  static Status Write(KernelContext*, Datum* out, Generator&& generator) {
+    ArrayData* out_arr = out->mutable_array();
+    auto out_data = out_arr->GetMutableValues<T>(1);
+    // TODO: Is this as fast as a more explicitly inlined function?
+    for (int64_t i = 0; i < out_arr->length; ++i) {
+      *out_data++ = generator();
+    }
+    return Status::OK();
+  }
+};
+*/

Review comment:
       Yes, will remove.




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

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

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



[GitHub] [arrow] cyb70289 commented on pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
cyb70289 commented on pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#issuecomment-997649339


   @bkmgit will do, may be a bit delay.


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

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

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



[GitHub] [arrow] bkmgit commented on a change in pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
bkmgit commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r775339296



##########
File path: cpp/src/arrow/compute/kernels/scalar_compare_test.cc
##########
@@ -1850,5 +1846,439 @@ TEST(TestMaxElementWiseMinElementWise, CommonTemporal) {
               ResultWith(ScalarFromJSON(date64(), "86400000")));
 }
 
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const Datum& val, const Datum& lhs,
+                            const Datum& rhs, const Datum& expected) {
+  ASSERT_OK_AND_ASSIGN(
+      Datum result,
+      CallFunction(BetweenOperatorToFunctionName(options.op), {val, lhs, rhs}));
+  AssertArraysEqual(*expected.make_array(), *result.make_array(),
+                    /*verbose=*/true);
+}
+
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const Datum& val, const Datum& lhs,
+                            const Datum& rhs, const char* expected_str) {
+  auto expected = ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), expected_str);
+  ValidateBetween<ArrowType>(options, val, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const Datum& val, const Datum& lhs,
+                            const char* rhs_str, const Datum& expected) {
+  auto rhs = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), rhs_str);
+  ValidateBetween<ArrowType>(options, val, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const Datum& val, const char* lhs_str,
+                            const Datum& rhs, const Datum& expected) {
+  auto lhs = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), lhs_str);
+  ValidateBetween<ArrowType>(options, val, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const char* val_str, const Datum& lhs,
+                            const Datum& rhs, const Datum& expected) {
+  auto val = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), val_str);
+  ValidateBetween<ArrowType>(options, val, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const Datum& val, const Datum& lhs,
+                            const char* rhs_str, const char* expected_str) {
+  auto rhs = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), rhs_str);
+  auto expected = ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), expected_str);
+  ValidateBetween<ArrowType>(options, val, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const Datum& val, const char* lhs_str,
+                            const Datum& rhs, const char* expected_str) {
+  auto lhs = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), lhs_str);
+  auto expected = ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), expected_str);
+  ValidateBetween<ArrowType>(options, val, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const Datum& val, const char* lhs_str,
+                            const char* rhs_str, const Datum& expected) {
+  auto lhs = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), lhs_str);
+  auto rhs = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), rhs_str);
+  ValidateBetween<ArrowType>(options, val, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const char* val_str, const Datum& lhs,
+                            const Datum& rhs, const char* expected_str) {
+  auto val = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), val_str);
+  auto expected = ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), expected_str);
+  ValidateBetween<ArrowType>(options, val, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const char* val_str,
+                            const char* lhs_str, const Datum& rhs,
+                            const Datum& expected) {
+  auto val = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), val_str);
+  auto lhs = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), lhs_str);
+  ValidateBetween<ArrowType>(options, val, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const char* val_str, const Datum& lhs,
+                            const char* rhs_str, const Datum& expected) {
+  auto val = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), val_str);
+  auto rhs = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), rhs_str);
+  ValidateBetween<ArrowType>(options, val, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const Datum& val, const char* lhs_str,
+                            const char* rhs_str, const char* expected_str) {
+  auto lhs = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), lhs_str);
+  auto rhs = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), rhs_str);
+  auto expected = ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), expected_str);
+  ValidateBetween<ArrowType>(options, val, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const char* val_str, const Datum& lhs,
+                            const char* rhs_str, const char* expected_str) {
+  auto val = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), val_str);
+  auto rhs = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), rhs_str);
+  auto expected = ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), expected_str);
+  ValidateBetween<ArrowType>(options, val, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const char* val_str,
+                            const char* lhs_str, const Datum& rhs,
+                            const char* expected_str) {
+  auto val = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), val_str);
+  auto lhs = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), lhs_str);
+  auto expected = ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), expected_str);
+  ValidateBetween<ArrowType>(options, val, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const char* val_str,
+                            const char* lhs_str, const char* rhs_str,
+                            const Datum& expected) {
+  auto val = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), val_str);
+  auto lhs = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), lhs_str);
+  auto rhs = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), rhs_str);
+  ValidateBetween<ArrowType>(options, val, lhs, rhs, expected);
+}
+
+template <typename T>
+static inline bool SlowBetween(BetweenOperator op, const T& val, const T& lhs,
+                               const T& rhs) {
+  switch (op) {
+    case BETWEEN_LESS_EQUAL_LESS_EQUAL:
+      return ((lhs <= val) && (val <= rhs));
+    case BETWEEN_LESS_EQUAL_LESS_THAN:
+      return ((lhs <= val) && (val < rhs));
+    case BETWEEN_LESS_THAN_LESS_EQUAL:
+      return ((lhs < val) && (val <= rhs));
+    case BETWEEN_LESS_THAN_LESS_THAN:
+      return ((lhs < val) && (val < rhs));
+    default:
+      return false;
+  }
+}
+
+template <typename ArrayType>
+std::vector<bool> NullBitmapFromThreeArrays(const ArrayType& val, const ArrayType& lhs,
+                                            const ArrayType& rhs) {
+  auto value_lambda = [&val](int64_t i) {
+    return val.null_count() == 0 ? true : val.IsValid(i);
+  };
+
+  auto left_lambda = [&lhs](int64_t i) {
+    return lhs.null_count() == 0 ? true : lhs.IsValid(i);
+  };
+
+  auto right_lambda = [&rhs](int64_t i) {
+    return rhs.null_count() == 0 ? true : rhs.IsValid(i);
+  };
+
+  const int64_t length = lhs.length();
+  std::vector<bool> null_bitmap(length);
+
+  for (int64_t i = 0; i < length; i++) {
+    null_bitmap[i] = value_lambda(i) && left_lambda(i) && right_lambda(i);
+  }
+
+  return null_bitmap;
+}
+
+template <typename ArrowType>
+Datum SimpleArrayScalarScalarBetween(BetweenOptions options, const Datum& val,

Review comment:
       Works now.




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

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

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



[GitHub] [arrow] bkmgit commented on a change in pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
bkmgit commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r775545459



##########
File path: cpp/src/arrow/compute/kernels/codegen_internal.h
##########
@@ -982,6 +1021,407 @@ template <typename OutType, typename ArgType, typename Op>
 using ScalarBinaryNotNullStatefulEqualTypes =
     ScalarBinaryNotNullStateful<OutType, ArgType, ArgType, Op>;
 
+// A kernel exec generator for ternary functions that addresses both array and
+// scalar inputs and dispatches input iteration and output writing to other
+// templates
+//
+// This template executes the operator even on the data behind null values,
+// therefore it is generally only suitable for operators that are safe to apply
+// even on the null slot values.
+//
+// The "Op" functor should have the form
+//
+// struct Op {
+//   template <typename OutValue, typename Arg0Value, typename Arg1Value, typename
+//   Arg2Value> static OutValue Call(KernelContext* ctx, Arg0Value arg0, Arg1Value arg1,
+//   Arg2Value arg2, Status *st) {
+//     // implementation
+//     // NOTE: "status" should only be populated with errors,
+//     //       leave it unmodified to indicate Status::OK()
+//   }
+// };
+template <typename OutType, typename Arg0Type, typename Arg1Type, typename Arg2Type,
+          typename Op>
+struct ScalarTernary {
+  using OutValue = typename GetOutputType<OutType>::T;
+  using Arg0Value = typename GetViewType<Arg0Type>::T;
+  using Arg1Value = typename GetViewType<Arg1Type>::T;
+  using Arg2Value = typename GetViewType<Arg2Type>::T;
+
+  static Status ArrayArrayArray(KernelContext* ctx, const ArrayData& arg0,
+                                const ArrayData& arg1, const ArrayData& arg2,
+                                Datum* out) {
+    Status st = Status::OK();
+    ArrayIterator<Arg0Type> arg0_it(arg0);
+    ArrayIterator<Arg1Type> arg1_it(arg1);
+    ArrayIterator<Arg2Type> arg2_it(arg2);
+    RETURN_NOT_OK(OutputAdapter<OutType>::Write(ctx, out, [&]() -> OutValue {
+      return Op::template Call<OutValue, Arg0Value, Arg1Value, Arg2Value>(
+          ctx, arg0_it(), arg1_it(), arg2_it(), &st);
+    }));
+    return st;
+  }
+
+  static Status ArrayArrayScalar(KernelContext* ctx, const ArrayData& arg0,
+                                 const ArrayData& arg1, const Scalar& arg2, Datum* out) {
+    Status st = Status::OK();
+    ArrayIterator<Arg0Type> arg0_it(arg0);
+    ArrayIterator<Arg1Type> arg1_it(arg1);
+    auto arg2_val = UnboxScalar<Arg2Type>::Unbox(arg2);
+    RETURN_NOT_OK(OutputAdapter<OutType>::Write(ctx, out, [&]() -> OutValue {
+      return Op::template Call<OutValue, Arg0Value, Arg1Value, Arg2Value>(
+          ctx, arg0_it(), arg1_it(), arg2_val, &st);
+    }));
+    return st;
+  }
+
+  static Status ArrayScalarArray(KernelContext* ctx, const ArrayData& arg0,
+                                 const Scalar& arg1, const ArrayData& arg2, Datum* out) {
+    Status st = Status::OK();
+    ArrayIterator<Arg0Type> arg0_it(arg0);
+    auto arg1_val = UnboxScalar<Arg1Type>::Unbox(arg1);
+    ArrayIterator<Arg2Type> arg2_it(arg2);
+    RETURN_NOT_OK(OutputAdapter<OutType>::Write(ctx, out, [&]() -> OutValue {
+      return Op::template Call<OutValue, Arg0Value, Arg1Value, Arg2Value>(
+          ctx, arg0_it(), arg1_val, arg2_it(), &st);
+    }));
+    return st;
+  }
+
+  static Status ScalarArrayArray(KernelContext* ctx, const Scalar& arg0,
+                                 const ArrayData& arg1, const ArrayData& arg2,
+                                 Datum* out) {
+    Status st = Status::OK();
+    auto arg0_val = UnboxScalar<Arg0Type>::Unbox(arg0);
+    ArrayIterator<Arg1Type> arg1_it(arg1);
+    ArrayIterator<Arg2Type> arg2_it(arg2);
+    RETURN_NOT_OK(OutputAdapter<OutType>::Write(ctx, out, [&]() -> OutValue {
+      return Op::template Call<OutValue, Arg0Value, Arg1Value, Arg2Value>(
+          ctx, arg0_val, arg1_it(), arg2_it(), &st);
+    }));
+    return st;
+  }
+
+  static Status ArrayScalarScalar(KernelContext* ctx, const ArrayData& arg0,
+                                  const Scalar& arg1, const Scalar& arg2, Datum* out) {
+    Status st = Status::OK();
+    ArrayIterator<Arg0Type> arg0_it(arg0);
+    auto arg1_val = UnboxScalar<Arg1Type>::Unbox(arg1);
+    auto arg2_val = UnboxScalar<Arg2Type>::Unbox(arg2);
+    RETURN_NOT_OK(OutputAdapter<OutType>::Write(ctx, out, [&]() -> OutValue {
+      return Op::template Call<OutValue, Arg0Value, Arg1Value, Arg2Value>(
+          ctx, arg0_it(), arg1_val, arg2_val, &st);
+    }));
+    return st;
+  }
+
+  static Status ScalarScalarArray(KernelContext* ctx, const Scalar& arg0,
+                                  const Scalar& arg1, const ArrayData& arg2, Datum* out) {
+    Status st = Status::OK();
+    auto arg0_val = UnboxScalar<Arg0Type>::Unbox(arg0);
+    auto arg1_val = UnboxScalar<Arg1Type>::Unbox(arg1);
+    ArrayIterator<Arg2Type> arg2_it(arg2);
+    RETURN_NOT_OK(OutputAdapter<OutType>::Write(ctx, out, [&]() -> OutValue {
+      return Op::template Call<OutValue, Arg0Value, Arg1Value, Arg2Value>(
+          ctx, arg0_val, arg1_val, arg2_it(), &st);
+    }));
+    return st;
+  }
+
+  static Status ScalarArrayScalar(KernelContext* ctx, const Scalar& arg0,
+                                  const ArrayData& arg1, const Scalar& arg2, Datum* out) {
+    Status st = Status::OK();
+    auto arg0_val = UnboxScalar<Arg0Type>::Unbox(arg0);
+    ArrayIterator<Arg1Type> arg1_it(arg1);
+    auto arg2_val = UnboxScalar<Arg2Type>::Unbox(arg2);
+    RETURN_NOT_OK(OutputAdapter<OutType>::Write(ctx, out, [&]() -> OutValue {
+      return Op::template Call<OutValue, Arg0Value, Arg1Value, Arg2Value>(
+          ctx, arg0_val, arg1_it(), arg2_val, &st);
+    }));
+    return st;
+  }
+
+  static Status ScalarScalarScalar(KernelContext* ctx, const Scalar& arg0,
+                                   const Scalar& arg1, const Scalar& arg2, Datum* out) {
+    Status st = Status::OK();
+    if (out->scalar()->is_valid) {
+      auto arg0_val = UnboxScalar<Arg0Type>::Unbox(arg0);
+      auto arg1_val = UnboxScalar<Arg1Type>::Unbox(arg1);
+      auto arg2_val = UnboxScalar<Arg2Type>::Unbox(arg2);
+      BoxScalar<OutType>::Box(
+          Op::template Call<OutValue, Arg0Value, Arg1Value, Arg2Value>(
+              ctx, arg0_val, arg1_val, arg2_val, &st),
+          out->scalar().get());
+    }
+    return st;
+  }
+
+  static Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    if (batch[0].kind() == Datum::ARRAY) {
+      if (batch[1].kind() == Datum::ARRAY) {
+        if (batch[2].kind() == Datum::ARRAY) {
+          return ArrayArrayArray(ctx, *batch[0].array(), *batch[1].array(),
+                                 *batch[2].array(), out);
+        } else {
+          return ArrayArrayScalar(ctx, *batch[0].array(), *batch[1].array(),
+                                  *batch[2].scalar(), out);
+        }
+      } else {
+        if (batch[2].kind() == Datum::ARRAY) {
+          return ArrayScalarArray(ctx, *batch[0].array(), *batch[1].scalar(),
+                                  *batch[2].array(), out);
+        } else {
+          return ArrayScalarScalar(ctx, *batch[0].array(), *batch[1].scalar(),
+                                   *batch[2].scalar(), out);
+        }
+      }
+    } else {
+      if (batch[1].kind() == Datum::ARRAY) {
+        if (batch[2].kind() == Datum::ARRAY) {
+          return ScalarArrayArray(ctx, *batch[0].scalar(), *batch[1].array(),
+                                  *batch[2].array(), out);
+        } else {
+          return ScalarArrayScalar(ctx, *batch[0].scalar(), *batch[1].array(),
+                                   *batch[2].scalar(), out);
+        }
+      } else {
+        if (batch[2].kind() == Datum::ARRAY) {
+          return ScalarScalarArray(ctx, *batch[0].scalar(), *batch[1].scalar(),
+                                   *batch[2].array(), out);
+        } else {
+          return ScalarScalarScalar(ctx, *batch[0].scalar(), *batch[1].scalar(),
+                                    *batch[2].scalar(), out);
+        }
+      }
+    }
+  }
+};
+
+// An alternative to ScalarTernary 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 Arg2Type,
+          typename Op>
+struct ScalarTernaryNotNullStateful {

Review comment:
       Removed unused code, this seems not needed for decimal support.




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

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

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



[GitHub] [arrow] edponce commented on a change in pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
edponce commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r776673334



##########
File path: cpp/src/arrow/util/bit_block_counter.h
##########
@@ -87,6 +87,147 @@ struct BitBlockOrNot<bool> {
   static bool Call(bool left, bool right) { return left || !right; }
 };
 
+// Three Arguments
+template <typename T>
+struct BitBlockAndAnd {
+  static T Call(T left, T mid, T right) { return left & mid & right; }
+};
+
+template <>
+struct BitBlockAndAnd<bool> {
+  static bool Call(bool left, bool mid, bool right) { return left && mid && right; }

Review comment:
       Note, the only bool specialization needed would be for negating bool variables. I submitted [this PR with these changes](https://github.com/apache/arrow/pull/12052) that if they get accepted you can make use of the added`BitNot()` to wrap negations.




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

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

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



[GitHub] [arrow] lidavidm commented on a change in pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
lidavidm commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r775903695



##########
File path: cpp/src/arrow/compute/kernels/scalar_compare.cc
##########
@@ -68,6 +72,50 @@ struct GreaterEqual {
   }
 };
 
+struct BetweenLessEqualLessEqual {
+  template <typename T, typename Arg0, typename Arg1, typename Arg2>
+  static constexpr T Call(KernelContext*, const Arg0& middle, const Arg1& left,
+                          const Arg2& right, Status*) {
+    static_assert(std::is_same<T, bool>::value && std::is_same<Arg0, Arg1>::value &&
+                      std::is_same<Arg1, Arg2>::value,
+                  "");
+    return (left <= middle) && (middle <= right);
+  }
+};
+
+struct BetweenLessThanLessEqual {
+  template <typename T, typename Arg0, typename Arg1, typename Arg2>
+  static constexpr T Call(KernelContext*, const Arg0& middle, const Arg1& left,
+                          const Arg2& right, Status*) {
+    static_assert(std::is_same<T, bool>::value && std::is_same<Arg0, Arg1>::value &&
+                      std::is_same<Arg1, Arg2>::value,
+                  "");
+    return (left < middle) && (middle <= right);
+  }
+};
+
+struct BetweenLessEqualLessThan {
+  template <typename T, typename Arg0, typename Arg1, typename Arg2>
+  static constexpr T Call(KernelContext*, const Arg0& middle, const Arg1& left,
+                          const Arg2& right, Status*) {
+    static_assert(std::is_same<T, bool>::value && std::is_same<Arg0, Arg1>::value &&
+                      std::is_same<Arg1, Arg2>::value,
+                  "");
+    return (left <= middle) && (middle < right);
+  }
+};
+
+struct BetweenLessThanLessThan {
+  template <typename T, typename Arg0, typename Arg1, typename Arg2>
+  static constexpr T Call(KernelContext*, const Arg0& middle, const Arg1& left,
+                          const Arg2& right, Status*) {
+    static_assert(std::is_same<T, bool>::value && std::is_same<Arg0, Arg1>::value &&
+                      std::is_same<Arg1, Arg2>::value,
+                  "");
+    return (left < middle) && (middle < right);
+  }
+};

Review comment:
       The CompareOptions weren't really options, it was just a facade in C++ over the underlying functions (somewhat similar to ArithmeticOptions in fact). When the options classes were "upgraded" to make them serializable, CompareOptions was deemed redundant and mostly removed.
   
   Looking at it now, I can see the argument for four separate functions for consistency, but that means we probably don't want/need BetweenOptions or any enum at all.




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

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

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



[GitHub] [arrow] bkmgit commented on a change in pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
bkmgit commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r775938819



##########
File path: cpp/src/arrow/compute/kernels/scalar_compare.cc
##########
@@ -68,6 +72,50 @@ struct GreaterEqual {
   }
 };
 
+struct BetweenLessEqualLessEqual {
+  template <typename T, typename Arg0, typename Arg1, typename Arg2>
+  static constexpr T Call(KernelContext*, const Arg0& middle, const Arg1& left,
+                          const Arg2& right, Status*) {
+    static_assert(std::is_same<T, bool>::value && std::is_same<Arg0, Arg1>::value &&
+                      std::is_same<Arg1, Arg2>::value,
+                  "");
+    return (left <= middle) && (middle <= right);
+  }
+};
+
+struct BetweenLessThanLessEqual {
+  template <typename T, typename Arg0, typename Arg1, typename Arg2>
+  static constexpr T Call(KernelContext*, const Arg0& middle, const Arg1& left,
+                          const Arg2& right, Status*) {
+    static_assert(std::is_same<T, bool>::value && std::is_same<Arg0, Arg1>::value &&
+                      std::is_same<Arg1, Arg2>::value,
+                  "");
+    return (left < middle) && (middle <= right);
+  }
+};
+
+struct BetweenLessEqualLessThan {
+  template <typename T, typename Arg0, typename Arg1, typename Arg2>
+  static constexpr T Call(KernelContext*, const Arg0& middle, const Arg1& left,
+                          const Arg2& right, Status*) {
+    static_assert(std::is_same<T, bool>::value && std::is_same<Arg0, Arg1>::value &&
+                      std::is_same<Arg1, Arg2>::value,
+                  "");
+    return (left <= middle) && (middle < right);
+  }
+};
+
+struct BetweenLessThanLessThan {
+  template <typename T, typename Arg0, typename Arg1, typename Arg2>
+  static constexpr T Call(KernelContext*, const Arg0& middle, const Arg1& left,
+                          const Arg2& right, Status*) {
+    static_assert(std::is_same<T, bool>::value && std::is_same<Arg0, Arg1>::value &&
+                      std::is_same<Arg1, Arg2>::value,
+                  "");
+    return (left < middle) && (middle < right);
+  }
+};

Review comment:
       Strings are sorted by iso codes. The codes may not be the appropriate ordering in all cases, see for example, - https://en.wikipedia.org/wiki/Alphabetical_order#Language-specific_conventions
   - https://www.postgresql.org/docs/current/collation.html
   - https://www.rdocumentation.org/packages/base/versions/3.6.2/topics/xtfrm




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

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

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



[GitHub] [arrow] bkmgit commented on a change in pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
bkmgit commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r776655406



##########
File path: cpp/src/arrow/compute/kernels/scalar_compare.cc
##########
@@ -746,6 +787,111 @@ std::shared_ptr<ScalarFunction> MakeScalarMinMax(std::string name,
   return func;
 }
 
+template <typename Op>
+void AddIntegerBetween(const std::shared_ptr<DataType>& ty, ScalarFunction* func) {
+  auto exec = GeneratePhysicalInteger<ScalarTernaryEqualTypes, BooleanType, Op>(*ty);
+  DCHECK_OK(func->AddKernel({ty, ty, ty}, boolean(), std::move(exec)));
+}
+
+template <typename InType, typename Op>
+void AddGenericBetween(const std::shared_ptr<DataType>& ty, ScalarFunction* func) {
+  DCHECK_OK(func->AddKernel({ty, ty, ty}, boolean(),
+                            ScalarTernaryEqualTypes<BooleanType, InType, Op>::Exec));
+}
+
+template <typename OutType, typename ArgType, typename Op>
+struct BetweenTimestamps : public ScalarTernaryEqualTypes<OutType, ArgType, Op> {
+  using Base = ScalarTernaryEqualTypes<OutType, ArgType, Op>;
+
+  static Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    const auto& var = checked_cast<const TimestampType&>(*batch[0].type());
+    const auto& lhs = checked_cast<const TimestampType&>(*batch[1].type());
+    const auto& rhs = checked_cast<const TimestampType&>(*batch[2].type());
+    if ((var.timezone().empty() != lhs.timezone().empty()) ||
+        (var.timezone().empty() != rhs.timezone().empty()) ||
+        (lhs.timezone().empty() != rhs.timezone().empty())) {
+      return Status::Invalid(
+          "Cannot use timestamps with timezone and timestamps without timezones, got: ",
+          var.timezone().empty(), " ", lhs.timezone().empty(), " and ",
+          rhs.timezone().empty());
+    }
+    return Base::Exec(ctx, batch, out);
+  }
+};
+
+template <typename Op>
+std::shared_ptr<ScalarFunction> MakeBetweenFunction(std::string name,
+                                                    const FunctionDoc* doc) {
+  auto func = std::make_shared<CompareFunction>(name, Arity::Ternary(), doc);
+
+  DCHECK_OK(func->AddKernel(
+      {boolean(), boolean(), boolean()}, boolean(),
+      ScalarTernary<BooleanType, BooleanType, BooleanType, BooleanType, Op>::Exec));
+
+  for (const std::shared_ptr<DataType>& ty : IntTypes()) {
+    AddIntegerBetween<Op>(ty, func.get());
+  }
+
+  AddIntegerBetween<Op>(date32(), func.get());
+  AddIntegerBetween<Op>(date64(), func.get());
+
+  AddGenericBetween<FloatType, Op>(float32(), func.get());
+  AddGenericBetween<DoubleType, Op>(float64(), func.get());
+
+  // Add timestamp kernels
+  for (auto unit : TimeUnit::values()) {
+    InputType in_type(match::TimestampTypeUnit(unit));
+    DCHECK_OK(func->AddKernel({in_type, in_type, in_type}, boolean(),
+                              BetweenTimestamps<BooleanType, TimestampType, Op>::Exec));
+  }
+
+  // Duration
+  for (auto unit : TimeUnit::values()) {
+    InputType in_type(match::DurationTypeUnit(unit));
+    auto exec =
+        GeneratePhysicalInteger<ScalarTernaryEqualTypes, BooleanType, Op>(int64());
+    DCHECK_OK(func->AddKernel({in_type, in_type, in_type}, boolean(), std::move(exec)));
+  }
+
+  // Time32 and Time64
+  for (auto unit : {TimeUnit::SECOND, TimeUnit::MILLI}) {
+    InputType in_type(match::Time32TypeUnit(unit));
+    auto exec =
+        GeneratePhysicalInteger<ScalarTernaryEqualTypes, BooleanType, Op>(int32());
+    DCHECK_OK(func->AddKernel({in_type, in_type, in_type}, boolean(), std::move(exec)));
+  }
+  for (auto unit : {TimeUnit::MICRO, TimeUnit::NANO}) {
+    InputType in_type(match::Time64TypeUnit(unit));
+    auto exec =
+        GeneratePhysicalInteger<ScalarTernaryEqualTypes, BooleanType, Op>(int64());
+    DCHECK_OK(func->AddKernel({in_type, in_type, in_type}, boolean(), std::move(exec)));
+  }
+
+  for (const std::shared_ptr<DataType>& ty : BaseBinaryTypes()) {
+    auto exec = GenerateVarBinaryBase<ScalarTernaryEqualTypes, BooleanType, Op>(*ty);

Review comment:
       Ok. This followed line 335 for ScalarBinaryEqualTypes, so I guess this can be changed as well.




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

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

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



[GitHub] [arrow] bkmgit commented on a change in pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
bkmgit commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r776690939



##########
File path: cpp/src/arrow/util/bit_block_counter.h
##########
@@ -87,6 +87,147 @@ struct BitBlockOrNot<bool> {
   static bool Call(bool left, bool right) { return left || !right; }
 };
 
+// Three Arguments
+template <typename T>
+struct BitBlockAndAnd {
+  static T Call(T left, T mid, T right) { return left & mid & right; }
+};
+
+template <>
+struct BitBlockAndAnd<bool> {
+  static bool Call(bool left, bool mid, bool right) { return left && mid && right; }

Review comment:
       Reverted this change until https://github.com/apache/arrow/pull/12052 is resolved.




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

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

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



[GitHub] [arrow] bkmgit commented on a change in pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
bkmgit commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r776705668



##########
File path: cpp/src/arrow/compute/api_scalar.h
##########
@@ -1240,5 +1255,22 @@ ARROW_EXPORT Result<Datum> AssumeTimezone(const Datum& values,
                                           AssumeTimezoneOptions options,
                                           ExecContext* ctx = NULLPTR);
 
+/// \brief Between compares each element in `values`
+/// with `left` as a lower bound and 'right' as an upperbound
+///
+/// \param[in] values input to compare between left and right
+/// \param[in] left used as the lower bound for comparison
+/// \param[in] right used as the upper bound for comparison
+//  \param[in] options between options, optional
+/// \param[in] ctx the function execution context, optional
+///
+/// \return the resulting datum
+///
+/// \note API not yet finalized
+ARROW_EXPORT
+Result<Datum> Between(const Datum& values, const Datum& left, const Datum& right,
+                      BetweenOptions options = BetweenOptions(),

Review comment:
       Done




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

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

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



[GitHub] [arrow] edponce commented on pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
edponce commented on pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#issuecomment-1003110634


   From the previous discussion on compute function's API, recall that the main entry point for invoking a function is `CallFunction(name, args, options, ctx)`. This allows passing function-specific `options`. If functions were to be grouped and the `options` parameter used to select a particular function, then expressing function-specific `options` would be difficult.
   
   For this particular kernels, `between`, I recommend to have a single `between` function that uses the `options` from `CallFunction` to select the desired kernel implementation. This approach will provide the structure for the `not_between` function as well.
   
   @bkmgit I created ARROW-15223 as a follow-up JIRA for adding the `not_between` compute function. Thanks!


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

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

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



[GitHub] [arrow] github-actions[bot] commented on pull request #11882: ARROW-9843: [C++] Between Kernel

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#issuecomment-987898907


   https://issues.apache.org/jira/browse/ARROW-9843


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

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

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



[GitHub] [arrow] kou commented on a change in pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
kou commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r767413145



##########
File path: cpp/src/arrow/compute/kernels/scalar_compare_test.cc
##########
@@ -1850,5 +1851,154 @@ TEST(TestMaxElementWiseMinElementWise, CommonTemporal) {
               ResultWith(ScalarFromJSON(date64(), "86400000")));
 }
 
+template <typename ArrowType>
+static void ValidateBetween(const Datum& val, const Datum& lhs, const Datum& rhs,
+                            const Datum& expected) {
+  ASSERT_OK_AND_ASSIGN(Datum result, Between(val, lhs, rhs));
+  AssertArraysEqual(*expected.make_array(), *result.make_array(),
+                    /*verbose=*/true);
+}
+
+template <typename ArrowType>
+static void ValidateBetween(const char* value_str, const Datum& lhs, const Datum& rhs,
+                            const char* expected_str) {
+  auto value = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), value_str);
+  auto expected = ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), expected_str);
+  ValidateBetween<ArrowType>(value, lhs, rhs, expected);
+}
+
+template <>
+void ValidateBetween<StringType>(const char* value_str, const Datum& lhs,
+                                 const Datum& rhs, const char* expected_str) {
+  auto value = ArrayFromJSON(utf8(), value_str);
+  auto expected = ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), expected_str);
+  ValidateBetween<StringType>(value, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+class TestNumericBetweenKernel : public ::testing::Test {};
+
+TYPED_TEST_SUITE(TestNumericBetweenKernel, NumericArrowTypes);
+TYPED_TEST(TestNumericBetweenKernel, SimpleBetweenArrayScalarScalar) {
+  using ScalarType = typename TypeTraits<TypeParam>::ScalarType;
+  using CType = typename TypeTraits<TypeParam>::CType;
+
+  Datum zero(std::make_shared<ScalarType>(CType(0)));
+  Datum four(std::make_shared<ScalarType>(CType(4)));
+  ValidateBetween<TypeParam>("[]", zero, four, "[]");
+  ValidateBetween<TypeParam>("[null]", zero, four, "[null]");
+  ValidateBetween<TypeParam>("[0,0,1,1,2,2]", zero, four, "[0,0,1,1,1,1]");
+  ValidateBetween<TypeParam>("[0,1,2,3,4,5]", zero, four, "[0,1,1,1,0,0]");
+  ValidateBetween<TypeParam>("[5,4,3,2,1,0]", zero, four, "[0,0,1,1,1,0]");
+  ValidateBetween<TypeParam>("[null,0,1,1]", zero, four, "[null,0,1,1]");

Review comment:
       Should we also add tests for `lhs` is NULL and/or `rhs` is NULL cases?

##########
File path: cpp/src/arrow/compute/kernels/scalar_compare.cc
##########
@@ -746,6 +768,76 @@ std::shared_ptr<ScalarFunction> MakeScalarMinMax(std::string name,
   return func;
 }
 
+template <typename Op>
+void AddIntegerBetween(const std::shared_ptr<DataType>& ty, ScalarFunction* func) {
+  auto exec = GeneratePhysicalInteger<ScalarTernaryEqualTypes, BooleanType, Op>(*ty);
+  DCHECK_OK(func->AddKernel({ty, ty, ty}, boolean(), std::move(exec)));
+}
+
+template <typename InType, typename Op>
+void AddGenericBetween(const std::shared_ptr<DataType>& ty, ScalarFunction* func) {
+  DCHECK_OK(func->AddKernel({ty, ty, ty}, boolean(),
+                            ScalarTernaryEqualTypes<BooleanType, InType, Op>::Exec));
+}
+
+template <typename Op>
+std::shared_ptr<ScalarFunction> MakeBetweenFunction(std::string name,
+                                                    const FunctionDoc* doc) {
+  auto func = std::make_shared<CompareFunction>(name, Arity::Ternary(), doc);
+
+  for (const std::shared_ptr<DataType>& ty : IntTypes()) {
+    AddIntegerBetween<Op>(ty, func.get());
+  }
+
+  AddIntegerBetween<Op>(date32(), func.get());
+  AddIntegerBetween<Op>(date64(), func.get());
+
+  AddGenericBetween<FloatType, Op>(float32(), func.get());
+  AddGenericBetween<DoubleType, Op>(float64(), func.get());
+
+  // Add timestamp kernels
+  for (auto unit : TimeUnit::values()) {
+    InputType in_type(match::TimestampTypeUnit(unit));
+    auto exec =
+        GeneratePhysicalInteger<ScalarTernaryEqualTypes, BooleanType, Op>(int64());
+    DCHECK_OK(func->AddKernel({in_type, in_type, in_type}, boolean(), std::move(exec)));

Review comment:
       It seems that we need to care time zone here like #11541.

##########
File path: cpp/src/arrow/compute/kernels/scalar_compare.cc
##########
@@ -68,6 +72,32 @@ struct GreaterEqual {
   }
 };
 
+struct Between {
+  template <typename T, typename Arg0, typename Arg1, typename Arg2>
+  static constexpr T Call(KernelContext*, const Arg0& middle, const Arg1& left,
+                          const Arg2& right, Status*) {
+    static_assert(std::is_same<T, bool>::value && std::is_same<Arg0, Arg1>::value &&
+                      std::is_same<Arg1, Arg2>::value,
+                  "");
+    return (left < middle) && (middle < right);
+  }
+};
+
+template <typename T>
+using is_unsigned_integer = std::integral_constant<bool, std::is_integral<T>::value &&
+                                                             std::is_unsigned<T>::value>;

Review comment:
       Did you copy this from `codegen_internal.h`?
   Can we reuse existing declaration?

##########
File path: cpp/src/arrow/compute/kernels/scalar_compare_test.cc
##########
@@ -1850,5 +1851,154 @@ TEST(TestMaxElementWiseMinElementWise, CommonTemporal) {
               ResultWith(ScalarFromJSON(date64(), "86400000")));
 }
 
+template <typename ArrowType>
+static void ValidateBetween(const Datum& val, const Datum& lhs, const Datum& rhs,
+                            const Datum& expected) {
+  ASSERT_OK_AND_ASSIGN(Datum result, Between(val, lhs, rhs));
+  AssertArraysEqual(*expected.make_array(), *result.make_array(),
+                    /*verbose=*/true);
+}
+
+template <typename ArrowType>
+static void ValidateBetween(const char* value_str, const Datum& lhs, const Datum& rhs,
+                            const char* expected_str) {
+  auto value = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), value_str);
+  auto expected = ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), expected_str);
+  ValidateBetween<ArrowType>(value, lhs, rhs, expected);
+}
+
+template <>
+void ValidateBetween<StringType>(const char* value_str, const Datum& lhs,
+                                 const Datum& rhs, const char* expected_str) {
+  auto value = ArrayFromJSON(utf8(), value_str);
+  auto expected = ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), expected_str);
+  ValidateBetween<StringType>(value, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+class TestNumericBetweenKernel : public ::testing::Test {};
+
+TYPED_TEST_SUITE(TestNumericBetweenKernel, NumericArrowTypes);
+TYPED_TEST(TestNumericBetweenKernel, SimpleBetweenArrayScalarScalar) {

Review comment:
       Can we also add `RandomBetweenArrayScalarScalar` like `RandomComapreArrayScalar`?

##########
File path: cpp/src/arrow/compute/kernels/scalar_compare.cc
##########
@@ -786,6 +878,10 @@ const FunctionDoc max_element_wise_doc{
     {"*args"},
     "ElementWiseAggregateOptions"};
 
+const FunctionDoc between_doc{"Check if values are in a range x <= y <= z",

Review comment:
       Is this correct?
   
   ```suggestion
   const FunctionDoc between_doc{"Check if values are in a range x < y < z",
   ```

##########
File path: cpp/src/arrow/compute/kernels/scalar_compare_test.cc
##########
@@ -1850,5 +1851,154 @@ TEST(TestMaxElementWiseMinElementWise, CommonTemporal) {
               ResultWith(ScalarFromJSON(date64(), "86400000")));
 }
 
+template <typename ArrowType>
+static void ValidateBetween(const Datum& val, const Datum& lhs, const Datum& rhs,
+                            const Datum& expected) {
+  ASSERT_OK_AND_ASSIGN(Datum result, Between(val, lhs, rhs));
+  AssertArraysEqual(*expected.make_array(), *result.make_array(),
+                    /*verbose=*/true);
+}
+
+template <typename ArrowType>
+static void ValidateBetween(const char* value_str, const Datum& lhs, const Datum& rhs,
+                            const char* expected_str) {
+  auto value = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), value_str);
+  auto expected = ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), expected_str);
+  ValidateBetween<ArrowType>(value, lhs, rhs, expected);
+}
+
+template <>
+void ValidateBetween<StringType>(const char* value_str, const Datum& lhs,
+                                 const Datum& rhs, const char* expected_str) {
+  auto value = ArrayFromJSON(utf8(), value_str);
+  auto expected = ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), expected_str);
+  ValidateBetween<StringType>(value, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+class TestNumericBetweenKernel : public ::testing::Test {};
+
+TYPED_TEST_SUITE(TestNumericBetweenKernel, NumericArrowTypes);
+TYPED_TEST(TestNumericBetweenKernel, SimpleBetweenArrayScalarScalar) {
+  using ScalarType = typename TypeTraits<TypeParam>::ScalarType;
+  using CType = typename TypeTraits<TypeParam>::CType;
+
+  Datum zero(std::make_shared<ScalarType>(CType(0)));
+  Datum four(std::make_shared<ScalarType>(CType(4)));
+  ValidateBetween<TypeParam>("[]", zero, four, "[]");
+  ValidateBetween<TypeParam>("[null]", zero, four, "[null]");
+  ValidateBetween<TypeParam>("[0,0,1,1,2,2]", zero, four, "[0,0,1,1,1,1]");
+  ValidateBetween<TypeParam>("[0,1,2,3,4,5]", zero, four, "[0,1,1,1,0,0]");
+  ValidateBetween<TypeParam>("[5,4,3,2,1,0]", zero, four, "[0,0,1,1,1,0]");
+  ValidateBetween<TypeParam>("[null,0,1,1]", zero, four, "[null,0,1,1]");
+}
+
+TEST(TestSimpleBetweenKernel, SimpleStringTest) {
+  using ScalarType = typename TypeTraits<StringType>::ScalarType;
+  auto l = Datum(std::make_shared<ScalarType>("abc"));
+  auto r = Datum(std::make_shared<ScalarType>("zzz"));
+  ValidateBetween<StringType>("[]", l, r, "[]");
+  ValidateBetween<StringType>("[null]", l, r, "[null]");
+  ValidateBetween<StringType>(R"(["aaa", "aaaa", "ccc", "z"])", l, r,
+                              R"([false, false, true, true])");
+  ValidateBetween<StringType>(R"(["a", "aaaa", "c", "z"])", l, r,
+                              R"([false, false, true, true])");
+  ValidateBetween<StringType>(R"(["a", "aaaa", "fff", "zzzz"])", l, r,
+                              R"([false, false, true, false])");
+  ValidateBetween<StringType>(R"(["abd", null, null, "zzx"])", l, r,
+                              R"([true, null, null, true])");
+}
+
+TEST(TestSimpleBetweenKernel, SimpleTimestampTest) {
+  using ScalarType = typename TypeTraits<TimestampType>::ScalarType;
+  auto checkTimestampArray = [](std::shared_ptr<DataType> type, const char* input_str,
+                                const Datum& lhs, const Datum& rhs,
+                                const char* expected_str) {
+    auto value = ArrayFromJSON(type, input_str);
+    auto expected = ArrayFromJSON(boolean(), expected_str);
+    ValidateBetween<TimestampType>(value, lhs, rhs, expected);
+  };
+  auto unit = TimeUnit::SECOND;
+  auto l = Datum(std::make_shared<ScalarType>(923184000, timestamp(unit)));
+  auto r = Datum(std::make_shared<ScalarType>(1602032602, timestamp(unit)));
+  checkTimestampArray(timestamp(unit), "[]", l, r, "[]");
+  checkTimestampArray(timestamp(unit), "[null]", l, r, "[null]");
+  checkTimestampArray(timestamp(unit), R"(["1970-01-01","2000-02-29","1900-02-28"])", l,
+                      r, "[false,true,false]");
+  checkTimestampArray(timestamp(unit), R"(["1970-01-01","2000-02-29","2004-02-28"])", l,
+                      r, "[false,true,true]");
+  checkTimestampArray(timestamp(unit), R"(["2018-01-01","1999-04-04","1900-02-28"])", l,
+                      r, "[true,false,false]");
+}
+
+TYPED_TEST(TestNumericBetweenKernel, SimpleBetweenArrayArrayArray) {
+  ValidateBetween<TypeParam>(
+      "[]", ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"), "[]");
+  ValidateBetween<TypeParam>(
+      "[null]", ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"), "[null]");
+  ValidateBetween<TypeParam>(
+      "[1,1,2,2,2]",
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      "[true,true,false,false,false]");
+  ValidateBetween<TypeParam>(
+      "[1,1,2,2,2,2]",
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,2,5,5]"),
+      "[true,true,false,null,false,false]");
+}
+
+TEST(TestSimpleBetweenKernel, StringArrayArrayArrayTest) {
+  ValidateBetween<StringType>(
+      R"(["david","hello","world"])",
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["adam","hi","whirl"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                    R"(["robert","goeiemoreen","whirlwind"])"),
+      "[true, false, false]");
+  ValidateBetween<StringType>(
+      R"(["x","a","f"])",
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["w","a","e"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["z","a","g"])"),
+      "[true, false, true]");
+  ValidateBetween<StringType>(
+      R"(["block","bit","binary"])",
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                    R"(["bit","nibble","ternary"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["word","d","xyz"])"),
+      "[true, false, false]");
+  ValidateBetween<StringType>(R"(["Ayumi","アユミ","王梦莹"])",
+                              ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                            R"(["たなか","あゆみ","歩美"])"),
+                              ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                            R"(["李平之","田中","たなか"])"),
+                              "[false, true, false]");

Review comment:
       Do we need this case?

##########
File path: cpp/src/arrow/util/bit_block_counter.h
##########
@@ -586,5 +961,144 @@ static void VisitTwoBitBlocksVoid(const std::shared_ptr<Buffer>& left_bitmap_buf
   }
 }
 
+template <typename VisitNotNull, typename VisitNull>
+static void VisitThreeBitBlocksVoid(
+    const std::shared_ptr<Buffer>& left_bitmap_buf, int64_t left_offset,
+    const std::shared_ptr<Buffer>& mid_bitmap_buf, int64_t mid_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) && (mid_bitmap_buf == NULLPTR)) ||
+      ((mid_bitmap_buf == NULLPTR) && (right_bitmap_buf == NULLPTR)) ||
+      ((left_bitmap_buf == NULLPTR) && (right_bitmap_buf == NULLPTR))) {
+    // At most one bitmap is present
+    if ((left_bitmap_buf == NULLPTR) && (mid_bitmap_buf == NULLPTR)) {
+      return VisitBitBlocksVoid(right_bitmap_buf, right_offset, length,
+                                std::forward<VisitNotNull>(visit_not_null),
+                                std::forward<VisitNull>(visit_null));
+    } else if ((mid_bitmap_buf == NULLPTR) && (right_bitmap_buf == NULLPTR)) {
+      return VisitBitBlocksVoid(left_bitmap_buf, left_offset, length,
+                                std::forward<VisitNotNull>(visit_not_null),
+                                std::forward<VisitNull>(visit_null));
+    } else {
+      return VisitBitBlocksVoid(mid_bitmap_buf, mid_offset, length,
+                                std::forward<VisitNotNull>(visit_not_null),
+                                std::forward<VisitNull>(visit_null));
+    }
+  }
+  // Two bitmaps are present
+  if (left_bitmap_buf == NULLPTR) {
+    const uint8_t* mid_bitmap = mid_bitmap_buf->data();

Review comment:
       Can we use `VisitTwoBitBlocksVoid()` here?

##########
File path: cpp/src/arrow/compute/kernels/scalar_compare_test.cc
##########
@@ -1850,5 +1851,154 @@ TEST(TestMaxElementWiseMinElementWise, CommonTemporal) {
               ResultWith(ScalarFromJSON(date64(), "86400000")));
 }
 
+template <typename ArrowType>
+static void ValidateBetween(const Datum& val, const Datum& lhs, const Datum& rhs,
+                            const Datum& expected) {
+  ASSERT_OK_AND_ASSIGN(Datum result, Between(val, lhs, rhs));
+  AssertArraysEqual(*expected.make_array(), *result.make_array(),
+                    /*verbose=*/true);
+}
+
+template <typename ArrowType>
+static void ValidateBetween(const char* value_str, const Datum& lhs, const Datum& rhs,
+                            const char* expected_str) {
+  auto value = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), value_str);
+  auto expected = ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), expected_str);
+  ValidateBetween<ArrowType>(value, lhs, rhs, expected);
+}
+
+template <>
+void ValidateBetween<StringType>(const char* value_str, const Datum& lhs,
+                                 const Datum& rhs, const char* expected_str) {
+  auto value = ArrayFromJSON(utf8(), value_str);
+  auto expected = ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), expected_str);
+  ValidateBetween<StringType>(value, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+class TestNumericBetweenKernel : public ::testing::Test {};
+
+TYPED_TEST_SUITE(TestNumericBetweenKernel, NumericArrowTypes);
+TYPED_TEST(TestNumericBetweenKernel, SimpleBetweenArrayScalarScalar) {
+  using ScalarType = typename TypeTraits<TypeParam>::ScalarType;
+  using CType = typename TypeTraits<TypeParam>::CType;
+
+  Datum zero(std::make_shared<ScalarType>(CType(0)));
+  Datum four(std::make_shared<ScalarType>(CType(4)));
+  ValidateBetween<TypeParam>("[]", zero, four, "[]");
+  ValidateBetween<TypeParam>("[null]", zero, four, "[null]");
+  ValidateBetween<TypeParam>("[0,0,1,1,2,2]", zero, four, "[0,0,1,1,1,1]");
+  ValidateBetween<TypeParam>("[0,1,2,3,4,5]", zero, four, "[0,1,1,1,0,0]");
+  ValidateBetween<TypeParam>("[5,4,3,2,1,0]", zero, four, "[0,0,1,1,1,0]");
+  ValidateBetween<TypeParam>("[null,0,1,1]", zero, four, "[null,0,1,1]");
+}
+
+TEST(TestSimpleBetweenKernel, SimpleStringTest) {
+  using ScalarType = typename TypeTraits<StringType>::ScalarType;
+  auto l = Datum(std::make_shared<ScalarType>("abc"));
+  auto r = Datum(std::make_shared<ScalarType>("zzz"));
+  ValidateBetween<StringType>("[]", l, r, "[]");
+  ValidateBetween<StringType>("[null]", l, r, "[null]");
+  ValidateBetween<StringType>(R"(["aaa", "aaaa", "ccc", "z"])", l, r,
+                              R"([false, false, true, true])");
+  ValidateBetween<StringType>(R"(["a", "aaaa", "c", "z"])", l, r,
+                              R"([false, false, true, true])");
+  ValidateBetween<StringType>(R"(["a", "aaaa", "fff", "zzzz"])", l, r,
+                              R"([false, false, true, false])");
+  ValidateBetween<StringType>(R"(["abd", null, null, "zzx"])", l, r,
+                              R"([true, null, null, true])");
+}
+
+TEST(TestSimpleBetweenKernel, SimpleTimestampTest) {

Review comment:
       Did you copy this from `TEST(TestCompareTimestamps, Basics)`?
   Do you want to add between kernel version of `TEST(TestCompareTimestamps, DifferentParameters)` too?

##########
File path: cpp/src/arrow/compute/kernels/codegen_internal.h
##########
@@ -583,7 +606,23 @@ struct OutputAdapter<Type, enable_if_base_binary<Type>> {
     return Status::NotImplemented("NYI");
   }
 };
+/*
+template <typename Type>
+struct OutputAdapter<Type, enable_if_base_binary<Type>> {
+  using T = typename TypeTraits<Type>::ScalarType::ValueType;
 
+  template <typename Generator>
+  static Status Write(KernelContext*, Datum* out, Generator&& generator) {
+    ArrayData* out_arr = out->mutable_array();
+    auto out_data = out_arr->GetMutableValues<T>(1);
+    // TODO: Is this as fast as a more explicitly inlined function?
+    for (int64_t i = 0; i < out_arr->length; ++i) {
+      *out_data++ = generator();
+    }
+    return Status::OK();
+  }
+};
+*/

Review comment:
       Should we remove this?

##########
File path: cpp/src/arrow/compute/kernels/scalar_compare_test.cc
##########
@@ -1850,5 +1851,154 @@ TEST(TestMaxElementWiseMinElementWise, CommonTemporal) {
               ResultWith(ScalarFromJSON(date64(), "86400000")));
 }
 
+template <typename ArrowType>
+static void ValidateBetween(const Datum& val, const Datum& lhs, const Datum& rhs,
+                            const Datum& expected) {
+  ASSERT_OK_AND_ASSIGN(Datum result, Between(val, lhs, rhs));
+  AssertArraysEqual(*expected.make_array(), *result.make_array(),
+                    /*verbose=*/true);
+}
+
+template <typename ArrowType>
+static void ValidateBetween(const char* value_str, const Datum& lhs, const Datum& rhs,
+                            const char* expected_str) {
+  auto value = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), value_str);
+  auto expected = ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), expected_str);
+  ValidateBetween<ArrowType>(value, lhs, rhs, expected);
+}
+
+template <>
+void ValidateBetween<StringType>(const char* value_str, const Datum& lhs,
+                                 const Datum& rhs, const char* expected_str) {
+  auto value = ArrayFromJSON(utf8(), value_str);
+  auto expected = ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), expected_str);
+  ValidateBetween<StringType>(value, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+class TestNumericBetweenKernel : public ::testing::Test {};
+
+TYPED_TEST_SUITE(TestNumericBetweenKernel, NumericArrowTypes);
+TYPED_TEST(TestNumericBetweenKernel, SimpleBetweenArrayScalarScalar) {
+  using ScalarType = typename TypeTraits<TypeParam>::ScalarType;
+  using CType = typename TypeTraits<TypeParam>::CType;
+
+  Datum zero(std::make_shared<ScalarType>(CType(0)));
+  Datum four(std::make_shared<ScalarType>(CType(4)));
+  ValidateBetween<TypeParam>("[]", zero, four, "[]");
+  ValidateBetween<TypeParam>("[null]", zero, four, "[null]");
+  ValidateBetween<TypeParam>("[0,0,1,1,2,2]", zero, four, "[0,0,1,1,1,1]");
+  ValidateBetween<TypeParam>("[0,1,2,3,4,5]", zero, four, "[0,1,1,1,0,0]");
+  ValidateBetween<TypeParam>("[5,4,3,2,1,0]", zero, four, "[0,0,1,1,1,0]");
+  ValidateBetween<TypeParam>("[null,0,1,1]", zero, four, "[null,0,1,1]");
+}
+
+TEST(TestSimpleBetweenKernel, SimpleStringTest) {

Review comment:
       Do we need `Simple`s here?
   Do you want to add more tests?

##########
File path: cpp/src/arrow/compute/kernels/scalar_compare_test.cc
##########
@@ -1850,5 +1851,154 @@ TEST(TestMaxElementWiseMinElementWise, CommonTemporal) {
               ResultWith(ScalarFromJSON(date64(), "86400000")));
 }
 
+template <typename ArrowType>
+static void ValidateBetween(const Datum& val, const Datum& lhs, const Datum& rhs,
+                            const Datum& expected) {
+  ASSERT_OK_AND_ASSIGN(Datum result, Between(val, lhs, rhs));
+  AssertArraysEqual(*expected.make_array(), *result.make_array(),
+                    /*verbose=*/true);
+}
+
+template <typename ArrowType>
+static void ValidateBetween(const char* value_str, const Datum& lhs, const Datum& rhs,
+                            const char* expected_str) {
+  auto value = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), value_str);
+  auto expected = ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), expected_str);
+  ValidateBetween<ArrowType>(value, lhs, rhs, expected);
+}
+
+template <>
+void ValidateBetween<StringType>(const char* value_str, const Datum& lhs,
+                                 const Datum& rhs, const char* expected_str) {
+  auto value = ArrayFromJSON(utf8(), value_str);
+  auto expected = ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), expected_str);
+  ValidateBetween<StringType>(value, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+class TestNumericBetweenKernel : public ::testing::Test {};
+
+TYPED_TEST_SUITE(TestNumericBetweenKernel, NumericArrowTypes);
+TYPED_TEST(TestNumericBetweenKernel, SimpleBetweenArrayScalarScalar) {
+  using ScalarType = typename TypeTraits<TypeParam>::ScalarType;
+  using CType = typename TypeTraits<TypeParam>::CType;
+
+  Datum zero(std::make_shared<ScalarType>(CType(0)));
+  Datum four(std::make_shared<ScalarType>(CType(4)));
+  ValidateBetween<TypeParam>("[]", zero, four, "[]");
+  ValidateBetween<TypeParam>("[null]", zero, four, "[null]");
+  ValidateBetween<TypeParam>("[0,0,1,1,2,2]", zero, four, "[0,0,1,1,1,1]");
+  ValidateBetween<TypeParam>("[0,1,2,3,4,5]", zero, four, "[0,1,1,1,0,0]");
+  ValidateBetween<TypeParam>("[5,4,3,2,1,0]", zero, four, "[0,0,1,1,1,0]");
+  ValidateBetween<TypeParam>("[null,0,1,1]", zero, four, "[null,0,1,1]");
+}
+
+TEST(TestSimpleBetweenKernel, SimpleStringTest) {
+  using ScalarType = typename TypeTraits<StringType>::ScalarType;
+  auto l = Datum(std::make_shared<ScalarType>("abc"));
+  auto r = Datum(std::make_shared<ScalarType>("zzz"));
+  ValidateBetween<StringType>("[]", l, r, "[]");
+  ValidateBetween<StringType>("[null]", l, r, "[null]");
+  ValidateBetween<StringType>(R"(["aaa", "aaaa", "ccc", "z"])", l, r,
+                              R"([false, false, true, true])");
+  ValidateBetween<StringType>(R"(["a", "aaaa", "c", "z"])", l, r,
+                              R"([false, false, true, true])");
+  ValidateBetween<StringType>(R"(["a", "aaaa", "fff", "zzzz"])", l, r,
+                              R"([false, false, true, false])");
+  ValidateBetween<StringType>(R"(["abd", null, null, "zzx"])", l, r,
+                              R"([true, null, null, true])");
+}
+
+TEST(TestSimpleBetweenKernel, SimpleTimestampTest) {
+  using ScalarType = typename TypeTraits<TimestampType>::ScalarType;
+  auto checkTimestampArray = [](std::shared_ptr<DataType> type, const char* input_str,
+                                const Datum& lhs, const Datum& rhs,
+                                const char* expected_str) {
+    auto value = ArrayFromJSON(type, input_str);
+    auto expected = ArrayFromJSON(boolean(), expected_str);
+    ValidateBetween<TimestampType>(value, lhs, rhs, expected);
+  };
+  auto unit = TimeUnit::SECOND;
+  auto l = Datum(std::make_shared<ScalarType>(923184000, timestamp(unit)));
+  auto r = Datum(std::make_shared<ScalarType>(1602032602, timestamp(unit)));
+  checkTimestampArray(timestamp(unit), "[]", l, r, "[]");
+  checkTimestampArray(timestamp(unit), "[null]", l, r, "[null]");
+  checkTimestampArray(timestamp(unit), R"(["1970-01-01","2000-02-29","1900-02-28"])", l,
+                      r, "[false,true,false]");
+  checkTimestampArray(timestamp(unit), R"(["1970-01-01","2000-02-29","2004-02-28"])", l,
+                      r, "[false,true,true]");
+  checkTimestampArray(timestamp(unit), R"(["2018-01-01","1999-04-04","1900-02-28"])", l,
+                      r, "[true,false,false]");
+}
+
+TYPED_TEST(TestNumericBetweenKernel, SimpleBetweenArrayArrayArray) {
+  ValidateBetween<TypeParam>(
+      "[]", ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"), "[]");
+  ValidateBetween<TypeParam>(
+      "[null]", ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"), "[null]");
+  ValidateBetween<TypeParam>(
+      "[1,1,2,2,2]",
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      "[true,true,false,false,false]");
+  ValidateBetween<TypeParam>(
+      "[1,1,2,2,2,2]",
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,2,5,5]"),
+      "[true,true,false,null,false,false]");
+}
+
+TEST(TestSimpleBetweenKernel, StringArrayArrayArrayTest) {
+  ValidateBetween<StringType>(
+      R"(["david","hello","world"])",
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["adam","hi","whirl"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                    R"(["robert","goeiemoreen","whirlwind"])"),
+      "[true, false, false]");
+  ValidateBetween<StringType>(
+      R"(["x","a","f"])",
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["w","a","e"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["z","a","g"])"),
+      "[true, false, true]");
+  ValidateBetween<StringType>(
+      R"(["block","bit","binary"])",
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                    R"(["bit","nibble","ternary"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["word","d","xyz"])"),
+      "[true, false, false]");
+  ValidateBetween<StringType>(R"(["Ayumi","アユミ","王梦莹"])",
+                              ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                            R"(["たなか","あゆみ","歩美"])"),
+                              ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                            R"(["李平之","田中","たなか"])"),
+                              "[false, true, false]");
+}

Review comment:
       Do we need to add `null` in `lhs` and/or `rhs` cases?

##########
File path: cpp/src/arrow/compute/kernels/scalar_compare_benchmark.cc
##########
@@ -77,5 +77,42 @@ BENCHMARK(GreaterArrayScalarInt64)->Apply(RegressionSetArgs);
 BENCHMARK(GreaterArrayArrayString)->Apply(RegressionSetArgs);
 BENCHMARK(GreaterArrayScalarString)->Apply(RegressionSetArgs);
 
+template <typename Type>
+static void BetweenScalarArrayScalar(benchmark::State& state) {
+  RegressionArgs args(state, /*size_is_bytes=*/false);
+  auto ty = TypeTraits<Type>::type_singleton();
+  auto rand = random::RandomArrayGenerator(kSeed);
+  auto array = rand.ArrayOf(ty, args.size, args.null_proportion);
+  auto scalar_left = *rand.ArrayOf(ty, 1, 0)->GetScalar(0);
+  auto scalar_right = *rand.ArrayOf(ty, 1, .0)->GetScalar(0);

Review comment:
       ```suggestion
     auto scalar_right = *rand.ArrayOf(ty, 1, 0)->GetScalar(0);
   ```

##########
File path: cpp/src/arrow/compute/kernels/scalar_compare_test.cc
##########
@@ -1850,5 +1851,154 @@ TEST(TestMaxElementWiseMinElementWise, CommonTemporal) {
               ResultWith(ScalarFromJSON(date64(), "86400000")));
 }
 
+template <typename ArrowType>
+static void ValidateBetween(const Datum& val, const Datum& lhs, const Datum& rhs,
+                            const Datum& expected) {
+  ASSERT_OK_AND_ASSIGN(Datum result, Between(val, lhs, rhs));
+  AssertArraysEqual(*expected.make_array(), *result.make_array(),
+                    /*verbose=*/true);
+}
+
+template <typename ArrowType>
+static void ValidateBetween(const char* value_str, const Datum& lhs, const Datum& rhs,
+                            const char* expected_str) {
+  auto value = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), value_str);
+  auto expected = ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), expected_str);
+  ValidateBetween<ArrowType>(value, lhs, rhs, expected);
+}
+
+template <>
+void ValidateBetween<StringType>(const char* value_str, const Datum& lhs,
+                                 const Datum& rhs, const char* expected_str) {
+  auto value = ArrayFromJSON(utf8(), value_str);
+  auto expected = ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), expected_str);
+  ValidateBetween<StringType>(value, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+class TestNumericBetweenKernel : public ::testing::Test {};
+
+TYPED_TEST_SUITE(TestNumericBetweenKernel, NumericArrowTypes);
+TYPED_TEST(TestNumericBetweenKernel, SimpleBetweenArrayScalarScalar) {
+  using ScalarType = typename TypeTraits<TypeParam>::ScalarType;
+  using CType = typename TypeTraits<TypeParam>::CType;
+
+  Datum zero(std::make_shared<ScalarType>(CType(0)));
+  Datum four(std::make_shared<ScalarType>(CType(4)));
+  ValidateBetween<TypeParam>("[]", zero, four, "[]");
+  ValidateBetween<TypeParam>("[null]", zero, four, "[null]");
+  ValidateBetween<TypeParam>("[0,0,1,1,2,2]", zero, four, "[0,0,1,1,1,1]");
+  ValidateBetween<TypeParam>("[0,1,2,3,4,5]", zero, four, "[0,1,1,1,0,0]");
+  ValidateBetween<TypeParam>("[5,4,3,2,1,0]", zero, four, "[0,0,1,1,1,0]");
+  ValidateBetween<TypeParam>("[null,0,1,1]", zero, four, "[null,0,1,1]");
+}
+
+TEST(TestSimpleBetweenKernel, SimpleStringTest) {
+  using ScalarType = typename TypeTraits<StringType>::ScalarType;
+  auto l = Datum(std::make_shared<ScalarType>("abc"));
+  auto r = Datum(std::make_shared<ScalarType>("zzz"));
+  ValidateBetween<StringType>("[]", l, r, "[]");
+  ValidateBetween<StringType>("[null]", l, r, "[null]");
+  ValidateBetween<StringType>(R"(["aaa", "aaaa", "ccc", "z"])", l, r,
+                              R"([false, false, true, true])");
+  ValidateBetween<StringType>(R"(["a", "aaaa", "c", "z"])", l, r,
+                              R"([false, false, true, true])");
+  ValidateBetween<StringType>(R"(["a", "aaaa", "fff", "zzzz"])", l, r,
+                              R"([false, false, true, false])");
+  ValidateBetween<StringType>(R"(["abd", null, null, "zzx"])", l, r,
+                              R"([true, null, null, true])");
+}
+
+TEST(TestSimpleBetweenKernel, SimpleTimestampTest) {
+  using ScalarType = typename TypeTraits<TimestampType>::ScalarType;
+  auto checkTimestampArray = [](std::shared_ptr<DataType> type, const char* input_str,
+                                const Datum& lhs, const Datum& rhs,
+                                const char* expected_str) {
+    auto value = ArrayFromJSON(type, input_str);
+    auto expected = ArrayFromJSON(boolean(), expected_str);
+    ValidateBetween<TimestampType>(value, lhs, rhs, expected);
+  };
+  auto unit = TimeUnit::SECOND;
+  auto l = Datum(std::make_shared<ScalarType>(923184000, timestamp(unit)));
+  auto r = Datum(std::make_shared<ScalarType>(1602032602, timestamp(unit)));
+  checkTimestampArray(timestamp(unit), "[]", l, r, "[]");
+  checkTimestampArray(timestamp(unit), "[null]", l, r, "[null]");
+  checkTimestampArray(timestamp(unit), R"(["1970-01-01","2000-02-29","1900-02-28"])", l,
+                      r, "[false,true,false]");
+  checkTimestampArray(timestamp(unit), R"(["1970-01-01","2000-02-29","2004-02-28"])", l,
+                      r, "[false,true,true]");
+  checkTimestampArray(timestamp(unit), R"(["2018-01-01","1999-04-04","1900-02-28"])", l,
+                      r, "[true,false,false]");
+}
+
+TYPED_TEST(TestNumericBetweenKernel, SimpleBetweenArrayArrayArray) {

Review comment:
       Could you move this to right after `TYPED_TEST(TestNumericBetweenKernel, SimpleBetweenArrayScalarScalar)`?

##########
File path: cpp/src/arrow/compute/kernels/codegen_internal.h
##########
@@ -982,6 +1021,407 @@ template <typename OutType, typename ArgType, typename Op>
 using ScalarBinaryNotNullStatefulEqualTypes =
     ScalarBinaryNotNullStateful<OutType, ArgType, ArgType, Op>;
 
+// A kernel exec generator for ternary functions that addresses both array and
+// scalar inputs and dispatches input iteration and output writing to other
+// templates
+//
+// This template executes the operator even on the data behind null values,
+// therefore it is generally only suitable for operators that are safe to apply
+// even on the null slot values.
+//
+// The "Op" functor should have the form
+//
+// struct Op {
+//   template <typename OutValue, typename Arg0Value, typename Arg1Value, typename
+//   Arg2Value> static OutValue Call(KernelContext* ctx, Arg0Value arg0, Arg1Value arg1,
+//   Arg2Value arg2, Status *st) {

Review comment:
       Could you format this?
   This is difficult to read.

##########
File path: cpp/src/arrow/compute/kernels/scalar_compare_test.cc
##########
@@ -1850,5 +1851,154 @@ TEST(TestMaxElementWiseMinElementWise, CommonTemporal) {
               ResultWith(ScalarFromJSON(date64(), "86400000")));
 }
 
+template <typename ArrowType>
+static void ValidateBetween(const Datum& val, const Datum& lhs, const Datum& rhs,
+                            const Datum& expected) {
+  ASSERT_OK_AND_ASSIGN(Datum result, Between(val, lhs, rhs));
+  AssertArraysEqual(*expected.make_array(), *result.make_array(),
+                    /*verbose=*/true);
+}
+
+template <typename ArrowType>
+static void ValidateBetween(const char* value_str, const Datum& lhs, const Datum& rhs,
+                            const char* expected_str) {
+  auto value = ArrayFromJSON(TypeTraits<ArrowType>::type_singleton(), value_str);
+  auto expected = ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), expected_str);
+  ValidateBetween<ArrowType>(value, lhs, rhs, expected);
+}
+
+template <>
+void ValidateBetween<StringType>(const char* value_str, const Datum& lhs,
+                                 const Datum& rhs, const char* expected_str) {
+  auto value = ArrayFromJSON(utf8(), value_str);
+  auto expected = ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), expected_str);
+  ValidateBetween<StringType>(value, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+class TestNumericBetweenKernel : public ::testing::Test {};
+
+TYPED_TEST_SUITE(TestNumericBetweenKernel, NumericArrowTypes);
+TYPED_TEST(TestNumericBetweenKernel, SimpleBetweenArrayScalarScalar) {
+  using ScalarType = typename TypeTraits<TypeParam>::ScalarType;
+  using CType = typename TypeTraits<TypeParam>::CType;
+
+  Datum zero(std::make_shared<ScalarType>(CType(0)));
+  Datum four(std::make_shared<ScalarType>(CType(4)));
+  ValidateBetween<TypeParam>("[]", zero, four, "[]");
+  ValidateBetween<TypeParam>("[null]", zero, four, "[null]");
+  ValidateBetween<TypeParam>("[0,0,1,1,2,2]", zero, four, "[0,0,1,1,1,1]");
+  ValidateBetween<TypeParam>("[0,1,2,3,4,5]", zero, four, "[0,1,1,1,0,0]");
+  ValidateBetween<TypeParam>("[5,4,3,2,1,0]", zero, four, "[0,0,1,1,1,0]");
+  ValidateBetween<TypeParam>("[null,0,1,1]", zero, four, "[null,0,1,1]");
+}
+
+TEST(TestSimpleBetweenKernel, SimpleStringTest) {
+  using ScalarType = typename TypeTraits<StringType>::ScalarType;
+  auto l = Datum(std::make_shared<ScalarType>("abc"));
+  auto r = Datum(std::make_shared<ScalarType>("zzz"));
+  ValidateBetween<StringType>("[]", l, r, "[]");
+  ValidateBetween<StringType>("[null]", l, r, "[null]");
+  ValidateBetween<StringType>(R"(["aaa", "aaaa", "ccc", "z"])", l, r,
+                              R"([false, false, true, true])");
+  ValidateBetween<StringType>(R"(["a", "aaaa", "c", "z"])", l, r,
+                              R"([false, false, true, true])");
+  ValidateBetween<StringType>(R"(["a", "aaaa", "fff", "zzzz"])", l, r,
+                              R"([false, false, true, false])");
+  ValidateBetween<StringType>(R"(["abd", null, null, "zzx"])", l, r,
+                              R"([true, null, null, true])");
+}
+
+TEST(TestSimpleBetweenKernel, SimpleTimestampTest) {
+  using ScalarType = typename TypeTraits<TimestampType>::ScalarType;
+  auto checkTimestampArray = [](std::shared_ptr<DataType> type, const char* input_str,
+                                const Datum& lhs, const Datum& rhs,
+                                const char* expected_str) {
+    auto value = ArrayFromJSON(type, input_str);
+    auto expected = ArrayFromJSON(boolean(), expected_str);
+    ValidateBetween<TimestampType>(value, lhs, rhs, expected);
+  };
+  auto unit = TimeUnit::SECOND;
+  auto l = Datum(std::make_shared<ScalarType>(923184000, timestamp(unit)));
+  auto r = Datum(std::make_shared<ScalarType>(1602032602, timestamp(unit)));
+  checkTimestampArray(timestamp(unit), "[]", l, r, "[]");
+  checkTimestampArray(timestamp(unit), "[null]", l, r, "[null]");
+  checkTimestampArray(timestamp(unit), R"(["1970-01-01","2000-02-29","1900-02-28"])", l,
+                      r, "[false,true,false]");
+  checkTimestampArray(timestamp(unit), R"(["1970-01-01","2000-02-29","2004-02-28"])", l,
+                      r, "[false,true,true]");
+  checkTimestampArray(timestamp(unit), R"(["2018-01-01","1999-04-04","1900-02-28"])", l,
+                      r, "[true,false,false]");
+}
+
+TYPED_TEST(TestNumericBetweenKernel, SimpleBetweenArrayArrayArray) {
+  ValidateBetween<TypeParam>(
+      "[]", ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"), "[]");
+  ValidateBetween<TypeParam>(
+      "[null]", ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"), "[null]");
+  ValidateBetween<TypeParam>(
+      "[1,1,2,2,2]",
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      "[true,true,false,false,false]");
+  ValidateBetween<TypeParam>(
+      "[1,1,2,2,2,2]",
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,2,5,5]"),
+      "[true,true,false,null,false,false]");
+}
+
+TEST(TestSimpleBetweenKernel, StringArrayArrayArrayTest) {
+  ValidateBetween<StringType>(
+      R"(["david","hello","world"])",
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["adam","hi","whirl"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                    R"(["robert","goeiemoreen","whirlwind"])"),
+      "[true, false, false]");
+  ValidateBetween<StringType>(
+      R"(["x","a","f"])",
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["w","a","e"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["z","a","g"])"),
+      "[true, false, true]");
+  ValidateBetween<StringType>(
+      R"(["block","bit","binary"])",
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                    R"(["bit","nibble","ternary"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["word","d","xyz"])"),
+      "[true, false, false]");
+  ValidateBetween<StringType>(R"(["Ayumi","アユミ","王梦莹"])",
+                              ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                            R"(["たなか","あゆみ","歩美"])"),
+                              ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                            R"(["李平之","田中","たなか"])"),
+                              "[false, true, false]");
+}
+
+TEST(TestSimpleBetweenKernel, TimestampArrayArrayArrayTest) {

Review comment:
       Do we also need different time unit case?

##########
File path: cpp/src/arrow/compute/kernels/codegen_internal.h
##########
@@ -982,6 +1021,407 @@ template <typename OutType, typename ArgType, typename Op>
 using ScalarBinaryNotNullStatefulEqualTypes =
     ScalarBinaryNotNullStateful<OutType, ArgType, ArgType, Op>;
 
+// A kernel exec generator for ternary functions that addresses both array and
+// scalar inputs and dispatches input iteration and output writing to other
+// templates
+//
+// This template executes the operator even on the data behind null values,
+// therefore it is generally only suitable for operators that are safe to apply
+// even on the null slot values.
+//
+// The "Op" functor should have the form
+//
+// struct Op {
+//   template <typename OutValue, typename Arg0Value, typename Arg1Value, typename
+//   Arg2Value> static OutValue Call(KernelContext* ctx, Arg0Value arg0, Arg1Value arg1,
+//   Arg2Value arg2, Status *st) {
+//     // implementation
+//     // NOTE: "status" should only be populated with errors,
+//     //       leave it unmodified to indicate Status::OK()
+//   }
+// };
+template <typename OutType, typename Arg0Type, typename Arg1Type, typename Arg2Type,
+          typename Op>
+struct ScalarTernary {
+  using OutValue = typename GetOutputType<OutType>::T;
+  using Arg0Value = typename GetViewType<Arg0Type>::T;
+  using Arg1Value = typename GetViewType<Arg1Type>::T;
+  using Arg2Value = typename GetViewType<Arg2Type>::T;
+
+  static Status ArrayArrayArray(KernelContext* ctx, const ArrayData& arg0,
+                                const ArrayData& arg1, const ArrayData& arg2,
+                                Datum* out) {
+    Status st = Status::OK();
+    ArrayIterator<Arg0Type> arg0_it(arg0);
+    ArrayIterator<Arg1Type> arg1_it(arg1);
+    ArrayIterator<Arg2Type> arg2_it(arg2);
+    RETURN_NOT_OK(OutputAdapter<OutType>::Write(ctx, out, [&]() -> OutValue {
+      return Op::template Call<OutValue, Arg0Value, Arg1Value, Arg2Value>(
+          ctx, arg0_it(), arg1_it(), arg2_it(), &st);
+    }));
+    return st;
+  }
+
+  static Status ArrayArrayScalar(KernelContext* ctx, const ArrayData& arg0,
+                                 const ArrayData& arg1, const Scalar& arg2, Datum* out) {
+    Status st = Status::OK();
+    ArrayIterator<Arg0Type> arg0_it(arg0);
+    ArrayIterator<Arg1Type> arg1_it(arg1);
+    auto arg2_val = UnboxScalar<Arg2Type>::Unbox(arg2);
+    RETURN_NOT_OK(OutputAdapter<OutType>::Write(ctx, out, [&]() -> OutValue {
+      return Op::template Call<OutValue, Arg0Value, Arg1Value, Arg2Value>(
+          ctx, arg0_it(), arg1_it(), arg2_val, &st);
+    }));
+    return st;
+  }
+
+  static Status ArrayScalarArray(KernelContext* ctx, const ArrayData& arg0,
+                                 const Scalar& arg1, const ArrayData& arg2, Datum* out) {
+    Status st = Status::OK();
+    ArrayIterator<Arg0Type> arg0_it(arg0);
+    auto arg1_val = UnboxScalar<Arg1Type>::Unbox(arg1);
+    ArrayIterator<Arg2Type> arg2_it(arg2);
+    RETURN_NOT_OK(OutputAdapter<OutType>::Write(ctx, out, [&]() -> OutValue {
+      return Op::template Call<OutValue, Arg0Value, Arg1Value, Arg2Value>(
+          ctx, arg0_it(), arg1_val, arg2_it(), &st);
+    }));
+    return st;
+  }
+
+  static Status ScalarArrayArray(KernelContext* ctx, const Scalar& arg0,
+                                 const ArrayData& arg1, const ArrayData& arg2,
+                                 Datum* out) {
+    Status st = Status::OK();
+    auto arg0_val = UnboxScalar<Arg0Type>::Unbox(arg0);
+    ArrayIterator<Arg1Type> arg1_it(arg1);
+    ArrayIterator<Arg2Type> arg2_it(arg2);
+    RETURN_NOT_OK(OutputAdapter<OutType>::Write(ctx, out, [&]() -> OutValue {
+      return Op::template Call<OutValue, Arg0Value, Arg1Value, Arg2Value>(
+          ctx, arg0_val, arg1_it(), arg2_it(), &st);
+    }));
+    return st;
+  }
+
+  static Status ArrayScalarScalar(KernelContext* ctx, const ArrayData& arg0,
+                                  const Scalar& arg1, const Scalar& arg2, Datum* out) {
+    Status st = Status::OK();
+    ArrayIterator<Arg0Type> arg0_it(arg0);
+    auto arg1_val = UnboxScalar<Arg1Type>::Unbox(arg1);
+    auto arg2_val = UnboxScalar<Arg2Type>::Unbox(arg2);
+    RETURN_NOT_OK(OutputAdapter<OutType>::Write(ctx, out, [&]() -> OutValue {
+      return Op::template Call<OutValue, Arg0Value, Arg1Value, Arg2Value>(
+          ctx, arg0_it(), arg1_val, arg2_val, &st);
+    }));
+    return st;
+  }
+
+  static Status ScalarScalarArray(KernelContext* ctx, const Scalar& arg0,
+                                  const Scalar& arg1, const ArrayData& arg2, Datum* out) {
+    Status st = Status::OK();
+    auto arg0_val = UnboxScalar<Arg0Type>::Unbox(arg0);
+    auto arg1_val = UnboxScalar<Arg1Type>::Unbox(arg1);
+    ArrayIterator<Arg2Type> arg2_it(arg2);
+    RETURN_NOT_OK(OutputAdapter<OutType>::Write(ctx, out, [&]() -> OutValue {
+      return Op::template Call<OutValue, Arg0Value, Arg1Value, Arg2Value>(
+          ctx, arg0_val, arg1_val, arg2_it(), &st);
+    }));
+    return st;
+  }
+
+  static Status ScalarArrayScalar(KernelContext* ctx, const Scalar& arg0,
+                                  const ArrayData& arg1, const Scalar& arg2, Datum* out) {
+    Status st = Status::OK();
+    auto arg0_val = UnboxScalar<Arg0Type>::Unbox(arg0);
+    ArrayIterator<Arg1Type> arg1_it(arg1);
+    auto arg2_val = UnboxScalar<Arg2Type>::Unbox(arg2);
+    RETURN_NOT_OK(OutputAdapter<OutType>::Write(ctx, out, [&]() -> OutValue {
+      return Op::template Call<OutValue, Arg0Value, Arg1Value, Arg2Value>(
+          ctx, arg0_val, arg1_it(), arg2_val, &st);
+    }));
+    return st;
+  }
+
+  static Status ScalarScalarScalar(KernelContext* ctx, const Scalar& arg0,
+                                   const Scalar& arg1, const Scalar& arg2, Datum* out) {
+    Status st = Status::OK();
+    if (out->scalar()->is_valid) {
+      auto arg0_val = UnboxScalar<Arg0Type>::Unbox(arg0);
+      auto arg1_val = UnboxScalar<Arg1Type>::Unbox(arg1);
+      auto arg2_val = UnboxScalar<Arg2Type>::Unbox(arg2);
+      BoxScalar<OutType>::Box(
+          Op::template Call<OutValue, Arg0Value, Arg1Value, Arg2Value>(
+              ctx, arg0_val, arg1_val, arg2_val, &st),
+          out->scalar().get());
+    }
+    return st;
+  }
+
+  static Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    if (batch[0].kind() == Datum::ARRAY) {
+      if (batch[1].kind() == Datum::ARRAY) {
+        if (batch[2].kind() == Datum::ARRAY) {
+          return ArrayArrayArray(ctx, *batch[0].array(), *batch[1].array(),
+                                 *batch[2].array(), out);
+        } else {
+          return ArrayArrayScalar(ctx, *batch[0].array(), *batch[1].array(),
+                                  *batch[2].scalar(), out);
+        }
+      } else {
+        if (batch[2].kind() == Datum::ARRAY) {
+          return ArrayScalarArray(ctx, *batch[0].array(), *batch[1].scalar(),
+                                  *batch[2].array(), out);
+        } else {
+          return ArrayScalarScalar(ctx, *batch[0].array(), *batch[1].scalar(),
+                                   *batch[2].scalar(), out);
+        }
+      }
+    } else {
+      if (batch[1].kind() == Datum::ARRAY) {
+        if (batch[2].kind() == Datum::ARRAY) {
+          return ScalarArrayArray(ctx, *batch[0].scalar(), *batch[1].array(),
+                                  *batch[2].array(), out);
+        } else {
+          return ScalarArrayScalar(ctx, *batch[0].scalar(), *batch[1].array(),
+                                   *batch[2].scalar(), out);
+        }
+      } else {
+        if (batch[2].kind() == Datum::ARRAY) {
+          return ScalarScalarArray(ctx, *batch[0].scalar(), *batch[1].scalar(),
+                                   *batch[2].array(), out);
+        } else {
+          return ScalarScalarScalar(ctx, *batch[0].scalar(), *batch[1].scalar(),
+                                    *batch[2].scalar(), out);
+        }
+      }
+    }
+  }
+};
+
+// An alternative to ScalarTernary 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 Arg2Type,
+          typename Op>
+struct ScalarTernaryNotNullStateful {

Review comment:
       It seems that this is not used in this pull request?
   Should we remove this or use this (by adding support for decimal types)?

##########
File path: cpp/src/arrow/util/bit_block_counter.h
##########
@@ -586,5 +961,144 @@ static void VisitTwoBitBlocksVoid(const std::shared_ptr<Buffer>& left_bitmap_buf
   }
 }
 
+template <typename VisitNotNull, typename VisitNull>
+static void VisitThreeBitBlocksVoid(
+    const std::shared_ptr<Buffer>& left_bitmap_buf, int64_t left_offset,
+    const std::shared_ptr<Buffer>& mid_bitmap_buf, int64_t mid_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) && (mid_bitmap_buf == NULLPTR)) ||
+      ((mid_bitmap_buf == NULLPTR) && (right_bitmap_buf == NULLPTR)) ||
+      ((left_bitmap_buf == NULLPTR) && (right_bitmap_buf == NULLPTR))) {
+    // At most one bitmap is present
+    if ((left_bitmap_buf == NULLPTR) && (mid_bitmap_buf == NULLPTR)) {
+      return VisitBitBlocksVoid(right_bitmap_buf, right_offset, length,
+                                std::forward<VisitNotNull>(visit_not_null),
+                                std::forward<VisitNull>(visit_null));
+    } else if ((mid_bitmap_buf == NULLPTR) && (right_bitmap_buf == NULLPTR)) {
+      return VisitBitBlocksVoid(left_bitmap_buf, left_offset, length,
+                                std::forward<VisitNotNull>(visit_not_null),
+                                std::forward<VisitNull>(visit_null));
+    } else {
+      return VisitBitBlocksVoid(mid_bitmap_buf, mid_offset, length,
+                                std::forward<VisitNotNull>(visit_not_null),
+                                std::forward<VisitNull>(visit_null));
+    }
+  }
+  // Two bitmaps are present
+  if (left_bitmap_buf == NULLPTR) {
+    const uint8_t* mid_bitmap = mid_bitmap_buf->data();
+    const uint8_t* right_bitmap = right_bitmap_buf->data();
+    BinaryBitBlockCounter bit_counter(mid_bitmap, mid_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 (bit_util::GetBit(mid_bitmap, mid_offset + position) &&
+              bit_util::GetBit(right_bitmap, right_offset + position)) {
+            visit_not_null(position);
+          } else {
+            visit_null();
+          }
+        }
+      }
+    }
+  } else if (mid_bitmap_buf == NULLPTR) {
+    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 (bit_util::GetBit(left_bitmap, left_offset + position) &&
+              bit_util::GetBit(right_bitmap, right_offset + position)) {
+            visit_not_null(position);
+          } else {
+            visit_null();
+          }
+        }
+      }
+    }
+  } else {
+    const uint8_t* left_bitmap = left_bitmap_buf->data();

Review comment:
       Is this clause correct?
   It seems that `right_bitmap_buf` may be ignored.




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

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

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



[GitHub] [arrow] bkmgit commented on a change in pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
bkmgit commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r780780625



##########
File path: cpp/src/arrow/compute/kernels/scalar_compare.cc
##########
@@ -746,6 +787,111 @@ std::shared_ptr<ScalarFunction> MakeScalarMinMax(std::string name,
   return func;
 }
 
+template <typename Op>
+void AddIntegerBetween(const std::shared_ptr<DataType>& ty, ScalarFunction* func) {
+  auto exec = GeneratePhysicalInteger<ScalarTernaryEqualTypes, BooleanType, Op>(*ty);
+  DCHECK_OK(func->AddKernel({ty, ty, ty}, boolean(), std::move(exec)));
+}
+
+template <typename InType, typename Op>
+void AddGenericBetween(const std::shared_ptr<DataType>& ty, ScalarFunction* func) {
+  DCHECK_OK(func->AddKernel({ty, ty, ty}, boolean(),
+                            ScalarTernaryEqualTypes<BooleanType, InType, Op>::Exec));
+}
+
+template <typename OutType, typename ArgType, typename Op>
+struct BetweenTimestamps : public ScalarTernaryEqualTypes<OutType, ArgType, Op> {
+  using Base = ScalarTernaryEqualTypes<OutType, ArgType, Op>;
+
+  static Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    const auto& var = checked_cast<const TimestampType&>(*batch[0].type());
+    const auto& lhs = checked_cast<const TimestampType&>(*batch[1].type());
+    const auto& rhs = checked_cast<const TimestampType&>(*batch[2].type());
+    if ((var.timezone().empty() != lhs.timezone().empty()) ||
+        (var.timezone().empty() != rhs.timezone().empty()) ||
+        (lhs.timezone().empty() != rhs.timezone().empty())) {
+      return Status::Invalid(
+          "Cannot use timestamps with timezone and timestamps without timezones, got: ",
+          var.timezone().empty(), " ", lhs.timezone().empty(), " and ",
+          rhs.timezone().empty());
+    }
+    return Base::Exec(ctx, batch, out);
+  }
+};
+
+template <typename Op>
+std::shared_ptr<ScalarFunction> MakeBetweenFunction(std::string name,
+                                                    const FunctionDoc* doc) {
+  auto func = std::make_shared<CompareFunction>(name, Arity::Ternary(), doc);
+
+  DCHECK_OK(func->AddKernel(
+      {boolean(), boolean(), boolean()}, boolean(),
+      ScalarTernary<BooleanType, BooleanType, BooleanType, BooleanType, Op>::Exec));
+
+  for (const std::shared_ptr<DataType>& ty : IntTypes()) {
+    AddIntegerBetween<Op>(ty, func.get());
+  }
+
+  AddIntegerBetween<Op>(date32(), func.get());
+  AddIntegerBetween<Op>(date64(), func.get());
+
+  AddGenericBetween<FloatType, Op>(float32(), func.get());
+  AddGenericBetween<DoubleType, Op>(float64(), func.get());
+
+  // Add timestamp kernels
+  for (auto unit : TimeUnit::values()) {
+    InputType in_type(match::TimestampTypeUnit(unit));
+    DCHECK_OK(func->AddKernel({in_type, in_type, in_type}, boolean(),
+                              BetweenTimestamps<BooleanType, TimestampType, Op>::Exec));
+  }
+
+  // Duration
+  for (auto unit : TimeUnit::values()) {
+    InputType in_type(match::DurationTypeUnit(unit));
+    auto exec =
+        GeneratePhysicalInteger<ScalarTernaryEqualTypes, BooleanType, Op>(int64());
+    DCHECK_OK(func->AddKernel({in_type, in_type, in_type}, boolean(), std::move(exec)));
+  }
+
+  // Time32 and Time64
+  for (auto unit : {TimeUnit::SECOND, TimeUnit::MILLI}) {
+    InputType in_type(match::Time32TypeUnit(unit));
+    auto exec =
+        GeneratePhysicalInteger<ScalarTernaryEqualTypes, BooleanType, Op>(int32());
+    DCHECK_OK(func->AddKernel({in_type, in_type, in_type}, boolean(), std::move(exec)));
+  }
+  for (auto unit : {TimeUnit::MICRO, TimeUnit::NANO}) {
+    InputType in_type(match::Time64TypeUnit(unit));
+    auto exec =
+        GeneratePhysicalInteger<ScalarTernaryEqualTypes, BooleanType, Op>(int64());
+    DCHECK_OK(func->AddKernel({in_type, in_type, in_type}, boolean(), std::move(exec)));
+  }
+
+  for (const std::shared_ptr<DataType>& ty : BaseBinaryTypes()) {
+    auto exec = GenerateVarBinaryBase<ScalarTernaryEqualTypes, BooleanType, Op>(*ty);
+    DCHECK_OK(func->AddKernel(
+        {InputType::Array(ty), InputType::Scalar(ty), InputType::Scalar(ty)}, boolean(),
+        exec));
+    DCHECK_OK(func->AddKernel(
+        {InputType::Array(ty), InputType::Array(ty), InputType::Array(ty)}, boolean(),
+        std::move(exec)));
+  }

Review comment:
       Thanks for adding these.




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

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

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



[GitHub] [arrow] edponce edited a comment on pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
edponce edited a comment on pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#issuecomment-1008355611


   @bkmgit It is a team effort 👍🏾! For Python, you would only need to add bindings for `BetweenOptions` because the compute function is "binded" automatically (`call_function` mechanism). I suggest that you [`grep -R RoundOptions arrow/python/pyarrow/`](https://github.com/apache/arrow/tree/master/python/pyarrow) so you can see what are the things needed in Python to provide full support for `between` function. The following are main actions to complete for Python bindings:
   * [Cython header binding for options `pyarrow/includes/libarrow.pxd`](https://github.com/apache/arrow/blob/master/python/pyarrow/includes/libarrow.pxd#L1866-L1893) - Note that the enum is in global scope because it is used by `Round` and `RoundToMultiple`. In this case, `BetweenOptions` will be used by `Between` and `NotBetween`, so maybe it is wise in C++ to put `BetweenOptions` in global space now instead of in `NotBetween` PR.
   * [Cython/Python definitions for options `pyarrow/_compute.pyx`](https://github.com/apache/arrow/blob/master/python/pyarrow/_compute.pyx#L743-L776)
   * [Python test for `BetweenOptions`](https://github.com/apache/arrow/blob/master/python/pyarrow/tests/test_compute.py#L153)
   * [Python tests for compute function](https://github.com/apache/arrow/blob/master/python/pyarrow/tests/test_compute.py#L1409-L1472)
   * [Add function to Python docs](https://github.com/apache/arrow/blob/master/docs/source/python/api/compute.rst#comparisons)


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

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

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



[GitHub] [arrow] bkmgit commented on a change in pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
bkmgit commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r785343098



##########
File path: dev/tasks/java-jars/README.md
##########
@@ -16,7 +16,7 @@ See the License for the specific language governing permissions and
 limitations under the License.
 -->
 
-# Java Jars Task
+# Jars.

Review comment:
       Fixed poor rebase




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

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

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



[GitHub] [arrow] bkmgit commented on pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
bkmgit commented on pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#issuecomment-1013735426


   @github-actions autotune


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

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

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



[GitHub] [arrow] bkmgit commented on a change in pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
bkmgit commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r782688014



##########
File path: python/pyarrow/tests/test_compute.py
##########
@@ -1262,6 +1263,300 @@ def test_filter_null_type():
     assert len(table.filter(mask).column(0)) == 5
 
 
+@pytest.mark.parametrize("ty", ["inclusive"])
+def test_between_array_array_array(ty):
+    BetweenOptions = partial(pc.BetweenOptions)
+
+    val = pa.array([1, 1, 4, 3, 2, 6])
+    arr1 = pa.array([1, 1, 3, 4, None, 5])
+    arr2 = pa.array([1, 2, 4, None, 4, 7])
+
+    inclusive_and_expected = {
+        "both": [True, True, True, None, None, True],
+        "left": [False, True, False, None, None, True],
+        "right": [False, False, True, None, None, True],
+        "neither": [False, False, False, None, None, True],
+    }
+
+    for inclusive, expected in inclusive_and_expected.items():
+        options = BetweenOptions(inclusive=inclusive)
+        result = pc.between(val, arr1, arr2, options=options)
+        np.testing.assert_array_equal(result, pa.array(expected))

Review comment:
       Probably this is best done as a separate issue.




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

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

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



[GitHub] [arrow] bkmgit removed a comment on pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
bkmgit removed a comment on pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#issuecomment-1010646334


   @lidavidm Thanks for the feedback. Inclusive is in global space so that it can also be used in NotBetween - follow up issue https://issues.apache.org/jira/browse/ARROW-15223


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

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

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



[GitHub] [arrow] bkmgit commented on a change in pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
bkmgit commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r782784767



##########
File path: cpp/src/arrow/compute/kernels/scalar_compare_benchmark.cc
##########
@@ -77,5 +77,44 @@ BENCHMARK(GreaterArrayScalarInt64)->Apply(RegressionSetArgs);
 BENCHMARK(GreaterArrayArrayString)->Apply(RegressionSetArgs);
 BENCHMARK(GreaterArrayScalarString)->Apply(RegressionSetArgs);
 
+template <typename Type>
+static void BetweenScalarArrayScalar(benchmark::State& state) {
+  RegressionArgs args(state, /*size_is_bytes=*/false);
+  auto ty = TypeTraits<Type>::type_singleton();
+  auto rand = random::RandomArrayGenerator(kSeed);
+  auto array = rand.ArrayOf(ty, args.size, args.null_proportion);
+  auto scalar_left = *rand.ArrayOf(ty, 1, 0)->GetScalar(0);
+  auto scalar_right = *rand.ArrayOf(ty, 1, 0)->GetScalar(0);
+  for (auto _ : state) {
+    ABORT_NOT_OK(
+        CallFunction("between_less_equal_less_equal", {array, scalar_left, scalar_right})
+            .status());
+  }
+}
+
+template <typename Type>
+static void BetweenArrayArrayArray(benchmark::State& state) {
+  RegressionArgs args(state, /*size_is_bytes=*/false);
+  auto ty = TypeTraits<Type>::type_singleton();
+  auto rand = random::RandomArrayGenerator(kSeed);
+  auto lhs = rand.ArrayOf(ty, args.size, args.null_proportion);
+  auto mid = rand.ArrayOf(ty, args.size, args.null_proportion);
+  auto rhs = rand.ArrayOf(ty, args.size, args.null_proportion);
+  for (auto _ : state) {
+    ABORT_NOT_OK(CallFunction("between_less_equal_less_equal", {mid, lhs, rhs}).status());
+  }
+}
+
+// static void BetweenArrayArrayArrayInt64(benchmark::State& state) {
+//   BetweenArrayArrayArray<Int64Type>(state);
+// }
+//
+// static void BetweenScalarArrayScalarInt64(benchmark::State& state) {
+//   BetweenScalarArrayScalar<Int64Type>(state);
+// }
+
+// BENCHMARK(BetweenArrayArrayArrayInt64)->Apply(RegressionSetArgs);
+// BENCHMARK(BetweenScalarArrayScalarInt64)->Apply(RegressionSetArgs);

Review comment:
       Working now. Thanks.




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

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

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



[GitHub] [arrow] edponce commented on a change in pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
edponce commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r784211897



##########
File path: cpp/src/arrow/compute/kernels/scalar_compare_test.cc
##########
@@ -1865,5 +1861,636 @@ TEST(TestMaxElementWiseMinElementWise, CommonTemporal) {
               ResultWith(ScalarFromJSON(date64(), "86400000")));
 }
 
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const Datum& val, const Datum& lhs,
+                            const Datum& rhs, const Datum& expected) {
+  ASSERT_OK_AND_ASSIGN(Datum result, Between(val, lhs, rhs, options, nullptr));
+  AssertArraysEqual(*expected.make_array(), *result.make_array(),
+                    /*verbose=*/true);
+}
+
+template <typename ArrowType>
+void ValidateBetween(BetweenOptions options, const Datum& val, const Datum& lhs,
+                     const Datum& rhs) {
+  CompareOperator lhs_val;
+  CompareOperator val_rhs;
+  BetweenOptions::Inclusive include_endpoints = options.inclusive;
+
+  if (include_endpoints == BetweenOptions::Inclusive::NEITHER) {
+    lhs_val = LESS;
+    val_rhs = LESS;
+  } else if (include_endpoints == BetweenOptions::Inclusive::LEFT) {
+    lhs_val = LESS_EQUAL;
+    val_rhs = LESS;
+  } else if (include_endpoints == BetweenOptions::Inclusive::RIGHT) {
+    lhs_val = LESS;
+    val_rhs = LESS_EQUAL;
+  } else {
+    lhs_val = LESS_EQUAL;
+    val_rhs = LESS_EQUAL;
+  }
+
+  ASSERT_OK_AND_ASSIGN(Datum resultl,
+                       CallFunction(CompareOperatorToFunctionName(lhs_val), {lhs, val}));
+  ASSERT_OK_AND_ASSIGN(Datum resultr,
+                       CallFunction(CompareOperatorToFunctionName(val_rhs), {val, rhs}));
+  ASSERT_OK_AND_ASSIGN(Datum expected, CallFunction("and", {resultl, resultr}));
+
+  ValidateBetween<ArrowType>(options, val, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+class TestNumericBetweenKernel : public ::testing::Test {};
+
+TYPED_TEST_SUITE(TestNumericBetweenKernel, NumericArrowTypes);
+TYPED_TEST(TestNumericBetweenKernel, SimpleBetweenArrayScalarScalar) {
+  using ScalarType = typename TypeTraits<TypeParam>::ScalarType;
+  using CType = typename TypeTraits<TypeParam>::CType;
+
+  Datum zero(std::make_shared<ScalarType>(CType(0)));
+  Datum four(std::make_shared<ScalarType>(CType(4)));
+  Datum null(std::make_shared<ScalarType>());
+  BetweenOptions InclusiveBoth(BetweenOptions::Inclusive::BOTH);
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"), zero,
+      four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      zero, four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,1,2,2]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[1,1,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,3,4,5]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[1,1,1,1,1,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,1,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null,0,1,1]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), null, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, null,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  BetweenOptions InclusiveLeft(BetweenOptions::Inclusive::LEFT);
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"), zero,
+      four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      zero, four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,1,2,2]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[1,1,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,3,4,5]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[1,1,1,1,0,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,0,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null,0,1,1]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), null, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, null,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  BetweenOptions InclusiveRight(BetweenOptions::Inclusive::RIGHT);
+  ValidateBetween<TypeParam>(
+      InclusiveRight, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"), zero,
+      four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      zero, four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,1,2,2]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,0,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,3,4,5]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,1,1,1,1,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,1,1,1,1,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null,0,1,1]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null,0,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), null, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, null,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  BetweenOptions InclusiveNeither(BetweenOptions::Inclusive::NEITHER);
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      zero, four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      zero, four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,1,2,2]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,0,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,3,4,5]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,1,1,1,0,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,0,1,1,1,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null,0,1,1]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null,0,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), null, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, null,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+}
+
+TYPED_TEST(TestNumericBetweenKernel, SimpleBetweenScalarArrayArray) {
+  using ScalarType = typename TypeTraits<TypeParam>::ScalarType;
+  using CType = typename TypeTraits<TypeParam>::CType;
+
+  Datum one(std::make_shared<ScalarType>(CType(1)));
+
+  BetweenOptions InclusiveBoth(BetweenOptions::Inclusive::BOTH);
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, one, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,true,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,true,null,false,false]"));
+  BetweenOptions InclusiveLeft(BetweenOptions::Inclusive::LEFT);
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, one, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,true,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,true,null,false,false]"));
+  BetweenOptions InclusiveRight(BetweenOptions::Inclusive::RIGHT);
+  ValidateBetween<TypeParam>(
+      InclusiveRight, one, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,false,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,false,null,false,false]"));
+  BetweenOptions InclusiveNeither(BetweenOptions::Inclusive::NEITHER);
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, one, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,false,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,false,null,false,false]"));
+}
+
+TYPED_TEST(TestNumericBetweenKernel, SimpleBetweenArrayArrayArray) {
+  BetweenOptions InclusiveBoth(BetweenOptions::Inclusive::BOTH);
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[1,1,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,true,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,true,null,false,false]"));
+  BetweenOptions InclusiveLeft(BetweenOptions::Inclusive::LEFT);
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[1,1,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,false,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,false,null,false,false]"));
+  BetweenOptions InclusiveRight(BetweenOptions::Inclusive::RIGHT);
+  ValidateBetween<TypeParam>(
+      InclusiveRight, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[1,1,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,true,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,true,null,false,false]"));
+  BetweenOptions InclusiveNeither(BetweenOptions::Inclusive::NEITHER);
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[1,1,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,false,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,false,null,false,false]"));
+}
+
+template <typename Type>
+struct BetweenRandomNumeric {
+  static void Test(const std::shared_ptr<DataType>& type) {
+    using ScalarType = typename TypeTraits<Type>::ScalarType;
+    using CType = typename TypeTraits<Type>::CType;
+    auto rand = random::RandomArrayGenerator(0x5416447);
+    const int64_t length = 1000;
+    for (auto null_probability : {0.0, 0.01, 0.1, 0.25, 0.5, 1.0}) {
+      for (auto inclusive :
+           {BetweenOptions::Inclusive::BOTH, BetweenOptions::Inclusive::LEFT,
+            BetweenOptions::Inclusive::RIGHT, BetweenOptions::Inclusive::NEITHER}) {
+        auto data1 =
+            rand.Numeric<typename Type::PhysicalType>(length, 0, 100, null_probability);
+        auto data2 =
+            rand.Numeric<typename Type::PhysicalType>(length, 0, 100, null_probability);
+        auto data3 =
+            rand.Numeric<typename Type::PhysicalType>(length, 0, 100, null_probability);
+
+        // Create view of data as the type (e.g. timestamp)
+        auto array1 = Datum(*data1->View(type));
+        auto array2 = Datum(*data2->View(type));
+        auto array3 = Datum(*data3->View(type));
+        auto scalar1 = Datum(std::make_shared<ScalarType>(CType(10), type));
+        auto scalar2 = Datum(std::make_shared<ScalarType>(CType(30), type));
+        auto scalar3 = Datum(std::make_shared<ScalarType>(CType(50), type));
+        auto options = BetweenOptions(inclusive);
+        ValidateBetween<Type>(options, array1, scalar2, scalar3);
+        ValidateBetween<Type>(options, array1, array2, scalar3);
+        ValidateBetween<Type>(options, array1, array2, array3);
+        ValidateBetween<Type>(options, array1, scalar2, scalar3);
+        ValidateBetween<Type>(options, scalar1, array2, array3);
+        ValidateBetween<Type>(options, scalar1, array2, scalar3);
+        ValidateBetween<Type>(options, scalar1, scalar2, array3);
+        ValidateBetween<Type>(options, array1, scalar2, array3);
+      }
+    }
+  }
+};
+
+TEST(TestNumericBetweenKernel, BetweenPrimitiveRandomTests) {
+  TestRandomPrimitiveCTypes<BetweenRandomNumeric>();
+}
+
+class TestStringBetweenKernel : public ::testing::Test {};
+
+TEST(TestStringBetweenKernel, RandomBetween) {
+  using ScalarType = typename TypeTraits<StringType>::ScalarType;
+
+  auto rand = random::RandomArrayGenerator(0x5416447);
+  for (size_t i = 3; i < 10; i++) {
+    for (auto null_probability : {0.0, 0.01, 0.1, 0.25, 0.5, 1.0}) {
+      for (auto inclusive :
+           {BetweenOptions::Inclusive::BOTH, BetweenOptions::Inclusive::LEFT,
+            BetweenOptions::Inclusive::RIGHT, BetweenOptions::Inclusive::NEITHER}) {
+        const int64_t length = static_cast<int64_t>(1ULL << i);
+        auto array1 = Datum(rand.String(length, 0, 16, null_probability));
+        auto array2 = Datum(rand.String(length, 0, 16, null_probability));
+        auto array3 = Datum(rand.String(length, 0, 16, null_probability));
+        auto scalar1 = Datum(std::make_shared<ScalarType>("fupi"));
+        auto scalar2 = Datum(std::make_shared<ScalarType>("tupu"));
+        auto scalar3 = Datum(std::make_shared<ScalarType>("zito"));
+        auto options = BetweenOptions(inclusive);
+        ValidateBetween<StringType>(options, array1, scalar2, scalar3);
+        ValidateBetween<StringType>(options, scalar1, array2, scalar3);
+        ValidateBetween<StringType>(options, scalar1, scalar2, array3);
+        ValidateBetween<StringType>(options, scalar1, array2, array3);
+        ValidateBetween<StringType>(options, array1, scalar2, array3);
+        ValidateBetween<StringType>(options, array1, array2, scalar3);
+        ValidateBetween<StringType>(options, array1, array2, array3);
+      }
+    }
+  }
+}
+
+TEST(TestStringBetweenKernel, StringArrayScalarScalarTest) {
+  using ScalarType = typename TypeTraits<StringType>::ScalarType;
+  auto l = Datum(std::make_shared<ScalarType>("abc"));
+  auto r = Datum(std::make_shared<ScalarType>("zzz"));
+  BetweenOptions InclusiveOption(BetweenOptions::Inclusive::BOTH);
+  ValidateBetween<StringType>(
+      InclusiveOption, ArrayFromJSON(TypeTraits<StringType>::type_singleton(), "[]"), l,
+      r, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<StringType>(
+      InclusiveOption, ArrayFromJSON(TypeTraits<StringType>::type_singleton(), "[null]"),
+      l, r, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<StringType>(InclusiveOption,
+                              ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                            R"(["aaa", "aaaa", "ccc", "z"])"),
+                              l, r,
+                              ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                                            R"([false, false, true, true])"));
+  ValidateBetween<StringType>(InclusiveOption,
+                              ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                            R"(["a", "aaaa", "c", "z"])"),
+                              l, r,
+                              ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                                            R"([false, false, true, true])"));
+  ValidateBetween<StringType>(InclusiveOption,
+                              ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                            R"(["a", "aaaa", "fff", "zzzz"])"),
+                              l, r,
+                              ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                                            R"([false, false, true, false])"));
+  ValidateBetween<StringType>(InclusiveOption,
+                              ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                            R"(["abc", "baa", "fff", "zzz"])"),
+                              l, r,
+                              ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                                            R"([true, true, true, true])"));
+  ValidateBetween<StringType>(InclusiveOption,
+                              ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                            R"(["abd", null, null, "zzx"])"),
+                              l, r,
+                              ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                                            R"([true, null, null, true])"));
+}
+
+TEST(TestStringBetweenKernel, StringArrayArrayArrayTest) {
+  BetweenOptions InclusiveOption(BetweenOptions::Inclusive::BOTH);
+  ValidateBetween<StringType>(
+      InclusiveOption,
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                    R"(["david","hello","world"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["adam","hi","whirl"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                    R"(["robert","goeiemoreen","whirlwind"])"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[true, false, false]"));
+  ValidateBetween<StringType>(
+      InclusiveOption,
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["x","a","f"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["w","a","e"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["z","a","g"])"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[true, true, true]"));
+  ValidateBetween<StringType>(
+      InclusiveOption,
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                    R"(["block","bit","binary"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                    R"(["bit","nibble","ternary"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["word","d","xyz"])"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[true, false, false]"));
+  ValidateBetween<StringType>(
+      InclusiveOption,
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                    R"(["よしもと","の","ち"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["は","へ","あ"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["な","を","ち"])"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[false, false, true]"));
+  ValidateBetween<StringType>(
+      InclusiveOption,
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["A","ア","王"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["た","あ","歩"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["李","田",null])"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[false, true, null]"));
+  ValidateBetween<StringType>(
+      InclusiveOption,
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["Б",null,"Я"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["А","Ж","Щ"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["Д","Л","Ф"])"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[true, null, false]"));
+}
+
+TEST(TestTimestampsBetweenKernel, TimestampsArrayArrayArrayTest) {
+  const std::string arr_json = R"(["1970-01-01","2000-02-02","1900-02-28"])";
+  const std::string lhs_json = R"(["1970-01-01","2000-02-01","1900-02-28"])";
+  const std::string rhs_json = R"(["1970-01-02","2000-02-02","1900-02-28"])";
+  // Between Options
+  BetweenOptions both(BetweenOptions::Inclusive::BOTH);
+  BetweenOptions left(BetweenOptions::Inclusive::LEFT);
+  BetweenOptions right(BetweenOptions::Inclusive::RIGHT);
+  BetweenOptions neither(BetweenOptions::Inclusive::NEITHER);
+  // Same units should be fine
+  auto arr = ArrayFromJSON(timestamp(TimeUnit::SECOND), arr_json);
+  auto lhs = ArrayFromJSON(timestamp(TimeUnit::SECOND), lhs_json);
+  auto rhs = ArrayFromJSON(timestamp(TimeUnit::SECOND), rhs_json);
+  // BOTH
+  ValidateBetween<TimestampType>(both, arr, lhs, rhs);
+  // LEFT
+  ValidateBetween<TimestampType>(left, arr, lhs, rhs);
+  // RIGHT
+  ValidateBetween<TimestampType>(right, arr, lhs, rhs);
+  // NEITHER
+  ValidateBetween<TimestampType>(neither, arr, lhs, rhs);

Review comment:
       Nit: No need for these comments as the code already is clear enough.




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

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

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



[GitHub] [arrow] edponce commented on a change in pull request #11882: ARROW-9843: [C++] Implement Between ternary kernel

Posted by GitBox <gi...@apache.org>.
edponce commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r784341952



##########
File path: cpp/src/arrow/compute/kernels/scalar_compare_test.cc
##########
@@ -1865,5 +1861,636 @@ TEST(TestMaxElementWiseMinElementWise, CommonTemporal) {
               ResultWith(ScalarFromJSON(date64(), "86400000")));
 }
 
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const Datum& val, const Datum& lhs,
+                            const Datum& rhs, const Datum& expected) {
+  ASSERT_OK_AND_ASSIGN(Datum result, Between(val, lhs, rhs, options, nullptr));
+  AssertArraysEqual(*expected.make_array(), *result.make_array(),
+                    /*verbose=*/true);
+}
+
+template <typename ArrowType>
+void ValidateBetween(BetweenOptions options, const Datum& val, const Datum& lhs,
+                     const Datum& rhs) {
+  CompareOperator lhs_val;
+  CompareOperator val_rhs;
+  BetweenOptions::Inclusive include_endpoints = options.inclusive;
+
+  if (include_endpoints == BetweenOptions::Inclusive::NEITHER) {
+    lhs_val = LESS;
+    val_rhs = LESS;
+  } else if (include_endpoints == BetweenOptions::Inclusive::LEFT) {
+    lhs_val = LESS_EQUAL;
+    val_rhs = LESS;
+  } else if (include_endpoints == BetweenOptions::Inclusive::RIGHT) {
+    lhs_val = LESS;
+    val_rhs = LESS_EQUAL;
+  } else {
+    lhs_val = LESS_EQUAL;
+    val_rhs = LESS_EQUAL;
+  }
+
+  ASSERT_OK_AND_ASSIGN(Datum resultl,
+                       CallFunction(CompareOperatorToFunctionName(lhs_val), {lhs, val}));
+  ASSERT_OK_AND_ASSIGN(Datum resultr,
+                       CallFunction(CompareOperatorToFunctionName(val_rhs), {val, rhs}));
+  ASSERT_OK_AND_ASSIGN(Datum expected, CallFunction("and", {resultl, resultr}));
+
+  ValidateBetween<ArrowType>(options, val, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+class TestNumericBetweenKernel : public ::testing::Test {};
+
+TYPED_TEST_SUITE(TestNumericBetweenKernel, NumericArrowTypes);
+TYPED_TEST(TestNumericBetweenKernel, SimpleBetweenArrayScalarScalar) {
+  using ScalarType = typename TypeTraits<TypeParam>::ScalarType;
+  using CType = typename TypeTraits<TypeParam>::CType;
+
+  Datum zero(std::make_shared<ScalarType>(CType(0)));
+  Datum four(std::make_shared<ScalarType>(CType(4)));
+  Datum null(std::make_shared<ScalarType>());
+  BetweenOptions InclusiveBoth(BetweenOptions::Inclusive::BOTH);
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"), zero,
+      four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      zero, four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,1,2,2]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[1,1,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,3,4,5]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[1,1,1,1,1,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,1,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null,0,1,1]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), null, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, null,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  BetweenOptions InclusiveLeft(BetweenOptions::Inclusive::LEFT);
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"), zero,
+      four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      zero, four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,1,2,2]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[1,1,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,3,4,5]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[1,1,1,1,0,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,0,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null,0,1,1]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), null, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, null,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  BetweenOptions InclusiveRight(BetweenOptions::Inclusive::RIGHT);
+  ValidateBetween<TypeParam>(
+      InclusiveRight, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"), zero,
+      four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      zero, four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,1,2,2]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,0,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,3,4,5]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,1,1,1,1,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,1,1,1,1,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null,0,1,1]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null,0,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), null, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, null,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  BetweenOptions InclusiveNeither(BetweenOptions::Inclusive::NEITHER);
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      zero, four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      zero, four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,1,2,2]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,0,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,3,4,5]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,1,1,1,0,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,0,1,1,1,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null,0,1,1]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null,0,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), null, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, null,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[null,null,null,null,null,null]"));
+}
+
+TYPED_TEST(TestNumericBetweenKernel, SimpleBetweenScalarArrayArray) {
+  using ScalarType = typename TypeTraits<TypeParam>::ScalarType;
+  using CType = typename TypeTraits<TypeParam>::CType;
+
+  Datum one(std::make_shared<ScalarType>(CType(1)));
+
+  BetweenOptions InclusiveBoth(BetweenOptions::Inclusive::BOTH);
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, one, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,true,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,true,null,false,false]"));
+  BetweenOptions InclusiveLeft(BetweenOptions::Inclusive::LEFT);
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, one, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,true,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,true,null,false,false]"));
+  BetweenOptions InclusiveRight(BetweenOptions::Inclusive::RIGHT);
+  ValidateBetween<TypeParam>(
+      InclusiveRight, one, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,false,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,false,null,false,false]"));
+  BetweenOptions InclusiveNeither(BetweenOptions::Inclusive::NEITHER);
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, one, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,false,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, one,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,false,null,false,false]"));
+}
+
+TYPED_TEST(TestNumericBetweenKernel, SimpleBetweenArrayArrayArray) {
+  BetweenOptions InclusiveBoth(BetweenOptions::Inclusive::BOTH);
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[1,1,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,true,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,true,null,false,false]"));
+  BetweenOptions InclusiveLeft(BetweenOptions::Inclusive::LEFT);
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[1,1,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,false,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveLeft,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,false,null,false,false]"));
+  BetweenOptions InclusiveRight(BetweenOptions::Inclusive::RIGHT);
+  ValidateBetween<TypeParam>(
+      InclusiveRight, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[1,1,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,true,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveRight,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,true,null,false,false]"));
+  BetweenOptions InclusiveNeither(BetweenOptions::Inclusive::NEITHER);
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[1,1,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[10,10,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[true,true,false,false,false]"));
+  ValidateBetween<TypeParam>(
+      InclusiveNeither,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,2,2,2]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,null,3,3]"),
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,10,2,2,5,5]"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                    "[false,true,false,null,false,false]"));
+}
+
+template <typename Type>
+struct BetweenRandomNumeric {
+  static void Test(const std::shared_ptr<DataType>& type) {
+    using ScalarType = typename TypeTraits<Type>::ScalarType;
+    using CType = typename TypeTraits<Type>::CType;
+    auto rand = random::RandomArrayGenerator(0x5416447);
+    const int64_t length = 1000;
+    for (auto null_probability : {0.0, 0.01, 0.1, 0.25, 0.5, 1.0}) {
+      for (auto inclusive :
+           {BetweenOptions::Inclusive::BOTH, BetweenOptions::Inclusive::LEFT,
+            BetweenOptions::Inclusive::RIGHT, BetweenOptions::Inclusive::NEITHER}) {
+        auto data1 =
+            rand.Numeric<typename Type::PhysicalType>(length, 0, 100, null_probability);
+        auto data2 =
+            rand.Numeric<typename Type::PhysicalType>(length, 0, 100, null_probability);
+        auto data3 =
+            rand.Numeric<typename Type::PhysicalType>(length, 0, 100, null_probability);
+
+        // Create view of data as the type (e.g. timestamp)
+        auto array1 = Datum(*data1->View(type));
+        auto array2 = Datum(*data2->View(type));
+        auto array3 = Datum(*data3->View(type));
+        auto scalar1 = Datum(std::make_shared<ScalarType>(CType(10), type));
+        auto scalar2 = Datum(std::make_shared<ScalarType>(CType(30), type));
+        auto scalar3 = Datum(std::make_shared<ScalarType>(CType(50), type));
+        auto options = BetweenOptions(inclusive);
+        ValidateBetween<Type>(options, array1, scalar2, scalar3);
+        ValidateBetween<Type>(options, array1, array2, scalar3);
+        ValidateBetween<Type>(options, array1, array2, array3);
+        ValidateBetween<Type>(options, array1, scalar2, scalar3);
+        ValidateBetween<Type>(options, scalar1, array2, array3);
+        ValidateBetween<Type>(options, scalar1, array2, scalar3);
+        ValidateBetween<Type>(options, scalar1, scalar2, array3);
+        ValidateBetween<Type>(options, array1, scalar2, array3);
+      }
+    }
+  }
+};
+
+TEST(TestNumericBetweenKernel, BetweenPrimitiveRandomTests) {
+  TestRandomPrimitiveCTypes<BetweenRandomNumeric>();
+}
+
+class TestStringBetweenKernel : public ::testing::Test {};
+
+TEST(TestStringBetweenKernel, RandomBetween) {
+  using ScalarType = typename TypeTraits<StringType>::ScalarType;
+
+  auto rand = random::RandomArrayGenerator(0x5416447);
+  for (size_t i = 3; i < 10; i++) {
+    for (auto null_probability : {0.0, 0.01, 0.1, 0.25, 0.5, 1.0}) {
+      for (auto inclusive :
+           {BetweenOptions::Inclusive::BOTH, BetweenOptions::Inclusive::LEFT,
+            BetweenOptions::Inclusive::RIGHT, BetweenOptions::Inclusive::NEITHER}) {
+        const int64_t length = static_cast<int64_t>(1ULL << i);
+        auto array1 = Datum(rand.String(length, 0, 16, null_probability));
+        auto array2 = Datum(rand.String(length, 0, 16, null_probability));
+        auto array3 = Datum(rand.String(length, 0, 16, null_probability));
+        auto scalar1 = Datum(std::make_shared<ScalarType>("fupi"));
+        auto scalar2 = Datum(std::make_shared<ScalarType>("tupu"));
+        auto scalar3 = Datum(std::make_shared<ScalarType>("zito"));
+        auto options = BetweenOptions(inclusive);
+        ValidateBetween<StringType>(options, array1, scalar2, scalar3);
+        ValidateBetween<StringType>(options, scalar1, array2, scalar3);
+        ValidateBetween<StringType>(options, scalar1, scalar2, array3);
+        ValidateBetween<StringType>(options, scalar1, array2, array3);
+        ValidateBetween<StringType>(options, array1, scalar2, array3);
+        ValidateBetween<StringType>(options, array1, array2, scalar3);
+        ValidateBetween<StringType>(options, array1, array2, array3);
+      }
+    }
+  }
+}
+
+TEST(TestStringBetweenKernel, StringArrayScalarScalarTest) {
+  using ScalarType = typename TypeTraits<StringType>::ScalarType;
+  auto l = Datum(std::make_shared<ScalarType>("abc"));
+  auto r = Datum(std::make_shared<ScalarType>("zzz"));
+  BetweenOptions InclusiveOption(BetweenOptions::Inclusive::BOTH);
+  ValidateBetween<StringType>(
+      InclusiveOption, ArrayFromJSON(TypeTraits<StringType>::type_singleton(), "[]"), l,
+      r, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<StringType>(
+      InclusiveOption, ArrayFromJSON(TypeTraits<StringType>::type_singleton(), "[null]"),
+      l, r, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<StringType>(InclusiveOption,
+                              ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                            R"(["aaa", "aaaa", "ccc", "z"])"),
+                              l, r,
+                              ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                                            R"([false, false, true, true])"));
+  ValidateBetween<StringType>(InclusiveOption,
+                              ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                            R"(["a", "aaaa", "c", "z"])"),
+                              l, r,
+                              ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                                            R"([false, false, true, true])"));
+  ValidateBetween<StringType>(InclusiveOption,
+                              ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                            R"(["a", "aaaa", "fff", "zzzz"])"),
+                              l, r,
+                              ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                                            R"([false, false, true, false])"));
+  ValidateBetween<StringType>(InclusiveOption,
+                              ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                            R"(["abc", "baa", "fff", "zzz"])"),
+                              l, r,
+                              ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                                            R"([true, true, true, true])"));
+  ValidateBetween<StringType>(InclusiveOption,
+                              ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                                            R"(["abd", null, null, "zzx"])"),
+                              l, r,
+                              ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(),
+                                            R"([true, null, null, true])"));
+}
+
+TEST(TestStringBetweenKernel, StringArrayArrayArrayTest) {
+  BetweenOptions InclusiveOption(BetweenOptions::Inclusive::BOTH);
+  ValidateBetween<StringType>(
+      InclusiveOption,
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                    R"(["david","hello","world"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["adam","hi","whirl"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                    R"(["robert","goeiemoreen","whirlwind"])"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[true, false, false]"));
+  ValidateBetween<StringType>(
+      InclusiveOption,
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["x","a","f"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["w","a","e"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["z","a","g"])"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[true, true, true]"));
+  ValidateBetween<StringType>(
+      InclusiveOption,
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                    R"(["block","bit","binary"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                    R"(["bit","nibble","ternary"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["word","d","xyz"])"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[true, false, false]"));
+  ValidateBetween<StringType>(
+      InclusiveOption,
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(),
+                    R"(["よしもと","の","ち"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["は","へ","あ"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["な","を","ち"])"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[false, false, true]"));
+  ValidateBetween<StringType>(
+      InclusiveOption,
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["A","ア","王"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["た","あ","歩"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["李","田",null])"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[false, true, null]"));
+  ValidateBetween<StringType>(
+      InclusiveOption,
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["Б",null,"Я"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["А","Ж","Щ"])"),
+      ArrayFromJSON(TypeTraits<StringType>::type_singleton(), R"(["Д","Л","Ф"])"),
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[true, null, false]"));
+}
+
+TEST(TestTimestampsBetweenKernel, TimestampsArrayArrayArrayTest) {
+  const std::string arr_json = R"(["1970-01-01","2000-02-02","1900-02-28"])";
+  const std::string lhs_json = R"(["1970-01-01","2000-02-01","1900-02-28"])";
+  const std::string rhs_json = R"(["1970-01-02","2000-02-02","1900-02-28"])";
+  // Between Options
+  BetweenOptions both(BetweenOptions::Inclusive::BOTH);
+  BetweenOptions left(BetweenOptions::Inclusive::LEFT);
+  BetweenOptions right(BetweenOptions::Inclusive::RIGHT);
+  BetweenOptions neither(BetweenOptions::Inclusive::NEITHER);
+  // Same units should be fine
+  auto arr = ArrayFromJSON(timestamp(TimeUnit::SECOND), arr_json);
+  auto lhs = ArrayFromJSON(timestamp(TimeUnit::SECOND), lhs_json);
+  auto rhs = ArrayFromJSON(timestamp(TimeUnit::SECOND), rhs_json);
+  // BOTH
+  ValidateBetween<TimestampType>(both, arr, lhs, rhs);
+  // LEFT
+  ValidateBetween<TimestampType>(left, arr, lhs, rhs);
+  // RIGHT
+  ValidateBetween<TimestampType>(right, arr, lhs, rhs);
+  // NEITHER
+  ValidateBetween<TimestampType>(neither, arr, lhs, rhs);
+/*  // So are same time zones
+  arr = ArrayFromJSON(timestamp(TimeUnit::SECOND, "America/Chicago"), arr_json);
+  lhs = ArrayFromJSON(timestamp(TimeUnit::SECOND, "America/Chicago"), lhs_json);
+  rhs = ArrayFromJSON(timestamp(TimeUnit::SECOND, "America/Chicago"), rhs_json);
+  // BOTH
+  ValidateBetween<TimestampType>(both, arr, lhs, rhs);

Review comment:
       The error is due to a redeclaration of variables, see my comment below.




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

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

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



[GitHub] [arrow] bkmgit commented on a change in pull request #11882: ARROW-9843: [C++][Python] Implement Between ternary kernel and Python bindings

Posted by GitBox <gi...@apache.org>.
bkmgit commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r786979074



##########
File path: cpp/src/arrow/compute/api_scalar.h
##########
@@ -342,6 +342,17 @@ struct ARROW_EXPORT CompareOptions {
   enum CompareOperator op;
 };
 
+class ARROW_EXPORT BetweenOptions : public FunctionOptions {
+ public:
+  enum Inclusive { BOTH, LEFT, RIGHT, NEITHER };
+  explicit BetweenOptions(Inclusive inclusive = BOTH);
+  static BetweenOptions Defaults() { return BetweenOptions(); }
+  constexpr static char const kTypeName[] = "BetweenOptions";
+
+  /// Inclusive option to apply

Review comment:
       Updated to
   ``` 
     /// Indicate boundary points to include, both (left <= value <= right), 
     /// left (left <= value < right) , right (left < value <= right)
     /// or neither (left < value < right)
   ```




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

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

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



[GitHub] [arrow] bkmgit commented on a change in pull request #11882: ARROW-9843: [C++][Python] Implement Between ternary kernel and Python bindings

Posted by GitBox <gi...@apache.org>.
bkmgit commented on a change in pull request #11882:
URL: https://github.com/apache/arrow/pull/11882#discussion_r787393753



##########
File path: cpp/src/arrow/compute/kernels/scalar_compare_test.cc
##########
@@ -1865,5 +1867,683 @@ TEST(TestMaxElementWiseMinElementWise, CommonTemporal) {
               ResultWith(ScalarFromJSON(date64(), "86400000")));
 }
 
+template <typename ArrowType>
+static void ValidateBetween(BetweenOptions options, const Datum& val, const Datum& lhs,
+                            const Datum& rhs, const Datum& expected) {
+  ASSERT_OK_AND_ASSIGN(Datum result, Between(val, lhs, rhs, options, nullptr));
+  AssertArraysEqual(*expected.make_array(), *result.make_array(),
+                    /*verbose=*/true);
+}
+
+template <typename ArrowType>
+void ValidateBetween(BetweenOptions options, const Datum& val, const Datum& lhs,
+                     const Datum& rhs) {
+  CompareOperator lhs_val;
+  CompareOperator val_rhs;
+  BetweenOptions::Inclusive include_endpoints = options.inclusive;
+
+  if (include_endpoints == BetweenOptions::Inclusive::NEITHER) {
+    lhs_val = LESS;
+    val_rhs = LESS;
+  } else if (include_endpoints == BetweenOptions::Inclusive::LEFT) {
+    lhs_val = LESS_EQUAL;
+    val_rhs = LESS;
+  } else if (include_endpoints == BetweenOptions::Inclusive::RIGHT) {
+    lhs_val = LESS;
+    val_rhs = LESS_EQUAL;
+  } else {
+    lhs_val = LESS_EQUAL;
+    val_rhs = LESS_EQUAL;
+  }
+
+  ASSERT_OK_AND_ASSIGN(Datum resultl,
+                       CallFunction(CompareOperatorToFunctionName(lhs_val), {lhs, val}));
+  ASSERT_OK_AND_ASSIGN(Datum resultr,
+                       CallFunction(CompareOperatorToFunctionName(val_rhs), {val, rhs}));
+  ASSERT_OK_AND_ASSIGN(Datum expected, CallFunction("and", {resultl, resultr}));
+
+  ValidateBetween<ArrowType>(options, val, lhs, rhs, expected);
+}
+
+template <typename ArrowType>
+class TestNumericBetweenKernel : public ::testing::Test {};
+
+TYPED_TEST_SUITE(TestNumericBetweenKernel, NumericArrowTypes);
+TYPED_TEST(TestNumericBetweenKernel, SimpleBetweenArrayScalarScalar) {
+  using ScalarType = typename TypeTraits<TypeParam>::ScalarType;
+  using CType = typename TypeTraits<TypeParam>::CType;
+
+  Datum zero(std::make_shared<ScalarType>(CType(0)));
+  Datum four(std::make_shared<ScalarType>(CType(4)));
+  Datum null(std::make_shared<ScalarType>());
+  BetweenOptions InclusiveBoth(BetweenOptions::Inclusive::BOTH);
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[]"), zero,
+      four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth, ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[null]"),
+      zero, four, ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[null]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,0,1,1,2,2]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[1,1,1,1,1,1]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[0,1,2,3,4,5]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[1,1,1,1,1,0]"));
+  ValidateBetween<TypeParam>(
+      InclusiveBoth,
+      ArrayFromJSON(TypeTraits<TypeParam>::type_singleton(), "[5,4,3,2,1,0]"), zero, four,
+      ArrayFromJSON(TypeTraits<BooleanType>::type_singleton(), "[0,1,1,1,1,1]"));

Review comment:
       Ok. Updated.




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

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

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