You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@arrow.apache.org by GitBox <gi...@apache.org> on 2020/06/17 12:49:28 UTC

[GitHub] [arrow] pitrou commented on a change in pull request #7461: ARROW-8969: [C++] Reduce binary size of kernels/scalar_compare.cc.o by reusing more kernels between types, operators

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



##########
File path: cpp/src/arrow/compute/kernels/codegen_internal.h
##########
@@ -787,6 +830,41 @@ ArrayKernelExec Integer(detail::GetTypeId get_id) {
   }
 }
 
+template <template <typename...> class Generator, typename Type0, typename... Args>
+ArrayKernelExec IntegerBased(detail::GetTypeId get_id) {

Review comment:
       What's this? How is it supposed to be used? `codegen_internal.h` quickly seems to become write-only: it's difficult to make sense of why those things exist and what they're good at.

##########
File path: cpp/src/arrow/compute/kernels/codegen_internal.h
##########
@@ -121,18 +123,34 @@ struct ArrayIterator<Type, enable_if_boolean<Type>> {
 
 template <typename Type>
 struct ArrayIterator<Type, enable_if_base_binary<Type>> {
-  int64_t position = 0;
-  typename TypeTraits<Type>::ArrayType arr;
-  explicit ArrayIterator(const ArrayData& data) : arr(data.Copy()) {}
-  util::string_view operator()() { return arr.GetView(position++); }
+  using offset_type = typename Type::offset_type;
+  const ArrayData& arr;
+  const offset_type* offsets;
+  offset_type cur_offset;
+  const char* data;
+  int64_t position;
+  explicit ArrayIterator(const ArrayData& arr)
+      : arr(arr),
+        offsets(reinterpret_cast<const offset_type*>(arr.buffers[1]->data()) +
+                arr.offset),
+        cur_offset(offsets[0]),
+        data(reinterpret_cast<const char*>(arr.buffers[2]->data())),
+        position(0) {}
+
+  util::string_view operator()() {
+    offset_type next_offset = offsets[position++ + 1];
+    auto result = util::string_view(data + cur_offset, next_offset - cur_offset);
+    cur_offset = next_offset;
+    return result;
+  }

Review comment:
       It would probably depend on the compiler? Logically, it's quite similar.

##########
File path: cpp/src/arrow/compute/kernels/codegen_internal.h
##########
@@ -485,18 +502,44 @@ struct ScalarUnaryNotNullStateful {
   struct ArrayExec<Type, enable_if_base_binary<Type>> {
     static void Exec(const ThisType& functor, KernelContext* ctx, const ExecBatch& batch,
                      Datum* out) {
-      typename TypeTraits<Type>::BuilderType builder;
-      VisitArrayValuesInline<Arg0Type>(*batch[0].array(), [&](util::optional<ARG0> v) {
-        if (v.has_value()) {
-          KERNEL_RETURN_IF_ERROR(ctx, builder.Append(functor.op.Call(ctx, *v)));
-        } else {
-          KERNEL_RETURN_IF_ERROR(ctx, builder.AppendNull());
+      // TODO: This code path is currently inadequately tested.
+
+      using offset_type = typename Type::offset_type;

Review comment:
       I'm not convinced why this is doing everything by hand instead of relying on the array builder as before.

##########
File path: cpp/src/arrow/compute/kernels/scalar_compare_test.cc
##########
@@ -368,23 +368,55 @@ TYPED_TEST(TestNumericCompareKernel, TestNullScalar) {
 }
 
 TYPED_TEST_SUITE(TestNumericCompareKernel, NumericArrowTypes);
-TYPED_TEST(TestNumericCompareKernel, RandomCompareArrayScalar) {
-  using ScalarType = typename TypeTraits<TypeParam>::ScalarType;
-  using CType = typename TypeTraits<TypeParam>::CType;
 
+template <typename Type>
+void DoRandomCompare(const std::shared_ptr<DataType>& type) {
+  using ScalarType = typename TypeTraits<Type>::ScalarType;
+  using CType = typename TypeTraits<Type>::CType;
   auto rand = random::RandomArrayGenerator(0x5416447);
   for (size_t i = 3; i < 10; i++) {

Review comment:
       It doesn't make sense to try out a ton of different sizes (or probabilities, see below), IMHO.




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

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