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/05/26 18:19:36 UTC

[GitHub] [arrow] nirandaperera opened a new pull request #10410: ARROW-10640: [C++] A "where" kernel to combine two arrays based on a mask

nirandaperera opened a new pull request #10410:
URL: https://github.com/apache/arrow/pull/10410


   Adding a preliminary impl for an `if_else(cond: Datum, left: Datum, right: Datum)` function. It works as follows,
   ```python
   def if_else(cond, left, right):
       for c, true_val, false_val in zip(cond, left, right):
           if c:
               yield true_val
           else:
               yield false_val
   ```
   `null` values will be promoted to the output. 
   
   


-- 
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



[GitHub] [arrow] nirandaperera commented on a change in pull request #10410: ARROW-10640: [C++] A "where" kernel to combine two arrays based on a mask

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_if_else.cc
##########
@@ -0,0 +1,577 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <arrow/compute/api.h>
+#include <arrow/compute/kernels/codegen_internal.h>
+#include <arrow/compute/util_internal.h>
+#include <arrow/util/bit_block_counter.h>
+#include <arrow/util/bitmap.h>
+#include <arrow/util/bitmap_ops.h>
+
+namespace arrow {
+using internal::BitBlockCount;
+using internal::BitBlockCounter;
+using internal::Bitmap;
+
+namespace compute {
+
+namespace {
+
+util::optional<uint64_t> GetConstantValidityWord(const Datum& data) {
+  if (data.is_scalar()) {
+    return data.scalar()->is_valid ? ~uint64_t(0) : uint64_t(0);
+  }
+
+  if (data.array()->null_count == data.array()->length) return uint64_t(0);
+
+  if (!data.array()->MayHaveNulls()) return ~uint64_t(0);
+
+  // no constant validity word available
+  return {};
+}
+
+inline Bitmap GetBitmap(const Datum& datum, int i) {
+  if (datum.is_scalar()) return {};
+  const ArrayData& a = *datum.array();
+  return Bitmap{a.buffers[i], a.offset, a.length};
+}
+
+// if the condition is null then output is null otherwise we take validity from the
+// selected argument
+// ie. cond.valid & (cond.data & left.valid | ~cond.data & right.valid)
+Status PromoteNullsVisitor(KernelContext* ctx, const Datum& cond_d, const Datum& left_d,
+                           const Datum& right_d, ArrayData* output) {
+  auto cond_const = GetConstantValidityWord(cond_d);
+  auto left_const = GetConstantValidityWord(left_d);
+  auto right_const = GetConstantValidityWord(right_d);
+
+  enum { COND_CONST = 1, LEFT_CONST = 2, RIGHT_CONST = 4 };
+  auto flag = COND_CONST * cond_const.has_value() | LEFT_CONST * left_const.has_value() |
+              RIGHT_CONST * right_const.has_value();
+
+  const ArrayData& cond = *cond_d.array();
+  // cond.data will always be available
+  Bitmap cond_data{cond.buffers[1], cond.offset, cond.length};
+  Bitmap cond_valid{cond.buffers[0], cond.offset, cond.length};
+  Bitmap left_valid = GetBitmap(left_d, 0);
+  Bitmap right_valid = GetBitmap(right_d, 0);
+  // sometimes Bitmaps will be ignored, in which case we replace access to them with
+  // duplicated (probably elided) access to cond_data
+  const Bitmap& _ = cond_data;
+
+  // lambda function that will be used inside the visitor
+  uint64_t* out_validity = nullptr;
+  int64_t i = 0;
+  auto apply = [&](uint64_t c_valid, uint64_t c_data, uint64_t l_valid,
+                   uint64_t r_valid) {
+    out_validity[i] = c_valid & ((c_data & l_valid) | (~c_data & r_valid));
+    i++;
+  };
+
+  // cond.valid & (cond.data & left.valid | ~cond.data & right.valid)
+  // In the following cases, we dont need to allocate out_valid bitmap
+  switch (flag) {
+    case COND_CONST | LEFT_CONST | RIGHT_CONST:
+      // if cond & left & right all ones, then output is all valid --> out_valid = nullptr
+      if ((*cond_const & *left_const & *right_const) == UINT64_MAX) {
+        return Status::OK();
+      }
+    case LEFT_CONST | RIGHT_CONST:
+      // if both left and right are valid, no need to calculate out_valid bitmap. Pass
+      // cond validity buffer
+      if ((*left_const & *right_const) == UINT64_MAX) {
+        // if there's an offset, copy bitmap (cannot slice a bitmap)
+        if (cond.offset) {
+          ARROW_ASSIGN_OR_RAISE(
+              output->buffers[0],
+              arrow::internal::CopyBitmap(ctx->memory_pool(), cond.buffers[0]->data(),
+                                          cond.offset, cond.length));
+        } else {  // just copy assign cond validity buffer
+          output->buffers[0] = cond.buffers[0];
+        }
+        return Status::OK();
+      }
+  }
+
+  // following cases requires a separate out_valid buffer
+  ARROW_ASSIGN_OR_RAISE(output->buffers[0], ctx->AllocateBitmap(cond.length));
+  out_validity = output->GetMutableValues<uint64_t>(0);
+
+  enum { C_VALID, C_DATA, L_VALID, R_VALID };
+
+  switch (flag) {
+    case COND_CONST | LEFT_CONST | RIGHT_CONST:
+      Bitmap::VisitWords({_, cond_data, _, _}, [&](std::array<uint64_t, 4> words) {
+        apply(*cond_const, words[C_DATA], *left_const, *right_const);
+      });
+      break;
+    case LEFT_CONST | RIGHT_CONST:
+      Bitmap::VisitWords(
+          {cond_valid, cond_data, _, _}, [&](std::array<uint64_t, 4> words) {
+            apply(words[C_VALID], words[C_DATA], *left_const, *right_const);
+          });
+      break;
+    case COND_CONST | RIGHT_CONST:
+      // bitmaps[C_VALID], bitmaps[R_VALID] might be null; override to make it safe for
+      // Visit()
+      Bitmap::VisitWords(
+          {_, cond_data, left_valid, _}, [&](std::array<uint64_t, 4> words) {
+            apply(*cond_const, words[C_DATA], words[L_VALID], *right_const);
+          });
+      break;
+    case RIGHT_CONST:
+      // bitmaps[R_VALID] might be null; override to make it safe for Visit()
+      Bitmap::VisitWords(
+          {cond_valid, cond_data, left_valid, _}, [&](std::array<uint64_t, 4> words) {
+            apply(words[C_VALID], words[C_DATA], words[L_VALID], *right_const);
+          });
+      break;
+    case COND_CONST | LEFT_CONST:
+      // bitmaps[C_VALID], bitmaps[L_VALID] might be null; override to make it safe for
+      // Visit()
+      Bitmap::VisitWords({_, cond_data, _, right_valid},
+                         [&](std::array<uint64_t, 4> words) {
+                           apply(*cond_const, words[C_DATA], *left_const, words[R_VALID]);
+                         });
+      break;
+    case LEFT_CONST:
+      // bitmaps[L_VALID] might be null; override to make it safe for Visit()
+      Bitmap::VisitWords(
+          {cond_valid, cond_data, _, right_valid}, [&](std::array<uint64_t, 4> words) {
+            apply(words[C_VALID], words[C_DATA], *left_const, words[R_VALID]);
+          });
+      break;
+    case COND_CONST:
+      // bitmaps[C_VALID] might be null; override to make it safe for Visit()
+      Bitmap::VisitWords(
+          {_, cond_data, left_valid, right_valid}, [&](std::array<uint64_t, 4> words) {
+            apply(*cond_const, words[C_DATA], words[L_VALID], words[R_VALID]);
+          });
+      break;
+    case 0:
+      Bitmap::VisitWords({cond_valid, cond_data, left_valid, right_valid},
+                         [&](std::array<uint64_t, 4> words) {
+                           apply(words[C_VALID], words[C_DATA], words[L_VALID],
+                                 words[R_VALID]);
+                         });
+      break;
+  }
+  return Status::OK();
+}
+
+template <typename Type, typename Enable = void>
+struct IfElseFunctor {};
+
+// only number types needs to be handled for Fixed sized primitive data types because,
+// internal::GenerateTypeAgnosticPrimitive forwards types to the corresponding unsigned
+// int type
+template <typename Type>
+struct IfElseFunctor<Type, enable_if_number<Type>> {
+  using T = typename TypeTraits<Type>::CType;
+  // A - Array
+  // S - Scalar
+
+  //  AAA
+  static Status Call(KernelContext* ctx, const ArrayData& cond, const ArrayData& left,
+                     const ArrayData& right, ArrayData* out) {
+    ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Buffer> out_buf,
+                          ctx->Allocate(cond.length * sizeof(T)));
+    T* out_values = reinterpret_cast<T*>(out_buf->mutable_data());
+
+    // copy right data to out_buff
+    const T* right_data = right.GetValues<T>(1);
+    std::memcpy(out_values, right_data, right.length * sizeof(T));
+
+    const auto* cond_data = cond.buffers[1]->data();  // this is a BoolArray
+    BitBlockCounter bit_counter(cond_data, cond.offset, cond.length);
+
+    // selectively copy values from left data
+    const T* left_data = left.GetValues<T>(1);
+    int64_t offset = cond.offset;
+
+    // todo this can be improved by intrinsics. ex: _mm*_mask_store_e* (vmovdqa*)
+    while (offset < cond.offset + cond.length) {
+      const BitBlockCount& block = bit_counter.NextWord();
+      if (block.AllSet()) {  // all from left
+        std::memcpy(out_values, left_data, block.length * sizeof(T));
+      } else if (block.popcount) {  // selectively copy from left
+        for (int64_t i = 0; i < block.length; ++i) {
+          if (BitUtil::GetBit(cond_data, offset + i)) {
+            out_values[i] = left_data[i];
+          }
+        }
+      }
+
+      offset += block.length;
+      out_values += block.length;
+      left_data += block.length;
+    }
+
+    out->buffers[1] = std::move(out_buf);
+    return Status::OK();
+  }
+
+  // ASA
+  static Status Call(KernelContext* ctx, const ArrayData& cond, const Scalar& left,
+                     const ArrayData& right, ArrayData* out) {
+    ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Buffer> out_buf,
+                          ctx->Allocate(cond.length * sizeof(T)));
+    T* out_values = reinterpret_cast<T*>(out_buf->mutable_data());
+
+    // copy right data to out_buff
+    const T* right_data = right.GetValues<T>(1);
+    std::memcpy(out_values, right_data, right.length * sizeof(T));
+
+    const auto* cond_data = cond.buffers[1]->data();  // this is a BoolArray
+    BitBlockCounter bit_counter(cond_data, cond.offset, cond.length);
+
+    // selectively copy values from left data
+    T left_data = internal::UnboxScalar<Type>::Unbox(left);
+    int64_t offset = cond.offset;
+
+    // todo this can be improved by intrinsics. ex: _mm*_mask_store_e* (vmovdqa*)
+    while (offset < cond.offset + cond.length) {
+      const BitBlockCount& block = bit_counter.NextWord();
+      if (block.AllSet()) {  // all from left
+        std::fill(out_values, out_values + block.length, left_data);
+      } else if (block.popcount) {  // selectively copy from left
+        for (int64_t i = 0; i < block.length; ++i) {
+          if (BitUtil::GetBit(cond_data, offset + i)) {
+            out_values[i] = left_data;
+          }
+        }
+      }
+
+      offset += block.length;
+      out_values += block.length;
+    }
+
+    out->buffers[1] = std::move(out_buf);
+    return Status::OK();
+  }
+
+  // AAS
+  static Status Call(KernelContext* ctx, const ArrayData& cond, const ArrayData& left,
+                     const Scalar& right, ArrayData* out) {
+    ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Buffer> out_buf,
+                          ctx->Allocate(cond.length * sizeof(T)));
+    T* out_values = reinterpret_cast<T*>(out_buf->mutable_data());
+
+    // copy left data to out_buff
+    const T* left_data = left.GetValues<T>(1);
+    std::memcpy(out_values, left_data, left.length * sizeof(T));
+
+    const auto* cond_data = cond.buffers[1]->data();  // this is a BoolArray
+    BitBlockCounter bit_counter(cond_data, cond.offset, cond.length);
+
+    // selectively copy values from left data
+    T right_data = internal::UnboxScalar<Type>::Unbox(right);
+    int64_t offset = cond.offset;
+
+    // todo this can be improved by intrinsics. ex: _mm*_mask_store_e* (vmovdqa*)
+    // left data is already in the output buffer. Therefore, mask needs to be inverted
+    while (offset < cond.offset + cond.length) {
+      const BitBlockCount& block = bit_counter.NextWord();
+      if (block.NoneSet()) {  // all from right
+        std::fill(out_values, out_values + block.length, right_data);
+      } else if (block.popcount) {  // selectively copy from right
+        for (int64_t i = 0; i < block.length; ++i) {
+          if (!BitUtil::GetBit(cond_data, offset + i)) {
+            out_values[i] = right_data;
+          }
+        }
+      }
+
+      offset += block.length;
+      out_values += block.length;
+    }
+
+    out->buffers[1] = std::move(out_buf);
+    return Status::OK();
+  }
+
+  // ASS
+  static Status Call(KernelContext* ctx, const ArrayData& cond, const Scalar& left,
+                     const Scalar& right, ArrayData* out) {
+    ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Buffer> out_buf,
+                          ctx->Allocate(cond.length * sizeof(T)));
+    T* out_values = reinterpret_cast<T*>(out_buf->mutable_data());
+
+    // copy right data to out_buff
+    T right_data = internal::UnboxScalar<Type>::Unbox(right);
+    std::fill(out_values, out_values + cond.length, right_data);
+
+    const auto* cond_data = cond.buffers[1]->data();  // this is a BoolArray
+    BitBlockCounter bit_counter(cond_data, cond.offset, cond.length);
+
+    // selectively copy values from left data
+    T left_data = internal::UnboxScalar<Type>::Unbox(left);
+    int64_t offset = cond.offset;
+
+    // todo this can be improved by intrinsics. ex: _mm*_mask_store_e* (vmovdqa*)
+    while (offset < cond.offset + cond.length) {
+      const BitBlockCount& block = bit_counter.NextWord();
+      if (block.AllSet()) {  // all from left
+        std::fill(out_values, out_values + block.length, left_data);
+      } else if (block.popcount) {  // selectively copy from left
+        for (int64_t i = 0; i < block.length; ++i) {
+          if (BitUtil::GetBit(cond_data, offset + i)) {
+            out_values[i] = left_data;
+          }
+        }
+      }
+
+      offset += block.length;
+      out_values += block.length;
+    }
+
+    out->buffers[1] = std::move(out_buf);
+    return Status::OK();
+  }
+};
+
+template <typename Type>
+struct IfElseFunctor<Type, enable_if_boolean<Type>> {
+  // AAA
+  static Status Call(KernelContext* ctx, const ArrayData& cond, const ArrayData& left,
+                     const ArrayData& right, ArrayData* out) {
+    // out_buff = right & ~cond
+    ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Buffer> out_buf,
+                          arrow::internal::BitmapAndNot(
+                              ctx->memory_pool(), right.buffers[1]->data(), right.offset,
+                              cond.buffers[1]->data(), cond.offset, cond.length, 0));
+
+    // out_buff = left & cond
+    ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Buffer> temp_buf,
+                          arrow::internal::BitmapAnd(
+                              ctx->memory_pool(), left.buffers[1]->data(), left.offset,
+                              cond.buffers[1]->data(), cond.offset, cond.length, 0));
+
+    arrow::internal::BitmapOr(out_buf->data(), 0, temp_buf->data(), 0, cond.length, 0,
+                              out_buf->mutable_data());
+    out->buffers[1] = std::move(out_buf);
+    return Status::OK();
+  }
+
+  // ASA
+  static Status Call(KernelContext* ctx, const ArrayData& cond, const Scalar& left,
+                     const ArrayData& right, ArrayData* out) {
+    // out_buff = right & ~cond
+    ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Buffer> out_buf,
+                          arrow::internal::BitmapAndNot(
+                              ctx->memory_pool(), right.buffers[1]->data(), right.offset,
+                              cond.buffers[1]->data(), cond.offset, cond.length, 0));
+
+    // out_buff = left & cond
+    bool left_data = internal::UnboxScalar<BooleanType>::Unbox(left);
+    if (left_data) {
+      arrow::internal::BitmapOr(out_buf->data(), 0, cond.buffers[1]->data(), cond.offset,
+                                cond.length, 0, out_buf->mutable_data());
+    }
+
+    out->buffers[1] = std::move(out_buf);
+    return Status::OK();
+  }
+
+  // AAS
+  static Status Call(KernelContext* ctx, const ArrayData& cond, const ArrayData& left,
+                     const Scalar& right, ArrayData* out) {
+    // out_buff = left & cond
+    ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Buffer> out_buf,
+                          arrow::internal::BitmapAnd(
+                              ctx->memory_pool(), left.buffers[1]->data(), left.offset,
+                              cond.buffers[1]->data(), cond.offset, cond.length, 0));
+
+    bool right_data = internal::UnboxScalar<BooleanType>::Unbox(right);
+
+    // out_buff = left & cond | right & ~cond
+    if (right_data) {
+      arrow::internal::BitmapOrNot(out_buf->data(), 0, cond.buffers[1]->data(),
+                                   cond.offset, cond.length, 0, out_buf->mutable_data());
+    }
+
+    out->buffers[1] = std::move(out_buf);
+    return Status::OK();
+  }
+
+  // ASS
+  static Status Call(KernelContext* ctx, const ArrayData& cond, const Scalar& left,
+                     const Scalar& right, ArrayData* out) {
+    bool left_data = internal::UnboxScalar<BooleanType>::Unbox(left);
+    bool right_data = internal::UnboxScalar<BooleanType>::Unbox(right);
+
+    // out_buf = left & cond | right & ~cond
+    std::shared_ptr<Buffer> out_buf = nullptr;
+    if (left_data) {
+      if (right_data) {
+        // out_buf = ones
+        ARROW_ASSIGN_OR_RAISE(out_buf, ctx->AllocateBitmap(cond.length));
+        // filling with UINT8_MAX upto the buffer's size (in bytes)
+        arrow::compute::internal::SetMemory<UINT8_MAX>(out_buf.get());
+      } else {
+        // out_buf = cond
+        out_buf = SliceBuffer(cond.buffers[1], cond.offset, cond.length);
+      }
+    } else {
+      if (right_data) {
+        // out_buf = ~cond
+        ARROW_ASSIGN_OR_RAISE(out_buf, arrow::internal::InvertBitmap(
+                                           ctx->memory_pool(), cond.buffers[1]->data(),
+                                           cond.offset, cond.length))
+      } else {
+        // out_buf = zeros
+        ARROW_ASSIGN_OR_RAISE(out_buf, ctx->AllocateBitmap(cond.length));
+      }
+    }
+    out->buffers[1] = std::move(out_buf);
+    return Status::OK();
+  }
+};
+
+template <typename Type>
+struct IfElseFunctor<Type, enable_if_null<Type>> {
+  template <typename T>
+  static inline Status ReturnCopy(const T& in, T* out) {
+    // Nothing preallocated, so we assign in into the output
+    *out = in;
+    return Status::OK();
+  }
+
+  // AAA
+  static Status Call(KernelContext* ctx, const ArrayData& cond, const ArrayData& left,
+                     const ArrayData& right, ArrayData* out) {
+    return ReturnCopy(left, out);
+  }
+
+  // ASA
+  static Status Call(KernelContext* ctx, const ArrayData& cond, const Scalar& left,
+                     const ArrayData& right, ArrayData* out) {
+    return ReturnCopy(right, out);
+  }
+
+  // AAS
+  static Status Call(KernelContext* ctx, const ArrayData& cond, const ArrayData& left,
+                     const Scalar& right, ArrayData* out) {
+    return ReturnCopy(left, out);
+  }
+
+  // ASS
+  static Status Call(KernelContext* ctx, const ArrayData& cond, const Scalar& left,
+                     const Scalar& right, ArrayData* out) {
+    return ReturnCopy(cond, out);
+  }
+};
+
+template <typename Type>
+struct ResolveIfElseExec {
+  static Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    // cond is scalar
+    if (batch[0].is_scalar()) {
+      const auto& cond = batch[0].scalar_as<BooleanScalar>();
+      if (batch[1].is_scalar() && batch[2].is_scalar()) {
+        if (cond.is_valid) {
+          *out = cond.value ? batch[1].scalar() : batch[2].scalar();
+        } else {
+          *out = MakeNullScalar(batch[1].type());
+        }
+        return Status::OK();
+      } else {  // either left or right is an array. output is always an array
+        // output size is the size of the array arg
+        auto bcast_size = batch.length;
+        if (cond.is_valid) {
+          const auto& valid_data = cond.value ? batch[1] : batch[2];
+          if (valid_data.is_array()) {
+            *out = valid_data;
+          } else {  // valid data is a scalar that needs to be broadcasted
+            ARROW_ASSIGN_OR_RAISE(*out,
+                                  MakeArrayFromScalar(*valid_data.scalar(), bcast_size,
+                                                      ctx->memory_pool()));
+          }
+        } else {  // cond is null. create null array
+          ARROW_ASSIGN_OR_RAISE(
+              *out, MakeArrayOfNull(batch[1].type(), bcast_size, ctx->memory_pool()))
+        }
+        return Status::OK();
+      }
+    }

Review comment:
       @bkietz would this work even when these `*_const` dont have any values in them?
   ```
   if (cond_const == kAllValid && left_const == kAllValid && right_const == kAllValid) {
     // no nulls in any input: don't bother allocating a null bitmap
     return Status::OK();
   }
   ```

##########
File path: cpp/src/arrow/compute/kernels/scalar_if_else.cc
##########
@@ -0,0 +1,577 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <arrow/compute/api.h>
+#include <arrow/compute/kernels/codegen_internal.h>
+#include <arrow/compute/util_internal.h>
+#include <arrow/util/bit_block_counter.h>
+#include <arrow/util/bitmap.h>
+#include <arrow/util/bitmap_ops.h>
+
+namespace arrow {
+using internal::BitBlockCount;
+using internal::BitBlockCounter;
+using internal::Bitmap;
+
+namespace compute {
+
+namespace {
+
+util::optional<uint64_t> GetConstantValidityWord(const Datum& data) {
+  if (data.is_scalar()) {
+    return data.scalar()->is_valid ? ~uint64_t(0) : uint64_t(0);
+  }
+
+  if (data.array()->null_count == data.array()->length) return uint64_t(0);
+
+  if (!data.array()->MayHaveNulls()) return ~uint64_t(0);
+
+  // no constant validity word available
+  return {};
+}
+
+inline Bitmap GetBitmap(const Datum& datum, int i) {
+  if (datum.is_scalar()) return {};
+  const ArrayData& a = *datum.array();
+  return Bitmap{a.buffers[i], a.offset, a.length};
+}
+
+// if the condition is null then output is null otherwise we take validity from the
+// selected argument
+// ie. cond.valid & (cond.data & left.valid | ~cond.data & right.valid)
+Status PromoteNullsVisitor(KernelContext* ctx, const Datum& cond_d, const Datum& left_d,
+                           const Datum& right_d, ArrayData* output) {
+  auto cond_const = GetConstantValidityWord(cond_d);
+  auto left_const = GetConstantValidityWord(left_d);
+  auto right_const = GetConstantValidityWord(right_d);
+
+  enum { COND_CONST = 1, LEFT_CONST = 2, RIGHT_CONST = 4 };
+  auto flag = COND_CONST * cond_const.has_value() | LEFT_CONST * left_const.has_value() |
+              RIGHT_CONST * right_const.has_value();
+
+  const ArrayData& cond = *cond_d.array();
+  // cond.data will always be available
+  Bitmap cond_data{cond.buffers[1], cond.offset, cond.length};
+  Bitmap cond_valid{cond.buffers[0], cond.offset, cond.length};
+  Bitmap left_valid = GetBitmap(left_d, 0);
+  Bitmap right_valid = GetBitmap(right_d, 0);
+  // sometimes Bitmaps will be ignored, in which case we replace access to them with
+  // duplicated (probably elided) access to cond_data
+  const Bitmap& _ = cond_data;
+
+  // lambda function that will be used inside the visitor
+  uint64_t* out_validity = nullptr;
+  int64_t i = 0;
+  auto apply = [&](uint64_t c_valid, uint64_t c_data, uint64_t l_valid,
+                   uint64_t r_valid) {
+    out_validity[i] = c_valid & ((c_data & l_valid) | (~c_data & r_valid));
+    i++;
+  };
+
+  // cond.valid & (cond.data & left.valid | ~cond.data & right.valid)
+  // In the following cases, we dont need to allocate out_valid bitmap
+  switch (flag) {
+    case COND_CONST | LEFT_CONST | RIGHT_CONST:
+      // if cond & left & right all ones, then output is all valid --> out_valid = nullptr
+      if ((*cond_const & *left_const & *right_const) == UINT64_MAX) {
+        return Status::OK();
+      }
+    case LEFT_CONST | RIGHT_CONST:
+      // if both left and right are valid, no need to calculate out_valid bitmap. Pass
+      // cond validity buffer
+      if ((*left_const & *right_const) == UINT64_MAX) {
+        // if there's an offset, copy bitmap (cannot slice a bitmap)
+        if (cond.offset) {
+          ARROW_ASSIGN_OR_RAISE(
+              output->buffers[0],
+              arrow::internal::CopyBitmap(ctx->memory_pool(), cond.buffers[0]->data(),
+                                          cond.offset, cond.length));
+        } else {  // just copy assign cond validity buffer
+          output->buffers[0] = cond.buffers[0];
+        }
+        return Status::OK();
+      }
+  }
+
+  // following cases requires a separate out_valid buffer
+  ARROW_ASSIGN_OR_RAISE(output->buffers[0], ctx->AllocateBitmap(cond.length));
+  out_validity = output->GetMutableValues<uint64_t>(0);
+
+  enum { C_VALID, C_DATA, L_VALID, R_VALID };
+
+  switch (flag) {
+    case COND_CONST | LEFT_CONST | RIGHT_CONST:
+      Bitmap::VisitWords({_, cond_data, _, _}, [&](std::array<uint64_t, 4> words) {
+        apply(*cond_const, words[C_DATA], *left_const, *right_const);
+      });
+      break;
+    case LEFT_CONST | RIGHT_CONST:
+      Bitmap::VisitWords(
+          {cond_valid, cond_data, _, _}, [&](std::array<uint64_t, 4> words) {
+            apply(words[C_VALID], words[C_DATA], *left_const, *right_const);
+          });
+      break;
+    case COND_CONST | RIGHT_CONST:
+      // bitmaps[C_VALID], bitmaps[R_VALID] might be null; override to make it safe for
+      // Visit()
+      Bitmap::VisitWords(
+          {_, cond_data, left_valid, _}, [&](std::array<uint64_t, 4> words) {
+            apply(*cond_const, words[C_DATA], words[L_VALID], *right_const);
+          });
+      break;
+    case RIGHT_CONST:
+      // bitmaps[R_VALID] might be null; override to make it safe for Visit()
+      Bitmap::VisitWords(
+          {cond_valid, cond_data, left_valid, _}, [&](std::array<uint64_t, 4> words) {
+            apply(words[C_VALID], words[C_DATA], words[L_VALID], *right_const);
+          });
+      break;
+    case COND_CONST | LEFT_CONST:
+      // bitmaps[C_VALID], bitmaps[L_VALID] might be null; override to make it safe for
+      // Visit()
+      Bitmap::VisitWords({_, cond_data, _, right_valid},
+                         [&](std::array<uint64_t, 4> words) {
+                           apply(*cond_const, words[C_DATA], *left_const, words[R_VALID]);
+                         });
+      break;
+    case LEFT_CONST:
+      // bitmaps[L_VALID] might be null; override to make it safe for Visit()
+      Bitmap::VisitWords(
+          {cond_valid, cond_data, _, right_valid}, [&](std::array<uint64_t, 4> words) {
+            apply(words[C_VALID], words[C_DATA], *left_const, words[R_VALID]);
+          });
+      break;
+    case COND_CONST:
+      // bitmaps[C_VALID] might be null; override to make it safe for Visit()
+      Bitmap::VisitWords(
+          {_, cond_data, left_valid, right_valid}, [&](std::array<uint64_t, 4> words) {
+            apply(*cond_const, words[C_DATA], words[L_VALID], words[R_VALID]);
+          });
+      break;
+    case 0:
+      Bitmap::VisitWords({cond_valid, cond_data, left_valid, right_valid},
+                         [&](std::array<uint64_t, 4> words) {
+                           apply(words[C_VALID], words[C_DATA], words[L_VALID],
+                                 words[R_VALID]);
+                         });
+      break;
+  }
+  return Status::OK();
+}
+
+template <typename Type, typename Enable = void>
+struct IfElseFunctor {};
+
+// only number types needs to be handled for Fixed sized primitive data types because,
+// internal::GenerateTypeAgnosticPrimitive forwards types to the corresponding unsigned
+// int type
+template <typename Type>
+struct IfElseFunctor<Type, enable_if_number<Type>> {
+  using T = typename TypeTraits<Type>::CType;
+  // A - Array
+  // S - Scalar
+
+  //  AAA
+  static Status Call(KernelContext* ctx, const ArrayData& cond, const ArrayData& left,
+                     const ArrayData& right, ArrayData* out) {
+    ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Buffer> out_buf,
+                          ctx->Allocate(cond.length * sizeof(T)));
+    T* out_values = reinterpret_cast<T*>(out_buf->mutable_data());
+
+    // copy right data to out_buff
+    const T* right_data = right.GetValues<T>(1);
+    std::memcpy(out_values, right_data, right.length * sizeof(T));
+
+    const auto* cond_data = cond.buffers[1]->data();  // this is a BoolArray
+    BitBlockCounter bit_counter(cond_data, cond.offset, cond.length);
+
+    // selectively copy values from left data
+    const T* left_data = left.GetValues<T>(1);
+    int64_t offset = cond.offset;
+
+    // todo this can be improved by intrinsics. ex: _mm*_mask_store_e* (vmovdqa*)
+    while (offset < cond.offset + cond.length) {
+      const BitBlockCount& block = bit_counter.NextWord();
+      if (block.AllSet()) {  // all from left
+        std::memcpy(out_values, left_data, block.length * sizeof(T));
+      } else if (block.popcount) {  // selectively copy from left
+        for (int64_t i = 0; i < block.length; ++i) {
+          if (BitUtil::GetBit(cond_data, offset + i)) {
+            out_values[i] = left_data[i];
+          }
+        }
+      }
+
+      offset += block.length;
+      out_values += block.length;
+      left_data += block.length;
+    }
+
+    out->buffers[1] = std::move(out_buf);
+    return Status::OK();
+  }
+
+  // ASA
+  static Status Call(KernelContext* ctx, const ArrayData& cond, const Scalar& left,
+                     const ArrayData& right, ArrayData* out) {
+    ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Buffer> out_buf,
+                          ctx->Allocate(cond.length * sizeof(T)));
+    T* out_values = reinterpret_cast<T*>(out_buf->mutable_data());
+
+    // copy right data to out_buff
+    const T* right_data = right.GetValues<T>(1);
+    std::memcpy(out_values, right_data, right.length * sizeof(T));
+
+    const auto* cond_data = cond.buffers[1]->data();  // this is a BoolArray
+    BitBlockCounter bit_counter(cond_data, cond.offset, cond.length);
+
+    // selectively copy values from left data
+    T left_data = internal::UnboxScalar<Type>::Unbox(left);
+    int64_t offset = cond.offset;
+
+    // todo this can be improved by intrinsics. ex: _mm*_mask_store_e* (vmovdqa*)
+    while (offset < cond.offset + cond.length) {
+      const BitBlockCount& block = bit_counter.NextWord();
+      if (block.AllSet()) {  // all from left
+        std::fill(out_values, out_values + block.length, left_data);
+      } else if (block.popcount) {  // selectively copy from left
+        for (int64_t i = 0; i < block.length; ++i) {
+          if (BitUtil::GetBit(cond_data, offset + i)) {
+            out_values[i] = left_data;
+          }
+        }
+      }
+
+      offset += block.length;
+      out_values += block.length;
+    }
+
+    out->buffers[1] = std::move(out_buf);
+    return Status::OK();
+  }
+
+  // AAS
+  static Status Call(KernelContext* ctx, const ArrayData& cond, const ArrayData& left,
+                     const Scalar& right, ArrayData* out) {
+    ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Buffer> out_buf,
+                          ctx->Allocate(cond.length * sizeof(T)));
+    T* out_values = reinterpret_cast<T*>(out_buf->mutable_data());
+
+    // copy left data to out_buff
+    const T* left_data = left.GetValues<T>(1);
+    std::memcpy(out_values, left_data, left.length * sizeof(T));
+
+    const auto* cond_data = cond.buffers[1]->data();  // this is a BoolArray
+    BitBlockCounter bit_counter(cond_data, cond.offset, cond.length);
+
+    // selectively copy values from left data
+    T right_data = internal::UnboxScalar<Type>::Unbox(right);
+    int64_t offset = cond.offset;
+
+    // todo this can be improved by intrinsics. ex: _mm*_mask_store_e* (vmovdqa*)
+    // left data is already in the output buffer. Therefore, mask needs to be inverted
+    while (offset < cond.offset + cond.length) {
+      const BitBlockCount& block = bit_counter.NextWord();
+      if (block.NoneSet()) {  // all from right
+        std::fill(out_values, out_values + block.length, right_data);
+      } else if (block.popcount) {  // selectively copy from right
+        for (int64_t i = 0; i < block.length; ++i) {
+          if (!BitUtil::GetBit(cond_data, offset + i)) {
+            out_values[i] = right_data;
+          }
+        }
+      }
+
+      offset += block.length;
+      out_values += block.length;
+    }
+
+    out->buffers[1] = std::move(out_buf);
+    return Status::OK();
+  }
+
+  // ASS
+  static Status Call(KernelContext* ctx, const ArrayData& cond, const Scalar& left,
+                     const Scalar& right, ArrayData* out) {
+    ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Buffer> out_buf,
+                          ctx->Allocate(cond.length * sizeof(T)));
+    T* out_values = reinterpret_cast<T*>(out_buf->mutable_data());
+
+    // copy right data to out_buff
+    T right_data = internal::UnboxScalar<Type>::Unbox(right);
+    std::fill(out_values, out_values + cond.length, right_data);
+
+    const auto* cond_data = cond.buffers[1]->data();  // this is a BoolArray
+    BitBlockCounter bit_counter(cond_data, cond.offset, cond.length);
+
+    // selectively copy values from left data
+    T left_data = internal::UnboxScalar<Type>::Unbox(left);
+    int64_t offset = cond.offset;
+
+    // todo this can be improved by intrinsics. ex: _mm*_mask_store_e* (vmovdqa*)
+    while (offset < cond.offset + cond.length) {
+      const BitBlockCount& block = bit_counter.NextWord();
+      if (block.AllSet()) {  // all from left
+        std::fill(out_values, out_values + block.length, left_data);
+      } else if (block.popcount) {  // selectively copy from left
+        for (int64_t i = 0; i < block.length; ++i) {
+          if (BitUtil::GetBit(cond_data, offset + i)) {
+            out_values[i] = left_data;
+          }
+        }
+      }
+
+      offset += block.length;
+      out_values += block.length;
+    }
+
+    out->buffers[1] = std::move(out_buf);
+    return Status::OK();
+  }
+};
+
+template <typename Type>
+struct IfElseFunctor<Type, enable_if_boolean<Type>> {
+  // AAA
+  static Status Call(KernelContext* ctx, const ArrayData& cond, const ArrayData& left,
+                     const ArrayData& right, ArrayData* out) {
+    // out_buff = right & ~cond
+    ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Buffer> out_buf,
+                          arrow::internal::BitmapAndNot(
+                              ctx->memory_pool(), right.buffers[1]->data(), right.offset,
+                              cond.buffers[1]->data(), cond.offset, cond.length, 0));
+
+    // out_buff = left & cond
+    ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Buffer> temp_buf,
+                          arrow::internal::BitmapAnd(
+                              ctx->memory_pool(), left.buffers[1]->data(), left.offset,
+                              cond.buffers[1]->data(), cond.offset, cond.length, 0));
+
+    arrow::internal::BitmapOr(out_buf->data(), 0, temp_buf->data(), 0, cond.length, 0,
+                              out_buf->mutable_data());
+    out->buffers[1] = std::move(out_buf);
+    return Status::OK();
+  }
+
+  // ASA
+  static Status Call(KernelContext* ctx, const ArrayData& cond, const Scalar& left,
+                     const ArrayData& right, ArrayData* out) {
+    // out_buff = right & ~cond
+    ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Buffer> out_buf,
+                          arrow::internal::BitmapAndNot(
+                              ctx->memory_pool(), right.buffers[1]->data(), right.offset,
+                              cond.buffers[1]->data(), cond.offset, cond.length, 0));
+
+    // out_buff = left & cond
+    bool left_data = internal::UnboxScalar<BooleanType>::Unbox(left);
+    if (left_data) {
+      arrow::internal::BitmapOr(out_buf->data(), 0, cond.buffers[1]->data(), cond.offset,
+                                cond.length, 0, out_buf->mutable_data());
+    }
+
+    out->buffers[1] = std::move(out_buf);
+    return Status::OK();
+  }
+
+  // AAS
+  static Status Call(KernelContext* ctx, const ArrayData& cond, const ArrayData& left,
+                     const Scalar& right, ArrayData* out) {
+    // out_buff = left & cond
+    ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Buffer> out_buf,
+                          arrow::internal::BitmapAnd(
+                              ctx->memory_pool(), left.buffers[1]->data(), left.offset,
+                              cond.buffers[1]->data(), cond.offset, cond.length, 0));
+
+    bool right_data = internal::UnboxScalar<BooleanType>::Unbox(right);
+
+    // out_buff = left & cond | right & ~cond
+    if (right_data) {
+      arrow::internal::BitmapOrNot(out_buf->data(), 0, cond.buffers[1]->data(),
+                                   cond.offset, cond.length, 0, out_buf->mutable_data());
+    }
+
+    out->buffers[1] = std::move(out_buf);
+    return Status::OK();
+  }
+
+  // ASS
+  static Status Call(KernelContext* ctx, const ArrayData& cond, const Scalar& left,
+                     const Scalar& right, ArrayData* out) {
+    bool left_data = internal::UnboxScalar<BooleanType>::Unbox(left);
+    bool right_data = internal::UnboxScalar<BooleanType>::Unbox(right);
+
+    // out_buf = left & cond | right & ~cond
+    std::shared_ptr<Buffer> out_buf = nullptr;
+    if (left_data) {
+      if (right_data) {
+        // out_buf = ones
+        ARROW_ASSIGN_OR_RAISE(out_buf, ctx->AllocateBitmap(cond.length));
+        // filling with UINT8_MAX upto the buffer's size (in bytes)
+        arrow::compute::internal::SetMemory<UINT8_MAX>(out_buf.get());
+      } else {
+        // out_buf = cond
+        out_buf = SliceBuffer(cond.buffers[1], cond.offset, cond.length);
+      }
+    } else {
+      if (right_data) {
+        // out_buf = ~cond
+        ARROW_ASSIGN_OR_RAISE(out_buf, arrow::internal::InvertBitmap(
+                                           ctx->memory_pool(), cond.buffers[1]->data(),
+                                           cond.offset, cond.length))
+      } else {
+        // out_buf = zeros
+        ARROW_ASSIGN_OR_RAISE(out_buf, ctx->AllocateBitmap(cond.length));
+      }
+    }
+    out->buffers[1] = std::move(out_buf);
+    return Status::OK();
+  }
+};
+
+template <typename Type>
+struct IfElseFunctor<Type, enable_if_null<Type>> {
+  template <typename T>
+  static inline Status ReturnCopy(const T& in, T* out) {
+    // Nothing preallocated, so we assign in into the output
+    *out = in;
+    return Status::OK();
+  }
+
+  // AAA
+  static Status Call(KernelContext* ctx, const ArrayData& cond, const ArrayData& left,
+                     const ArrayData& right, ArrayData* out) {
+    return ReturnCopy(left, out);
+  }
+
+  // ASA
+  static Status Call(KernelContext* ctx, const ArrayData& cond, const Scalar& left,
+                     const ArrayData& right, ArrayData* out) {
+    return ReturnCopy(right, out);
+  }
+
+  // AAS
+  static Status Call(KernelContext* ctx, const ArrayData& cond, const ArrayData& left,
+                     const Scalar& right, ArrayData* out) {
+    return ReturnCopy(left, out);
+  }
+
+  // ASS
+  static Status Call(KernelContext* ctx, const ArrayData& cond, const Scalar& left,
+                     const Scalar& right, ArrayData* out) {
+    return ReturnCopy(cond, out);
+  }
+};
+
+template <typename Type>
+struct ResolveIfElseExec {
+  static Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    // cond is scalar
+    if (batch[0].is_scalar()) {
+      const auto& cond = batch[0].scalar_as<BooleanScalar>();
+      if (batch[1].is_scalar() && batch[2].is_scalar()) {
+        if (cond.is_valid) {
+          *out = cond.value ? batch[1].scalar() : batch[2].scalar();
+        } else {
+          *out = MakeNullScalar(batch[1].type());
+        }
+        return Status::OK();
+      } else {  // either left or right is an array. output is always an array
+        // output size is the size of the array arg
+        auto bcast_size = batch.length;
+        if (cond.is_valid) {
+          const auto& valid_data = cond.value ? batch[1] : batch[2];
+          if (valid_data.is_array()) {
+            *out = valid_data;
+          } else {  // valid data is a scalar that needs to be broadcasted
+            ARROW_ASSIGN_OR_RAISE(*out,
+                                  MakeArrayFromScalar(*valid_data.scalar(), bcast_size,
+                                                      ctx->memory_pool()));
+          }
+        } else {  // cond is null. create null array
+          ARROW_ASSIGN_OR_RAISE(
+              *out, MakeArrayOfNull(batch[1].type(), bcast_size, ctx->memory_pool()))
+        }
+        return Status::OK();
+      }
+    }

Review comment:
       @bkietz would this work even when these `*_const` dont have any values in them?
   ```c++
   if (cond_const == kAllValid && left_const == kAllValid && right_const == kAllValid) {
     // no nulls in any input: don't bother allocating a null bitmap
     return Status::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.

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



[GitHub] [arrow] lidavidm closed pull request #10410: ARROW-10640: [C++] A, "if_else" ("where") kernel to combine two arrays based on a mask

Posted by GitBox <gi...@apache.org>.
lidavidm closed pull request #10410:
URL: https://github.com/apache/arrow/pull/10410


   


-- 
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



[GitHub] [arrow] lidavidm commented on a change in pull request #10410: ARROW-10640: [C++] A "where" kernel to combine two arrays based on a mask

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_if_else_test.cc
##########
@@ -0,0 +1,266 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <arrow/array.h>
+#include <arrow/compute/api_scalar.h>
+#include <arrow/compute/kernels/test_util.h>
+#include <arrow/testing/gtest_util.h>
+#include <gtest/gtest.h>
+
+namespace arrow {
+namespace compute {
+
+void CheckIfElseOutput(const Datum& cond, const Datum& left, const Datum& right,
+                       const Datum& expected) {
+  ASSERT_OK_AND_ASSIGN(Datum datum_out, IfElse(cond, left, right));
+  if (datum_out.is_array()) {
+    std::shared_ptr<Array> result = datum_out.make_array();
+    ASSERT_OK(result->ValidateFull());
+    std::shared_ptr<Array> expected_ = expected.make_array();
+    AssertArraysEqual(*expected_, *result, /*verbose=*/true);
+  } else {  // expecting scalar
+    const std::shared_ptr<Scalar>& result = datum_out.scalar();
+    const std::shared_ptr<Scalar>& expected_ = expected.scalar();
+    AssertScalarsEqual(*expected_, *result, /*verbose=*/true);
+  }
+}
+
+class TestIfElseKernel : public ::testing::Test {};
+
+template <typename Type>
+class TestIfElsePrimitive : public ::testing::Test {};
+
+using PrimitiveTypes = ::testing::Types<Int8Type, UInt8Type, Int16Type, UInt16Type,
+                                        Int32Type, UInt32Type, Int64Type, UInt64Type,
+                                        FloatType, DoubleType, Date32Type, Date64Type>;
+
+TYPED_TEST_SUITE(TestIfElsePrimitive, PrimitiveTypes);
+
+TYPED_TEST(TestIfElsePrimitive, IfElseFixedSizeRand) {
+  using ArrayType = typename TypeTraits<TypeParam>::ArrayType;
+  auto type = TypeTraits<TypeParam>::type_singleton();
+
+  random::RandomArrayGenerator rand(/*seed=*/0);
+  int64_t len = 1000;
+  auto cond = std::static_pointer_cast<BooleanArray>(
+      rand.ArrayOf(boolean(), len, /*null_probability=*/0.01));
+  auto left = std::static_pointer_cast<ArrayType>(
+      rand.ArrayOf(type, len, /*null_probability=*/0.01));
+  auto right = std::static_pointer_cast<ArrayType>(
+      rand.ArrayOf(type, len, /*null_probability=*/0.01));
+
+  typename TypeTraits<TypeParam>::BuilderType builder;
+
+  for (int64_t i = 0; i < len; ++i) {
+    if (!cond->IsValid(i) || (cond->Value(i) && !left->IsValid(i)) ||
+        (!cond->Value(i) && !right->IsValid(i))) {
+      ASSERT_OK(builder.AppendNull());
+      continue;
+    }
+
+    if (cond->Value(i)) {
+      ASSERT_OK(builder.Append(left->Value(i)));
+    } else {
+      ASSERT_OK(builder.Append(right->Value(i)));
+    }
+  }
+  ASSERT_OK_AND_ASSIGN(auto expected_data, builder.Finish());
+
+  CheckIfElseOutput(cond, left, right, expected_data);
+}
+
+template <typename Type>
+struct DatumWrapper {
+  using CType = typename TypeTraits<Type>::CType;
+  using ArrayType = typename TypeTraits<Type>::ArrayType;
+  using ScalarType = typename TypeTraits<Type>::ScalarType;
+
+  util::Variant<std::shared_ptr<ScalarType>, std::shared_ptr<ArrayType>> datum;
+  bool is_scalar;
+
+  explicit DatumWrapper(const Datum& datum_) : is_scalar(datum_.is_scalar()) {
+    if (is_scalar) {
+      datum = std::move(std::static_pointer_cast<ScalarType>(datum_.scalar()));
+    } else {
+      datum = std::move(std::static_pointer_cast<ArrayType>(datum_.make_array()));
+    }
+  }
+
+  bool IsValid(int64_t i) const {
+    return is_scalar ? util::get<std::shared_ptr<ScalarType>>(datum)->is_valid
+                     : util::get<std::shared_ptr<ArrayType>>(datum)->IsValid(i);
+  }
+
+  CType Value(int64_t i) const {
+    return is_scalar ? util::get<std::shared_ptr<ScalarType>>(datum)->value
+                     : util::get<std::shared_ptr<ArrayType>>(datum)->Value(i);
+  }
+};
+
+template <typename Type>
+void GenerateExpected(const Datum& cond, const Datum& left, const Datum& right,
+                      Datum* out) {
+  int64_t len = cond.is_array() ? cond.length()
+                                : left.is_array() ? left.length()
+                                                  : right.is_array() ? right.length() : 1;
+
+  DatumWrapper<BooleanType> cond_(cond);
+  DatumWrapper<Type> left_(left);
+  DatumWrapper<Type> right_(right);
+
+  int64_t i = 0;
+
+  // if all scalars
+  if (cond.is_scalar() && left.is_scalar() && right.is_scalar()) {
+    if (!cond_.IsValid(i) || (cond_.Value(i) && !left_.IsValid(i)) ||
+        (!cond_.Value(i) && !right_.IsValid(i))) {
+      *out = MakeNullScalar(left.type());
+      return;
+    }
+
+    if (cond_.Value(i)) {
+      *out = left;
+      return;
+    } else {
+      *out = right;
+      return;
+    }
+  }
+
+  typename TypeTraits<Type>::BuilderType builder;
+
+  for (; i < len; ++i) {
+    if (!cond_.IsValid(i) || (cond_.Value(i) && !left_.IsValid(i)) ||
+        (!cond_.Value(i) && !right_.IsValid(i))) {
+      ASSERT_OK(builder.AppendNull());
+      continue;
+    }
+
+    if (cond_.Value(i)) {
+      ASSERT_OK(builder.Append(left_.Value(i)));
+    } else {
+      ASSERT_OK(builder.Append(right_.Value(i)));
+    }
+  }
+  ASSERT_OK_AND_ASSIGN(auto expected_data, builder.Finish());
+
+  *out = expected_data;
+}
+
+TYPED_TEST(TestIfElsePrimitive, IfElseFixedSizeGen) {
+  auto type = TypeTraits<TypeParam>::type_singleton();
+
+  std::vector<Datum> cond_datums{ArrayFromJSON(boolean(), "[true, true, true, false]"),
+                                 ArrayFromJSON(boolean(), "[true, null, true, false]"),
+                                 MakeScalar(boolean(), true).ValueOrDie(),
+                                 MakeNullScalar(boolean())};
+
+  std::vector<Datum> left_datums{
+      ArrayFromJSON(type, "[1, 2, 3, 4]"), ArrayFromJSON(type, "[1, 2, null, 4]"),
+      MakeScalar(type, 100).ValueOrDie(), MakeNullScalar(type)};
+
+  std::vector<Datum> right_datums{
+      ArrayFromJSON(type, "[5, 6, 7, 8]"), ArrayFromJSON(type, "[5, 6, 7, null]"),
+      MakeScalar(type, 111).ValueOrDie(), MakeNullScalar(type)};
+
+  for (auto&& cond : cond_datums) {
+    for (auto&& left : left_datums) {
+      for (auto&& right : right_datums) {
+        Datum exp;
+        GenerateExpected<TypeParam>(cond, left, right, &exp);
+        CheckIfElseOutput(cond, left, right, exp);
+      }
+    }
+  }
+}
+
+TEST_F(TestIfElseKernel, IfElseBooleanGen) {
+  auto type = boolean();
+
+  std::vector<Datum> cond_datums{ArrayFromJSON(boolean(), "[true, true, true, false]"),
+                                 ArrayFromJSON(boolean(), "[true, true, null, false]"),
+                                 MakeScalar(boolean(), true).ValueOrDie(),
+                                 MakeNullScalar(boolean())};
+
+  std::vector<Datum> left_datums{ArrayFromJSON(type, "[false, false, false, false]"),
+                                 ArrayFromJSON(type, "[false, false, null, false]"),
+                                 MakeScalar(type, false).ValueOrDie(),
+                                 MakeNullScalar(type)};
+
+  std::vector<Datum> right_datums{ArrayFromJSON(type, "[true, true, true, true]"),
+                                  ArrayFromJSON(type, "[true, true, true, null]"),
+                                  MakeScalar(type, true).ValueOrDie(),
+                                  MakeNullScalar(type)};
+
+  for (auto&& cond : cond_datums) {
+    for (auto&& left : left_datums) {
+      for (auto&& right : right_datums) {
+        Datum exp;
+        GenerateExpected<BooleanType>(cond, left, right, &exp);
+        CheckIfElseOutput(cond, left, right, exp);
+      }
+    }
+  }
+}
+
+TYPED_TEST(TestIfElsePrimitive, IfElseBooleanRand) {
+  auto type = boolean();
+  random::RandomArrayGenerator rand(/*seed=*/0);
+  int64_t len = 1000;
+  auto cond = std::static_pointer_cast<BooleanArray>(
+      rand.ArrayOf(boolean(), len, /*null_probability=*/0.01));
+  auto left = std::static_pointer_cast<BooleanArray>(
+      rand.ArrayOf(type, len, /*null_probability=*/0.01));
+  auto right = std::static_pointer_cast<BooleanArray>(
+      rand.ArrayOf(type, len, /*null_probability=*/0.01));
+
+  BooleanBuilder builder;
+  for (int64_t i = 0; i < len; ++i) {
+    if (!cond->IsValid(i) || (cond->Value(i) && !left->IsValid(i)) ||
+        (!cond->Value(i) && !right->IsValid(i))) {
+      ASSERT_OK(builder.AppendNull());
+      continue;
+    }
+
+    if (cond->Value(i)) {
+      ASSERT_OK(builder.Append(left->Value(i)));
+    } else {
+      ASSERT_OK(builder.Append(right->Value(i)));
+    }
+  }
+  ASSERT_OK_AND_ASSIGN(auto expected_data, builder.Finish());
+
+  CheckIfElseOutput(cond, left, right, expected_data);
+}
+
+TEST_F(TestIfElseKernel, IfElseNull) {
+  CheckIfElseOutput(ArrayFromJSON(boolean(), "[null, null, null, null]"),
+                    ArrayFromJSON(null(), "[null, null, null, null]"),
+                    ArrayFromJSON(null(), "[null, null, null, null]"),
+                    ArrayFromJSON(null(), "[null, null, null, null]"));
+}
+
+TEST_F(TestIfElseKernel, IfElseWithOffset) {

Review comment:
       You could also do this in the random tests by testing in a loop, say, every offset from 0 to 8 at the end.

##########
File path: cpp/src/arrow/compute/kernels/scalar_if_else_test.cc
##########
@@ -0,0 +1,266 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <arrow/array.h>
+#include <arrow/compute/api_scalar.h>
+#include <arrow/compute/kernels/test_util.h>
+#include <arrow/testing/gtest_util.h>
+#include <gtest/gtest.h>
+
+namespace arrow {
+namespace compute {
+
+void CheckIfElseOutput(const Datum& cond, const Datum& left, const Datum& right,
+                       const Datum& expected) {
+  ASSERT_OK_AND_ASSIGN(Datum datum_out, IfElse(cond, left, right));
+  if (datum_out.is_array()) {
+    std::shared_ptr<Array> result = datum_out.make_array();
+    ASSERT_OK(result->ValidateFull());
+    std::shared_ptr<Array> expected_ = expected.make_array();
+    AssertArraysEqual(*expected_, *result, /*verbose=*/true);
+  } else {  // expecting scalar
+    const std::shared_ptr<Scalar>& result = datum_out.scalar();
+    const std::shared_ptr<Scalar>& expected_ = expected.scalar();
+    AssertScalarsEqual(*expected_, *result, /*verbose=*/true);
+  }
+}
+
+class TestIfElseKernel : public ::testing::Test {};
+
+template <typename Type>
+class TestIfElsePrimitive : public ::testing::Test {};
+
+using PrimitiveTypes = ::testing::Types<Int8Type, UInt8Type, Int16Type, UInt16Type,
+                                        Int32Type, UInt32Type, Int64Type, UInt64Type,
+                                        FloatType, DoubleType, Date32Type, Date64Type>;
+
+TYPED_TEST_SUITE(TestIfElsePrimitive, PrimitiveTypes);
+
+TYPED_TEST(TestIfElsePrimitive, IfElseFixedSizeRand) {
+  using ArrayType = typename TypeTraits<TypeParam>::ArrayType;
+  auto type = TypeTraits<TypeParam>::type_singleton();
+
+  random::RandomArrayGenerator rand(/*seed=*/0);
+  int64_t len = 1000;
+  auto cond = std::static_pointer_cast<BooleanArray>(
+      rand.ArrayOf(boolean(), len, /*null_probability=*/0.01));
+  auto left = std::static_pointer_cast<ArrayType>(
+      rand.ArrayOf(type, len, /*null_probability=*/0.01));
+  auto right = std::static_pointer_cast<ArrayType>(
+      rand.ArrayOf(type, len, /*null_probability=*/0.01));
+
+  typename TypeTraits<TypeParam>::BuilderType builder;
+
+  for (int64_t i = 0; i < len; ++i) {
+    if (!cond->IsValid(i) || (cond->Value(i) && !left->IsValid(i)) ||
+        (!cond->Value(i) && !right->IsValid(i))) {
+      ASSERT_OK(builder.AppendNull());
+      continue;
+    }
+
+    if (cond->Value(i)) {
+      ASSERT_OK(builder.Append(left->Value(i)));
+    } else {
+      ASSERT_OK(builder.Append(right->Value(i)));
+    }
+  }
+  ASSERT_OK_AND_ASSIGN(auto expected_data, builder.Finish());
+
+  CheckIfElseOutput(cond, left, right, expected_data);
+}
+
+template <typename Type>
+struct DatumWrapper {
+  using CType = typename TypeTraits<Type>::CType;
+  using ArrayType = typename TypeTraits<Type>::ArrayType;
+  using ScalarType = typename TypeTraits<Type>::ScalarType;
+
+  util::Variant<std::shared_ptr<ScalarType>, std::shared_ptr<ArrayType>> datum;
+  bool is_scalar;
+
+  explicit DatumWrapper(const Datum& datum_) : is_scalar(datum_.is_scalar()) {
+    if (is_scalar) {
+      datum = std::move(std::static_pointer_cast<ScalarType>(datum_.scalar()));
+    } else {
+      datum = std::move(std::static_pointer_cast<ArrayType>(datum_.make_array()));
+    }
+  }
+
+  bool IsValid(int64_t i) const {
+    return is_scalar ? util::get<std::shared_ptr<ScalarType>>(datum)->is_valid
+                     : util::get<std::shared_ptr<ArrayType>>(datum)->IsValid(i);
+  }
+
+  CType Value(int64_t i) const {
+    return is_scalar ? util::get<std::shared_ptr<ScalarType>>(datum)->value
+                     : util::get<std::shared_ptr<ArrayType>>(datum)->Value(i);
+  }
+};
+
+template <typename Type>
+void GenerateExpected(const Datum& cond, const Datum& left, const Datum& right,

Review comment:
       nit: why not just return the scalar?

##########
File path: cpp/src/arrow/compute/kernels/scalar_if_else.cc
##########
@@ -0,0 +1,577 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <arrow/compute/api.h>
+#include <arrow/compute/kernels/codegen_internal.h>
+#include <arrow/compute/util_internal.h>
+#include <arrow/util/bit_block_counter.h>
+#include <arrow/util/bitmap.h>
+#include <arrow/util/bitmap_ops.h>
+
+namespace arrow {
+using internal::BitBlockCount;
+using internal::BitBlockCounter;
+using internal::Bitmap;
+
+namespace compute {
+
+namespace {
+
+util::optional<uint64_t> GetConstantValidityWord(const Datum& data) {
+  if (data.is_scalar()) {
+    return data.scalar()->is_valid ? ~uint64_t(0) : uint64_t(0);
+  }
+
+  if (data.array()->null_count == data.array()->length) return uint64_t(0);
+
+  if (!data.array()->MayHaveNulls()) return ~uint64_t(0);
+
+  // no constant validity word available
+  return {};
+}
+
+inline Bitmap GetBitmap(const Datum& datum, int i) {
+  if (datum.is_scalar()) return {};
+  const ArrayData& a = *datum.array();
+  return Bitmap{a.buffers[i], a.offset, a.length};
+}
+
+// if the condition is null then output is null otherwise we take validity from the
+// selected argument
+// ie. cond.valid & (cond.data & left.valid | ~cond.data & right.valid)
+Status PromoteNullsVisitor(KernelContext* ctx, const Datum& cond_d, const Datum& left_d,
+                           const Datum& right_d, ArrayData* output) {
+  auto cond_const = GetConstantValidityWord(cond_d);
+  auto left_const = GetConstantValidityWord(left_d);
+  auto right_const = GetConstantValidityWord(right_d);
+
+  enum { COND_CONST = 1, LEFT_CONST = 2, RIGHT_CONST = 4 };
+  auto flag = COND_CONST * cond_const.has_value() | LEFT_CONST * left_const.has_value() |
+              RIGHT_CONST * right_const.has_value();
+
+  const ArrayData& cond = *cond_d.array();
+  // cond.data will always be available
+  Bitmap cond_data{cond.buffers[1], cond.offset, cond.length};
+  Bitmap cond_valid{cond.buffers[0], cond.offset, cond.length};
+  Bitmap left_valid = GetBitmap(left_d, 0);
+  Bitmap right_valid = GetBitmap(right_d, 0);
+  // sometimes Bitmaps will be ignored, in which case we replace access to them with
+  // duplicated (probably elided) access to cond_data
+  const Bitmap& _ = cond_data;
+
+  // lambda function that will be used inside the visitor
+  uint64_t* out_validity = nullptr;
+  int64_t i = 0;
+  auto apply = [&](uint64_t c_valid, uint64_t c_data, uint64_t l_valid,
+                   uint64_t r_valid) {
+    out_validity[i] = c_valid & ((c_data & l_valid) | (~c_data & r_valid));
+    i++;
+  };
+
+  // cond.valid & (cond.data & left.valid | ~cond.data & right.valid)
+  // In the following cases, we dont need to allocate out_valid bitmap
+  switch (flag) {
+    case COND_CONST | LEFT_CONST | RIGHT_CONST:
+      // if cond & left & right all ones, then output is all valid --> out_valid = nullptr
+      if ((*cond_const & *left_const & *right_const) == UINT64_MAX) {
+        return Status::OK();
+      }

Review comment:
       Can we put `// fallthrough` here to make it explicit that this is intentional?




-- 
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



[GitHub] [arrow] lidavidm commented on a change in pull request #10410: ARROW-10640: [C++] A "where" kernel to combine two arrays based on a mask

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_if_else.cc
##########
@@ -0,0 +1,572 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <arrow/compute/api.h>
+#include <arrow/compute/kernels/codegen_internal.h>
+#include <arrow/compute/util_internal.h>
+#include <arrow/util/bit_block_counter.h>
+#include <arrow/util/bitmap.h>
+#include <arrow/util/bitmap_ops.h>
+
+namespace arrow {
+using internal::BitBlockCount;
+using internal::BitBlockCounter;
+using internal::Bitmap;
+
+namespace compute {
+
+namespace {
+
+util::optional<uint64_t> GetConstantValidityWord(const Datum& data) {
+  if (data.is_scalar()) {
+    return data.scalar()->is_valid ? ~uint64_t(0) : uint64_t(0);
+  }
+
+  if (data.array()->null_count == data.array()->length) return uint64_t(0);
+
+  if (!data.array()->MayHaveNulls()) return ~uint64_t(0);
+
+  // no constant validity word available
+  return {};
+}
+
+inline Bitmap GetBitmap(const Datum& datum, int i) {
+  if (datum.is_scalar()) return {};
+  const ArrayData& a = *datum.array();
+  return Bitmap{a.buffers[i], a.offset, a.length};
+}
+
+// if the condition is null then output is null otherwise we take validity from the
+// selected argument
+// ie. cond.valid & (cond.data & left.valid | ~cond.data & right.valid)
+Status PromoteNullsVisitor(KernelContext* ctx, const Datum& cond_d, const Datum& left_d,
+                           const Datum& right_d, ArrayData* output) {
+  auto cond_const = GetConstantValidityWord(cond_d);
+  auto left_const = GetConstantValidityWord(left_d);
+  auto right_const = GetConstantValidityWord(right_d);
+
+  enum { COND_CONST = 1, LEFT_CONST = 2, RIGHT_CONST = 4 };
+  auto flag = COND_CONST * cond_const.has_value() | LEFT_CONST * left_const.has_value() |
+              RIGHT_CONST * right_const.has_value();
+
+  const ArrayData& cond = *cond_d.array();
+  // cond.data will always be available
+  Bitmap cond_data{cond.buffers[1], cond.offset, cond.length};
+  Bitmap cond_valid{cond.buffers[0], cond.offset, cond.length};
+  Bitmap left_valid = GetBitmap(left_d, 0);
+  Bitmap right_valid = GetBitmap(right_d, 0);
+  // sometimes Bitmaps will be ignored, in which case we replace access to them with
+  // duplicated (probably elided) access to cond_data
+  const Bitmap& _ = cond_data;
+
+  // lambda function that will be used inside the visitor
+  uint64_t* out_validity = nullptr;
+  int64_t i = 0;
+  auto apply = [&](uint64_t c_valid, uint64_t c_data, uint64_t l_valid,
+                   uint64_t r_valid) {
+    out_validity[i] = c_valid & ((c_data & l_valid) | (~c_data & r_valid));
+    i++;
+  };
+
+  // if the condition is null then output is null otherwise we take validity from the
+  // selected argument
+  // ie. cond.valid & (cond.data & left.valid | ~cond.data & right.valid)
+
+  // In the following cases, we dont need to allocate out_valid bitmap
+  switch (flag) {
+    case COND_CONST | LEFT_CONST | RIGHT_CONST:
+      // if cond & left & right all ones, then output is all valid --> out_valid = nullptr

Review comment:
       Sorry about the confusion - can we explicitly put `// fallthrough` at the end of this case?




-- 
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



[GitHub] [arrow] nirandaperera commented on a change in pull request #10410: ARROW-10640: [C++] A "where" kernel to combine two arrays based on a mask

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_if_else.cc
##########
@@ -0,0 +1,742 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <arrow/compute/api.h>
+#include <arrow/compute/kernels/codegen_internal.h>
+#include <arrow/compute/util_internal.h>
+#include <arrow/util/bit_block_counter.h>
+#include <arrow/util/bitmap.h>
+#include <arrow/util/bitmap_ops.h>
+
+namespace arrow {
+using internal::BitBlockCount;
+using internal::BitBlockCounter;
+using internal::Bitmap;
+
+namespace compute {
+
+namespace {
+
+enum { COND_ALL_VALID = 1, LEFT_ALL_VALID = 2, RIGHT_ALL_VALID = 4 };
+
+// if the condition is null then output is null otherwise we take validity from the
+// selected argument
+// ie. cond.valid & (cond.data & left.valid | ~cond.data & right.valid)
+Status PromoteNullsVisitor(KernelContext* ctx, const ArrayData& cond, const Scalar& left,
+                           const Scalar& right, ArrayData* output) {
+  uint8_t flag = right.is_valid * RIGHT_ALL_VALID |
+                        left.is_valid * LEFT_ALL_VALID |
+                         !cond.MayHaveNulls() * COND_ALL_VALID;
+
+  if (flag < 6 && flag != 3) {
+    // there will be a validity buffer in the output
+    ARROW_ASSIGN_OR_RAISE(output->buffers[0], ctx->AllocateBitmap(cond.length));
+  }
+
+  // if the condition is null then output is null otherwise we take validity from the
+  // selected argument
+  // ie. cond.valid & (cond.data & left.valid | ~cond.data & right.valid)
+  switch (flag) {
+    case COND_ALL_VALID | LEFT_ALL_VALID | RIGHT_ALL_VALID:  // = 7
+      break;
+    case LEFT_ALL_VALID | RIGHT_ALL_VALID:  // = 6
+      // out_valid = c_valid
+      output->buffers[0] = SliceBuffer(cond.buffers[0], cond.offset, cond.length);
+      break;
+    case COND_ALL_VALID | RIGHT_ALL_VALID:  // = 5
+      // out_valid = ~cond.data
+      arrow::internal::InvertBitmap(cond.buffers[1]->data(), cond.offset, cond.length,
+                                    output->buffers[0]->mutable_data(), 0);
+      break;
+    case RIGHT_ALL_VALID:  // = 4
+      // out_valid = c_valid & ~cond.data
+      arrow::internal::BitmapAndNot(cond.buffers[0]->data(), cond.offset,
+                                    cond.buffers[1]->data(), cond.offset, cond.length, 0,
+                                    output->buffers[0]->mutable_data());
+      break;
+    case COND_ALL_VALID | LEFT_ALL_VALID:  // = 3
+      // out_valid = cond.data
+      output->buffers[0] = SliceBuffer(cond.buffers[1], cond.offset, cond.length);
+      break;
+    case LEFT_ALL_VALID:  // = 2
+      // out_valid = cond.valid & cond.data
+      arrow::internal::BitmapAnd(cond.buffers[0]->data(), cond.offset,
+                                 cond.buffers[1]->data(), cond.offset, cond.length, 0,
+                                 output->buffers[0]->mutable_data());
+      break;
+    case COND_ALL_VALID:  // = 1
+      // out_valid = 0 --> nothing to do; but requires out_valid to be a all-zero buffer
+      break;
+    case 0:
+      // out_valid = 0 --> nothing to do; but requires out_valid to be a all-zero buffer
+      break;
+  }
+  return Status::OK();
+}
+
+Status PromoteNullsVisitor(KernelContext* ctx, const ArrayData& cond,
+                           const ArrayData& left, const Scalar& right,
+                           ArrayData* output) {
+  uint8_t flag = right.is_valid * 4 + !left.MayHaveNulls() * 2 + !cond.MayHaveNulls();
+
+  enum { C_VALID, C_DATA, L_VALID };
+
+  Bitmap bitmaps[3];
+  bitmaps[C_VALID] = {cond.buffers[0], cond.offset, cond.length};
+  bitmaps[C_DATA] = {cond.buffers[1], cond.offset, cond.length};
+  bitmaps[L_VALID] = {left.buffers[0], left.offset, left.length};
+
+  uint64_t* out_validity = nullptr;
+  if (flag < 6 && flag != 3) {
+    // there will be a validity buffer in the output
+    ARROW_ASSIGN_OR_RAISE(output->buffers[0], ctx->AllocateBitmap(cond.length));
+    out_validity = output->GetMutableValues<uint64_t>(0);
+  }
+
+  // lambda function that will be used inside the visitor
+  int64_t i = 0;
+  auto apply = [&](uint64_t c_valid, uint64_t c_data, uint64_t l_valid,
+                   uint64_t r_valid) {
+    out_validity[i] = c_valid & ((c_data & l_valid) | (~c_data & r_valid));
+    i++;
+  };
+
+  // if the condition is null then output is null otherwise we take validity from the
+  // selected argument
+  // ie. cond.valid & (cond.data & left.valid | ~cond.data & right.valid)
+  switch (flag) {
+    case COND_ALL_VALID | LEFT_ALL_VALID | RIGHT_ALL_VALID:
+      break;
+    case LEFT_ALL_VALID | RIGHT_ALL_VALID:
+      output->buffers[0] = SliceBuffer(cond.buffers[0], cond.offset, cond.length);
+      break;
+    case COND_ALL_VALID | RIGHT_ALL_VALID:
+      // bitmaps[C_VALID] might be null; override to make it safe for Visit()
+      bitmaps[C_VALID] = bitmaps[C_DATA];
+      Bitmap::VisitWords(bitmaps, [&](std::array<uint64_t, 3> words) {
+        apply(UINT64_MAX, words[C_DATA], words[L_VALID], UINT64_MAX);
+      });
+      break;
+    case RIGHT_ALL_VALID:
+      Bitmap::VisitWords(bitmaps, [&](std::array<uint64_t, 3> words) {
+        apply(words[C_VALID], words[C_DATA], words[L_VALID], UINT64_MAX);
+      });
+      break;
+    case COND_ALL_VALID | LEFT_ALL_VALID:
+      // only cond.data is passed
+      output->buffers[0] = SliceBuffer(cond.buffers[1], cond.offset, cond.length);
+      break;
+    case LEFT_ALL_VALID:
+      // out_valid = cond.valid & cond.data
+      arrow::internal::BitmapAnd(cond.buffers[0]->data(), cond.offset,
+                                 cond.buffers[1]->data(), cond.offset, cond.length, 0,
+                                 output->buffers[0]->mutable_data());
+      break;
+    case COND_ALL_VALID:
+      // out_valid = cond.data & left.valid
+      arrow::internal::BitmapAnd(cond.buffers[1]->data(), cond.offset,
+                                 left.buffers[0]->data(), left.offset, cond.length, 0,
+                                 output->buffers[0]->mutable_data());
+      break;
+    case 0:
+      Bitmap::VisitWords(bitmaps, [&](std::array<uint64_t, 3> words) {
+        apply(words[C_VALID], words[C_DATA], words[L_VALID], 0);
+      });
+      break;
+  }
+  return Status::OK();
+}
+
+// if the condition is null then output is null otherwise we take validity from the
+// selected argument
+// ie. cond.valid & (cond.data & left.valid | ~cond.data & right.valid)
+Status PromoteNullsVisitor(KernelContext* ctx, const ArrayData& cond, const Scalar& left,
+                           const ArrayData& right, ArrayData* output) {
+  uint8_t flag = !right.MayHaveNulls() * 4 + left.is_valid * 2 + !cond.MayHaveNulls();
+
+  enum { C_VALID, C_DATA, R_VALID };
+
+  Bitmap bitmaps[3];
+  bitmaps[C_VALID] = {cond.buffers[0], cond.offset, cond.length};
+  bitmaps[C_DATA] = {cond.buffers[1], cond.offset, cond.length};
+  bitmaps[R_VALID] = {right.buffers[0], right.offset, right.length};
+
+  uint64_t* out_validity = nullptr;
+  if (flag < 6) {
+    // there will be a validity buffer in the output
+    ARROW_ASSIGN_OR_RAISE(output->buffers[0], ctx->AllocateBitmap(cond.length));
+    out_validity = output->GetMutableValues<uint64_t>(0);
+  }
+
+  // lambda function that will be used inside the visitor
+  int64_t i = 0;
+  auto apply = [&](uint64_t c_valid, uint64_t c_data, uint64_t l_valid,
+                   uint64_t r_valid) {
+    out_validity[i] = c_valid & ((c_data & l_valid) | (~c_data & r_valid));
+    i++;
+  };
+
+  // if the condition is null then output is null otherwise we take validity from the
+  // selected argument
+  // ie. cond.valid & (cond.data & left.valid | ~cond.data & right.valid)
+  switch (flag) {
+    case COND_ALL_VALID | LEFT_ALL_VALID | RIGHT_ALL_VALID:
+      break;
+    case LEFT_ALL_VALID | RIGHT_ALL_VALID:
+      output->buffers[0] = SliceBuffer(cond.buffers[0], cond.offset, cond.length);
+      break;
+    case COND_ALL_VALID | RIGHT_ALL_VALID:
+      // out_valid = ~cond.data
+      arrow::internal::InvertBitmap(cond.buffers[1]->data(), cond.offset, cond.length,
+                                    output->buffers[0]->mutable_data(), 0);
+      break;
+    case RIGHT_ALL_VALID:
+      // out_valid = c_valid & ~cond.data
+      arrow::internal::BitmapAndNot(cond.buffers[0]->data(), cond.offset,
+                                    cond.buffers[1]->data(), cond.offset, cond.length, 0,
+                                    output->buffers[0]->mutable_data());
+      break;
+    case COND_ALL_VALID | LEFT_ALL_VALID:
+      // bitmaps[C_VALID] might be null; override to make it safe for Visit()
+      bitmaps[C_VALID] = bitmaps[C_DATA];
+      Bitmap::VisitWords(bitmaps, [&](std::array<uint64_t, 3> words) {
+        apply(UINT64_MAX, words[C_DATA], UINT64_MAX, words[R_VALID]);
+      });
+      break;
+    case LEFT_ALL_VALID:
+      Bitmap::VisitWords(bitmaps, [&](std::array<uint64_t, 3> words) {
+        apply(words[C_VALID], words[C_DATA], UINT64_MAX, words[R_VALID]);
+      });
+      break;
+    case COND_ALL_VALID:
+      // out_valid =  ~cond.data & right.valid
+      arrow::internal::BitmapAndNot(right.buffers[0]->data(), right.offset,
+                                    cond.buffers[1]->data(), cond.offset, cond.length, 0,
+                                    output->buffers[0]->mutable_data());
+      break;
+    case 0:
+      Bitmap::VisitWords(bitmaps, [&](std::array<uint64_t, 3> words) {
+        apply(words[C_VALID], words[C_DATA], 0, words[R_VALID]);
+      });
+      break;
+  }
+  return Status::OK();
+}
+
+// if the condition is null then output is null otherwise we take validity from the
+// selected argument
+// ie. cond.valid & (cond.data & left.valid | ~cond.data & right.valid)
+Status PromoteNullsVisitor(KernelContext* ctx, const ArrayData& cond,
+                           const ArrayData& left, const ArrayData& right,
+                           ArrayData* output) {
+  uint8_t flag =
+      !right.MayHaveNulls() * 4 + !left.MayHaveNulls() * 2 + !cond.MayHaveNulls();
+
+  enum { C_VALID, C_DATA, L_VALID, R_VALID };
+
+  Bitmap bitmaps[4];
+  bitmaps[C_VALID] = {cond.buffers[0], cond.offset, cond.length};
+  bitmaps[C_DATA] = {cond.buffers[1], cond.offset, cond.length};
+  bitmaps[L_VALID] = {left.buffers[0], left.offset, left.length};
+  bitmaps[R_VALID] = {right.buffers[0], right.offset, right.length};
+
+  uint64_t* out_validity = nullptr;
+  if (flag < 6) {
+    // there will be a validity buffer in the output
+    ARROW_ASSIGN_OR_RAISE(output->buffers[0], ctx->AllocateBitmap(cond.length));
+    out_validity = output->GetMutableValues<uint64_t>(0);
+  }
+
+  // lambda function that will be used inside the visitor
+  int64_t i = 0;
+  auto apply = [&](uint64_t c_valid, uint64_t c_data, uint64_t l_valid,
+                   uint64_t r_valid) {
+    out_validity[i] = c_valid & ((c_data & l_valid) | (~c_data & r_valid));
+    i++;
+  };
+
+  // if the condition is null then output is null otherwise we take validity from the
+  // selected argument
+  // ie. cond.valid & (cond.data & left.valid | ~cond.data & right.valid)
+  switch (flag) {
+    case COND_ALL_VALID | LEFT_ALL_VALID | RIGHT_ALL_VALID:
+      break;
+    case LEFT_ALL_VALID | RIGHT_ALL_VALID:
+      output->buffers[0] = SliceBuffer(cond.buffers[0], cond.offset, cond.length);
+      break;
+    case COND_ALL_VALID | RIGHT_ALL_VALID:
+      // bitmaps[C_VALID], bitmaps[R_VALID] might be null; override to make it safe for
+      // Visit()
+      bitmaps[C_VALID] = bitmaps[C_DATA];
+      bitmaps[R_VALID] = bitmaps[C_DATA];
+      Bitmap::VisitWords(bitmaps, [&](std::array<uint64_t, 4> words) {
+        apply(UINT64_MAX, words[C_DATA], words[L_VALID], UINT64_MAX);
+      });
+      break;
+    case RIGHT_ALL_VALID:
+      // bitmaps[R_VALID] might be null; override to make it safe for Visit()
+      bitmaps[R_VALID] = bitmaps[C_DATA];
+      Bitmap::VisitWords(bitmaps, [&](std::array<uint64_t, 4> words) {
+        apply(words[C_VALID], words[C_DATA], words[L_VALID], UINT64_MAX);
+      });
+      break;
+    case COND_ALL_VALID | LEFT_ALL_VALID:
+      // bitmaps[C_VALID], bitmaps[L_VALID] might be null; override to make it safe for
+      // Visit()
+      bitmaps[C_VALID] = bitmaps[C_DATA];
+      bitmaps[L_VALID] = bitmaps[C_DATA];
+      Bitmap::VisitWords(bitmaps, [&](std::array<uint64_t, 4> words) {
+        apply(UINT64_MAX, words[C_DATA], UINT64_MAX, words[R_VALID]);
+      });
+      break;
+    case LEFT_ALL_VALID:
+      // bitmaps[L_VALID] might be null; override to make it safe for Visit()
+      bitmaps[L_VALID] = bitmaps[C_DATA];
+      Bitmap::VisitWords(bitmaps, [&](std::array<uint64_t, 4> words) {
+        apply(words[C_VALID], words[C_DATA], UINT64_MAX, words[R_VALID]);
+      });
+      break;
+    case COND_ALL_VALID:
+      // bitmaps[C_VALID] might be null; override to make it safe for Visit()
+      bitmaps[C_VALID] = bitmaps[C_DATA];
+      Bitmap::VisitWords(bitmaps, [&](std::array<uint64_t, 4> words) {
+        apply(UINT64_MAX, words[C_DATA], words[L_VALID], words[R_VALID]);
+      });
+      break;
+    case 0:
+      Bitmap::VisitWords(bitmaps, [&](std::array<uint64_t, 4> words) {
+        apply(words[C_VALID], words[C_DATA], words[L_VALID], words[R_VALID]);
+      });
+      break;
+  }
+  return Status::OK();
+}
+
+template <typename Type, typename Enable = void>
+struct IfElseFunctor {};
+
+// only number types needs to be handled for Fixed sized primitive data types because,
+// internal::GenerateTypeAgnosticPrimitive forwards types to the corresponding unsigned
+// int type
+template <typename Type>
+struct IfElseFunctor<Type, enable_if_number<Type>> {
+  using T = typename TypeTraits<Type>::CType;
+  // A - Array
+  // S - Scalar
+
+  //  AAA
+  static Status Call(KernelContext* ctx, const ArrayData& cond, const ArrayData& left,
+                     const ArrayData& right, ArrayData* out) {
+    ARROW_RETURN_NOT_OK(PromoteNullsVisitor(ctx, cond, left, right, out));
+
+    ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Buffer> out_buf,
+                          ctx->Allocate(cond.length * sizeof(T)));
+    T* out_values = reinterpret_cast<T*>(out_buf->mutable_data());
+
+    // copy right data to out_buff
+    const T* right_data = right.GetValues<T>(1);
+    std::memcpy(out_values, right_data, right.length * sizeof(T));
+
+    const auto* cond_data = cond.buffers[1]->data();  // this is a BoolArray
+    BitBlockCounter bit_counter(cond_data, cond.offset, cond.length);
+
+    // selectively copy values from left data
+    const T* left_data = left.GetValues<T>(1);
+    int64_t offset = cond.offset;
+
+    // todo this can be improved by intrinsics. ex: _mm*_mask_store_e* (vmovdqa*)
+    while (offset < cond.offset + cond.length) {
+      const BitBlockCount& block = bit_counter.NextWord();
+      if (block.AllSet()) {  // all from left
+        std::memcpy(out_values, left_data, block.length * sizeof(T));
+      } else if (block.popcount) {  // selectively copy from left
+        for (int64_t i = 0; i < block.length; ++i) {
+          if (BitUtil::GetBit(cond_data, offset + i)) {
+            out_values[i] = left_data[i];
+          }
+        }
+      }
+
+      offset += block.length;
+      out_values += block.length;
+      left_data += block.length;
+    }
+
+    out->buffers[1] = std::move(out_buf);
+    return Status::OK();
+  }
+
+  // ASA
+  static Status Call(KernelContext* ctx, const ArrayData& cond, const Scalar& left,
+                     const ArrayData& right, ArrayData* out) {
+    ARROW_RETURN_NOT_OK(PromoteNullsVisitor(ctx, cond, left, right, out));
+
+    ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Buffer> out_buf,
+                          ctx->Allocate(cond.length * sizeof(T)));
+    T* out_values = reinterpret_cast<T*>(out_buf->mutable_data());
+
+    // copy right data to out_buff
+    const T* right_data = right.GetValues<T>(1);
+    std::memcpy(out_values, right_data, right.length * sizeof(T));
+
+    const auto* cond_data = cond.buffers[1]->data();  // this is a BoolArray
+    BitBlockCounter bit_counter(cond_data, cond.offset, cond.length);
+
+    // selectively copy values from left data
+    T left_data = internal::UnboxScalar<Type>::Unbox(left);
+    int64_t offset = cond.offset;
+
+    // todo this can be improved by intrinsics. ex: _mm*_mask_store_e* (vmovdqa*)
+    while (offset < cond.offset + cond.length) {
+      const BitBlockCount& block = bit_counter.NextWord();
+      if (block.AllSet()) {  // all from left
+        std::fill(out_values, out_values + block.length, left_data);
+      } else if (block.popcount) {  // selectively copy from left
+        for (int64_t i = 0; i < block.length; ++i) {
+          if (BitUtil::GetBit(cond_data, offset + i)) {
+            out_values[i] = left_data;
+          }
+        }
+      }
+
+      offset += block.length;
+      out_values += block.length;
+    }
+
+    out->buffers[1] = std::move(out_buf);
+    return Status::OK();
+  }
+
+  // AAS
+  static Status Call(KernelContext* ctx, const ArrayData& cond, const ArrayData& left,
+                     const Scalar& right, ArrayData* out) {
+    ARROW_RETURN_NOT_OK(PromoteNullsVisitor(ctx, cond, left, right, out));
+
+    ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Buffer> out_buf,
+                          ctx->Allocate(cond.length * sizeof(T)));
+    T* out_values = reinterpret_cast<T*>(out_buf->mutable_data());
+
+    // copy left data to out_buff
+    const T* left_data = left.GetValues<T>(1);
+    std::memcpy(out_values, left_data, left.length * sizeof(T));
+
+    const auto* cond_data = cond.buffers[1]->data();  // this is a BoolArray
+    BitBlockCounter bit_counter(cond_data, cond.offset, cond.length);
+
+    // selectively copy values from left data
+    T right_data = internal::UnboxScalar<Type>::Unbox(right);
+    int64_t offset = cond.offset;
+
+    // todo this can be improved by intrinsics. ex: _mm*_mask_store_e* (vmovdqa*)
+    // left data is already in the output buffer. Therefore, mask needs to be inverted
+    while (offset < cond.offset + cond.length) {
+      const BitBlockCount& block = bit_counter.NextWord();
+      if (block.NoneSet()) {  // all from right
+        std::fill(out_values, out_values + block.length, right_data);
+      } else if (block.popcount) {  // selectively copy from right
+        for (int64_t i = 0; i < block.length; ++i) {
+          if (!BitUtil::GetBit(cond_data, offset + i)) {
+            out_values[i] = right_data;
+          }
+        }
+      }
+
+      offset += block.length;
+      out_values += block.length;
+    }
+
+    out->buffers[1] = std::move(out_buf);
+    return Status::OK();
+  }
+
+  // ASS
+  static Status Call(KernelContext* ctx, const ArrayData& cond, const Scalar& left,
+                     const Scalar& right, ArrayData* out) {
+    ARROW_RETURN_NOT_OK(PromoteNullsVisitor(ctx, cond, left, right, out));
+
+    ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Buffer> out_buf,
+                          ctx->Allocate(cond.length * sizeof(T)));
+    T* out_values = reinterpret_cast<T*>(out_buf->mutable_data());
+
+    // copy right data to out_buff
+    T right_data = internal::UnboxScalar<Type>::Unbox(right);
+    std::fill(out_values, out_values + cond.length, right_data);
+
+    const auto* cond_data = cond.buffers[1]->data();  // this is a BoolArray
+    BitBlockCounter bit_counter(cond_data, cond.offset, cond.length);
+
+    // selectively copy values from left data
+    T left_data = internal::UnboxScalar<Type>::Unbox(left);
+    int64_t offset = cond.offset;
+
+    // todo this can be improved by intrinsics. ex: _mm*_mask_store_e* (vmovdqa*)
+    while (offset < cond.offset + cond.length) {
+      const BitBlockCount& block = bit_counter.NextWord();
+      if (block.AllSet()) {  // all from left
+        std::fill(out_values, out_values + block.length, left_data);
+      } else if (block.popcount) {  // selectively copy from left
+        for (int64_t i = 0; i < block.length; ++i) {
+          if (BitUtil::GetBit(cond_data, offset + i)) {
+            out_values[i] = left_data;
+          }
+        }
+      }
+
+      offset += block.length;
+      out_values += block.length;
+    }
+
+    out->buffers[1] = std::move(out_buf);
+    return Status::OK();
+  }
+};
+
+template <typename Type>
+struct IfElseFunctor<Type, enable_if_boolean<Type>> {
+  // AAA
+  static Status Call(KernelContext* ctx, const ArrayData& cond, const ArrayData& left,
+                     const ArrayData& right, ArrayData* out) {
+    ARROW_RETURN_NOT_OK(PromoteNullsVisitor(ctx, cond, left, right, out));
+
+    // out_buff = right & ~cond
+    ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Buffer> out_buf,
+                          arrow::internal::BitmapAndNot(
+                              ctx->memory_pool(), right.buffers[1]->data(), right.offset,
+                              cond.buffers[1]->data(), cond.offset, cond.length, 0));
+
+    // out_buff = left & cond
+    ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Buffer> temp_buf,
+                          arrow::internal::BitmapAnd(
+                              ctx->memory_pool(), left.buffers[1]->data(), left.offset,
+                              cond.buffers[1]->data(), cond.offset, cond.length, 0));
+
+    arrow::internal::BitmapOr(out_buf->data(), 0, temp_buf->data(), 0, cond.length, 0,
+                              out_buf->mutable_data());
+    out->buffers[1] = std::move(out_buf);
+    return Status::OK();
+  }
+
+  // ASA
+  static Status Call(KernelContext* ctx, const ArrayData& cond, const Scalar& left,
+                     const ArrayData& right, ArrayData* out) {
+    ARROW_RETURN_NOT_OK(PromoteNullsVisitor(ctx, cond, left, right, out));
+
+    // out_buff = right & ~cond
+    ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Buffer> out_buf,
+                          arrow::internal::BitmapAndNot(
+                              ctx->memory_pool(), right.buffers[1]->data(), right.offset,
+                              cond.buffers[1]->data(), cond.offset, cond.length, 0));
+
+    // out_buff = left & cond
+    bool left_data = internal::UnboxScalar<BooleanType>::Unbox(left);
+    if (left_data) {
+      arrow::internal::BitmapOr(out_buf->data(), 0, cond.buffers[1]->data(), cond.offset,
+                                cond.length, 0, out_buf->mutable_data());
+    }
+
+    out->buffers[1] = std::move(out_buf);
+    return Status::OK();
+  }
+
+  // AAS
+  static Status Call(KernelContext* ctx, const ArrayData& cond, const ArrayData& left,
+                     const Scalar& right, ArrayData* out) {
+    ARROW_RETURN_NOT_OK(PromoteNullsVisitor(ctx, cond, left, right, out));
+
+    // out_buff = left & cond
+    ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Buffer> out_buf,
+                          arrow::internal::BitmapAnd(
+                              ctx->memory_pool(), left.buffers[1]->data(), left.offset,
+                              cond.buffers[1]->data(), cond.offset, cond.length, 0));
+
+    bool right_data = internal::UnboxScalar<BooleanType>::Unbox(right);
+
+    // out_buff = left & cond | right & ~cond
+    if (right_data) {
+      arrow::internal::BitmapOrNot(out_buf->data(), 0, cond.buffers[1]->data(),
+                                   cond.offset, cond.length, 0, out_buf->mutable_data());
+    }
+
+    out->buffers[1] = std::move(out_buf);
+    return Status::OK();
+  }
+
+  // ASS
+  static Status Call(KernelContext* ctx, const ArrayData& cond, const Scalar& left,
+                     const Scalar& right, ArrayData* out) {
+    ARROW_RETURN_NOT_OK(PromoteNullsVisitor(ctx, cond, left, right, out));
+
+    bool left_data = internal::UnboxScalar<BooleanType>::Unbox(left);
+    bool right_data = internal::UnboxScalar<BooleanType>::Unbox(right);
+
+    // out_buf = left & cond | right & ~cond
+    std::shared_ptr<Buffer> out_buf = nullptr;
+    if (left_data) {
+      if (right_data) {
+        // out_buf = ones
+        ARROW_ASSIGN_OR_RAISE(out_buf, ctx->AllocateBitmap(cond.length));
+        // filling with UINT8_MAX upto the buffer's size (in bytes)
+        arrow::compute::internal::SetMemory<UINT8_MAX>(out_buf.get());
+      } else {
+        // out_buf = cond
+        out_buf = SliceBuffer(cond.buffers[1], cond.offset, cond.length);
+      }
+    } else {
+      if (right_data) {
+        // out_buf = ~cond
+        ARROW_ASSIGN_OR_RAISE(out_buf, arrow::internal::InvertBitmap(
+                                           ctx->memory_pool(), cond.buffers[1]->data(),
+                                           cond.offset, cond.length))
+      } else {
+        // out_buf = zeros
+        ARROW_ASSIGN_OR_RAISE(out_buf, ctx->AllocateBitmap(cond.length));
+      }
+    }
+    out->buffers[1] = std::move(out_buf);
+    return Status::OK();
+  }
+};
+
+template <typename Type>
+struct IfElseFunctor<Type, enable_if_null<Type>> {
+  template <typename T>
+  static inline Status ReturnCopy(const T& in, T* out) {
+    // Nothing preallocated, so we assign in into the output
+    *out = in;
+    return Status::OK();
+  }
+
+  // AAA
+  static Status Call(KernelContext* ctx, const ArrayData& cond, const ArrayData& left,
+                     const ArrayData& right, ArrayData* out) {
+    return ReturnCopy(left, out);
+  }
+
+  // ASA
+  static Status Call(KernelContext* ctx, const ArrayData& cond, const Scalar& left,
+                     const ArrayData& right, ArrayData* out) {
+    return ReturnCopy(right, out);
+  }
+
+  // AAS
+  static Status Call(KernelContext* ctx, const ArrayData& cond, const ArrayData& left,
+                     const Scalar& right, ArrayData* out) {
+    return ReturnCopy(left, out);
+  }
+
+  // ASS
+  static Status Call(KernelContext* ctx, const ArrayData& cond, const Scalar& left,
+                     const Scalar& right, ArrayData* out) {
+    return ReturnCopy(cond, out);
+  }
+};
+
+template <typename Type>
+struct ResolveIfElseExec {
+  static Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    // cond is scalar
+    if (batch[0].is_scalar()) {
+      const auto& cond = batch[0].scalar_as<BooleanScalar>();
+      if (batch[1].is_scalar() && batch[2].is_scalar()) {
+        if (cond.is_valid) {
+          *out = cond.value ? batch[1].scalar() : batch[2].scalar();
+        } else {
+          *out = MakeNullScalar(batch[1].type());
+        }
+      } else {  // either left or right is an array. output is always an array
+        // output size is the size of the array arg
+        int64_t bcast_size = batch[1].is_array() ? batch[1].length() : batch[2].length();

Review comment:
       ah! `ExecBatch::length`... didn't know that until now. :smile: I used this, because `Datum::length` didn't work properly with empty arrays. 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.

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



[GitHub] [arrow] nirandaperera commented on a change in pull request #10410: ARROW-10640: [C++] A "where" kernel to combine two arrays based on a mask

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_if_else.cc
##########
@@ -0,0 +1,572 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <arrow/compute/api.h>
+#include <arrow/compute/kernels/codegen_internal.h>
+#include <arrow/compute/util_internal.h>
+#include <arrow/util/bit_block_counter.h>
+#include <arrow/util/bitmap.h>
+#include <arrow/util/bitmap_ops.h>
+
+namespace arrow {
+using internal::BitBlockCount;
+using internal::BitBlockCounter;
+using internal::Bitmap;
+
+namespace compute {
+
+namespace {
+
+util::optional<uint64_t> GetConstantValidityWord(const Datum& data) {
+  if (data.is_scalar()) {
+    return data.scalar()->is_valid ? ~uint64_t(0) : uint64_t(0);
+  }
+
+  if (data.array()->null_count == data.array()->length) return uint64_t(0);
+
+  if (!data.array()->MayHaveNulls()) return ~uint64_t(0);
+
+  // no constant validity word available
+  return {};
+}
+
+inline Bitmap GetBitmap(const Datum& datum, int i) {
+  if (datum.is_scalar()) return {};
+  const ArrayData& a = *datum.array();
+  return Bitmap{a.buffers[i], a.offset, a.length};
+}
+
+// if the condition is null then output is null otherwise we take validity from the
+// selected argument
+// ie. cond.valid & (cond.data & left.valid | ~cond.data & right.valid)
+Status PromoteNullsVisitor(KernelContext* ctx, const Datum& cond_d, const Datum& left_d,
+                           const Datum& right_d, ArrayData* output) {
+  auto cond_const = GetConstantValidityWord(cond_d);
+  auto left_const = GetConstantValidityWord(left_d);
+  auto right_const = GetConstantValidityWord(right_d);
+
+  enum { COND_CONST = 1, LEFT_CONST = 2, RIGHT_CONST = 4 };
+  auto flag = COND_CONST * cond_const.has_value() | LEFT_CONST * left_const.has_value() |
+              RIGHT_CONST * right_const.has_value();
+
+  const ArrayData& cond = *cond_d.array();
+  // cond.data will always be available
+  Bitmap cond_data{cond.buffers[1], cond.offset, cond.length};
+  Bitmap cond_valid{cond.buffers[0], cond.offset, cond.length};
+  Bitmap left_valid = GetBitmap(left_d, 0);
+  Bitmap right_valid = GetBitmap(right_d, 0);
+  // sometimes Bitmaps will be ignored, in which case we replace access to them with
+  // duplicated (probably elided) access to cond_data
+  const Bitmap& _ = cond_data;
+
+  // lambda function that will be used inside the visitor
+  uint64_t* out_validity = nullptr;
+  int64_t i = 0;
+  auto apply = [&](uint64_t c_valid, uint64_t c_data, uint64_t l_valid,
+                   uint64_t r_valid) {
+    out_validity[i] = c_valid & ((c_data & l_valid) | (~c_data & r_valid));
+    i++;
+  };
+
+  // if the condition is null then output is null otherwise we take validity from the
+  // selected argument
+  // ie. cond.valid & (cond.data & left.valid | ~cond.data & right.valid)
+
+  // In the following cases, we dont need to allocate out_valid bitmap
+  switch (flag) {
+    case COND_CONST | LEFT_CONST | RIGHT_CONST:
+      // if cond & left & right all ones, then output is all valid --> out_valid = nullptr
+      if ((*cond_const & *left_const & *right_const) == UINT64_MAX) {
+        return Status::OK();
+      }
+    case LEFT_CONST | RIGHT_CONST:
+      // if both left and right are valid, no need to calculate out_valid bitmap. Pass
+      // cond validity buffer
+      if ((*left_const & *right_const) == UINT64_MAX) {
+        output->buffers[0] = SliceBuffer(cond.buffers[0], cond.offset, cond.length);

Review comment:
       I see! Thanks David! 




-- 
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



[GitHub] [arrow] bkietz commented on a change in pull request #10410: ARROW-10640: [C++] A "where" kernel to combine two arrays based on a mask

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_if_else_test.cc
##########
@@ -0,0 +1,474 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <arrow/array.h>
+#include <arrow/compute/api_scalar.h>
+#include <arrow/compute/kernels/test_util.h>
+#include <arrow/testing/gtest_util.h>
+#include <gtest/gtest.h>
+
+namespace arrow {
+namespace compute {
+
+void CheckIfElseOutput(const Datum& cond, const Datum& left, const Datum& right,
+                       const Datum& expected) {
+  ASSERT_OK_AND_ASSIGN(Datum datum_out, IfElse(cond, left, right));
+  if (datum_out.is_array()) {
+    std::shared_ptr<Array> result = datum_out.make_array();
+    ASSERT_OK(result->ValidateFull());
+    std::shared_ptr<Array> expected_ = expected.make_array();
+    AssertArraysEqual(*expected_, *result, /*verbose=*/true);
+  } else {  // expecting scalar
+    const std::shared_ptr<Scalar>& result = datum_out.scalar();
+    const std::shared_ptr<Scalar>& expected_ = expected.scalar();
+    AssertScalarsEqual(*expected_, *result, /*verbose=*/true);
+  }
+}
+
+void CheckIfElseOutputAAA(const std::shared_ptr<DataType>& type, const std::string& cond,
+                          const std::string& left, const std::string& right,
+                          const std::string& expected) {
+  const std::shared_ptr<Array>& cond_ = ArrayFromJSON(boolean(), cond);
+  const std::shared_ptr<Array>& left_ = ArrayFromJSON(type, left);
+  const std::shared_ptr<Array>& right_ = ArrayFromJSON(type, right);
+  const std::shared_ptr<Array>& expected_ = ArrayFromJSON(type, expected);
+  CheckIfElseOutput(cond_, left_, right_, expected_);
+}
+
+void CheckIfElseOutputAAS(const std::shared_ptr<DataType>& type, const std::string& cond,
+                          const std::string& left, const std::shared_ptr<Scalar>& right,
+                          const std::string& expected) {
+  const std::shared_ptr<Array>& cond_ = ArrayFromJSON(boolean(), cond);
+  const std::shared_ptr<Array>& left_ = ArrayFromJSON(type, left);
+  const std::shared_ptr<Array>& expected_ = ArrayFromJSON(type, expected);
+  CheckIfElseOutput(cond_, left_, right, expected_);
+}
+
+void CheckIfElseOutputASA(const std::shared_ptr<DataType>& type, const std::string& cond,
+                          const std::shared_ptr<Scalar>& left, const std::string& right,
+                          const std::string& expected) {
+  const std::shared_ptr<Array>& cond_ = ArrayFromJSON(boolean(), cond);
+  const std::shared_ptr<Array>& right_ = ArrayFromJSON(type, right);
+  const std::shared_ptr<Array>& expected_ = ArrayFromJSON(type, expected);
+  CheckIfElseOutput(cond_, left, right_, expected_);
+}
+
+void CheckIfElseOutputASS(const std::shared_ptr<DataType>& type, const std::string& cond,
+                          const std::shared_ptr<Scalar>& left,
+                          const std::shared_ptr<Scalar>& right,
+                          const std::string& expected) {
+  const std::shared_ptr<Array>& cond_ = ArrayFromJSON(boolean(), cond);
+  const std::shared_ptr<Array>& expected_ = ArrayFromJSON(type, expected);
+  CheckIfElseOutput(cond_, left, right, expected_);
+}
+
+void CheckIfElseOutputSAA(const std::shared_ptr<DataType>& type,
+                          const std::shared_ptr<Scalar>& cond, const std::string& left,
+                          const std::string& right, const std::string& expected) {
+  const std::shared_ptr<Array>& left_ = ArrayFromJSON(type, left);
+  const std::shared_ptr<Array>& right_ = ArrayFromJSON(type, right);
+  const std::shared_ptr<Array>& expected_ = ArrayFromJSON(type, expected);
+  CheckIfElseOutput(cond, left_, right_, expected_);
+}
+
+void CheckIfElseOutputSAS(const std::shared_ptr<DataType>& type,
+                          const std::shared_ptr<Scalar>& cond, const std::string& left,
+                          const std::shared_ptr<Scalar>& right,
+                          const std::string& expected) {
+  const std::shared_ptr<Array>& left_ = ArrayFromJSON(type, left);
+  const std::shared_ptr<Array>& expected_ = ArrayFromJSON(type, expected);
+  CheckIfElseOutput(cond, left_, right, expected_);
+}
+
+void CheckIfElseOutputSSA(const std::shared_ptr<DataType>& type,
+                          const std::shared_ptr<Scalar>& cond,
+                          const std::shared_ptr<Scalar>& left, const std::string& right,
+                          const std::string& expected) {
+  const std::shared_ptr<Array>& right_ = ArrayFromJSON(type, right);
+  const std::shared_ptr<Array>& expected_ = ArrayFromJSON(type, expected);
+  CheckIfElseOutput(cond, left, right_, expected_);
+}
+
+class TestIfElseKernel : public ::testing::Test {};
+
+template <typename Type>
+class TestIfElsePrimitive : public ::testing::Test {};
+
+using PrimitiveTypes = ::testing::Types<Int8Type, UInt8Type, Int16Type, UInt16Type,
+                                        Int32Type, UInt32Type, Int64Type, UInt64Type,
+                                        FloatType, DoubleType, Date32Type, Date64Type>;
+
+TYPED_TEST_SUITE(TestIfElsePrimitive, PrimitiveTypes);
+
+TYPED_TEST(TestIfElsePrimitive, IfElseFixedSizeRand) {
+  using ArrayType = typename TypeTraits<TypeParam>::ArrayType;
+  auto type = TypeTraits<TypeParam>::type_singleton();
+
+  random::RandomArrayGenerator rand(/*seed=*/0);
+  int64_t len = 1000;
+  auto cond = std::static_pointer_cast<BooleanArray>(
+      rand.ArrayOf(boolean(), len, /*null_probability=*/0.01));
+  auto left = std::static_pointer_cast<ArrayType>(
+      rand.ArrayOf(type, len, /*null_probability=*/0.01));
+  auto right = std::static_pointer_cast<ArrayType>(
+      rand.ArrayOf(type, len, /*null_probability=*/0.01));
+
+  typename TypeTraits<TypeParam>::BuilderType builder;
+
+  for (int64_t i = 0; i < len; ++i) {
+    if (!cond->IsValid(i) || (cond->Value(i) && !left->IsValid(i)) ||
+        (!cond->Value(i) && !right->IsValid(i))) {
+      ASSERT_OK(builder.AppendNull());
+      continue;
+    }
+
+    if (cond->Value(i)) {
+      ASSERT_OK(builder.Append(left->Value(i)));
+    } else {
+      ASSERT_OK(builder.Append(right->Value(i)));
+    }
+  }
+  ASSERT_OK_AND_ASSIGN(auto expected_data, builder.Finish());
+
+  CheckIfElseOutput(cond, left, right, expected_data);
+}
+
+#define IF_ELSE_TEST_GEN(type, l0, l1, l2, l3, r0, r1, r2, r3, valid, valid1)            \

Review comment:
       No, I mean you can generate the other cases given correct array-array-array using `Array::GetScalar` and `MakeArrayFromScalar`
   
   ```c++
   std::shared_ptr<Array> cond = ..., left = ..., right = ..., expected = ...;
   // Explicit, hand written assertion for array-array-array and scalar-scalar-scalar cases
   CheckScalarUnary("if_else", {cond, left, right}, expected);
   
   // Generate Array-Scalar-Array case
   for (int64_t i = 0; i < left->length(); ++i) {
     ASSERT_OK_AND_ASSIGN(auto left_scalar, left->GetScalar(i));
     ASSERT_OK_AND_ASSIGN(auto left_array, MakeArrayFromScalar(*left_scalar, left->length()));
     ASSERT_OK_AND_ASSIGN(auto expected, IfElse(cond, left_array, right)); // array-array-array is known to be correct
     ASSERT_OK_AND_ASSIGN(auto actual, IfElse(cond, left_scalar, right));
     AssertDatumsEqual(expected, actual);
   }
   ```




-- 
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



[GitHub] [arrow] jorisvandenbossche commented on pull request #10410: ARROW-10640: [C++] A, "if_else" ("where") kernel to combine two arrays based on a mask

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


   > should I add string type support to this as well, or can I leave it for later development?
   
   I think it is important to add support for strings as well, but it's probably a good idea to leave that for a follow-up PR so we can get this merged? (I can imagine the implementation to be a bit different / more complex for variable length array elements)


-- 
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



[GitHub] [arrow] nirandaperera commented on pull request #10410: ARROW-10640: [C++] A "where" kernel to combine two arrays based on a mask

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


   @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.

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



[GitHub] [arrow] lidavidm commented on a change in pull request #10410: ARROW-10640: [C++] A "where" kernel to combine two arrays based on a mask

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_if_else.cc
##########
@@ -0,0 +1,572 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <arrow/compute/api.h>
+#include <arrow/compute/kernels/codegen_internal.h>
+#include <arrow/compute/util_internal.h>
+#include <arrow/util/bit_block_counter.h>
+#include <arrow/util/bitmap.h>
+#include <arrow/util/bitmap_ops.h>
+
+namespace arrow {
+using internal::BitBlockCount;
+using internal::BitBlockCounter;
+using internal::Bitmap;
+
+namespace compute {
+
+namespace {
+
+util::optional<uint64_t> GetConstantValidityWord(const Datum& data) {
+  if (data.is_scalar()) {
+    return data.scalar()->is_valid ? ~uint64_t(0) : uint64_t(0);
+  }
+
+  if (data.array()->null_count == data.array()->length) return uint64_t(0);
+
+  if (!data.array()->MayHaveNulls()) return ~uint64_t(0);
+
+  // no constant validity word available
+  return {};
+}
+
+inline Bitmap GetBitmap(const Datum& datum, int i) {
+  if (datum.is_scalar()) return {};
+  const ArrayData& a = *datum.array();
+  return Bitmap{a.buffers[i], a.offset, a.length};
+}
+
+// if the condition is null then output is null otherwise we take validity from the
+// selected argument
+// ie. cond.valid & (cond.data & left.valid | ~cond.data & right.valid)
+Status PromoteNullsVisitor(KernelContext* ctx, const Datum& cond_d, const Datum& left_d,
+                           const Datum& right_d, ArrayData* output) {
+  auto cond_const = GetConstantValidityWord(cond_d);
+  auto left_const = GetConstantValidityWord(left_d);
+  auto right_const = GetConstantValidityWord(right_d);
+
+  enum { COND_CONST = 1, LEFT_CONST = 2, RIGHT_CONST = 4 };
+  auto flag = COND_CONST * cond_const.has_value() | LEFT_CONST * left_const.has_value() |
+              RIGHT_CONST * right_const.has_value();
+
+  const ArrayData& cond = *cond_d.array();
+  // cond.data will always be available
+  Bitmap cond_data{cond.buffers[1], cond.offset, cond.length};
+  Bitmap cond_valid{cond.buffers[0], cond.offset, cond.length};
+  Bitmap left_valid = GetBitmap(left_d, 0);
+  Bitmap right_valid = GetBitmap(right_d, 0);
+  // sometimes Bitmaps will be ignored, in which case we replace access to them with
+  // duplicated (probably elided) access to cond_data
+  const Bitmap& _ = cond_data;
+
+  // lambda function that will be used inside the visitor
+  uint64_t* out_validity = nullptr;
+  int64_t i = 0;
+  auto apply = [&](uint64_t c_valid, uint64_t c_data, uint64_t l_valid,
+                   uint64_t r_valid) {
+    out_validity[i] = c_valid & ((c_data & l_valid) | (~c_data & r_valid));
+    i++;
+  };
+
+  // if the condition is null then output is null otherwise we take validity from the
+  // selected argument
+  // ie. cond.valid & (cond.data & left.valid | ~cond.data & right.valid)
+
+  // In the following cases, we dont need to allocate out_valid bitmap
+  switch (flag) {
+    case COND_CONST | LEFT_CONST | RIGHT_CONST:
+      // if cond & left & right all ones, then output is all valid --> out_valid = nullptr

Review comment:
       Ah wait, I didn't notice that there was an if and that this branch falls through…




-- 
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



[GitHub] [arrow] lidavidm commented on a change in pull request #10410: ARROW-10640: [C++] A "where" kernel to combine two arrays based on a mask

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



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

Review comment:
       I don't think we need `all_valid` given it's pretty much always `false`. It made sense for the original `fill_null` test because you expect no nulls afterwards, but not here. 

##########
File path: cpp/src/arrow/compute/api_scalar.h
##########
@@ -462,5 +462,21 @@ ARROW_EXPORT
 Result<Datum> FillNull(const Datum& values, const Datum& fill_value,
                        ExecContext* ctx = NULLPTR);
 
+/// \brief IfElse returns elements chosen from `left` or `right`
+/// depending on `cond`. `Null` values would be promoted to the result
+///
+/// \param[in] cond `BooleanArray` condition array
+/// \param[in] left scalar/ Array
+/// \param[in] right scalar/ Array
+/// \param[in] ctx the function execution context, optional
+///
+/// \return the resulting datum
+///
+/// \since x.x.x
+/// \note API not yet finalized

Review comment:
       ```suggestion
   /// \brief IfElse returns elements chosen from `left` or `right`
   /// depending on `cond`. `null` values would be promoted to the result
   ///
   /// \param[in] cond `BooleanArray` condition array
   /// \param[in] left scalar/ Array
   /// \param[in] right scalar/ Array
   /// \param[in] ctx the function execution context, optional
   ///
   /// \return the resulting datum
   ///
   /// \since 5.0.0
   /// \note API not yet finalized
   ```

##########
File path: cpp/src/arrow/compute/kernels/scalar_if_else.cc
##########
@@ -0,0 +1,838 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <arrow/compute/api.h>
+#include <arrow/util/bit_block_counter.h>
+#include <arrow/util/bitmap.h>
+#include <arrow/util/bitmap_ops.h>
+
+#include "codegen_internal.h"
+
+namespace arrow {
+using internal::BitBlockCount;
+using internal::BitBlockCounter;
+using internal::Bitmap;
+
+namespace compute {
+
+namespace {
+
+enum { COND_ALL_VALID = 1, LEFT_ALL_VALID = 2, RIGHT_ALL_VALID = 4 };
+
+// if the condition is null then output is null otherwise we take validity from the
+// selected argument
+// ie. cond.valid & (cond.data & left.valid | ~cond.data & right.valid)
+Status PromoteNullsVisitor(KernelContext* ctx, const ArrayData& cond, const Scalar& left,
+                           const Scalar& right, ArrayData* output) {
+  uint8_t flag = right.is_valid * 4 + left.is_valid * 2 + !cond.MayHaveNulls();
+
+  if (flag < 6 && flag != 3) {
+    // there will be a validity buffer in the output
+    ARROW_ASSIGN_OR_RAISE(output->buffers[0], ctx->AllocateBitmap(cond.length));
+  }
+
+  // if the condition is null then output is null otherwise we take validity from the
+  // selected argument
+  // ie. cond.valid & (cond.data & left.valid | ~cond.data & right.valid)
+  switch (flag) {
+    case COND_ALL_VALID | LEFT_ALL_VALID | RIGHT_ALL_VALID:  // = 7

Review comment:
       Also because RLC/LRC/etc. is a confusing abbreviation especially when the order of arguments is CLR.

##########
File path: cpp/src/arrow/compute/kernels/scalar_if_else_test.cc
##########
@@ -0,0 +1,291 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <arrow/array.h>
+#include <arrow/compute/api_scalar.h>
+#include <arrow/compute/kernels/test_util.h>
+#include <arrow/testing/gtest_util.h>
+#include <gtest/gtest.h>
+
+namespace arrow {
+namespace compute {
+
+void CheckIfElseOutputArray(const Datum& cond, const Datum& left, const Datum& right,
+                            const Datum& expected, bool all_valid = true) {
+  ASSERT_OK_AND_ASSIGN(Datum datum_out, IfElse(cond, left, right));
+  std::shared_ptr<Array> result = datum_out.make_array();
+  ASSERT_OK(result->ValidateFull());
+  AssertArraysEqual(*expected.make_array(), *result, /*verbose=*/true);
+  if (all_valid) {
+    // Check null count of ArrayData is set, not the computed Array.null_count
+    ASSERT_EQ(result->data()->null_count, 0);
+  }
+}
+
+void CheckIfElseOutputAAA(const std::shared_ptr<DataType>& type, const std::string& cond,
+                          const std::string& left, const std::string& right,
+                          const std::string& expected, bool all_valid = true) {
+  const std::shared_ptr<Array>& cond_ = ArrayFromJSON(boolean(), cond);
+  const std::shared_ptr<Array>& left_ = ArrayFromJSON(type, left);
+  const std::shared_ptr<Array>& right_ = ArrayFromJSON(type, right);
+  const std::shared_ptr<Array>& expected_ = ArrayFromJSON(type, expected);
+  CheckIfElseOutputArray(cond_, left_, right_, expected_, all_valid);
+}
+
+void CheckIfElseOutputAAS(const std::shared_ptr<DataType>& type, const std::string& cond,
+                          const std::string& left, const std::shared_ptr<Scalar>& right,
+                          const std::string& expected, bool all_valid = true) {
+  const std::shared_ptr<Array>& cond_ = ArrayFromJSON(boolean(), cond);
+  const std::shared_ptr<Array>& left_ = ArrayFromJSON(type, left);
+  const std::shared_ptr<Array>& expected_ = ArrayFromJSON(type, expected);
+  CheckIfElseOutputArray(cond_, left_, right, expected_, all_valid);
+}
+
+void CheckIfElseOutputASA(const std::shared_ptr<DataType>& type, const std::string& cond,
+                          const std::shared_ptr<Scalar>& left, const std::string& right,
+                          const std::string& expected, bool all_valid = true) {
+  const std::shared_ptr<Array>& cond_ = ArrayFromJSON(boolean(), cond);
+  const std::shared_ptr<Array>& right_ = ArrayFromJSON(type, right);
+  const std::shared_ptr<Array>& expected_ = ArrayFromJSON(type, expected);
+  CheckIfElseOutputArray(cond_, left, right_, expected_, all_valid);
+}
+
+void CheckIfElseOutputASS(const std::shared_ptr<DataType>& type, const std::string& cond,
+                          const std::shared_ptr<Scalar>& left,
+                          const std::shared_ptr<Scalar>& right,
+                          const std::string& expected, bool all_valid = true) {
+  const std::shared_ptr<Array>& cond_ = ArrayFromJSON(boolean(), cond);
+  const std::shared_ptr<Array>& expected_ = ArrayFromJSON(type, expected);
+  CheckIfElseOutputArray(cond_, left, right, expected_, all_valid);
+}
+
+class TestIfElseKernel : public ::testing::Test {};
+
+template <typename Type>
+class TestIfElsePrimitive : public ::testing::Test {};
+
+using PrimitiveTypes = ::testing::Types<Int8Type, UInt8Type, Int16Type, UInt16Type,
+                                        Int32Type, UInt32Type, Int64Type, UInt64Type,
+                                        FloatType, DoubleType, Date32Type, Date64Type>;
+
+TYPED_TEST_SUITE(TestIfElsePrimitive, PrimitiveTypes);
+
+TYPED_TEST(TestIfElsePrimitive, IfElseFixedSize) {
+  auto type = TypeTraits<TypeParam>::type_singleton();
+
+  // No Nulls
+  CheckIfElseOutputAAA(type, "[]", "[]", "[]", "[]");
+
+  // -------- All arrays ---------
+  // RLC = 111
+  CheckIfElseOutputAAA(type, "[true, true, true, false]", "[1, 2, 3, 4]", "[5, 6, 7, 8]",
+                       "[1, 2, 3, 8]");
+  // RLC = 110
+  CheckIfElseOutputAAA(type, "[true, true, null, false]", "[1, 2, 3, 4]", "[5, 6, 7, 8]",
+                       "[1, 2, null, 8]", false);
+  // RLC = 101
+  CheckIfElseOutputAAA(type, "[true, true, true, false]", "[1, null, 3, 4]",
+                       "[5, 6, 7, 8]", "[1, null, 3, 8]", false);
+  // RLC = 100
+  CheckIfElseOutputAAA(type, "[true, true, null, false]", "[1, null, 3, 4]",
+                       "[5, 6, 7, 8]", "[1, null, null, 8]", false);
+  // RLC = 011
+  CheckIfElseOutputAAA(type, "[true, true, true, false]", "[1, 2, 3, 4]",
+                       "[5, 6, 7, null]", "[1, 2, 3, null]", false);
+  // RLC = 010
+  CheckIfElseOutputAAA(type, "[null, true, true, false]", "[1, 2, 3, 4]",
+                       "[5, 6, 7, null]", "[null, 2, 3, null]", false);
+  // RLC = 001
+  CheckIfElseOutputAAA(type, "[true, true, true, false]", "[1, 2, null, null]",
+                       "[null, 6, 7, null]", "[1, 2, null, null]", false);
+  // RLC = 000
+  CheckIfElseOutputAAA(type, "[null, true, true, false]", "[1, 2, null, null]",
+                       "[null, 6, 7, null]", "[null, 2, null, null]", false);
+
+  using ArrayType = typename TypeTraits<TypeParam>::ArrayType;
+  random::RandomArrayGenerator rand(/*seed=*/0);
+  int64_t len = 1000;
+  auto cond = std::static_pointer_cast<BooleanArray>(
+      rand.ArrayOf(boolean(), len, /*null_probability=*/0.01));
+  auto left = std::static_pointer_cast<ArrayType>(
+      rand.ArrayOf(type, len, /*null_probability=*/0.01));
+  auto right = std::static_pointer_cast<ArrayType>(
+      rand.ArrayOf(type, len, /*null_probability=*/0.01));
+
+  typename TypeTraits<TypeParam>::BuilderType builder;
+
+  for (int64_t i = 0; i < len; ++i) {
+    if (!cond->IsValid(i) || (cond->Value(i) && !left->IsValid(i)) ||
+        (!cond->Value(i) && !right->IsValid(i))) {
+      ASSERT_OK(builder.AppendNull());
+      continue;
+    }
+
+    if (cond->Value(i)) {
+      ASSERT_OK(builder.Append(left->Value(i)));
+    } else {
+      ASSERT_OK(builder.Append(right->Value(i)));
+    }
+  }
+  ASSERT_OK_AND_ASSIGN(auto expected_data, builder.Finish());
+
+  CheckIfElseOutputArray(cond, left, right, expected_data, false);
+
+  // -------- Cond - Array, Left- Array, Right - Scalar ---------
+
+  ASSERT_OK_AND_ASSIGN(std::shared_ptr<Scalar> valid_scalar, MakeScalar(type, 100));
+  std::shared_ptr<Scalar> null_scalar = MakeNullScalar(type);
+
+  // empty
+  CheckIfElseOutputAAS(type, "[]", "[]", valid_scalar, "[]");
+
+  // RLC = 111
+  CheckIfElseOutputAAS(type, "[true, true, true, false]", "[1, 2, 3, 4]", valid_scalar,
+                       "[1, 2, 3, 100]");
+  // RLC = 110
+  CheckIfElseOutputAAS(type, "[true, true, null, false]", "[1, 2, 3, 4]", valid_scalar,
+                       "[1, 2, null, 100]", false);
+  // RLC = 101
+  CheckIfElseOutputAAS(type, "[true, true, true, false]", "[1, null, 3, 4]", valid_scalar,
+                       "[1, null, 3, 100]", false);
+  // RLC = 100
+  CheckIfElseOutputAAS(type, "[true, true, null, false]", "[1, null, 3, 4]", valid_scalar,
+                       "[1, null, null, 100]", false);

Review comment:
       It's also unclear to me what RLC/LRC stand for.

##########
File path: cpp/src/arrow/compute/kernels/scalar_if_else.cc
##########
@@ -0,0 +1,838 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <arrow/compute/api.h>
+#include <arrow/util/bit_block_counter.h>
+#include <arrow/util/bitmap.h>
+#include <arrow/util/bitmap_ops.h>
+
+#include "codegen_internal.h"
+
+namespace arrow {
+using internal::BitBlockCount;
+using internal::BitBlockCounter;
+using internal::Bitmap;
+
+namespace compute {
+
+namespace {
+
+enum { COND_ALL_VALID = 1, LEFT_ALL_VALID = 2, RIGHT_ALL_VALID = 4 };
+
+// if the condition is null then output is null otherwise we take validity from the
+// selected argument
+// ie. cond.valid & (cond.data & left.valid | ~cond.data & right.valid)
+Status PromoteNullsVisitor(KernelContext* ctx, const ArrayData& cond, const Scalar& left,
+                           const Scalar& right, ArrayData* output) {
+  uint8_t flag = right.is_valid * 4 + left.is_valid * 2 + !cond.MayHaveNulls();
+
+  if (flag < 6 && flag != 3) {

Review comment:
       This is a little confusing: can we move this into the switch below as another case?

##########
File path: cpp/src/arrow/compute/kernels/scalar_if_else.cc
##########
@@ -0,0 +1,838 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <arrow/compute/api.h>
+#include <arrow/util/bit_block_counter.h>
+#include <arrow/util/bitmap.h>
+#include <arrow/util/bitmap_ops.h>
+
+#include "codegen_internal.h"
+
+namespace arrow {
+using internal::BitBlockCount;
+using internal::BitBlockCounter;
+using internal::Bitmap;
+
+namespace compute {
+
+namespace {
+
+enum { COND_ALL_VALID = 1, LEFT_ALL_VALID = 2, RIGHT_ALL_VALID = 4 };
+
+// if the condition is null then output is null otherwise we take validity from the
+// selected argument
+// ie. cond.valid & (cond.data & left.valid | ~cond.data & right.valid)
+Status PromoteNullsVisitor(KernelContext* ctx, const ArrayData& cond, const Scalar& left,
+                           const Scalar& right, ArrayData* output) {
+  uint8_t flag = right.is_valid * 4 + left.is_valid * 2 + !cond.MayHaveNulls();
+
+  if (flag < 6 && flag != 3) {
+    // there will be a validity buffer in the output
+    ARROW_ASSIGN_OR_RAISE(output->buffers[0], ctx->AllocateBitmap(cond.length));
+  }
+
+  // if the condition is null then output is null otherwise we take validity from the
+  // selected argument
+  // ie. cond.valid & (cond.data & left.valid | ~cond.data & right.valid)
+  switch (flag) {
+    case COND_ALL_VALID | LEFT_ALL_VALID | RIGHT_ALL_VALID:  // = 7

Review comment:
       nit: below you abbreviate these as RLC/LRC/etc. I don't think we need any of those comments since the case itself should be self explanatory.

##########
File path: cpp/src/arrow/compute/kernels/scalar_if_else.cc
##########
@@ -0,0 +1,838 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <arrow/compute/api.h>
+#include <arrow/util/bit_block_counter.h>
+#include <arrow/util/bitmap.h>
+#include <arrow/util/bitmap_ops.h>
+
+#include "codegen_internal.h"
+
+namespace arrow {
+using internal::BitBlockCount;
+using internal::BitBlockCounter;
+using internal::Bitmap;
+
+namespace compute {
+
+namespace {
+
+enum { COND_ALL_VALID = 1, LEFT_ALL_VALID = 2, RIGHT_ALL_VALID = 4 };
+
+// if the condition is null then output is null otherwise we take validity from the
+// selected argument
+// ie. cond.valid & (cond.data & left.valid | ~cond.data & right.valid)
+Status PromoteNullsVisitor(KernelContext* ctx, const ArrayData& cond, const Scalar& left,
+                           const Scalar& right, ArrayData* output) {
+  uint8_t flag = right.is_valid * 4 + left.is_valid * 2 + !cond.MayHaveNulls();
+
+  if (flag < 6 && flag != 3) {
+    // there will be a validity buffer in the output
+    ARROW_ASSIGN_OR_RAISE(output->buffers[0], ctx->AllocateBitmap(cond.length));
+  }
+
+  // if the condition is null then output is null otherwise we take validity from the
+  // selected argument
+  // ie. cond.valid & (cond.data & left.valid | ~cond.data & right.valid)
+  switch (flag) {
+    case COND_ALL_VALID | LEFT_ALL_VALID | RIGHT_ALL_VALID:  // = 7
+      break;
+    case LEFT_ALL_VALID | RIGHT_ALL_VALID:  // = 6
+      // out_valid = c_valid
+      output->buffers[0] = SliceBuffer(cond.buffers[0], cond.offset, cond.length);
+      break;
+    case COND_ALL_VALID | RIGHT_ALL_VALID:  // = 5
+      // out_valid = ~cond.data
+      arrow::internal::InvertBitmap(cond.buffers[1]->data(), cond.offset, cond.length,
+                                    output->buffers[0]->mutable_data(), 0);
+      break;
+    case RIGHT_ALL_VALID:  // = 4
+      // out_valid = c_valid & ~cond.data
+      arrow::internal::BitmapAndNot(cond.buffers[0]->data(), cond.offset,
+                                    cond.buffers[1]->data(), cond.offset, cond.length, 0,
+                                    output->buffers[0]->mutable_data());
+      break;
+    case COND_ALL_VALID | LEFT_ALL_VALID:  // = 3
+      // out_valid = cond.data
+      output->buffers[0] = SliceBuffer(cond.buffers[1], cond.offset, cond.length);
+      break;
+    case LEFT_ALL_VALID:  // = 2
+      // out_valid = cond.valid & cond.data
+      arrow::internal::BitmapAnd(cond.buffers[0]->data(), cond.offset,
+                                 cond.buffers[1]->data(), cond.offset, cond.length, 0,
+                                 output->buffers[0]->mutable_data());
+      break;
+    case COND_ALL_VALID:  // = 1
+      // out_valid = 0 --> nothing to do; but requires out_valid to be a all-zero buffer

Review comment:
       AllocateBitmap doesn't zero the allocation
   https://github.com/apache/arrow/blob/176988893e182ac418072ef8cd9a4bc598784d97/cpp/src/arrow/compute/kernel.h#L61-L64

##########
File path: cpp/src/arrow/compute/kernels/scalar_if_else_test.cc
##########
@@ -0,0 +1,291 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <arrow/array.h>
+#include <arrow/compute/api_scalar.h>
+#include <arrow/compute/kernels/test_util.h>
+#include <arrow/testing/gtest_util.h>
+#include <gtest/gtest.h>
+
+namespace arrow {
+namespace compute {
+
+void CheckIfElseOutputArray(const Datum& cond, const Datum& left, const Datum& right,
+                            const Datum& expected, bool all_valid = true) {
+  ASSERT_OK_AND_ASSIGN(Datum datum_out, IfElse(cond, left, right));
+  std::shared_ptr<Array> result = datum_out.make_array();
+  ASSERT_OK(result->ValidateFull());
+  AssertArraysEqual(*expected.make_array(), *result, /*verbose=*/true);
+  if (all_valid) {
+    // Check null count of ArrayData is set, not the computed Array.null_count
+    ASSERT_EQ(result->data()->null_count, 0);
+  }
+}
+
+void CheckIfElseOutputAAA(const std::shared_ptr<DataType>& type, const std::string& cond,
+                          const std::string& left, const std::string& right,
+                          const std::string& expected, bool all_valid = true) {
+  const std::shared_ptr<Array>& cond_ = ArrayFromJSON(boolean(), cond);
+  const std::shared_ptr<Array>& left_ = ArrayFromJSON(type, left);
+  const std::shared_ptr<Array>& right_ = ArrayFromJSON(type, right);
+  const std::shared_ptr<Array>& expected_ = ArrayFromJSON(type, expected);
+  CheckIfElseOutputArray(cond_, left_, right_, expected_, all_valid);
+}
+
+void CheckIfElseOutputAAS(const std::shared_ptr<DataType>& type, const std::string& cond,
+                          const std::string& left, const std::shared_ptr<Scalar>& right,
+                          const std::string& expected, bool all_valid = true) {
+  const std::shared_ptr<Array>& cond_ = ArrayFromJSON(boolean(), cond);
+  const std::shared_ptr<Array>& left_ = ArrayFromJSON(type, left);
+  const std::shared_ptr<Array>& expected_ = ArrayFromJSON(type, expected);
+  CheckIfElseOutputArray(cond_, left_, right, expected_, all_valid);
+}
+
+void CheckIfElseOutputASA(const std::shared_ptr<DataType>& type, const std::string& cond,
+                          const std::shared_ptr<Scalar>& left, const std::string& right,
+                          const std::string& expected, bool all_valid = true) {
+  const std::shared_ptr<Array>& cond_ = ArrayFromJSON(boolean(), cond);
+  const std::shared_ptr<Array>& right_ = ArrayFromJSON(type, right);
+  const std::shared_ptr<Array>& expected_ = ArrayFromJSON(type, expected);
+  CheckIfElseOutputArray(cond_, left, right_, expected_, all_valid);
+}
+
+void CheckIfElseOutputASS(const std::shared_ptr<DataType>& type, const std::string& cond,
+                          const std::shared_ptr<Scalar>& left,
+                          const std::shared_ptr<Scalar>& right,
+                          const std::string& expected, bool all_valid = true) {
+  const std::shared_ptr<Array>& cond_ = ArrayFromJSON(boolean(), cond);
+  const std::shared_ptr<Array>& expected_ = ArrayFromJSON(type, expected);
+  CheckIfElseOutputArray(cond_, left, right, expected_, all_valid);
+}
+
+class TestIfElseKernel : public ::testing::Test {};
+
+template <typename Type>
+class TestIfElsePrimitive : public ::testing::Test {};
+
+using PrimitiveTypes = ::testing::Types<Int8Type, UInt8Type, Int16Type, UInt16Type,
+                                        Int32Type, UInt32Type, Int64Type, UInt64Type,
+                                        FloatType, DoubleType, Date32Type, Date64Type>;
+
+TYPED_TEST_SUITE(TestIfElsePrimitive, PrimitiveTypes);
+
+TYPED_TEST(TestIfElsePrimitive, IfElseFixedSize) {
+  auto type = TypeTraits<TypeParam>::type_singleton();
+
+  // No Nulls
+  CheckIfElseOutputAAA(type, "[]", "[]", "[]", "[]");
+
+  // -------- All arrays ---------
+  // RLC = 111
+  CheckIfElseOutputAAA(type, "[true, true, true, false]", "[1, 2, 3, 4]", "[5, 6, 7, 8]",
+                       "[1, 2, 3, 8]");
+  // RLC = 110
+  CheckIfElseOutputAAA(type, "[true, true, null, false]", "[1, 2, 3, 4]", "[5, 6, 7, 8]",
+                       "[1, 2, null, 8]", false);
+  // RLC = 101
+  CheckIfElseOutputAAA(type, "[true, true, true, false]", "[1, null, 3, 4]",
+                       "[5, 6, 7, 8]", "[1, null, 3, 8]", false);
+  // RLC = 100
+  CheckIfElseOutputAAA(type, "[true, true, null, false]", "[1, null, 3, 4]",
+                       "[5, 6, 7, 8]", "[1, null, null, 8]", false);
+  // RLC = 011
+  CheckIfElseOutputAAA(type, "[true, true, true, false]", "[1, 2, 3, 4]",
+                       "[5, 6, 7, null]", "[1, 2, 3, null]", false);
+  // RLC = 010
+  CheckIfElseOutputAAA(type, "[null, true, true, false]", "[1, 2, 3, 4]",
+                       "[5, 6, 7, null]", "[null, 2, 3, null]", false);
+  // RLC = 001
+  CheckIfElseOutputAAA(type, "[true, true, true, false]", "[1, 2, null, null]",
+                       "[null, 6, 7, null]", "[1, 2, null, null]", false);
+  // RLC = 000
+  CheckIfElseOutputAAA(type, "[null, true, true, false]", "[1, 2, null, null]",
+                       "[null, 6, 7, null]", "[null, 2, null, null]", false);
+
+  using ArrayType = typename TypeTraits<TypeParam>::ArrayType;
+  random::RandomArrayGenerator rand(/*seed=*/0);
+  int64_t len = 1000;
+  auto cond = std::static_pointer_cast<BooleanArray>(
+      rand.ArrayOf(boolean(), len, /*null_probability=*/0.01));
+  auto left = std::static_pointer_cast<ArrayType>(
+      rand.ArrayOf(type, len, /*null_probability=*/0.01));
+  auto right = std::static_pointer_cast<ArrayType>(
+      rand.ArrayOf(type, len, /*null_probability=*/0.01));
+
+  typename TypeTraits<TypeParam>::BuilderType builder;
+
+  for (int64_t i = 0; i < len; ++i) {
+    if (!cond->IsValid(i) || (cond->Value(i) && !left->IsValid(i)) ||
+        (!cond->Value(i) && !right->IsValid(i))) {
+      ASSERT_OK(builder.AppendNull());
+      continue;
+    }
+
+    if (cond->Value(i)) {
+      ASSERT_OK(builder.Append(left->Value(i)));
+    } else {
+      ASSERT_OK(builder.Append(right->Value(i)));
+    }
+  }
+  ASSERT_OK_AND_ASSIGN(auto expected_data, builder.Finish());
+
+  CheckIfElseOutputArray(cond, left, right, expected_data, false);
+
+  // -------- Cond - Array, Left- Array, Right - Scalar ---------
+
+  ASSERT_OK_AND_ASSIGN(std::shared_ptr<Scalar> valid_scalar, MakeScalar(type, 100));
+  std::shared_ptr<Scalar> null_scalar = MakeNullScalar(type);
+
+  // empty
+  CheckIfElseOutputAAS(type, "[]", "[]", valid_scalar, "[]");
+
+  // RLC = 111
+  CheckIfElseOutputAAS(type, "[true, true, true, false]", "[1, 2, 3, 4]", valid_scalar,
+                       "[1, 2, 3, 100]");
+  // RLC = 110
+  CheckIfElseOutputAAS(type, "[true, true, null, false]", "[1, 2, 3, 4]", valid_scalar,
+                       "[1, 2, null, 100]", false);
+  // RLC = 101
+  CheckIfElseOutputAAS(type, "[true, true, true, false]", "[1, null, 3, 4]", valid_scalar,
+                       "[1, null, 3, 100]", false);
+  // RLC = 100
+  CheckIfElseOutputAAS(type, "[true, true, null, false]", "[1, null, 3, 4]", valid_scalar,
+                       "[1, null, null, 100]", false);

Review comment:
       Couldn't these cases be combined? e.g. `[true, true, null, null, false, false]`, `[1, null, 2, null, 3, null]`, and `valid_scalar` -> `[1, null, null, null, 100, null]`?
   
   And ditto overall.

##########
File path: cpp/src/arrow/compute/kernels/scalar_if_else_test.cc
##########
@@ -0,0 +1,291 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <arrow/array.h>
+#include <arrow/compute/api_scalar.h>
+#include <arrow/compute/kernels/test_util.h>
+#include <arrow/testing/gtest_util.h>
+#include <gtest/gtest.h>
+
+namespace arrow {
+namespace compute {
+
+void CheckIfElseOutputArray(const Datum& cond, const Datum& left, const Datum& right,
+                            const Datum& expected, bool all_valid = true) {
+  ASSERT_OK_AND_ASSIGN(Datum datum_out, IfElse(cond, left, right));
+  std::shared_ptr<Array> result = datum_out.make_array();
+  ASSERT_OK(result->ValidateFull());
+  AssertArraysEqual(*expected.make_array(), *result, /*verbose=*/true);
+  if (all_valid) {
+    // Check null count of ArrayData is set, not the computed Array.null_count
+    ASSERT_EQ(result->data()->null_count, 0);
+  }
+}
+
+void CheckIfElseOutputAAA(const std::shared_ptr<DataType>& type, const std::string& cond,
+                          const std::string& left, const std::string& right,
+                          const std::string& expected, bool all_valid = true) {
+  const std::shared_ptr<Array>& cond_ = ArrayFromJSON(boolean(), cond);
+  const std::shared_ptr<Array>& left_ = ArrayFromJSON(type, left);
+  const std::shared_ptr<Array>& right_ = ArrayFromJSON(type, right);
+  const std::shared_ptr<Array>& expected_ = ArrayFromJSON(type, expected);
+  CheckIfElseOutputArray(cond_, left_, right_, expected_, all_valid);
+}
+
+void CheckIfElseOutputAAS(const std::shared_ptr<DataType>& type, const std::string& cond,
+                          const std::string& left, const std::shared_ptr<Scalar>& right,
+                          const std::string& expected, bool all_valid = true) {
+  const std::shared_ptr<Array>& cond_ = ArrayFromJSON(boolean(), cond);
+  const std::shared_ptr<Array>& left_ = ArrayFromJSON(type, left);
+  const std::shared_ptr<Array>& expected_ = ArrayFromJSON(type, expected);
+  CheckIfElseOutputArray(cond_, left_, right, expected_, all_valid);
+}
+
+void CheckIfElseOutputASA(const std::shared_ptr<DataType>& type, const std::string& cond,
+                          const std::shared_ptr<Scalar>& left, const std::string& right,
+                          const std::string& expected, bool all_valid = true) {
+  const std::shared_ptr<Array>& cond_ = ArrayFromJSON(boolean(), cond);
+  const std::shared_ptr<Array>& right_ = ArrayFromJSON(type, right);
+  const std::shared_ptr<Array>& expected_ = ArrayFromJSON(type, expected);
+  CheckIfElseOutputArray(cond_, left, right_, expected_, all_valid);
+}
+
+void CheckIfElseOutputASS(const std::shared_ptr<DataType>& type, const std::string& cond,
+                          const std::shared_ptr<Scalar>& left,
+                          const std::shared_ptr<Scalar>& right,
+                          const std::string& expected, bool all_valid = true) {
+  const std::shared_ptr<Array>& cond_ = ArrayFromJSON(boolean(), cond);
+  const std::shared_ptr<Array>& expected_ = ArrayFromJSON(type, expected);
+  CheckIfElseOutputArray(cond_, left, right, expected_, all_valid);
+}
+
+class TestIfElseKernel : public ::testing::Test {};
+
+template <typename Type>
+class TestIfElsePrimitive : public ::testing::Test {};
+
+using PrimitiveTypes = ::testing::Types<Int8Type, UInt8Type, Int16Type, UInt16Type,
+                                        Int32Type, UInt32Type, Int64Type, UInt64Type,
+                                        FloatType, DoubleType, Date32Type, Date64Type>;
+
+TYPED_TEST_SUITE(TestIfElsePrimitive, PrimitiveTypes);
+
+TYPED_TEST(TestIfElsePrimitive, IfElseFixedSize) {
+  auto type = TypeTraits<TypeParam>::type_singleton();
+
+  // No Nulls
+  CheckIfElseOutputAAA(type, "[]", "[]", "[]", "[]");
+
+  // -------- All arrays ---------
+  // RLC = 111
+  CheckIfElseOutputAAA(type, "[true, true, true, false]", "[1, 2, 3, 4]", "[5, 6, 7, 8]",
+                       "[1, 2, 3, 8]");
+  // RLC = 110
+  CheckIfElseOutputAAA(type, "[true, true, null, false]", "[1, 2, 3, 4]", "[5, 6, 7, 8]",
+                       "[1, 2, null, 8]", false);
+  // RLC = 101
+  CheckIfElseOutputAAA(type, "[true, true, true, false]", "[1, null, 3, 4]",
+                       "[5, 6, 7, 8]", "[1, null, 3, 8]", false);
+  // RLC = 100
+  CheckIfElseOutputAAA(type, "[true, true, null, false]", "[1, null, 3, 4]",
+                       "[5, 6, 7, 8]", "[1, null, null, 8]", false);
+  // RLC = 011
+  CheckIfElseOutputAAA(type, "[true, true, true, false]", "[1, 2, 3, 4]",
+                       "[5, 6, 7, null]", "[1, 2, 3, null]", false);
+  // RLC = 010
+  CheckIfElseOutputAAA(type, "[null, true, true, false]", "[1, 2, 3, 4]",
+                       "[5, 6, 7, null]", "[null, 2, 3, null]", false);
+  // RLC = 001
+  CheckIfElseOutputAAA(type, "[true, true, true, false]", "[1, 2, null, null]",
+                       "[null, 6, 7, null]", "[1, 2, null, null]", false);
+  // RLC = 000
+  CheckIfElseOutputAAA(type, "[null, true, true, false]", "[1, 2, null, null]",
+                       "[null, 6, 7, null]", "[null, 2, null, null]", false);
+
+  using ArrayType = typename TypeTraits<TypeParam>::ArrayType;

Review comment:
       nit: do you mind extracting out the random array test into a separate test case?

##########
File path: cpp/src/arrow/compute/kernels/scalar_if_else.cc
##########
@@ -0,0 +1,838 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <arrow/compute/api.h>
+#include <arrow/util/bit_block_counter.h>
+#include <arrow/util/bitmap.h>
+#include <arrow/util/bitmap_ops.h>
+
+#include "codegen_internal.h"
+
+namespace arrow {
+using internal::BitBlockCount;
+using internal::BitBlockCounter;
+using internal::Bitmap;
+
+namespace compute {
+
+namespace {
+
+enum { COND_ALL_VALID = 1, LEFT_ALL_VALID = 2, RIGHT_ALL_VALID = 4 };
+
+// if the condition is null then output is null otherwise we take validity from the
+// selected argument
+// ie. cond.valid & (cond.data & left.valid | ~cond.data & right.valid)
+Status PromoteNullsVisitor(KernelContext* ctx, const ArrayData& cond, const Scalar& left,
+                           const Scalar& right, ArrayData* output) {
+  uint8_t flag = right.is_valid * 4 + left.is_valid * 2 + !cond.MayHaveNulls();
+
+  if (flag < 6 && flag != 3) {

Review comment:
       You could move the allocation into the cases as well.

##########
File path: cpp/src/arrow/compute/kernels/scalar_if_else_test.cc
##########
@@ -0,0 +1,291 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <arrow/array.h>
+#include <arrow/compute/api_scalar.h>
+#include <arrow/compute/kernels/test_util.h>
+#include <arrow/testing/gtest_util.h>
+#include <gtest/gtest.h>
+
+namespace arrow {
+namespace compute {
+
+void CheckIfElseOutputArray(const Datum& cond, const Datum& left, const Datum& right,
+                            const Datum& expected, bool all_valid = true) {
+  ASSERT_OK_AND_ASSIGN(Datum datum_out, IfElse(cond, left, right));
+  std::shared_ptr<Array> result = datum_out.make_array();
+  ASSERT_OK(result->ValidateFull());
+  AssertArraysEqual(*expected.make_array(), *result, /*verbose=*/true);
+  if (all_valid) {
+    // Check null count of ArrayData is set, not the computed Array.null_count
+    ASSERT_EQ(result->data()->null_count, 0);
+  }
+}
+
+void CheckIfElseOutputAAA(const std::shared_ptr<DataType>& type, const std::string& cond,
+                          const std::string& left, const std::string& right,
+                          const std::string& expected, bool all_valid = true) {
+  const std::shared_ptr<Array>& cond_ = ArrayFromJSON(boolean(), cond);
+  const std::shared_ptr<Array>& left_ = ArrayFromJSON(type, left);
+  const std::shared_ptr<Array>& right_ = ArrayFromJSON(type, right);
+  const std::shared_ptr<Array>& expected_ = ArrayFromJSON(type, expected);
+  CheckIfElseOutputArray(cond_, left_, right_, expected_, all_valid);
+}
+
+void CheckIfElseOutputAAS(const std::shared_ptr<DataType>& type, const std::string& cond,
+                          const std::string& left, const std::shared_ptr<Scalar>& right,
+                          const std::string& expected, bool all_valid = true) {
+  const std::shared_ptr<Array>& cond_ = ArrayFromJSON(boolean(), cond);
+  const std::shared_ptr<Array>& left_ = ArrayFromJSON(type, left);
+  const std::shared_ptr<Array>& expected_ = ArrayFromJSON(type, expected);
+  CheckIfElseOutputArray(cond_, left_, right, expected_, all_valid);
+}
+
+void CheckIfElseOutputASA(const std::shared_ptr<DataType>& type, const std::string& cond,
+                          const std::shared_ptr<Scalar>& left, const std::string& right,
+                          const std::string& expected, bool all_valid = true) {
+  const std::shared_ptr<Array>& cond_ = ArrayFromJSON(boolean(), cond);
+  const std::shared_ptr<Array>& right_ = ArrayFromJSON(type, right);
+  const std::shared_ptr<Array>& expected_ = ArrayFromJSON(type, expected);
+  CheckIfElseOutputArray(cond_, left, right_, expected_, all_valid);
+}
+
+void CheckIfElseOutputASS(const std::shared_ptr<DataType>& type, const std::string& cond,
+                          const std::shared_ptr<Scalar>& left,
+                          const std::shared_ptr<Scalar>& right,
+                          const std::string& expected, bool all_valid = true) {
+  const std::shared_ptr<Array>& cond_ = ArrayFromJSON(boolean(), cond);
+  const std::shared_ptr<Array>& expected_ = ArrayFromJSON(type, expected);
+  CheckIfElseOutputArray(cond_, left, right, expected_, all_valid);
+}
+
+class TestIfElseKernel : public ::testing::Test {};
+
+template <typename Type>
+class TestIfElsePrimitive : public ::testing::Test {};
+
+using PrimitiveTypes = ::testing::Types<Int8Type, UInt8Type, Int16Type, UInt16Type,
+                                        Int32Type, UInt32Type, Int64Type, UInt64Type,
+                                        FloatType, DoubleType, Date32Type, Date64Type>;
+
+TYPED_TEST_SUITE(TestIfElsePrimitive, PrimitiveTypes);
+
+TYPED_TEST(TestIfElsePrimitive, IfElseFixedSize) {
+  auto type = TypeTraits<TypeParam>::type_singleton();
+
+  // No Nulls
+  CheckIfElseOutputAAA(type, "[]", "[]", "[]", "[]");
+
+  // -------- All arrays ---------
+  // RLC = 111
+  CheckIfElseOutputAAA(type, "[true, true, true, false]", "[1, 2, 3, 4]", "[5, 6, 7, 8]",
+                       "[1, 2, 3, 8]");
+  // RLC = 110
+  CheckIfElseOutputAAA(type, "[true, true, null, false]", "[1, 2, 3, 4]", "[5, 6, 7, 8]",
+                       "[1, 2, null, 8]", false);
+  // RLC = 101
+  CheckIfElseOutputAAA(type, "[true, true, true, false]", "[1, null, 3, 4]",
+                       "[5, 6, 7, 8]", "[1, null, 3, 8]", false);
+  // RLC = 100
+  CheckIfElseOutputAAA(type, "[true, true, null, false]", "[1, null, 3, 4]",
+                       "[5, 6, 7, 8]", "[1, null, null, 8]", false);
+  // RLC = 011
+  CheckIfElseOutputAAA(type, "[true, true, true, false]", "[1, 2, 3, 4]",
+                       "[5, 6, 7, null]", "[1, 2, 3, null]", false);
+  // RLC = 010
+  CheckIfElseOutputAAA(type, "[null, true, true, false]", "[1, 2, 3, 4]",
+                       "[5, 6, 7, null]", "[null, 2, 3, null]", false);
+  // RLC = 001
+  CheckIfElseOutputAAA(type, "[true, true, true, false]", "[1, 2, null, null]",
+                       "[null, 6, 7, null]", "[1, 2, null, null]", false);
+  // RLC = 000
+  CheckIfElseOutputAAA(type, "[null, true, true, false]", "[1, 2, null, null]",
+                       "[null, 6, 7, null]", "[null, 2, null, null]", false);
+
+  using ArrayType = typename TypeTraits<TypeParam>::ArrayType;
+  random::RandomArrayGenerator rand(/*seed=*/0);
+  int64_t len = 1000;
+  auto cond = std::static_pointer_cast<BooleanArray>(
+      rand.ArrayOf(boolean(), len, /*null_probability=*/0.01));
+  auto left = std::static_pointer_cast<ArrayType>(
+      rand.ArrayOf(type, len, /*null_probability=*/0.01));
+  auto right = std::static_pointer_cast<ArrayType>(
+      rand.ArrayOf(type, len, /*null_probability=*/0.01));
+
+  typename TypeTraits<TypeParam>::BuilderType builder;
+
+  for (int64_t i = 0; i < len; ++i) {
+    if (!cond->IsValid(i) || (cond->Value(i) && !left->IsValid(i)) ||
+        (!cond->Value(i) && !right->IsValid(i))) {
+      ASSERT_OK(builder.AppendNull());
+      continue;
+    }
+
+    if (cond->Value(i)) {
+      ASSERT_OK(builder.Append(left->Value(i)));
+    } else {
+      ASSERT_OK(builder.Append(right->Value(i)));
+    }
+  }
+  ASSERT_OK_AND_ASSIGN(auto expected_data, builder.Finish());
+
+  CheckIfElseOutputArray(cond, left, right, expected_data, false);
+
+  // -------- Cond - Array, Left- Array, Right - Scalar ---------
+
+  ASSERT_OK_AND_ASSIGN(std::shared_ptr<Scalar> valid_scalar, MakeScalar(type, 100));
+  std::shared_ptr<Scalar> null_scalar = MakeNullScalar(type);
+
+  // empty
+  CheckIfElseOutputAAS(type, "[]", "[]", valid_scalar, "[]");
+
+  // RLC = 111
+  CheckIfElseOutputAAS(type, "[true, true, true, false]", "[1, 2, 3, 4]", valid_scalar,
+                       "[1, 2, 3, 100]");
+  // RLC = 110
+  CheckIfElseOutputAAS(type, "[true, true, null, false]", "[1, 2, 3, 4]", valid_scalar,
+                       "[1, 2, null, 100]", false);
+  // RLC = 101
+  CheckIfElseOutputAAS(type, "[true, true, true, false]", "[1, null, 3, 4]", valid_scalar,
+                       "[1, null, 3, 100]", false);
+  // RLC = 100
+  CheckIfElseOutputAAS(type, "[true, true, null, false]", "[1, null, 3, 4]", valid_scalar,
+                       "[1, null, null, 100]", false);

Review comment:
       Ok, I see from above that it's about the validity of cond/left/right. IMO if you're going to abbreviate it, at least always write it as C/L/R since that's the order of the arguments.

##########
File path: cpp/src/arrow/compute/kernels/scalar_if_else.cc
##########
@@ -0,0 +1,838 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <arrow/compute/api.h>
+#include <arrow/util/bit_block_counter.h>
+#include <arrow/util/bitmap.h>
+#include <arrow/util/bitmap_ops.h>
+
+#include "codegen_internal.h"
+
+namespace arrow {
+using internal::BitBlockCount;
+using internal::BitBlockCounter;
+using internal::Bitmap;
+
+namespace compute {
+
+namespace {
+
+enum { COND_ALL_VALID = 1, LEFT_ALL_VALID = 2, RIGHT_ALL_VALID = 4 };
+
+// if the condition is null then output is null otherwise we take validity from the
+// selected argument
+// ie. cond.valid & (cond.data & left.valid | ~cond.data & right.valid)
+Status PromoteNullsVisitor(KernelContext* ctx, const ArrayData& cond, const Scalar& left,
+                           const Scalar& right, ArrayData* output) {
+  uint8_t flag = right.is_valid * 4 + left.is_valid * 2 + !cond.MayHaveNulls();
+
+  if (flag < 6 && flag != 3) {
+    // there will be a validity buffer in the output
+    ARROW_ASSIGN_OR_RAISE(output->buffers[0], ctx->AllocateBitmap(cond.length));
+  }
+
+  // if the condition is null then output is null otherwise we take validity from the
+  // selected argument
+  // ie. cond.valid & (cond.data & left.valid | ~cond.data & right.valid)
+  switch (flag) {
+    case COND_ALL_VALID | LEFT_ALL_VALID | RIGHT_ALL_VALID:  // = 7
+      break;
+    case LEFT_ALL_VALID | RIGHT_ALL_VALID:  // = 6
+      // out_valid = c_valid
+      output->buffers[0] = SliceBuffer(cond.buffers[0], cond.offset, cond.length);
+      break;
+    case COND_ALL_VALID | RIGHT_ALL_VALID:  // = 5
+      // out_valid = ~cond.data
+      arrow::internal::InvertBitmap(cond.buffers[1]->data(), cond.offset, cond.length,
+                                    output->buffers[0]->mutable_data(), 0);
+      break;
+    case RIGHT_ALL_VALID:  // = 4
+      // out_valid = c_valid & ~cond.data
+      arrow::internal::BitmapAndNot(cond.buffers[0]->data(), cond.offset,
+                                    cond.buffers[1]->data(), cond.offset, cond.length, 0,
+                                    output->buffers[0]->mutable_data());
+      break;
+    case COND_ALL_VALID | LEFT_ALL_VALID:  // = 3
+      // out_valid = cond.data
+      output->buffers[0] = SliceBuffer(cond.buffers[1], cond.offset, cond.length);
+      break;
+    case LEFT_ALL_VALID:  // = 2
+      // out_valid = cond.valid & cond.data
+      arrow::internal::BitmapAnd(cond.buffers[0]->data(), cond.offset,
+                                 cond.buffers[1]->data(), cond.offset, cond.length, 0,
+                                 output->buffers[0]->mutable_data());
+      break;
+    case COND_ALL_VALID:  // = 1
+      // out_valid = 0 --> nothing to do; but requires out_valid to be a all-zero buffer
+      break;
+    case 0:  // RLC = 000
+      // out_valid = 0 --> nothing to do; but requires out_valid to be a all-zero buffer
+      break;
+  }
+  return Status::OK();
+}
+
+Status PromoteNullsVisitor(KernelContext* ctx, const ArrayData& cond,
+                           const ArrayData& left, const Scalar& right,
+                           ArrayData* output) {
+  uint8_t flag = right.is_valid * 4 + !left.MayHaveNulls() * 2 + !cond.MayHaveNulls();
+
+  enum { C_VALID, C_DATA, L_VALID };
+
+  Bitmap bitmaps[3];
+  bitmaps[C_VALID] = {cond.buffers[0], cond.offset, cond.length};
+  bitmaps[C_DATA] = {cond.buffers[1], cond.offset, cond.length};
+  bitmaps[L_VALID] = {left.buffers[0], left.offset, left.length};
+
+  uint64_t* out_validity = nullptr;
+  if (flag < 6 && flag != 3) {
+    // there will be a validity buffer in the output
+    ARROW_ASSIGN_OR_RAISE(output->buffers[0], ctx->AllocateBitmap(cond.length));
+    out_validity = output->GetMutableValues<uint64_t>(0);
+  }
+
+  // lambda function that will be used inside the visitor
+  int64_t i = 0;
+  auto apply = [&](uint64_t c_valid, uint64_t c_data, uint64_t l_valid,
+                   uint64_t r_valid) {
+    out_validity[i] = c_valid & ((c_data & l_valid) | (~c_data & r_valid));
+    i++;
+  };
+
+  // if the condition is null then output is null otherwise we take validity from the
+  // selected argument
+  // ie. cond.valid & (cond.data & left.valid | ~cond.data & right.valid)
+  switch (flag) {
+    case COND_ALL_VALID | LEFT_ALL_VALID | RIGHT_ALL_VALID:  // RLC = 111
+      break;
+    case LEFT_ALL_VALID | RIGHT_ALL_VALID:  // RLC = 110
+      output->buffers[0] = SliceBuffer(cond.buffers[0], cond.offset, cond.length);
+      break;
+    case COND_ALL_VALID | RIGHT_ALL_VALID:  // RLC = 101
+      // bitmaps[C_VALID] might be null; override to make it safe for Visit()
+      bitmaps[C_VALID] = bitmaps[C_DATA];
+      Bitmap::VisitWords(bitmaps, [&](std::array<uint64_t, 3> words) {
+        apply(UINT64_MAX, words[C_DATA], words[L_VALID], UINT64_MAX);
+      });
+      break;
+    case RIGHT_ALL_VALID:  // RLC = 100
+      Bitmap::VisitWords(bitmaps, [&](std::array<uint64_t, 3> words) {
+        apply(words[C_VALID], words[C_DATA], words[L_VALID], UINT64_MAX);
+      });
+      break;
+    case COND_ALL_VALID | LEFT_ALL_VALID:  // RLC = 011
+      // only cond.data is passed
+      output->buffers[0] = SliceBuffer(cond.buffers[1], cond.offset, cond.length);
+      break;
+    case LEFT_ALL_VALID:  // RLC = 010
+      // out_valid = cond.valid & cond.data
+      arrow::internal::BitmapAnd(cond.buffers[0]->data(), cond.offset,
+                                 cond.buffers[1]->data(), cond.offset, cond.length, 0,
+                                 output->buffers[0]->mutable_data());
+      break;
+    case COND_ALL_VALID:  // RLC = 001
+      // out_valid = cond.data & left.valid
+      arrow::internal::BitmapAnd(cond.buffers[1]->data(), cond.offset,
+                                 left.buffers[0]->data(), left.offset, cond.length, 0,
+                                 output->buffers[0]->mutable_data());
+      break;
+    case 0:  // RLC = 000
+      Bitmap::VisitWords(bitmaps, [&](std::array<uint64_t, 3> words) {
+        apply(words[C_VALID], words[C_DATA], words[L_VALID], 0);
+      });
+      break;
+  }
+  return Status::OK();
+}
+
+// if the condition is null then output is null otherwise we take validity from the
+// selected argument
+// ie. cond.valid & (cond.data & left.valid | ~cond.data & right.valid)
+Status PromoteNullsVisitor(KernelContext* ctx, const ArrayData& cond, const Scalar& left,
+                           const ArrayData& right, ArrayData* output) {
+  uint8_t flag = !right.MayHaveNulls() * 4 + left.is_valid * 2 + !cond.MayHaveNulls();
+
+  enum { C_VALID, C_DATA, R_VALID };
+
+  Bitmap bitmaps[3];
+  bitmaps[C_VALID] = {cond.buffers[0], cond.offset, cond.length};
+  bitmaps[C_DATA] = {cond.buffers[1], cond.offset, cond.length};
+  bitmaps[R_VALID] = {right.buffers[0], right.offset, right.length};
+
+  uint64_t* out_validity = nullptr;
+  if (flag < 6) {
+    // there will be a validity buffer in the output
+    ARROW_ASSIGN_OR_RAISE(output->buffers[0], ctx->AllocateBitmap(cond.length));
+    out_validity = output->GetMutableValues<uint64_t>(0);
+  }
+
+  // lambda function that will be used inside the visitor
+  int64_t i = 0;
+  auto apply = [&](uint64_t c_valid, uint64_t c_data, uint64_t l_valid,
+                   uint64_t r_valid) {
+    out_validity[i] = c_valid & ((c_data & l_valid) | (~c_data & r_valid));
+    i++;
+  };
+
+  // if the condition is null then output is null otherwise we take validity from the
+  // selected argument
+  // ie. cond.valid & (cond.data & left.valid | ~cond.data & right.valid)
+  switch (flag) {
+    case COND_ALL_VALID | LEFT_ALL_VALID | RIGHT_ALL_VALID:  // RLC = 111
+      break;
+    case LEFT_ALL_VALID | RIGHT_ALL_VALID:  // RLC = 110
+      output->buffers[0] = SliceBuffer(cond.buffers[0], cond.offset, cond.length);
+      break;
+    case COND_ALL_VALID | RIGHT_ALL_VALID:  // RLC = 101
+      // out_valid = ~cond.data
+      arrow::internal::InvertBitmap(cond.buffers[1]->data(), cond.offset, cond.length,
+                                    output->buffers[0]->mutable_data(), 0);
+      break;
+    case RIGHT_ALL_VALID:  // RLC = 100
+      // out_valid = c_valid & ~cond.data
+      arrow::internal::BitmapAndNot(cond.buffers[0]->data(), cond.offset,
+                                    cond.buffers[1]->data(), cond.offset, cond.length, 0,
+                                    output->buffers[0]->mutable_data());
+      break;
+    case COND_ALL_VALID | LEFT_ALL_VALID:  // RLC = 011
+      // bitmaps[C_VALID] might be null; override to make it safe for Visit()
+      bitmaps[C_VALID] = bitmaps[C_DATA];
+      Bitmap::VisitWords(bitmaps, [&](std::array<uint64_t, 3> words) {
+        apply(UINT64_MAX, words[C_DATA], UINT64_MAX, words[R_VALID]);
+      });
+      break;
+    case LEFT_ALL_VALID:  // RLC = 010
+      Bitmap::VisitWords(bitmaps, [&](std::array<uint64_t, 3> words) {
+        apply(words[C_VALID], words[C_DATA], UINT64_MAX, words[R_VALID]);
+      });
+      break;
+    case COND_ALL_VALID:  // RLC = 001
+      // out_valid =  ~cond.data & right.valid
+      arrow::internal::BitmapAndNot(right.buffers[0]->data(), right.offset,
+                                    cond.buffers[1]->data(), cond.offset, cond.length, 0,
+                                    output->buffers[0]->mutable_data());
+      break;
+    case 0:  // RLC = 000
+      Bitmap::VisitWords(bitmaps, [&](std::array<uint64_t, 3> words) {
+        apply(words[C_VALID], words[C_DATA], 0, words[R_VALID]);
+      });
+      break;
+  }
+  return Status::OK();
+}
+
+// if the condition is null then output is null otherwise we take validity from the
+// selected argument
+// ie. cond.valid & (cond.data & left.valid | ~cond.data & right.valid)
+Status PromoteNullsVisitor(KernelContext* ctx, const ArrayData& cond,
+                           const ArrayData& left, const ArrayData& right,
+                           ArrayData* output) {
+  uint8_t flag =
+      !right.MayHaveNulls() * 4 + !left.MayHaveNulls() * 2 + !cond.MayHaveNulls();
+
+  enum { C_VALID, C_DATA, L_VALID, R_VALID };
+
+  Bitmap bitmaps[4];
+  bitmaps[C_VALID] = {cond.buffers[0], cond.offset, cond.length};
+  bitmaps[C_DATA] = {cond.buffers[1], cond.offset, cond.length};
+  bitmaps[L_VALID] = {left.buffers[0], left.offset, left.length};
+  bitmaps[R_VALID] = {right.buffers[0], right.offset, right.length};
+
+  uint64_t* out_validity = nullptr;
+  if (flag < 6) {
+    // there will be a validity buffer in the output
+    ARROW_ASSIGN_OR_RAISE(output->buffers[0], ctx->AllocateBitmap(cond.length));
+    out_validity = output->GetMutableValues<uint64_t>(0);
+  }
+
+  // lambda function that will be used inside the visitor
+  int64_t i = 0;
+  auto apply = [&](uint64_t c_valid, uint64_t c_data, uint64_t l_valid,
+                   uint64_t r_valid) {
+    out_validity[i] = c_valid & ((c_data & l_valid) | (~c_data & r_valid));
+    i++;
+  };
+
+  // if the condition is null then output is null otherwise we take validity from the
+  // selected argument
+  // ie. cond.valid & (cond.data & left.valid | ~cond.data & right.valid)
+  switch (flag) {
+    case COND_ALL_VALID | LEFT_ALL_VALID | RIGHT_ALL_VALID:  // RLC = 111
+      break;
+    case LEFT_ALL_VALID | RIGHT_ALL_VALID:  // RLC = 110
+      output->buffers[0] = SliceBuffer(cond.buffers[0], cond.offset, cond.length);
+      break;
+    case COND_ALL_VALID | RIGHT_ALL_VALID:  // RLC = 101
+      // bitmaps[C_VALID], bitmaps[R_VALID] might be null; override to make it safe for
+      // Visit()
+      bitmaps[C_VALID] = bitmaps[C_DATA];
+      bitmaps[R_VALID] = bitmaps[C_DATA];
+      Bitmap::VisitWords(bitmaps, [&](std::array<uint64_t, 4> words) {
+        apply(UINT64_MAX, words[C_DATA], words[L_VALID], UINT64_MAX);
+      });
+      break;
+    case RIGHT_ALL_VALID:  // RLC = 100
+      // bitmaps[R_VALID] might be null; override to make it safe for Visit()
+      bitmaps[R_VALID] = bitmaps[C_DATA];
+      Bitmap::VisitWords(bitmaps, [&](std::array<uint64_t, 4> words) {
+        apply(words[C_VALID], words[C_DATA], words[L_VALID], UINT64_MAX);
+      });
+      break;
+    case COND_ALL_VALID | LEFT_ALL_VALID:  // RLC = 011
+      // bitmaps[C_VALID], bitmaps[L_VALID] might be null; override to make it safe for
+      // Visit()
+      bitmaps[C_VALID] = bitmaps[C_DATA];
+      bitmaps[L_VALID] = bitmaps[C_DATA];
+      Bitmap::VisitWords(bitmaps, [&](std::array<uint64_t, 4> words) {
+        apply(UINT64_MAX, words[C_DATA], UINT64_MAX, words[R_VALID]);
+      });
+      break;
+    case LEFT_ALL_VALID:  // RLC = 010
+      // bitmaps[L_VALID] might be null; override to make it safe for Visit()
+      bitmaps[L_VALID] = bitmaps[C_DATA];
+      Bitmap::VisitWords(bitmaps, [&](std::array<uint64_t, 4> words) {
+        apply(words[C_VALID], words[C_DATA], UINT64_MAX, words[R_VALID]);
+      });
+      break;
+    case COND_ALL_VALID:  // RLC = 001
+      // bitmaps[C_VALID] might be null; override to make it safe for Visit()
+      bitmaps[C_VALID] = bitmaps[C_DATA];
+      Bitmap::VisitWords(bitmaps, [&](std::array<uint64_t, 4> words) {
+        apply(UINT64_MAX, words[C_DATA], words[L_VALID], words[R_VALID]);
+      });
+      break;
+    case 0:  // RLC = 000
+      Bitmap::VisitWords(bitmaps, [&](std::array<uint64_t, 4> words) {
+        apply(words[C_VALID], words[C_DATA], words[L_VALID], words[R_VALID]);
+      });
+      break;
+  }
+  return Status::OK();
+}
+
+// nulls will be promoted as follows:
+// cond.valid && (cond.data && left.valid || ~cond.data && right.valid)
+// Note: we have to work on ArrayData. Otherwise we won't be able to handle array
+// offsets AAA
+/*Status PromoteNulls(KernelContext* ctx, const ArrayData& cond, const ArrayData& left,
+                    const ArrayData& right, ArrayData* output) {
+  if (!cond.MayHaveNulls() && !left.MayHaveNulls() && !right.MayHaveNulls()) {
+    return Status::OK();  // no nulls to handle
+  }
+  const int64_t len = cond.length;
+
+  // out_validity = ~cond.data --> mask right values
+  ARROW_ASSIGN_OR_RAISE(
+      std::shared_ptr<Buffer> out_validity,
+      arrow::internal::InvertBitmap(ctx->memory_pool(), cond.buffers[1]->data(),
+                                    cond.offset, len));
+
+  if (right.MayHaveNulls()) {  // out_validity = right.valid && ~cond.data
+    arrow::internal::BitmapAnd(right.buffers[0]->data(), right.offset,
+                               out_validity->data(), 0, len, 0,
+                               out_validity->mutable_data());
+  }
+
+  std::shared_ptr<Buffer> tmp_buf;
+  if (left.MayHaveNulls()) {
+    // tmp_buf = left.valid && cond.data
+    ARROW_ASSIGN_OR_RAISE(
+        tmp_buf, arrow::internal::BitmapAnd(ctx->memory_pool(), left.buffers[0]->data(),
+                                            left.offset, cond.buffers[1]->data(),
+                                            cond.offset, len, 0));
+  } else {  // if left all valid --> tmp_buf = cond.data (zero copy slice)
+    tmp_buf = SliceBuffer(cond.buffers[1], cond.offset, cond.length);
+  }
+
+  // out_validity = cond.data && left.valid || ~cond.data && right.valid
+  arrow::internal::BitmapOr(out_validity->data(), 0, tmp_buf->data(), 0, len, 0,
+                            out_validity->mutable_data());
+
+  if (cond.MayHaveNulls()) {
+    // out_validity = cond.valid && (cond.data && left.valid || ~cond.data && right.valid)
+    ::arrow::internal::BitmapAnd(out_validity->data(), 0, cond.buffers[0]->data(),
+                                 cond.offset, len, 0, out_validity->mutable_data());
+  }
+
+  output->buffers[0] = std::move(out_validity);
+  output->GetNullCount();  // update null count
+  return Status::OK();
+}
+
+// cond.valid && (cond.data && left.valid || ~cond.data && right.valid)
+// ASA and AAS
+Status PromoteNulls(KernelContext* ctx, const ArrayData& cond, const Scalar& left,
+                    const ArrayData& right, ArrayData* output) {
+  if (!cond.MayHaveNulls() && left.is_valid && !right.MayHaveNulls()) {
+    return Status::OK();  // no nulls to handle
+  }
+  const int64_t len = cond.length;
+
+  // out_validity = ~cond.data
+  ARROW_ASSIGN_OR_RAISE(
+      std::shared_ptr<Buffer> out_validity,
+      arrow::internal::InvertBitmap(ctx->memory_pool(), cond.buffers[1]->data(),
+                                    cond.offset, len));
+  // out_validity = ~cond.data && right.valid
+  if (right.MayHaveNulls()) {  // out_validity = right.valid && ~cond.data
+    arrow::internal::BitmapAnd(right.buffers[0]->data(), right.offset,
+                               out_validity->data(), 0, len, 0,
+                               out_validity->mutable_data());
+  }
+
+  // out_validity = cond.data && left.valid || ~cond.data && right.valid
+  if (left.is_valid) {
+    arrow::internal::BitmapOr(out_validity->data(), 0, cond.buffers[1]->data(),
+                              cond.offset, len, 0, out_validity->mutable_data());
+  }
+
+  // out_validity = cond.valid && (cond.data && left.valid || ~cond.data && right.valid)
+  if (cond.MayHaveNulls()) {
+    ::arrow::internal::BitmapAnd(out_validity->data(), 0, cond.buffers[0]->data(),
+                                 cond.offset, len, 0, out_validity->mutable_data());
+  }
+
+  output->buffers[0] = std::move(out_validity);
+  output->GetNullCount();  // update null count
+  return Status::OK();
+}
+
+// cond.valid && (cond.data && left.valid || ~cond.data && right.valid)
+// ASS
+Status PromoteNulls(KernelContext* ctx, const ArrayData& cond, const Scalar& left,
+                    const Scalar& right, ArrayData* output) {
+  if (!cond.MayHaveNulls() && left.is_valid && right.is_valid) {
+    return Status::OK();  // no nulls to handle
+  }
+  const int64_t len = cond.length;
+
+  std::shared_ptr<Buffer> out_validity;
+  if (right.is_valid) {
+    // out_validity = ~cond.data
+    ARROW_ASSIGN_OR_RAISE(
+        out_validity, arrow::internal::InvertBitmap(
+                          ctx->memory_pool(), cond.buffers[1]->data(), cond.offset, len));
+  } else {
+    // out_validity = [0...]
+    ARROW_ASSIGN_OR_RAISE(out_validity, ctx->AllocateBitmap(len));
+  }
+
+  // out_validity = cond.data && left.valid || ~cond.data && right.valid
+  if (left.is_valid) {
+    arrow::internal::BitmapOr(out_validity->data(), 0, cond.buffers[1]->data(),
+                              cond.offset, len, 0, out_validity->mutable_data());
+  }
+
+  // out_validity = cond.valid && (cond.data && left.valid || ~cond.data && right.valid)
+  if (cond.MayHaveNulls()) {
+    ::arrow::internal::BitmapAnd(out_validity->data(), 0, cond.buffers[0]->data(),
+                                 cond.offset, len, 0, out_validity->mutable_data());
+  }
+
+  output->buffers[0] = std::move(out_validity);
+  output->GetNullCount();  // update null count
+  return Status::OK();
+}
+
+// todo: this could be dangerous because the inverted arraydata buffer[1] may not be
+//  available outside Exec's scope
+Status InvertBoolArrayData(KernelContext* ctx, const ArrayData& input,
+                           ArrayData* output) {
+  // null buffer
+  if (input.MayHaveNulls()) {
+    output->buffers.emplace_back(
+        SliceBuffer(input.buffers[0], input.offset, input.length));
+  } else {
+    output->buffers.push_back(NULLPTR);
+  }
+
+  // data buffer
+  ARROW_ASSIGN_OR_RAISE(
+      std::shared_ptr<Buffer> inv_data,
+      arrow::internal::InvertBitmap(ctx->memory_pool(), input.buffers[1]->data(),
+                                    input.offset, input.length));
+  output->buffers.emplace_back(std::move(inv_data));
+  return Status::OK();
+}
+ */
+
+template <typename Type, typename Enable = void>
+struct IfElseFunctor {};
+
+// only number types needs to be handled for Fixed sized primitive data types because,
+// internal::GenerateTypeAgnosticPrimitive forwards types to the corresponding unsigned
+// int type
+template <typename Type>
+struct IfElseFunctor<Type, enable_if_number<Type>> {
+  using T = typename TypeTraits<Type>::CType;
+  // A - Array
+  // S - Scalar
+
+  //  AAA
+  static Status Call(KernelContext* ctx, const ArrayData& cond, const ArrayData& left,
+                     const ArrayData& right, ArrayData* out) {
+    ARROW_RETURN_NOT_OK(PromoteNullsVisitor(ctx, cond, left, right, out));
+
+    ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Buffer> out_buf,
+                          ctx->Allocate(cond.length * sizeof(T)));
+    T* out_values = reinterpret_cast<T*>(out_buf->mutable_data());
+
+    // copy right data to out_buff
+    const T* right_data = right.GetValues<T>(1);
+    std::memcpy(out_values, right_data, right.length * sizeof(T));
+
+    const auto* cond_data = cond.buffers[1]->data();  // this is a BoolArray
+    BitBlockCounter bit_counter(cond_data, cond.offset, cond.length);
+
+    // selectively copy values from left data
+    const T* left_data = left.GetValues<T>(1);
+    int64_t offset = cond.offset;
+
+    // todo this can be improved by intrinsics. ex: _mm*_mask_store_e* (vmovdqa*)
+    while (offset < cond.offset + cond.length) {
+      const BitBlockCount& block = bit_counter.NextWord();
+      if (block.AllSet()) {  // all from left
+        std::memcpy(out_values, left_data, block.length * sizeof(T));
+      } else if (block.popcount) {  // selectively copy from left
+        for (int64_t i = 0; i < block.length; ++i) {
+          if (BitUtil::GetBit(cond_data, offset + i)) {
+            out_values[i] = left_data[i];
+          }
+        }
+      }
+
+      offset += block.length;
+      out_values += block.length;
+      left_data += block.length;
+    }
+
+    out->buffers[1] = std::move(out_buf);
+    return Status::OK();
+  }
+
+  // ASA
+  static Status Call(KernelContext* ctx, const ArrayData& cond, const Scalar& left,
+                     const ArrayData& right, ArrayData* out) {
+    ARROW_RETURN_NOT_OK(PromoteNullsVisitor(ctx, cond, left, right, out));
+
+    ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Buffer> out_buf,
+                          ctx->Allocate(cond.length * sizeof(T)));
+    T* out_values = reinterpret_cast<T*>(out_buf->mutable_data());
+
+    // copy right data to out_buff
+    const T* right_data = right.GetValues<T>(1);
+    std::memcpy(out_values, right_data, right.length * sizeof(T));
+
+    const auto* cond_data = cond.buffers[1]->data();  // this is a BoolArray
+    BitBlockCounter bit_counter(cond_data, cond.offset, cond.length);
+
+    // selectively copy values from left data
+    T left_data = internal::UnboxScalar<Type>::Unbox(left);
+    int64_t offset = cond.offset;
+
+    // todo this can be improved by intrinsics. ex: _mm*_mask_store_e* (vmovdqa*)
+    while (offset < cond.offset + cond.length) {
+      const BitBlockCount& block = bit_counter.NextWord();
+      if (block.AllSet()) {  // all from left
+        std::fill(out_values, out_values + block.length, left_data);
+      } else if (block.popcount) {  // selectively copy from left
+        for (int64_t i = 0; i < block.length; ++i) {
+          if (BitUtil::GetBit(cond_data, offset + i)) {
+            out_values[i] = left_data;
+          }
+        }
+      }
+
+      offset += block.length;
+      out_values += block.length;
+    }
+
+    out->buffers[1] = std::move(out_buf);
+    return Status::OK();
+  }
+
+  // AAS
+  static Status Call(KernelContext* ctx, const ArrayData& cond, const ArrayData& left,
+                     const Scalar& right, ArrayData* out) {
+    ARROW_RETURN_NOT_OK(PromoteNullsVisitor(ctx, cond, left, right, out));
+
+    ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Buffer> out_buf,
+                          ctx->Allocate(cond.length * sizeof(T)));
+    T* out_values = reinterpret_cast<T*>(out_buf->mutable_data());
+
+    // copy left data to out_buff
+    const T* left_data = left.GetValues<T>(1);
+    std::memcpy(out_values, left_data, left.length * sizeof(T));
+
+    const auto* cond_data = cond.buffers[1]->data();  // this is a BoolArray
+    BitBlockCounter bit_counter(cond_data, cond.offset, cond.length);
+
+    // selectively copy values from left data
+    T right_data = internal::UnboxScalar<Type>::Unbox(right);
+    int64_t offset = cond.offset;
+
+    // todo this can be improved by intrinsics. ex: _mm*_mask_store_e* (vmovdqa*)
+    // left data is already in the output buffer. Therefore, mask needs to be inverted
+    while (offset < cond.offset + cond.length) {
+      const BitBlockCount& block = bit_counter.NextWord();
+      if (block.NoneSet()) {  // all from right
+        std::fill(out_values, out_values + block.length, right_data);
+      } else if (block.popcount) {  // selectively copy from right
+        for (int64_t i = 0; i < block.length; ++i) {
+          if (!BitUtil::GetBit(cond_data, offset + i)) {
+            out_values[i] = right_data;
+          }
+        }
+      }
+
+      offset += block.length;
+      out_values += block.length;
+    }
+
+    out->buffers[1] = std::move(out_buf);
+    return Status::OK();
+  }
+
+  // ASS
+  static Status Call(KernelContext* ctx, const ArrayData& cond, const Scalar& left,
+                     const Scalar& right, ArrayData* out) {
+    ARROW_RETURN_NOT_OK(PromoteNullsVisitor(ctx, cond, left, right, out));
+
+    ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Buffer> out_buf,
+                          ctx->Allocate(cond.length * sizeof(T)));
+    T* out_values = reinterpret_cast<T*>(out_buf->mutable_data());
+
+    // copy right data to out_buff
+    T right_data = internal::UnboxScalar<Type>::Unbox(right);
+    std::fill(out_values, out_values + cond.length, right_data);
+
+    const auto* cond_data = cond.buffers[1]->data();  // this is a BoolArray
+    BitBlockCounter bit_counter(cond_data, cond.offset, cond.length);
+
+    // selectively copy values from left data
+    T left_data = internal::UnboxScalar<Type>::Unbox(left);
+    int64_t offset = cond.offset;
+
+    // todo this can be improved by intrinsics. ex: _mm*_mask_store_e* (vmovdqa*)
+    while (offset < cond.offset + cond.length) {
+      const BitBlockCount& block = bit_counter.NextWord();
+      if (block.AllSet()) {  // all from left
+        std::fill(out_values, out_values + block.length, left_data);
+      } else if (block.popcount) {  // selectively copy from left
+        for (int64_t i = 0; i < block.length; ++i) {
+          if (BitUtil::GetBit(cond_data, offset + i)) {
+            out_values[i] = left_data;
+          }
+        }
+      }
+
+      offset += block.length;
+      out_values += block.length;
+    }
+
+    out->buffers[1] = std::move(out_buf);
+    return Status::OK();
+  }
+};
+
+template <typename Type>
+struct IfElseFunctor<Type, enable_if_boolean<Type>> {
+  // AAA
+  static Status Call(KernelContext* ctx, const ArrayData& cond, const ArrayData& left,
+                     const ArrayData& right, ArrayData* out) {
+    ARROW_RETURN_NOT_OK(PromoteNullsVisitor(ctx, cond, left, right, out));
+
+    // out_buff = right & ~cond
+    ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Buffer> out_buf,
+                          arrow::internal::BitmapAndNot(
+                              ctx->memory_pool(), right.buffers[1]->data(), right.offset,
+                              cond.buffers[1]->data(), cond.offset, cond.length, 0));
+
+    // out_buff = left & cond
+    ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Buffer> temp_buf,
+                          arrow::internal::BitmapAnd(
+                              ctx->memory_pool(), left.buffers[1]->data(), left.offset,
+                              cond.buffers[1]->data(), cond.offset, cond.length, 0));
+
+    arrow::internal::BitmapOr(out_buf->data(), 0, temp_buf->data(), 0, cond.length, 0,
+                              out_buf->mutable_data());
+    out->buffers[1] = std::move(out_buf);
+    return Status::OK();
+  }
+
+  // ASA
+  static Status Call(KernelContext* ctx, const ArrayData& cond, const Scalar& left,
+                     const ArrayData& right, ArrayData* out) {
+    ARROW_RETURN_NOT_OK(PromoteNullsVisitor(ctx, cond, left, right, out));
+
+    // out_buff = right & ~cond
+    ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Buffer> out_buf,
+                          arrow::internal::BitmapAndNot(
+                              ctx->memory_pool(), right.buffers[1]->data(), right.offset,
+                              cond.buffers[1]->data(), cond.offset, cond.length, 0));
+
+    // out_buff = left & cond
+    bool left_data = internal::UnboxScalar<BooleanType>::Unbox(left);
+    if (left_data) {
+      arrow::internal::BitmapOr(out_buf->data(), 0, cond.buffers[1]->data(), cond.offset,
+                                cond.length, 0, out_buf->mutable_data());
+    }
+
+    out->buffers[1] = std::move(out_buf);
+    return Status::OK();
+  }
+
+  // AAS
+  static Status Call(KernelContext* ctx, const ArrayData& cond, const ArrayData& left,
+                     const Scalar& right, ArrayData* out) {
+    // todo impl
+    return Status::OK();
+  }
+
+  // ASS
+  static Status Call(KernelContext* ctx, const ArrayData& cond, const Scalar& left,
+                     const Scalar& right, ArrayData* out) {
+    // todo impl
+    return Status::OK();
+  }
+};
+
+template <typename Type>
+struct IfElseFunctor<Type, enable_if_null<Type>> {
+  template <typename T>
+  static inline Status ReturnCopy(const T& in, T* out) {
+    // Nothing preallocated, so we assign in into the output
+    *out = in;
+    return Status::OK();
+  }
+
+  // AAA
+  static Status Call(KernelContext* ctx, const ArrayData& cond, const ArrayData& left,
+                     const ArrayData& right, ArrayData* out) {
+    return ReturnCopy(left, out);
+  }
+
+  // ASA
+  static Status Call(KernelContext* ctx, const ArrayData& cond, const Scalar& left,
+                     const ArrayData& right, ArrayData* out) {
+    return ReturnCopy(right, out);
+  }
+
+  // AAS
+  static Status Call(KernelContext* ctx, const ArrayData& cond, const ArrayData& left,
+                     const Scalar& right, ArrayData* out) {
+    return ReturnCopy(left, out);
+  }
+
+  // ASS
+  static Status Call(KernelContext* ctx, const ArrayData& cond, const Scalar& left,
+                     const Scalar& right, ArrayData* out) {
+    return ReturnCopy(cond, out);
+  }
+};
+
+template <typename Type>
+struct ResolveIfElseExec {
+  static Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    // cond is scalar

Review comment:
       Scalar conditions don't seem tested 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.

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



[GitHub] [arrow] nirandaperera commented on a change in pull request #10410: ARROW-10640: [C++] A "where" kernel to combine two arrays based on a mask

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_if_else.cc
##########
@@ -0,0 +1,740 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <arrow/compute/api.h>
+#include <arrow/compute/kernels/codegen_internal.h>
+#include <arrow/compute/util_internal.h>
+#include <arrow/util/bit_block_counter.h>
+#include <arrow/util/bitmap.h>
+#include <arrow/util/bitmap_ops.h>
+
+namespace arrow {
+using internal::BitBlockCount;
+using internal::BitBlockCounter;
+using internal::Bitmap;
+
+namespace compute {
+
+namespace {
+
+enum { COND_ALL_VALID = 1, LEFT_ALL_VALID = 2, RIGHT_ALL_VALID = 4 };
+
+// if the condition is null then output is null otherwise we take validity from the
+// selected argument
+// ie. cond.valid & (cond.data & left.valid | ~cond.data & right.valid)
+Status PromoteNullsVisitor(KernelContext* ctx, const ArrayData& cond, const Scalar& left,
+                           const Scalar& right, ArrayData* output) {
+  uint8_t flag = right.is_valid * 4 + left.is_valid * 2 + !cond.MayHaveNulls();
+
+  if (flag < 6 && flag != 3) {
+    // there will be a validity buffer in the output
+    ARROW_ASSIGN_OR_RAISE(output->buffers[0], ctx->AllocateBitmap(cond.length));
+  }
+
+  // if the condition is null then output is null otherwise we take validity from the
+  // selected argument
+  // ie. cond.valid & (cond.data & left.valid | ~cond.data & right.valid)
+  switch (flag) {
+    case COND_ALL_VALID | LEFT_ALL_VALID | RIGHT_ALL_VALID:  // = 7
+      break;
+    case LEFT_ALL_VALID | RIGHT_ALL_VALID:  // = 6
+      // out_valid = c_valid
+      output->buffers[0] = SliceBuffer(cond.buffers[0], cond.offset, cond.length);
+      break;
+    case COND_ALL_VALID | RIGHT_ALL_VALID:  // = 5
+      // out_valid = ~cond.data
+      arrow::internal::InvertBitmap(cond.buffers[1]->data(), cond.offset, cond.length,
+                                    output->buffers[0]->mutable_data(), 0);
+      break;
+    case RIGHT_ALL_VALID:  // = 4
+      // out_valid = c_valid & ~cond.data
+      arrow::internal::BitmapAndNot(cond.buffers[0]->data(), cond.offset,
+                                    cond.buffers[1]->data(), cond.offset, cond.length, 0,
+                                    output->buffers[0]->mutable_data());
+      break;
+    case COND_ALL_VALID | LEFT_ALL_VALID:  // = 3
+      // out_valid = cond.data
+      output->buffers[0] = SliceBuffer(cond.buffers[1], cond.offset, cond.length);
+      break;
+    case LEFT_ALL_VALID:  // = 2
+      // out_valid = cond.valid & cond.data
+      arrow::internal::BitmapAnd(cond.buffers[0]->data(), cond.offset,
+                                 cond.buffers[1]->data(), cond.offset, cond.length, 0,
+                                 output->buffers[0]->mutable_data());
+      break;
+    case COND_ALL_VALID:  // = 1
+      // out_valid = 0 --> nothing to do; but requires out_valid to be a all-zero buffer
+      break;
+    case 0:
+      // out_valid = 0 --> nothing to do; but requires out_valid to be a all-zero buffer
+      break;
+  }
+  return Status::OK();
+}
+
+Status PromoteNullsVisitor(KernelContext* ctx, const ArrayData& cond,
+                           const ArrayData& left, const Scalar& right,
+                           ArrayData* output) {
+  uint8_t flag = right.is_valid * 4 + !left.MayHaveNulls() * 2 + !cond.MayHaveNulls();
+
+  enum { C_VALID, C_DATA, L_VALID };
+
+  Bitmap bitmaps[3];
+  bitmaps[C_VALID] = {cond.buffers[0], cond.offset, cond.length};
+  bitmaps[C_DATA] = {cond.buffers[1], cond.offset, cond.length};
+  bitmaps[L_VALID] = {left.buffers[0], left.offset, left.length};
+
+  uint64_t* out_validity = nullptr;
+  if (flag < 6 && flag != 3) {
+    // there will be a validity buffer in the output
+    ARROW_ASSIGN_OR_RAISE(output->buffers[0], ctx->AllocateBitmap(cond.length));
+    out_validity = output->GetMutableValues<uint64_t>(0);
+  }
+
+  // lambda function that will be used inside the visitor
+  int64_t i = 0;
+  auto apply = [&](uint64_t c_valid, uint64_t c_data, uint64_t l_valid,
+                   uint64_t r_valid) {
+    out_validity[i] = c_valid & ((c_data & l_valid) | (~c_data & r_valid));
+    i++;
+  };
+
+  // if the condition is null then output is null otherwise we take validity from the
+  // selected argument
+  // ie. cond.valid & (cond.data & left.valid | ~cond.data & right.valid)
+  switch (flag) {
+    case COND_ALL_VALID | LEFT_ALL_VALID | RIGHT_ALL_VALID:
+      break;
+    case LEFT_ALL_VALID | RIGHT_ALL_VALID:
+      output->buffers[0] = SliceBuffer(cond.buffers[0], cond.offset, cond.length);
+      break;
+    case COND_ALL_VALID | RIGHT_ALL_VALID:
+      // bitmaps[C_VALID] might be null; override to make it safe for Visit()
+      bitmaps[C_VALID] = bitmaps[C_DATA];
+      Bitmap::VisitWords(bitmaps, [&](std::array<uint64_t, 3> words) {
+        apply(UINT64_MAX, words[C_DATA], words[L_VALID], UINT64_MAX);
+      });
+      break;
+    case RIGHT_ALL_VALID:
+      Bitmap::VisitWords(bitmaps, [&](std::array<uint64_t, 3> words) {
+        apply(words[C_VALID], words[C_DATA], words[L_VALID], UINT64_MAX);
+      });
+      break;
+    case COND_ALL_VALID | LEFT_ALL_VALID:
+      // only cond.data is passed
+      output->buffers[0] = SliceBuffer(cond.buffers[1], cond.offset, cond.length);
+      break;
+    case LEFT_ALL_VALID:
+      // out_valid = cond.valid & cond.data
+      arrow::internal::BitmapAnd(cond.buffers[0]->data(), cond.offset,
+                                 cond.buffers[1]->data(), cond.offset, cond.length, 0,
+                                 output->buffers[0]->mutable_data());
+      break;
+    case COND_ALL_VALID:
+      // out_valid = cond.data & left.valid
+      arrow::internal::BitmapAnd(cond.buffers[1]->data(), cond.offset,
+                                 left.buffers[0]->data(), left.offset, cond.length, 0,
+                                 output->buffers[0]->mutable_data());
+      break;
+    case 0:
+      Bitmap::VisitWords(bitmaps, [&](std::array<uint64_t, 3> words) {
+        apply(words[C_VALID], words[C_DATA], words[L_VALID], 0);
+      });
+      break;
+  }
+  return Status::OK();
+}
+
+// if the condition is null then output is null otherwise we take validity from the
+// selected argument
+// ie. cond.valid & (cond.data & left.valid | ~cond.data & right.valid)
+Status PromoteNullsVisitor(KernelContext* ctx, const ArrayData& cond, const Scalar& left,
+                           const ArrayData& right, ArrayData* output) {
+  uint8_t flag = !right.MayHaveNulls() * 4 + left.is_valid * 2 + !cond.MayHaveNulls();
+
+  enum { C_VALID, C_DATA, R_VALID };
+
+  Bitmap bitmaps[3];
+  bitmaps[C_VALID] = {cond.buffers[0], cond.offset, cond.length};
+  bitmaps[C_DATA] = {cond.buffers[1], cond.offset, cond.length};
+  bitmaps[R_VALID] = {right.buffers[0], right.offset, right.length};
+
+  uint64_t* out_validity = nullptr;
+  if (flag < 6) {
+    // there will be a validity buffer in the output
+    ARROW_ASSIGN_OR_RAISE(output->buffers[0], ctx->AllocateBitmap(cond.length));
+    out_validity = output->GetMutableValues<uint64_t>(0);
+  }
+
+  // lambda function that will be used inside the visitor
+  int64_t i = 0;
+  auto apply = [&](uint64_t c_valid, uint64_t c_data, uint64_t l_valid,
+                   uint64_t r_valid) {
+    out_validity[i] = c_valid & ((c_data & l_valid) | (~c_data & r_valid));
+    i++;
+  };
+
+  // if the condition is null then output is null otherwise we take validity from the
+  // selected argument
+  // ie. cond.valid & (cond.data & left.valid | ~cond.data & right.valid)
+  switch (flag) {
+    case COND_ALL_VALID | LEFT_ALL_VALID | RIGHT_ALL_VALID:
+      break;
+    case LEFT_ALL_VALID | RIGHT_ALL_VALID:
+      output->buffers[0] = SliceBuffer(cond.buffers[0], cond.offset, cond.length);
+      break;
+    case COND_ALL_VALID | RIGHT_ALL_VALID:
+      // out_valid = ~cond.data
+      arrow::internal::InvertBitmap(cond.buffers[1]->data(), cond.offset, cond.length,
+                                    output->buffers[0]->mutable_data(), 0);
+      break;
+    case RIGHT_ALL_VALID:
+      // out_valid = c_valid & ~cond.data
+      arrow::internal::BitmapAndNot(cond.buffers[0]->data(), cond.offset,
+                                    cond.buffers[1]->data(), cond.offset, cond.length, 0,
+                                    output->buffers[0]->mutable_data());
+      break;
+    case COND_ALL_VALID | LEFT_ALL_VALID:
+      // bitmaps[C_VALID] might be null; override to make it safe for Visit()
+      bitmaps[C_VALID] = bitmaps[C_DATA];
+      Bitmap::VisitWords(bitmaps, [&](std::array<uint64_t, 3> words) {
+        apply(UINT64_MAX, words[C_DATA], UINT64_MAX, words[R_VALID]);
+      });
+      break;
+    case LEFT_ALL_VALID:
+      Bitmap::VisitWords(bitmaps, [&](std::array<uint64_t, 3> words) {
+        apply(words[C_VALID], words[C_DATA], UINT64_MAX, words[R_VALID]);
+      });
+      break;
+    case COND_ALL_VALID:
+      // out_valid =  ~cond.data & right.valid
+      arrow::internal::BitmapAndNot(right.buffers[0]->data(), right.offset,
+                                    cond.buffers[1]->data(), cond.offset, cond.length, 0,
+                                    output->buffers[0]->mutable_data());
+      break;
+    case 0:
+      Bitmap::VisitWords(bitmaps, [&](std::array<uint64_t, 3> words) {
+        apply(words[C_VALID], words[C_DATA], 0, words[R_VALID]);
+      });
+      break;
+  }
+  return Status::OK();
+}
+
+// if the condition is null then output is null otherwise we take validity from the
+// selected argument
+// ie. cond.valid & (cond.data & left.valid | ~cond.data & right.valid)
+Status PromoteNullsVisitor(KernelContext* ctx, const ArrayData& cond,
+                           const ArrayData& left, const ArrayData& right,
+                           ArrayData* output) {
+  uint8_t flag =
+      !right.MayHaveNulls() * 4 + !left.MayHaveNulls() * 2 + !cond.MayHaveNulls();
+
+  enum { C_VALID, C_DATA, L_VALID, R_VALID };
+
+  Bitmap bitmaps[4];
+  bitmaps[C_VALID] = {cond.buffers[0], cond.offset, cond.length};
+  bitmaps[C_DATA] = {cond.buffers[1], cond.offset, cond.length};
+  bitmaps[L_VALID] = {left.buffers[0], left.offset, left.length};
+  bitmaps[R_VALID] = {right.buffers[0], right.offset, right.length};
+
+  uint64_t* out_validity = nullptr;
+  if (flag < 6) {
+    // there will be a validity buffer in the output
+    ARROW_ASSIGN_OR_RAISE(output->buffers[0], ctx->AllocateBitmap(cond.length));
+    out_validity = output->GetMutableValues<uint64_t>(0);
+  }
+
+  // lambda function that will be used inside the visitor
+  int64_t i = 0;
+  auto apply = [&](uint64_t c_valid, uint64_t c_data, uint64_t l_valid,
+                   uint64_t r_valid) {
+    out_validity[i] = c_valid & ((c_data & l_valid) | (~c_data & r_valid));
+    i++;
+  };
+
+  // if the condition is null then output is null otherwise we take validity from the
+  // selected argument
+  // ie. cond.valid & (cond.data & left.valid | ~cond.data & right.valid)
+  switch (flag) {
+    case COND_ALL_VALID | LEFT_ALL_VALID | RIGHT_ALL_VALID:
+      break;
+    case LEFT_ALL_VALID | RIGHT_ALL_VALID:
+      output->buffers[0] = SliceBuffer(cond.buffers[0], cond.offset, cond.length);
+      break;
+    case COND_ALL_VALID | RIGHT_ALL_VALID:
+      // bitmaps[C_VALID], bitmaps[R_VALID] might be null; override to make it safe for
+      // Visit()
+      bitmaps[C_VALID] = bitmaps[C_DATA];
+      bitmaps[R_VALID] = bitmaps[C_DATA];
+      Bitmap::VisitWords(bitmaps, [&](std::array<uint64_t, 4> words) {
+        apply(UINT64_MAX, words[C_DATA], words[L_VALID], UINT64_MAX);
+      });
+      break;
+    case RIGHT_ALL_VALID:
+      // bitmaps[R_VALID] might be null; override to make it safe for Visit()
+      bitmaps[R_VALID] = bitmaps[C_DATA];
+      Bitmap::VisitWords(bitmaps, [&](std::array<uint64_t, 4> words) {
+        apply(words[C_VALID], words[C_DATA], words[L_VALID], UINT64_MAX);
+      });
+      break;
+    case COND_ALL_VALID | LEFT_ALL_VALID:
+      // bitmaps[C_VALID], bitmaps[L_VALID] might be null; override to make it safe for
+      // Visit()
+      bitmaps[C_VALID] = bitmaps[C_DATA];
+      bitmaps[L_VALID] = bitmaps[C_DATA];
+      Bitmap::VisitWords(bitmaps, [&](std::array<uint64_t, 4> words) {
+        apply(UINT64_MAX, words[C_DATA], UINT64_MAX, words[R_VALID]);
+      });
+      break;
+    case LEFT_ALL_VALID:
+      // bitmaps[L_VALID] might be null; override to make it safe for Visit()
+      bitmaps[L_VALID] = bitmaps[C_DATA];
+      Bitmap::VisitWords(bitmaps, [&](std::array<uint64_t, 4> words) {
+        apply(words[C_VALID], words[C_DATA], UINT64_MAX, words[R_VALID]);
+      });
+      break;
+    case COND_ALL_VALID:
+      // bitmaps[C_VALID] might be null; override to make it safe for Visit()
+      bitmaps[C_VALID] = bitmaps[C_DATA];
+      Bitmap::VisitWords(bitmaps, [&](std::array<uint64_t, 4> words) {
+        apply(UINT64_MAX, words[C_DATA], words[L_VALID], words[R_VALID]);
+      });
+      break;
+    case 0:
+      Bitmap::VisitWords(bitmaps, [&](std::array<uint64_t, 4> words) {
+        apply(words[C_VALID], words[C_DATA], words[L_VALID], words[R_VALID]);
+      });
+      break;
+  }
+  return Status::OK();
+}
+
+template <typename Type, typename Enable = void>
+struct IfElseFunctor {};
+
+// only number types needs to be handled for Fixed sized primitive data types because,
+// internal::GenerateTypeAgnosticPrimitive forwards types to the corresponding unsigned
+// int type
+template <typename Type>
+struct IfElseFunctor<Type, enable_if_number<Type>> {
+  using T = typename TypeTraits<Type>::CType;
+  // A - Array
+  // S - Scalar
+
+  //  AAA
+  static Status Call(KernelContext* ctx, const ArrayData& cond, const ArrayData& left,
+                     const ArrayData& right, ArrayData* out) {
+    ARROW_RETURN_NOT_OK(PromoteNullsVisitor(ctx, cond, left, right, out));
+
+    ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Buffer> out_buf,
+                          ctx->Allocate(cond.length * sizeof(T)));
+    T* out_values = reinterpret_cast<T*>(out_buf->mutable_data());
+
+    // copy right data to out_buff
+    const T* right_data = right.GetValues<T>(1);
+    std::memcpy(out_values, right_data, right.length * sizeof(T));
+
+    const auto* cond_data = cond.buffers[1]->data();  // this is a BoolArray
+    BitBlockCounter bit_counter(cond_data, cond.offset, cond.length);
+
+    // selectively copy values from left data
+    const T* left_data = left.GetValues<T>(1);
+    int64_t offset = cond.offset;
+
+    // todo this can be improved by intrinsics. ex: _mm*_mask_store_e* (vmovdqa*)
+    while (offset < cond.offset + cond.length) {
+      const BitBlockCount& block = bit_counter.NextWord();
+      if (block.AllSet()) {  // all from left
+        std::memcpy(out_values, left_data, block.length * sizeof(T));
+      } else if (block.popcount) {  // selectively copy from left
+        for (int64_t i = 0; i < block.length; ++i) {
+          if (BitUtil::GetBit(cond_data, offset + i)) {
+            out_values[i] = left_data[i];
+          }
+        }
+      }
+
+      offset += block.length;
+      out_values += block.length;
+      left_data += block.length;
+    }
+
+    out->buffers[1] = std::move(out_buf);
+    return Status::OK();
+  }
+
+  // ASA
+  static Status Call(KernelContext* ctx, const ArrayData& cond, const Scalar& left,
+                     const ArrayData& right, ArrayData* out) {
+    ARROW_RETURN_NOT_OK(PromoteNullsVisitor(ctx, cond, left, right, out));
+
+    ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Buffer> out_buf,
+                          ctx->Allocate(cond.length * sizeof(T)));
+    T* out_values = reinterpret_cast<T*>(out_buf->mutable_data());
+
+    // copy right data to out_buff
+    const T* right_data = right.GetValues<T>(1);
+    std::memcpy(out_values, right_data, right.length * sizeof(T));
+
+    const auto* cond_data = cond.buffers[1]->data();  // this is a BoolArray
+    BitBlockCounter bit_counter(cond_data, cond.offset, cond.length);
+
+    // selectively copy values from left data
+    T left_data = internal::UnboxScalar<Type>::Unbox(left);
+    int64_t offset = cond.offset;
+
+    // todo this can be improved by intrinsics. ex: _mm*_mask_store_e* (vmovdqa*)
+    while (offset < cond.offset + cond.length) {
+      const BitBlockCount& block = bit_counter.NextWord();
+      if (block.AllSet()) {  // all from left
+        std::fill(out_values, out_values + block.length, left_data);
+      } else if (block.popcount) {  // selectively copy from left
+        for (int64_t i = 0; i < block.length; ++i) {
+          if (BitUtil::GetBit(cond_data, offset + i)) {
+            out_values[i] = left_data;
+          }
+        }
+      }
+
+      offset += block.length;
+      out_values += block.length;
+    }
+
+    out->buffers[1] = std::move(out_buf);
+    return Status::OK();
+  }
+
+  // AAS
+  static Status Call(KernelContext* ctx, const ArrayData& cond, const ArrayData& left,
+                     const Scalar& right, ArrayData* out) {
+    ARROW_RETURN_NOT_OK(PromoteNullsVisitor(ctx, cond, left, right, out));
+
+    ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Buffer> out_buf,
+                          ctx->Allocate(cond.length * sizeof(T)));
+    T* out_values = reinterpret_cast<T*>(out_buf->mutable_data());
+
+    // copy left data to out_buff
+    const T* left_data = left.GetValues<T>(1);
+    std::memcpy(out_values, left_data, left.length * sizeof(T));
+
+    const auto* cond_data = cond.buffers[1]->data();  // this is a BoolArray
+    BitBlockCounter bit_counter(cond_data, cond.offset, cond.length);
+
+    // selectively copy values from left data
+    T right_data = internal::UnboxScalar<Type>::Unbox(right);
+    int64_t offset = cond.offset;
+
+    // todo this can be improved by intrinsics. ex: _mm*_mask_store_e* (vmovdqa*)
+    // left data is already in the output buffer. Therefore, mask needs to be inverted
+    while (offset < cond.offset + cond.length) {
+      const BitBlockCount& block = bit_counter.NextWord();
+      if (block.NoneSet()) {  // all from right
+        std::fill(out_values, out_values + block.length, right_data);
+      } else if (block.popcount) {  // selectively copy from right
+        for (int64_t i = 0; i < block.length; ++i) {
+          if (!BitUtil::GetBit(cond_data, offset + i)) {
+            out_values[i] = right_data;
+          }
+        }
+      }
+
+      offset += block.length;
+      out_values += block.length;
+    }
+
+    out->buffers[1] = std::move(out_buf);
+    return Status::OK();
+  }
+
+  // ASS
+  static Status Call(KernelContext* ctx, const ArrayData& cond, const Scalar& left,
+                     const Scalar& right, ArrayData* out) {
+    ARROW_RETURN_NOT_OK(PromoteNullsVisitor(ctx, cond, left, right, out));
+
+    ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Buffer> out_buf,
+                          ctx->Allocate(cond.length * sizeof(T)));
+    T* out_values = reinterpret_cast<T*>(out_buf->mutable_data());
+
+    // copy right data to out_buff
+    T right_data = internal::UnboxScalar<Type>::Unbox(right);
+    std::fill(out_values, out_values + cond.length, right_data);
+
+    const auto* cond_data = cond.buffers[1]->data();  // this is a BoolArray
+    BitBlockCounter bit_counter(cond_data, cond.offset, cond.length);
+
+    // selectively copy values from left data
+    T left_data = internal::UnboxScalar<Type>::Unbox(left);
+    int64_t offset = cond.offset;
+
+    // todo this can be improved by intrinsics. ex: _mm*_mask_store_e* (vmovdqa*)
+    while (offset < cond.offset + cond.length) {
+      const BitBlockCount& block = bit_counter.NextWord();
+      if (block.AllSet()) {  // all from left
+        std::fill(out_values, out_values + block.length, left_data);
+      } else if (block.popcount) {  // selectively copy from left
+        for (int64_t i = 0; i < block.length; ++i) {
+          if (BitUtil::GetBit(cond_data, offset + i)) {
+            out_values[i] = left_data;
+          }
+        }
+      }
+
+      offset += block.length;
+      out_values += block.length;
+    }
+
+    out->buffers[1] = std::move(out_buf);
+    return Status::OK();
+  }
+};
+
+template <typename Type>
+struct IfElseFunctor<Type, enable_if_boolean<Type>> {
+  // AAA
+  static Status Call(KernelContext* ctx, const ArrayData& cond, const ArrayData& left,
+                     const ArrayData& right, ArrayData* out) {
+    ARROW_RETURN_NOT_OK(PromoteNullsVisitor(ctx, cond, left, right, out));
+
+    // out_buff = right & ~cond
+    ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Buffer> out_buf,
+                          arrow::internal::BitmapAndNot(
+                              ctx->memory_pool(), right.buffers[1]->data(), right.offset,
+                              cond.buffers[1]->data(), cond.offset, cond.length, 0));
+
+    // out_buff = left & cond
+    ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Buffer> temp_buf,
+                          arrow::internal::BitmapAnd(
+                              ctx->memory_pool(), left.buffers[1]->data(), left.offset,
+                              cond.buffers[1]->data(), cond.offset, cond.length, 0));
+
+    arrow::internal::BitmapOr(out_buf->data(), 0, temp_buf->data(), 0, cond.length, 0,
+                              out_buf->mutable_data());
+    out->buffers[1] = std::move(out_buf);
+    return Status::OK();
+  }
+
+  // ASA
+  static Status Call(KernelContext* ctx, const ArrayData& cond, const Scalar& left,
+                     const ArrayData& right, ArrayData* out) {
+    ARROW_RETURN_NOT_OK(PromoteNullsVisitor(ctx, cond, left, right, out));
+
+    // out_buff = right & ~cond
+    ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Buffer> out_buf,
+                          arrow::internal::BitmapAndNot(
+                              ctx->memory_pool(), right.buffers[1]->data(), right.offset,
+                              cond.buffers[1]->data(), cond.offset, cond.length, 0));
+
+    // out_buff = left & cond
+    bool left_data = internal::UnboxScalar<BooleanType>::Unbox(left);
+    if (left_data) {
+      arrow::internal::BitmapOr(out_buf->data(), 0, cond.buffers[1]->data(), cond.offset,
+                                cond.length, 0, out_buf->mutable_data());
+    }
+
+    out->buffers[1] = std::move(out_buf);
+    return Status::OK();
+  }
+
+  // AAS
+  static Status Call(KernelContext* ctx, const ArrayData& cond, const ArrayData& left,
+                     const Scalar& right, ArrayData* out) {
+    ARROW_RETURN_NOT_OK(PromoteNullsVisitor(ctx, cond, left, right, out));
+
+    // out_buff = left & cond
+    ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Buffer> out_buf,
+                          arrow::internal::BitmapAnd(
+                              ctx->memory_pool(), left.buffers[1]->data(), left.offset,
+                              cond.buffers[1]->data(), cond.offset, cond.length, 0));
+
+    bool right_data = internal::UnboxScalar<BooleanType>::Unbox(right);
+
+    // out_buff = left & cond | right & ~cond
+    if (right_data) {
+      arrow::internal::BitmapOrNot(out_buf->data(), 0, cond.buffers[1]->data(),
+                                   cond.offset, cond.length, 0, out_buf->mutable_data());
+    }
+
+    out->buffers[1] = std::move(out_buf);
+    return Status::OK();
+  }
+
+  // ASS
+  static Status Call(KernelContext* ctx, const ArrayData& cond, const Scalar& left,
+                     const Scalar& right, ArrayData* out) {
+    ARROW_RETURN_NOT_OK(PromoteNullsVisitor(ctx, cond, left, right, out));
+
+    bool left_data = internal::UnboxScalar<BooleanType>::Unbox(left);
+    bool right_data = internal::UnboxScalar<BooleanType>::Unbox(right);
+
+    // out_buf = left & cond | right & ~cond
+    std::shared_ptr<Buffer> out_buf = nullptr;
+    if (left_data) {
+      if (right_data) {
+        // out_buf = ones
+        ARROW_ASSIGN_OR_RAISE(out_buf, ctx->AllocateBitmap(cond.length));
+        // filling with UINT8_MAX upto the buffer's size (in bytes)
+        arrow::compute::internal::SetMemory<UINT8_MAX>(out_buf.get());
+      } else {
+        // out_buf = cond
+        out_buf = SliceBuffer(cond.buffers[1], cond.offset, cond.length);
+      }
+    } else {
+      if (right_data) {
+        // out_buf = ~cond
+        ARROW_ASSIGN_OR_RAISE(out_buf, arrow::internal::InvertBitmap(
+                                           ctx->memory_pool(), cond.buffers[1]->data(),
+                                           cond.offset, cond.length))
+      } else {
+        // out_buf = zeros
+        ARROW_ASSIGN_OR_RAISE(out_buf, ctx->AllocateBitmap(cond.length));
+      }
+    }
+    out->buffers[1] = std::move(out_buf);
+    return Status::OK();
+  }
+};
+
+template <typename Type>
+struct IfElseFunctor<Type, enable_if_null<Type>> {
+  template <typename T>
+  static inline Status ReturnCopy(const T& in, T* out) {
+    // Nothing preallocated, so we assign in into the output
+    *out = in;
+    return Status::OK();
+  }
+
+  // AAA
+  static Status Call(KernelContext* ctx, const ArrayData& cond, const ArrayData& left,
+                     const ArrayData& right, ArrayData* out) {
+    return ReturnCopy(left, out);
+  }
+
+  // ASA
+  static Status Call(KernelContext* ctx, const ArrayData& cond, const Scalar& left,
+                     const ArrayData& right, ArrayData* out) {
+    return ReturnCopy(right, out);
+  }
+
+  // AAS
+  static Status Call(KernelContext* ctx, const ArrayData& cond, const ArrayData& left,
+                     const Scalar& right, ArrayData* out) {
+    return ReturnCopy(left, out);
+  }
+
+  // ASS
+  static Status Call(KernelContext* ctx, const ArrayData& cond, const Scalar& left,
+                     const Scalar& right, ArrayData* out) {
+    return ReturnCopy(cond, out);
+  }
+};
+
+template <typename Type>
+struct ResolveIfElseExec {
+  static Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    // cond is scalar
+    if (batch[0].is_scalar()) {
+      const auto& cond = batch[0].scalar_as<BooleanScalar>();
+      if (batch[1].is_scalar() && batch[2].is_scalar()) {
+        if (cond.is_valid) {
+          *out = cond.value ? batch[1].scalar() : batch[2].scalar();
+        } else {
+          *out = MakeNullScalar(batch[1].type());
+        }
+      } else {  // either left or right is an array. output is always an array
+        // output size is the size of the array arg
+        int64_t bcast_size = batch[1].is_array() ? batch[1].length() : batch[2].length();
+        if (cond.is_valid) {
+          const auto& valid_data = cond.value ? batch[1] : batch[2];
+          if (valid_data.is_array()) {
+            *out = valid_data;
+          } else {  // valid data is a scalar that needs to be broadcasted
+            ARROW_ASSIGN_OR_RAISE(*out,
+                                  MakeArrayFromScalar(*valid_data.scalar(), bcast_size,
+                                                      ctx->memory_pool()));
+          }
+        } else {  // cond is null. create null array
+          ARROW_ASSIGN_OR_RAISE(
+              *out, MakeArrayOfNull(batch[1].type(), bcast_size, ctx->memory_pool()))
+        }
+      }
+      return Status::OK();
+    }
+

Review comment:
       Thanks @bkietz for guiding me through this. I added that in the latest commit




-- 
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



[GitHub] [arrow] lidavidm commented on a change in pull request #10410: ARROW-10640: [C++] A "where" kernel to combine two arrays based on a mask

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_if_else.cc
##########
@@ -0,0 +1,572 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <arrow/compute/api.h>
+#include <arrow/compute/kernels/codegen_internal.h>
+#include <arrow/compute/util_internal.h>
+#include <arrow/util/bit_block_counter.h>
+#include <arrow/util/bitmap.h>
+#include <arrow/util/bitmap_ops.h>
+
+namespace arrow {
+using internal::BitBlockCount;
+using internal::BitBlockCounter;
+using internal::Bitmap;
+
+namespace compute {
+
+namespace {
+
+util::optional<uint64_t> GetConstantValidityWord(const Datum& data) {
+  if (data.is_scalar()) {
+    return data.scalar()->is_valid ? ~uint64_t(0) : uint64_t(0);
+  }
+
+  if (data.array()->null_count == data.array()->length) return uint64_t(0);
+
+  if (!data.array()->MayHaveNulls()) return ~uint64_t(0);
+
+  // no constant validity word available
+  return {};
+}
+
+inline Bitmap GetBitmap(const Datum& datum, int i) {
+  if (datum.is_scalar()) return {};
+  const ArrayData& a = *datum.array();
+  return Bitmap{a.buffers[i], a.offset, a.length};
+}
+
+// if the condition is null then output is null otherwise we take validity from the
+// selected argument
+// ie. cond.valid & (cond.data & left.valid | ~cond.data & right.valid)
+Status PromoteNullsVisitor(KernelContext* ctx, const Datum& cond_d, const Datum& left_d,
+                           const Datum& right_d, ArrayData* output) {
+  auto cond_const = GetConstantValidityWord(cond_d);
+  auto left_const = GetConstantValidityWord(left_d);
+  auto right_const = GetConstantValidityWord(right_d);
+
+  enum { COND_CONST = 1, LEFT_CONST = 2, RIGHT_CONST = 4 };
+  auto flag = COND_CONST * cond_const.has_value() | LEFT_CONST * left_const.has_value() |
+              RIGHT_CONST * right_const.has_value();
+
+  const ArrayData& cond = *cond_d.array();
+  // cond.data will always be available
+  Bitmap cond_data{cond.buffers[1], cond.offset, cond.length};
+  Bitmap cond_valid{cond.buffers[0], cond.offset, cond.length};
+  Bitmap left_valid = GetBitmap(left_d, 0);
+  Bitmap right_valid = GetBitmap(right_d, 0);
+  // sometimes Bitmaps will be ignored, in which case we replace access to them with
+  // duplicated (probably elided) access to cond_data
+  const Bitmap& _ = cond_data;
+
+  // lambda function that will be used inside the visitor
+  uint64_t* out_validity = nullptr;
+  int64_t i = 0;
+  auto apply = [&](uint64_t c_valid, uint64_t c_data, uint64_t l_valid,
+                   uint64_t r_valid) {
+    out_validity[i] = c_valid & ((c_data & l_valid) | (~c_data & r_valid));
+    i++;
+  };
+
+  // if the condition is null then output is null otherwise we take validity from the
+  // selected argument
+  // ie. cond.valid & (cond.data & left.valid | ~cond.data & right.valid)
+
+  // In the following cases, we dont need to allocate out_valid bitmap
+  switch (flag) {
+    case COND_CONST | LEFT_CONST | RIGHT_CONST:
+      // if cond & left & right all ones, then output is all valid --> out_valid = nullptr
+      if ((*cond_const & *left_const & *right_const) == UINT64_MAX) {
+        return Status::OK();
+      }
+    case LEFT_CONST | RIGHT_CONST:
+      // if both left and right are valid, no need to calculate out_valid bitmap. Pass
+      // cond validity buffer
+      if ((*left_const & *right_const) == UINT64_MAX) {
+        output->buffers[0] = SliceBuffer(cond.buffers[0], cond.offset, cond.length);

Review comment:
       Example:
   
   ```
   > cat test.py
   import pyarrow as pa
   import pyarrow.compute as pc
   cond = pa.array([None, True, True]).slice(1, 2)
   left = pa.array([10, 11], pa.int64())
   right = pa.array([1, 2], pa.int64())
   print(pc.if_else(cond, left, right))
   > python test.py
   [
     null,
     null
   ]
   ```




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

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



[GitHub] [arrow] lidavidm commented on pull request #10410: ARROW-10640: [C++] A, "if_else" ("where") kernel to combine two arrays based on a mask

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


   I agree, and also Niranda already filed ARROW-12955 to take care of that followup.


-- 
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



[GitHub] [arrow] bkietz commented on a change in pull request #10410: ARROW-10640: [C++] A "where" kernel to combine two arrays based on a mask

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_if_else_test.cc
##########
@@ -0,0 +1,474 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <arrow/array.h>
+#include <arrow/compute/api_scalar.h>
+#include <arrow/compute/kernels/test_util.h>
+#include <arrow/testing/gtest_util.h>
+#include <gtest/gtest.h>
+
+namespace arrow {
+namespace compute {
+
+void CheckIfElseOutput(const Datum& cond, const Datum& left, const Datum& right,
+                       const Datum& expected) {
+  ASSERT_OK_AND_ASSIGN(Datum datum_out, IfElse(cond, left, right));
+  if (datum_out.is_array()) {
+    std::shared_ptr<Array> result = datum_out.make_array();
+    ASSERT_OK(result->ValidateFull());
+    std::shared_ptr<Array> expected_ = expected.make_array();
+    AssertArraysEqual(*expected_, *result, /*verbose=*/true);
+  } else {  // expecting scalar
+    const std::shared_ptr<Scalar>& result = datum_out.scalar();
+    const std::shared_ptr<Scalar>& expected_ = expected.scalar();
+    AssertScalarsEqual(*expected_, *result, /*verbose=*/true);
+  }
+}
+
+void CheckIfElseOutputAAA(const std::shared_ptr<DataType>& type, const std::string& cond,
+                          const std::string& left, const std::string& right,
+                          const std::string& expected) {
+  const std::shared_ptr<Array>& cond_ = ArrayFromJSON(boolean(), cond);
+  const std::shared_ptr<Array>& left_ = ArrayFromJSON(type, left);
+  const std::shared_ptr<Array>& right_ = ArrayFromJSON(type, right);
+  const std::shared_ptr<Array>& expected_ = ArrayFromJSON(type, expected);
+  CheckIfElseOutput(cond_, left_, right_, expected_);
+}
+
+void CheckIfElseOutputAAS(const std::shared_ptr<DataType>& type, const std::string& cond,
+                          const std::string& left, const std::shared_ptr<Scalar>& right,
+                          const std::string& expected) {
+  const std::shared_ptr<Array>& cond_ = ArrayFromJSON(boolean(), cond);
+  const std::shared_ptr<Array>& left_ = ArrayFromJSON(type, left);
+  const std::shared_ptr<Array>& expected_ = ArrayFromJSON(type, expected);
+  CheckIfElseOutput(cond_, left_, right, expected_);
+}
+
+void CheckIfElseOutputASA(const std::shared_ptr<DataType>& type, const std::string& cond,
+                          const std::shared_ptr<Scalar>& left, const std::string& right,
+                          const std::string& expected) {
+  const std::shared_ptr<Array>& cond_ = ArrayFromJSON(boolean(), cond);
+  const std::shared_ptr<Array>& right_ = ArrayFromJSON(type, right);
+  const std::shared_ptr<Array>& expected_ = ArrayFromJSON(type, expected);
+  CheckIfElseOutput(cond_, left, right_, expected_);
+}
+
+void CheckIfElseOutputASS(const std::shared_ptr<DataType>& type, const std::string& cond,
+                          const std::shared_ptr<Scalar>& left,
+                          const std::shared_ptr<Scalar>& right,
+                          const std::string& expected) {
+  const std::shared_ptr<Array>& cond_ = ArrayFromJSON(boolean(), cond);
+  const std::shared_ptr<Array>& expected_ = ArrayFromJSON(type, expected);
+  CheckIfElseOutput(cond_, left, right, expected_);
+}
+
+void CheckIfElseOutputSAA(const std::shared_ptr<DataType>& type,
+                          const std::shared_ptr<Scalar>& cond, const std::string& left,
+                          const std::string& right, const std::string& expected) {
+  const std::shared_ptr<Array>& left_ = ArrayFromJSON(type, left);
+  const std::shared_ptr<Array>& right_ = ArrayFromJSON(type, right);
+  const std::shared_ptr<Array>& expected_ = ArrayFromJSON(type, expected);
+  CheckIfElseOutput(cond, left_, right_, expected_);
+}
+
+void CheckIfElseOutputSAS(const std::shared_ptr<DataType>& type,
+                          const std::shared_ptr<Scalar>& cond, const std::string& left,
+                          const std::shared_ptr<Scalar>& right,
+                          const std::string& expected) {
+  const std::shared_ptr<Array>& left_ = ArrayFromJSON(type, left);
+  const std::shared_ptr<Array>& expected_ = ArrayFromJSON(type, expected);
+  CheckIfElseOutput(cond, left_, right, expected_);
+}
+
+void CheckIfElseOutputSSA(const std::shared_ptr<DataType>& type,
+                          const std::shared_ptr<Scalar>& cond,
+                          const std::shared_ptr<Scalar>& left, const std::string& right,
+                          const std::string& expected) {
+  const std::shared_ptr<Array>& right_ = ArrayFromJSON(type, right);
+  const std::shared_ptr<Array>& expected_ = ArrayFromJSON(type, expected);
+  CheckIfElseOutput(cond, left, right_, expected_);
+}
+
+class TestIfElseKernel : public ::testing::Test {};
+
+template <typename Type>
+class TestIfElsePrimitive : public ::testing::Test {};
+
+using PrimitiveTypes = ::testing::Types<Int8Type, UInt8Type, Int16Type, UInt16Type,
+                                        Int32Type, UInt32Type, Int64Type, UInt64Type,
+                                        FloatType, DoubleType, Date32Type, Date64Type>;
+
+TYPED_TEST_SUITE(TestIfElsePrimitive, PrimitiveTypes);
+
+TYPED_TEST(TestIfElsePrimitive, IfElseFixedSizeRand) {
+  using ArrayType = typename TypeTraits<TypeParam>::ArrayType;
+  auto type = TypeTraits<TypeParam>::type_singleton();
+
+  random::RandomArrayGenerator rand(/*seed=*/0);
+  int64_t len = 1000;
+  auto cond = std::static_pointer_cast<BooleanArray>(
+      rand.ArrayOf(boolean(), len, /*null_probability=*/0.01));
+  auto left = std::static_pointer_cast<ArrayType>(
+      rand.ArrayOf(type, len, /*null_probability=*/0.01));
+  auto right = std::static_pointer_cast<ArrayType>(
+      rand.ArrayOf(type, len, /*null_probability=*/0.01));
+
+  typename TypeTraits<TypeParam>::BuilderType builder;
+
+  for (int64_t i = 0; i < len; ++i) {
+    if (!cond->IsValid(i) || (cond->Value(i) && !left->IsValid(i)) ||
+        (!cond->Value(i) && !right->IsValid(i))) {
+      ASSERT_OK(builder.AppendNull());
+      continue;
+    }
+
+    if (cond->Value(i)) {
+      ASSERT_OK(builder.Append(left->Value(i)));
+    } else {
+      ASSERT_OK(builder.Append(right->Value(i)));
+    }
+  }
+  ASSERT_OK_AND_ASSIGN(auto expected_data, builder.Finish());
+
+  CheckIfElseOutput(cond, left, right, expected_data);
+}
+
+#define IF_ELSE_TEST_GEN(type, l0, l1, l2, l3, r0, r1, r2, r3, valid, valid1)            \

Review comment:
       I'd break this into two stages: first use CheckScalar should suffice to verify the (array array array) and (scalar scalar scalar) cases. The remaining cases can compare the results with manually broadcast 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.

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



[GitHub] [arrow] lidavidm commented on a change in pull request #10410: ARROW-10640: [C++] A "where" kernel to combine two arrays based on a mask

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_if_else_test.cc
##########
@@ -0,0 +1,291 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <arrow/array.h>
+#include <arrow/compute/api_scalar.h>
+#include <arrow/compute/kernels/test_util.h>
+#include <arrow/testing/gtest_util.h>
+#include <gtest/gtest.h>
+
+namespace arrow {
+namespace compute {
+
+void CheckIfElseOutputArray(const Datum& cond, const Datum& left, const Datum& right,
+                            const Datum& expected, bool all_valid = true) {
+  ASSERT_OK_AND_ASSIGN(Datum datum_out, IfElse(cond, left, right));
+  std::shared_ptr<Array> result = datum_out.make_array();
+  ASSERT_OK(result->ValidateFull());
+  AssertArraysEqual(*expected.make_array(), *result, /*verbose=*/true);
+  if (all_valid) {
+    // Check null count of ArrayData is set, not the computed Array.null_count
+    ASSERT_EQ(result->data()->null_count, 0);
+  }
+}
+
+void CheckIfElseOutputAAA(const std::shared_ptr<DataType>& type, const std::string& cond,
+                          const std::string& left, const std::string& right,
+                          const std::string& expected, bool all_valid = true) {
+  const std::shared_ptr<Array>& cond_ = ArrayFromJSON(boolean(), cond);
+  const std::shared_ptr<Array>& left_ = ArrayFromJSON(type, left);
+  const std::shared_ptr<Array>& right_ = ArrayFromJSON(type, right);
+  const std::shared_ptr<Array>& expected_ = ArrayFromJSON(type, expected);
+  CheckIfElseOutputArray(cond_, left_, right_, expected_, all_valid);
+}
+
+void CheckIfElseOutputAAS(const std::shared_ptr<DataType>& type, const std::string& cond,
+                          const std::string& left, const std::shared_ptr<Scalar>& right,
+                          const std::string& expected, bool all_valid = true) {
+  const std::shared_ptr<Array>& cond_ = ArrayFromJSON(boolean(), cond);
+  const std::shared_ptr<Array>& left_ = ArrayFromJSON(type, left);
+  const std::shared_ptr<Array>& expected_ = ArrayFromJSON(type, expected);
+  CheckIfElseOutputArray(cond_, left_, right, expected_, all_valid);
+}
+
+void CheckIfElseOutputASA(const std::shared_ptr<DataType>& type, const std::string& cond,
+                          const std::shared_ptr<Scalar>& left, const std::string& right,
+                          const std::string& expected, bool all_valid = true) {
+  const std::shared_ptr<Array>& cond_ = ArrayFromJSON(boolean(), cond);
+  const std::shared_ptr<Array>& right_ = ArrayFromJSON(type, right);
+  const std::shared_ptr<Array>& expected_ = ArrayFromJSON(type, expected);
+  CheckIfElseOutputArray(cond_, left, right_, expected_, all_valid);
+}
+
+void CheckIfElseOutputASS(const std::shared_ptr<DataType>& type, const std::string& cond,
+                          const std::shared_ptr<Scalar>& left,
+                          const std::shared_ptr<Scalar>& right,
+                          const std::string& expected, bool all_valid = true) {
+  const std::shared_ptr<Array>& cond_ = ArrayFromJSON(boolean(), cond);
+  const std::shared_ptr<Array>& expected_ = ArrayFromJSON(type, expected);
+  CheckIfElseOutputArray(cond_, left, right, expected_, all_valid);
+}
+
+class TestIfElseKernel : public ::testing::Test {};
+
+template <typename Type>
+class TestIfElsePrimitive : public ::testing::Test {};
+
+using PrimitiveTypes = ::testing::Types<Int8Type, UInt8Type, Int16Type, UInt16Type,
+                                        Int32Type, UInt32Type, Int64Type, UInt64Type,
+                                        FloatType, DoubleType, Date32Type, Date64Type>;
+
+TYPED_TEST_SUITE(TestIfElsePrimitive, PrimitiveTypes);
+
+TYPED_TEST(TestIfElsePrimitive, IfElseFixedSize) {
+  auto type = TypeTraits<TypeParam>::type_singleton();
+
+  // No Nulls
+  CheckIfElseOutputAAA(type, "[]", "[]", "[]", "[]");
+
+  // -------- All arrays ---------
+  // RLC = 111
+  CheckIfElseOutputAAA(type, "[true, true, true, false]", "[1, 2, 3, 4]", "[5, 6, 7, 8]",
+                       "[1, 2, 3, 8]");
+  // RLC = 110
+  CheckIfElseOutputAAA(type, "[true, true, null, false]", "[1, 2, 3, 4]", "[5, 6, 7, 8]",
+                       "[1, 2, null, 8]", false);
+  // RLC = 101
+  CheckIfElseOutputAAA(type, "[true, true, true, false]", "[1, null, 3, 4]",
+                       "[5, 6, 7, 8]", "[1, null, 3, 8]", false);
+  // RLC = 100
+  CheckIfElseOutputAAA(type, "[true, true, null, false]", "[1, null, 3, 4]",
+                       "[5, 6, 7, 8]", "[1, null, null, 8]", false);
+  // RLC = 011
+  CheckIfElseOutputAAA(type, "[true, true, true, false]", "[1, 2, 3, 4]",
+                       "[5, 6, 7, null]", "[1, 2, 3, null]", false);
+  // RLC = 010
+  CheckIfElseOutputAAA(type, "[null, true, true, false]", "[1, 2, 3, 4]",
+                       "[5, 6, 7, null]", "[null, 2, 3, null]", false);
+  // RLC = 001
+  CheckIfElseOutputAAA(type, "[true, true, true, false]", "[1, 2, null, null]",
+                       "[null, 6, 7, null]", "[1, 2, null, null]", false);
+  // RLC = 000
+  CheckIfElseOutputAAA(type, "[null, true, true, false]", "[1, 2, null, null]",
+                       "[null, 6, 7, null]", "[null, 2, null, null]", false);
+
+  using ArrayType = typename TypeTraits<TypeParam>::ArrayType;
+  random::RandomArrayGenerator rand(/*seed=*/0);
+  int64_t len = 1000;
+  auto cond = std::static_pointer_cast<BooleanArray>(
+      rand.ArrayOf(boolean(), len, /*null_probability=*/0.01));
+  auto left = std::static_pointer_cast<ArrayType>(
+      rand.ArrayOf(type, len, /*null_probability=*/0.01));
+  auto right = std::static_pointer_cast<ArrayType>(
+      rand.ArrayOf(type, len, /*null_probability=*/0.01));
+
+  typename TypeTraits<TypeParam>::BuilderType builder;
+
+  for (int64_t i = 0; i < len; ++i) {
+    if (!cond->IsValid(i) || (cond->Value(i) && !left->IsValid(i)) ||
+        (!cond->Value(i) && !right->IsValid(i))) {
+      ASSERT_OK(builder.AppendNull());
+      continue;
+    }
+
+    if (cond->Value(i)) {
+      ASSERT_OK(builder.Append(left->Value(i)));
+    } else {
+      ASSERT_OK(builder.Append(right->Value(i)));
+    }
+  }
+  ASSERT_OK_AND_ASSIGN(auto expected_data, builder.Finish());
+
+  CheckIfElseOutputArray(cond, left, right, expected_data, false);
+
+  // -------- Cond - Array, Left- Array, Right - Scalar ---------
+
+  ASSERT_OK_AND_ASSIGN(std::shared_ptr<Scalar> valid_scalar, MakeScalar(type, 100));
+  std::shared_ptr<Scalar> null_scalar = MakeNullScalar(type);
+
+  // empty
+  CheckIfElseOutputAAS(type, "[]", "[]", valid_scalar, "[]");
+
+  // RLC = 111
+  CheckIfElseOutputAAS(type, "[true, true, true, false]", "[1, 2, 3, 4]", valid_scalar,
+                       "[1, 2, 3, 100]");
+  // RLC = 110
+  CheckIfElseOutputAAS(type, "[true, true, null, false]", "[1, 2, 3, 4]", valid_scalar,
+                       "[1, 2, null, 100]", false);
+  // RLC = 101
+  CheckIfElseOutputAAS(type, "[true, true, true, false]", "[1, null, 3, 4]", valid_scalar,
+                       "[1, null, 3, 100]", false);
+  // RLC = 100
+  CheckIfElseOutputAAS(type, "[true, true, null, false]", "[1, null, 3, 4]", valid_scalar,
+                       "[1, null, null, 100]", false);

Review comment:
       Ah, and these probably can't be combined as you want to test null handling for different cases as well. Ok, got 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.

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



[GitHub] [arrow] lidavidm commented on a change in pull request #10410: ARROW-10640: [C++] A "where" kernel to combine two arrays based on a mask

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_if_else.cc
##########
@@ -0,0 +1,838 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <arrow/compute/api.h>
+#include <arrow/util/bit_block_counter.h>
+#include <arrow/util/bitmap.h>
+#include <arrow/util/bitmap_ops.h>
+
+#include "codegen_internal.h"
+
+namespace arrow {
+using internal::BitBlockCount;
+using internal::BitBlockCounter;
+using internal::Bitmap;
+
+namespace compute {
+
+namespace {
+
+enum { COND_ALL_VALID = 1, LEFT_ALL_VALID = 2, RIGHT_ALL_VALID = 4 };
+
+// if the condition is null then output is null otherwise we take validity from the
+// selected argument
+// ie. cond.valid & (cond.data & left.valid | ~cond.data & right.valid)
+Status PromoteNullsVisitor(KernelContext* ctx, const ArrayData& cond, const Scalar& left,
+                           const Scalar& right, ArrayData* output) {
+  uint8_t flag = right.is_valid * 4 + left.is_valid * 2 + !cond.MayHaveNulls();
+
+  if (flag < 6 && flag != 3) {
+    // there will be a validity buffer in the output
+    ARROW_ASSIGN_OR_RAISE(output->buffers[0], ctx->AllocateBitmap(cond.length));
+  }
+
+  // if the condition is null then output is null otherwise we take validity from the
+  // selected argument
+  // ie. cond.valid & (cond.data & left.valid | ~cond.data & right.valid)
+  switch (flag) {
+    case COND_ALL_VALID | LEFT_ALL_VALID | RIGHT_ALL_VALID:  // = 7
+      break;
+    case LEFT_ALL_VALID | RIGHT_ALL_VALID:  // = 6
+      // out_valid = c_valid
+      output->buffers[0] = SliceBuffer(cond.buffers[0], cond.offset, cond.length);
+      break;
+    case COND_ALL_VALID | RIGHT_ALL_VALID:  // = 5
+      // out_valid = ~cond.data
+      arrow::internal::InvertBitmap(cond.buffers[1]->data(), cond.offset, cond.length,
+                                    output->buffers[0]->mutable_data(), 0);
+      break;
+    case RIGHT_ALL_VALID:  // = 4
+      // out_valid = c_valid & ~cond.data
+      arrow::internal::BitmapAndNot(cond.buffers[0]->data(), cond.offset,
+                                    cond.buffers[1]->data(), cond.offset, cond.length, 0,
+                                    output->buffers[0]->mutable_data());
+      break;
+    case COND_ALL_VALID | LEFT_ALL_VALID:  // = 3
+      // out_valid = cond.data
+      output->buffers[0] = SliceBuffer(cond.buffers[1], cond.offset, cond.length);
+      break;
+    case LEFT_ALL_VALID:  // = 2
+      // out_valid = cond.valid & cond.data
+      arrow::internal::BitmapAnd(cond.buffers[0]->data(), cond.offset,
+                                 cond.buffers[1]->data(), cond.offset, cond.length, 0,
+                                 output->buffers[0]->mutable_data());
+      break;
+    case COND_ALL_VALID:  // = 1
+      // out_valid = 0 --> nothing to do; but requires out_valid to be a all-zero buffer

Review comment:
       Hmm, we should probably update comment then. CC @bkietz given KernelContext::AllocateBitmap has to zero the allocation to satisfy Valgrind et al, should we just go ahead and guarantee that it returns a pre-cleared bitmap?




-- 
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



[GitHub] [arrow] lidavidm commented on a change in pull request #10410: ARROW-10640: [C++] A "where" kernel to combine two arrays based on a mask

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_if_else.cc
##########
@@ -0,0 +1,572 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <arrow/compute/api.h>
+#include <arrow/compute/kernels/codegen_internal.h>
+#include <arrow/compute/util_internal.h>
+#include <arrow/util/bit_block_counter.h>
+#include <arrow/util/bitmap.h>
+#include <arrow/util/bitmap_ops.h>
+
+namespace arrow {
+using internal::BitBlockCount;
+using internal::BitBlockCounter;
+using internal::Bitmap;
+
+namespace compute {
+
+namespace {
+
+util::optional<uint64_t> GetConstantValidityWord(const Datum& data) {
+  if (data.is_scalar()) {
+    return data.scalar()->is_valid ? ~uint64_t(0) : uint64_t(0);
+  }
+
+  if (data.array()->null_count == data.array()->length) return uint64_t(0);
+
+  if (!data.array()->MayHaveNulls()) return ~uint64_t(0);
+
+  // no constant validity word available
+  return {};
+}
+
+inline Bitmap GetBitmap(const Datum& datum, int i) {
+  if (datum.is_scalar()) return {};
+  const ArrayData& a = *datum.array();
+  return Bitmap{a.buffers[i], a.offset, a.length};
+}
+
+// if the condition is null then output is null otherwise we take validity from the
+// selected argument
+// ie. cond.valid & (cond.data & left.valid | ~cond.data & right.valid)
+Status PromoteNullsVisitor(KernelContext* ctx, const Datum& cond_d, const Datum& left_d,
+                           const Datum& right_d, ArrayData* output) {
+  auto cond_const = GetConstantValidityWord(cond_d);
+  auto left_const = GetConstantValidityWord(left_d);
+  auto right_const = GetConstantValidityWord(right_d);
+
+  enum { COND_CONST = 1, LEFT_CONST = 2, RIGHT_CONST = 4 };
+  auto flag = COND_CONST * cond_const.has_value() | LEFT_CONST * left_const.has_value() |
+              RIGHT_CONST * right_const.has_value();
+
+  const ArrayData& cond = *cond_d.array();
+  // cond.data will always be available
+  Bitmap cond_data{cond.buffers[1], cond.offset, cond.length};
+  Bitmap cond_valid{cond.buffers[0], cond.offset, cond.length};
+  Bitmap left_valid = GetBitmap(left_d, 0);
+  Bitmap right_valid = GetBitmap(right_d, 0);
+  // sometimes Bitmaps will be ignored, in which case we replace access to them with
+  // duplicated (probably elided) access to cond_data
+  const Bitmap& _ = cond_data;
+
+  // lambda function that will be used inside the visitor
+  uint64_t* out_validity = nullptr;
+  int64_t i = 0;
+  auto apply = [&](uint64_t c_valid, uint64_t c_data, uint64_t l_valid,
+                   uint64_t r_valid) {
+    out_validity[i] = c_valid & ((c_data & l_valid) | (~c_data & r_valid));
+    i++;
+  };
+
+  // if the condition is null then output is null otherwise we take validity from the
+  // selected argument
+  // ie. cond.valid & (cond.data & left.valid | ~cond.data & right.valid)
+
+  // In the following cases, we dont need to allocate out_valid bitmap
+  switch (flag) {
+    case COND_CONST | LEFT_CONST | RIGHT_CONST:
+      // if cond & left & right all ones, then output is all valid --> out_valid = nullptr

Review comment:
       What I mean is that just because all of the validity bitmaps are constant, doesn't mean that all of the validity bitmaps are all 1s. GetConstantValidityWord returns a non-null optional for both valid and invalid scalars, for instance.




-- 
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



[GitHub] [arrow] nirandaperera commented on a change in pull request #10410: ARROW-10640: [C++] A "where" kernel to combine two arrays based on a mask

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_if_else.cc
##########
@@ -0,0 +1,577 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <arrow/compute/api.h>
+#include <arrow/compute/kernels/codegen_internal.h>
+#include <arrow/compute/util_internal.h>
+#include <arrow/util/bit_block_counter.h>
+#include <arrow/util/bitmap.h>
+#include <arrow/util/bitmap_ops.h>
+
+namespace arrow {
+using internal::BitBlockCount;
+using internal::BitBlockCounter;
+using internal::Bitmap;
+
+namespace compute {
+
+namespace {
+
+util::optional<uint64_t> GetConstantValidityWord(const Datum& data) {
+  if (data.is_scalar()) {
+    return data.scalar()->is_valid ? ~uint64_t(0) : uint64_t(0);
+  }
+
+  if (data.array()->null_count == data.array()->length) return uint64_t(0);
+
+  if (!data.array()->MayHaveNulls()) return ~uint64_t(0);
+
+  // no constant validity word available
+  return {};
+}
+
+inline Bitmap GetBitmap(const Datum& datum, int i) {
+  if (datum.is_scalar()) return {};
+  const ArrayData& a = *datum.array();
+  return Bitmap{a.buffers[i], a.offset, a.length};
+}
+
+// if the condition is null then output is null otherwise we take validity from the
+// selected argument
+// ie. cond.valid & (cond.data & left.valid | ~cond.data & right.valid)
+Status PromoteNullsVisitor(KernelContext* ctx, const Datum& cond_d, const Datum& left_d,
+                           const Datum& right_d, ArrayData* output) {
+  auto cond_const = GetConstantValidityWord(cond_d);
+  auto left_const = GetConstantValidityWord(left_d);
+  auto right_const = GetConstantValidityWord(right_d);
+
+  enum { COND_CONST = 1, LEFT_CONST = 2, RIGHT_CONST = 4 };
+  auto flag = COND_CONST * cond_const.has_value() | LEFT_CONST * left_const.has_value() |
+              RIGHT_CONST * right_const.has_value();
+
+  const ArrayData& cond = *cond_d.array();
+  // cond.data will always be available
+  Bitmap cond_data{cond.buffers[1], cond.offset, cond.length};
+  Bitmap cond_valid{cond.buffers[0], cond.offset, cond.length};
+  Bitmap left_valid = GetBitmap(left_d, 0);
+  Bitmap right_valid = GetBitmap(right_d, 0);
+  // sometimes Bitmaps will be ignored, in which case we replace access to them with
+  // duplicated (probably elided) access to cond_data
+  const Bitmap& _ = cond_data;
+
+  // lambda function that will be used inside the visitor
+  uint64_t* out_validity = nullptr;
+  int64_t i = 0;
+  auto apply = [&](uint64_t c_valid, uint64_t c_data, uint64_t l_valid,
+                   uint64_t r_valid) {
+    out_validity[i] = c_valid & ((c_data & l_valid) | (~c_data & r_valid));
+    i++;
+  };
+
+  // cond.valid & (cond.data & left.valid | ~cond.data & right.valid)
+  // In the following cases, we dont need to allocate out_valid bitmap
+  switch (flag) {
+    case COND_CONST | LEFT_CONST | RIGHT_CONST:
+      // if cond & left & right all ones, then output is all valid --> out_valid = nullptr
+      if ((*cond_const & *left_const & *right_const) == UINT64_MAX) {
+        return Status::OK();
+      }
+    case LEFT_CONST | RIGHT_CONST:
+      // if both left and right are valid, no need to calculate out_valid bitmap. Pass
+      // cond validity buffer
+      if ((*left_const & *right_const) == UINT64_MAX) {
+        // if there's an offset, copy bitmap (cannot slice a bitmap)
+        if (cond.offset) {
+          ARROW_ASSIGN_OR_RAISE(
+              output->buffers[0],
+              arrow::internal::CopyBitmap(ctx->memory_pool(), cond.buffers[0]->data(),
+                                          cond.offset, cond.length));
+        } else {  // just copy assign cond validity buffer
+          output->buffers[0] = cond.buffers[0];
+        }
+        return Status::OK();
+      }
+  }
+
+  // following cases requires a separate out_valid buffer
+  ARROW_ASSIGN_OR_RAISE(output->buffers[0], ctx->AllocateBitmap(cond.length));
+  out_validity = output->GetMutableValues<uint64_t>(0);
+
+  enum { C_VALID, C_DATA, L_VALID, R_VALID };
+
+  switch (flag) {
+    case COND_CONST | LEFT_CONST | RIGHT_CONST:
+      Bitmap::VisitWords({_, cond_data, _, _}, [&](std::array<uint64_t, 4> words) {
+        apply(*cond_const, words[C_DATA], *left_const, *right_const);
+      });
+      break;
+    case LEFT_CONST | RIGHT_CONST:
+      Bitmap::VisitWords(
+          {cond_valid, cond_data, _, _}, [&](std::array<uint64_t, 4> words) {
+            apply(words[C_VALID], words[C_DATA], *left_const, *right_const);
+          });
+      break;
+    case COND_CONST | RIGHT_CONST:
+      // bitmaps[C_VALID], bitmaps[R_VALID] might be null; override to make it safe for
+      // Visit()
+      Bitmap::VisitWords(
+          {_, cond_data, left_valid, _}, [&](std::array<uint64_t, 4> words) {
+            apply(*cond_const, words[C_DATA], words[L_VALID], *right_const);
+          });
+      break;
+    case RIGHT_CONST:
+      // bitmaps[R_VALID] might be null; override to make it safe for Visit()
+      Bitmap::VisitWords(
+          {cond_valid, cond_data, left_valid, _}, [&](std::array<uint64_t, 4> words) {
+            apply(words[C_VALID], words[C_DATA], words[L_VALID], *right_const);
+          });
+      break;
+    case COND_CONST | LEFT_CONST:
+      // bitmaps[C_VALID], bitmaps[L_VALID] might be null; override to make it safe for
+      // Visit()
+      Bitmap::VisitWords({_, cond_data, _, right_valid},
+                         [&](std::array<uint64_t, 4> words) {
+                           apply(*cond_const, words[C_DATA], *left_const, words[R_VALID]);
+                         });
+      break;
+    case LEFT_CONST:
+      // bitmaps[L_VALID] might be null; override to make it safe for Visit()
+      Bitmap::VisitWords(
+          {cond_valid, cond_data, _, right_valid}, [&](std::array<uint64_t, 4> words) {
+            apply(words[C_VALID], words[C_DATA], *left_const, words[R_VALID]);
+          });
+      break;
+    case COND_CONST:
+      // bitmaps[C_VALID] might be null; override to make it safe for Visit()
+      Bitmap::VisitWords(
+          {_, cond_data, left_valid, right_valid}, [&](std::array<uint64_t, 4> words) {
+            apply(*cond_const, words[C_DATA], words[L_VALID], words[R_VALID]);
+          });
+      break;
+    case 0:
+      Bitmap::VisitWords({cond_valid, cond_data, left_valid, right_valid},
+                         [&](std::array<uint64_t, 4> words) {
+                           apply(words[C_VALID], words[C_DATA], words[L_VALID],
+                                 words[R_VALID]);
+                         });
+      break;
+  }
+  return Status::OK();
+}
+
+template <typename Type, typename Enable = void>
+struct IfElseFunctor {};
+
+// only number types needs to be handled for Fixed sized primitive data types because,
+// internal::GenerateTypeAgnosticPrimitive forwards types to the corresponding unsigned
+// int type
+template <typename Type>
+struct IfElseFunctor<Type, enable_if_number<Type>> {
+  using T = typename TypeTraits<Type>::CType;
+  // A - Array
+  // S - Scalar
+
+  //  AAA
+  static Status Call(KernelContext* ctx, const ArrayData& cond, const ArrayData& left,
+                     const ArrayData& right, ArrayData* out) {
+    ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Buffer> out_buf,
+                          ctx->Allocate(cond.length * sizeof(T)));
+    T* out_values = reinterpret_cast<T*>(out_buf->mutable_data());
+
+    // copy right data to out_buff
+    const T* right_data = right.GetValues<T>(1);
+    std::memcpy(out_values, right_data, right.length * sizeof(T));
+
+    const auto* cond_data = cond.buffers[1]->data();  // this is a BoolArray
+    BitBlockCounter bit_counter(cond_data, cond.offset, cond.length);
+
+    // selectively copy values from left data
+    const T* left_data = left.GetValues<T>(1);
+    int64_t offset = cond.offset;
+
+    // todo this can be improved by intrinsics. ex: _mm*_mask_store_e* (vmovdqa*)
+    while (offset < cond.offset + cond.length) {
+      const BitBlockCount& block = bit_counter.NextWord();
+      if (block.AllSet()) {  // all from left
+        std::memcpy(out_values, left_data, block.length * sizeof(T));
+      } else if (block.popcount) {  // selectively copy from left
+        for (int64_t i = 0; i < block.length; ++i) {
+          if (BitUtil::GetBit(cond_data, offset + i)) {
+            out_values[i] = left_data[i];
+          }
+        }
+      }
+
+      offset += block.length;
+      out_values += block.length;
+      left_data += block.length;
+    }
+
+    out->buffers[1] = std::move(out_buf);
+    return Status::OK();
+  }
+
+  // ASA
+  static Status Call(KernelContext* ctx, const ArrayData& cond, const Scalar& left,
+                     const ArrayData& right, ArrayData* out) {
+    ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Buffer> out_buf,
+                          ctx->Allocate(cond.length * sizeof(T)));
+    T* out_values = reinterpret_cast<T*>(out_buf->mutable_data());
+
+    // copy right data to out_buff
+    const T* right_data = right.GetValues<T>(1);
+    std::memcpy(out_values, right_data, right.length * sizeof(T));
+
+    const auto* cond_data = cond.buffers[1]->data();  // this is a BoolArray
+    BitBlockCounter bit_counter(cond_data, cond.offset, cond.length);
+
+    // selectively copy values from left data
+    T left_data = internal::UnboxScalar<Type>::Unbox(left);
+    int64_t offset = cond.offset;
+
+    // todo this can be improved by intrinsics. ex: _mm*_mask_store_e* (vmovdqa*)
+    while (offset < cond.offset + cond.length) {
+      const BitBlockCount& block = bit_counter.NextWord();
+      if (block.AllSet()) {  // all from left
+        std::fill(out_values, out_values + block.length, left_data);
+      } else if (block.popcount) {  // selectively copy from left
+        for (int64_t i = 0; i < block.length; ++i) {
+          if (BitUtil::GetBit(cond_data, offset + i)) {
+            out_values[i] = left_data;
+          }
+        }
+      }
+
+      offset += block.length;
+      out_values += block.length;
+    }
+
+    out->buffers[1] = std::move(out_buf);
+    return Status::OK();
+  }
+
+  // AAS
+  static Status Call(KernelContext* ctx, const ArrayData& cond, const ArrayData& left,
+                     const Scalar& right, ArrayData* out) {
+    ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Buffer> out_buf,
+                          ctx->Allocate(cond.length * sizeof(T)));
+    T* out_values = reinterpret_cast<T*>(out_buf->mutable_data());
+
+    // copy left data to out_buff
+    const T* left_data = left.GetValues<T>(1);
+    std::memcpy(out_values, left_data, left.length * sizeof(T));
+
+    const auto* cond_data = cond.buffers[1]->data();  // this is a BoolArray
+    BitBlockCounter bit_counter(cond_data, cond.offset, cond.length);
+
+    // selectively copy values from left data
+    T right_data = internal::UnboxScalar<Type>::Unbox(right);
+    int64_t offset = cond.offset;
+
+    // todo this can be improved by intrinsics. ex: _mm*_mask_store_e* (vmovdqa*)
+    // left data is already in the output buffer. Therefore, mask needs to be inverted
+    while (offset < cond.offset + cond.length) {
+      const BitBlockCount& block = bit_counter.NextWord();
+      if (block.NoneSet()) {  // all from right
+        std::fill(out_values, out_values + block.length, right_data);
+      } else if (block.popcount) {  // selectively copy from right
+        for (int64_t i = 0; i < block.length; ++i) {
+          if (!BitUtil::GetBit(cond_data, offset + i)) {
+            out_values[i] = right_data;
+          }
+        }
+      }
+
+      offset += block.length;
+      out_values += block.length;
+    }
+
+    out->buffers[1] = std::move(out_buf);
+    return Status::OK();
+  }
+
+  // ASS
+  static Status Call(KernelContext* ctx, const ArrayData& cond, const Scalar& left,
+                     const Scalar& right, ArrayData* out) {
+    ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Buffer> out_buf,
+                          ctx->Allocate(cond.length * sizeof(T)));
+    T* out_values = reinterpret_cast<T*>(out_buf->mutable_data());
+
+    // copy right data to out_buff
+    T right_data = internal::UnboxScalar<Type>::Unbox(right);
+    std::fill(out_values, out_values + cond.length, right_data);
+
+    const auto* cond_data = cond.buffers[1]->data();  // this is a BoolArray
+    BitBlockCounter bit_counter(cond_data, cond.offset, cond.length);
+
+    // selectively copy values from left data
+    T left_data = internal::UnboxScalar<Type>::Unbox(left);
+    int64_t offset = cond.offset;
+
+    // todo this can be improved by intrinsics. ex: _mm*_mask_store_e* (vmovdqa*)
+    while (offset < cond.offset + cond.length) {
+      const BitBlockCount& block = bit_counter.NextWord();
+      if (block.AllSet()) {  // all from left
+        std::fill(out_values, out_values + block.length, left_data);
+      } else if (block.popcount) {  // selectively copy from left
+        for (int64_t i = 0; i < block.length; ++i) {
+          if (BitUtil::GetBit(cond_data, offset + i)) {
+            out_values[i] = left_data;
+          }
+        }
+      }
+
+      offset += block.length;
+      out_values += block.length;
+    }
+
+    out->buffers[1] = std::move(out_buf);
+    return Status::OK();
+  }
+};
+
+template <typename Type>
+struct IfElseFunctor<Type, enable_if_boolean<Type>> {
+  // AAA
+  static Status Call(KernelContext* ctx, const ArrayData& cond, const ArrayData& left,
+                     const ArrayData& right, ArrayData* out) {
+    // out_buff = right & ~cond
+    ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Buffer> out_buf,
+                          arrow::internal::BitmapAndNot(
+                              ctx->memory_pool(), right.buffers[1]->data(), right.offset,
+                              cond.buffers[1]->data(), cond.offset, cond.length, 0));
+
+    // out_buff = left & cond
+    ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Buffer> temp_buf,
+                          arrow::internal::BitmapAnd(
+                              ctx->memory_pool(), left.buffers[1]->data(), left.offset,
+                              cond.buffers[1]->data(), cond.offset, cond.length, 0));
+
+    arrow::internal::BitmapOr(out_buf->data(), 0, temp_buf->data(), 0, cond.length, 0,
+                              out_buf->mutable_data());
+    out->buffers[1] = std::move(out_buf);
+    return Status::OK();
+  }
+
+  // ASA
+  static Status Call(KernelContext* ctx, const ArrayData& cond, const Scalar& left,
+                     const ArrayData& right, ArrayData* out) {
+    // out_buff = right & ~cond
+    ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Buffer> out_buf,
+                          arrow::internal::BitmapAndNot(
+                              ctx->memory_pool(), right.buffers[1]->data(), right.offset,
+                              cond.buffers[1]->data(), cond.offset, cond.length, 0));
+
+    // out_buff = left & cond
+    bool left_data = internal::UnboxScalar<BooleanType>::Unbox(left);
+    if (left_data) {
+      arrow::internal::BitmapOr(out_buf->data(), 0, cond.buffers[1]->data(), cond.offset,
+                                cond.length, 0, out_buf->mutable_data());
+    }
+
+    out->buffers[1] = std::move(out_buf);
+    return Status::OK();
+  }
+
+  // AAS
+  static Status Call(KernelContext* ctx, const ArrayData& cond, const ArrayData& left,
+                     const Scalar& right, ArrayData* out) {
+    // out_buff = left & cond
+    ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Buffer> out_buf,
+                          arrow::internal::BitmapAnd(
+                              ctx->memory_pool(), left.buffers[1]->data(), left.offset,
+                              cond.buffers[1]->data(), cond.offset, cond.length, 0));
+
+    bool right_data = internal::UnboxScalar<BooleanType>::Unbox(right);
+
+    // out_buff = left & cond | right & ~cond
+    if (right_data) {
+      arrow::internal::BitmapOrNot(out_buf->data(), 0, cond.buffers[1]->data(),
+                                   cond.offset, cond.length, 0, out_buf->mutable_data());
+    }
+
+    out->buffers[1] = std::move(out_buf);
+    return Status::OK();
+  }
+
+  // ASS
+  static Status Call(KernelContext* ctx, const ArrayData& cond, const Scalar& left,
+                     const Scalar& right, ArrayData* out) {
+    bool left_data = internal::UnboxScalar<BooleanType>::Unbox(left);
+    bool right_data = internal::UnboxScalar<BooleanType>::Unbox(right);
+
+    // out_buf = left & cond | right & ~cond
+    std::shared_ptr<Buffer> out_buf = nullptr;
+    if (left_data) {
+      if (right_data) {
+        // out_buf = ones
+        ARROW_ASSIGN_OR_RAISE(out_buf, ctx->AllocateBitmap(cond.length));
+        // filling with UINT8_MAX upto the buffer's size (in bytes)
+        arrow::compute::internal::SetMemory<UINT8_MAX>(out_buf.get());
+      } else {
+        // out_buf = cond
+        out_buf = SliceBuffer(cond.buffers[1], cond.offset, cond.length);
+      }
+    } else {
+      if (right_data) {
+        // out_buf = ~cond
+        ARROW_ASSIGN_OR_RAISE(out_buf, arrow::internal::InvertBitmap(
+                                           ctx->memory_pool(), cond.buffers[1]->data(),
+                                           cond.offset, cond.length))
+      } else {
+        // out_buf = zeros
+        ARROW_ASSIGN_OR_RAISE(out_buf, ctx->AllocateBitmap(cond.length));
+      }
+    }
+    out->buffers[1] = std::move(out_buf);
+    return Status::OK();
+  }
+};
+
+template <typename Type>
+struct IfElseFunctor<Type, enable_if_null<Type>> {
+  template <typename T>
+  static inline Status ReturnCopy(const T& in, T* out) {
+    // Nothing preallocated, so we assign in into the output
+    *out = in;
+    return Status::OK();
+  }
+
+  // AAA
+  static Status Call(KernelContext* ctx, const ArrayData& cond, const ArrayData& left,
+                     const ArrayData& right, ArrayData* out) {
+    return ReturnCopy(left, out);
+  }
+
+  // ASA
+  static Status Call(KernelContext* ctx, const ArrayData& cond, const Scalar& left,
+                     const ArrayData& right, ArrayData* out) {
+    return ReturnCopy(right, out);
+  }
+
+  // AAS
+  static Status Call(KernelContext* ctx, const ArrayData& cond, const ArrayData& left,
+                     const Scalar& right, ArrayData* out) {
+    return ReturnCopy(left, out);
+  }
+
+  // ASS
+  static Status Call(KernelContext* ctx, const ArrayData& cond, const Scalar& left,
+                     const Scalar& right, ArrayData* out) {
+    return ReturnCopy(cond, out);
+  }
+};
+
+template <typename Type>
+struct ResolveIfElseExec {
+  static Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    // cond is scalar
+    if (batch[0].is_scalar()) {
+      const auto& cond = batch[0].scalar_as<BooleanScalar>();
+      if (batch[1].is_scalar() && batch[2].is_scalar()) {
+        if (cond.is_valid) {
+          *out = cond.value ? batch[1].scalar() : batch[2].scalar();
+        } else {
+          *out = MakeNullScalar(batch[1].type());
+        }
+        return Status::OK();
+      } else {  // either left or right is an array. output is always an array
+        // output size is the size of the array arg
+        auto bcast_size = batch.length;
+        if (cond.is_valid) {
+          const auto& valid_data = cond.value ? batch[1] : batch[2];
+          if (valid_data.is_array()) {
+            *out = valid_data;
+          } else {  // valid data is a scalar that needs to be broadcasted
+            ARROW_ASSIGN_OR_RAISE(*out,
+                                  MakeArrayFromScalar(*valid_data.scalar(), bcast_size,
+                                                      ctx->memory_pool()));
+          }
+        } else {  // cond is null. create null array
+          ARROW_ASSIGN_OR_RAISE(
+              *out, MakeArrayOfNull(batch[1].type(), bcast_size, ctx->memory_pool()))
+        }
+        return Status::OK();
+      }
+    }

Review comment:
       ah okay. I think it 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.

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



[GitHub] [arrow] bkietz commented on a change in pull request #10410: ARROW-10640: [C++] A "where" kernel to combine two arrays based on a mask

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_if_else.cc
##########
@@ -0,0 +1,838 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <arrow/compute/api.h>
+#include <arrow/util/bit_block_counter.h>
+#include <arrow/util/bitmap.h>
+#include <arrow/util/bitmap_ops.h>
+
+#include "codegen_internal.h"
+
+namespace arrow {
+using internal::BitBlockCount;
+using internal::BitBlockCounter;
+using internal::Bitmap;
+
+namespace compute {
+
+namespace {
+
+enum { COND_ALL_VALID = 1, LEFT_ALL_VALID = 2, RIGHT_ALL_VALID = 4 };
+
+// if the condition is null then output is null otherwise we take validity from the
+// selected argument
+// ie. cond.valid & (cond.data & left.valid | ~cond.data & right.valid)
+Status PromoteNullsVisitor(KernelContext* ctx, const ArrayData& cond, const Scalar& left,
+                           const Scalar& right, ArrayData* output) {
+  uint8_t flag = right.is_valid * 4 + left.is_valid * 2 + !cond.MayHaveNulls();
+
+  if (flag < 6 && flag != 3) {
+    // there will be a validity buffer in the output
+    ARROW_ASSIGN_OR_RAISE(output->buffers[0], ctx->AllocateBitmap(cond.length));
+  }
+
+  // if the condition is null then output is null otherwise we take validity from the
+  // selected argument
+  // ie. cond.valid & (cond.data & left.valid | ~cond.data & right.valid)
+  switch (flag) {
+    case COND_ALL_VALID | LEFT_ALL_VALID | RIGHT_ALL_VALID:  // = 7
+      break;
+    case LEFT_ALL_VALID | RIGHT_ALL_VALID:  // = 6
+      // out_valid = c_valid
+      output->buffers[0] = SliceBuffer(cond.buffers[0], cond.offset, cond.length);
+      break;
+    case COND_ALL_VALID | RIGHT_ALL_VALID:  // = 5
+      // out_valid = ~cond.data
+      arrow::internal::InvertBitmap(cond.buffers[1]->data(), cond.offset, cond.length,
+                                    output->buffers[0]->mutable_data(), 0);
+      break;
+    case RIGHT_ALL_VALID:  // = 4
+      // out_valid = c_valid & ~cond.data
+      arrow::internal::BitmapAndNot(cond.buffers[0]->data(), cond.offset,
+                                    cond.buffers[1]->data(), cond.offset, cond.length, 0,
+                                    output->buffers[0]->mutable_data());
+      break;
+    case COND_ALL_VALID | LEFT_ALL_VALID:  // = 3
+      // out_valid = cond.data
+      output->buffers[0] = SliceBuffer(cond.buffers[1], cond.offset, cond.length);
+      break;
+    case LEFT_ALL_VALID:  // = 2
+      // out_valid = cond.valid & cond.data
+      arrow::internal::BitmapAnd(cond.buffers[0]->data(), cond.offset,
+                                 cond.buffers[1]->data(), cond.offset, cond.length, 0,
+                                 output->buffers[0]->mutable_data());
+      break;
+    case COND_ALL_VALID:  // = 1
+      // out_valid = 0 --> nothing to do; but requires out_valid to be a all-zero buffer

Review comment:
       I'd prefer to leave this for a follow up where we can try to zero only the final byte (as described by the comment) and see if Valgrind is  appeased. If not, the comment can be 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.

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



[GitHub] [arrow] nirandaperera commented on pull request #10410: ARROW-10640: [C++] A, "if_else" ("where") kernel to combine two arrays based on a mask

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


   > I did a quick comparison with `np.where` (using plain float64 arrays without nulls to have it comparable), and it seems this implementation is already doing quite good compared to numpy (using a release build):
   > 
   > ```
   > In [19]: N = 10_000_000
   >     ...: arr1 = np.random.randn(N)
   >     ...: arr2 = np.random.randn(N)
   >     ...: mask = np.random.randint(0, 2, N).astype(bool)
   >     ...: 
   >     ...: pa_arr1 = pa.array(arr1)
   >     ...: pa_arr2 = pa.array(arr2)
   >     ...: pa_mask = pa.array(mask)
   >     ...: 
   > 
   > In [20]: %timeit np.where(mask, arr1, arr2)
   > 82.3 ms ± 8.76 ms per loop (mean ± std. dev. of 7 runs, 10 loops each)
   > 
   > In [21]: %timeit pc.if_else(pa_mask, pa_arr1, pa_arr2)
   > 50.4 ms ± 6.08 ms per loop (mean ± std. dev. of 7 runs, 10 loops each)
   > ```
   
   We may be able to further improve this if we directly use vector operations inside the kernel (I haven't checked the compiled code yet, may be compiler does that already), because if-else use case directly map to `mask_move` operations in AVX512


-- 
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



[GitHub] [arrow] nirandaperera commented on a change in pull request #10410: ARROW-10640: [C++] A "where" kernel to combine two arrays based on a mask

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_if_else.cc
##########
@@ -0,0 +1,572 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <arrow/compute/api.h>
+#include <arrow/compute/kernels/codegen_internal.h>
+#include <arrow/compute/util_internal.h>
+#include <arrow/util/bit_block_counter.h>
+#include <arrow/util/bitmap.h>
+#include <arrow/util/bitmap_ops.h>
+
+namespace arrow {
+using internal::BitBlockCount;
+using internal::BitBlockCounter;
+using internal::Bitmap;
+
+namespace compute {
+
+namespace {
+
+util::optional<uint64_t> GetConstantValidityWord(const Datum& data) {
+  if (data.is_scalar()) {
+    return data.scalar()->is_valid ? ~uint64_t(0) : uint64_t(0);
+  }
+
+  if (data.array()->null_count == data.array()->length) return uint64_t(0);
+
+  if (!data.array()->MayHaveNulls()) return ~uint64_t(0);
+
+  // no constant validity word available
+  return {};
+}
+
+inline Bitmap GetBitmap(const Datum& datum, int i) {
+  if (datum.is_scalar()) return {};
+  const ArrayData& a = *datum.array();
+  return Bitmap{a.buffers[i], a.offset, a.length};
+}
+
+// if the condition is null then output is null otherwise we take validity from the
+// selected argument
+// ie. cond.valid & (cond.data & left.valid | ~cond.data & right.valid)
+Status PromoteNullsVisitor(KernelContext* ctx, const Datum& cond_d, const Datum& left_d,
+                           const Datum& right_d, ArrayData* output) {
+  auto cond_const = GetConstantValidityWord(cond_d);
+  auto left_const = GetConstantValidityWord(left_d);
+  auto right_const = GetConstantValidityWord(right_d);
+
+  enum { COND_CONST = 1, LEFT_CONST = 2, RIGHT_CONST = 4 };
+  auto flag = COND_CONST * cond_const.has_value() | LEFT_CONST * left_const.has_value() |
+              RIGHT_CONST * right_const.has_value();
+
+  const ArrayData& cond = *cond_d.array();
+  // cond.data will always be available
+  Bitmap cond_data{cond.buffers[1], cond.offset, cond.length};
+  Bitmap cond_valid{cond.buffers[0], cond.offset, cond.length};
+  Bitmap left_valid = GetBitmap(left_d, 0);
+  Bitmap right_valid = GetBitmap(right_d, 0);
+  // sometimes Bitmaps will be ignored, in which case we replace access to them with
+  // duplicated (probably elided) access to cond_data
+  const Bitmap& _ = cond_data;
+
+  // lambda function that will be used inside the visitor
+  uint64_t* out_validity = nullptr;
+  int64_t i = 0;
+  auto apply = [&](uint64_t c_valid, uint64_t c_data, uint64_t l_valid,
+                   uint64_t r_valid) {
+    out_validity[i] = c_valid & ((c_data & l_valid) | (~c_data & r_valid));
+    i++;
+  };
+
+  // if the condition is null then output is null otherwise we take validity from the
+  // selected argument
+  // ie. cond.valid & (cond.data & left.valid | ~cond.data & right.valid)
+
+  // In the following cases, we dont need to allocate out_valid bitmap
+  switch (flag) {
+    case COND_CONST | LEFT_CONST | RIGHT_CONST:
+      // if cond & left & right all ones, then output is all valid --> out_valid = nullptr
+      if ((*cond_const & *left_const & *right_const) == UINT64_MAX) {
+        return Status::OK();
+      }
+    case LEFT_CONST | RIGHT_CONST:
+      // if both left and right are valid, no need to calculate out_valid bitmap. Pass
+      // cond validity buffer
+      if ((*left_const & *right_const) == UINT64_MAX) {
+        output->buffers[0] = SliceBuffer(cond.buffers[0], cond.offset, cond.length);

Review comment:
       I don't test arrays with offsets. I think it would be better to add some test cases there
   




-- 
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



[GitHub] [arrow] nirandaperera commented on a change in pull request #10410: ARROW-10640: [C++] A "where" kernel to combine two arrays based on a mask

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_if_else.cc
##########
@@ -0,0 +1,838 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <arrow/compute/api.h>
+#include <arrow/util/bit_block_counter.h>
+#include <arrow/util/bitmap.h>
+#include <arrow/util/bitmap_ops.h>
+
+#include "codegen_internal.h"
+
+namespace arrow {
+using internal::BitBlockCount;
+using internal::BitBlockCounter;
+using internal::Bitmap;
+
+namespace compute {
+
+namespace {
+
+enum { COND_ALL_VALID = 1, LEFT_ALL_VALID = 2, RIGHT_ALL_VALID = 4 };
+
+// if the condition is null then output is null otherwise we take validity from the
+// selected argument
+// ie. cond.valid & (cond.data & left.valid | ~cond.data & right.valid)
+Status PromoteNullsVisitor(KernelContext* ctx, const ArrayData& cond, const Scalar& left,
+                           const Scalar& right, ArrayData* output) {
+  uint8_t flag = right.is_valid * 4 + left.is_valid * 2 + !cond.MayHaveNulls();
+
+  if (flag < 6 && flag != 3) {

Review comment:
       Idea is to allocate buffer only for 5 cases. but yes, I will do 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.

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



[GitHub] [arrow] jorisvandenbossche commented on pull request #10410: ARROW-10640: [C++] A, "if_else" ("where") kernel to combine two arrays based on a mask

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


   > should I add string type support to this as well, or can I leave it for later development?
   
   I think it is important to add support for strings as well, but it's probably a good idea to leave that for a follow-up PR so we can get this merged? (I can imagine the implementation to be a bit different / more complex for variable length array elements)


-- 
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



[GitHub] [arrow] nirandaperera commented on pull request #10410: ARROW-10640: [C++] A "where" kernel to combine two arrays based on a mask

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


   @bkietz I think the bitmap ops approach is simpler than the bitmap visitor approach. WDYT?
   bitmap ops - 
   https://github.com/apache/arrow/blob/8c0e8d9749f171da3826d9b194eacbd897909873/cpp/src/arrow/compute/kernels/scalar_if_else.cc#L307 onward
   
   bitmap visitor - 
   https://github.com/apache/arrow/blob/8c0e8d9749f171da3826d9b194eacbd897909873/cpp/src/arrow/compute/kernels/scalar_if_else.cc#L156 onward
   


-- 
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



[GitHub] [arrow] nirandaperera commented on a change in pull request #10410: ARROW-10640: [C++] A "where" kernel to combine two arrays based on a mask

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_if_else.cc
##########
@@ -0,0 +1,838 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <arrow/compute/api.h>
+#include <arrow/util/bit_block_counter.h>
+#include <arrow/util/bitmap.h>
+#include <arrow/util/bitmap_ops.h>
+
+#include "codegen_internal.h"
+
+namespace arrow {
+using internal::BitBlockCount;
+using internal::BitBlockCounter;
+using internal::Bitmap;
+
+namespace compute {
+
+namespace {
+
+enum { COND_ALL_VALID = 1, LEFT_ALL_VALID = 2, RIGHT_ALL_VALID = 4 };
+
+// if the condition is null then output is null otherwise we take validity from the
+// selected argument
+// ie. cond.valid & (cond.data & left.valid | ~cond.data & right.valid)
+Status PromoteNullsVisitor(KernelContext* ctx, const ArrayData& cond, const Scalar& left,
+                           const Scalar& right, ArrayData* output) {
+  uint8_t flag = right.is_valid * 4 + left.is_valid * 2 + !cond.MayHaveNulls();
+
+  if (flag < 6 && flag != 3) {
+    // there will be a validity buffer in the output
+    ARROW_ASSIGN_OR_RAISE(output->buffers[0], ctx->AllocateBitmap(cond.length));
+  }
+
+  // if the condition is null then output is null otherwise we take validity from the
+  // selected argument
+  // ie. cond.valid & (cond.data & left.valid | ~cond.data & right.valid)
+  switch (flag) {
+    case COND_ALL_VALID | LEFT_ALL_VALID | RIGHT_ALL_VALID:  // = 7
+      break;
+    case LEFT_ALL_VALID | RIGHT_ALL_VALID:  // = 6
+      // out_valid = c_valid
+      output->buffers[0] = SliceBuffer(cond.buffers[0], cond.offset, cond.length);
+      break;
+    case COND_ALL_VALID | RIGHT_ALL_VALID:  // = 5
+      // out_valid = ~cond.data
+      arrow::internal::InvertBitmap(cond.buffers[1]->data(), cond.offset, cond.length,
+                                    output->buffers[0]->mutable_data(), 0);
+      break;
+    case RIGHT_ALL_VALID:  // = 4
+      // out_valid = c_valid & ~cond.data
+      arrow::internal::BitmapAndNot(cond.buffers[0]->data(), cond.offset,
+                                    cond.buffers[1]->data(), cond.offset, cond.length, 0,
+                                    output->buffers[0]->mutable_data());
+      break;
+    case COND_ALL_VALID | LEFT_ALL_VALID:  // = 3
+      // out_valid = cond.data
+      output->buffers[0] = SliceBuffer(cond.buffers[1], cond.offset, cond.length);
+      break;
+    case LEFT_ALL_VALID:  // = 2
+      // out_valid = cond.valid & cond.data
+      arrow::internal::BitmapAnd(cond.buffers[0]->data(), cond.offset,
+                                 cond.buffers[1]->data(), cond.offset, cond.length, 0,
+                                 output->buffers[0]->mutable_data());
+      break;
+    case COND_ALL_VALID:  // = 1
+      // out_valid = 0 --> nothing to do; but requires out_valid to be a all-zero buffer

Review comment:
       I think there is a mismatch in the kernel comment and the impl. 
   https://github.com/apache/arrow/blob/29130ca54cd773f6f52e17bc78d37fb72d53eb49/cpp/src/arrow/compute/kernel.cc#L56
   It looks like it is zeroed out for bitmaps




-- 
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



[GitHub] [arrow] nirandaperera commented on a change in pull request #10410: ARROW-10640: [C++] A "where" kernel to combine two arrays based on a mask

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_if_else.cc
##########
@@ -0,0 +1,838 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <arrow/compute/api.h>
+#include <arrow/util/bit_block_counter.h>
+#include <arrow/util/bitmap.h>
+#include <arrow/util/bitmap_ops.h>
+
+#include "codegen_internal.h"
+
+namespace arrow {
+using internal::BitBlockCount;
+using internal::BitBlockCounter;
+using internal::Bitmap;
+
+namespace compute {
+
+namespace {
+
+enum { COND_ALL_VALID = 1, LEFT_ALL_VALID = 2, RIGHT_ALL_VALID = 4 };
+
+// if the condition is null then output is null otherwise we take validity from the
+// selected argument
+// ie. cond.valid & (cond.data & left.valid | ~cond.data & right.valid)
+Status PromoteNullsVisitor(KernelContext* ctx, const ArrayData& cond, const Scalar& left,
+                           const Scalar& right, ArrayData* output) {
+  uint8_t flag = right.is_valid * 4 + left.is_valid * 2 + !cond.MayHaveNulls();
+
+  if (flag < 6 && flag != 3) {
+    // there will be a validity buffer in the output
+    ARROW_ASSIGN_OR_RAISE(output->buffers[0], ctx->AllocateBitmap(cond.length));
+  }
+
+  // if the condition is null then output is null otherwise we take validity from the
+  // selected argument
+  // ie. cond.valid & (cond.data & left.valid | ~cond.data & right.valid)
+  switch (flag) {
+    case COND_ALL_VALID | LEFT_ALL_VALID | RIGHT_ALL_VALID:  // = 7
+      break;
+    case LEFT_ALL_VALID | RIGHT_ALL_VALID:  // = 6
+      // out_valid = c_valid
+      output->buffers[0] = SliceBuffer(cond.buffers[0], cond.offset, cond.length);
+      break;
+    case COND_ALL_VALID | RIGHT_ALL_VALID:  // = 5
+      // out_valid = ~cond.data
+      arrow::internal::InvertBitmap(cond.buffers[1]->data(), cond.offset, cond.length,
+                                    output->buffers[0]->mutable_data(), 0);
+      break;
+    case RIGHT_ALL_VALID:  // = 4
+      // out_valid = c_valid & ~cond.data
+      arrow::internal::BitmapAndNot(cond.buffers[0]->data(), cond.offset,
+                                    cond.buffers[1]->data(), cond.offset, cond.length, 0,
+                                    output->buffers[0]->mutable_data());
+      break;
+    case COND_ALL_VALID | LEFT_ALL_VALID:  // = 3
+      // out_valid = cond.data
+      output->buffers[0] = SliceBuffer(cond.buffers[1], cond.offset, cond.length);
+      break;
+    case LEFT_ALL_VALID:  // = 2
+      // out_valid = cond.valid & cond.data
+      arrow::internal::BitmapAnd(cond.buffers[0]->data(), cond.offset,
+                                 cond.buffers[1]->data(), cond.offset, cond.length, 0,
+                                 output->buffers[0]->mutable_data());
+      break;
+    case COND_ALL_VALID:  // = 1
+      // out_valid = 0 --> nothing to do; but requires out_valid to be a all-zero buffer
+      break;
+    case 0:  // RLC = 000
+      // out_valid = 0 --> nothing to do; but requires out_valid to be a all-zero buffer
+      break;
+  }
+  return Status::OK();
+}
+
+Status PromoteNullsVisitor(KernelContext* ctx, const ArrayData& cond,
+                           const ArrayData& left, const Scalar& right,
+                           ArrayData* output) {
+  uint8_t flag = right.is_valid * 4 + !left.MayHaveNulls() * 2 + !cond.MayHaveNulls();
+
+  enum { C_VALID, C_DATA, L_VALID };
+
+  Bitmap bitmaps[3];
+  bitmaps[C_VALID] = {cond.buffers[0], cond.offset, cond.length};
+  bitmaps[C_DATA] = {cond.buffers[1], cond.offset, cond.length};
+  bitmaps[L_VALID] = {left.buffers[0], left.offset, left.length};
+
+  uint64_t* out_validity = nullptr;
+  if (flag < 6 && flag != 3) {
+    // there will be a validity buffer in the output
+    ARROW_ASSIGN_OR_RAISE(output->buffers[0], ctx->AllocateBitmap(cond.length));
+    out_validity = output->GetMutableValues<uint64_t>(0);
+  }
+
+  // lambda function that will be used inside the visitor
+  int64_t i = 0;
+  auto apply = [&](uint64_t c_valid, uint64_t c_data, uint64_t l_valid,
+                   uint64_t r_valid) {
+    out_validity[i] = c_valid & ((c_data & l_valid) | (~c_data & r_valid));
+    i++;
+  };
+
+  // if the condition is null then output is null otherwise we take validity from the
+  // selected argument
+  // ie. cond.valid & (cond.data & left.valid | ~cond.data & right.valid)
+  switch (flag) {
+    case COND_ALL_VALID | LEFT_ALL_VALID | RIGHT_ALL_VALID:  // RLC = 111
+      break;
+    case LEFT_ALL_VALID | RIGHT_ALL_VALID:  // RLC = 110
+      output->buffers[0] = SliceBuffer(cond.buffers[0], cond.offset, cond.length);
+      break;
+    case COND_ALL_VALID | RIGHT_ALL_VALID:  // RLC = 101
+      // bitmaps[C_VALID] might be null; override to make it safe for Visit()
+      bitmaps[C_VALID] = bitmaps[C_DATA];
+      Bitmap::VisitWords(bitmaps, [&](std::array<uint64_t, 3> words) {
+        apply(UINT64_MAX, words[C_DATA], words[L_VALID], UINT64_MAX);
+      });
+      break;
+    case RIGHT_ALL_VALID:  // RLC = 100
+      Bitmap::VisitWords(bitmaps, [&](std::array<uint64_t, 3> words) {
+        apply(words[C_VALID], words[C_DATA], words[L_VALID], UINT64_MAX);
+      });
+      break;
+    case COND_ALL_VALID | LEFT_ALL_VALID:  // RLC = 011
+      // only cond.data is passed
+      output->buffers[0] = SliceBuffer(cond.buffers[1], cond.offset, cond.length);
+      break;
+    case LEFT_ALL_VALID:  // RLC = 010
+      // out_valid = cond.valid & cond.data
+      arrow::internal::BitmapAnd(cond.buffers[0]->data(), cond.offset,
+                                 cond.buffers[1]->data(), cond.offset, cond.length, 0,
+                                 output->buffers[0]->mutable_data());
+      break;
+    case COND_ALL_VALID:  // RLC = 001
+      // out_valid = cond.data & left.valid
+      arrow::internal::BitmapAnd(cond.buffers[1]->data(), cond.offset,
+                                 left.buffers[0]->data(), left.offset, cond.length, 0,
+                                 output->buffers[0]->mutable_data());
+      break;
+    case 0:  // RLC = 000
+      Bitmap::VisitWords(bitmaps, [&](std::array<uint64_t, 3> words) {
+        apply(words[C_VALID], words[C_DATA], words[L_VALID], 0);
+      });
+      break;
+  }
+  return Status::OK();
+}
+
+// if the condition is null then output is null otherwise we take validity from the
+// selected argument
+// ie. cond.valid & (cond.data & left.valid | ~cond.data & right.valid)
+Status PromoteNullsVisitor(KernelContext* ctx, const ArrayData& cond, const Scalar& left,
+                           const ArrayData& right, ArrayData* output) {
+  uint8_t flag = !right.MayHaveNulls() * 4 + left.is_valid * 2 + !cond.MayHaveNulls();
+
+  enum { C_VALID, C_DATA, R_VALID };
+
+  Bitmap bitmaps[3];
+  bitmaps[C_VALID] = {cond.buffers[0], cond.offset, cond.length};
+  bitmaps[C_DATA] = {cond.buffers[1], cond.offset, cond.length};
+  bitmaps[R_VALID] = {right.buffers[0], right.offset, right.length};
+
+  uint64_t* out_validity = nullptr;
+  if (flag < 6) {
+    // there will be a validity buffer in the output
+    ARROW_ASSIGN_OR_RAISE(output->buffers[0], ctx->AllocateBitmap(cond.length));
+    out_validity = output->GetMutableValues<uint64_t>(0);
+  }
+
+  // lambda function that will be used inside the visitor
+  int64_t i = 0;
+  auto apply = [&](uint64_t c_valid, uint64_t c_data, uint64_t l_valid,
+                   uint64_t r_valid) {
+    out_validity[i] = c_valid & ((c_data & l_valid) | (~c_data & r_valid));
+    i++;
+  };
+
+  // if the condition is null then output is null otherwise we take validity from the
+  // selected argument
+  // ie. cond.valid & (cond.data & left.valid | ~cond.data & right.valid)
+  switch (flag) {
+    case COND_ALL_VALID | LEFT_ALL_VALID | RIGHT_ALL_VALID:  // RLC = 111
+      break;
+    case LEFT_ALL_VALID | RIGHT_ALL_VALID:  // RLC = 110
+      output->buffers[0] = SliceBuffer(cond.buffers[0], cond.offset, cond.length);
+      break;
+    case COND_ALL_VALID | RIGHT_ALL_VALID:  // RLC = 101
+      // out_valid = ~cond.data
+      arrow::internal::InvertBitmap(cond.buffers[1]->data(), cond.offset, cond.length,
+                                    output->buffers[0]->mutable_data(), 0);
+      break;
+    case RIGHT_ALL_VALID:  // RLC = 100
+      // out_valid = c_valid & ~cond.data
+      arrow::internal::BitmapAndNot(cond.buffers[0]->data(), cond.offset,
+                                    cond.buffers[1]->data(), cond.offset, cond.length, 0,
+                                    output->buffers[0]->mutable_data());
+      break;
+    case COND_ALL_VALID | LEFT_ALL_VALID:  // RLC = 011
+      // bitmaps[C_VALID] might be null; override to make it safe for Visit()
+      bitmaps[C_VALID] = bitmaps[C_DATA];
+      Bitmap::VisitWords(bitmaps, [&](std::array<uint64_t, 3> words) {
+        apply(UINT64_MAX, words[C_DATA], UINT64_MAX, words[R_VALID]);
+      });
+      break;
+    case LEFT_ALL_VALID:  // RLC = 010
+      Bitmap::VisitWords(bitmaps, [&](std::array<uint64_t, 3> words) {
+        apply(words[C_VALID], words[C_DATA], UINT64_MAX, words[R_VALID]);
+      });
+      break;
+    case COND_ALL_VALID:  // RLC = 001
+      // out_valid =  ~cond.data & right.valid
+      arrow::internal::BitmapAndNot(right.buffers[0]->data(), right.offset,
+                                    cond.buffers[1]->data(), cond.offset, cond.length, 0,
+                                    output->buffers[0]->mutable_data());
+      break;
+    case 0:  // RLC = 000
+      Bitmap::VisitWords(bitmaps, [&](std::array<uint64_t, 3> words) {
+        apply(words[C_VALID], words[C_DATA], 0, words[R_VALID]);
+      });
+      break;
+  }
+  return Status::OK();
+}
+
+// if the condition is null then output is null otherwise we take validity from the
+// selected argument
+// ie. cond.valid & (cond.data & left.valid | ~cond.data & right.valid)
+Status PromoteNullsVisitor(KernelContext* ctx, const ArrayData& cond,
+                           const ArrayData& left, const ArrayData& right,
+                           ArrayData* output) {
+  uint8_t flag =
+      !right.MayHaveNulls() * 4 + !left.MayHaveNulls() * 2 + !cond.MayHaveNulls();
+
+  enum { C_VALID, C_DATA, L_VALID, R_VALID };
+
+  Bitmap bitmaps[4];
+  bitmaps[C_VALID] = {cond.buffers[0], cond.offset, cond.length};
+  bitmaps[C_DATA] = {cond.buffers[1], cond.offset, cond.length};
+  bitmaps[L_VALID] = {left.buffers[0], left.offset, left.length};
+  bitmaps[R_VALID] = {right.buffers[0], right.offset, right.length};
+
+  uint64_t* out_validity = nullptr;
+  if (flag < 6) {
+    // there will be a validity buffer in the output
+    ARROW_ASSIGN_OR_RAISE(output->buffers[0], ctx->AllocateBitmap(cond.length));
+    out_validity = output->GetMutableValues<uint64_t>(0);
+  }
+
+  // lambda function that will be used inside the visitor
+  int64_t i = 0;
+  auto apply = [&](uint64_t c_valid, uint64_t c_data, uint64_t l_valid,
+                   uint64_t r_valid) {
+    out_validity[i] = c_valid & ((c_data & l_valid) | (~c_data & r_valid));
+    i++;
+  };
+
+  // if the condition is null then output is null otherwise we take validity from the
+  // selected argument
+  // ie. cond.valid & (cond.data & left.valid | ~cond.data & right.valid)
+  switch (flag) {
+    case COND_ALL_VALID | LEFT_ALL_VALID | RIGHT_ALL_VALID:  // RLC = 111
+      break;
+    case LEFT_ALL_VALID | RIGHT_ALL_VALID:  // RLC = 110
+      output->buffers[0] = SliceBuffer(cond.buffers[0], cond.offset, cond.length);
+      break;
+    case COND_ALL_VALID | RIGHT_ALL_VALID:  // RLC = 101
+      // bitmaps[C_VALID], bitmaps[R_VALID] might be null; override to make it safe for
+      // Visit()
+      bitmaps[C_VALID] = bitmaps[C_DATA];
+      bitmaps[R_VALID] = bitmaps[C_DATA];
+      Bitmap::VisitWords(bitmaps, [&](std::array<uint64_t, 4> words) {
+        apply(UINT64_MAX, words[C_DATA], words[L_VALID], UINT64_MAX);
+      });
+      break;
+    case RIGHT_ALL_VALID:  // RLC = 100
+      // bitmaps[R_VALID] might be null; override to make it safe for Visit()
+      bitmaps[R_VALID] = bitmaps[C_DATA];
+      Bitmap::VisitWords(bitmaps, [&](std::array<uint64_t, 4> words) {
+        apply(words[C_VALID], words[C_DATA], words[L_VALID], UINT64_MAX);
+      });
+      break;
+    case COND_ALL_VALID | LEFT_ALL_VALID:  // RLC = 011
+      // bitmaps[C_VALID], bitmaps[L_VALID] might be null; override to make it safe for
+      // Visit()
+      bitmaps[C_VALID] = bitmaps[C_DATA];
+      bitmaps[L_VALID] = bitmaps[C_DATA];
+      Bitmap::VisitWords(bitmaps, [&](std::array<uint64_t, 4> words) {
+        apply(UINT64_MAX, words[C_DATA], UINT64_MAX, words[R_VALID]);
+      });
+      break;
+    case LEFT_ALL_VALID:  // RLC = 010
+      // bitmaps[L_VALID] might be null; override to make it safe for Visit()
+      bitmaps[L_VALID] = bitmaps[C_DATA];
+      Bitmap::VisitWords(bitmaps, [&](std::array<uint64_t, 4> words) {
+        apply(words[C_VALID], words[C_DATA], UINT64_MAX, words[R_VALID]);
+      });
+      break;
+    case COND_ALL_VALID:  // RLC = 001
+      // bitmaps[C_VALID] might be null; override to make it safe for Visit()
+      bitmaps[C_VALID] = bitmaps[C_DATA];
+      Bitmap::VisitWords(bitmaps, [&](std::array<uint64_t, 4> words) {
+        apply(UINT64_MAX, words[C_DATA], words[L_VALID], words[R_VALID]);
+      });
+      break;
+    case 0:  // RLC = 000
+      Bitmap::VisitWords(bitmaps, [&](std::array<uint64_t, 4> words) {
+        apply(words[C_VALID], words[C_DATA], words[L_VALID], words[R_VALID]);
+      });
+      break;
+  }
+  return Status::OK();
+}
+
+// nulls will be promoted as follows:
+// cond.valid && (cond.data && left.valid || ~cond.data && right.valid)
+// Note: we have to work on ArrayData. Otherwise we won't be able to handle array
+// offsets AAA
+/*Status PromoteNulls(KernelContext* ctx, const ArrayData& cond, const ArrayData& left,
+                    const ArrayData& right, ArrayData* output) {
+  if (!cond.MayHaveNulls() && !left.MayHaveNulls() && !right.MayHaveNulls()) {
+    return Status::OK();  // no nulls to handle
+  }
+  const int64_t len = cond.length;
+
+  // out_validity = ~cond.data --> mask right values
+  ARROW_ASSIGN_OR_RAISE(
+      std::shared_ptr<Buffer> out_validity,
+      arrow::internal::InvertBitmap(ctx->memory_pool(), cond.buffers[1]->data(),
+                                    cond.offset, len));
+
+  if (right.MayHaveNulls()) {  // out_validity = right.valid && ~cond.data
+    arrow::internal::BitmapAnd(right.buffers[0]->data(), right.offset,
+                               out_validity->data(), 0, len, 0,
+                               out_validity->mutable_data());
+  }
+
+  std::shared_ptr<Buffer> tmp_buf;
+  if (left.MayHaveNulls()) {
+    // tmp_buf = left.valid && cond.data
+    ARROW_ASSIGN_OR_RAISE(
+        tmp_buf, arrow::internal::BitmapAnd(ctx->memory_pool(), left.buffers[0]->data(),
+                                            left.offset, cond.buffers[1]->data(),
+                                            cond.offset, len, 0));
+  } else {  // if left all valid --> tmp_buf = cond.data (zero copy slice)
+    tmp_buf = SliceBuffer(cond.buffers[1], cond.offset, cond.length);
+  }
+
+  // out_validity = cond.data && left.valid || ~cond.data && right.valid
+  arrow::internal::BitmapOr(out_validity->data(), 0, tmp_buf->data(), 0, len, 0,
+                            out_validity->mutable_data());
+
+  if (cond.MayHaveNulls()) {
+    // out_validity = cond.valid && (cond.data && left.valid || ~cond.data && right.valid)
+    ::arrow::internal::BitmapAnd(out_validity->data(), 0, cond.buffers[0]->data(),
+                                 cond.offset, len, 0, out_validity->mutable_data());
+  }
+
+  output->buffers[0] = std::move(out_validity);
+  output->GetNullCount();  // update null count
+  return Status::OK();
+}
+
+// cond.valid && (cond.data && left.valid || ~cond.data && right.valid)
+// ASA and AAS
+Status PromoteNulls(KernelContext* ctx, const ArrayData& cond, const Scalar& left,
+                    const ArrayData& right, ArrayData* output) {
+  if (!cond.MayHaveNulls() && left.is_valid && !right.MayHaveNulls()) {
+    return Status::OK();  // no nulls to handle
+  }
+  const int64_t len = cond.length;
+
+  // out_validity = ~cond.data
+  ARROW_ASSIGN_OR_RAISE(
+      std::shared_ptr<Buffer> out_validity,
+      arrow::internal::InvertBitmap(ctx->memory_pool(), cond.buffers[1]->data(),
+                                    cond.offset, len));
+  // out_validity = ~cond.data && right.valid
+  if (right.MayHaveNulls()) {  // out_validity = right.valid && ~cond.data
+    arrow::internal::BitmapAnd(right.buffers[0]->data(), right.offset,
+                               out_validity->data(), 0, len, 0,
+                               out_validity->mutable_data());
+  }
+
+  // out_validity = cond.data && left.valid || ~cond.data && right.valid
+  if (left.is_valid) {
+    arrow::internal::BitmapOr(out_validity->data(), 0, cond.buffers[1]->data(),
+                              cond.offset, len, 0, out_validity->mutable_data());
+  }
+
+  // out_validity = cond.valid && (cond.data && left.valid || ~cond.data && right.valid)
+  if (cond.MayHaveNulls()) {
+    ::arrow::internal::BitmapAnd(out_validity->data(), 0, cond.buffers[0]->data(),
+                                 cond.offset, len, 0, out_validity->mutable_data());
+  }
+
+  output->buffers[0] = std::move(out_validity);
+  output->GetNullCount();  // update null count
+  return Status::OK();
+}
+
+// cond.valid && (cond.data && left.valid || ~cond.data && right.valid)
+// ASS
+Status PromoteNulls(KernelContext* ctx, const ArrayData& cond, const Scalar& left,
+                    const Scalar& right, ArrayData* output) {
+  if (!cond.MayHaveNulls() && left.is_valid && right.is_valid) {
+    return Status::OK();  // no nulls to handle
+  }
+  const int64_t len = cond.length;
+
+  std::shared_ptr<Buffer> out_validity;
+  if (right.is_valid) {
+    // out_validity = ~cond.data
+    ARROW_ASSIGN_OR_RAISE(
+        out_validity, arrow::internal::InvertBitmap(
+                          ctx->memory_pool(), cond.buffers[1]->data(), cond.offset, len));
+  } else {
+    // out_validity = [0...]
+    ARROW_ASSIGN_OR_RAISE(out_validity, ctx->AllocateBitmap(len));
+  }
+
+  // out_validity = cond.data && left.valid || ~cond.data && right.valid
+  if (left.is_valid) {
+    arrow::internal::BitmapOr(out_validity->data(), 0, cond.buffers[1]->data(),
+                              cond.offset, len, 0, out_validity->mutable_data());
+  }
+
+  // out_validity = cond.valid && (cond.data && left.valid || ~cond.data && right.valid)
+  if (cond.MayHaveNulls()) {
+    ::arrow::internal::BitmapAnd(out_validity->data(), 0, cond.buffers[0]->data(),
+                                 cond.offset, len, 0, out_validity->mutable_data());
+  }
+
+  output->buffers[0] = std::move(out_validity);
+  output->GetNullCount();  // update null count
+  return Status::OK();
+}
+
+// todo: this could be dangerous because the inverted arraydata buffer[1] may not be
+//  available outside Exec's scope
+Status InvertBoolArrayData(KernelContext* ctx, const ArrayData& input,
+                           ArrayData* output) {
+  // null buffer
+  if (input.MayHaveNulls()) {
+    output->buffers.emplace_back(
+        SliceBuffer(input.buffers[0], input.offset, input.length));
+  } else {
+    output->buffers.push_back(NULLPTR);
+  }
+
+  // data buffer
+  ARROW_ASSIGN_OR_RAISE(
+      std::shared_ptr<Buffer> inv_data,
+      arrow::internal::InvertBitmap(ctx->memory_pool(), input.buffers[1]->data(),
+                                    input.offset, input.length));
+  output->buffers.emplace_back(std::move(inv_data));
+  return Status::OK();
+}
+ */
+
+template <typename Type, typename Enable = void>
+struct IfElseFunctor {};
+
+// only number types needs to be handled for Fixed sized primitive data types because,
+// internal::GenerateTypeAgnosticPrimitive forwards types to the corresponding unsigned
+// int type
+template <typename Type>
+struct IfElseFunctor<Type, enable_if_number<Type>> {
+  using T = typename TypeTraits<Type>::CType;
+  // A - Array
+  // S - Scalar
+
+  //  AAA
+  static Status Call(KernelContext* ctx, const ArrayData& cond, const ArrayData& left,
+                     const ArrayData& right, ArrayData* out) {
+    ARROW_RETURN_NOT_OK(PromoteNullsVisitor(ctx, cond, left, right, out));
+
+    ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Buffer> out_buf,
+                          ctx->Allocate(cond.length * sizeof(T)));
+    T* out_values = reinterpret_cast<T*>(out_buf->mutable_data());
+
+    // copy right data to out_buff
+    const T* right_data = right.GetValues<T>(1);
+    std::memcpy(out_values, right_data, right.length * sizeof(T));
+
+    const auto* cond_data = cond.buffers[1]->data();  // this is a BoolArray
+    BitBlockCounter bit_counter(cond_data, cond.offset, cond.length);
+
+    // selectively copy values from left data
+    const T* left_data = left.GetValues<T>(1);
+    int64_t offset = cond.offset;
+
+    // todo this can be improved by intrinsics. ex: _mm*_mask_store_e* (vmovdqa*)
+    while (offset < cond.offset + cond.length) {
+      const BitBlockCount& block = bit_counter.NextWord();
+      if (block.AllSet()) {  // all from left
+        std::memcpy(out_values, left_data, block.length * sizeof(T));
+      } else if (block.popcount) {  // selectively copy from left
+        for (int64_t i = 0; i < block.length; ++i) {
+          if (BitUtil::GetBit(cond_data, offset + i)) {
+            out_values[i] = left_data[i];
+          }
+        }
+      }
+
+      offset += block.length;
+      out_values += block.length;
+      left_data += block.length;
+    }
+
+    out->buffers[1] = std::move(out_buf);
+    return Status::OK();
+  }
+
+  // ASA
+  static Status Call(KernelContext* ctx, const ArrayData& cond, const Scalar& left,
+                     const ArrayData& right, ArrayData* out) {
+    ARROW_RETURN_NOT_OK(PromoteNullsVisitor(ctx, cond, left, right, out));
+
+    ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Buffer> out_buf,
+                          ctx->Allocate(cond.length * sizeof(T)));
+    T* out_values = reinterpret_cast<T*>(out_buf->mutable_data());
+
+    // copy right data to out_buff
+    const T* right_data = right.GetValues<T>(1);
+    std::memcpy(out_values, right_data, right.length * sizeof(T));
+
+    const auto* cond_data = cond.buffers[1]->data();  // this is a BoolArray
+    BitBlockCounter bit_counter(cond_data, cond.offset, cond.length);
+
+    // selectively copy values from left data
+    T left_data = internal::UnboxScalar<Type>::Unbox(left);
+    int64_t offset = cond.offset;
+
+    // todo this can be improved by intrinsics. ex: _mm*_mask_store_e* (vmovdqa*)
+    while (offset < cond.offset + cond.length) {
+      const BitBlockCount& block = bit_counter.NextWord();
+      if (block.AllSet()) {  // all from left
+        std::fill(out_values, out_values + block.length, left_data);
+      } else if (block.popcount) {  // selectively copy from left
+        for (int64_t i = 0; i < block.length; ++i) {
+          if (BitUtil::GetBit(cond_data, offset + i)) {
+            out_values[i] = left_data;
+          }
+        }
+      }
+
+      offset += block.length;
+      out_values += block.length;
+    }
+
+    out->buffers[1] = std::move(out_buf);
+    return Status::OK();
+  }
+
+  // AAS
+  static Status Call(KernelContext* ctx, const ArrayData& cond, const ArrayData& left,
+                     const Scalar& right, ArrayData* out) {
+    ARROW_RETURN_NOT_OK(PromoteNullsVisitor(ctx, cond, left, right, out));
+
+    ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Buffer> out_buf,
+                          ctx->Allocate(cond.length * sizeof(T)));
+    T* out_values = reinterpret_cast<T*>(out_buf->mutable_data());
+
+    // copy left data to out_buff
+    const T* left_data = left.GetValues<T>(1);
+    std::memcpy(out_values, left_data, left.length * sizeof(T));
+
+    const auto* cond_data = cond.buffers[1]->data();  // this is a BoolArray
+    BitBlockCounter bit_counter(cond_data, cond.offset, cond.length);
+
+    // selectively copy values from left data
+    T right_data = internal::UnboxScalar<Type>::Unbox(right);
+    int64_t offset = cond.offset;
+
+    // todo this can be improved by intrinsics. ex: _mm*_mask_store_e* (vmovdqa*)
+    // left data is already in the output buffer. Therefore, mask needs to be inverted
+    while (offset < cond.offset + cond.length) {
+      const BitBlockCount& block = bit_counter.NextWord();
+      if (block.NoneSet()) {  // all from right
+        std::fill(out_values, out_values + block.length, right_data);
+      } else if (block.popcount) {  // selectively copy from right
+        for (int64_t i = 0; i < block.length; ++i) {
+          if (!BitUtil::GetBit(cond_data, offset + i)) {
+            out_values[i] = right_data;
+          }
+        }
+      }
+
+      offset += block.length;
+      out_values += block.length;
+    }
+
+    out->buffers[1] = std::move(out_buf);
+    return Status::OK();
+  }
+
+  // ASS
+  static Status Call(KernelContext* ctx, const ArrayData& cond, const Scalar& left,
+                     const Scalar& right, ArrayData* out) {
+    ARROW_RETURN_NOT_OK(PromoteNullsVisitor(ctx, cond, left, right, out));
+
+    ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Buffer> out_buf,
+                          ctx->Allocate(cond.length * sizeof(T)));
+    T* out_values = reinterpret_cast<T*>(out_buf->mutable_data());
+
+    // copy right data to out_buff
+    T right_data = internal::UnboxScalar<Type>::Unbox(right);
+    std::fill(out_values, out_values + cond.length, right_data);
+
+    const auto* cond_data = cond.buffers[1]->data();  // this is a BoolArray
+    BitBlockCounter bit_counter(cond_data, cond.offset, cond.length);
+
+    // selectively copy values from left data
+    T left_data = internal::UnboxScalar<Type>::Unbox(left);
+    int64_t offset = cond.offset;
+
+    // todo this can be improved by intrinsics. ex: _mm*_mask_store_e* (vmovdqa*)
+    while (offset < cond.offset + cond.length) {
+      const BitBlockCount& block = bit_counter.NextWord();
+      if (block.AllSet()) {  // all from left
+        std::fill(out_values, out_values + block.length, left_data);
+      } else if (block.popcount) {  // selectively copy from left
+        for (int64_t i = 0; i < block.length; ++i) {
+          if (BitUtil::GetBit(cond_data, offset + i)) {
+            out_values[i] = left_data;
+          }
+        }
+      }
+
+      offset += block.length;
+      out_values += block.length;
+    }
+
+    out->buffers[1] = std::move(out_buf);
+    return Status::OK();
+  }
+};
+
+template <typename Type>
+struct IfElseFunctor<Type, enable_if_boolean<Type>> {
+  // AAA
+  static Status Call(KernelContext* ctx, const ArrayData& cond, const ArrayData& left,
+                     const ArrayData& right, ArrayData* out) {
+    ARROW_RETURN_NOT_OK(PromoteNullsVisitor(ctx, cond, left, right, out));
+
+    // out_buff = right & ~cond
+    ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Buffer> out_buf,
+                          arrow::internal::BitmapAndNot(
+                              ctx->memory_pool(), right.buffers[1]->data(), right.offset,
+                              cond.buffers[1]->data(), cond.offset, cond.length, 0));
+
+    // out_buff = left & cond
+    ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Buffer> temp_buf,
+                          arrow::internal::BitmapAnd(
+                              ctx->memory_pool(), left.buffers[1]->data(), left.offset,
+                              cond.buffers[1]->data(), cond.offset, cond.length, 0));
+
+    arrow::internal::BitmapOr(out_buf->data(), 0, temp_buf->data(), 0, cond.length, 0,
+                              out_buf->mutable_data());
+    out->buffers[1] = std::move(out_buf);
+    return Status::OK();
+  }
+
+  // ASA
+  static Status Call(KernelContext* ctx, const ArrayData& cond, const Scalar& left,
+                     const ArrayData& right, ArrayData* out) {
+    ARROW_RETURN_NOT_OK(PromoteNullsVisitor(ctx, cond, left, right, out));
+
+    // out_buff = right & ~cond
+    ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Buffer> out_buf,
+                          arrow::internal::BitmapAndNot(
+                              ctx->memory_pool(), right.buffers[1]->data(), right.offset,
+                              cond.buffers[1]->data(), cond.offset, cond.length, 0));
+
+    // out_buff = left & cond
+    bool left_data = internal::UnboxScalar<BooleanType>::Unbox(left);
+    if (left_data) {
+      arrow::internal::BitmapOr(out_buf->data(), 0, cond.buffers[1]->data(), cond.offset,
+                                cond.length, 0, out_buf->mutable_data());
+    }
+
+    out->buffers[1] = std::move(out_buf);
+    return Status::OK();
+  }
+
+  // AAS
+  static Status Call(KernelContext* ctx, const ArrayData& cond, const ArrayData& left,
+                     const Scalar& right, ArrayData* out) {
+    // todo impl
+    return Status::OK();
+  }
+
+  // ASS
+  static Status Call(KernelContext* ctx, const ArrayData& cond, const Scalar& left,
+                     const Scalar& right, ArrayData* out) {
+    // todo impl
+    return Status::OK();
+  }
+};
+
+template <typename Type>
+struct IfElseFunctor<Type, enable_if_null<Type>> {
+  template <typename T>
+  static inline Status ReturnCopy(const T& in, T* out) {
+    // Nothing preallocated, so we assign in into the output
+    *out = in;
+    return Status::OK();
+  }
+
+  // AAA
+  static Status Call(KernelContext* ctx, const ArrayData& cond, const ArrayData& left,
+                     const ArrayData& right, ArrayData* out) {
+    return ReturnCopy(left, out);
+  }
+
+  // ASA
+  static Status Call(KernelContext* ctx, const ArrayData& cond, const Scalar& left,
+                     const ArrayData& right, ArrayData* out) {
+    return ReturnCopy(right, out);
+  }
+
+  // AAS
+  static Status Call(KernelContext* ctx, const ArrayData& cond, const ArrayData& left,
+                     const Scalar& right, ArrayData* out) {
+    return ReturnCopy(left, out);
+  }
+
+  // ASS
+  static Status Call(KernelContext* ctx, const ArrayData& cond, const Scalar& left,
+                     const Scalar& right, ArrayData* out) {
+    return ReturnCopy(cond, out);
+  }
+};
+
+template <typename Type>
+struct ResolveIfElseExec {
+  static Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    // cond is scalar

Review comment:
       yep.. still didnt add that. So, there are 4^3 = 64 (each param can be `all_valid_array, array_with_nulls, valid_scalar, null_scalar`) test cases for a single type-category :grin: 




-- 
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



[GitHub] [arrow] bkietz commented on a change in pull request #10410: ARROW-10640: [C++] A "where" kernel to combine two arrays based on a mask

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_if_else_test.cc
##########
@@ -0,0 +1,474 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <arrow/array.h>
+#include <arrow/compute/api_scalar.h>
+#include <arrow/compute/kernels/test_util.h>
+#include <arrow/testing/gtest_util.h>
+#include <gtest/gtest.h>
+
+namespace arrow {
+namespace compute {
+
+void CheckIfElseOutput(const Datum& cond, const Datum& left, const Datum& right,
+                       const Datum& expected) {
+  ASSERT_OK_AND_ASSIGN(Datum datum_out, IfElse(cond, left, right));
+  if (datum_out.is_array()) {
+    std::shared_ptr<Array> result = datum_out.make_array();
+    ASSERT_OK(result->ValidateFull());
+    std::shared_ptr<Array> expected_ = expected.make_array();
+    AssertArraysEqual(*expected_, *result, /*verbose=*/true);
+  } else {  // expecting scalar
+    const std::shared_ptr<Scalar>& result = datum_out.scalar();
+    const std::shared_ptr<Scalar>& expected_ = expected.scalar();
+    AssertScalarsEqual(*expected_, *result, /*verbose=*/true);
+  }
+}
+
+void CheckIfElseOutputAAA(const std::shared_ptr<DataType>& type, const std::string& cond,
+                          const std::string& left, const std::string& right,
+                          const std::string& expected) {
+  const std::shared_ptr<Array>& cond_ = ArrayFromJSON(boolean(), cond);
+  const std::shared_ptr<Array>& left_ = ArrayFromJSON(type, left);
+  const std::shared_ptr<Array>& right_ = ArrayFromJSON(type, right);
+  const std::shared_ptr<Array>& expected_ = ArrayFromJSON(type, expected);
+  CheckIfElseOutput(cond_, left_, right_, expected_);
+}
+
+void CheckIfElseOutputAAS(const std::shared_ptr<DataType>& type, const std::string& cond,
+                          const std::string& left, const std::shared_ptr<Scalar>& right,
+                          const std::string& expected) {
+  const std::shared_ptr<Array>& cond_ = ArrayFromJSON(boolean(), cond);
+  const std::shared_ptr<Array>& left_ = ArrayFromJSON(type, left);
+  const std::shared_ptr<Array>& expected_ = ArrayFromJSON(type, expected);
+  CheckIfElseOutput(cond_, left_, right, expected_);
+}
+
+void CheckIfElseOutputASA(const std::shared_ptr<DataType>& type, const std::string& cond,
+                          const std::shared_ptr<Scalar>& left, const std::string& right,
+                          const std::string& expected) {
+  const std::shared_ptr<Array>& cond_ = ArrayFromJSON(boolean(), cond);
+  const std::shared_ptr<Array>& right_ = ArrayFromJSON(type, right);
+  const std::shared_ptr<Array>& expected_ = ArrayFromJSON(type, expected);
+  CheckIfElseOutput(cond_, left, right_, expected_);
+}
+
+void CheckIfElseOutputASS(const std::shared_ptr<DataType>& type, const std::string& cond,
+                          const std::shared_ptr<Scalar>& left,
+                          const std::shared_ptr<Scalar>& right,
+                          const std::string& expected) {
+  const std::shared_ptr<Array>& cond_ = ArrayFromJSON(boolean(), cond);
+  const std::shared_ptr<Array>& expected_ = ArrayFromJSON(type, expected);
+  CheckIfElseOutput(cond_, left, right, expected_);
+}
+
+void CheckIfElseOutputSAA(const std::shared_ptr<DataType>& type,
+                          const std::shared_ptr<Scalar>& cond, const std::string& left,
+                          const std::string& right, const std::string& expected) {
+  const std::shared_ptr<Array>& left_ = ArrayFromJSON(type, left);
+  const std::shared_ptr<Array>& right_ = ArrayFromJSON(type, right);
+  const std::shared_ptr<Array>& expected_ = ArrayFromJSON(type, expected);
+  CheckIfElseOutput(cond, left_, right_, expected_);
+}
+
+void CheckIfElseOutputSAS(const std::shared_ptr<DataType>& type,
+                          const std::shared_ptr<Scalar>& cond, const std::string& left,
+                          const std::shared_ptr<Scalar>& right,
+                          const std::string& expected) {
+  const std::shared_ptr<Array>& left_ = ArrayFromJSON(type, left);
+  const std::shared_ptr<Array>& expected_ = ArrayFromJSON(type, expected);
+  CheckIfElseOutput(cond, left_, right, expected_);
+}
+
+void CheckIfElseOutputSSA(const std::shared_ptr<DataType>& type,
+                          const std::shared_ptr<Scalar>& cond,
+                          const std::shared_ptr<Scalar>& left, const std::string& right,
+                          const std::string& expected) {
+  const std::shared_ptr<Array>& right_ = ArrayFromJSON(type, right);
+  const std::shared_ptr<Array>& expected_ = ArrayFromJSON(type, expected);
+  CheckIfElseOutput(cond, left, right_, expected_);
+}
+
+class TestIfElseKernel : public ::testing::Test {};
+
+template <typename Type>
+class TestIfElsePrimitive : public ::testing::Test {};
+
+using PrimitiveTypes = ::testing::Types<Int8Type, UInt8Type, Int16Type, UInt16Type,
+                                        Int32Type, UInt32Type, Int64Type, UInt64Type,
+                                        FloatType, DoubleType, Date32Type, Date64Type>;
+
+TYPED_TEST_SUITE(TestIfElsePrimitive, PrimitiveTypes);
+
+TYPED_TEST(TestIfElsePrimitive, IfElseFixedSizeRand) {
+  using ArrayType = typename TypeTraits<TypeParam>::ArrayType;
+  auto type = TypeTraits<TypeParam>::type_singleton();
+
+  random::RandomArrayGenerator rand(/*seed=*/0);
+  int64_t len = 1000;
+  auto cond = std::static_pointer_cast<BooleanArray>(
+      rand.ArrayOf(boolean(), len, /*null_probability=*/0.01));
+  auto left = std::static_pointer_cast<ArrayType>(
+      rand.ArrayOf(type, len, /*null_probability=*/0.01));
+  auto right = std::static_pointer_cast<ArrayType>(
+      rand.ArrayOf(type, len, /*null_probability=*/0.01));
+
+  typename TypeTraits<TypeParam>::BuilderType builder;
+
+  for (int64_t i = 0; i < len; ++i) {
+    if (!cond->IsValid(i) || (cond->Value(i) && !left->IsValid(i)) ||
+        (!cond->Value(i) && !right->IsValid(i))) {
+      ASSERT_OK(builder.AppendNull());
+      continue;
+    }
+
+    if (cond->Value(i)) {
+      ASSERT_OK(builder.Append(left->Value(i)));
+    } else {
+      ASSERT_OK(builder.Append(right->Value(i)));
+    }
+  }
+  ASSERT_OK_AND_ASSIGN(auto expected_data, builder.Finish());
+
+  CheckIfElseOutput(cond, left, right, expected_data);
+}
+
+#define IF_ELSE_TEST_GEN(type, l0, l1, l2, l3, r0, r1, r2, r3, valid, valid1)            \

Review comment:
       `GenerateExpected` represents a reiteration of the function's logic. I'd prefer to see baseline assertions of correct behavior in the (array array array) case using CheckScalar() with explicit expected values; I think it'd be easier to read assertions like:
   
   ```c++
   CheckScalar("if_else", {boolean(), int32(), int32()}, {"[true, false, null]", "[1, 2, 3]", "[4, 5, 6]"}, int32(), "[1, 5, null]");
   ```
   
   Checking various permutations of input shapes can then be separate assertions or even cases.




-- 
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



[GitHub] [arrow] lidavidm commented on a change in pull request #10410: ARROW-10640: [C++] A "where" kernel to combine two arrays based on a mask

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_if_else_test.cc
##########
@@ -25,52 +25,51 @@ namespace arrow {
 namespace compute {
 
 void CheckIfElseOutputArray(const Datum& cond, const Datum& left, const Datum& right,
-                            const Datum& expected, bool all_valid = true) {
+                            const Datum& expected) {
   ASSERT_OK_AND_ASSIGN(Datum datum_out, IfElse(cond, left, right));
   std::shared_ptr<Array> result = datum_out.make_array();
   ASSERT_OK(result->ValidateFull());
+  std::shared_ptr<Array> expected_ = expected.make_array();
   AssertArraysEqual(*expected.make_array(), *result, /*verbose=*/true);
-  if (all_valid) {
-    // Check null count of ArrayData is set, not the computed Array.null_count
-    ASSERT_EQ(result->data()->null_count, 0);
-  }
+
+  ASSERT_EQ(result->data()->null_count, expected_->data()->null_count);

Review comment:
       Note AssertArraysEqual already performs this check for you.

##########
File path: cpp/src/arrow/compute/kernels/scalar_if_else.cc
##########
@@ -699,14 +699,58 @@ struct IfElseFunctor<Type, enable_if_boolean<Type>> {
   // AAS
   static Status Call(KernelContext* ctx, const ArrayData& cond, const ArrayData& left,
                      const Scalar& right, ArrayData* out) {
-    // todo impl
+    ARROW_RETURN_NOT_OK(PromoteNullsVisitor(ctx, cond, left, right, out));
+
+    // out_buff = left & cond
+    ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Buffer> out_buf,
+                          arrow::internal::BitmapAnd(
+                              ctx->memory_pool(), left.buffers[1]->data(), left.offset,
+                              cond.buffers[1]->data(), cond.offset, cond.length, 0));
+
+    bool right_data = internal::UnboxScalar<BooleanType>::Unbox(right);
+
+    // out_buff = left & cond | right & ~cond
+    if (right_data) {
+      arrow::internal::BitmapOrNot(out_buf->data(), 0, cond.buffers[1]->data(),
+                                   cond.offset, cond.length, 0, out_buf->mutable_data());
+    }
+
+    out->buffers[1] = std::move(out_buf);
     return Status::OK();
   }
 
   // ASS
   static Status Call(KernelContext* ctx, const ArrayData& cond, const Scalar& left,
                      const Scalar& right, ArrayData* out) {
-    // todo impl
+    ARROW_RETURN_NOT_OK(PromoteNullsVisitor(ctx, cond, left, right, out));
+
+    bool left_data = internal::UnboxScalar<BooleanType>::Unbox(left);
+    bool right_data = internal::UnboxScalar<BooleanType>::Unbox(right);
+
+    // out_buf = left & cond | right & ~cond
+    std::shared_ptr<Buffer> out_buf = nullptr;
+    if (left_data) {
+      if (right_data) {
+        // out_buf = ones
+        ARROW_ASSIGN_OR_RAISE(out_buf, ctx->AllocateBitmap(cond.length));
+        // filling with UINT8_MAX upto the buffer's size (in bytes)
+        arrow::compute::internal::SetMemory<UINT8_MAX>(out_buf.get());

Review comment:
       You could just inline the memset call here - no need to put a helper in a header IMO unless we're going to use it a lot. 




-- 
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



[GitHub] [arrow] lidavidm closed pull request #10410: ARROW-10640: [C++] A, "if_else" ("where") kernel to combine two arrays based on a mask

Posted by GitBox <gi...@apache.org>.
lidavidm closed pull request #10410:
URL: https://github.com/apache/arrow/pull/10410


   


-- 
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



[GitHub] [arrow] lidavidm commented on pull request #10410: ARROW-10640: [C++] A, "if_else" ("where") kernel to combine two arrays based on a mask

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


   It looks like there's an actual build failure on CentOS/GCC4.8 (expand "Dump install logs" to get the actual failure):
   
   <details>
   
   ```
   In file included from /tmp/Rtmpmpy1Y5/file7d77bb9a74/src/arrow/CMakeFiles/arrow_objlib.dir/Unity/unity_17_cxx.cxx:4:0:
   /arrow/cpp/src/arrow/compute/kernels/scalar_if_else.cc: In function ‘arrow::Status arrow::compute::{anonymous}::PromoteNullsVisitor(arrow::compute::KernelContext*, const arrow::Datum&, const arrow::Datum&, const arrow::Datum&, arrow::ArrayData*)’:
   /arrow/cpp/src/arrow/compute/kernels/scalar_if_else.cc:121:8: error: no matching function for call to ‘arrow::internal::Bitmap::VisitWords(<brace-enclosed initializer list>, arrow::compute::{anonymous}::PromoteNullsVisitor(arrow::compute::KernelContext*, const arrow::Datum&, const arrow::Datum&, const arrow::Datum&, arrow::ArrayData*)::__lambda44)’
          });
           ^
   /arrow/cpp/src/arrow/compute/kernels/scalar_if_else.cc:121:8: note: candidate is:
   In file included from /arrow/cpp/src/arrow/compute/kernels/scalar_if_else.cc:22:0,
                    from /tmp/Rtmpmpy1Y5/file7d77bb9a74/src/arrow/CMakeFiles/arrow_objlib.dir/Unity/unity_17_cxx.cxx:4:
   /arrow/cpp/src/arrow/util/bitmap.h:128:18: note: template<long unsigned int N, class Visitor, class Word> static int64_t arrow::internal::Bitmap::VisitWords(const arrow::internal::Bitmap (&)[N], Visitor&&)
      static int64_t VisitWords(const Bitmap (&bitmaps_arg)[N], Visitor&& visitor) {
                     ^
   /arrow/cpp/src/arrow/util/bitmap.h:128:18: note:   template argument deduction/substitution failed:
   In file included from /tmp/Rtmpmpy1Y5/file7d77bb9a74/src/arrow/CMakeFiles/arrow_objlib.dir/Unity/unity_17_cxx.cxx:4:0:
   /arrow/cpp/src/arrow/compute/kernels/scalar_if_else.cc:121:8: note:   couldn't deduce template parameter ‘N’
          });
           ^
   /arrow/cpp/src/arrow/compute/kernels/scalar_if_else.cc:127:12: error: no matching function for call to ‘arrow::internal::Bitmap::VisitWords(<brace-enclosed initializer list>, arrow::compute::{anonymous}::PromoteNullsVisitor(arrow::compute::KernelContext*, const arrow::Datum&, const arrow::Datum&, const arrow::Datum&, arrow::ArrayData*)::__lambda45)’
              });
               ^
   /arrow/cpp/src/arrow/compute/kernels/scalar_if_else.cc:127:12: note: candidate is:
   In file included from /arrow/cpp/src/arrow/compute/kernels/scalar_if_else.cc:22:0,
                    from /tmp/Rtmpmpy1Y5/file7d77bb9a74/src/arrow/CMakeFiles/arrow_objlib.dir/Unity/unity_17_cxx.cxx:4:
   /arrow/cpp/src/arrow/util/bitmap.h:128:18: note: template<long unsigned int N, class Visitor, class Word> static int64_t arrow::internal::Bitmap::VisitWords(const arrow::internal::Bitmap (&)[N], Visitor&&)
      static int64_t VisitWords(const Bitmap (&bitmaps_arg)[N], Visitor&& visitor) {
               ^
   /arrow/cpp/src/arrow/compute/kernels/scalar_if_else.cc:157:12: note: candidate is:
   In file included from /arrow/cpp/src/arrow/compute/kernels/scalar_if_else.cc:22:0,
                    from /tmp/Rtmpmpy1Y5/file7d77bb9a74/src/arrow/CMakeFiles/arrow_objlib.dir/Unity/unity_17_cxx.cxx:4:
   /arrow/cpp/src/arrow/util/bitmap.h:128:18: note: template<long unsigned int N, class Visitor, class Word> static int64_t arrow::internal::Bitmap::VisitWords(const arrow::internal::Bitmap (&)[N], Visitor&&)
      static int64_t VisitWords(const Bitmap (&bitmaps_arg)[N], Visitor&& visitor) {
                     ^
   /arrow/cpp/src/arrow/util/bitmap.h:128:18: note:   template argument deduction/substitution failed:
   In file included from /tmp/Rtmpmpy1Y5/file7d77bb9a74/src/arrow/CMakeFiles/arrow_objlib.dir/Unity/unity_17_cxx.cxx:4:0:
   /arrow/cpp/src/arrow/compute/kernels/scalar_if_else.cc:157:12: note:   couldn't deduce template parameter ‘N’
              });
               ^
   /arrow/cpp/src/arrow/compute/kernels/scalar_if_else.cc:164:12: error: no matching function for call to ‘arrow::internal::Bitmap::VisitWords(<brace-enclosed initializer list>, arrow::compute::{anonymous}::PromoteNullsVisitor(arrow::compute::KernelContext*, const arrow::Datum&, const arrow::Datum&, const arrow::Datum&, arrow::ArrayData*)::__lambda50)’
              });
               ^
   /arrow/cpp/src/arrow/compute/kernels/scalar_if_else.cc:164:12: note: candidate is:
   In file included from /arrow/cpp/src/arrow/compute/kernels/scalar_if_else.cc:22:0,
                    from /tmp/Rtmpmpy1Y5/file7d77bb9a74/src/arrow/CMakeFiles/arrow_objlib.dir/Unity/unity_17_cxx.cxx:4:
   /arrow/cpp/src/arrow/util/bitmap.h:128:18: note: template<long unsigned int N, class Visitor, class Word> static int64_t arrow::internal::Bitmap::VisitWords(const arrow::internal::Bitmap (&)[N], Visitor&&)
      static int64_t VisitWords(const Bitmap (&bitmaps_arg)[N], Visitor&& visitor) {
                     ^
   /arrow/cpp/src/arrow/util/bitmap.h:128:18: note:   template argument deduction/substitution failed:
   In file included from /tmp/Rtmpmpy1Y5/file7d77bb9a74/src/arrow/CMakeFiles/arrow_objlib.dir/Unity/unity_17_cxx.cxx:4:0:
   /arrow/cpp/src/arrow/compute/kernels/scalar_if_else.cc:164:12: note:   couldn't deduce template parameter ‘N’
              });
               ^
   /arrow/cpp/src/arrow/compute/kernels/scalar_if_else.cc:171:27: error: no matching function for call to ‘arrow::internal::Bitmap::VisitWords(<brace-enclosed initializer list>, arrow::compute::{anonymous}::PromoteNullsVisitor(arrow::compute::KernelContext*, const arrow::Datum&, const arrow::Datum&, const arrow::Datum&, arrow::ArrayData*)::__lambda51)’
                             });
                              ^
   /arrow/cpp/src/arrow/compute/kernels/scalar_if_else.cc:171:27: note: candidate is:
   In file included from /arrow/cpp/src/arrow/compute/kernels/scalar_if_else.cc:22:0,
                    from /tmp/Rtmpmpy1Y5/file7d77bb9a74/src/arrow/CMakeFiles/arrow_objlib.dir/Unity/unity_17_cxx.cxx:4:
   /arrow/cpp/src/arrow/util/bitmap.h:128:18: note: template<long unsigned int N, class Visitor, class Word> static int64_t arrow::internal::Bitmap::VisitWords(const arrow::internal::Bitmap (&)[N], Visitor&&)
      static int64_t VisitWords(const Bitmap (&bitmaps_arg)[N], Visitor&& visitor) {
                     ^
   /arrow/cpp/src/arrow/util/bitmap.h:128:18: note:   template argument deduction/substitution failed:
   In file included from /tmp/Rtmpmpy1Y5/file7d77bb9a74/src/arrow/CMakeFiles/arrow_objlib.dir/Unity/unity_17_cxx.cxx:4:0:
   /arrow/cpp/src/arrow/compute/kernels/scalar_if_else.cc:171:27: note:   couldn't deduce template parameter ‘N’
                             });
   ```
   
   </details>


-- 
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



[GitHub] [arrow] nirandaperera commented on pull request #10410: ARROW-10640: [C++] A, "if_else" ("where") kernel to combine two arrays based on a mask

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


   > It looks like there's an actual build failure on CentOS/GCC4.8 (expand "Dump install logs" to get the actual failure):
   > 
   > ```
   > In file included from /tmp/Rtmpmpy1Y5/file7d77bb9a74/src/arrow/CMakeFiles/arrow_objlib.dir/Unity/unity_17_cxx.cxx:4:0:
   > /arrow/cpp/src/arrow/compute/kernels/scalar_if_else.cc: In function ‘arrow::Status arrow::compute::{anonymous}::PromoteNullsVisitor(arrow::compute::KernelContext*, const arrow::Datum&, const arrow::Datum&, const arrow::Datum&, arrow::ArrayData*)’:
   > /arrow/cpp/src/arrow/compute/kernels/scalar_if_else.cc:121:8: error: no matching function for call to ‘arrow::internal::Bitmap::VisitWords(<brace-enclosed initializer list>, arrow::compute::{anonymous}::PromoteNullsVisitor(arrow::compute::KernelContext*, const arrow::Datum&, const arrow::Datum&, const arrow::Datum&, arrow::ArrayData*)::__lambda44)’
   >        });
   >         ^
   > /arrow/cpp/src/arrow/compute/kernels/scalar_if_else.cc:121:8: note: candidate is:
   > In file included from /arrow/cpp/src/arrow/compute/kernels/scalar_if_else.cc:22:0,
   >                  from /tmp/Rtmpmpy1Y5/file7d77bb9a74/src/arrow/CMakeFiles/arrow_objlib.dir/Unity/unity_17_cxx.cxx:4:
   > /arrow/cpp/src/arrow/util/bitmap.h:128:18: note: template<long unsigned int N, class Visitor, class Word> static int64_t arrow::internal::Bitmap::VisitWords(const arrow::internal::Bitmap (&)[N], Visitor&&)
   >    static int64_t VisitWords(const Bitmap (&bitmaps_arg)[N], Visitor&& visitor) {
   >                   ^
   > /arrow/cpp/src/arrow/util/bitmap.h:128:18: note:   template argument deduction/substitution failed:
   > In file included from /tmp/Rtmpmpy1Y5/file7d77bb9a74/src/arrow/CMakeFiles/arrow_objlib.dir/Unity/unity_17_cxx.cxx:4:0:
   > /arrow/cpp/src/arrow/compute/kernels/scalar_if_else.cc:121:8: note:   couldn't deduce template parameter ‘N’
   >        });
   >         ^
   > /arrow/cpp/src/arrow/compute/kernels/scalar_if_else.cc:127:12: error: no matching function for call to ‘arrow::internal::Bitmap::VisitWords(<brace-enclosed initializer list>, arrow::compute::{anonymous}::PromoteNullsVisitor(arrow::compute::KernelContext*, const arrow::Datum&, const arrow::Datum&, const arrow::Datum&, arrow::ArrayData*)::__lambda45)’
   >            });
   >             ^
   > /arrow/cpp/src/arrow/compute/kernels/scalar_if_else.cc:127:12: note: candidate is:
   > In file included from /arrow/cpp/src/arrow/compute/kernels/scalar_if_else.cc:22:0,
   >                  from /tmp/Rtmpmpy1Y5/file7d77bb9a74/src/arrow/CMakeFiles/arrow_objlib.dir/Unity/unity_17_cxx.cxx:4:
   > /arrow/cpp/src/arrow/util/bitmap.h:128:18: note: template<long unsigned int N, class Visitor, class Word> static int64_t arrow::internal::Bitmap::VisitWords(const arrow::internal::Bitmap (&)[N], Visitor&&)
   >    static int64_t VisitWords(const Bitmap (&bitmaps_arg)[N], Visitor&& visitor) {
   >             ^
   > /arrow/cpp/src/arrow/compute/kernels/scalar_if_else.cc:157:12: note: candidate is:
   > In file included from /arrow/cpp/src/arrow/compute/kernels/scalar_if_else.cc:22:0,
   >                  from /tmp/Rtmpmpy1Y5/file7d77bb9a74/src/arrow/CMakeFiles/arrow_objlib.dir/Unity/unity_17_cxx.cxx:4:
   > /arrow/cpp/src/arrow/util/bitmap.h:128:18: note: template<long unsigned int N, class Visitor, class Word> static int64_t arrow::internal::Bitmap::VisitWords(const arrow::internal::Bitmap (&)[N], Visitor&&)
   >    static int64_t VisitWords(const Bitmap (&bitmaps_arg)[N], Visitor&& visitor) {
   >                   ^
   > /arrow/cpp/src/arrow/util/bitmap.h:128:18: note:   template argument deduction/substitution failed:
   > In file included from /tmp/Rtmpmpy1Y5/file7d77bb9a74/src/arrow/CMakeFiles/arrow_objlib.dir/Unity/unity_17_cxx.cxx:4:0:
   > /arrow/cpp/src/arrow/compute/kernels/scalar_if_else.cc:157:12: note:   couldn't deduce template parameter ‘N’
   >            });
   >             ^
   > /arrow/cpp/src/arrow/compute/kernels/scalar_if_else.cc:164:12: error: no matching function for call to ‘arrow::internal::Bitmap::VisitWords(<brace-enclosed initializer list>, arrow::compute::{anonymous}::PromoteNullsVisitor(arrow::compute::KernelContext*, const arrow::Datum&, const arrow::Datum&, const arrow::Datum&, arrow::ArrayData*)::__lambda50)’
   >            });
   >             ^
   > /arrow/cpp/src/arrow/compute/kernels/scalar_if_else.cc:164:12: note: candidate is:
   > In file included from /arrow/cpp/src/arrow/compute/kernels/scalar_if_else.cc:22:0,
   >                  from /tmp/Rtmpmpy1Y5/file7d77bb9a74/src/arrow/CMakeFiles/arrow_objlib.dir/Unity/unity_17_cxx.cxx:4:
   > /arrow/cpp/src/arrow/util/bitmap.h:128:18: note: template<long unsigned int N, class Visitor, class Word> static int64_t arrow::internal::Bitmap::VisitWords(const arrow::internal::Bitmap (&)[N], Visitor&&)
   >    static int64_t VisitWords(const Bitmap (&bitmaps_arg)[N], Visitor&& visitor) {
   >                   ^
   > /arrow/cpp/src/arrow/util/bitmap.h:128:18: note:   template argument deduction/substitution failed:
   > In file included from /tmp/Rtmpmpy1Y5/file7d77bb9a74/src/arrow/CMakeFiles/arrow_objlib.dir/Unity/unity_17_cxx.cxx:4:0:
   > /arrow/cpp/src/arrow/compute/kernels/scalar_if_else.cc:164:12: note:   couldn't deduce template parameter ‘N’
   >            });
   >             ^
   > /arrow/cpp/src/arrow/compute/kernels/scalar_if_else.cc:171:27: error: no matching function for call to ‘arrow::internal::Bitmap::VisitWords(<brace-enclosed initializer list>, arrow::compute::{anonymous}::PromoteNullsVisitor(arrow::compute::KernelContext*, const arrow::Datum&, const arrow::Datum&, const arrow::Datum&, arrow::ArrayData*)::__lambda51)’
   >                           });
   >                            ^
   > /arrow/cpp/src/arrow/compute/kernels/scalar_if_else.cc:171:27: note: candidate is:
   > In file included from /arrow/cpp/src/arrow/compute/kernels/scalar_if_else.cc:22:0,
   >                  from /tmp/Rtmpmpy1Y5/file7d77bb9a74/src/arrow/CMakeFiles/arrow_objlib.dir/Unity/unity_17_cxx.cxx:4:
   > /arrow/cpp/src/arrow/util/bitmap.h:128:18: note: template<long unsigned int N, class Visitor, class Word> static int64_t arrow::internal::Bitmap::VisitWords(const arrow::internal::Bitmap (&)[N], Visitor&&)
   >    static int64_t VisitWords(const Bitmap (&bitmaps_arg)[N], Visitor&& visitor) {
   >                   ^
   > /arrow/cpp/src/arrow/util/bitmap.h:128:18: note:   template argument deduction/substitution failed:
   > In file included from /tmp/Rtmpmpy1Y5/file7d77bb9a74/src/arrow/CMakeFiles/arrow_objlib.dir/Unity/unity_17_cxx.cxx:4:0:
   > /arrow/cpp/src/arrow/compute/kernels/scalar_if_else.cc:171:27: note:   couldn't deduce template parameter ‘N’
   >                           });
   > ```
   
   Ah! thanks @lidavidm I missed this... it seems to be a gcc 4.x thing, isn't it. 
   
   BTW @bkietz should I add string type support to this as well, or can I leave it for later development?


-- 
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



[GitHub] [arrow] jorisvandenbossche commented on a change in pull request #10410: ARROW-10640: [C++] A, "if_else" ("where") kernel to combine two arrays based on a mask

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



##########
File path: cpp/src/arrow/compute/api_scalar.h
##########
@@ -462,5 +462,21 @@ ARROW_EXPORT
 Result<Datum> FillNull(const Datum& values, const Datum& fill_value,
                        ExecContext* ctx = NULLPTR);
 
+/// \brief IfElse returns elements chosen from `left` or `right`
+/// depending on `cond`. `null` values would be promoted to the result

Review comment:
       ```suggestion
   /// depending on `cond`. `null` values in `cond` will be promoted to the result
   ```




-- 
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



[GitHub] [arrow] github-actions[bot] commented on pull request #10410: ARROW-10640: [C++] A "where" kernel to combine two arrays based on a mask

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #10410:
URL: https://github.com/apache/arrow/pull/10410#issuecomment-849013467


   https://issues.apache.org/jira/browse/ARROW-10640


-- 
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



[GitHub] [arrow] bkietz commented on a change in pull request #10410: ARROW-10640: [C++] A "where" kernel to combine two arrays based on a mask

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_if_else.cc
##########
@@ -0,0 +1,742 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <arrow/compute/api.h>
+#include <arrow/compute/kernels/codegen_internal.h>
+#include <arrow/compute/util_internal.h>
+#include <arrow/util/bit_block_counter.h>
+#include <arrow/util/bitmap.h>
+#include <arrow/util/bitmap_ops.h>
+
+namespace arrow {
+using internal::BitBlockCount;
+using internal::BitBlockCounter;
+using internal::Bitmap;
+
+namespace compute {
+
+namespace {
+
+enum { COND_ALL_VALID = 1, LEFT_ALL_VALID = 2, RIGHT_ALL_VALID = 4 };
+
+// if the condition is null then output is null otherwise we take validity from the
+// selected argument
+// ie. cond.valid & (cond.data & left.valid | ~cond.data & right.valid)
+Status PromoteNullsVisitor(KernelContext* ctx, const ArrayData& cond, const Scalar& left,
+                           const Scalar& right, ArrayData* output) {
+  uint8_t flag = right.is_valid * RIGHT_ALL_VALID |
+                        left.is_valid * LEFT_ALL_VALID |
+                         !cond.MayHaveNulls() * COND_ALL_VALID;
+
+  if (flag < 6 && flag != 3) {
+    // there will be a validity buffer in the output
+    ARROW_ASSIGN_OR_RAISE(output->buffers[0], ctx->AllocateBitmap(cond.length));
+  }
+
+  // if the condition is null then output is null otherwise we take validity from the
+  // selected argument
+  // ie. cond.valid & (cond.data & left.valid | ~cond.data & right.valid)
+  switch (flag) {
+    case COND_ALL_VALID | LEFT_ALL_VALID | RIGHT_ALL_VALID:  // = 7
+      break;
+    case LEFT_ALL_VALID | RIGHT_ALL_VALID:  // = 6
+      // out_valid = c_valid
+      output->buffers[0] = SliceBuffer(cond.buffers[0], cond.offset, cond.length);
+      break;
+    case COND_ALL_VALID | RIGHT_ALL_VALID:  // = 5
+      // out_valid = ~cond.data
+      arrow::internal::InvertBitmap(cond.buffers[1]->data(), cond.offset, cond.length,
+                                    output->buffers[0]->mutable_data(), 0);
+      break;
+    case RIGHT_ALL_VALID:  // = 4
+      // out_valid = c_valid & ~cond.data
+      arrow::internal::BitmapAndNot(cond.buffers[0]->data(), cond.offset,
+                                    cond.buffers[1]->data(), cond.offset, cond.length, 0,
+                                    output->buffers[0]->mutable_data());
+      break;
+    case COND_ALL_VALID | LEFT_ALL_VALID:  // = 3
+      // out_valid = cond.data
+      output->buffers[0] = SliceBuffer(cond.buffers[1], cond.offset, cond.length);
+      break;
+    case LEFT_ALL_VALID:  // = 2
+      // out_valid = cond.valid & cond.data
+      arrow::internal::BitmapAnd(cond.buffers[0]->data(), cond.offset,
+                                 cond.buffers[1]->data(), cond.offset, cond.length, 0,
+                                 output->buffers[0]->mutable_data());
+      break;
+    case COND_ALL_VALID:  // = 1
+      // out_valid = 0 --> nothing to do; but requires out_valid to be a all-zero buffer
+      break;
+    case 0:
+      // out_valid = 0 --> nothing to do; but requires out_valid to be a all-zero buffer
+      break;
+  }
+  return Status::OK();
+}
+
+Status PromoteNullsVisitor(KernelContext* ctx, const ArrayData& cond,
+                           const ArrayData& left, const Scalar& right,
+                           ArrayData* output) {
+  uint8_t flag = right.is_valid * 4 + !left.MayHaveNulls() * 2 + !cond.MayHaveNulls();
+
+  enum { C_VALID, C_DATA, L_VALID };
+
+  Bitmap bitmaps[3];
+  bitmaps[C_VALID] = {cond.buffers[0], cond.offset, cond.length};
+  bitmaps[C_DATA] = {cond.buffers[1], cond.offset, cond.length};
+  bitmaps[L_VALID] = {left.buffers[0], left.offset, left.length};
+
+  uint64_t* out_validity = nullptr;
+  if (flag < 6 && flag != 3) {
+    // there will be a validity buffer in the output
+    ARROW_ASSIGN_OR_RAISE(output->buffers[0], ctx->AllocateBitmap(cond.length));
+    out_validity = output->GetMutableValues<uint64_t>(0);
+  }
+
+  // lambda function that will be used inside the visitor
+  int64_t i = 0;
+  auto apply = [&](uint64_t c_valid, uint64_t c_data, uint64_t l_valid,
+                   uint64_t r_valid) {
+    out_validity[i] = c_valid & ((c_data & l_valid) | (~c_data & r_valid));
+    i++;
+  };
+
+  // if the condition is null then output is null otherwise we take validity from the
+  // selected argument
+  // ie. cond.valid & (cond.data & left.valid | ~cond.data & right.valid)
+  switch (flag) {
+    case COND_ALL_VALID | LEFT_ALL_VALID | RIGHT_ALL_VALID:
+      break;
+    case LEFT_ALL_VALID | RIGHT_ALL_VALID:
+      output->buffers[0] = SliceBuffer(cond.buffers[0], cond.offset, cond.length);
+      break;
+    case COND_ALL_VALID | RIGHT_ALL_VALID:
+      // bitmaps[C_VALID] might be null; override to make it safe for Visit()
+      bitmaps[C_VALID] = bitmaps[C_DATA];
+      Bitmap::VisitWords(bitmaps, [&](std::array<uint64_t, 3> words) {
+        apply(UINT64_MAX, words[C_DATA], words[L_VALID], UINT64_MAX);
+      });
+      break;
+    case RIGHT_ALL_VALID:
+      Bitmap::VisitWords(bitmaps, [&](std::array<uint64_t, 3> words) {
+        apply(words[C_VALID], words[C_DATA], words[L_VALID], UINT64_MAX);
+      });
+      break;
+    case COND_ALL_VALID | LEFT_ALL_VALID:
+      // only cond.data is passed
+      output->buffers[0] = SliceBuffer(cond.buffers[1], cond.offset, cond.length);
+      break;
+    case LEFT_ALL_VALID:
+      // out_valid = cond.valid & cond.data
+      arrow::internal::BitmapAnd(cond.buffers[0]->data(), cond.offset,
+                                 cond.buffers[1]->data(), cond.offset, cond.length, 0,
+                                 output->buffers[0]->mutable_data());
+      break;
+    case COND_ALL_VALID:
+      // out_valid = cond.data & left.valid
+      arrow::internal::BitmapAnd(cond.buffers[1]->data(), cond.offset,
+                                 left.buffers[0]->data(), left.offset, cond.length, 0,
+                                 output->buffers[0]->mutable_data());
+      break;
+    case 0:
+      Bitmap::VisitWords(bitmaps, [&](std::array<uint64_t, 3> words) {
+        apply(words[C_VALID], words[C_DATA], words[L_VALID], 0);
+      });
+      break;
+  }
+  return Status::OK();
+}
+
+// if the condition is null then output is null otherwise we take validity from the
+// selected argument
+// ie. cond.valid & (cond.data & left.valid | ~cond.data & right.valid)
+Status PromoteNullsVisitor(KernelContext* ctx, const ArrayData& cond, const Scalar& left,
+                           const ArrayData& right, ArrayData* output) {
+  uint8_t flag = !right.MayHaveNulls() * 4 + left.is_valid * 2 + !cond.MayHaveNulls();
+
+  enum { C_VALID, C_DATA, R_VALID };
+
+  Bitmap bitmaps[3];
+  bitmaps[C_VALID] = {cond.buffers[0], cond.offset, cond.length};
+  bitmaps[C_DATA] = {cond.buffers[1], cond.offset, cond.length};
+  bitmaps[R_VALID] = {right.buffers[0], right.offset, right.length};
+
+  uint64_t* out_validity = nullptr;
+  if (flag < 6) {
+    // there will be a validity buffer in the output
+    ARROW_ASSIGN_OR_RAISE(output->buffers[0], ctx->AllocateBitmap(cond.length));
+    out_validity = output->GetMutableValues<uint64_t>(0);
+  }
+
+  // lambda function that will be used inside the visitor
+  int64_t i = 0;
+  auto apply = [&](uint64_t c_valid, uint64_t c_data, uint64_t l_valid,
+                   uint64_t r_valid) {
+    out_validity[i] = c_valid & ((c_data & l_valid) | (~c_data & r_valid));
+    i++;
+  };
+
+  // if the condition is null then output is null otherwise we take validity from the
+  // selected argument
+  // ie. cond.valid & (cond.data & left.valid | ~cond.data & right.valid)
+  switch (flag) {
+    case COND_ALL_VALID | LEFT_ALL_VALID | RIGHT_ALL_VALID:
+      break;
+    case LEFT_ALL_VALID | RIGHT_ALL_VALID:
+      output->buffers[0] = SliceBuffer(cond.buffers[0], cond.offset, cond.length);
+      break;
+    case COND_ALL_VALID | RIGHT_ALL_VALID:
+      // out_valid = ~cond.data
+      arrow::internal::InvertBitmap(cond.buffers[1]->data(), cond.offset, cond.length,
+                                    output->buffers[0]->mutable_data(), 0);
+      break;
+    case RIGHT_ALL_VALID:
+      // out_valid = c_valid & ~cond.data
+      arrow::internal::BitmapAndNot(cond.buffers[0]->data(), cond.offset,
+                                    cond.buffers[1]->data(), cond.offset, cond.length, 0,
+                                    output->buffers[0]->mutable_data());
+      break;
+    case COND_ALL_VALID | LEFT_ALL_VALID:
+      // bitmaps[C_VALID] might be null; override to make it safe for Visit()
+      bitmaps[C_VALID] = bitmaps[C_DATA];
+      Bitmap::VisitWords(bitmaps, [&](std::array<uint64_t, 3> words) {
+        apply(UINT64_MAX, words[C_DATA], UINT64_MAX, words[R_VALID]);
+      });
+      break;
+    case LEFT_ALL_VALID:
+      Bitmap::VisitWords(bitmaps, [&](std::array<uint64_t, 3> words) {
+        apply(words[C_VALID], words[C_DATA], UINT64_MAX, words[R_VALID]);
+      });
+      break;
+    case COND_ALL_VALID:
+      // out_valid =  ~cond.data & right.valid
+      arrow::internal::BitmapAndNot(right.buffers[0]->data(), right.offset,
+                                    cond.buffers[1]->data(), cond.offset, cond.length, 0,
+                                    output->buffers[0]->mutable_data());
+      break;
+    case 0:
+      Bitmap::VisitWords(bitmaps, [&](std::array<uint64_t, 3> words) {
+        apply(words[C_VALID], words[C_DATA], 0, words[R_VALID]);
+      });
+      break;
+  }
+  return Status::OK();
+}
+
+// if the condition is null then output is null otherwise we take validity from the
+// selected argument
+// ie. cond.valid & (cond.data & left.valid | ~cond.data & right.valid)
+Status PromoteNullsVisitor(KernelContext* ctx, const ArrayData& cond,
+                           const ArrayData& left, const ArrayData& right,
+                           ArrayData* output) {
+  uint8_t flag =
+      !right.MayHaveNulls() * 4 + !left.MayHaveNulls() * 2 + !cond.MayHaveNulls();
+
+  enum { C_VALID, C_DATA, L_VALID, R_VALID };
+
+  Bitmap bitmaps[4];
+  bitmaps[C_VALID] = {cond.buffers[0], cond.offset, cond.length};
+  bitmaps[C_DATA] = {cond.buffers[1], cond.offset, cond.length};
+  bitmaps[L_VALID] = {left.buffers[0], left.offset, left.length};
+  bitmaps[R_VALID] = {right.buffers[0], right.offset, right.length};
+
+  uint64_t* out_validity = nullptr;
+  if (flag < 6) {
+    // there will be a validity buffer in the output
+    ARROW_ASSIGN_OR_RAISE(output->buffers[0], ctx->AllocateBitmap(cond.length));
+    out_validity = output->GetMutableValues<uint64_t>(0);
+  }
+
+  // lambda function that will be used inside the visitor
+  int64_t i = 0;
+  auto apply = [&](uint64_t c_valid, uint64_t c_data, uint64_t l_valid,
+                   uint64_t r_valid) {
+    out_validity[i] = c_valid & ((c_data & l_valid) | (~c_data & r_valid));
+    i++;
+  };
+
+  // if the condition is null then output is null otherwise we take validity from the
+  // selected argument
+  // ie. cond.valid & (cond.data & left.valid | ~cond.data & right.valid)
+  switch (flag) {
+    case COND_ALL_VALID | LEFT_ALL_VALID | RIGHT_ALL_VALID:
+      break;
+    case LEFT_ALL_VALID | RIGHT_ALL_VALID:
+      output->buffers[0] = SliceBuffer(cond.buffers[0], cond.offset, cond.length);
+      break;
+    case COND_ALL_VALID | RIGHT_ALL_VALID:
+      // bitmaps[C_VALID], bitmaps[R_VALID] might be null; override to make it safe for
+      // Visit()
+      bitmaps[C_VALID] = bitmaps[C_DATA];
+      bitmaps[R_VALID] = bitmaps[C_DATA];
+      Bitmap::VisitWords(bitmaps, [&](std::array<uint64_t, 4> words) {
+        apply(UINT64_MAX, words[C_DATA], words[L_VALID], UINT64_MAX);
+      });
+      break;
+    case RIGHT_ALL_VALID:
+      // bitmaps[R_VALID] might be null; override to make it safe for Visit()
+      bitmaps[R_VALID] = bitmaps[C_DATA];
+      Bitmap::VisitWords(bitmaps, [&](std::array<uint64_t, 4> words) {
+        apply(words[C_VALID], words[C_DATA], words[L_VALID], UINT64_MAX);
+      });
+      break;
+    case COND_ALL_VALID | LEFT_ALL_VALID:
+      // bitmaps[C_VALID], bitmaps[L_VALID] might be null; override to make it safe for
+      // Visit()
+      bitmaps[C_VALID] = bitmaps[C_DATA];
+      bitmaps[L_VALID] = bitmaps[C_DATA];
+      Bitmap::VisitWords(bitmaps, [&](std::array<uint64_t, 4> words) {
+        apply(UINT64_MAX, words[C_DATA], UINT64_MAX, words[R_VALID]);
+      });
+      break;
+    case LEFT_ALL_VALID:
+      // bitmaps[L_VALID] might be null; override to make it safe for Visit()
+      bitmaps[L_VALID] = bitmaps[C_DATA];
+      Bitmap::VisitWords(bitmaps, [&](std::array<uint64_t, 4> words) {
+        apply(words[C_VALID], words[C_DATA], UINT64_MAX, words[R_VALID]);
+      });
+      break;
+    case COND_ALL_VALID:
+      // bitmaps[C_VALID] might be null; override to make it safe for Visit()
+      bitmaps[C_VALID] = bitmaps[C_DATA];
+      Bitmap::VisitWords(bitmaps, [&](std::array<uint64_t, 4> words) {
+        apply(UINT64_MAX, words[C_DATA], words[L_VALID], words[R_VALID]);
+      });
+      break;
+    case 0:
+      Bitmap::VisitWords(bitmaps, [&](std::array<uint64_t, 4> words) {
+        apply(words[C_VALID], words[C_DATA], words[L_VALID], words[R_VALID]);
+      });
+      break;
+  }
+  return Status::OK();
+}
+
+template <typename Type, typename Enable = void>
+struct IfElseFunctor {};
+
+// only number types needs to be handled for Fixed sized primitive data types because,
+// internal::GenerateTypeAgnosticPrimitive forwards types to the corresponding unsigned
+// int type
+template <typename Type>
+struct IfElseFunctor<Type, enable_if_number<Type>> {
+  using T = typename TypeTraits<Type>::CType;
+  // A - Array
+  // S - Scalar
+
+  //  AAA
+  static Status Call(KernelContext* ctx, const ArrayData& cond, const ArrayData& left,
+                     const ArrayData& right, ArrayData* out) {
+    ARROW_RETURN_NOT_OK(PromoteNullsVisitor(ctx, cond, left, right, out));
+
+    ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Buffer> out_buf,
+                          ctx->Allocate(cond.length * sizeof(T)));
+    T* out_values = reinterpret_cast<T*>(out_buf->mutable_data());
+
+    // copy right data to out_buff
+    const T* right_data = right.GetValues<T>(1);
+    std::memcpy(out_values, right_data, right.length * sizeof(T));
+
+    const auto* cond_data = cond.buffers[1]->data();  // this is a BoolArray
+    BitBlockCounter bit_counter(cond_data, cond.offset, cond.length);
+
+    // selectively copy values from left data
+    const T* left_data = left.GetValues<T>(1);
+    int64_t offset = cond.offset;
+
+    // todo this can be improved by intrinsics. ex: _mm*_mask_store_e* (vmovdqa*)
+    while (offset < cond.offset + cond.length) {
+      const BitBlockCount& block = bit_counter.NextWord();
+      if (block.AllSet()) {  // all from left
+        std::memcpy(out_values, left_data, block.length * sizeof(T));
+      } else if (block.popcount) {  // selectively copy from left
+        for (int64_t i = 0; i < block.length; ++i) {
+          if (BitUtil::GetBit(cond_data, offset + i)) {
+            out_values[i] = left_data[i];
+          }
+        }
+      }
+
+      offset += block.length;
+      out_values += block.length;
+      left_data += block.length;
+    }
+
+    out->buffers[1] = std::move(out_buf);
+    return Status::OK();
+  }
+
+  // ASA
+  static Status Call(KernelContext* ctx, const ArrayData& cond, const Scalar& left,
+                     const ArrayData& right, ArrayData* out) {
+    ARROW_RETURN_NOT_OK(PromoteNullsVisitor(ctx, cond, left, right, out));
+
+    ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Buffer> out_buf,
+                          ctx->Allocate(cond.length * sizeof(T)));
+    T* out_values = reinterpret_cast<T*>(out_buf->mutable_data());
+
+    // copy right data to out_buff
+    const T* right_data = right.GetValues<T>(1);
+    std::memcpy(out_values, right_data, right.length * sizeof(T));
+
+    const auto* cond_data = cond.buffers[1]->data();  // this is a BoolArray
+    BitBlockCounter bit_counter(cond_data, cond.offset, cond.length);
+
+    // selectively copy values from left data
+    T left_data = internal::UnboxScalar<Type>::Unbox(left);
+    int64_t offset = cond.offset;
+
+    // todo this can be improved by intrinsics. ex: _mm*_mask_store_e* (vmovdqa*)
+    while (offset < cond.offset + cond.length) {
+      const BitBlockCount& block = bit_counter.NextWord();
+      if (block.AllSet()) {  // all from left
+        std::fill(out_values, out_values + block.length, left_data);
+      } else if (block.popcount) {  // selectively copy from left
+        for (int64_t i = 0; i < block.length; ++i) {
+          if (BitUtil::GetBit(cond_data, offset + i)) {
+            out_values[i] = left_data;
+          }
+        }
+      }
+
+      offset += block.length;
+      out_values += block.length;
+    }
+
+    out->buffers[1] = std::move(out_buf);
+    return Status::OK();
+  }
+
+  // AAS
+  static Status Call(KernelContext* ctx, const ArrayData& cond, const ArrayData& left,
+                     const Scalar& right, ArrayData* out) {
+    ARROW_RETURN_NOT_OK(PromoteNullsVisitor(ctx, cond, left, right, out));
+
+    ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Buffer> out_buf,
+                          ctx->Allocate(cond.length * sizeof(T)));
+    T* out_values = reinterpret_cast<T*>(out_buf->mutable_data());
+
+    // copy left data to out_buff
+    const T* left_data = left.GetValues<T>(1);
+    std::memcpy(out_values, left_data, left.length * sizeof(T));
+
+    const auto* cond_data = cond.buffers[1]->data();  // this is a BoolArray
+    BitBlockCounter bit_counter(cond_data, cond.offset, cond.length);
+
+    // selectively copy values from left data
+    T right_data = internal::UnboxScalar<Type>::Unbox(right);
+    int64_t offset = cond.offset;
+
+    // todo this can be improved by intrinsics. ex: _mm*_mask_store_e* (vmovdqa*)
+    // left data is already in the output buffer. Therefore, mask needs to be inverted
+    while (offset < cond.offset + cond.length) {
+      const BitBlockCount& block = bit_counter.NextWord();
+      if (block.NoneSet()) {  // all from right
+        std::fill(out_values, out_values + block.length, right_data);
+      } else if (block.popcount) {  // selectively copy from right
+        for (int64_t i = 0; i < block.length; ++i) {
+          if (!BitUtil::GetBit(cond_data, offset + i)) {
+            out_values[i] = right_data;
+          }
+        }
+      }
+
+      offset += block.length;
+      out_values += block.length;
+    }
+
+    out->buffers[1] = std::move(out_buf);
+    return Status::OK();
+  }
+
+  // ASS
+  static Status Call(KernelContext* ctx, const ArrayData& cond, const Scalar& left,
+                     const Scalar& right, ArrayData* out) {
+    ARROW_RETURN_NOT_OK(PromoteNullsVisitor(ctx, cond, left, right, out));
+
+    ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Buffer> out_buf,
+                          ctx->Allocate(cond.length * sizeof(T)));
+    T* out_values = reinterpret_cast<T*>(out_buf->mutable_data());
+
+    // copy right data to out_buff
+    T right_data = internal::UnboxScalar<Type>::Unbox(right);
+    std::fill(out_values, out_values + cond.length, right_data);
+
+    const auto* cond_data = cond.buffers[1]->data();  // this is a BoolArray
+    BitBlockCounter bit_counter(cond_data, cond.offset, cond.length);
+
+    // selectively copy values from left data
+    T left_data = internal::UnboxScalar<Type>::Unbox(left);
+    int64_t offset = cond.offset;
+
+    // todo this can be improved by intrinsics. ex: _mm*_mask_store_e* (vmovdqa*)
+    while (offset < cond.offset + cond.length) {
+      const BitBlockCount& block = bit_counter.NextWord();
+      if (block.AllSet()) {  // all from left
+        std::fill(out_values, out_values + block.length, left_data);
+      } else if (block.popcount) {  // selectively copy from left
+        for (int64_t i = 0; i < block.length; ++i) {
+          if (BitUtil::GetBit(cond_data, offset + i)) {
+            out_values[i] = left_data;
+          }
+        }
+      }
+
+      offset += block.length;
+      out_values += block.length;
+    }
+
+    out->buffers[1] = std::move(out_buf);
+    return Status::OK();
+  }
+};
+
+template <typename Type>
+struct IfElseFunctor<Type, enable_if_boolean<Type>> {
+  // AAA
+  static Status Call(KernelContext* ctx, const ArrayData& cond, const ArrayData& left,
+                     const ArrayData& right, ArrayData* out) {
+    ARROW_RETURN_NOT_OK(PromoteNullsVisitor(ctx, cond, left, right, out));
+
+    // out_buff = right & ~cond
+    ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Buffer> out_buf,
+                          arrow::internal::BitmapAndNot(
+                              ctx->memory_pool(), right.buffers[1]->data(), right.offset,
+                              cond.buffers[1]->data(), cond.offset, cond.length, 0));
+
+    // out_buff = left & cond
+    ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Buffer> temp_buf,
+                          arrow::internal::BitmapAnd(
+                              ctx->memory_pool(), left.buffers[1]->data(), left.offset,
+                              cond.buffers[1]->data(), cond.offset, cond.length, 0));
+
+    arrow::internal::BitmapOr(out_buf->data(), 0, temp_buf->data(), 0, cond.length, 0,
+                              out_buf->mutable_data());
+    out->buffers[1] = std::move(out_buf);
+    return Status::OK();
+  }
+
+  // ASA
+  static Status Call(KernelContext* ctx, const ArrayData& cond, const Scalar& left,
+                     const ArrayData& right, ArrayData* out) {
+    ARROW_RETURN_NOT_OK(PromoteNullsVisitor(ctx, cond, left, right, out));
+
+    // out_buff = right & ~cond
+    ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Buffer> out_buf,
+                          arrow::internal::BitmapAndNot(
+                              ctx->memory_pool(), right.buffers[1]->data(), right.offset,
+                              cond.buffers[1]->data(), cond.offset, cond.length, 0));
+
+    // out_buff = left & cond
+    bool left_data = internal::UnboxScalar<BooleanType>::Unbox(left);
+    if (left_data) {
+      arrow::internal::BitmapOr(out_buf->data(), 0, cond.buffers[1]->data(), cond.offset,
+                                cond.length, 0, out_buf->mutable_data());
+    }
+
+    out->buffers[1] = std::move(out_buf);
+    return Status::OK();
+  }
+
+  // AAS
+  static Status Call(KernelContext* ctx, const ArrayData& cond, const ArrayData& left,
+                     const Scalar& right, ArrayData* out) {
+    ARROW_RETURN_NOT_OK(PromoteNullsVisitor(ctx, cond, left, right, out));
+
+    // out_buff = left & cond
+    ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Buffer> out_buf,
+                          arrow::internal::BitmapAnd(
+                              ctx->memory_pool(), left.buffers[1]->data(), left.offset,
+                              cond.buffers[1]->data(), cond.offset, cond.length, 0));
+
+    bool right_data = internal::UnboxScalar<BooleanType>::Unbox(right);
+
+    // out_buff = left & cond | right & ~cond
+    if (right_data) {
+      arrow::internal::BitmapOrNot(out_buf->data(), 0, cond.buffers[1]->data(),
+                                   cond.offset, cond.length, 0, out_buf->mutable_data());
+    }
+
+    out->buffers[1] = std::move(out_buf);
+    return Status::OK();
+  }
+
+  // ASS
+  static Status Call(KernelContext* ctx, const ArrayData& cond, const Scalar& left,
+                     const Scalar& right, ArrayData* out) {
+    ARROW_RETURN_NOT_OK(PromoteNullsVisitor(ctx, cond, left, right, out));
+
+    bool left_data = internal::UnboxScalar<BooleanType>::Unbox(left);
+    bool right_data = internal::UnboxScalar<BooleanType>::Unbox(right);
+
+    // out_buf = left & cond | right & ~cond
+    std::shared_ptr<Buffer> out_buf = nullptr;
+    if (left_data) {
+      if (right_data) {
+        // out_buf = ones
+        ARROW_ASSIGN_OR_RAISE(out_buf, ctx->AllocateBitmap(cond.length));
+        // filling with UINT8_MAX upto the buffer's size (in bytes)
+        arrow::compute::internal::SetMemory<UINT8_MAX>(out_buf.get());
+      } else {
+        // out_buf = cond
+        out_buf = SliceBuffer(cond.buffers[1], cond.offset, cond.length);
+      }
+    } else {
+      if (right_data) {
+        // out_buf = ~cond
+        ARROW_ASSIGN_OR_RAISE(out_buf, arrow::internal::InvertBitmap(
+                                           ctx->memory_pool(), cond.buffers[1]->data(),
+                                           cond.offset, cond.length))
+      } else {
+        // out_buf = zeros
+        ARROW_ASSIGN_OR_RAISE(out_buf, ctx->AllocateBitmap(cond.length));
+      }
+    }
+    out->buffers[1] = std::move(out_buf);
+    return Status::OK();
+  }
+};
+
+template <typename Type>
+struct IfElseFunctor<Type, enable_if_null<Type>> {
+  template <typename T>
+  static inline Status ReturnCopy(const T& in, T* out) {
+    // Nothing preallocated, so we assign in into the output
+    *out = in;
+    return Status::OK();
+  }
+
+  // AAA
+  static Status Call(KernelContext* ctx, const ArrayData& cond, const ArrayData& left,
+                     const ArrayData& right, ArrayData* out) {
+    return ReturnCopy(left, out);
+  }
+
+  // ASA
+  static Status Call(KernelContext* ctx, const ArrayData& cond, const Scalar& left,
+                     const ArrayData& right, ArrayData* out) {
+    return ReturnCopy(right, out);
+  }
+
+  // AAS
+  static Status Call(KernelContext* ctx, const ArrayData& cond, const ArrayData& left,
+                     const Scalar& right, ArrayData* out) {
+    return ReturnCopy(left, out);
+  }
+
+  // ASS
+  static Status Call(KernelContext* ctx, const ArrayData& cond, const Scalar& left,
+                     const Scalar& right, ArrayData* out) {
+    return ReturnCopy(cond, out);
+  }
+};
+
+template <typename Type>
+struct ResolveIfElseExec {
+  static Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    // cond is scalar
+    if (batch[0].is_scalar()) {
+      const auto& cond = batch[0].scalar_as<BooleanScalar>();
+      if (batch[1].is_scalar() && batch[2].is_scalar()) {
+        if (cond.is_valid) {
+          *out = cond.value ? batch[1].scalar() : batch[2].scalar();
+        } else {
+          *out = MakeNullScalar(batch[1].type());
+        }
+      } else {  // either left or right is an array. output is always an array
+        // output size is the size of the array arg
+        int64_t bcast_size = batch[1].is_array() ? batch[1].length() : batch[2].length();

Review comment:
       Correct, batch.lenth 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.

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



[GitHub] [arrow] bkietz commented on a change in pull request #10410: ARROW-10640: [C++] A "where" kernel to combine two arrays based on a mask

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_if_else.cc
##########
@@ -0,0 +1,740 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <arrow/compute/api.h>
+#include <arrow/compute/kernels/codegen_internal.h>
+#include <arrow/compute/util_internal.h>
+#include <arrow/util/bit_block_counter.h>
+#include <arrow/util/bitmap.h>
+#include <arrow/util/bitmap_ops.h>
+
+namespace arrow {
+using internal::BitBlockCount;
+using internal::BitBlockCounter;
+using internal::Bitmap;
+
+namespace compute {
+
+namespace {
+
+enum { COND_ALL_VALID = 1, LEFT_ALL_VALID = 2, RIGHT_ALL_VALID = 4 };
+
+// if the condition is null then output is null otherwise we take validity from the
+// selected argument
+// ie. cond.valid & (cond.data & left.valid | ~cond.data & right.valid)
+Status PromoteNullsVisitor(KernelContext* ctx, const ArrayData& cond, const Scalar& left,
+                           const Scalar& right, ArrayData* output) {
+  uint8_t flag = right.is_valid * 4 + left.is_valid * 2 + !cond.MayHaveNulls();
+
+  if (flag < 6 && flag != 3) {
+    // there will be a validity buffer in the output
+    ARROW_ASSIGN_OR_RAISE(output->buffers[0], ctx->AllocateBitmap(cond.length));
+  }
+
+  // if the condition is null then output is null otherwise we take validity from the
+  // selected argument
+  // ie. cond.valid & (cond.data & left.valid | ~cond.data & right.valid)
+  switch (flag) {
+    case COND_ALL_VALID | LEFT_ALL_VALID | RIGHT_ALL_VALID:  // = 7
+      break;
+    case LEFT_ALL_VALID | RIGHT_ALL_VALID:  // = 6
+      // out_valid = c_valid
+      output->buffers[0] = SliceBuffer(cond.buffers[0], cond.offset, cond.length);
+      break;
+    case COND_ALL_VALID | RIGHT_ALL_VALID:  // = 5
+      // out_valid = ~cond.data
+      arrow::internal::InvertBitmap(cond.buffers[1]->data(), cond.offset, cond.length,
+                                    output->buffers[0]->mutable_data(), 0);
+      break;
+    case RIGHT_ALL_VALID:  // = 4
+      // out_valid = c_valid & ~cond.data
+      arrow::internal::BitmapAndNot(cond.buffers[0]->data(), cond.offset,
+                                    cond.buffers[1]->data(), cond.offset, cond.length, 0,
+                                    output->buffers[0]->mutable_data());
+      break;
+    case COND_ALL_VALID | LEFT_ALL_VALID:  // = 3
+      // out_valid = cond.data
+      output->buffers[0] = SliceBuffer(cond.buffers[1], cond.offset, cond.length);
+      break;
+    case LEFT_ALL_VALID:  // = 2
+      // out_valid = cond.valid & cond.data
+      arrow::internal::BitmapAnd(cond.buffers[0]->data(), cond.offset,
+                                 cond.buffers[1]->data(), cond.offset, cond.length, 0,
+                                 output->buffers[0]->mutable_data());
+      break;
+    case COND_ALL_VALID:  // = 1
+      // out_valid = 0 --> nothing to do; but requires out_valid to be a all-zero buffer
+      break;
+    case 0:
+      // out_valid = 0 --> nothing to do; but requires out_valid to be a all-zero buffer
+      break;
+  }
+  return Status::OK();
+}
+
+Status PromoteNullsVisitor(KernelContext* ctx, const ArrayData& cond,
+                           const ArrayData& left, const Scalar& right,
+                           ArrayData* output) {
+  uint8_t flag = right.is_valid * 4 + !left.MayHaveNulls() * 2 + !cond.MayHaveNulls();
+
+  enum { C_VALID, C_DATA, L_VALID };
+
+  Bitmap bitmaps[3];
+  bitmaps[C_VALID] = {cond.buffers[0], cond.offset, cond.length};
+  bitmaps[C_DATA] = {cond.buffers[1], cond.offset, cond.length};
+  bitmaps[L_VALID] = {left.buffers[0], left.offset, left.length};
+
+  uint64_t* out_validity = nullptr;
+  if (flag < 6 && flag != 3) {
+    // there will be a validity buffer in the output
+    ARROW_ASSIGN_OR_RAISE(output->buffers[0], ctx->AllocateBitmap(cond.length));
+    out_validity = output->GetMutableValues<uint64_t>(0);
+  }
+
+  // lambda function that will be used inside the visitor
+  int64_t i = 0;
+  auto apply = [&](uint64_t c_valid, uint64_t c_data, uint64_t l_valid,
+                   uint64_t r_valid) {
+    out_validity[i] = c_valid & ((c_data & l_valid) | (~c_data & r_valid));
+    i++;
+  };
+
+  // if the condition is null then output is null otherwise we take validity from the
+  // selected argument
+  // ie. cond.valid & (cond.data & left.valid | ~cond.data & right.valid)
+  switch (flag) {
+    case COND_ALL_VALID | LEFT_ALL_VALID | RIGHT_ALL_VALID:
+      break;
+    case LEFT_ALL_VALID | RIGHT_ALL_VALID:
+      output->buffers[0] = SliceBuffer(cond.buffers[0], cond.offset, cond.length);
+      break;
+    case COND_ALL_VALID | RIGHT_ALL_VALID:
+      // bitmaps[C_VALID] might be null; override to make it safe for Visit()
+      bitmaps[C_VALID] = bitmaps[C_DATA];
+      Bitmap::VisitWords(bitmaps, [&](std::array<uint64_t, 3> words) {
+        apply(UINT64_MAX, words[C_DATA], words[L_VALID], UINT64_MAX);
+      });
+      break;
+    case RIGHT_ALL_VALID:
+      Bitmap::VisitWords(bitmaps, [&](std::array<uint64_t, 3> words) {
+        apply(words[C_VALID], words[C_DATA], words[L_VALID], UINT64_MAX);
+      });
+      break;
+    case COND_ALL_VALID | LEFT_ALL_VALID:
+      // only cond.data is passed
+      output->buffers[0] = SliceBuffer(cond.buffers[1], cond.offset, cond.length);
+      break;
+    case LEFT_ALL_VALID:
+      // out_valid = cond.valid & cond.data
+      arrow::internal::BitmapAnd(cond.buffers[0]->data(), cond.offset,
+                                 cond.buffers[1]->data(), cond.offset, cond.length, 0,
+                                 output->buffers[0]->mutable_data());
+      break;
+    case COND_ALL_VALID:
+      // out_valid = cond.data & left.valid
+      arrow::internal::BitmapAnd(cond.buffers[1]->data(), cond.offset,
+                                 left.buffers[0]->data(), left.offset, cond.length, 0,
+                                 output->buffers[0]->mutable_data());
+      break;
+    case 0:
+      Bitmap::VisitWords(bitmaps, [&](std::array<uint64_t, 3> words) {
+        apply(words[C_VALID], words[C_DATA], words[L_VALID], 0);
+      });
+      break;
+  }
+  return Status::OK();
+}
+
+// if the condition is null then output is null otherwise we take validity from the
+// selected argument
+// ie. cond.valid & (cond.data & left.valid | ~cond.data & right.valid)
+Status PromoteNullsVisitor(KernelContext* ctx, const ArrayData& cond, const Scalar& left,
+                           const ArrayData& right, ArrayData* output) {
+  uint8_t flag = !right.MayHaveNulls() * 4 + left.is_valid * 2 + !cond.MayHaveNulls();
+
+  enum { C_VALID, C_DATA, R_VALID };
+
+  Bitmap bitmaps[3];
+  bitmaps[C_VALID] = {cond.buffers[0], cond.offset, cond.length};
+  bitmaps[C_DATA] = {cond.buffers[1], cond.offset, cond.length};
+  bitmaps[R_VALID] = {right.buffers[0], right.offset, right.length};
+
+  uint64_t* out_validity = nullptr;
+  if (flag < 6) {
+    // there will be a validity buffer in the output
+    ARROW_ASSIGN_OR_RAISE(output->buffers[0], ctx->AllocateBitmap(cond.length));
+    out_validity = output->GetMutableValues<uint64_t>(0);
+  }
+
+  // lambda function that will be used inside the visitor
+  int64_t i = 0;
+  auto apply = [&](uint64_t c_valid, uint64_t c_data, uint64_t l_valid,
+                   uint64_t r_valid) {
+    out_validity[i] = c_valid & ((c_data & l_valid) | (~c_data & r_valid));
+    i++;
+  };
+
+  // if the condition is null then output is null otherwise we take validity from the
+  // selected argument
+  // ie. cond.valid & (cond.data & left.valid | ~cond.data & right.valid)
+  switch (flag) {
+    case COND_ALL_VALID | LEFT_ALL_VALID | RIGHT_ALL_VALID:
+      break;
+    case LEFT_ALL_VALID | RIGHT_ALL_VALID:
+      output->buffers[0] = SliceBuffer(cond.buffers[0], cond.offset, cond.length);
+      break;
+    case COND_ALL_VALID | RIGHT_ALL_VALID:
+      // out_valid = ~cond.data
+      arrow::internal::InvertBitmap(cond.buffers[1]->data(), cond.offset, cond.length,
+                                    output->buffers[0]->mutable_data(), 0);
+      break;
+    case RIGHT_ALL_VALID:
+      // out_valid = c_valid & ~cond.data
+      arrow::internal::BitmapAndNot(cond.buffers[0]->data(), cond.offset,
+                                    cond.buffers[1]->data(), cond.offset, cond.length, 0,
+                                    output->buffers[0]->mutable_data());
+      break;
+    case COND_ALL_VALID | LEFT_ALL_VALID:
+      // bitmaps[C_VALID] might be null; override to make it safe for Visit()
+      bitmaps[C_VALID] = bitmaps[C_DATA];
+      Bitmap::VisitWords(bitmaps, [&](std::array<uint64_t, 3> words) {
+        apply(UINT64_MAX, words[C_DATA], UINT64_MAX, words[R_VALID]);
+      });
+      break;
+    case LEFT_ALL_VALID:
+      Bitmap::VisitWords(bitmaps, [&](std::array<uint64_t, 3> words) {
+        apply(words[C_VALID], words[C_DATA], UINT64_MAX, words[R_VALID]);
+      });
+      break;
+    case COND_ALL_VALID:
+      // out_valid =  ~cond.data & right.valid
+      arrow::internal::BitmapAndNot(right.buffers[0]->data(), right.offset,
+                                    cond.buffers[1]->data(), cond.offset, cond.length, 0,
+                                    output->buffers[0]->mutable_data());
+      break;
+    case 0:
+      Bitmap::VisitWords(bitmaps, [&](std::array<uint64_t, 3> words) {
+        apply(words[C_VALID], words[C_DATA], 0, words[R_VALID]);
+      });
+      break;
+  }
+  return Status::OK();
+}
+
+// if the condition is null then output is null otherwise we take validity from the
+// selected argument
+// ie. cond.valid & (cond.data & left.valid | ~cond.data & right.valid)
+Status PromoteNullsVisitor(KernelContext* ctx, const ArrayData& cond,
+                           const ArrayData& left, const ArrayData& right,
+                           ArrayData* output) {
+  uint8_t flag =
+      !right.MayHaveNulls() * 4 + !left.MayHaveNulls() * 2 + !cond.MayHaveNulls();
+
+  enum { C_VALID, C_DATA, L_VALID, R_VALID };
+
+  Bitmap bitmaps[4];
+  bitmaps[C_VALID] = {cond.buffers[0], cond.offset, cond.length};
+  bitmaps[C_DATA] = {cond.buffers[1], cond.offset, cond.length};
+  bitmaps[L_VALID] = {left.buffers[0], left.offset, left.length};
+  bitmaps[R_VALID] = {right.buffers[0], right.offset, right.length};
+
+  uint64_t* out_validity = nullptr;
+  if (flag < 6) {
+    // there will be a validity buffer in the output
+    ARROW_ASSIGN_OR_RAISE(output->buffers[0], ctx->AllocateBitmap(cond.length));
+    out_validity = output->GetMutableValues<uint64_t>(0);
+  }
+
+  // lambda function that will be used inside the visitor
+  int64_t i = 0;
+  auto apply = [&](uint64_t c_valid, uint64_t c_data, uint64_t l_valid,
+                   uint64_t r_valid) {
+    out_validity[i] = c_valid & ((c_data & l_valid) | (~c_data & r_valid));
+    i++;
+  };
+
+  // if the condition is null then output is null otherwise we take validity from the
+  // selected argument
+  // ie. cond.valid & (cond.data & left.valid | ~cond.data & right.valid)
+  switch (flag) {
+    case COND_ALL_VALID | LEFT_ALL_VALID | RIGHT_ALL_VALID:
+      break;
+    case LEFT_ALL_VALID | RIGHT_ALL_VALID:
+      output->buffers[0] = SliceBuffer(cond.buffers[0], cond.offset, cond.length);
+      break;
+    case COND_ALL_VALID | RIGHT_ALL_VALID:
+      // bitmaps[C_VALID], bitmaps[R_VALID] might be null; override to make it safe for
+      // Visit()
+      bitmaps[C_VALID] = bitmaps[C_DATA];
+      bitmaps[R_VALID] = bitmaps[C_DATA];
+      Bitmap::VisitWords(bitmaps, [&](std::array<uint64_t, 4> words) {
+        apply(UINT64_MAX, words[C_DATA], words[L_VALID], UINT64_MAX);
+      });
+      break;
+    case RIGHT_ALL_VALID:
+      // bitmaps[R_VALID] might be null; override to make it safe for Visit()
+      bitmaps[R_VALID] = bitmaps[C_DATA];
+      Bitmap::VisitWords(bitmaps, [&](std::array<uint64_t, 4> words) {
+        apply(words[C_VALID], words[C_DATA], words[L_VALID], UINT64_MAX);
+      });
+      break;
+    case COND_ALL_VALID | LEFT_ALL_VALID:
+      // bitmaps[C_VALID], bitmaps[L_VALID] might be null; override to make it safe for
+      // Visit()
+      bitmaps[C_VALID] = bitmaps[C_DATA];
+      bitmaps[L_VALID] = bitmaps[C_DATA];
+      Bitmap::VisitWords(bitmaps, [&](std::array<uint64_t, 4> words) {
+        apply(UINT64_MAX, words[C_DATA], UINT64_MAX, words[R_VALID]);
+      });
+      break;
+    case LEFT_ALL_VALID:
+      // bitmaps[L_VALID] might be null; override to make it safe for Visit()
+      bitmaps[L_VALID] = bitmaps[C_DATA];
+      Bitmap::VisitWords(bitmaps, [&](std::array<uint64_t, 4> words) {
+        apply(words[C_VALID], words[C_DATA], UINT64_MAX, words[R_VALID]);
+      });
+      break;
+    case COND_ALL_VALID:
+      // bitmaps[C_VALID] might be null; override to make it safe for Visit()
+      bitmaps[C_VALID] = bitmaps[C_DATA];
+      Bitmap::VisitWords(bitmaps, [&](std::array<uint64_t, 4> words) {
+        apply(UINT64_MAX, words[C_DATA], words[L_VALID], words[R_VALID]);
+      });
+      break;
+    case 0:
+      Bitmap::VisitWords(bitmaps, [&](std::array<uint64_t, 4> words) {
+        apply(words[C_VALID], words[C_DATA], words[L_VALID], words[R_VALID]);
+      });
+      break;
+  }
+  return Status::OK();
+}
+
+template <typename Type, typename Enable = void>
+struct IfElseFunctor {};
+
+// only number types needs to be handled for Fixed sized primitive data types because,
+// internal::GenerateTypeAgnosticPrimitive forwards types to the corresponding unsigned
+// int type
+template <typename Type>
+struct IfElseFunctor<Type, enable_if_number<Type>> {
+  using T = typename TypeTraits<Type>::CType;
+  // A - Array
+  // S - Scalar
+
+  //  AAA
+  static Status Call(KernelContext* ctx, const ArrayData& cond, const ArrayData& left,
+                     const ArrayData& right, ArrayData* out) {
+    ARROW_RETURN_NOT_OK(PromoteNullsVisitor(ctx, cond, left, right, out));
+
+    ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Buffer> out_buf,
+                          ctx->Allocate(cond.length * sizeof(T)));
+    T* out_values = reinterpret_cast<T*>(out_buf->mutable_data());
+
+    // copy right data to out_buff
+    const T* right_data = right.GetValues<T>(1);
+    std::memcpy(out_values, right_data, right.length * sizeof(T));
+
+    const auto* cond_data = cond.buffers[1]->data();  // this is a BoolArray
+    BitBlockCounter bit_counter(cond_data, cond.offset, cond.length);
+
+    // selectively copy values from left data
+    const T* left_data = left.GetValues<T>(1);
+    int64_t offset = cond.offset;
+
+    // todo this can be improved by intrinsics. ex: _mm*_mask_store_e* (vmovdqa*)
+    while (offset < cond.offset + cond.length) {
+      const BitBlockCount& block = bit_counter.NextWord();
+      if (block.AllSet()) {  // all from left
+        std::memcpy(out_values, left_data, block.length * sizeof(T));
+      } else if (block.popcount) {  // selectively copy from left
+        for (int64_t i = 0; i < block.length; ++i) {
+          if (BitUtil::GetBit(cond_data, offset + i)) {
+            out_values[i] = left_data[i];
+          }
+        }
+      }
+
+      offset += block.length;
+      out_values += block.length;
+      left_data += block.length;
+    }
+
+    out->buffers[1] = std::move(out_buf);
+    return Status::OK();
+  }
+
+  // ASA
+  static Status Call(KernelContext* ctx, const ArrayData& cond, const Scalar& left,
+                     const ArrayData& right, ArrayData* out) {
+    ARROW_RETURN_NOT_OK(PromoteNullsVisitor(ctx, cond, left, right, out));
+
+    ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Buffer> out_buf,
+                          ctx->Allocate(cond.length * sizeof(T)));
+    T* out_values = reinterpret_cast<T*>(out_buf->mutable_data());
+
+    // copy right data to out_buff
+    const T* right_data = right.GetValues<T>(1);
+    std::memcpy(out_values, right_data, right.length * sizeof(T));
+
+    const auto* cond_data = cond.buffers[1]->data();  // this is a BoolArray
+    BitBlockCounter bit_counter(cond_data, cond.offset, cond.length);
+
+    // selectively copy values from left data
+    T left_data = internal::UnboxScalar<Type>::Unbox(left);
+    int64_t offset = cond.offset;
+
+    // todo this can be improved by intrinsics. ex: _mm*_mask_store_e* (vmovdqa*)
+    while (offset < cond.offset + cond.length) {
+      const BitBlockCount& block = bit_counter.NextWord();
+      if (block.AllSet()) {  // all from left
+        std::fill(out_values, out_values + block.length, left_data);
+      } else if (block.popcount) {  // selectively copy from left
+        for (int64_t i = 0; i < block.length; ++i) {
+          if (BitUtil::GetBit(cond_data, offset + i)) {
+            out_values[i] = left_data;
+          }
+        }
+      }
+
+      offset += block.length;
+      out_values += block.length;
+    }
+
+    out->buffers[1] = std::move(out_buf);
+    return Status::OK();
+  }
+
+  // AAS
+  static Status Call(KernelContext* ctx, const ArrayData& cond, const ArrayData& left,
+                     const Scalar& right, ArrayData* out) {
+    ARROW_RETURN_NOT_OK(PromoteNullsVisitor(ctx, cond, left, right, out));
+
+    ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Buffer> out_buf,
+                          ctx->Allocate(cond.length * sizeof(T)));
+    T* out_values = reinterpret_cast<T*>(out_buf->mutable_data());
+
+    // copy left data to out_buff
+    const T* left_data = left.GetValues<T>(1);
+    std::memcpy(out_values, left_data, left.length * sizeof(T));
+
+    const auto* cond_data = cond.buffers[1]->data();  // this is a BoolArray
+    BitBlockCounter bit_counter(cond_data, cond.offset, cond.length);
+
+    // selectively copy values from left data
+    T right_data = internal::UnboxScalar<Type>::Unbox(right);
+    int64_t offset = cond.offset;
+
+    // todo this can be improved by intrinsics. ex: _mm*_mask_store_e* (vmovdqa*)
+    // left data is already in the output buffer. Therefore, mask needs to be inverted
+    while (offset < cond.offset + cond.length) {
+      const BitBlockCount& block = bit_counter.NextWord();
+      if (block.NoneSet()) {  // all from right
+        std::fill(out_values, out_values + block.length, right_data);
+      } else if (block.popcount) {  // selectively copy from right
+        for (int64_t i = 0; i < block.length; ++i) {
+          if (!BitUtil::GetBit(cond_data, offset + i)) {
+            out_values[i] = right_data;
+          }
+        }
+      }
+
+      offset += block.length;
+      out_values += block.length;
+    }
+
+    out->buffers[1] = std::move(out_buf);
+    return Status::OK();
+  }
+
+  // ASS
+  static Status Call(KernelContext* ctx, const ArrayData& cond, const Scalar& left,
+                     const Scalar& right, ArrayData* out) {
+    ARROW_RETURN_NOT_OK(PromoteNullsVisitor(ctx, cond, left, right, out));
+
+    ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Buffer> out_buf,
+                          ctx->Allocate(cond.length * sizeof(T)));
+    T* out_values = reinterpret_cast<T*>(out_buf->mutable_data());
+
+    // copy right data to out_buff
+    T right_data = internal::UnboxScalar<Type>::Unbox(right);
+    std::fill(out_values, out_values + cond.length, right_data);
+
+    const auto* cond_data = cond.buffers[1]->data();  // this is a BoolArray
+    BitBlockCounter bit_counter(cond_data, cond.offset, cond.length);
+
+    // selectively copy values from left data
+    T left_data = internal::UnboxScalar<Type>::Unbox(left);
+    int64_t offset = cond.offset;
+
+    // todo this can be improved by intrinsics. ex: _mm*_mask_store_e* (vmovdqa*)
+    while (offset < cond.offset + cond.length) {
+      const BitBlockCount& block = bit_counter.NextWord();
+      if (block.AllSet()) {  // all from left
+        std::fill(out_values, out_values + block.length, left_data);
+      } else if (block.popcount) {  // selectively copy from left
+        for (int64_t i = 0; i < block.length; ++i) {
+          if (BitUtil::GetBit(cond_data, offset + i)) {
+            out_values[i] = left_data;
+          }
+        }
+      }
+
+      offset += block.length;
+      out_values += block.length;
+    }
+
+    out->buffers[1] = std::move(out_buf);
+    return Status::OK();
+  }
+};
+
+template <typename Type>
+struct IfElseFunctor<Type, enable_if_boolean<Type>> {
+  // AAA
+  static Status Call(KernelContext* ctx, const ArrayData& cond, const ArrayData& left,
+                     const ArrayData& right, ArrayData* out) {
+    ARROW_RETURN_NOT_OK(PromoteNullsVisitor(ctx, cond, left, right, out));
+
+    // out_buff = right & ~cond
+    ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Buffer> out_buf,
+                          arrow::internal::BitmapAndNot(
+                              ctx->memory_pool(), right.buffers[1]->data(), right.offset,
+                              cond.buffers[1]->data(), cond.offset, cond.length, 0));
+
+    // out_buff = left & cond
+    ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Buffer> temp_buf,
+                          arrow::internal::BitmapAnd(
+                              ctx->memory_pool(), left.buffers[1]->data(), left.offset,
+                              cond.buffers[1]->data(), cond.offset, cond.length, 0));
+
+    arrow::internal::BitmapOr(out_buf->data(), 0, temp_buf->data(), 0, cond.length, 0,
+                              out_buf->mutable_data());
+    out->buffers[1] = std::move(out_buf);
+    return Status::OK();
+  }
+
+  // ASA
+  static Status Call(KernelContext* ctx, const ArrayData& cond, const Scalar& left,
+                     const ArrayData& right, ArrayData* out) {
+    ARROW_RETURN_NOT_OK(PromoteNullsVisitor(ctx, cond, left, right, out));
+
+    // out_buff = right & ~cond
+    ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Buffer> out_buf,
+                          arrow::internal::BitmapAndNot(
+                              ctx->memory_pool(), right.buffers[1]->data(), right.offset,
+                              cond.buffers[1]->data(), cond.offset, cond.length, 0));
+
+    // out_buff = left & cond
+    bool left_data = internal::UnboxScalar<BooleanType>::Unbox(left);
+    if (left_data) {
+      arrow::internal::BitmapOr(out_buf->data(), 0, cond.buffers[1]->data(), cond.offset,
+                                cond.length, 0, out_buf->mutable_data());
+    }
+
+    out->buffers[1] = std::move(out_buf);
+    return Status::OK();
+  }
+
+  // AAS
+  static Status Call(KernelContext* ctx, const ArrayData& cond, const ArrayData& left,
+                     const Scalar& right, ArrayData* out) {
+    ARROW_RETURN_NOT_OK(PromoteNullsVisitor(ctx, cond, left, right, out));
+
+    // out_buff = left & cond
+    ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Buffer> out_buf,
+                          arrow::internal::BitmapAnd(
+                              ctx->memory_pool(), left.buffers[1]->data(), left.offset,
+                              cond.buffers[1]->data(), cond.offset, cond.length, 0));
+
+    bool right_data = internal::UnboxScalar<BooleanType>::Unbox(right);
+
+    // out_buff = left & cond | right & ~cond
+    if (right_data) {
+      arrow::internal::BitmapOrNot(out_buf->data(), 0, cond.buffers[1]->data(),
+                                   cond.offset, cond.length, 0, out_buf->mutable_data());
+    }
+
+    out->buffers[1] = std::move(out_buf);
+    return Status::OK();
+  }
+
+  // ASS
+  static Status Call(KernelContext* ctx, const ArrayData& cond, const Scalar& left,
+                     const Scalar& right, ArrayData* out) {
+    ARROW_RETURN_NOT_OK(PromoteNullsVisitor(ctx, cond, left, right, out));
+
+    bool left_data = internal::UnboxScalar<BooleanType>::Unbox(left);
+    bool right_data = internal::UnboxScalar<BooleanType>::Unbox(right);
+
+    // out_buf = left & cond | right & ~cond
+    std::shared_ptr<Buffer> out_buf = nullptr;
+    if (left_data) {
+      if (right_data) {
+        // out_buf = ones
+        ARROW_ASSIGN_OR_RAISE(out_buf, ctx->AllocateBitmap(cond.length));
+        // filling with UINT8_MAX upto the buffer's size (in bytes)
+        arrow::compute::internal::SetMemory<UINT8_MAX>(out_buf.get());
+      } else {
+        // out_buf = cond
+        out_buf = SliceBuffer(cond.buffers[1], cond.offset, cond.length);
+      }
+    } else {
+      if (right_data) {
+        // out_buf = ~cond
+        ARROW_ASSIGN_OR_RAISE(out_buf, arrow::internal::InvertBitmap(
+                                           ctx->memory_pool(), cond.buffers[1]->data(),
+                                           cond.offset, cond.length))
+      } else {
+        // out_buf = zeros
+        ARROW_ASSIGN_OR_RAISE(out_buf, ctx->AllocateBitmap(cond.length));
+      }
+    }
+    out->buffers[1] = std::move(out_buf);
+    return Status::OK();
+  }
+};
+
+template <typename Type>
+struct IfElseFunctor<Type, enable_if_null<Type>> {
+  template <typename T>
+  static inline Status ReturnCopy(const T& in, T* out) {
+    // Nothing preallocated, so we assign in into the output
+    *out = in;
+    return Status::OK();
+  }
+
+  // AAA
+  static Status Call(KernelContext* ctx, const ArrayData& cond, const ArrayData& left,
+                     const ArrayData& right, ArrayData* out) {
+    return ReturnCopy(left, out);
+  }
+
+  // ASA
+  static Status Call(KernelContext* ctx, const ArrayData& cond, const Scalar& left,
+                     const ArrayData& right, ArrayData* out) {
+    return ReturnCopy(right, out);
+  }
+
+  // AAS
+  static Status Call(KernelContext* ctx, const ArrayData& cond, const ArrayData& left,
+                     const Scalar& right, ArrayData* out) {
+    return ReturnCopy(left, out);
+  }
+
+  // ASS
+  static Status Call(KernelContext* ctx, const ArrayData& cond, const Scalar& left,
+                     const Scalar& right, ArrayData* out) {
+    return ReturnCopy(cond, out);
+  }
+};
+
+template <typename Type>
+struct ResolveIfElseExec {
+  static Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    // cond is scalar
+    if (batch[0].is_scalar()) {
+      const auto& cond = batch[0].scalar_as<BooleanScalar>();
+      if (batch[1].is_scalar() && batch[2].is_scalar()) {
+        if (cond.is_valid) {
+          *out = cond.value ? batch[1].scalar() : batch[2].scalar();
+        } else {
+          *out = MakeNullScalar(batch[1].type());
+        }
+      } else {  // either left or right is an array. output is always an array

Review comment:
       nit: prefer returning early over `else` blocks
   ```suggestion
           return Status::OK();
         }
         // either left or right is an array. output is always an array
   ```




-- 
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



[GitHub] [arrow] lidavidm commented on a change in pull request #10410: ARROW-10640: [C++] A "where" kernel to combine two arrays based on a mask

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_if_else.cc
##########
@@ -0,0 +1,572 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <arrow/compute/api.h>
+#include <arrow/compute/kernels/codegen_internal.h>
+#include <arrow/compute/util_internal.h>
+#include <arrow/util/bit_block_counter.h>
+#include <arrow/util/bitmap.h>
+#include <arrow/util/bitmap_ops.h>
+
+namespace arrow {
+using internal::BitBlockCount;
+using internal::BitBlockCounter;
+using internal::Bitmap;
+
+namespace compute {
+
+namespace {
+
+util::optional<uint64_t> GetConstantValidityWord(const Datum& data) {
+  if (data.is_scalar()) {
+    return data.scalar()->is_valid ? ~uint64_t(0) : uint64_t(0);
+  }
+
+  if (data.array()->null_count == data.array()->length) return uint64_t(0);
+
+  if (!data.array()->MayHaveNulls()) return ~uint64_t(0);
+
+  // no constant validity word available
+  return {};
+}
+
+inline Bitmap GetBitmap(const Datum& datum, int i) {
+  if (datum.is_scalar()) return {};
+  const ArrayData& a = *datum.array();
+  return Bitmap{a.buffers[i], a.offset, a.length};
+}
+
+// if the condition is null then output is null otherwise we take validity from the
+// selected argument
+// ie. cond.valid & (cond.data & left.valid | ~cond.data & right.valid)
+Status PromoteNullsVisitor(KernelContext* ctx, const Datum& cond_d, const Datum& left_d,
+                           const Datum& right_d, ArrayData* output) {
+  auto cond_const = GetConstantValidityWord(cond_d);
+  auto left_const = GetConstantValidityWord(left_d);
+  auto right_const = GetConstantValidityWord(right_d);
+
+  enum { COND_CONST = 1, LEFT_CONST = 2, RIGHT_CONST = 4 };
+  auto flag = COND_CONST * cond_const.has_value() | LEFT_CONST * left_const.has_value() |
+              RIGHT_CONST * right_const.has_value();
+
+  const ArrayData& cond = *cond_d.array();
+  // cond.data will always be available
+  Bitmap cond_data{cond.buffers[1], cond.offset, cond.length};
+  Bitmap cond_valid{cond.buffers[0], cond.offset, cond.length};
+  Bitmap left_valid = GetBitmap(left_d, 0);
+  Bitmap right_valid = GetBitmap(right_d, 0);
+  // sometimes Bitmaps will be ignored, in which case we replace access to them with
+  // duplicated (probably elided) access to cond_data
+  const Bitmap& _ = cond_data;
+
+  // lambda function that will be used inside the visitor
+  uint64_t* out_validity = nullptr;
+  int64_t i = 0;
+  auto apply = [&](uint64_t c_valid, uint64_t c_data, uint64_t l_valid,
+                   uint64_t r_valid) {
+    out_validity[i] = c_valid & ((c_data & l_valid) | (~c_data & r_valid));
+    i++;
+  };
+
+  // if the condition is null then output is null otherwise we take validity from the
+  // selected argument
+  // ie. cond.valid & (cond.data & left.valid | ~cond.data & right.valid)
+
+  // In the following cases, we dont need to allocate out_valid bitmap
+  switch (flag) {
+    case COND_CONST | LEFT_CONST | RIGHT_CONST:
+      // if cond & left & right all ones, then output is all valid --> out_valid = nullptr
+      if ((*cond_const & *left_const & *right_const) == UINT64_MAX) {
+        return Status::OK();
+      }
+    case LEFT_CONST | RIGHT_CONST:
+      // if both left and right are valid, no need to calculate out_valid bitmap. Pass
+      // cond validity buffer
+      if ((*left_const & *right_const) == UINT64_MAX) {
+        output->buffers[0] = SliceBuffer(cond.buffers[0], cond.offset, cond.length);

Review comment:
       I don't think this does what you want: cond.offset/cond.length are bitwise here but SliceBuffer is bytewise. You probably do need to allocate a validity bitmap and copy it using CopyBitmap.

##########
File path: cpp/src/arrow/compute/kernels/scalar_if_else.cc
##########
@@ -0,0 +1,572 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <arrow/compute/api.h>
+#include <arrow/compute/kernels/codegen_internal.h>
+#include <arrow/compute/util_internal.h>
+#include <arrow/util/bit_block_counter.h>
+#include <arrow/util/bitmap.h>
+#include <arrow/util/bitmap_ops.h>
+
+namespace arrow {
+using internal::BitBlockCount;
+using internal::BitBlockCounter;
+using internal::Bitmap;
+
+namespace compute {
+
+namespace {
+
+util::optional<uint64_t> GetConstantValidityWord(const Datum& data) {
+  if (data.is_scalar()) {
+    return data.scalar()->is_valid ? ~uint64_t(0) : uint64_t(0);
+  }
+
+  if (data.array()->null_count == data.array()->length) return uint64_t(0);
+
+  if (!data.array()->MayHaveNulls()) return ~uint64_t(0);
+
+  // no constant validity word available
+  return {};
+}
+
+inline Bitmap GetBitmap(const Datum& datum, int i) {
+  if (datum.is_scalar()) return {};
+  const ArrayData& a = *datum.array();
+  return Bitmap{a.buffers[i], a.offset, a.length};
+}
+
+// if the condition is null then output is null otherwise we take validity from the
+// selected argument
+// ie. cond.valid & (cond.data & left.valid | ~cond.data & right.valid)
+Status PromoteNullsVisitor(KernelContext* ctx, const Datum& cond_d, const Datum& left_d,
+                           const Datum& right_d, ArrayData* output) {
+  auto cond_const = GetConstantValidityWord(cond_d);
+  auto left_const = GetConstantValidityWord(left_d);
+  auto right_const = GetConstantValidityWord(right_d);
+
+  enum { COND_CONST = 1, LEFT_CONST = 2, RIGHT_CONST = 4 };
+  auto flag = COND_CONST * cond_const.has_value() | LEFT_CONST * left_const.has_value() |
+              RIGHT_CONST * right_const.has_value();
+
+  const ArrayData& cond = *cond_d.array();
+  // cond.data will always be available
+  Bitmap cond_data{cond.buffers[1], cond.offset, cond.length};
+  Bitmap cond_valid{cond.buffers[0], cond.offset, cond.length};
+  Bitmap left_valid = GetBitmap(left_d, 0);
+  Bitmap right_valid = GetBitmap(right_d, 0);
+  // sometimes Bitmaps will be ignored, in which case we replace access to them with
+  // duplicated (probably elided) access to cond_data
+  const Bitmap& _ = cond_data;
+
+  // lambda function that will be used inside the visitor
+  uint64_t* out_validity = nullptr;
+  int64_t i = 0;
+  auto apply = [&](uint64_t c_valid, uint64_t c_data, uint64_t l_valid,
+                   uint64_t r_valid) {
+    out_validity[i] = c_valid & ((c_data & l_valid) | (~c_data & r_valid));
+    i++;
+  };
+
+  // if the condition is null then output is null otherwise we take validity from the
+  // selected argument
+  // ie. cond.valid & (cond.data & left.valid | ~cond.data & right.valid)
+
+  // In the following cases, we dont need to allocate out_valid bitmap
+  switch (flag) {
+    case COND_CONST | LEFT_CONST | RIGHT_CONST:
+      // if cond & left & right all ones, then output is all valid --> out_valid = nullptr

Review comment:
       Is this true? It could be that they're all ones, or all zeroes.

##########
File path: cpp/src/arrow/compute/kernels/scalar_if_else.cc
##########
@@ -0,0 +1,572 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <arrow/compute/api.h>
+#include <arrow/compute/kernels/codegen_internal.h>
+#include <arrow/compute/util_internal.h>
+#include <arrow/util/bit_block_counter.h>
+#include <arrow/util/bitmap.h>
+#include <arrow/util/bitmap_ops.h>
+
+namespace arrow {
+using internal::BitBlockCount;
+using internal::BitBlockCounter;
+using internal::Bitmap;
+
+namespace compute {
+
+namespace {
+
+util::optional<uint64_t> GetConstantValidityWord(const Datum& data) {
+  if (data.is_scalar()) {
+    return data.scalar()->is_valid ? ~uint64_t(0) : uint64_t(0);
+  }
+
+  if (data.array()->null_count == data.array()->length) return uint64_t(0);
+
+  if (!data.array()->MayHaveNulls()) return ~uint64_t(0);
+
+  // no constant validity word available
+  return {};
+}
+
+inline Bitmap GetBitmap(const Datum& datum, int i) {
+  if (datum.is_scalar()) return {};
+  const ArrayData& a = *datum.array();
+  return Bitmap{a.buffers[i], a.offset, a.length};
+}
+
+// if the condition is null then output is null otherwise we take validity from the
+// selected argument
+// ie. cond.valid & (cond.data & left.valid | ~cond.data & right.valid)
+Status PromoteNullsVisitor(KernelContext* ctx, const Datum& cond_d, const Datum& left_d,
+                           const Datum& right_d, ArrayData* output) {
+  auto cond_const = GetConstantValidityWord(cond_d);
+  auto left_const = GetConstantValidityWord(left_d);
+  auto right_const = GetConstantValidityWord(right_d);
+
+  enum { COND_CONST = 1, LEFT_CONST = 2, RIGHT_CONST = 4 };
+  auto flag = COND_CONST * cond_const.has_value() | LEFT_CONST * left_const.has_value() |
+              RIGHT_CONST * right_const.has_value();
+
+  const ArrayData& cond = *cond_d.array();
+  // cond.data will always be available
+  Bitmap cond_data{cond.buffers[1], cond.offset, cond.length};
+  Bitmap cond_valid{cond.buffers[0], cond.offset, cond.length};
+  Bitmap left_valid = GetBitmap(left_d, 0);
+  Bitmap right_valid = GetBitmap(right_d, 0);
+  // sometimes Bitmaps will be ignored, in which case we replace access to them with
+  // duplicated (probably elided) access to cond_data
+  const Bitmap& _ = cond_data;
+
+  // lambda function that will be used inside the visitor
+  uint64_t* out_validity = nullptr;
+  int64_t i = 0;
+  auto apply = [&](uint64_t c_valid, uint64_t c_data, uint64_t l_valid,
+                   uint64_t r_valid) {
+    out_validity[i] = c_valid & ((c_data & l_valid) | (~c_data & r_valid));
+    i++;
+  };
+
+  // if the condition is null then output is null otherwise we take validity from the

Review comment:
       nit: duplicated 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.

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



[GitHub] [arrow] jorisvandenbossche commented on pull request #10410: ARROW-10640: [C++] A, "if_else" ("where") kernel to combine two arrays based on a mask

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


   I did a quick comparison with `np.where` (using plain float64 arrays without nulls to have it comparable), and it seems this implementation is already doing quite good compared to numpy (using a release build):
   
   ```
   In [19]: N = 10_000_000
       ...: arr1 = np.random.randn(N)
       ...: arr2 = np.random.randn(N)
       ...: mask = np.random.randint(0, 2, N).astype(bool)
       ...: 
       ...: pa_arr1 = pa.array(arr1)
       ...: pa_arr2 = pa.array(arr2)
       ...: pa_mask = pa.array(mask)
       ...: 
   
   In [20]: %timeit np.where(mask, arr1, arr2)
   82.3 ms ± 8.76 ms per loop (mean ± std. dev. of 7 runs, 10 loops each)
   
   In [21]: %timeit pc.if_else(pa_mask, pa_arr1, pa_arr2)
   50.4 ms ± 6.08 ms per loop (mean ± std. dev. of 7 runs, 10 loops each)
   ```
   


-- 
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



[GitHub] [arrow] bkietz commented on a change in pull request #10410: ARROW-10640: [C++] A "where" kernel to combine two arrays based on a mask

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_if_else.cc
##########
@@ -0,0 +1,577 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <arrow/compute/api.h>
+#include <arrow/compute/kernels/codegen_internal.h>
+#include <arrow/compute/util_internal.h>
+#include <arrow/util/bit_block_counter.h>
+#include <arrow/util/bitmap.h>
+#include <arrow/util/bitmap_ops.h>
+
+namespace arrow {
+using internal::BitBlockCount;
+using internal::BitBlockCounter;
+using internal::Bitmap;
+
+namespace compute {
+
+namespace {
+
+util::optional<uint64_t> GetConstantValidityWord(const Datum& data) {
+  if (data.is_scalar()) {
+    return data.scalar()->is_valid ? ~uint64_t(0) : uint64_t(0);

Review comment:
       Instead of using `~uint64_t(0)` or `UINT64_MAX`, please declare
   
   ```c++
   constexpr uint64_t kAllNull = 0;
   constexpr uint64_t kAllValid = ~kAllNull;
   ```
   
   or so

##########
File path: cpp/src/arrow/compute/kernels/scalar_if_else.cc
##########
@@ -0,0 +1,577 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <arrow/compute/api.h>
+#include <arrow/compute/kernels/codegen_internal.h>
+#include <arrow/compute/util_internal.h>
+#include <arrow/util/bit_block_counter.h>
+#include <arrow/util/bitmap.h>
+#include <arrow/util/bitmap_ops.h>
+
+namespace arrow {
+using internal::BitBlockCount;
+using internal::BitBlockCounter;
+using internal::Bitmap;
+
+namespace compute {
+
+namespace {
+
+util::optional<uint64_t> GetConstantValidityWord(const Datum& data) {
+  if (data.is_scalar()) {
+    return data.scalar()->is_valid ? ~uint64_t(0) : uint64_t(0);
+  }
+
+  if (data.array()->null_count == data.array()->length) return uint64_t(0);
+
+  if (!data.array()->MayHaveNulls()) return ~uint64_t(0);
+
+  // no constant validity word available
+  return {};
+}
+
+inline Bitmap GetBitmap(const Datum& datum, int i) {
+  if (datum.is_scalar()) return {};
+  const ArrayData& a = *datum.array();
+  return Bitmap{a.buffers[i], a.offset, a.length};
+}
+
+// if the condition is null then output is null otherwise we take validity from the
+// selected argument
+// ie. cond.valid & (cond.data & left.valid | ~cond.data & right.valid)
+Status PromoteNullsVisitor(KernelContext* ctx, const Datum& cond_d, const Datum& left_d,
+                           const Datum& right_d, ArrayData* output) {
+  auto cond_const = GetConstantValidityWord(cond_d);
+  auto left_const = GetConstantValidityWord(left_d);
+  auto right_const = GetConstantValidityWord(right_d);
+
+  enum { COND_CONST = 1, LEFT_CONST = 2, RIGHT_CONST = 4 };
+  auto flag = COND_CONST * cond_const.has_value() | LEFT_CONST * left_const.has_value() |
+              RIGHT_CONST * right_const.has_value();
+
+  const ArrayData& cond = *cond_d.array();
+  // cond.data will always be available
+  Bitmap cond_data{cond.buffers[1], cond.offset, cond.length};
+  Bitmap cond_valid{cond.buffers[0], cond.offset, cond.length};
+  Bitmap left_valid = GetBitmap(left_d, 0);
+  Bitmap right_valid = GetBitmap(right_d, 0);
+  // sometimes Bitmaps will be ignored, in which case we replace access to them with
+  // duplicated (probably elided) access to cond_data
+  const Bitmap& _ = cond_data;
+
+  // lambda function that will be used inside the visitor
+  uint64_t* out_validity = nullptr;
+  int64_t i = 0;
+  auto apply = [&](uint64_t c_valid, uint64_t c_data, uint64_t l_valid,
+                   uint64_t r_valid) {
+    out_validity[i] = c_valid & ((c_data & l_valid) | (~c_data & r_valid));
+    i++;
+  };
+
+  // cond.valid & (cond.data & left.valid | ~cond.data & right.valid)
+  // In the following cases, we dont need to allocate out_valid bitmap
+  switch (flag) {
+    case COND_CONST | LEFT_CONST | RIGHT_CONST:
+      // if cond & left & right all ones, then output is all valid --> out_valid = nullptr
+      if ((*cond_const & *left_const & *right_const) == UINT64_MAX) {
+        return Status::OK();
+      }

Review comment:
       What if instead we rewrite these two special cases as if statements:
   ```c++
   if (cond_const == kAllValid && left_const == kAllValid && right_const == kAllValid) {
     // no nulls in any input: don't bother allocating a null bitmap
     return Status::OK();
   }
   
   if (left_const == kAllValid && right_const == kAllValid) {
     // copy or slice the condition bitmap
     //...
     return Status::OK();
   }
   ```

##########
File path: cpp/src/arrow/compute/kernels/scalar_if_else.cc
##########
@@ -0,0 +1,577 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <arrow/compute/api.h>
+#include <arrow/compute/kernels/codegen_internal.h>
+#include <arrow/compute/util_internal.h>
+#include <arrow/util/bit_block_counter.h>
+#include <arrow/util/bitmap.h>
+#include <arrow/util/bitmap_ops.h>
+
+namespace arrow {
+using internal::BitBlockCount;
+using internal::BitBlockCounter;
+using internal::Bitmap;
+
+namespace compute {
+
+namespace {
+
+util::optional<uint64_t> GetConstantValidityWord(const Datum& data) {
+  if (data.is_scalar()) {
+    return data.scalar()->is_valid ? ~uint64_t(0) : uint64_t(0);
+  }
+
+  if (data.array()->null_count == data.array()->length) return uint64_t(0);
+
+  if (!data.array()->MayHaveNulls()) return ~uint64_t(0);
+
+  // no constant validity word available
+  return {};
+}
+
+inline Bitmap GetBitmap(const Datum& datum, int i) {
+  if (datum.is_scalar()) return {};
+  const ArrayData& a = *datum.array();
+  return Bitmap{a.buffers[i], a.offset, a.length};
+}
+
+// if the condition is null then output is null otherwise we take validity from the
+// selected argument
+// ie. cond.valid & (cond.data & left.valid | ~cond.data & right.valid)
+Status PromoteNullsVisitor(KernelContext* ctx, const Datum& cond_d, const Datum& left_d,
+                           const Datum& right_d, ArrayData* output) {
+  auto cond_const = GetConstantValidityWord(cond_d);
+  auto left_const = GetConstantValidityWord(left_d);
+  auto right_const = GetConstantValidityWord(right_d);
+
+  enum { COND_CONST = 1, LEFT_CONST = 2, RIGHT_CONST = 4 };
+  auto flag = COND_CONST * cond_const.has_value() | LEFT_CONST * left_const.has_value() |
+              RIGHT_CONST * right_const.has_value();
+
+  const ArrayData& cond = *cond_d.array();
+  // cond.data will always be available
+  Bitmap cond_data{cond.buffers[1], cond.offset, cond.length};
+  Bitmap cond_valid{cond.buffers[0], cond.offset, cond.length};
+  Bitmap left_valid = GetBitmap(left_d, 0);
+  Bitmap right_valid = GetBitmap(right_d, 0);
+  // sometimes Bitmaps will be ignored, in which case we replace access to them with
+  // duplicated (probably elided) access to cond_data
+  const Bitmap& _ = cond_data;
+
+  // lambda function that will be used inside the visitor
+  uint64_t* out_validity = nullptr;
+  int64_t i = 0;
+  auto apply = [&](uint64_t c_valid, uint64_t c_data, uint64_t l_valid,
+                   uint64_t r_valid) {
+    out_validity[i] = c_valid & ((c_data & l_valid) | (~c_data & r_valid));
+    i++;
+  };
+
+  // cond.valid & (cond.data & left.valid | ~cond.data & right.valid)
+  // In the following cases, we dont need to allocate out_valid bitmap
+  switch (flag) {
+    case COND_CONST | LEFT_CONST | RIGHT_CONST:
+      // if cond & left & right all ones, then output is all valid --> out_valid = nullptr
+      if ((*cond_const & *left_const & *right_const) == UINT64_MAX) {
+        return Status::OK();
+      }
+    case LEFT_CONST | RIGHT_CONST:
+      // if both left and right are valid, no need to calculate out_valid bitmap. Pass
+      // cond validity buffer
+      if ((*left_const & *right_const) == UINT64_MAX) {
+        // if there's an offset, copy bitmap (cannot slice a bitmap)
+        if (cond.offset) {
+          ARROW_ASSIGN_OR_RAISE(
+              output->buffers[0],
+              arrow::internal::CopyBitmap(ctx->memory_pool(), cond.buffers[0]->data(),
+                                          cond.offset, cond.length));
+        } else {  // just copy assign cond validity buffer
+          output->buffers[0] = cond.buffers[0];
+        }
+        return Status::OK();
+      }
+  }
+
+  // following cases requires a separate out_valid buffer
+  ARROW_ASSIGN_OR_RAISE(output->buffers[0], ctx->AllocateBitmap(cond.length));
+  out_validity = output->GetMutableValues<uint64_t>(0);
+
+  enum { C_VALID, C_DATA, L_VALID, R_VALID };
+
+  switch (flag) {
+    case COND_CONST | LEFT_CONST | RIGHT_CONST:
+      Bitmap::VisitWords({_, cond_data, _, _}, [&](std::array<uint64_t, 4> words) {
+        apply(*cond_const, words[C_DATA], *left_const, *right_const);
+      });
+      break;
+    case LEFT_CONST | RIGHT_CONST:
+      Bitmap::VisitWords(
+          {cond_valid, cond_data, _, _}, [&](std::array<uint64_t, 4> words) {
+            apply(words[C_VALID], words[C_DATA], *left_const, *right_const);
+          });
+      break;
+    case COND_CONST | RIGHT_CONST:
+      // bitmaps[C_VALID], bitmaps[R_VALID] might be null; override to make it safe for
+      // Visit()
+      Bitmap::VisitWords(
+          {_, cond_data, left_valid, _}, [&](std::array<uint64_t, 4> words) {
+            apply(*cond_const, words[C_DATA], words[L_VALID], *right_const);
+          });
+      break;
+    case RIGHT_CONST:
+      // bitmaps[R_VALID] might be null; override to make it safe for Visit()
+      Bitmap::VisitWords(
+          {cond_valid, cond_data, left_valid, _}, [&](std::array<uint64_t, 4> words) {
+            apply(words[C_VALID], words[C_DATA], words[L_VALID], *right_const);
+          });
+      break;
+    case COND_CONST | LEFT_CONST:
+      // bitmaps[C_VALID], bitmaps[L_VALID] might be null; override to make it safe for
+      // Visit()
+      Bitmap::VisitWords({_, cond_data, _, right_valid},
+                         [&](std::array<uint64_t, 4> words) {
+                           apply(*cond_const, words[C_DATA], *left_const, words[R_VALID]);
+                         });
+      break;
+    case LEFT_CONST:
+      // bitmaps[L_VALID] might be null; override to make it safe for Visit()
+      Bitmap::VisitWords(
+          {cond_valid, cond_data, _, right_valid}, [&](std::array<uint64_t, 4> words) {
+            apply(words[C_VALID], words[C_DATA], *left_const, words[R_VALID]);
+          });
+      break;
+    case COND_CONST:
+      // bitmaps[C_VALID] might be null; override to make it safe for Visit()
+      Bitmap::VisitWords(
+          {_, cond_data, left_valid, right_valid}, [&](std::array<uint64_t, 4> words) {
+            apply(*cond_const, words[C_DATA], words[L_VALID], words[R_VALID]);
+          });
+      break;
+    case 0:
+      Bitmap::VisitWords({cond_valid, cond_data, left_valid, right_valid},
+                         [&](std::array<uint64_t, 4> words) {
+                           apply(words[C_VALID], words[C_DATA], words[L_VALID],
+                                 words[R_VALID]);
+                         });
+      break;
+  }
+  return Status::OK();
+}
+
+template <typename Type, typename Enable = void>
+struct IfElseFunctor {};
+
+// only number types needs to be handled for Fixed sized primitive data types because,
+// internal::GenerateTypeAgnosticPrimitive forwards types to the corresponding unsigned
+// int type
+template <typename Type>
+struct IfElseFunctor<Type, enable_if_number<Type>> {
+  using T = typename TypeTraits<Type>::CType;
+  // A - Array
+  // S - Scalar
+
+  //  AAA
+  static Status Call(KernelContext* ctx, const ArrayData& cond, const ArrayData& left,
+                     const ArrayData& right, ArrayData* out) {
+    ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Buffer> out_buf,
+                          ctx->Allocate(cond.length * sizeof(T)));
+    T* out_values = reinterpret_cast<T*>(out_buf->mutable_data());
+
+    // copy right data to out_buff
+    const T* right_data = right.GetValues<T>(1);
+    std::memcpy(out_values, right_data, right.length * sizeof(T));
+
+    const auto* cond_data = cond.buffers[1]->data();  // this is a BoolArray
+    BitBlockCounter bit_counter(cond_data, cond.offset, cond.length);
+
+    // selectively copy values from left data
+    const T* left_data = left.GetValues<T>(1);
+    int64_t offset = cond.offset;
+
+    // todo this can be improved by intrinsics. ex: _mm*_mask_store_e* (vmovdqa*)
+    while (offset < cond.offset + cond.length) {
+      const BitBlockCount& block = bit_counter.NextWord();
+      if (block.AllSet()) {  // all from left
+        std::memcpy(out_values, left_data, block.length * sizeof(T));
+      } else if (block.popcount) {  // selectively copy from left
+        for (int64_t i = 0; i < block.length; ++i) {
+          if (BitUtil::GetBit(cond_data, offset + i)) {
+            out_values[i] = left_data[i];
+          }
+        }
+      }
+
+      offset += block.length;
+      out_values += block.length;
+      left_data += block.length;
+    }
+
+    out->buffers[1] = std::move(out_buf);
+    return Status::OK();
+  }
+
+  // ASA
+  static Status Call(KernelContext* ctx, const ArrayData& cond, const Scalar& left,
+                     const ArrayData& right, ArrayData* out) {
+    ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Buffer> out_buf,
+                          ctx->Allocate(cond.length * sizeof(T)));
+    T* out_values = reinterpret_cast<T*>(out_buf->mutable_data());
+
+    // copy right data to out_buff
+    const T* right_data = right.GetValues<T>(1);
+    std::memcpy(out_values, right_data, right.length * sizeof(T));
+
+    const auto* cond_data = cond.buffers[1]->data();  // this is a BoolArray
+    BitBlockCounter bit_counter(cond_data, cond.offset, cond.length);
+
+    // selectively copy values from left data
+    T left_data = internal::UnboxScalar<Type>::Unbox(left);
+    int64_t offset = cond.offset;
+
+    // todo this can be improved by intrinsics. ex: _mm*_mask_store_e* (vmovdqa*)
+    while (offset < cond.offset + cond.length) {
+      const BitBlockCount& block = bit_counter.NextWord();
+      if (block.AllSet()) {  // all from left
+        std::fill(out_values, out_values + block.length, left_data);
+      } else if (block.popcount) {  // selectively copy from left
+        for (int64_t i = 0; i < block.length; ++i) {
+          if (BitUtil::GetBit(cond_data, offset + i)) {
+            out_values[i] = left_data;
+          }
+        }
+      }
+
+      offset += block.length;
+      out_values += block.length;
+    }
+
+    out->buffers[1] = std::move(out_buf);
+    return Status::OK();
+  }
+
+  // AAS
+  static Status Call(KernelContext* ctx, const ArrayData& cond, const ArrayData& left,
+                     const Scalar& right, ArrayData* out) {
+    ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Buffer> out_buf,
+                          ctx->Allocate(cond.length * sizeof(T)));
+    T* out_values = reinterpret_cast<T*>(out_buf->mutable_data());
+
+    // copy left data to out_buff
+    const T* left_data = left.GetValues<T>(1);
+    std::memcpy(out_values, left_data, left.length * sizeof(T));
+
+    const auto* cond_data = cond.buffers[1]->data();  // this is a BoolArray
+    BitBlockCounter bit_counter(cond_data, cond.offset, cond.length);
+
+    // selectively copy values from left data
+    T right_data = internal::UnboxScalar<Type>::Unbox(right);
+    int64_t offset = cond.offset;
+
+    // todo this can be improved by intrinsics. ex: _mm*_mask_store_e* (vmovdqa*)
+    // left data is already in the output buffer. Therefore, mask needs to be inverted
+    while (offset < cond.offset + cond.length) {
+      const BitBlockCount& block = bit_counter.NextWord();
+      if (block.NoneSet()) {  // all from right
+        std::fill(out_values, out_values + block.length, right_data);
+      } else if (block.popcount) {  // selectively copy from right
+        for (int64_t i = 0; i < block.length; ++i) {
+          if (!BitUtil::GetBit(cond_data, offset + i)) {
+            out_values[i] = right_data;
+          }
+        }
+      }
+
+      offset += block.length;
+      out_values += block.length;
+    }
+
+    out->buffers[1] = std::move(out_buf);
+    return Status::OK();
+  }
+
+  // ASS
+  static Status Call(KernelContext* ctx, const ArrayData& cond, const Scalar& left,
+                     const Scalar& right, ArrayData* out) {
+    ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Buffer> out_buf,
+                          ctx->Allocate(cond.length * sizeof(T)));
+    T* out_values = reinterpret_cast<T*>(out_buf->mutable_data());
+
+    // copy right data to out_buff
+    T right_data = internal::UnboxScalar<Type>::Unbox(right);
+    std::fill(out_values, out_values + cond.length, right_data);
+
+    const auto* cond_data = cond.buffers[1]->data();  // this is a BoolArray
+    BitBlockCounter bit_counter(cond_data, cond.offset, cond.length);
+
+    // selectively copy values from left data
+    T left_data = internal::UnboxScalar<Type>::Unbox(left);
+    int64_t offset = cond.offset;
+
+    // todo this can be improved by intrinsics. ex: _mm*_mask_store_e* (vmovdqa*)
+    while (offset < cond.offset + cond.length) {
+      const BitBlockCount& block = bit_counter.NextWord();
+      if (block.AllSet()) {  // all from left
+        std::fill(out_values, out_values + block.length, left_data);
+      } else if (block.popcount) {  // selectively copy from left
+        for (int64_t i = 0; i < block.length; ++i) {
+          if (BitUtil::GetBit(cond_data, offset + i)) {
+            out_values[i] = left_data;
+          }
+        }
+      }
+
+      offset += block.length;
+      out_values += block.length;
+    }
+
+    out->buffers[1] = std::move(out_buf);
+    return Status::OK();
+  }
+};
+
+template <typename Type>
+struct IfElseFunctor<Type, enable_if_boolean<Type>> {
+  // AAA
+  static Status Call(KernelContext* ctx, const ArrayData& cond, const ArrayData& left,
+                     const ArrayData& right, ArrayData* out) {
+    // out_buff = right & ~cond
+    ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Buffer> out_buf,
+                          arrow::internal::BitmapAndNot(
+                              ctx->memory_pool(), right.buffers[1]->data(), right.offset,
+                              cond.buffers[1]->data(), cond.offset, cond.length, 0));
+
+    // out_buff = left & cond
+    ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Buffer> temp_buf,
+                          arrow::internal::BitmapAnd(
+                              ctx->memory_pool(), left.buffers[1]->data(), left.offset,
+                              cond.buffers[1]->data(), cond.offset, cond.length, 0));
+
+    arrow::internal::BitmapOr(out_buf->data(), 0, temp_buf->data(), 0, cond.length, 0,
+                              out_buf->mutable_data());
+    out->buffers[1] = std::move(out_buf);
+    return Status::OK();
+  }
+
+  // ASA
+  static Status Call(KernelContext* ctx, const ArrayData& cond, const Scalar& left,
+                     const ArrayData& right, ArrayData* out) {
+    // out_buff = right & ~cond
+    ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Buffer> out_buf,
+                          arrow::internal::BitmapAndNot(
+                              ctx->memory_pool(), right.buffers[1]->data(), right.offset,
+                              cond.buffers[1]->data(), cond.offset, cond.length, 0));
+
+    // out_buff = left & cond
+    bool left_data = internal::UnboxScalar<BooleanType>::Unbox(left);
+    if (left_data) {
+      arrow::internal::BitmapOr(out_buf->data(), 0, cond.buffers[1]->data(), cond.offset,
+                                cond.length, 0, out_buf->mutable_data());
+    }
+
+    out->buffers[1] = std::move(out_buf);
+    return Status::OK();
+  }
+
+  // AAS
+  static Status Call(KernelContext* ctx, const ArrayData& cond, const ArrayData& left,
+                     const Scalar& right, ArrayData* out) {
+    // out_buff = left & cond
+    ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Buffer> out_buf,
+                          arrow::internal::BitmapAnd(
+                              ctx->memory_pool(), left.buffers[1]->data(), left.offset,
+                              cond.buffers[1]->data(), cond.offset, cond.length, 0));
+
+    bool right_data = internal::UnboxScalar<BooleanType>::Unbox(right);
+
+    // out_buff = left & cond | right & ~cond
+    if (right_data) {
+      arrow::internal::BitmapOrNot(out_buf->data(), 0, cond.buffers[1]->data(),
+                                   cond.offset, cond.length, 0, out_buf->mutable_data());
+    }
+
+    out->buffers[1] = std::move(out_buf);
+    return Status::OK();
+  }
+
+  // ASS
+  static Status Call(KernelContext* ctx, const ArrayData& cond, const Scalar& left,
+                     const Scalar& right, ArrayData* out) {
+    bool left_data = internal::UnboxScalar<BooleanType>::Unbox(left);
+    bool right_data = internal::UnboxScalar<BooleanType>::Unbox(right);
+
+    // out_buf = left & cond | right & ~cond
+    std::shared_ptr<Buffer> out_buf = nullptr;
+    if (left_data) {
+      if (right_data) {
+        // out_buf = ones
+        ARROW_ASSIGN_OR_RAISE(out_buf, ctx->AllocateBitmap(cond.length));
+        // filling with UINT8_MAX upto the buffer's size (in bytes)
+        arrow::compute::internal::SetMemory<UINT8_MAX>(out_buf.get());
+      } else {
+        // out_buf = cond
+        out_buf = SliceBuffer(cond.buffers[1], cond.offset, cond.length);
+      }
+    } else {
+      if (right_data) {
+        // out_buf = ~cond
+        ARROW_ASSIGN_OR_RAISE(out_buf, arrow::internal::InvertBitmap(
+                                           ctx->memory_pool(), cond.buffers[1]->data(),
+                                           cond.offset, cond.length))
+      } else {
+        // out_buf = zeros
+        ARROW_ASSIGN_OR_RAISE(out_buf, ctx->AllocateBitmap(cond.length));
+      }
+    }
+    out->buffers[1] = std::move(out_buf);
+    return Status::OK();
+  }
+};
+
+template <typename Type>
+struct IfElseFunctor<Type, enable_if_null<Type>> {
+  template <typename T>
+  static inline Status ReturnCopy(const T& in, T* out) {
+    // Nothing preallocated, so we assign in into the output
+    *out = in;
+    return Status::OK();
+  }
+
+  // AAA
+  static Status Call(KernelContext* ctx, const ArrayData& cond, const ArrayData& left,
+                     const ArrayData& right, ArrayData* out) {
+    return ReturnCopy(left, out);
+  }
+
+  // ASA
+  static Status Call(KernelContext* ctx, const ArrayData& cond, const Scalar& left,
+                     const ArrayData& right, ArrayData* out) {
+    return ReturnCopy(right, out);
+  }
+
+  // AAS
+  static Status Call(KernelContext* ctx, const ArrayData& cond, const ArrayData& left,
+                     const Scalar& right, ArrayData* out) {
+    return ReturnCopy(left, out);
+  }
+
+  // ASS
+  static Status Call(KernelContext* ctx, const ArrayData& cond, const Scalar& left,
+                     const Scalar& right, ArrayData* out) {
+    return ReturnCopy(cond, out);
+  }
+};
+
+template <typename Type>
+struct ResolveIfElseExec {
+  static Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    // cond is scalar
+    if (batch[0].is_scalar()) {
+      const auto& cond = batch[0].scalar_as<BooleanScalar>();
+      if (batch[1].is_scalar() && batch[2].is_scalar()) {
+        if (cond.is_valid) {
+          *out = cond.value ? batch[1].scalar() : batch[2].scalar();
+        } else {
+          *out = MakeNullScalar(batch[1].type());
+        }
+        return Status::OK();
+      } else {  // either left or right is an array. output is always an array
+        // output size is the size of the array arg
+        auto bcast_size = batch.length;
+        if (cond.is_valid) {
+          const auto& valid_data = cond.value ? batch[1] : batch[2];
+          if (valid_data.is_array()) {
+            *out = valid_data;
+          } else {  // valid data is a scalar that needs to be broadcasted
+            ARROW_ASSIGN_OR_RAISE(*out,
+                                  MakeArrayFromScalar(*valid_data.scalar(), bcast_size,
+                                                      ctx->memory_pool()));
+          }
+        } else {  // cond is null. create null array
+          ARROW_ASSIGN_OR_RAISE(
+              *out, MakeArrayOfNull(batch[1].type(), bcast_size, ctx->memory_pool()))
+        }
+        return Status::OK();
+      }
+    }

Review comment:
       ```suggestion
         }
         // either left or right is an array. Output is always an array
         if (!cond.is_valid) {
           // cond is null; just create a null array
           ARROW_ASSIGN_OR_RAISE(
               *out, MakeArrayOfNull(batch[1].type(), bcast_size, ctx->memory_pool()))
           return Status::OK();
         }
   
         const auto& valid_data = cond.value ? batch[1] : batch[2];
         if (valid_data.is_array()) {
           *out = valid_data;
         } else {
           // valid data is a scalar that needs to be broadcasted
           ARROW_ASSIGN_OR_RAISE(*out,
                                 MakeArrayFromScalar(*valid_data.scalar(), batch.length,
                                                     ctx->memory_pool()));
         }
         return Status::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.

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



[GitHub] [arrow] bkietz commented on a change in pull request #10410: ARROW-10640: [C++] A "where" kernel to combine two arrays based on a mask

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_if_else.cc
##########
@@ -0,0 +1,740 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <arrow/compute/api.h>
+#include <arrow/compute/kernels/codegen_internal.h>
+#include <arrow/compute/util_internal.h>
+#include <arrow/util/bit_block_counter.h>
+#include <arrow/util/bitmap.h>
+#include <arrow/util/bitmap_ops.h>
+
+namespace arrow {
+using internal::BitBlockCount;
+using internal::BitBlockCounter;
+using internal::Bitmap;
+
+namespace compute {
+
+namespace {
+
+enum { COND_ALL_VALID = 1, LEFT_ALL_VALID = 2, RIGHT_ALL_VALID = 4 };
+
+// if the condition is null then output is null otherwise we take validity from the
+// selected argument
+// ie. cond.valid & (cond.data & left.valid | ~cond.data & right.valid)
+Status PromoteNullsVisitor(KernelContext* ctx, const ArrayData& cond, const Scalar& left,
+                           const Scalar& right, ArrayData* output) {
+  uint8_t flag = right.is_valid * 4 + left.is_valid * 2 + !cond.MayHaveNulls();
+
+  if (flag < 6 && flag != 3) {
+    // there will be a validity buffer in the output
+    ARROW_ASSIGN_OR_RAISE(output->buffers[0], ctx->AllocateBitmap(cond.length));
+  }
+
+  // if the condition is null then output is null otherwise we take validity from the
+  // selected argument
+  // ie. cond.valid & (cond.data & left.valid | ~cond.data & right.valid)
+  switch (flag) {
+    case COND_ALL_VALID | LEFT_ALL_VALID | RIGHT_ALL_VALID:  // = 7
+      break;
+    case LEFT_ALL_VALID | RIGHT_ALL_VALID:  // = 6
+      // out_valid = c_valid
+      output->buffers[0] = SliceBuffer(cond.buffers[0], cond.offset, cond.length);
+      break;
+    case COND_ALL_VALID | RIGHT_ALL_VALID:  // = 5
+      // out_valid = ~cond.data
+      arrow::internal::InvertBitmap(cond.buffers[1]->data(), cond.offset, cond.length,
+                                    output->buffers[0]->mutable_data(), 0);
+      break;
+    case RIGHT_ALL_VALID:  // = 4
+      // out_valid = c_valid & ~cond.data
+      arrow::internal::BitmapAndNot(cond.buffers[0]->data(), cond.offset,
+                                    cond.buffers[1]->data(), cond.offset, cond.length, 0,
+                                    output->buffers[0]->mutable_data());
+      break;
+    case COND_ALL_VALID | LEFT_ALL_VALID:  // = 3
+      // out_valid = cond.data
+      output->buffers[0] = SliceBuffer(cond.buffers[1], cond.offset, cond.length);
+      break;
+    case LEFT_ALL_VALID:  // = 2
+      // out_valid = cond.valid & cond.data
+      arrow::internal::BitmapAnd(cond.buffers[0]->data(), cond.offset,
+                                 cond.buffers[1]->data(), cond.offset, cond.length, 0,
+                                 output->buffers[0]->mutable_data());
+      break;
+    case COND_ALL_VALID:  // = 1
+      // out_valid = 0 --> nothing to do; but requires out_valid to be a all-zero buffer
+      break;
+    case 0:
+      // out_valid = 0 --> nothing to do; but requires out_valid to be a all-zero buffer
+      break;
+  }
+  return Status::OK();
+}
+
+Status PromoteNullsVisitor(KernelContext* ctx, const ArrayData& cond,
+                           const ArrayData& left, const Scalar& right,
+                           ArrayData* output) {
+  uint8_t flag = right.is_valid * 4 + !left.MayHaveNulls() * 2 + !cond.MayHaveNulls();
+
+  enum { C_VALID, C_DATA, L_VALID };
+
+  Bitmap bitmaps[3];
+  bitmaps[C_VALID] = {cond.buffers[0], cond.offset, cond.length};
+  bitmaps[C_DATA] = {cond.buffers[1], cond.offset, cond.length};
+  bitmaps[L_VALID] = {left.buffers[0], left.offset, left.length};
+
+  uint64_t* out_validity = nullptr;
+  if (flag < 6 && flag != 3) {
+    // there will be a validity buffer in the output
+    ARROW_ASSIGN_OR_RAISE(output->buffers[0], ctx->AllocateBitmap(cond.length));
+    out_validity = output->GetMutableValues<uint64_t>(0);
+  }
+
+  // lambda function that will be used inside the visitor
+  int64_t i = 0;
+  auto apply = [&](uint64_t c_valid, uint64_t c_data, uint64_t l_valid,
+                   uint64_t r_valid) {
+    out_validity[i] = c_valid & ((c_data & l_valid) | (~c_data & r_valid));
+    i++;
+  };
+
+  // if the condition is null then output is null otherwise we take validity from the
+  // selected argument
+  // ie. cond.valid & (cond.data & left.valid | ~cond.data & right.valid)
+  switch (flag) {
+    case COND_ALL_VALID | LEFT_ALL_VALID | RIGHT_ALL_VALID:
+      break;
+    case LEFT_ALL_VALID | RIGHT_ALL_VALID:
+      output->buffers[0] = SliceBuffer(cond.buffers[0], cond.offset, cond.length);
+      break;
+    case COND_ALL_VALID | RIGHT_ALL_VALID:
+      // bitmaps[C_VALID] might be null; override to make it safe for Visit()
+      bitmaps[C_VALID] = bitmaps[C_DATA];
+      Bitmap::VisitWords(bitmaps, [&](std::array<uint64_t, 3> words) {
+        apply(UINT64_MAX, words[C_DATA], words[L_VALID], UINT64_MAX);
+      });
+      break;
+    case RIGHT_ALL_VALID:
+      Bitmap::VisitWords(bitmaps, [&](std::array<uint64_t, 3> words) {
+        apply(words[C_VALID], words[C_DATA], words[L_VALID], UINT64_MAX);
+      });
+      break;
+    case COND_ALL_VALID | LEFT_ALL_VALID:
+      // only cond.data is passed
+      output->buffers[0] = SliceBuffer(cond.buffers[1], cond.offset, cond.length);
+      break;
+    case LEFT_ALL_VALID:
+      // out_valid = cond.valid & cond.data
+      arrow::internal::BitmapAnd(cond.buffers[0]->data(), cond.offset,
+                                 cond.buffers[1]->data(), cond.offset, cond.length, 0,
+                                 output->buffers[0]->mutable_data());
+      break;
+    case COND_ALL_VALID:
+      // out_valid = cond.data & left.valid
+      arrow::internal::BitmapAnd(cond.buffers[1]->data(), cond.offset,
+                                 left.buffers[0]->data(), left.offset, cond.length, 0,
+                                 output->buffers[0]->mutable_data());
+      break;
+    case 0:
+      Bitmap::VisitWords(bitmaps, [&](std::array<uint64_t, 3> words) {
+        apply(words[C_VALID], words[C_DATA], words[L_VALID], 0);
+      });
+      break;
+  }
+  return Status::OK();
+}
+
+// if the condition is null then output is null otherwise we take validity from the
+// selected argument
+// ie. cond.valid & (cond.data & left.valid | ~cond.data & right.valid)
+Status PromoteNullsVisitor(KernelContext* ctx, const ArrayData& cond, const Scalar& left,
+                           const ArrayData& right, ArrayData* output) {
+  uint8_t flag = !right.MayHaveNulls() * 4 + left.is_valid * 2 + !cond.MayHaveNulls();
+
+  enum { C_VALID, C_DATA, R_VALID };
+
+  Bitmap bitmaps[3];
+  bitmaps[C_VALID] = {cond.buffers[0], cond.offset, cond.length};
+  bitmaps[C_DATA] = {cond.buffers[1], cond.offset, cond.length};
+  bitmaps[R_VALID] = {right.buffers[0], right.offset, right.length};
+
+  uint64_t* out_validity = nullptr;
+  if (flag < 6) {
+    // there will be a validity buffer in the output
+    ARROW_ASSIGN_OR_RAISE(output->buffers[0], ctx->AllocateBitmap(cond.length));
+    out_validity = output->GetMutableValues<uint64_t>(0);
+  }
+
+  // lambda function that will be used inside the visitor
+  int64_t i = 0;
+  auto apply = [&](uint64_t c_valid, uint64_t c_data, uint64_t l_valid,
+                   uint64_t r_valid) {
+    out_validity[i] = c_valid & ((c_data & l_valid) | (~c_data & r_valid));
+    i++;
+  };
+
+  // if the condition is null then output is null otherwise we take validity from the
+  // selected argument
+  // ie. cond.valid & (cond.data & left.valid | ~cond.data & right.valid)
+  switch (flag) {
+    case COND_ALL_VALID | LEFT_ALL_VALID | RIGHT_ALL_VALID:
+      break;
+    case LEFT_ALL_VALID | RIGHT_ALL_VALID:
+      output->buffers[0] = SliceBuffer(cond.buffers[0], cond.offset, cond.length);
+      break;
+    case COND_ALL_VALID | RIGHT_ALL_VALID:
+      // out_valid = ~cond.data
+      arrow::internal::InvertBitmap(cond.buffers[1]->data(), cond.offset, cond.length,
+                                    output->buffers[0]->mutable_data(), 0);
+      break;
+    case RIGHT_ALL_VALID:
+      // out_valid = c_valid & ~cond.data
+      arrow::internal::BitmapAndNot(cond.buffers[0]->data(), cond.offset,
+                                    cond.buffers[1]->data(), cond.offset, cond.length, 0,
+                                    output->buffers[0]->mutable_data());
+      break;
+    case COND_ALL_VALID | LEFT_ALL_VALID:
+      // bitmaps[C_VALID] might be null; override to make it safe for Visit()
+      bitmaps[C_VALID] = bitmaps[C_DATA];
+      Bitmap::VisitWords(bitmaps, [&](std::array<uint64_t, 3> words) {
+        apply(UINT64_MAX, words[C_DATA], UINT64_MAX, words[R_VALID]);
+      });
+      break;
+    case LEFT_ALL_VALID:
+      Bitmap::VisitWords(bitmaps, [&](std::array<uint64_t, 3> words) {
+        apply(words[C_VALID], words[C_DATA], UINT64_MAX, words[R_VALID]);
+      });
+      break;
+    case COND_ALL_VALID:
+      // out_valid =  ~cond.data & right.valid
+      arrow::internal::BitmapAndNot(right.buffers[0]->data(), right.offset,
+                                    cond.buffers[1]->data(), cond.offset, cond.length, 0,
+                                    output->buffers[0]->mutable_data());
+      break;
+    case 0:
+      Bitmap::VisitWords(bitmaps, [&](std::array<uint64_t, 3> words) {
+        apply(words[C_VALID], words[C_DATA], 0, words[R_VALID]);
+      });
+      break;
+  }
+  return Status::OK();
+}
+
+// if the condition is null then output is null otherwise we take validity from the
+// selected argument
+// ie. cond.valid & (cond.data & left.valid | ~cond.data & right.valid)
+Status PromoteNullsVisitor(KernelContext* ctx, const ArrayData& cond,
+                           const ArrayData& left, const ArrayData& right,
+                           ArrayData* output) {
+  uint8_t flag =
+      !right.MayHaveNulls() * 4 + !left.MayHaveNulls() * 2 + !cond.MayHaveNulls();
+
+  enum { C_VALID, C_DATA, L_VALID, R_VALID };
+
+  Bitmap bitmaps[4];
+  bitmaps[C_VALID] = {cond.buffers[0], cond.offset, cond.length};
+  bitmaps[C_DATA] = {cond.buffers[1], cond.offset, cond.length};
+  bitmaps[L_VALID] = {left.buffers[0], left.offset, left.length};
+  bitmaps[R_VALID] = {right.buffers[0], right.offset, right.length};
+
+  uint64_t* out_validity = nullptr;
+  if (flag < 6) {
+    // there will be a validity buffer in the output
+    ARROW_ASSIGN_OR_RAISE(output->buffers[0], ctx->AllocateBitmap(cond.length));
+    out_validity = output->GetMutableValues<uint64_t>(0);
+  }
+
+  // lambda function that will be used inside the visitor
+  int64_t i = 0;
+  auto apply = [&](uint64_t c_valid, uint64_t c_data, uint64_t l_valid,
+                   uint64_t r_valid) {
+    out_validity[i] = c_valid & ((c_data & l_valid) | (~c_data & r_valid));
+    i++;
+  };
+
+  // if the condition is null then output is null otherwise we take validity from the
+  // selected argument
+  // ie. cond.valid & (cond.data & left.valid | ~cond.data & right.valid)
+  switch (flag) {
+    case COND_ALL_VALID | LEFT_ALL_VALID | RIGHT_ALL_VALID:
+      break;
+    case LEFT_ALL_VALID | RIGHT_ALL_VALID:
+      output->buffers[0] = SliceBuffer(cond.buffers[0], cond.offset, cond.length);
+      break;
+    case COND_ALL_VALID | RIGHT_ALL_VALID:
+      // bitmaps[C_VALID], bitmaps[R_VALID] might be null; override to make it safe for
+      // Visit()
+      bitmaps[C_VALID] = bitmaps[C_DATA];
+      bitmaps[R_VALID] = bitmaps[C_DATA];
+      Bitmap::VisitWords(bitmaps, [&](std::array<uint64_t, 4> words) {
+        apply(UINT64_MAX, words[C_DATA], words[L_VALID], UINT64_MAX);
+      });
+      break;
+    case RIGHT_ALL_VALID:
+      // bitmaps[R_VALID] might be null; override to make it safe for Visit()
+      bitmaps[R_VALID] = bitmaps[C_DATA];
+      Bitmap::VisitWords(bitmaps, [&](std::array<uint64_t, 4> words) {
+        apply(words[C_VALID], words[C_DATA], words[L_VALID], UINT64_MAX);
+      });
+      break;
+    case COND_ALL_VALID | LEFT_ALL_VALID:
+      // bitmaps[C_VALID], bitmaps[L_VALID] might be null; override to make it safe for
+      // Visit()
+      bitmaps[C_VALID] = bitmaps[C_DATA];
+      bitmaps[L_VALID] = bitmaps[C_DATA];
+      Bitmap::VisitWords(bitmaps, [&](std::array<uint64_t, 4> words) {
+        apply(UINT64_MAX, words[C_DATA], UINT64_MAX, words[R_VALID]);
+      });
+      break;
+    case LEFT_ALL_VALID:
+      // bitmaps[L_VALID] might be null; override to make it safe for Visit()
+      bitmaps[L_VALID] = bitmaps[C_DATA];
+      Bitmap::VisitWords(bitmaps, [&](std::array<uint64_t, 4> words) {
+        apply(words[C_VALID], words[C_DATA], UINT64_MAX, words[R_VALID]);
+      });
+      break;
+    case COND_ALL_VALID:
+      // bitmaps[C_VALID] might be null; override to make it safe for Visit()
+      bitmaps[C_VALID] = bitmaps[C_DATA];
+      Bitmap::VisitWords(bitmaps, [&](std::array<uint64_t, 4> words) {
+        apply(UINT64_MAX, words[C_DATA], words[L_VALID], words[R_VALID]);
+      });
+      break;
+    case 0:
+      Bitmap::VisitWords(bitmaps, [&](std::array<uint64_t, 4> words) {
+        apply(words[C_VALID], words[C_DATA], words[L_VALID], words[R_VALID]);
+      });
+      break;
+  }
+  return Status::OK();
+}
+
+template <typename Type, typename Enable = void>
+struct IfElseFunctor {};
+
+// only number types needs to be handled for Fixed sized primitive data types because,
+// internal::GenerateTypeAgnosticPrimitive forwards types to the corresponding unsigned
+// int type
+template <typename Type>
+struct IfElseFunctor<Type, enable_if_number<Type>> {
+  using T = typename TypeTraits<Type>::CType;
+  // A - Array
+  // S - Scalar
+
+  //  AAA
+  static Status Call(KernelContext* ctx, const ArrayData& cond, const ArrayData& left,
+                     const ArrayData& right, ArrayData* out) {
+    ARROW_RETURN_NOT_OK(PromoteNullsVisitor(ctx, cond, left, right, out));
+
+    ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Buffer> out_buf,
+                          ctx->Allocate(cond.length * sizeof(T)));
+    T* out_values = reinterpret_cast<T*>(out_buf->mutable_data());
+
+    // copy right data to out_buff
+    const T* right_data = right.GetValues<T>(1);
+    std::memcpy(out_values, right_data, right.length * sizeof(T));
+
+    const auto* cond_data = cond.buffers[1]->data();  // this is a BoolArray
+    BitBlockCounter bit_counter(cond_data, cond.offset, cond.length);
+
+    // selectively copy values from left data
+    const T* left_data = left.GetValues<T>(1);
+    int64_t offset = cond.offset;
+
+    // todo this can be improved by intrinsics. ex: _mm*_mask_store_e* (vmovdqa*)
+    while (offset < cond.offset + cond.length) {
+      const BitBlockCount& block = bit_counter.NextWord();
+      if (block.AllSet()) {  // all from left
+        std::memcpy(out_values, left_data, block.length * sizeof(T));
+      } else if (block.popcount) {  // selectively copy from left
+        for (int64_t i = 0; i < block.length; ++i) {
+          if (BitUtil::GetBit(cond_data, offset + i)) {
+            out_values[i] = left_data[i];
+          }
+        }
+      }
+
+      offset += block.length;
+      out_values += block.length;
+      left_data += block.length;
+    }
+
+    out->buffers[1] = std::move(out_buf);
+    return Status::OK();
+  }
+
+  // ASA
+  static Status Call(KernelContext* ctx, const ArrayData& cond, const Scalar& left,
+                     const ArrayData& right, ArrayData* out) {
+    ARROW_RETURN_NOT_OK(PromoteNullsVisitor(ctx, cond, left, right, out));
+
+    ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Buffer> out_buf,
+                          ctx->Allocate(cond.length * sizeof(T)));
+    T* out_values = reinterpret_cast<T*>(out_buf->mutable_data());
+
+    // copy right data to out_buff
+    const T* right_data = right.GetValues<T>(1);
+    std::memcpy(out_values, right_data, right.length * sizeof(T));
+
+    const auto* cond_data = cond.buffers[1]->data();  // this is a BoolArray
+    BitBlockCounter bit_counter(cond_data, cond.offset, cond.length);
+
+    // selectively copy values from left data
+    T left_data = internal::UnboxScalar<Type>::Unbox(left);
+    int64_t offset = cond.offset;
+
+    // todo this can be improved by intrinsics. ex: _mm*_mask_store_e* (vmovdqa*)
+    while (offset < cond.offset + cond.length) {
+      const BitBlockCount& block = bit_counter.NextWord();
+      if (block.AllSet()) {  // all from left
+        std::fill(out_values, out_values + block.length, left_data);
+      } else if (block.popcount) {  // selectively copy from left
+        for (int64_t i = 0; i < block.length; ++i) {
+          if (BitUtil::GetBit(cond_data, offset + i)) {
+            out_values[i] = left_data;
+          }
+        }
+      }
+
+      offset += block.length;
+      out_values += block.length;
+    }
+
+    out->buffers[1] = std::move(out_buf);
+    return Status::OK();
+  }
+
+  // AAS
+  static Status Call(KernelContext* ctx, const ArrayData& cond, const ArrayData& left,
+                     const Scalar& right, ArrayData* out) {
+    ARROW_RETURN_NOT_OK(PromoteNullsVisitor(ctx, cond, left, right, out));
+
+    ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Buffer> out_buf,
+                          ctx->Allocate(cond.length * sizeof(T)));
+    T* out_values = reinterpret_cast<T*>(out_buf->mutable_data());
+
+    // copy left data to out_buff
+    const T* left_data = left.GetValues<T>(1);
+    std::memcpy(out_values, left_data, left.length * sizeof(T));
+
+    const auto* cond_data = cond.buffers[1]->data();  // this is a BoolArray
+    BitBlockCounter bit_counter(cond_data, cond.offset, cond.length);
+
+    // selectively copy values from left data
+    T right_data = internal::UnboxScalar<Type>::Unbox(right);
+    int64_t offset = cond.offset;
+
+    // todo this can be improved by intrinsics. ex: _mm*_mask_store_e* (vmovdqa*)
+    // left data is already in the output buffer. Therefore, mask needs to be inverted
+    while (offset < cond.offset + cond.length) {
+      const BitBlockCount& block = bit_counter.NextWord();
+      if (block.NoneSet()) {  // all from right
+        std::fill(out_values, out_values + block.length, right_data);
+      } else if (block.popcount) {  // selectively copy from right
+        for (int64_t i = 0; i < block.length; ++i) {
+          if (!BitUtil::GetBit(cond_data, offset + i)) {
+            out_values[i] = right_data;
+          }
+        }
+      }
+
+      offset += block.length;
+      out_values += block.length;
+    }
+
+    out->buffers[1] = std::move(out_buf);
+    return Status::OK();
+  }
+
+  // ASS
+  static Status Call(KernelContext* ctx, const ArrayData& cond, const Scalar& left,
+                     const Scalar& right, ArrayData* out) {
+    ARROW_RETURN_NOT_OK(PromoteNullsVisitor(ctx, cond, left, right, out));
+
+    ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Buffer> out_buf,
+                          ctx->Allocate(cond.length * sizeof(T)));
+    T* out_values = reinterpret_cast<T*>(out_buf->mutable_data());
+
+    // copy right data to out_buff
+    T right_data = internal::UnboxScalar<Type>::Unbox(right);
+    std::fill(out_values, out_values + cond.length, right_data);
+
+    const auto* cond_data = cond.buffers[1]->data();  // this is a BoolArray
+    BitBlockCounter bit_counter(cond_data, cond.offset, cond.length);
+
+    // selectively copy values from left data
+    T left_data = internal::UnboxScalar<Type>::Unbox(left);
+    int64_t offset = cond.offset;
+
+    // todo this can be improved by intrinsics. ex: _mm*_mask_store_e* (vmovdqa*)
+    while (offset < cond.offset + cond.length) {
+      const BitBlockCount& block = bit_counter.NextWord();
+      if (block.AllSet()) {  // all from left
+        std::fill(out_values, out_values + block.length, left_data);
+      } else if (block.popcount) {  // selectively copy from left
+        for (int64_t i = 0; i < block.length; ++i) {
+          if (BitUtil::GetBit(cond_data, offset + i)) {
+            out_values[i] = left_data;
+          }
+        }
+      }
+
+      offset += block.length;
+      out_values += block.length;
+    }
+
+    out->buffers[1] = std::move(out_buf);
+    return Status::OK();
+  }
+};
+
+template <typename Type>
+struct IfElseFunctor<Type, enable_if_boolean<Type>> {
+  // AAA
+  static Status Call(KernelContext* ctx, const ArrayData& cond, const ArrayData& left,
+                     const ArrayData& right, ArrayData* out) {
+    ARROW_RETURN_NOT_OK(PromoteNullsVisitor(ctx, cond, left, right, out));
+
+    // out_buff = right & ~cond
+    ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Buffer> out_buf,
+                          arrow::internal::BitmapAndNot(
+                              ctx->memory_pool(), right.buffers[1]->data(), right.offset,
+                              cond.buffers[1]->data(), cond.offset, cond.length, 0));
+
+    // out_buff = left & cond
+    ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Buffer> temp_buf,
+                          arrow::internal::BitmapAnd(
+                              ctx->memory_pool(), left.buffers[1]->data(), left.offset,
+                              cond.buffers[1]->data(), cond.offset, cond.length, 0));
+
+    arrow::internal::BitmapOr(out_buf->data(), 0, temp_buf->data(), 0, cond.length, 0,
+                              out_buf->mutable_data());
+    out->buffers[1] = std::move(out_buf);
+    return Status::OK();
+  }
+
+  // ASA
+  static Status Call(KernelContext* ctx, const ArrayData& cond, const Scalar& left,
+                     const ArrayData& right, ArrayData* out) {
+    ARROW_RETURN_NOT_OK(PromoteNullsVisitor(ctx, cond, left, right, out));
+
+    // out_buff = right & ~cond
+    ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Buffer> out_buf,
+                          arrow::internal::BitmapAndNot(
+                              ctx->memory_pool(), right.buffers[1]->data(), right.offset,
+                              cond.buffers[1]->data(), cond.offset, cond.length, 0));
+
+    // out_buff = left & cond
+    bool left_data = internal::UnboxScalar<BooleanType>::Unbox(left);
+    if (left_data) {
+      arrow::internal::BitmapOr(out_buf->data(), 0, cond.buffers[1]->data(), cond.offset,
+                                cond.length, 0, out_buf->mutable_data());
+    }
+
+    out->buffers[1] = std::move(out_buf);
+    return Status::OK();
+  }
+
+  // AAS
+  static Status Call(KernelContext* ctx, const ArrayData& cond, const ArrayData& left,
+                     const Scalar& right, ArrayData* out) {
+    ARROW_RETURN_NOT_OK(PromoteNullsVisitor(ctx, cond, left, right, out));
+
+    // out_buff = left & cond
+    ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Buffer> out_buf,
+                          arrow::internal::BitmapAnd(
+                              ctx->memory_pool(), left.buffers[1]->data(), left.offset,
+                              cond.buffers[1]->data(), cond.offset, cond.length, 0));
+
+    bool right_data = internal::UnboxScalar<BooleanType>::Unbox(right);
+
+    // out_buff = left & cond | right & ~cond
+    if (right_data) {
+      arrow::internal::BitmapOrNot(out_buf->data(), 0, cond.buffers[1]->data(),
+                                   cond.offset, cond.length, 0, out_buf->mutable_data());
+    }
+
+    out->buffers[1] = std::move(out_buf);
+    return Status::OK();
+  }
+
+  // ASS
+  static Status Call(KernelContext* ctx, const ArrayData& cond, const Scalar& left,
+                     const Scalar& right, ArrayData* out) {
+    ARROW_RETURN_NOT_OK(PromoteNullsVisitor(ctx, cond, left, right, out));
+
+    bool left_data = internal::UnboxScalar<BooleanType>::Unbox(left);
+    bool right_data = internal::UnboxScalar<BooleanType>::Unbox(right);
+
+    // out_buf = left & cond | right & ~cond
+    std::shared_ptr<Buffer> out_buf = nullptr;
+    if (left_data) {
+      if (right_data) {
+        // out_buf = ones
+        ARROW_ASSIGN_OR_RAISE(out_buf, ctx->AllocateBitmap(cond.length));
+        // filling with UINT8_MAX upto the buffer's size (in bytes)
+        arrow::compute::internal::SetMemory<UINT8_MAX>(out_buf.get());
+      } else {
+        // out_buf = cond
+        out_buf = SliceBuffer(cond.buffers[1], cond.offset, cond.length);
+      }
+    } else {
+      if (right_data) {
+        // out_buf = ~cond
+        ARROW_ASSIGN_OR_RAISE(out_buf, arrow::internal::InvertBitmap(
+                                           ctx->memory_pool(), cond.buffers[1]->data(),
+                                           cond.offset, cond.length))
+      } else {
+        // out_buf = zeros
+        ARROW_ASSIGN_OR_RAISE(out_buf, ctx->AllocateBitmap(cond.length));
+      }
+    }
+    out->buffers[1] = std::move(out_buf);
+    return Status::OK();
+  }
+};
+
+template <typename Type>
+struct IfElseFunctor<Type, enable_if_null<Type>> {
+  template <typename T>
+  static inline Status ReturnCopy(const T& in, T* out) {
+    // Nothing preallocated, so we assign in into the output
+    *out = in;
+    return Status::OK();
+  }
+
+  // AAA
+  static Status Call(KernelContext* ctx, const ArrayData& cond, const ArrayData& left,
+                     const ArrayData& right, ArrayData* out) {
+    return ReturnCopy(left, out);
+  }
+
+  // ASA
+  static Status Call(KernelContext* ctx, const ArrayData& cond, const Scalar& left,
+                     const ArrayData& right, ArrayData* out) {
+    return ReturnCopy(right, out);
+  }
+
+  // AAS
+  static Status Call(KernelContext* ctx, const ArrayData& cond, const ArrayData& left,
+                     const Scalar& right, ArrayData* out) {
+    return ReturnCopy(left, out);
+  }
+
+  // ASS
+  static Status Call(KernelContext* ctx, const ArrayData& cond, const Scalar& left,
+                     const Scalar& right, ArrayData* out) {
+    return ReturnCopy(cond, out);
+  }
+};
+
+template <typename Type>
+struct ResolveIfElseExec {
+  static Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    // cond is scalar
+    if (batch[0].is_scalar()) {
+      const auto& cond = batch[0].scalar_as<BooleanScalar>();
+      if (batch[1].is_scalar() && batch[2].is_scalar()) {
+        if (cond.is_valid) {
+          *out = cond.value ? batch[1].scalar() : batch[2].scalar();
+        } else {
+          *out = MakeNullScalar(batch[1].type());
+        }
+      } else {  // either left or right is an array. output is always an array
+        // output size is the size of the array arg
+        int64_t bcast_size = batch[1].is_array() ? batch[1].length() : batch[2].length();
+        if (cond.is_valid) {
+          const auto& valid_data = cond.value ? batch[1] : batch[2];
+          if (valid_data.is_array()) {
+            *out = valid_data;
+          } else {  // valid data is a scalar that needs to be broadcasted
+            ARROW_ASSIGN_OR_RAISE(*out,
+                                  MakeArrayFromScalar(*valid_data.scalar(), bcast_size,
+                                                      ctx->memory_pool()));
+          }
+        } else {  // cond is null. create null array
+          ARROW_ASSIGN_OR_RAISE(
+              *out, MakeArrayOfNull(batch[1].type(), bcast_size, ctx->memory_pool()))
+        }
+      }
+      return Status::OK();
+    }
+

Review comment:
       Instead of needing all those overloads for PropagateNulls, you could handle that here. Whether they are scalar or arrays, the salient point to null propagation is that each of these inputs may be entirely null (a null scalar or an array containing only nulls) or entirely valid (a valid scalar or an array with no nulls) or they have a null bitmap which needs to be visited. I'd write `util::optional<uint64_t> GetConstantValidityWord(const Datum&)` which you can apply to each input. If a a constant validity word can be created, you can use that directly in your visitor

##########
File path: cpp/src/arrow/compute/kernels/scalar_if_else.cc
##########
@@ -0,0 +1,882 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <arrow/compute/api.h>
+#include <arrow/compute/kernels/codegen_internal.h>
+#include <arrow/compute/util_internal.h>
+#include <arrow/util/bit_block_counter.h>
+#include <arrow/util/bitmap.h>
+#include <arrow/util/bitmap_ops.h>
+
+namespace arrow {
+using internal::BitBlockCount;
+using internal::BitBlockCounter;
+using internal::Bitmap;
+
+namespace compute {
+
+namespace {
+
+enum { COND_ALL_VALID = 1, LEFT_ALL_VALID = 2, RIGHT_ALL_VALID = 4 };
+
+// if the condition is null then output is null otherwise we take validity from the
+// selected argument
+// ie. cond.valid & (cond.data & left.valid | ~cond.data & right.valid)
+Status PromoteNullsVisitor(KernelContext* ctx, const ArrayData& cond, const Scalar& left,
+                           const Scalar& right, ArrayData* output) {
+  uint8_t flag = right.is_valid * 4 + left.is_valid * 2 + !cond.MayHaveNulls();

Review comment:
       ```suggestion
     uint8_t flag = right.is_valid * RIGHT_ALL_VALID |
                           left.is_valid * LEFT_ALL_VALID |
                            !cond.MayHaveNulls() * COND_ALL_VALID;
   ```




-- 
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



[GitHub] [arrow] nirandaperera commented on a change in pull request #10410: ARROW-10640: [C++] A "where" kernel to combine two arrays based on a mask

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_if_else_test.cc
##########
@@ -0,0 +1,474 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <arrow/array.h>
+#include <arrow/compute/api_scalar.h>
+#include <arrow/compute/kernels/test_util.h>
+#include <arrow/testing/gtest_util.h>
+#include <gtest/gtest.h>
+
+namespace arrow {
+namespace compute {
+
+void CheckIfElseOutput(const Datum& cond, const Datum& left, const Datum& right,
+                       const Datum& expected) {
+  ASSERT_OK_AND_ASSIGN(Datum datum_out, IfElse(cond, left, right));
+  if (datum_out.is_array()) {
+    std::shared_ptr<Array> result = datum_out.make_array();
+    ASSERT_OK(result->ValidateFull());
+    std::shared_ptr<Array> expected_ = expected.make_array();
+    AssertArraysEqual(*expected_, *result, /*verbose=*/true);
+  } else {  // expecting scalar
+    const std::shared_ptr<Scalar>& result = datum_out.scalar();
+    const std::shared_ptr<Scalar>& expected_ = expected.scalar();
+    AssertScalarsEqual(*expected_, *result, /*verbose=*/true);
+  }
+}
+
+void CheckIfElseOutputAAA(const std::shared_ptr<DataType>& type, const std::string& cond,
+                          const std::string& left, const std::string& right,
+                          const std::string& expected) {
+  const std::shared_ptr<Array>& cond_ = ArrayFromJSON(boolean(), cond);
+  const std::shared_ptr<Array>& left_ = ArrayFromJSON(type, left);
+  const std::shared_ptr<Array>& right_ = ArrayFromJSON(type, right);
+  const std::shared_ptr<Array>& expected_ = ArrayFromJSON(type, expected);
+  CheckIfElseOutput(cond_, left_, right_, expected_);
+}
+
+void CheckIfElseOutputAAS(const std::shared_ptr<DataType>& type, const std::string& cond,
+                          const std::string& left, const std::shared_ptr<Scalar>& right,
+                          const std::string& expected) {
+  const std::shared_ptr<Array>& cond_ = ArrayFromJSON(boolean(), cond);
+  const std::shared_ptr<Array>& left_ = ArrayFromJSON(type, left);
+  const std::shared_ptr<Array>& expected_ = ArrayFromJSON(type, expected);
+  CheckIfElseOutput(cond_, left_, right, expected_);
+}
+
+void CheckIfElseOutputASA(const std::shared_ptr<DataType>& type, const std::string& cond,
+                          const std::shared_ptr<Scalar>& left, const std::string& right,
+                          const std::string& expected) {
+  const std::shared_ptr<Array>& cond_ = ArrayFromJSON(boolean(), cond);
+  const std::shared_ptr<Array>& right_ = ArrayFromJSON(type, right);
+  const std::shared_ptr<Array>& expected_ = ArrayFromJSON(type, expected);
+  CheckIfElseOutput(cond_, left, right_, expected_);
+}
+
+void CheckIfElseOutputASS(const std::shared_ptr<DataType>& type, const std::string& cond,
+                          const std::shared_ptr<Scalar>& left,
+                          const std::shared_ptr<Scalar>& right,
+                          const std::string& expected) {
+  const std::shared_ptr<Array>& cond_ = ArrayFromJSON(boolean(), cond);
+  const std::shared_ptr<Array>& expected_ = ArrayFromJSON(type, expected);
+  CheckIfElseOutput(cond_, left, right, expected_);
+}
+
+void CheckIfElseOutputSAA(const std::shared_ptr<DataType>& type,
+                          const std::shared_ptr<Scalar>& cond, const std::string& left,
+                          const std::string& right, const std::string& expected) {
+  const std::shared_ptr<Array>& left_ = ArrayFromJSON(type, left);
+  const std::shared_ptr<Array>& right_ = ArrayFromJSON(type, right);
+  const std::shared_ptr<Array>& expected_ = ArrayFromJSON(type, expected);
+  CheckIfElseOutput(cond, left_, right_, expected_);
+}
+
+void CheckIfElseOutputSAS(const std::shared_ptr<DataType>& type,
+                          const std::shared_ptr<Scalar>& cond, const std::string& left,
+                          const std::shared_ptr<Scalar>& right,
+                          const std::string& expected) {
+  const std::shared_ptr<Array>& left_ = ArrayFromJSON(type, left);
+  const std::shared_ptr<Array>& expected_ = ArrayFromJSON(type, expected);
+  CheckIfElseOutput(cond, left_, right, expected_);
+}
+
+void CheckIfElseOutputSSA(const std::shared_ptr<DataType>& type,
+                          const std::shared_ptr<Scalar>& cond,
+                          const std::shared_ptr<Scalar>& left, const std::string& right,
+                          const std::string& expected) {
+  const std::shared_ptr<Array>& right_ = ArrayFromJSON(type, right);
+  const std::shared_ptr<Array>& expected_ = ArrayFromJSON(type, expected);
+  CheckIfElseOutput(cond, left, right_, expected_);
+}
+
+class TestIfElseKernel : public ::testing::Test {};
+
+template <typename Type>
+class TestIfElsePrimitive : public ::testing::Test {};
+
+using PrimitiveTypes = ::testing::Types<Int8Type, UInt8Type, Int16Type, UInt16Type,
+                                        Int32Type, UInt32Type, Int64Type, UInt64Type,
+                                        FloatType, DoubleType, Date32Type, Date64Type>;
+
+TYPED_TEST_SUITE(TestIfElsePrimitive, PrimitiveTypes);
+
+TYPED_TEST(TestIfElsePrimitive, IfElseFixedSizeRand) {
+  using ArrayType = typename TypeTraits<TypeParam>::ArrayType;
+  auto type = TypeTraits<TypeParam>::type_singleton();
+
+  random::RandomArrayGenerator rand(/*seed=*/0);
+  int64_t len = 1000;
+  auto cond = std::static_pointer_cast<BooleanArray>(
+      rand.ArrayOf(boolean(), len, /*null_probability=*/0.01));
+  auto left = std::static_pointer_cast<ArrayType>(
+      rand.ArrayOf(type, len, /*null_probability=*/0.01));
+  auto right = std::static_pointer_cast<ArrayType>(
+      rand.ArrayOf(type, len, /*null_probability=*/0.01));
+
+  typename TypeTraits<TypeParam>::BuilderType builder;
+
+  for (int64_t i = 0; i < len; ++i) {
+    if (!cond->IsValid(i) || (cond->Value(i) && !left->IsValid(i)) ||
+        (!cond->Value(i) && !right->IsValid(i))) {
+      ASSERT_OK(builder.AppendNull());
+      continue;
+    }
+
+    if (cond->Value(i)) {
+      ASSERT_OK(builder.Append(left->Value(i)));
+    } else {
+      ASSERT_OK(builder.Append(right->Value(i)));
+    }
+  }
+  ASSERT_OK_AND_ASSIGN(auto expected_data, builder.Finish());
+
+  CheckIfElseOutput(cond, left, right, expected_data);
+}
+
+#define IF_ELSE_TEST_GEN(type, l0, l1, l2, l3, r0, r1, r2, r3, valid, valid1)            \

Review comment:
       I rewrote the test cases in a more generic way. Could you please take a look at 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.

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



[GitHub] [arrow] nirandaperera commented on a change in pull request #10410: ARROW-10640: [C++] A "where" kernel to combine two arrays based on a mask

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_if_else_test.cc
##########
@@ -0,0 +1,266 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <arrow/array.h>
+#include <arrow/compute/api_scalar.h>
+#include <arrow/compute/kernels/test_util.h>
+#include <arrow/testing/gtest_util.h>
+#include <gtest/gtest.h>
+
+namespace arrow {
+namespace compute {
+
+void CheckIfElseOutput(const Datum& cond, const Datum& left, const Datum& right,
+                       const Datum& expected) {
+  ASSERT_OK_AND_ASSIGN(Datum datum_out, IfElse(cond, left, right));
+  if (datum_out.is_array()) {
+    std::shared_ptr<Array> result = datum_out.make_array();
+    ASSERT_OK(result->ValidateFull());
+    std::shared_ptr<Array> expected_ = expected.make_array();
+    AssertArraysEqual(*expected_, *result, /*verbose=*/true);
+  } else {  // expecting scalar
+    const std::shared_ptr<Scalar>& result = datum_out.scalar();
+    const std::shared_ptr<Scalar>& expected_ = expected.scalar();
+    AssertScalarsEqual(*expected_, *result, /*verbose=*/true);
+  }
+}
+
+class TestIfElseKernel : public ::testing::Test {};
+
+template <typename Type>
+class TestIfElsePrimitive : public ::testing::Test {};
+
+using PrimitiveTypes = ::testing::Types<Int8Type, UInt8Type, Int16Type, UInt16Type,
+                                        Int32Type, UInt32Type, Int64Type, UInt64Type,
+                                        FloatType, DoubleType, Date32Type, Date64Type>;
+
+TYPED_TEST_SUITE(TestIfElsePrimitive, PrimitiveTypes);
+
+TYPED_TEST(TestIfElsePrimitive, IfElseFixedSizeRand) {
+  using ArrayType = typename TypeTraits<TypeParam>::ArrayType;
+  auto type = TypeTraits<TypeParam>::type_singleton();
+
+  random::RandomArrayGenerator rand(/*seed=*/0);
+  int64_t len = 1000;
+  auto cond = std::static_pointer_cast<BooleanArray>(
+      rand.ArrayOf(boolean(), len, /*null_probability=*/0.01));
+  auto left = std::static_pointer_cast<ArrayType>(
+      rand.ArrayOf(type, len, /*null_probability=*/0.01));
+  auto right = std::static_pointer_cast<ArrayType>(
+      rand.ArrayOf(type, len, /*null_probability=*/0.01));
+
+  typename TypeTraits<TypeParam>::BuilderType builder;
+
+  for (int64_t i = 0; i < len; ++i) {
+    if (!cond->IsValid(i) || (cond->Value(i) && !left->IsValid(i)) ||
+        (!cond->Value(i) && !right->IsValid(i))) {
+      ASSERT_OK(builder.AppendNull());
+      continue;
+    }
+
+    if (cond->Value(i)) {
+      ASSERT_OK(builder.Append(left->Value(i)));
+    } else {
+      ASSERT_OK(builder.Append(right->Value(i)));
+    }
+  }
+  ASSERT_OK_AND_ASSIGN(auto expected_data, builder.Finish());
+
+  CheckIfElseOutput(cond, left, right, expected_data);
+}
+
+template <typename Type>
+struct DatumWrapper {
+  using CType = typename TypeTraits<Type>::CType;
+  using ArrayType = typename TypeTraits<Type>::ArrayType;
+  using ScalarType = typename TypeTraits<Type>::ScalarType;
+
+  util::Variant<std::shared_ptr<ScalarType>, std::shared_ptr<ArrayType>> datum;
+  bool is_scalar;
+
+  explicit DatumWrapper(const Datum& datum_) : is_scalar(datum_.is_scalar()) {
+    if (is_scalar) {
+      datum = std::move(std::static_pointer_cast<ScalarType>(datum_.scalar()));
+    } else {
+      datum = std::move(std::static_pointer_cast<ArrayType>(datum_.make_array()));
+    }
+  }
+
+  bool IsValid(int64_t i) const {
+    return is_scalar ? util::get<std::shared_ptr<ScalarType>>(datum)->is_valid
+                     : util::get<std::shared_ptr<ArrayType>>(datum)->IsValid(i);
+  }
+
+  CType Value(int64_t i) const {
+    return is_scalar ? util::get<std::shared_ptr<ScalarType>>(datum)->value
+                     : util::get<std::shared_ptr<ArrayType>>(datum)->Value(i);
+  }
+};
+
+template <typename Type>
+void GenerateExpected(const Datum& cond, const Datum& left, const Datum& right,

Review comment:
       `GenerateExpected` generate both `Scalar` and  `Array` outputs  (Array part is L144 onward) 
   




-- 
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



[GitHub] [arrow] lidavidm commented on a change in pull request #10410: ARROW-10640: [C++] A "where" kernel to combine two arrays based on a mask

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_if_else.cc
##########
@@ -795,15 +649,15 @@ struct ResolveIfElseExec {
     // cond is scalar
     if (batch[0].is_scalar()) {
       const auto& cond = batch[0].scalar_as<BooleanScalar>();
-
       if (batch[1].is_scalar() && batch[2].is_scalar()) {
         if (cond.is_valid) {
           *out = cond.value ? batch[1].scalar() : batch[2].scalar();
         } else {
           *out = MakeNullScalar(batch[1].type());
         }
       } else {  // either left or right is an array. output is always an array
-        int64_t bcast_size = std::max(batch[1].length(), batch[2].length());

Review comment:
       You probably just want `batch.length` which will be the length of the array arguments if there is one

##########
File path: cpp/src/arrow/compute/kernels/scalar_if_else_test.cc
##########
@@ -0,0 +1,474 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <arrow/array.h>
+#include <arrow/compute/api_scalar.h>
+#include <arrow/compute/kernels/test_util.h>
+#include <arrow/testing/gtest_util.h>
+#include <gtest/gtest.h>
+
+namespace arrow {
+namespace compute {
+
+void CheckIfElseOutput(const Datum& cond, const Datum& left, const Datum& right,
+                       const Datum& expected) {
+  ASSERT_OK_AND_ASSIGN(Datum datum_out, IfElse(cond, left, right));
+  if (datum_out.is_array()) {
+    std::shared_ptr<Array> result = datum_out.make_array();
+    ASSERT_OK(result->ValidateFull());
+    std::shared_ptr<Array> expected_ = expected.make_array();
+    AssertArraysEqual(*expected_, *result, /*verbose=*/true);
+  } else {  // expecting scalar
+    const std::shared_ptr<Scalar>& result = datum_out.scalar();
+    const std::shared_ptr<Scalar>& expected_ = expected.scalar();
+    AssertScalarsEqual(*expected_, *result, /*verbose=*/true);
+  }
+}
+
+void CheckIfElseOutputAAA(const std::shared_ptr<DataType>& type, const std::string& cond,
+                          const std::string& left, const std::string& right,
+                          const std::string& expected) {
+  const std::shared_ptr<Array>& cond_ = ArrayFromJSON(boolean(), cond);
+  const std::shared_ptr<Array>& left_ = ArrayFromJSON(type, left);
+  const std::shared_ptr<Array>& right_ = ArrayFromJSON(type, right);
+  const std::shared_ptr<Array>& expected_ = ArrayFromJSON(type, expected);
+  CheckIfElseOutput(cond_, left_, right_, expected_);
+}
+
+void CheckIfElseOutputAAS(const std::shared_ptr<DataType>& type, const std::string& cond,
+                          const std::string& left, const std::shared_ptr<Scalar>& right,
+                          const std::string& expected) {
+  const std::shared_ptr<Array>& cond_ = ArrayFromJSON(boolean(), cond);
+  const std::shared_ptr<Array>& left_ = ArrayFromJSON(type, left);
+  const std::shared_ptr<Array>& expected_ = ArrayFromJSON(type, expected);
+  CheckIfElseOutput(cond_, left_, right, expected_);
+}
+
+void CheckIfElseOutputASA(const std::shared_ptr<DataType>& type, const std::string& cond,
+                          const std::shared_ptr<Scalar>& left, const std::string& right,
+                          const std::string& expected) {
+  const std::shared_ptr<Array>& cond_ = ArrayFromJSON(boolean(), cond);
+  const std::shared_ptr<Array>& right_ = ArrayFromJSON(type, right);
+  const std::shared_ptr<Array>& expected_ = ArrayFromJSON(type, expected);
+  CheckIfElseOutput(cond_, left, right_, expected_);
+}
+
+void CheckIfElseOutputASS(const std::shared_ptr<DataType>& type, const std::string& cond,
+                          const std::shared_ptr<Scalar>& left,
+                          const std::shared_ptr<Scalar>& right,
+                          const std::string& expected) {
+  const std::shared_ptr<Array>& cond_ = ArrayFromJSON(boolean(), cond);
+  const std::shared_ptr<Array>& expected_ = ArrayFromJSON(type, expected);
+  CheckIfElseOutput(cond_, left, right, expected_);
+}
+
+void CheckIfElseOutputSAA(const std::shared_ptr<DataType>& type,
+                          const std::shared_ptr<Scalar>& cond, const std::string& left,
+                          const std::string& right, const std::string& expected) {
+  const std::shared_ptr<Array>& left_ = ArrayFromJSON(type, left);
+  const std::shared_ptr<Array>& right_ = ArrayFromJSON(type, right);
+  const std::shared_ptr<Array>& expected_ = ArrayFromJSON(type, expected);
+  CheckIfElseOutput(cond, left_, right_, expected_);
+}
+
+void CheckIfElseOutputSAS(const std::shared_ptr<DataType>& type,
+                          const std::shared_ptr<Scalar>& cond, const std::string& left,
+                          const std::shared_ptr<Scalar>& right,
+                          const std::string& expected) {
+  const std::shared_ptr<Array>& left_ = ArrayFromJSON(type, left);
+  const std::shared_ptr<Array>& expected_ = ArrayFromJSON(type, expected);
+  CheckIfElseOutput(cond, left_, right, expected_);
+}
+
+void CheckIfElseOutputSSA(const std::shared_ptr<DataType>& type,
+                          const std::shared_ptr<Scalar>& cond,
+                          const std::shared_ptr<Scalar>& left, const std::string& right,
+                          const std::string& expected) {
+  const std::shared_ptr<Array>& right_ = ArrayFromJSON(type, right);
+  const std::shared_ptr<Array>& expected_ = ArrayFromJSON(type, expected);
+  CheckIfElseOutput(cond, left, right_, expected_);
+}
+
+class TestIfElseKernel : public ::testing::Test {};
+
+template <typename Type>
+class TestIfElsePrimitive : public ::testing::Test {};
+
+using PrimitiveTypes = ::testing::Types<Int8Type, UInt8Type, Int16Type, UInt16Type,
+                                        Int32Type, UInt32Type, Int64Type, UInt64Type,
+                                        FloatType, DoubleType, Date32Type, Date64Type>;
+
+TYPED_TEST_SUITE(TestIfElsePrimitive, PrimitiveTypes);
+
+TYPED_TEST(TestIfElsePrimitive, IfElseFixedSizeRand) {
+  using ArrayType = typename TypeTraits<TypeParam>::ArrayType;
+  auto type = TypeTraits<TypeParam>::type_singleton();
+
+  random::RandomArrayGenerator rand(/*seed=*/0);
+  int64_t len = 1000;
+  auto cond = std::static_pointer_cast<BooleanArray>(
+      rand.ArrayOf(boolean(), len, /*null_probability=*/0.01));
+  auto left = std::static_pointer_cast<ArrayType>(
+      rand.ArrayOf(type, len, /*null_probability=*/0.01));
+  auto right = std::static_pointer_cast<ArrayType>(
+      rand.ArrayOf(type, len, /*null_probability=*/0.01));
+
+  typename TypeTraits<TypeParam>::BuilderType builder;
+
+  for (int64_t i = 0; i < len; ++i) {
+    if (!cond->IsValid(i) || (cond->Value(i) && !left->IsValid(i)) ||
+        (!cond->Value(i) && !right->IsValid(i))) {
+      ASSERT_OK(builder.AppendNull());
+      continue;
+    }
+
+    if (cond->Value(i)) {
+      ASSERT_OK(builder.Append(left->Value(i)));
+    } else {
+      ASSERT_OK(builder.Append(right->Value(i)));
+    }
+  }
+  ASSERT_OK_AND_ASSIGN(auto expected_data, builder.Finish());
+
+  CheckIfElseOutput(cond, left, right, expected_data);
+}
+
+#define IF_ELSE_TEST_GEN(type, l0, l1, l2, l3, r0, r1, r2, r3, valid, valid1)            \

Review comment:
       Given you have so many cases, I think a better approach might be to do this programatically - make a `CheckIfElse` that takes in Datum arguments, and have it compare the result against a 'naive' implementation. Then loop and construct some small scalars/arrays to check all the possible combinations. You can combine that with a few handwritten tests for cases like empty arrays.

##########
File path: cpp/src/arrow/compute/kernels/scalar_if_else.cc
##########
@@ -0,0 +1,742 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <arrow/compute/api.h>
+#include <arrow/compute/kernels/codegen_internal.h>
+#include <arrow/compute/util_internal.h>
+#include <arrow/util/bit_block_counter.h>
+#include <arrow/util/bitmap.h>
+#include <arrow/util/bitmap_ops.h>
+
+namespace arrow {
+using internal::BitBlockCount;
+using internal::BitBlockCounter;
+using internal::Bitmap;
+
+namespace compute {
+
+namespace {
+
+enum { COND_ALL_VALID = 1, LEFT_ALL_VALID = 2, RIGHT_ALL_VALID = 4 };
+
+// if the condition is null then output is null otherwise we take validity from the
+// selected argument
+// ie. cond.valid & (cond.data & left.valid | ~cond.data & right.valid)
+Status PromoteNullsVisitor(KernelContext* ctx, const ArrayData& cond, const Scalar& left,
+                           const Scalar& right, ArrayData* output) {
+  uint8_t flag = right.is_valid * RIGHT_ALL_VALID |
+                        left.is_valid * LEFT_ALL_VALID |
+                         !cond.MayHaveNulls() * COND_ALL_VALID;
+
+  if (flag < 6 && flag != 3) {
+    // there will be a validity buffer in the output
+    ARROW_ASSIGN_OR_RAISE(output->buffers[0], ctx->AllocateBitmap(cond.length));
+  }
+
+  // if the condition is null then output is null otherwise we take validity from the
+  // selected argument
+  // ie. cond.valid & (cond.data & left.valid | ~cond.data & right.valid)
+  switch (flag) {
+    case COND_ALL_VALID | LEFT_ALL_VALID | RIGHT_ALL_VALID:  // = 7
+      break;
+    case LEFT_ALL_VALID | RIGHT_ALL_VALID:  // = 6
+      // out_valid = c_valid
+      output->buffers[0] = SliceBuffer(cond.buffers[0], cond.offset, cond.length);
+      break;
+    case COND_ALL_VALID | RIGHT_ALL_VALID:  // = 5
+      // out_valid = ~cond.data
+      arrow::internal::InvertBitmap(cond.buffers[1]->data(), cond.offset, cond.length,
+                                    output->buffers[0]->mutable_data(), 0);
+      break;
+    case RIGHT_ALL_VALID:  // = 4
+      // out_valid = c_valid & ~cond.data
+      arrow::internal::BitmapAndNot(cond.buffers[0]->data(), cond.offset,
+                                    cond.buffers[1]->data(), cond.offset, cond.length, 0,
+                                    output->buffers[0]->mutable_data());
+      break;
+    case COND_ALL_VALID | LEFT_ALL_VALID:  // = 3
+      // out_valid = cond.data
+      output->buffers[0] = SliceBuffer(cond.buffers[1], cond.offset, cond.length);
+      break;
+    case LEFT_ALL_VALID:  // = 2
+      // out_valid = cond.valid & cond.data
+      arrow::internal::BitmapAnd(cond.buffers[0]->data(), cond.offset,
+                                 cond.buffers[1]->data(), cond.offset, cond.length, 0,
+                                 output->buffers[0]->mutable_data());
+      break;
+    case COND_ALL_VALID:  // = 1
+      // out_valid = 0 --> nothing to do; but requires out_valid to be a all-zero buffer
+      break;
+    case 0:
+      // out_valid = 0 --> nothing to do; but requires out_valid to be a all-zero buffer
+      break;
+  }
+  return Status::OK();
+}
+
+Status PromoteNullsVisitor(KernelContext* ctx, const ArrayData& cond,
+                           const ArrayData& left, const Scalar& right,
+                           ArrayData* output) {
+  uint8_t flag = right.is_valid * 4 + !left.MayHaveNulls() * 2 + !cond.MayHaveNulls();
+
+  enum { C_VALID, C_DATA, L_VALID };
+
+  Bitmap bitmaps[3];
+  bitmaps[C_VALID] = {cond.buffers[0], cond.offset, cond.length};
+  bitmaps[C_DATA] = {cond.buffers[1], cond.offset, cond.length};
+  bitmaps[L_VALID] = {left.buffers[0], left.offset, left.length};
+
+  uint64_t* out_validity = nullptr;
+  if (flag < 6 && flag != 3) {
+    // there will be a validity buffer in the output
+    ARROW_ASSIGN_OR_RAISE(output->buffers[0], ctx->AllocateBitmap(cond.length));
+    out_validity = output->GetMutableValues<uint64_t>(0);
+  }
+
+  // lambda function that will be used inside the visitor
+  int64_t i = 0;
+  auto apply = [&](uint64_t c_valid, uint64_t c_data, uint64_t l_valid,
+                   uint64_t r_valid) {
+    out_validity[i] = c_valid & ((c_data & l_valid) | (~c_data & r_valid));
+    i++;
+  };
+
+  // if the condition is null then output is null otherwise we take validity from the
+  // selected argument
+  // ie. cond.valid & (cond.data & left.valid | ~cond.data & right.valid)
+  switch (flag) {
+    case COND_ALL_VALID | LEFT_ALL_VALID | RIGHT_ALL_VALID:
+      break;
+    case LEFT_ALL_VALID | RIGHT_ALL_VALID:
+      output->buffers[0] = SliceBuffer(cond.buffers[0], cond.offset, cond.length);
+      break;
+    case COND_ALL_VALID | RIGHT_ALL_VALID:
+      // bitmaps[C_VALID] might be null; override to make it safe for Visit()
+      bitmaps[C_VALID] = bitmaps[C_DATA];
+      Bitmap::VisitWords(bitmaps, [&](std::array<uint64_t, 3> words) {
+        apply(UINT64_MAX, words[C_DATA], words[L_VALID], UINT64_MAX);
+      });
+      break;
+    case RIGHT_ALL_VALID:
+      Bitmap::VisitWords(bitmaps, [&](std::array<uint64_t, 3> words) {
+        apply(words[C_VALID], words[C_DATA], words[L_VALID], UINT64_MAX);
+      });
+      break;
+    case COND_ALL_VALID | LEFT_ALL_VALID:
+      // only cond.data is passed
+      output->buffers[0] = SliceBuffer(cond.buffers[1], cond.offset, cond.length);
+      break;
+    case LEFT_ALL_VALID:
+      // out_valid = cond.valid & cond.data
+      arrow::internal::BitmapAnd(cond.buffers[0]->data(), cond.offset,
+                                 cond.buffers[1]->data(), cond.offset, cond.length, 0,
+                                 output->buffers[0]->mutable_data());
+      break;
+    case COND_ALL_VALID:
+      // out_valid = cond.data & left.valid
+      arrow::internal::BitmapAnd(cond.buffers[1]->data(), cond.offset,
+                                 left.buffers[0]->data(), left.offset, cond.length, 0,
+                                 output->buffers[0]->mutable_data());
+      break;
+    case 0:
+      Bitmap::VisitWords(bitmaps, [&](std::array<uint64_t, 3> words) {
+        apply(words[C_VALID], words[C_DATA], words[L_VALID], 0);
+      });
+      break;
+  }
+  return Status::OK();
+}
+
+// if the condition is null then output is null otherwise we take validity from the
+// selected argument
+// ie. cond.valid & (cond.data & left.valid | ~cond.data & right.valid)
+Status PromoteNullsVisitor(KernelContext* ctx, const ArrayData& cond, const Scalar& left,
+                           const ArrayData& right, ArrayData* output) {
+  uint8_t flag = !right.MayHaveNulls() * 4 + left.is_valid * 2 + !cond.MayHaveNulls();
+
+  enum { C_VALID, C_DATA, R_VALID };
+
+  Bitmap bitmaps[3];
+  bitmaps[C_VALID] = {cond.buffers[0], cond.offset, cond.length};
+  bitmaps[C_DATA] = {cond.buffers[1], cond.offset, cond.length};
+  bitmaps[R_VALID] = {right.buffers[0], right.offset, right.length};
+
+  uint64_t* out_validity = nullptr;
+  if (flag < 6) {
+    // there will be a validity buffer in the output
+    ARROW_ASSIGN_OR_RAISE(output->buffers[0], ctx->AllocateBitmap(cond.length));
+    out_validity = output->GetMutableValues<uint64_t>(0);
+  }
+
+  // lambda function that will be used inside the visitor
+  int64_t i = 0;
+  auto apply = [&](uint64_t c_valid, uint64_t c_data, uint64_t l_valid,
+                   uint64_t r_valid) {
+    out_validity[i] = c_valid & ((c_data & l_valid) | (~c_data & r_valid));
+    i++;
+  };
+
+  // if the condition is null then output is null otherwise we take validity from the
+  // selected argument
+  // ie. cond.valid & (cond.data & left.valid | ~cond.data & right.valid)
+  switch (flag) {
+    case COND_ALL_VALID | LEFT_ALL_VALID | RIGHT_ALL_VALID:
+      break;
+    case LEFT_ALL_VALID | RIGHT_ALL_VALID:
+      output->buffers[0] = SliceBuffer(cond.buffers[0], cond.offset, cond.length);
+      break;
+    case COND_ALL_VALID | RIGHT_ALL_VALID:
+      // out_valid = ~cond.data
+      arrow::internal::InvertBitmap(cond.buffers[1]->data(), cond.offset, cond.length,
+                                    output->buffers[0]->mutable_data(), 0);
+      break;
+    case RIGHT_ALL_VALID:
+      // out_valid = c_valid & ~cond.data
+      arrow::internal::BitmapAndNot(cond.buffers[0]->data(), cond.offset,
+                                    cond.buffers[1]->data(), cond.offset, cond.length, 0,
+                                    output->buffers[0]->mutable_data());
+      break;
+    case COND_ALL_VALID | LEFT_ALL_VALID:
+      // bitmaps[C_VALID] might be null; override to make it safe for Visit()
+      bitmaps[C_VALID] = bitmaps[C_DATA];
+      Bitmap::VisitWords(bitmaps, [&](std::array<uint64_t, 3> words) {
+        apply(UINT64_MAX, words[C_DATA], UINT64_MAX, words[R_VALID]);
+      });
+      break;
+    case LEFT_ALL_VALID:
+      Bitmap::VisitWords(bitmaps, [&](std::array<uint64_t, 3> words) {
+        apply(words[C_VALID], words[C_DATA], UINT64_MAX, words[R_VALID]);
+      });
+      break;
+    case COND_ALL_VALID:
+      // out_valid =  ~cond.data & right.valid
+      arrow::internal::BitmapAndNot(right.buffers[0]->data(), right.offset,
+                                    cond.buffers[1]->data(), cond.offset, cond.length, 0,
+                                    output->buffers[0]->mutable_data());
+      break;
+    case 0:
+      Bitmap::VisitWords(bitmaps, [&](std::array<uint64_t, 3> words) {
+        apply(words[C_VALID], words[C_DATA], 0, words[R_VALID]);
+      });
+      break;
+  }
+  return Status::OK();
+}
+
+// if the condition is null then output is null otherwise we take validity from the
+// selected argument
+// ie. cond.valid & (cond.data & left.valid | ~cond.data & right.valid)
+Status PromoteNullsVisitor(KernelContext* ctx, const ArrayData& cond,
+                           const ArrayData& left, const ArrayData& right,
+                           ArrayData* output) {
+  uint8_t flag =
+      !right.MayHaveNulls() * 4 + !left.MayHaveNulls() * 2 + !cond.MayHaveNulls();
+
+  enum { C_VALID, C_DATA, L_VALID, R_VALID };
+
+  Bitmap bitmaps[4];
+  bitmaps[C_VALID] = {cond.buffers[0], cond.offset, cond.length};
+  bitmaps[C_DATA] = {cond.buffers[1], cond.offset, cond.length};
+  bitmaps[L_VALID] = {left.buffers[0], left.offset, left.length};
+  bitmaps[R_VALID] = {right.buffers[0], right.offset, right.length};
+
+  uint64_t* out_validity = nullptr;
+  if (flag < 6) {
+    // there will be a validity buffer in the output
+    ARROW_ASSIGN_OR_RAISE(output->buffers[0], ctx->AllocateBitmap(cond.length));
+    out_validity = output->GetMutableValues<uint64_t>(0);
+  }
+
+  // lambda function that will be used inside the visitor
+  int64_t i = 0;
+  auto apply = [&](uint64_t c_valid, uint64_t c_data, uint64_t l_valid,
+                   uint64_t r_valid) {
+    out_validity[i] = c_valid & ((c_data & l_valid) | (~c_data & r_valid));
+    i++;
+  };
+
+  // if the condition is null then output is null otherwise we take validity from the
+  // selected argument
+  // ie. cond.valid & (cond.data & left.valid | ~cond.data & right.valid)
+  switch (flag) {
+    case COND_ALL_VALID | LEFT_ALL_VALID | RIGHT_ALL_VALID:
+      break;
+    case LEFT_ALL_VALID | RIGHT_ALL_VALID:
+      output->buffers[0] = SliceBuffer(cond.buffers[0], cond.offset, cond.length);
+      break;
+    case COND_ALL_VALID | RIGHT_ALL_VALID:
+      // bitmaps[C_VALID], bitmaps[R_VALID] might be null; override to make it safe for
+      // Visit()
+      bitmaps[C_VALID] = bitmaps[C_DATA];
+      bitmaps[R_VALID] = bitmaps[C_DATA];
+      Bitmap::VisitWords(bitmaps, [&](std::array<uint64_t, 4> words) {
+        apply(UINT64_MAX, words[C_DATA], words[L_VALID], UINT64_MAX);
+      });
+      break;
+    case RIGHT_ALL_VALID:
+      // bitmaps[R_VALID] might be null; override to make it safe for Visit()
+      bitmaps[R_VALID] = bitmaps[C_DATA];
+      Bitmap::VisitWords(bitmaps, [&](std::array<uint64_t, 4> words) {
+        apply(words[C_VALID], words[C_DATA], words[L_VALID], UINT64_MAX);
+      });
+      break;
+    case COND_ALL_VALID | LEFT_ALL_VALID:
+      // bitmaps[C_VALID], bitmaps[L_VALID] might be null; override to make it safe for
+      // Visit()
+      bitmaps[C_VALID] = bitmaps[C_DATA];
+      bitmaps[L_VALID] = bitmaps[C_DATA];
+      Bitmap::VisitWords(bitmaps, [&](std::array<uint64_t, 4> words) {
+        apply(UINT64_MAX, words[C_DATA], UINT64_MAX, words[R_VALID]);
+      });
+      break;
+    case LEFT_ALL_VALID:
+      // bitmaps[L_VALID] might be null; override to make it safe for Visit()
+      bitmaps[L_VALID] = bitmaps[C_DATA];
+      Bitmap::VisitWords(bitmaps, [&](std::array<uint64_t, 4> words) {
+        apply(words[C_VALID], words[C_DATA], UINT64_MAX, words[R_VALID]);
+      });
+      break;
+    case COND_ALL_VALID:
+      // bitmaps[C_VALID] might be null; override to make it safe for Visit()
+      bitmaps[C_VALID] = bitmaps[C_DATA];
+      Bitmap::VisitWords(bitmaps, [&](std::array<uint64_t, 4> words) {
+        apply(UINT64_MAX, words[C_DATA], words[L_VALID], words[R_VALID]);
+      });
+      break;
+    case 0:
+      Bitmap::VisitWords(bitmaps, [&](std::array<uint64_t, 4> words) {
+        apply(words[C_VALID], words[C_DATA], words[L_VALID], words[R_VALID]);
+      });
+      break;
+  }
+  return Status::OK();
+}
+
+template <typename Type, typename Enable = void>
+struct IfElseFunctor {};
+
+// only number types needs to be handled for Fixed sized primitive data types because,
+// internal::GenerateTypeAgnosticPrimitive forwards types to the corresponding unsigned
+// int type
+template <typename Type>
+struct IfElseFunctor<Type, enable_if_number<Type>> {
+  using T = typename TypeTraits<Type>::CType;
+  // A - Array
+  // S - Scalar
+
+  //  AAA
+  static Status Call(KernelContext* ctx, const ArrayData& cond, const ArrayData& left,
+                     const ArrayData& right, ArrayData* out) {
+    ARROW_RETURN_NOT_OK(PromoteNullsVisitor(ctx, cond, left, right, out));
+
+    ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Buffer> out_buf,
+                          ctx->Allocate(cond.length * sizeof(T)));
+    T* out_values = reinterpret_cast<T*>(out_buf->mutable_data());
+
+    // copy right data to out_buff
+    const T* right_data = right.GetValues<T>(1);
+    std::memcpy(out_values, right_data, right.length * sizeof(T));
+
+    const auto* cond_data = cond.buffers[1]->data();  // this is a BoolArray
+    BitBlockCounter bit_counter(cond_data, cond.offset, cond.length);
+
+    // selectively copy values from left data
+    const T* left_data = left.GetValues<T>(1);
+    int64_t offset = cond.offset;
+
+    // todo this can be improved by intrinsics. ex: _mm*_mask_store_e* (vmovdqa*)
+    while (offset < cond.offset + cond.length) {
+      const BitBlockCount& block = bit_counter.NextWord();
+      if (block.AllSet()) {  // all from left
+        std::memcpy(out_values, left_data, block.length * sizeof(T));
+      } else if (block.popcount) {  // selectively copy from left
+        for (int64_t i = 0; i < block.length; ++i) {
+          if (BitUtil::GetBit(cond_data, offset + i)) {
+            out_values[i] = left_data[i];
+          }
+        }
+      }
+
+      offset += block.length;
+      out_values += block.length;
+      left_data += block.length;
+    }
+
+    out->buffers[1] = std::move(out_buf);
+    return Status::OK();
+  }
+
+  // ASA
+  static Status Call(KernelContext* ctx, const ArrayData& cond, const Scalar& left,
+                     const ArrayData& right, ArrayData* out) {
+    ARROW_RETURN_NOT_OK(PromoteNullsVisitor(ctx, cond, left, right, out));
+
+    ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Buffer> out_buf,
+                          ctx->Allocate(cond.length * sizeof(T)));
+    T* out_values = reinterpret_cast<T*>(out_buf->mutable_data());
+
+    // copy right data to out_buff
+    const T* right_data = right.GetValues<T>(1);
+    std::memcpy(out_values, right_data, right.length * sizeof(T));
+
+    const auto* cond_data = cond.buffers[1]->data();  // this is a BoolArray
+    BitBlockCounter bit_counter(cond_data, cond.offset, cond.length);
+
+    // selectively copy values from left data
+    T left_data = internal::UnboxScalar<Type>::Unbox(left);
+    int64_t offset = cond.offset;
+
+    // todo this can be improved by intrinsics. ex: _mm*_mask_store_e* (vmovdqa*)
+    while (offset < cond.offset + cond.length) {
+      const BitBlockCount& block = bit_counter.NextWord();
+      if (block.AllSet()) {  // all from left
+        std::fill(out_values, out_values + block.length, left_data);
+      } else if (block.popcount) {  // selectively copy from left
+        for (int64_t i = 0; i < block.length; ++i) {
+          if (BitUtil::GetBit(cond_data, offset + i)) {
+            out_values[i] = left_data;
+          }
+        }
+      }
+
+      offset += block.length;
+      out_values += block.length;
+    }
+
+    out->buffers[1] = std::move(out_buf);
+    return Status::OK();
+  }
+
+  // AAS
+  static Status Call(KernelContext* ctx, const ArrayData& cond, const ArrayData& left,
+                     const Scalar& right, ArrayData* out) {
+    ARROW_RETURN_NOT_OK(PromoteNullsVisitor(ctx, cond, left, right, out));
+
+    ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Buffer> out_buf,
+                          ctx->Allocate(cond.length * sizeof(T)));
+    T* out_values = reinterpret_cast<T*>(out_buf->mutable_data());
+
+    // copy left data to out_buff
+    const T* left_data = left.GetValues<T>(1);
+    std::memcpy(out_values, left_data, left.length * sizeof(T));
+
+    const auto* cond_data = cond.buffers[1]->data();  // this is a BoolArray
+    BitBlockCounter bit_counter(cond_data, cond.offset, cond.length);
+
+    // selectively copy values from left data
+    T right_data = internal::UnboxScalar<Type>::Unbox(right);
+    int64_t offset = cond.offset;
+
+    // todo this can be improved by intrinsics. ex: _mm*_mask_store_e* (vmovdqa*)
+    // left data is already in the output buffer. Therefore, mask needs to be inverted
+    while (offset < cond.offset + cond.length) {
+      const BitBlockCount& block = bit_counter.NextWord();
+      if (block.NoneSet()) {  // all from right
+        std::fill(out_values, out_values + block.length, right_data);
+      } else if (block.popcount) {  // selectively copy from right
+        for (int64_t i = 0; i < block.length; ++i) {
+          if (!BitUtil::GetBit(cond_data, offset + i)) {
+            out_values[i] = right_data;
+          }
+        }
+      }
+
+      offset += block.length;
+      out_values += block.length;
+    }
+
+    out->buffers[1] = std::move(out_buf);
+    return Status::OK();
+  }
+
+  // ASS
+  static Status Call(KernelContext* ctx, const ArrayData& cond, const Scalar& left,
+                     const Scalar& right, ArrayData* out) {
+    ARROW_RETURN_NOT_OK(PromoteNullsVisitor(ctx, cond, left, right, out));
+
+    ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Buffer> out_buf,
+                          ctx->Allocate(cond.length * sizeof(T)));
+    T* out_values = reinterpret_cast<T*>(out_buf->mutable_data());
+
+    // copy right data to out_buff
+    T right_data = internal::UnboxScalar<Type>::Unbox(right);
+    std::fill(out_values, out_values + cond.length, right_data);
+
+    const auto* cond_data = cond.buffers[1]->data();  // this is a BoolArray
+    BitBlockCounter bit_counter(cond_data, cond.offset, cond.length);
+
+    // selectively copy values from left data
+    T left_data = internal::UnboxScalar<Type>::Unbox(left);
+    int64_t offset = cond.offset;
+
+    // todo this can be improved by intrinsics. ex: _mm*_mask_store_e* (vmovdqa*)
+    while (offset < cond.offset + cond.length) {
+      const BitBlockCount& block = bit_counter.NextWord();
+      if (block.AllSet()) {  // all from left
+        std::fill(out_values, out_values + block.length, left_data);
+      } else if (block.popcount) {  // selectively copy from left
+        for (int64_t i = 0; i < block.length; ++i) {
+          if (BitUtil::GetBit(cond_data, offset + i)) {
+            out_values[i] = left_data;
+          }
+        }
+      }
+
+      offset += block.length;
+      out_values += block.length;
+    }
+
+    out->buffers[1] = std::move(out_buf);
+    return Status::OK();
+  }
+};
+
+template <typename Type>
+struct IfElseFunctor<Type, enable_if_boolean<Type>> {
+  // AAA
+  static Status Call(KernelContext* ctx, const ArrayData& cond, const ArrayData& left,
+                     const ArrayData& right, ArrayData* out) {
+    ARROW_RETURN_NOT_OK(PromoteNullsVisitor(ctx, cond, left, right, out));
+
+    // out_buff = right & ~cond
+    ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Buffer> out_buf,
+                          arrow::internal::BitmapAndNot(
+                              ctx->memory_pool(), right.buffers[1]->data(), right.offset,
+                              cond.buffers[1]->data(), cond.offset, cond.length, 0));
+
+    // out_buff = left & cond
+    ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Buffer> temp_buf,
+                          arrow::internal::BitmapAnd(
+                              ctx->memory_pool(), left.buffers[1]->data(), left.offset,
+                              cond.buffers[1]->data(), cond.offset, cond.length, 0));
+
+    arrow::internal::BitmapOr(out_buf->data(), 0, temp_buf->data(), 0, cond.length, 0,
+                              out_buf->mutable_data());
+    out->buffers[1] = std::move(out_buf);
+    return Status::OK();
+  }
+
+  // ASA
+  static Status Call(KernelContext* ctx, const ArrayData& cond, const Scalar& left,
+                     const ArrayData& right, ArrayData* out) {
+    ARROW_RETURN_NOT_OK(PromoteNullsVisitor(ctx, cond, left, right, out));
+
+    // out_buff = right & ~cond
+    ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Buffer> out_buf,
+                          arrow::internal::BitmapAndNot(
+                              ctx->memory_pool(), right.buffers[1]->data(), right.offset,
+                              cond.buffers[1]->data(), cond.offset, cond.length, 0));
+
+    // out_buff = left & cond
+    bool left_data = internal::UnboxScalar<BooleanType>::Unbox(left);
+    if (left_data) {
+      arrow::internal::BitmapOr(out_buf->data(), 0, cond.buffers[1]->data(), cond.offset,
+                                cond.length, 0, out_buf->mutable_data());
+    }
+
+    out->buffers[1] = std::move(out_buf);
+    return Status::OK();
+  }
+
+  // AAS
+  static Status Call(KernelContext* ctx, const ArrayData& cond, const ArrayData& left,
+                     const Scalar& right, ArrayData* out) {
+    ARROW_RETURN_NOT_OK(PromoteNullsVisitor(ctx, cond, left, right, out));
+
+    // out_buff = left & cond
+    ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Buffer> out_buf,
+                          arrow::internal::BitmapAnd(
+                              ctx->memory_pool(), left.buffers[1]->data(), left.offset,
+                              cond.buffers[1]->data(), cond.offset, cond.length, 0));
+
+    bool right_data = internal::UnboxScalar<BooleanType>::Unbox(right);
+
+    // out_buff = left & cond | right & ~cond
+    if (right_data) {
+      arrow::internal::BitmapOrNot(out_buf->data(), 0, cond.buffers[1]->data(),
+                                   cond.offset, cond.length, 0, out_buf->mutable_data());
+    }
+
+    out->buffers[1] = std::move(out_buf);
+    return Status::OK();
+  }
+
+  // ASS
+  static Status Call(KernelContext* ctx, const ArrayData& cond, const Scalar& left,
+                     const Scalar& right, ArrayData* out) {
+    ARROW_RETURN_NOT_OK(PromoteNullsVisitor(ctx, cond, left, right, out));
+
+    bool left_data = internal::UnboxScalar<BooleanType>::Unbox(left);
+    bool right_data = internal::UnboxScalar<BooleanType>::Unbox(right);
+
+    // out_buf = left & cond | right & ~cond
+    std::shared_ptr<Buffer> out_buf = nullptr;
+    if (left_data) {
+      if (right_data) {
+        // out_buf = ones
+        ARROW_ASSIGN_OR_RAISE(out_buf, ctx->AllocateBitmap(cond.length));
+        // filling with UINT8_MAX upto the buffer's size (in bytes)
+        arrow::compute::internal::SetMemory<UINT8_MAX>(out_buf.get());
+      } else {
+        // out_buf = cond
+        out_buf = SliceBuffer(cond.buffers[1], cond.offset, cond.length);
+      }
+    } else {
+      if (right_data) {
+        // out_buf = ~cond
+        ARROW_ASSIGN_OR_RAISE(out_buf, arrow::internal::InvertBitmap(
+                                           ctx->memory_pool(), cond.buffers[1]->data(),
+                                           cond.offset, cond.length))
+      } else {
+        // out_buf = zeros
+        ARROW_ASSIGN_OR_RAISE(out_buf, ctx->AllocateBitmap(cond.length));
+      }
+    }
+    out->buffers[1] = std::move(out_buf);
+    return Status::OK();
+  }
+};
+
+template <typename Type>
+struct IfElseFunctor<Type, enable_if_null<Type>> {
+  template <typename T>
+  static inline Status ReturnCopy(const T& in, T* out) {
+    // Nothing preallocated, so we assign in into the output
+    *out = in;
+    return Status::OK();
+  }
+
+  // AAA
+  static Status Call(KernelContext* ctx, const ArrayData& cond, const ArrayData& left,
+                     const ArrayData& right, ArrayData* out) {
+    return ReturnCopy(left, out);
+  }
+
+  // ASA
+  static Status Call(KernelContext* ctx, const ArrayData& cond, const Scalar& left,
+                     const ArrayData& right, ArrayData* out) {
+    return ReturnCopy(right, out);
+  }
+
+  // AAS
+  static Status Call(KernelContext* ctx, const ArrayData& cond, const ArrayData& left,
+                     const Scalar& right, ArrayData* out) {
+    return ReturnCopy(left, out);
+  }
+
+  // ASS
+  static Status Call(KernelContext* ctx, const ArrayData& cond, const Scalar& left,
+                     const Scalar& right, ArrayData* out) {
+    return ReturnCopy(cond, out);
+  }
+};
+
+template <typename Type>
+struct ResolveIfElseExec {
+  static Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    // cond is scalar
+    if (batch[0].is_scalar()) {
+      const auto& cond = batch[0].scalar_as<BooleanScalar>();
+      if (batch[1].is_scalar() && batch[2].is_scalar()) {
+        if (cond.is_valid) {
+          *out = cond.value ? batch[1].scalar() : batch[2].scalar();
+        } else {
+          *out = MakeNullScalar(batch[1].type());
+        }
+      } else {  // either left or right is an array. output is always an array
+        // output size is the size of the array arg
+        int64_t bcast_size = batch[1].is_array() ? batch[1].length() : batch[2].length();

Review comment:
       I think you can just use `batch.length` here. If either argument is an array it'll give you the length.




-- 
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



[GitHub] [arrow] bkietz commented on a change in pull request #10410: ARROW-10640: [C++] A "where" kernel to combine two arrays based on a mask

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_if_else_test.cc
##########
@@ -0,0 +1,264 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <arrow/array.h>
+#include <arrow/compute/api_scalar.h>
+#include <arrow/compute/kernels/test_util.h>
+#include <arrow/testing/gtest_util.h>
+#include <gtest/gtest.h>
+
+namespace arrow {
+namespace compute {
+
+void CheckIfElseOutput(const Datum& cond, const Datum& left, const Datum& right,
+                       const Datum& expected) {
+  ASSERT_OK_AND_ASSIGN(Datum datum_out, IfElse(cond, left, right));
+  if (datum_out.is_array()) {
+    std::shared_ptr<Array> result = datum_out.make_array();
+    ASSERT_OK(result->ValidateFull());
+    std::shared_ptr<Array> expected_ = expected.make_array();
+    AssertArraysEqual(*expected_, *result, /*verbose=*/true);
+  } else {  // expecting scalar
+    const std::shared_ptr<Scalar>& result = datum_out.scalar();
+    const std::shared_ptr<Scalar>& expected_ = expected.scalar();
+    AssertScalarsEqual(*expected_, *result, /*verbose=*/true);
+  }
+}
+
+class TestIfElseKernel : public ::testing::Test {};
+
+template <typename Type>
+class TestIfElsePrimitive : public ::testing::Test {};
+
+using PrimitiveTypes = ::testing::Types<Int8Type, UInt8Type, Int16Type, UInt16Type,
+                                        Int32Type, UInt32Type, Int64Type, UInt64Type,
+                                        FloatType, DoubleType, Date32Type, Date64Type>;
+
+TYPED_TEST_SUITE(TestIfElsePrimitive, PrimitiveTypes);
+
+TYPED_TEST(TestIfElsePrimitive, IfElseFixedSizeRand) {
+  using ArrayType = typename TypeTraits<TypeParam>::ArrayType;
+  auto type = TypeTraits<TypeParam>::type_singleton();
+
+  random::RandomArrayGenerator rand(/*seed=*/0);
+  int64_t len = 1000;
+  auto cond = std::static_pointer_cast<BooleanArray>(
+      rand.ArrayOf(boolean(), len, /*null_probability=*/0.01));
+  auto left = std::static_pointer_cast<ArrayType>(
+      rand.ArrayOf(type, len, /*null_probability=*/0.01));
+  auto right = std::static_pointer_cast<ArrayType>(
+      rand.ArrayOf(type, len, /*null_probability=*/0.01));
+
+  typename TypeTraits<TypeParam>::BuilderType builder;
+
+  for (int64_t i = 0; i < len; ++i) {
+    if (!cond->IsValid(i) || (cond->Value(i) && !left->IsValid(i)) ||
+        (!cond->Value(i) && !right->IsValid(i))) {
+      ASSERT_OK(builder.AppendNull());
+      continue;
+    }
+
+    if (cond->Value(i)) {
+      ASSERT_OK(builder.Append(left->Value(i)));
+    } else {
+      ASSERT_OK(builder.Append(right->Value(i)));
+    }
+  }
+  ASSERT_OK_AND_ASSIGN(auto expected_data, builder.Finish());
+
+  CheckIfElseOutput(cond, left, right, expected_data);
+}
+
+void CheckWithDifferentShapes(const std::shared_ptr<Array>& cond,
+                              const std::shared_ptr<Array>& left,
+                              const std::shared_ptr<Array>& right,
+                              const std::shared_ptr<Array>& expected) {
+  // this will check for whole arrays, every scalar at i'th index and slicing (offset)
+  CheckScalar("if_else", {cond, left, right}, expected);
+
+  auto len = left->length();
+
+  enum { COND_SCALAR = 1, LEFT_SCALAR = 2, RIGHT_SCALAR = 4 };
+  for (int mask = 0; mask < (COND_SCALAR | LEFT_SCALAR | RIGHT_SCALAR); ++mask) {
+    for (int64_t cond_idx = 0; cond_idx < len; ++cond_idx) {
+      Datum cond_in, cond_bcast;
+      if (mask & COND_SCALAR) {
+        ASSERT_OK_AND_ASSIGN(cond_in, cond->GetScalar(cond_idx));
+        ASSERT_OK_AND_ASSIGN(cond_bcast, MakeArrayFromScalar(*cond_in.scalar(), len));
+      } else {
+        cond_in = cond_bcast = cond;
+      }

Review comment:
       Please add SCOPED_TRACEs here and below for left/right so that failures within the loop are more informative about which iteration failed
   ```suggestion
         Datum cond_in, cond_bcast;
         std::string trace_msg = "Cond";
         if (mask & COND_SCALAR) {
           ASSERT_OK_AND_ASSIGN(cond_in, cond->GetScalar(cond_idx));
           ASSERT_OK_AND_ASSIGN(cond_bcast, MakeArrayFromScalar(*cond_in.scalar(), len));
           trace_msg += "@" + std::to_string(cond_idx) + "=" + cond_in.scalar()->ToString();
         } else {
           cond_in = cond_bcast = cond;
         }
         SCOPED_TRACE(trace_msg);
   ```

##########
File path: cpp/src/arrow/compute/kernels/scalar_if_else.cc
##########
@@ -699,14 +699,58 @@ struct IfElseFunctor<Type, enable_if_boolean<Type>> {
   // AAS
   static Status Call(KernelContext* ctx, const ArrayData& cond, const ArrayData& left,
                      const Scalar& right, ArrayData* out) {
-    // todo impl
+    ARROW_RETURN_NOT_OK(PromoteNullsVisitor(ctx, cond, left, right, out));
+
+    // out_buff = left & cond
+    ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Buffer> out_buf,
+                          arrow::internal::BitmapAnd(
+                              ctx->memory_pool(), left.buffers[1]->data(), left.offset,
+                              cond.buffers[1]->data(), cond.offset, cond.length, 0));
+
+    bool right_data = internal::UnboxScalar<BooleanType>::Unbox(right);
+
+    // out_buff = left & cond | right & ~cond
+    if (right_data) {
+      arrow::internal::BitmapOrNot(out_buf->data(), 0, cond.buffers[1]->data(),
+                                   cond.offset, cond.length, 0, out_buf->mutable_data());
+    }
+
+    out->buffers[1] = std::move(out_buf);
     return Status::OK();
   }
 
   // ASS
   static Status Call(KernelContext* ctx, const ArrayData& cond, const Scalar& left,
                      const Scalar& right, ArrayData* out) {
-    // todo impl
+    ARROW_RETURN_NOT_OK(PromoteNullsVisitor(ctx, cond, left, right, out));
+
+    bool left_data = internal::UnboxScalar<BooleanType>::Unbox(left);
+    bool right_data = internal::UnboxScalar<BooleanType>::Unbox(right);
+
+    // out_buf = left & cond | right & ~cond
+    std::shared_ptr<Buffer> out_buf = nullptr;
+    if (left_data) {
+      if (right_data) {
+        // out_buf = ones
+        ARROW_ASSIGN_OR_RAISE(out_buf, ctx->AllocateBitmap(cond.length));
+        // filling with UINT8_MAX upto the buffer's size (in bytes)
+        arrow::compute::internal::SetMemory<UINT8_MAX>(out_buf.get());

Review comment:
       Please inline the call to memset 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.

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



[GitHub] [arrow] nirandaperera commented on a change in pull request #10410: ARROW-10640: [C++] A "where" kernel to combine two arrays based on a mask

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_if_else.cc
##########
@@ -0,0 +1,572 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <arrow/compute/api.h>
+#include <arrow/compute/kernels/codegen_internal.h>
+#include <arrow/compute/util_internal.h>
+#include <arrow/util/bit_block_counter.h>
+#include <arrow/util/bitmap.h>
+#include <arrow/util/bitmap_ops.h>
+
+namespace arrow {
+using internal::BitBlockCount;
+using internal::BitBlockCounter;
+using internal::Bitmap;
+
+namespace compute {
+
+namespace {
+
+util::optional<uint64_t> GetConstantValidityWord(const Datum& data) {
+  if (data.is_scalar()) {
+    return data.scalar()->is_valid ? ~uint64_t(0) : uint64_t(0);
+  }
+
+  if (data.array()->null_count == data.array()->length) return uint64_t(0);
+
+  if (!data.array()->MayHaveNulls()) return ~uint64_t(0);
+
+  // no constant validity word available
+  return {};
+}
+
+inline Bitmap GetBitmap(const Datum& datum, int i) {
+  if (datum.is_scalar()) return {};
+  const ArrayData& a = *datum.array();
+  return Bitmap{a.buffers[i], a.offset, a.length};
+}
+
+// if the condition is null then output is null otherwise we take validity from the
+// selected argument
+// ie. cond.valid & (cond.data & left.valid | ~cond.data & right.valid)
+Status PromoteNullsVisitor(KernelContext* ctx, const Datum& cond_d, const Datum& left_d,
+                           const Datum& right_d, ArrayData* output) {
+  auto cond_const = GetConstantValidityWord(cond_d);
+  auto left_const = GetConstantValidityWord(left_d);
+  auto right_const = GetConstantValidityWord(right_d);
+
+  enum { COND_CONST = 1, LEFT_CONST = 2, RIGHT_CONST = 4 };
+  auto flag = COND_CONST * cond_const.has_value() | LEFT_CONST * left_const.has_value() |
+              RIGHT_CONST * right_const.has_value();
+
+  const ArrayData& cond = *cond_d.array();
+  // cond.data will always be available
+  Bitmap cond_data{cond.buffers[1], cond.offset, cond.length};
+  Bitmap cond_valid{cond.buffers[0], cond.offset, cond.length};
+  Bitmap left_valid = GetBitmap(left_d, 0);
+  Bitmap right_valid = GetBitmap(right_d, 0);
+  // sometimes Bitmaps will be ignored, in which case we replace access to them with
+  // duplicated (probably elided) access to cond_data
+  const Bitmap& _ = cond_data;
+
+  // lambda function that will be used inside the visitor
+  uint64_t* out_validity = nullptr;
+  int64_t i = 0;
+  auto apply = [&](uint64_t c_valid, uint64_t c_data, uint64_t l_valid,
+                   uint64_t r_valid) {
+    out_validity[i] = c_valid & ((c_data & l_valid) | (~c_data & r_valid));
+    i++;
+  };
+
+  // if the condition is null then output is null otherwise we take validity from the
+  // selected argument
+  // ie. cond.valid & (cond.data & left.valid | ~cond.data & right.valid)
+
+  // In the following cases, we dont need to allocate out_valid bitmap
+  switch (flag) {
+    case COND_CONST | LEFT_CONST | RIGHT_CONST:
+      // if cond & left & right all ones, then output is all valid --> out_valid = nullptr

Review comment:
       This has 2 `switch` blocks. This block only handles 2 cases that does not require explicit bitmap allocation. 




-- 
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



[GitHub] [arrow] lidavidm commented on pull request #10410: ARROW-10640: [C++] A, "if_else" ("where") kernel to combine two arrays based on a mask

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


   I agree, and also Niranda already filed ARROW-12955 to take care of that followup.


-- 
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



[GitHub] [arrow] nirandaperera commented on pull request #10410: ARROW-10640: [C++] A "where" kernel to combine two arrays based on a mask

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


   @bkietz @lidavidm I think I have added all the changes requested for 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.

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



[GitHub] [arrow] nirandaperera edited a comment on pull request #10410: ARROW-10640: [C++] A "where" kernel to combine two arrays based on a mask

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


   @bkietz @lidavidm I think I have added all the changes requested for this PR so far. Anything else?


-- 
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



[GitHub] [arrow] nirandaperera commented on pull request #10410: ARROW-10640: [C++] A, "if_else" ("where") kernel to combine two arrays based on a mask

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


   > It looks like there's an actual build failure on CentOS/GCC4.8 (expand "Dump install logs" to get the actual failure):
   > 
   > ```
   > In file included from /tmp/Rtmpmpy1Y5/file7d77bb9a74/src/arrow/CMakeFiles/arrow_objlib.dir/Unity/unity_17_cxx.cxx:4:0:
   > /arrow/cpp/src/arrow/compute/kernels/scalar_if_else.cc: In function ‘arrow::Status arrow::compute::{anonymous}::PromoteNullsVisitor(arrow::compute::KernelContext*, const arrow::Datum&, const arrow::Datum&, const arrow::Datum&, arrow::ArrayData*)’:
   > /arrow/cpp/src/arrow/compute/kernels/scalar_if_else.cc:121:8: error: no matching function for call to ‘arrow::internal::Bitmap::VisitWords(<brace-enclosed initializer list>, arrow::compute::{anonymous}::PromoteNullsVisitor(arrow::compute::KernelContext*, const arrow::Datum&, const arrow::Datum&, const arrow::Datum&, arrow::ArrayData*)::__lambda44)’
   >        });
   >         ^
   > /arrow/cpp/src/arrow/compute/kernels/scalar_if_else.cc:121:8: note: candidate is:
   > In file included from /arrow/cpp/src/arrow/compute/kernels/scalar_if_else.cc:22:0,
   >                  from /tmp/Rtmpmpy1Y5/file7d77bb9a74/src/arrow/CMakeFiles/arrow_objlib.dir/Unity/unity_17_cxx.cxx:4:
   > /arrow/cpp/src/arrow/util/bitmap.h:128:18: note: template<long unsigned int N, class Visitor, class Word> static int64_t arrow::internal::Bitmap::VisitWords(const arrow::internal::Bitmap (&)[N], Visitor&&)
   >    static int64_t VisitWords(const Bitmap (&bitmaps_arg)[N], Visitor&& visitor) {
   >                   ^
   > /arrow/cpp/src/arrow/util/bitmap.h:128:18: note:   template argument deduction/substitution failed:
   > In file included from /tmp/Rtmpmpy1Y5/file7d77bb9a74/src/arrow/CMakeFiles/arrow_objlib.dir/Unity/unity_17_cxx.cxx:4:0:
   > /arrow/cpp/src/arrow/compute/kernels/scalar_if_else.cc:121:8: note:   couldn't deduce template parameter ‘N’
   >        });
   >         ^
   > /arrow/cpp/src/arrow/compute/kernels/scalar_if_else.cc:127:12: error: no matching function for call to ‘arrow::internal::Bitmap::VisitWords(<brace-enclosed initializer list>, arrow::compute::{anonymous}::PromoteNullsVisitor(arrow::compute::KernelContext*, const arrow::Datum&, const arrow::Datum&, const arrow::Datum&, arrow::ArrayData*)::__lambda45)’
   >            });
   >             ^
   > /arrow/cpp/src/arrow/compute/kernels/scalar_if_else.cc:127:12: note: candidate is:
   > In file included from /arrow/cpp/src/arrow/compute/kernels/scalar_if_else.cc:22:0,
   >                  from /tmp/Rtmpmpy1Y5/file7d77bb9a74/src/arrow/CMakeFiles/arrow_objlib.dir/Unity/unity_17_cxx.cxx:4:
   > /arrow/cpp/src/arrow/util/bitmap.h:128:18: note: template<long unsigned int N, class Visitor, class Word> static int64_t arrow::internal::Bitmap::VisitWords(const arrow::internal::Bitmap (&)[N], Visitor&&)
   >    static int64_t VisitWords(const Bitmap (&bitmaps_arg)[N], Visitor&& visitor) {
   >             ^
   > /arrow/cpp/src/arrow/compute/kernels/scalar_if_else.cc:157:12: note: candidate is:
   > In file included from /arrow/cpp/src/arrow/compute/kernels/scalar_if_else.cc:22:0,
   >                  from /tmp/Rtmpmpy1Y5/file7d77bb9a74/src/arrow/CMakeFiles/arrow_objlib.dir/Unity/unity_17_cxx.cxx:4:
   > /arrow/cpp/src/arrow/util/bitmap.h:128:18: note: template<long unsigned int N, class Visitor, class Word> static int64_t arrow::internal::Bitmap::VisitWords(const arrow::internal::Bitmap (&)[N], Visitor&&)
   >    static int64_t VisitWords(const Bitmap (&bitmaps_arg)[N], Visitor&& visitor) {
   >                   ^
   > /arrow/cpp/src/arrow/util/bitmap.h:128:18: note:   template argument deduction/substitution failed:
   > In file included from /tmp/Rtmpmpy1Y5/file7d77bb9a74/src/arrow/CMakeFiles/arrow_objlib.dir/Unity/unity_17_cxx.cxx:4:0:
   > /arrow/cpp/src/arrow/compute/kernels/scalar_if_else.cc:157:12: note:   couldn't deduce template parameter ‘N’
   >            });
   >             ^
   > /arrow/cpp/src/arrow/compute/kernels/scalar_if_else.cc:164:12: error: no matching function for call to ‘arrow::internal::Bitmap::VisitWords(<brace-enclosed initializer list>, arrow::compute::{anonymous}::PromoteNullsVisitor(arrow::compute::KernelContext*, const arrow::Datum&, const arrow::Datum&, const arrow::Datum&, arrow::ArrayData*)::__lambda50)’
   >            });
   >             ^
   > /arrow/cpp/src/arrow/compute/kernels/scalar_if_else.cc:164:12: note: candidate is:
   > In file included from /arrow/cpp/src/arrow/compute/kernels/scalar_if_else.cc:22:0,
   >                  from /tmp/Rtmpmpy1Y5/file7d77bb9a74/src/arrow/CMakeFiles/arrow_objlib.dir/Unity/unity_17_cxx.cxx:4:
   > /arrow/cpp/src/arrow/util/bitmap.h:128:18: note: template<long unsigned int N, class Visitor, class Word> static int64_t arrow::internal::Bitmap::VisitWords(const arrow::internal::Bitmap (&)[N], Visitor&&)
   >    static int64_t VisitWords(const Bitmap (&bitmaps_arg)[N], Visitor&& visitor) {
   >                   ^
   > /arrow/cpp/src/arrow/util/bitmap.h:128:18: note:   template argument deduction/substitution failed:
   > In file included from /tmp/Rtmpmpy1Y5/file7d77bb9a74/src/arrow/CMakeFiles/arrow_objlib.dir/Unity/unity_17_cxx.cxx:4:0:
   > /arrow/cpp/src/arrow/compute/kernels/scalar_if_else.cc:164:12: note:   couldn't deduce template parameter ‘N’
   >            });
   >             ^
   > /arrow/cpp/src/arrow/compute/kernels/scalar_if_else.cc:171:27: error: no matching function for call to ‘arrow::internal::Bitmap::VisitWords(<brace-enclosed initializer list>, arrow::compute::{anonymous}::PromoteNullsVisitor(arrow::compute::KernelContext*, const arrow::Datum&, const arrow::Datum&, const arrow::Datum&, arrow::ArrayData*)::__lambda51)’
   >                           });
   >                            ^
   > /arrow/cpp/src/arrow/compute/kernels/scalar_if_else.cc:171:27: note: candidate is:
   > In file included from /arrow/cpp/src/arrow/compute/kernels/scalar_if_else.cc:22:0,
   >                  from /tmp/Rtmpmpy1Y5/file7d77bb9a74/src/arrow/CMakeFiles/arrow_objlib.dir/Unity/unity_17_cxx.cxx:4:
   > /arrow/cpp/src/arrow/util/bitmap.h:128:18: note: template<long unsigned int N, class Visitor, class Word> static int64_t arrow::internal::Bitmap::VisitWords(const arrow::internal::Bitmap (&)[N], Visitor&&)
   >    static int64_t VisitWords(const Bitmap (&bitmaps_arg)[N], Visitor&& visitor) {
   >                   ^
   > /arrow/cpp/src/arrow/util/bitmap.h:128:18: note:   template argument deduction/substitution failed:
   > In file included from /tmp/Rtmpmpy1Y5/file7d77bb9a74/src/arrow/CMakeFiles/arrow_objlib.dir/Unity/unity_17_cxx.cxx:4:0:
   > /arrow/cpp/src/arrow/compute/kernels/scalar_if_else.cc:171:27: note:   couldn't deduce template parameter ‘N’
   >                           });
   > ```
   
   Ah! thanks @lidavidm I missed this... it seems to be a gcc 4.x thing, isn't it. 
   
   BTW @bkietz should I add string type support to this as well, or can I leave it for later development?


-- 
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



[GitHub] [arrow] bkietz commented on a change in pull request #10410: ARROW-10640: [C++] A "where" kernel to combine two arrays based on a mask

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_if_else_test.cc
##########
@@ -0,0 +1,474 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <arrow/array.h>
+#include <arrow/compute/api_scalar.h>
+#include <arrow/compute/kernels/test_util.h>
+#include <arrow/testing/gtest_util.h>
+#include <gtest/gtest.h>
+
+namespace arrow {
+namespace compute {
+
+void CheckIfElseOutput(const Datum& cond, const Datum& left, const Datum& right,
+                       const Datum& expected) {
+  ASSERT_OK_AND_ASSIGN(Datum datum_out, IfElse(cond, left, right));
+  if (datum_out.is_array()) {
+    std::shared_ptr<Array> result = datum_out.make_array();
+    ASSERT_OK(result->ValidateFull());
+    std::shared_ptr<Array> expected_ = expected.make_array();
+    AssertArraysEqual(*expected_, *result, /*verbose=*/true);
+  } else {  // expecting scalar
+    const std::shared_ptr<Scalar>& result = datum_out.scalar();
+    const std::shared_ptr<Scalar>& expected_ = expected.scalar();
+    AssertScalarsEqual(*expected_, *result, /*verbose=*/true);
+  }
+}
+
+void CheckIfElseOutputAAA(const std::shared_ptr<DataType>& type, const std::string& cond,
+                          const std::string& left, const std::string& right,
+                          const std::string& expected) {
+  const std::shared_ptr<Array>& cond_ = ArrayFromJSON(boolean(), cond);
+  const std::shared_ptr<Array>& left_ = ArrayFromJSON(type, left);
+  const std::shared_ptr<Array>& right_ = ArrayFromJSON(type, right);
+  const std::shared_ptr<Array>& expected_ = ArrayFromJSON(type, expected);
+  CheckIfElseOutput(cond_, left_, right_, expected_);
+}
+
+void CheckIfElseOutputAAS(const std::shared_ptr<DataType>& type, const std::string& cond,
+                          const std::string& left, const std::shared_ptr<Scalar>& right,
+                          const std::string& expected) {
+  const std::shared_ptr<Array>& cond_ = ArrayFromJSON(boolean(), cond);
+  const std::shared_ptr<Array>& left_ = ArrayFromJSON(type, left);
+  const std::shared_ptr<Array>& expected_ = ArrayFromJSON(type, expected);
+  CheckIfElseOutput(cond_, left_, right, expected_);
+}
+
+void CheckIfElseOutputASA(const std::shared_ptr<DataType>& type, const std::string& cond,
+                          const std::shared_ptr<Scalar>& left, const std::string& right,
+                          const std::string& expected) {
+  const std::shared_ptr<Array>& cond_ = ArrayFromJSON(boolean(), cond);
+  const std::shared_ptr<Array>& right_ = ArrayFromJSON(type, right);
+  const std::shared_ptr<Array>& expected_ = ArrayFromJSON(type, expected);
+  CheckIfElseOutput(cond_, left, right_, expected_);
+}
+
+void CheckIfElseOutputASS(const std::shared_ptr<DataType>& type, const std::string& cond,
+                          const std::shared_ptr<Scalar>& left,
+                          const std::shared_ptr<Scalar>& right,
+                          const std::string& expected) {
+  const std::shared_ptr<Array>& cond_ = ArrayFromJSON(boolean(), cond);
+  const std::shared_ptr<Array>& expected_ = ArrayFromJSON(type, expected);
+  CheckIfElseOutput(cond_, left, right, expected_);
+}
+
+void CheckIfElseOutputSAA(const std::shared_ptr<DataType>& type,
+                          const std::shared_ptr<Scalar>& cond, const std::string& left,
+                          const std::string& right, const std::string& expected) {
+  const std::shared_ptr<Array>& left_ = ArrayFromJSON(type, left);
+  const std::shared_ptr<Array>& right_ = ArrayFromJSON(type, right);
+  const std::shared_ptr<Array>& expected_ = ArrayFromJSON(type, expected);
+  CheckIfElseOutput(cond, left_, right_, expected_);
+}
+
+void CheckIfElseOutputSAS(const std::shared_ptr<DataType>& type,
+                          const std::shared_ptr<Scalar>& cond, const std::string& left,
+                          const std::shared_ptr<Scalar>& right,
+                          const std::string& expected) {
+  const std::shared_ptr<Array>& left_ = ArrayFromJSON(type, left);
+  const std::shared_ptr<Array>& expected_ = ArrayFromJSON(type, expected);
+  CheckIfElseOutput(cond, left_, right, expected_);
+}
+
+void CheckIfElseOutputSSA(const std::shared_ptr<DataType>& type,
+                          const std::shared_ptr<Scalar>& cond,
+                          const std::shared_ptr<Scalar>& left, const std::string& right,
+                          const std::string& expected) {
+  const std::shared_ptr<Array>& right_ = ArrayFromJSON(type, right);
+  const std::shared_ptr<Array>& expected_ = ArrayFromJSON(type, expected);
+  CheckIfElseOutput(cond, left, right_, expected_);
+}
+
+class TestIfElseKernel : public ::testing::Test {};
+
+template <typename Type>
+class TestIfElsePrimitive : public ::testing::Test {};
+
+using PrimitiveTypes = ::testing::Types<Int8Type, UInt8Type, Int16Type, UInt16Type,
+                                        Int32Type, UInt32Type, Int64Type, UInt64Type,
+                                        FloatType, DoubleType, Date32Type, Date64Type>;
+
+TYPED_TEST_SUITE(TestIfElsePrimitive, PrimitiveTypes);
+
+TYPED_TEST(TestIfElsePrimitive, IfElseFixedSizeRand) {
+  using ArrayType = typename TypeTraits<TypeParam>::ArrayType;
+  auto type = TypeTraits<TypeParam>::type_singleton();
+
+  random::RandomArrayGenerator rand(/*seed=*/0);
+  int64_t len = 1000;
+  auto cond = std::static_pointer_cast<BooleanArray>(
+      rand.ArrayOf(boolean(), len, /*null_probability=*/0.01));
+  auto left = std::static_pointer_cast<ArrayType>(
+      rand.ArrayOf(type, len, /*null_probability=*/0.01));
+  auto right = std::static_pointer_cast<ArrayType>(
+      rand.ArrayOf(type, len, /*null_probability=*/0.01));
+
+  typename TypeTraits<TypeParam>::BuilderType builder;
+
+  for (int64_t i = 0; i < len; ++i) {
+    if (!cond->IsValid(i) || (cond->Value(i) && !left->IsValid(i)) ||
+        (!cond->Value(i) && !right->IsValid(i))) {
+      ASSERT_OK(builder.AppendNull());
+      continue;
+    }
+
+    if (cond->Value(i)) {
+      ASSERT_OK(builder.Append(left->Value(i)));
+    } else {
+      ASSERT_OK(builder.Append(right->Value(i)));
+    }
+  }
+  ASSERT_OK_AND_ASSIGN(auto expected_data, builder.Finish());
+
+  CheckIfElseOutput(cond, left, right, expected_data);
+}
+
+#define IF_ELSE_TEST_GEN(type, l0, l1, l2, l3, r0, r1, r2, r3, valid, valid1)            \

Review comment:
       No, I mean you can generate the other cases given correct array-array-array using `Array::GetScalar` and `MakeArrayFromScalar`
   
   ```c++
   std::shared_ptr<Array> cond = ..., left = ..., right = ..., expected = ...;
   // Explicit, hand written assertion for array-array-array and scalar-scalar-scalar cases
   CheckScalar("if_else", {cond, left, right}, expected);
   
   // Generate Array-Scalar-Array case
   for (int64_t i = 0; i < left->length(); ++i) {
     ASSERT_OK_AND_ASSIGN(auto left_scalar, left->GetScalar(i));
     ASSERT_OK_AND_ASSIGN(auto left_array, MakeArrayFromScalar(*left_scalar, left->length()));
     ASSERT_OK_AND_ASSIGN(auto expected, IfElse(cond, left_array, right)); // array-array-array is known to be correct
     ASSERT_OK_AND_ASSIGN(auto actual, IfElse(cond, left_scalar, right));
     AssertDatumsEqual(expected, actual);
   }
   ```




-- 
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



[GitHub] [arrow] nirandaperera commented on a change in pull request #10410: ARROW-10640: [C++] A "where" kernel to combine two arrays based on a mask

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_if_else_test.cc
##########
@@ -0,0 +1,474 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <arrow/array.h>
+#include <arrow/compute/api_scalar.h>
+#include <arrow/compute/kernels/test_util.h>
+#include <arrow/testing/gtest_util.h>
+#include <gtest/gtest.h>
+
+namespace arrow {
+namespace compute {
+
+void CheckIfElseOutput(const Datum& cond, const Datum& left, const Datum& right,
+                       const Datum& expected) {
+  ASSERT_OK_AND_ASSIGN(Datum datum_out, IfElse(cond, left, right));
+  if (datum_out.is_array()) {
+    std::shared_ptr<Array> result = datum_out.make_array();
+    ASSERT_OK(result->ValidateFull());
+    std::shared_ptr<Array> expected_ = expected.make_array();
+    AssertArraysEqual(*expected_, *result, /*verbose=*/true);
+  } else {  // expecting scalar
+    const std::shared_ptr<Scalar>& result = datum_out.scalar();
+    const std::shared_ptr<Scalar>& expected_ = expected.scalar();
+    AssertScalarsEqual(*expected_, *result, /*verbose=*/true);
+  }
+}
+
+void CheckIfElseOutputAAA(const std::shared_ptr<DataType>& type, const std::string& cond,
+                          const std::string& left, const std::string& right,
+                          const std::string& expected) {
+  const std::shared_ptr<Array>& cond_ = ArrayFromJSON(boolean(), cond);
+  const std::shared_ptr<Array>& left_ = ArrayFromJSON(type, left);
+  const std::shared_ptr<Array>& right_ = ArrayFromJSON(type, right);
+  const std::shared_ptr<Array>& expected_ = ArrayFromJSON(type, expected);
+  CheckIfElseOutput(cond_, left_, right_, expected_);
+}
+
+void CheckIfElseOutputAAS(const std::shared_ptr<DataType>& type, const std::string& cond,
+                          const std::string& left, const std::shared_ptr<Scalar>& right,
+                          const std::string& expected) {
+  const std::shared_ptr<Array>& cond_ = ArrayFromJSON(boolean(), cond);
+  const std::shared_ptr<Array>& left_ = ArrayFromJSON(type, left);
+  const std::shared_ptr<Array>& expected_ = ArrayFromJSON(type, expected);
+  CheckIfElseOutput(cond_, left_, right, expected_);
+}
+
+void CheckIfElseOutputASA(const std::shared_ptr<DataType>& type, const std::string& cond,
+                          const std::shared_ptr<Scalar>& left, const std::string& right,
+                          const std::string& expected) {
+  const std::shared_ptr<Array>& cond_ = ArrayFromJSON(boolean(), cond);
+  const std::shared_ptr<Array>& right_ = ArrayFromJSON(type, right);
+  const std::shared_ptr<Array>& expected_ = ArrayFromJSON(type, expected);
+  CheckIfElseOutput(cond_, left, right_, expected_);
+}
+
+void CheckIfElseOutputASS(const std::shared_ptr<DataType>& type, const std::string& cond,
+                          const std::shared_ptr<Scalar>& left,
+                          const std::shared_ptr<Scalar>& right,
+                          const std::string& expected) {
+  const std::shared_ptr<Array>& cond_ = ArrayFromJSON(boolean(), cond);
+  const std::shared_ptr<Array>& expected_ = ArrayFromJSON(type, expected);
+  CheckIfElseOutput(cond_, left, right, expected_);
+}
+
+void CheckIfElseOutputSAA(const std::shared_ptr<DataType>& type,
+                          const std::shared_ptr<Scalar>& cond, const std::string& left,
+                          const std::string& right, const std::string& expected) {
+  const std::shared_ptr<Array>& left_ = ArrayFromJSON(type, left);
+  const std::shared_ptr<Array>& right_ = ArrayFromJSON(type, right);
+  const std::shared_ptr<Array>& expected_ = ArrayFromJSON(type, expected);
+  CheckIfElseOutput(cond, left_, right_, expected_);
+}
+
+void CheckIfElseOutputSAS(const std::shared_ptr<DataType>& type,
+                          const std::shared_ptr<Scalar>& cond, const std::string& left,
+                          const std::shared_ptr<Scalar>& right,
+                          const std::string& expected) {
+  const std::shared_ptr<Array>& left_ = ArrayFromJSON(type, left);
+  const std::shared_ptr<Array>& expected_ = ArrayFromJSON(type, expected);
+  CheckIfElseOutput(cond, left_, right, expected_);
+}
+
+void CheckIfElseOutputSSA(const std::shared_ptr<DataType>& type,
+                          const std::shared_ptr<Scalar>& cond,
+                          const std::shared_ptr<Scalar>& left, const std::string& right,
+                          const std::string& expected) {
+  const std::shared_ptr<Array>& right_ = ArrayFromJSON(type, right);
+  const std::shared_ptr<Array>& expected_ = ArrayFromJSON(type, expected);
+  CheckIfElseOutput(cond, left, right_, expected_);
+}
+
+class TestIfElseKernel : public ::testing::Test {};
+
+template <typename Type>
+class TestIfElsePrimitive : public ::testing::Test {};
+
+using PrimitiveTypes = ::testing::Types<Int8Type, UInt8Type, Int16Type, UInt16Type,
+                                        Int32Type, UInt32Type, Int64Type, UInt64Type,
+                                        FloatType, DoubleType, Date32Type, Date64Type>;
+
+TYPED_TEST_SUITE(TestIfElsePrimitive, PrimitiveTypes);
+
+TYPED_TEST(TestIfElsePrimitive, IfElseFixedSizeRand) {
+  using ArrayType = typename TypeTraits<TypeParam>::ArrayType;
+  auto type = TypeTraits<TypeParam>::type_singleton();
+
+  random::RandomArrayGenerator rand(/*seed=*/0);
+  int64_t len = 1000;
+  auto cond = std::static_pointer_cast<BooleanArray>(
+      rand.ArrayOf(boolean(), len, /*null_probability=*/0.01));
+  auto left = std::static_pointer_cast<ArrayType>(
+      rand.ArrayOf(type, len, /*null_probability=*/0.01));
+  auto right = std::static_pointer_cast<ArrayType>(
+      rand.ArrayOf(type, len, /*null_probability=*/0.01));
+
+  typename TypeTraits<TypeParam>::BuilderType builder;
+
+  for (int64_t i = 0; i < len; ++i) {
+    if (!cond->IsValid(i) || (cond->Value(i) && !left->IsValid(i)) ||
+        (!cond->Value(i) && !right->IsValid(i))) {
+      ASSERT_OK(builder.AppendNull());
+      continue;
+    }
+
+    if (cond->Value(i)) {
+      ASSERT_OK(builder.Append(left->Value(i)));
+    } else {
+      ASSERT_OK(builder.Append(right->Value(i)));
+    }
+  }
+  ASSERT_OK_AND_ASSIGN(auto expected_data, builder.Finish());
+
+  CheckIfElseOutput(cond, left, right, expected_data);
+}
+
+#define IF_ELSE_TEST_GEN(type, l0, l1, l2, l3, r0, r1, r2, r3, valid, valid1)            \

Review comment:
       I will try the `CheckScalar` method (haven't had seen this so far).  




-- 
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



[GitHub] [arrow] nirandaperera commented on a change in pull request #10410: ARROW-10640: [C++] A "where" kernel to combine two arrays based on a mask

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_if_else_test.cc
##########
@@ -0,0 +1,474 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <arrow/array.h>
+#include <arrow/compute/api_scalar.h>
+#include <arrow/compute/kernels/test_util.h>
+#include <arrow/testing/gtest_util.h>
+#include <gtest/gtest.h>
+
+namespace arrow {
+namespace compute {
+
+void CheckIfElseOutput(const Datum& cond, const Datum& left, const Datum& right,
+                       const Datum& expected) {
+  ASSERT_OK_AND_ASSIGN(Datum datum_out, IfElse(cond, left, right));
+  if (datum_out.is_array()) {
+    std::shared_ptr<Array> result = datum_out.make_array();
+    ASSERT_OK(result->ValidateFull());
+    std::shared_ptr<Array> expected_ = expected.make_array();
+    AssertArraysEqual(*expected_, *result, /*verbose=*/true);
+  } else {  // expecting scalar
+    const std::shared_ptr<Scalar>& result = datum_out.scalar();
+    const std::shared_ptr<Scalar>& expected_ = expected.scalar();
+    AssertScalarsEqual(*expected_, *result, /*verbose=*/true);
+  }
+}
+
+void CheckIfElseOutputAAA(const std::shared_ptr<DataType>& type, const std::string& cond,
+                          const std::string& left, const std::string& right,
+                          const std::string& expected) {
+  const std::shared_ptr<Array>& cond_ = ArrayFromJSON(boolean(), cond);
+  const std::shared_ptr<Array>& left_ = ArrayFromJSON(type, left);
+  const std::shared_ptr<Array>& right_ = ArrayFromJSON(type, right);
+  const std::shared_ptr<Array>& expected_ = ArrayFromJSON(type, expected);
+  CheckIfElseOutput(cond_, left_, right_, expected_);
+}
+
+void CheckIfElseOutputAAS(const std::shared_ptr<DataType>& type, const std::string& cond,
+                          const std::string& left, const std::shared_ptr<Scalar>& right,
+                          const std::string& expected) {
+  const std::shared_ptr<Array>& cond_ = ArrayFromJSON(boolean(), cond);
+  const std::shared_ptr<Array>& left_ = ArrayFromJSON(type, left);
+  const std::shared_ptr<Array>& expected_ = ArrayFromJSON(type, expected);
+  CheckIfElseOutput(cond_, left_, right, expected_);
+}
+
+void CheckIfElseOutputASA(const std::shared_ptr<DataType>& type, const std::string& cond,
+                          const std::shared_ptr<Scalar>& left, const std::string& right,
+                          const std::string& expected) {
+  const std::shared_ptr<Array>& cond_ = ArrayFromJSON(boolean(), cond);
+  const std::shared_ptr<Array>& right_ = ArrayFromJSON(type, right);
+  const std::shared_ptr<Array>& expected_ = ArrayFromJSON(type, expected);
+  CheckIfElseOutput(cond_, left, right_, expected_);
+}
+
+void CheckIfElseOutputASS(const std::shared_ptr<DataType>& type, const std::string& cond,
+                          const std::shared_ptr<Scalar>& left,
+                          const std::shared_ptr<Scalar>& right,
+                          const std::string& expected) {
+  const std::shared_ptr<Array>& cond_ = ArrayFromJSON(boolean(), cond);
+  const std::shared_ptr<Array>& expected_ = ArrayFromJSON(type, expected);
+  CheckIfElseOutput(cond_, left, right, expected_);
+}
+
+void CheckIfElseOutputSAA(const std::shared_ptr<DataType>& type,
+                          const std::shared_ptr<Scalar>& cond, const std::string& left,
+                          const std::string& right, const std::string& expected) {
+  const std::shared_ptr<Array>& left_ = ArrayFromJSON(type, left);
+  const std::shared_ptr<Array>& right_ = ArrayFromJSON(type, right);
+  const std::shared_ptr<Array>& expected_ = ArrayFromJSON(type, expected);
+  CheckIfElseOutput(cond, left_, right_, expected_);
+}
+
+void CheckIfElseOutputSAS(const std::shared_ptr<DataType>& type,
+                          const std::shared_ptr<Scalar>& cond, const std::string& left,
+                          const std::shared_ptr<Scalar>& right,
+                          const std::string& expected) {
+  const std::shared_ptr<Array>& left_ = ArrayFromJSON(type, left);
+  const std::shared_ptr<Array>& expected_ = ArrayFromJSON(type, expected);
+  CheckIfElseOutput(cond, left_, right, expected_);
+}
+
+void CheckIfElseOutputSSA(const std::shared_ptr<DataType>& type,
+                          const std::shared_ptr<Scalar>& cond,
+                          const std::shared_ptr<Scalar>& left, const std::string& right,
+                          const std::string& expected) {
+  const std::shared_ptr<Array>& right_ = ArrayFromJSON(type, right);
+  const std::shared_ptr<Array>& expected_ = ArrayFromJSON(type, expected);
+  CheckIfElseOutput(cond, left, right_, expected_);
+}
+
+class TestIfElseKernel : public ::testing::Test {};
+
+template <typename Type>
+class TestIfElsePrimitive : public ::testing::Test {};
+
+using PrimitiveTypes = ::testing::Types<Int8Type, UInt8Type, Int16Type, UInt16Type,
+                                        Int32Type, UInt32Type, Int64Type, UInt64Type,
+                                        FloatType, DoubleType, Date32Type, Date64Type>;
+
+TYPED_TEST_SUITE(TestIfElsePrimitive, PrimitiveTypes);
+
+TYPED_TEST(TestIfElsePrimitive, IfElseFixedSizeRand) {
+  using ArrayType = typename TypeTraits<TypeParam>::ArrayType;
+  auto type = TypeTraits<TypeParam>::type_singleton();
+
+  random::RandomArrayGenerator rand(/*seed=*/0);
+  int64_t len = 1000;
+  auto cond = std::static_pointer_cast<BooleanArray>(
+      rand.ArrayOf(boolean(), len, /*null_probability=*/0.01));
+  auto left = std::static_pointer_cast<ArrayType>(
+      rand.ArrayOf(type, len, /*null_probability=*/0.01));
+  auto right = std::static_pointer_cast<ArrayType>(
+      rand.ArrayOf(type, len, /*null_probability=*/0.01));
+
+  typename TypeTraits<TypeParam>::BuilderType builder;
+
+  for (int64_t i = 0; i < len; ++i) {
+    if (!cond->IsValid(i) || (cond->Value(i) && !left->IsValid(i)) ||
+        (!cond->Value(i) && !right->IsValid(i))) {
+      ASSERT_OK(builder.AppendNull());
+      continue;
+    }
+
+    if (cond->Value(i)) {
+      ASSERT_OK(builder.Append(left->Value(i)));
+    } else {
+      ASSERT_OK(builder.Append(right->Value(i)));
+    }
+  }
+  ASSERT_OK_AND_ASSIGN(auto expected_data, builder.Finish());
+
+  CheckIfElseOutput(cond, left, right, expected_data);
+}
+
+#define IF_ELSE_TEST_GEN(type, l0, l1, l2, l3, r0, r1, r2, r3, valid, valid1)            \

Review comment:
       okay. So, IIUC you mean, we'd take the Array-Array-Array and Scalar-Scalar-Scalar case out from the `GenerateExpected` and have it separately, isn't 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.

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