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/27 15:05:21 UTC

[GitHub] [arrow] lidavidm opened a new pull request #10412: ARROW-9430: [C++] Implement override_mask kernel

lidavidm opened a new pull request #10412:
URL: https://github.com/apache/arrow/pull/10412


   This implements a kernel equivalent to NumPy's `arr[mask] = [values]`, i.e. given an array and an equal-length (or scalar) boolean mask, along with an array of replacement values passed via options, each array item for which the corresponding mask value is `true` is replaced with the next value from the replacement value 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 #10412: ARROW-9430: [C++] Implement replace_with_mask kernel

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



##########
File path: cpp/src/arrow/compute/kernels/vector_replace.cc
##########
@@ -0,0 +1,466 @@
+// 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_scalar.h"
+#include "arrow/compute/kernels/common.h"
+#include "arrow/util/bitmap_ops.h"
+
+namespace arrow {
+namespace compute {
+namespace internal {
+
+namespace {
+
+Status ReplacementArrayTooShort(int64_t expected, int64_t actual) {
+  return Status::Invalid("Replacement array must be of appropriate length (expected ",
+                         expected, " items but got ", actual, " items)");
+}
+
+// Helper to implement replace_with kernel with scalar mask for fixed-width types,
+// using callbacks to handle both bool and byte-sized types
+Status ReplaceWithScalarMask(KernelContext* ctx, const ArrayData& array,
+                             const BooleanScalar& mask, const Datum& replacements,
+                             ArrayData* output) {
+  if (!mask.is_valid) {
+    // Output = null
+    ARROW_ASSIGN_OR_RAISE(auto replacement_array,
+                          MakeArrayOfNull(array.type, array.length, ctx->memory_pool()));
+    *output = *replacement_array->data();
+    return Status::OK();
+  }
+  if (mask.value) {
+    // Output = replacement
+    if (replacements.is_scalar()) {
+      ARROW_ASSIGN_OR_RAISE(
+          auto replacement_array,
+          MakeArrayFromScalar(*replacements.scalar(), array.length, ctx->memory_pool()));
+      *output = *replacement_array->data();
+    } else {
+      auto replacement_array = replacements.array();
+      if (replacement_array->length != array.length) {
+        return ReplacementArrayTooShort(array.length, replacement_array->length);
+      }
+      *output = *replacement_array;
+    }
+  } else {
+    // Output = input
+    *output = array;
+  }
+  return Status::OK();
+}
+
+// Helper to implement replace_with kernel with array mask for fixed-width types,
+// using callbacks to handle both bool and byte-sized types and to handle
+// scalar and array replacements
+template <typename Functor>
+Status ReplaceWithArrayMask(KernelContext* ctx, const ArrayData& array,
+                            const ArrayData& mask, const Datum& replacements,
+                            ArrayData* output) {
+  uint8_t* out_bitmap = nullptr;
+  uint8_t* out_values = output->buffers[1]->mutable_data();
+  const uint8_t* mask_bitmap = mask.MayHaveNulls() ? mask.buffers[0]->data() : nullptr;
+  const uint8_t* mask_values = mask.buffers[1]->data();
+  bool replacements_bitmap;
+  int64_t replacements_length;
+  if (replacements.is_array()) {
+    replacements_bitmap = replacements.array()->MayHaveNulls();
+    replacements_length = replacements.array()->length;
+  } else {
+    replacements_bitmap = !replacements.scalar()->is_valid;
+    replacements_length = std::numeric_limits<int64_t>::max();
+  }
+  if (array.MayHaveNulls() || mask.MayHaveNulls() || replacements_bitmap) {
+    ARROW_ASSIGN_OR_RAISE(output->buffers[0], ctx->AllocateBitmap(array.length));
+    out_bitmap = output->buffers[0]->mutable_data();
+    output->null_count = -1;
+    if (array.MayHaveNulls()) {
+      // Copy array's bitmap
+      arrow::internal::CopyBitmap(array.buffers[0]->data(), array.offset, array.length,
+                                  out_bitmap, /*dest_offset=*/0);
+    } else {
+      // Array has no bitmap but mask/replacements do, generate an all-valid bitmap
+      std::memset(out_bitmap, 0xFF, output->buffers[0]->size());
+    }
+  } else {
+    output->null_count = 0;
+  }
+  auto copy_bitmap = [&](int64_t out_offset, int64_t in_offset, int64_t length) {
+    DCHECK(out_bitmap);
+    if (replacements.is_array()) {
+      const auto& in_data = *replacements.array();
+      const auto in_bitmap = in_data.GetValues<uint8_t>(0, /*absolute_offset=*/0);
+      arrow::internal::CopyBitmap(in_bitmap, in_data.offset + in_offset, length,
+                                  out_bitmap, out_offset);
+    } else {
+      BitUtil::SetBitsTo(out_bitmap, out_offset, length, !replacements_bitmap);
+    }
+  };
+
+  Functor::CopyData(*array.type, out_values, /*out_offset=*/0, array, /*in_offset=*/0,
+                    array.length);
+  arrow::internal::BitBlockCounter value_counter(mask_values, mask.offset, mask.length);
+  arrow::internal::OptionalBitBlockCounter valid_counter(mask_bitmap, mask.offset,
+                                                         mask.length);
+  int64_t out_offset = 0;
+  int64_t replacements_offset = 0;
+  while (out_offset < array.length) {
+    BitBlockCount value_block = value_counter.NextWord();
+    BitBlockCount valid_block = valid_counter.NextWord();
+    DCHECK_EQ(value_block.length, valid_block.length);
+    if (value_block.AllSet() && valid_block.AllSet()) {
+      // Copy from replacement array
+      if (replacements_offset + valid_block.length > replacements_length) {
+        return ReplacementArrayTooShort(replacements_offset + valid_block.length,
+                                        replacements_length);
+      }
+      Functor::CopyData(*array.type, out_values, out_offset, replacements,
+                        replacements_offset, valid_block.length);
+      if (replacements_bitmap) {
+        copy_bitmap(out_offset, replacements_offset, valid_block.length);
+      } else if (!replacements_bitmap && out_bitmap) {
+        BitUtil::SetBitsTo(out_bitmap, out_offset, valid_block.length, true);
+      }
+      replacements_offset += valid_block.length;
+    } else if ((value_block.NoneSet() && valid_block.AllSet()) || valid_block.NoneSet()) {
+      // Do nothing
+    } else {
+      for (int64_t i = 0; i < valid_block.length; ++i) {
+        if (BitUtil::GetBit(mask_values, out_offset + mask.offset + i) &&
+            (!mask_bitmap ||
+             BitUtil::GetBit(mask_bitmap, out_offset + mask.offset + i))) {
+          if (replacements_offset >= replacements_length) {

Review comment:
       About the same speed, so I'll go with this.




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

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

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



[GitHub] [arrow] nirandaperera commented on a change in pull request #10412: ARROW-9430: [C++] Implement replace_with_mask kernel

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



##########
File path: cpp/src/arrow/compute/kernels/vector_replace.cc
##########
@@ -0,0 +1,510 @@
+// 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_scalar.h"
+#include "arrow/compute/kernels/common.h"
+#include "arrow/util/bitmap_ops.h"
+
+namespace arrow {
+namespace compute {
+namespace internal {
+
+namespace {
+
+Status ReplacementArrayTooShort(int64_t expected, int64_t actual) {
+  return Status::Invalid("Replacement array must be of appropriate length (expected ",
+                         expected, " items but got ", actual, " items)");
+}
+
+// Helper to implement replace_with kernel with scalar mask for fixed-width types,
+// using callbacks to handle both bool and byte-sized types
+Status ReplaceWithScalarMask(KernelContext* ctx, const ArrayData& array,
+                             const BooleanScalar& mask, const Datum& replacements,
+                             ArrayData* output) {
+  if (!mask.is_valid) {
+    // Output = null
+    ARROW_ASSIGN_OR_RAISE(auto replacement_array,
+                          MakeArrayOfNull(array.type, array.length, ctx->memory_pool()));
+    *output = *replacement_array->data();
+    return Status::OK();
+  }
+  if (mask.value) {
+    // Output = replacement
+    if (replacements.is_scalar()) {
+      ARROW_ASSIGN_OR_RAISE(
+          auto replacement_array,
+          MakeArrayFromScalar(*replacements.scalar(), array.length, ctx->memory_pool()));
+      *output = *replacement_array->data();
+    } else {
+      auto replacement_array = replacements.array();
+      if (replacement_array->length != array.length) {
+        return ReplacementArrayTooShort(array.length, replacement_array->length);
+      }
+      *output = *replacement_array;
+    }
+  } else {
+    // Output = input
+    *output = array;
+  }
+  return Status::OK();
+}
+
+struct CopyArrayBitmap {
+  const uint8_t* in_bitmap;
+  int64_t in_offset;
+
+  void CopyBitmap(uint8_t* out_bitmap, int64_t out_offset, int64_t offset,
+                  int64_t length) const {
+    arrow::internal::CopyBitmap(in_bitmap, in_offset + offset, length, out_bitmap,
+                                out_offset);
+  }
+
+  void SetBit(uint8_t* out_bitmap, int64_t out_offset, int64_t offset) const {
+    BitUtil::SetBitTo(out_bitmap, out_offset,
+                      BitUtil::GetBit(in_bitmap, in_offset + offset));
+  }
+};
+
+struct CopyScalarBitmap {
+  const bool is_valid;
+
+  void CopyBitmap(uint8_t* out_bitmap, int64_t out_offset, int64_t offset,
+                  int64_t length) const {
+    BitUtil::SetBitsTo(out_bitmap, out_offset, length, is_valid);
+  }
+
+  void SetBit(uint8_t* out_bitmap, int64_t out_offset, int64_t offset) const {
+    BitUtil::SetBitTo(out_bitmap, out_offset, is_valid);
+  }
+};
+
+// Helper to implement replace_with kernel with array mask for fixed-width types,
+// using callbacks to handle both bool and byte-sized types and to handle
+// scalar and array replacements
+template <typename Functor, typename Data, typename CopyBitmap>
+void ReplaceWithArrayMaskImpl(const ArrayData& array, const ArrayData& mask,
+                              const Data& replacements, bool replacements_bitmap,
+                              const CopyBitmap copy_bitmap, const uint8_t* mask_bitmap,

Review comment:
       ```suggestion
                                 const CopyBitmap &copy_bitmap, const uint8_t* mask_bitmap,
   ```

##########
File path: cpp/src/arrow/compute/kernels/vector_replace.cc
##########
@@ -0,0 +1,510 @@
+// 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_scalar.h"
+#include "arrow/compute/kernels/common.h"
+#include "arrow/util/bitmap_ops.h"
+
+namespace arrow {
+namespace compute {
+namespace internal {
+
+namespace {
+
+Status ReplacementArrayTooShort(int64_t expected, int64_t actual) {
+  return Status::Invalid("Replacement array must be of appropriate length (expected ",
+                         expected, " items but got ", actual, " items)");
+}
+
+// Helper to implement replace_with kernel with scalar mask for fixed-width types,
+// using callbacks to handle both bool and byte-sized types
+Status ReplaceWithScalarMask(KernelContext* ctx, const ArrayData& array,
+                             const BooleanScalar& mask, const Datum& replacements,
+                             ArrayData* output) {
+  if (!mask.is_valid) {
+    // Output = null
+    ARROW_ASSIGN_OR_RAISE(auto replacement_array,
+                          MakeArrayOfNull(array.type, array.length, ctx->memory_pool()));
+    *output = *replacement_array->data();

Review comment:
       would this be called for a FixedWidthType execution? If so, this might conflict with `computed_preallocate` option. :thinking: 

##########
File path: cpp/src/arrow/compute/kernels/vector_replace.cc
##########
@@ -0,0 +1,510 @@
+// 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_scalar.h"
+#include "arrow/compute/kernels/common.h"
+#include "arrow/util/bitmap_ops.h"
+
+namespace arrow {
+namespace compute {
+namespace internal {
+
+namespace {
+
+Status ReplacementArrayTooShort(int64_t expected, int64_t actual) {
+  return Status::Invalid("Replacement array must be of appropriate length (expected ",
+                         expected, " items but got ", actual, " items)");
+}
+
+// Helper to implement replace_with kernel with scalar mask for fixed-width types,
+// using callbacks to handle both bool and byte-sized types
+Status ReplaceWithScalarMask(KernelContext* ctx, const ArrayData& array,
+                             const BooleanScalar& mask, const Datum& replacements,
+                             ArrayData* output) {
+  if (!mask.is_valid) {
+    // Output = null
+    ARROW_ASSIGN_OR_RAISE(auto replacement_array,
+                          MakeArrayOfNull(array.type, array.length, ctx->memory_pool()));
+    *output = *replacement_array->data();
+    return Status::OK();
+  }
+  if (mask.value) {
+    // Output = replacement
+    if (replacements.is_scalar()) {
+      ARROW_ASSIGN_OR_RAISE(
+          auto replacement_array,
+          MakeArrayFromScalar(*replacements.scalar(), array.length, ctx->memory_pool()));
+      *output = *replacement_array->data();
+    } else {
+      auto replacement_array = replacements.array();
+      if (replacement_array->length != array.length) {
+        return ReplacementArrayTooShort(array.length, replacement_array->length);
+      }
+      *output = *replacement_array;
+    }
+  } else {
+    // Output = input
+    *output = array;
+  }
+  return Status::OK();
+}
+
+struct CopyArrayBitmap {
+  const uint8_t* in_bitmap;
+  int64_t in_offset;
+
+  void CopyBitmap(uint8_t* out_bitmap, int64_t out_offset, int64_t offset,
+                  int64_t length) const {
+    arrow::internal::CopyBitmap(in_bitmap, in_offset + offset, length, out_bitmap,
+                                out_offset);
+  }
+
+  void SetBit(uint8_t* out_bitmap, int64_t out_offset, int64_t offset) const {
+    BitUtil::SetBitTo(out_bitmap, out_offset,
+                      BitUtil::GetBit(in_bitmap, in_offset + offset));
+  }
+};
+
+struct CopyScalarBitmap {
+  const bool is_valid;
+
+  void CopyBitmap(uint8_t* out_bitmap, int64_t out_offset, int64_t offset,
+                  int64_t length) const {
+    BitUtil::SetBitsTo(out_bitmap, out_offset, length, is_valid);
+  }
+
+  void SetBit(uint8_t* out_bitmap, int64_t out_offset, int64_t offset) const {
+    BitUtil::SetBitTo(out_bitmap, out_offset, is_valid);
+  }
+};
+
+// Helper to implement replace_with kernel with array mask for fixed-width types,
+// using callbacks to handle both bool and byte-sized types and to handle
+// scalar and array replacements
+template <typename Functor, typename Data, typename CopyBitmap>
+void ReplaceWithArrayMaskImpl(const ArrayData& array, const ArrayData& mask,
+                              const Data& replacements, bool replacements_bitmap,
+                              const CopyBitmap copy_bitmap, const uint8_t* mask_bitmap,
+                              const uint8_t* mask_values, uint8_t* out_bitmap,
+                              uint8_t* out_values, const int64_t out_offset) {
+  Functor::CopyData(*array.type, out_values, /*out_offset=*/0, array, /*in_offset=*/0,
+                    array.length);
+  arrow::internal::OptionalBinaryBitBlockCounter counter(
+      mask_values, mask.offset, mask_bitmap, mask.offset, mask.length);
+  int64_t write_offset = 0;
+  int64_t replacements_offset = 0;
+  while (write_offset < array.length) {
+    BitBlockCount block = counter.NextAndBlock();
+    if (block.AllSet()) {
+      // Copy from replacement array
+      Functor::CopyData(*array.type, out_values, out_offset + write_offset, replacements,
+                        replacements_offset, block.length);
+      if (replacements_bitmap) {
+        copy_bitmap.CopyBitmap(out_bitmap, out_offset + write_offset, replacements_offset,
+                               block.length);
+      } else if (!replacements_bitmap && out_bitmap) {
+        BitUtil::SetBitsTo(out_bitmap, out_offset + write_offset, block.length, true);
+      }
+      replacements_offset += block.length;
+    } else if (block.popcount) {
+      for (int64_t i = 0; i < block.length; ++i) {
+        if (BitUtil::GetBit(mask_values, write_offset + mask.offset + i) &&
+            (!mask_bitmap ||
+             BitUtil::GetBit(mask_bitmap, write_offset + mask.offset + i))) {
+          Functor::CopyData(*array.type, out_values, out_offset + write_offset + i,
+                            replacements, replacements_offset, /*length=*/1);
+          if (replacements_bitmap) {
+            copy_bitmap.SetBit(out_bitmap, out_offset + write_offset + i,
+                               replacements_offset);
+          }
+          replacements_offset++;
+        }
+      }
+    }
+    write_offset += block.length;
+  }
+}
+
+template <typename Functor>
+Status ReplaceWithArrayMask(KernelContext* ctx, const ArrayData& array,
+                            const ArrayData& mask, const Datum& replacements,
+                            ArrayData* output) {
+  const int64_t out_offset = output->offset;
+  uint8_t* out_bitmap = nullptr;
+  uint8_t* out_values = output->buffers[1]->mutable_data();
+  const uint8_t* mask_bitmap = mask.MayHaveNulls() ? mask.buffers[0]->data() : nullptr;
+  const uint8_t* mask_values = mask.buffers[1]->data();
+  const bool replacements_bitmap = replacements.is_array()
+                                       ? replacements.array()->MayHaveNulls()
+                                       : !replacements.scalar()->is_valid;
+  if (replacements.is_array()) {
+    // Check that we have enough replacement values
+    const int64_t replacements_length = replacements.array()->length;
+
+    arrow::internal::OptionalBinaryBitBlockCounter counter(
+        mask_values, mask.offset, mask_bitmap, mask.offset, mask.length);
+    int64_t count = 0;
+    for (int64_t offset = 0; offset < mask.length;) {
+      BitBlockCount block = counter.NextAndBlock();
+      count += block.popcount;
+      offset += block.length;
+    }

Review comment:
       If we can create a `BooleanArray` from `mask`, then we can call `true_count` straight away! It does this internally IINM. 

##########
File path: cpp/src/arrow/compute/kernels/vector_replace.cc
##########
@@ -0,0 +1,510 @@
+// 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_scalar.h"
+#include "arrow/compute/kernels/common.h"
+#include "arrow/util/bitmap_ops.h"
+
+namespace arrow {
+namespace compute {
+namespace internal {
+
+namespace {
+
+Status ReplacementArrayTooShort(int64_t expected, int64_t actual) {
+  return Status::Invalid("Replacement array must be of appropriate length (expected ",
+                         expected, " items but got ", actual, " items)");
+}
+
+// Helper to implement replace_with kernel with scalar mask for fixed-width types,
+// using callbacks to handle both bool and byte-sized types
+Status ReplaceWithScalarMask(KernelContext* ctx, const ArrayData& array,
+                             const BooleanScalar& mask, const Datum& replacements,
+                             ArrayData* output) {
+  if (!mask.is_valid) {
+    // Output = null
+    ARROW_ASSIGN_OR_RAISE(auto replacement_array,
+                          MakeArrayOfNull(array.type, array.length, ctx->memory_pool()));
+    *output = *replacement_array->data();

Review comment:
       same applies to other mem allocations in the function. 

##########
File path: cpp/src/arrow/compute/kernels/vector_replace.cc
##########
@@ -0,0 +1,510 @@
+// 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_scalar.h"
+#include "arrow/compute/kernels/common.h"
+#include "arrow/util/bitmap_ops.h"
+
+namespace arrow {
+namespace compute {
+namespace internal {
+
+namespace {
+
+Status ReplacementArrayTooShort(int64_t expected, int64_t actual) {
+  return Status::Invalid("Replacement array must be of appropriate length (expected ",
+                         expected, " items but got ", actual, " items)");
+}
+
+// Helper to implement replace_with kernel with scalar mask for fixed-width types,
+// using callbacks to handle both bool and byte-sized types
+Status ReplaceWithScalarMask(KernelContext* ctx, const ArrayData& array,
+                             const BooleanScalar& mask, const Datum& replacements,
+                             ArrayData* output) {
+  if (!mask.is_valid) {
+    // Output = null
+    ARROW_ASSIGN_OR_RAISE(auto replacement_array,
+                          MakeArrayOfNull(array.type, array.length, ctx->memory_pool()));
+    *output = *replacement_array->data();
+    return Status::OK();
+  }
+  if (mask.value) {
+    // Output = replacement
+    if (replacements.is_scalar()) {
+      ARROW_ASSIGN_OR_RAISE(
+          auto replacement_array,
+          MakeArrayFromScalar(*replacements.scalar(), array.length, ctx->memory_pool()));
+      *output = *replacement_array->data();
+    } else {
+      auto replacement_array = replacements.array();
+      if (replacement_array->length != array.length) {
+        return ReplacementArrayTooShort(array.length, replacement_array->length);
+      }
+      *output = *replacement_array;
+    }
+  } else {
+    // Output = input
+    *output = array;
+  }
+  return Status::OK();
+}
+
+struct CopyArrayBitmap {
+  const uint8_t* in_bitmap;
+  int64_t in_offset;
+
+  void CopyBitmap(uint8_t* out_bitmap, int64_t out_offset, int64_t offset,
+                  int64_t length) const {
+    arrow::internal::CopyBitmap(in_bitmap, in_offset + offset, length, out_bitmap,
+                                out_offset);
+  }
+
+  void SetBit(uint8_t* out_bitmap, int64_t out_offset, int64_t offset) const {
+    BitUtil::SetBitTo(out_bitmap, out_offset,
+                      BitUtil::GetBit(in_bitmap, in_offset + offset));
+  }
+};
+
+struct CopyScalarBitmap {
+  const bool is_valid;
+
+  void CopyBitmap(uint8_t* out_bitmap, int64_t out_offset, int64_t offset,
+                  int64_t length) const {
+    BitUtil::SetBitsTo(out_bitmap, out_offset, length, is_valid);
+  }
+
+  void SetBit(uint8_t* out_bitmap, int64_t out_offset, int64_t offset) const {
+    BitUtil::SetBitTo(out_bitmap, out_offset, is_valid);
+  }
+};
+
+// Helper to implement replace_with kernel with array mask for fixed-width types,
+// using callbacks to handle both bool and byte-sized types and to handle
+// scalar and array replacements
+template <typename Functor, typename Data, typename CopyBitmap>
+void ReplaceWithArrayMaskImpl(const ArrayData& array, const ArrayData& mask,
+                              const Data& replacements, bool replacements_bitmap,
+                              const CopyBitmap copy_bitmap, const uint8_t* mask_bitmap,
+                              const uint8_t* mask_values, uint8_t* out_bitmap,
+                              uint8_t* out_values, const int64_t out_offset) {
+  Functor::CopyData(*array.type, out_values, /*out_offset=*/0, array, /*in_offset=*/0,
+                    array.length);
+  arrow::internal::OptionalBinaryBitBlockCounter counter(
+      mask_values, mask.offset, mask_bitmap, mask.offset, mask.length);
+  int64_t write_offset = 0;
+  int64_t replacements_offset = 0;
+  while (write_offset < array.length) {
+    BitBlockCount block = counter.NextAndBlock();
+    if (block.AllSet()) {
+      // Copy from replacement array
+      Functor::CopyData(*array.type, out_values, out_offset + write_offset, replacements,
+                        replacements_offset, block.length);
+      if (replacements_bitmap) {
+        copy_bitmap.CopyBitmap(out_bitmap, out_offset + write_offset, replacements_offset,
+                               block.length);
+      } else if (!replacements_bitmap && out_bitmap) {
+        BitUtil::SetBitsTo(out_bitmap, out_offset + write_offset, block.length, true);
+      }
+      replacements_offset += block.length;
+    } else if (block.popcount) {
+      for (int64_t i = 0; i < block.length; ++i) {
+        if (BitUtil::GetBit(mask_values, write_offset + mask.offset + i) &&
+            (!mask_bitmap ||
+             BitUtil::GetBit(mask_bitmap, write_offset + mask.offset + i))) {
+          Functor::CopyData(*array.type, out_values, out_offset + write_offset + i,
+                            replacements, replacements_offset, /*length=*/1);
+          if (replacements_bitmap) {
+            copy_bitmap.SetBit(out_bitmap, out_offset + write_offset + i,
+                               replacements_offset);
+          }
+          replacements_offset++;
+        }
+      }
+    }
+    write_offset += block.length;
+  }
+}
+
+template <typename Functor>
+Status ReplaceWithArrayMask(KernelContext* ctx, const ArrayData& array,
+                            const ArrayData& mask, const Datum& replacements,
+                            ArrayData* output) {
+  const int64_t out_offset = output->offset;
+  uint8_t* out_bitmap = nullptr;
+  uint8_t* out_values = output->buffers[1]->mutable_data();
+  const uint8_t* mask_bitmap = mask.MayHaveNulls() ? mask.buffers[0]->data() : nullptr;
+  const uint8_t* mask_values = mask.buffers[1]->data();
+  const bool replacements_bitmap = replacements.is_array()
+                                       ? replacements.array()->MayHaveNulls()
+                                       : !replacements.scalar()->is_valid;
+  if (replacements.is_array()) {
+    // Check that we have enough replacement values
+    const int64_t replacements_length = replacements.array()->length;
+
+    arrow::internal::OptionalBinaryBitBlockCounter counter(
+        mask_values, mask.offset, mask_bitmap, mask.offset, mask.length);
+    int64_t count = 0;
+    for (int64_t offset = 0; offset < mask.length;) {
+      BitBlockCount block = counter.NextAndBlock();
+      count += block.popcount;
+      offset += block.length;
+    }
+    if (count > replacements_length) {
+      return ReplacementArrayTooShort(count, replacements_length);
+    }
+  }
+  if (array.MayHaveNulls() || mask.MayHaveNulls() || replacements_bitmap) {
+    out_bitmap = output->buffers[0]->mutable_data();
+    output->null_count = -1;
+    if (array.MayHaveNulls()) {
+      // Copy array's bitmap
+      arrow::internal::CopyBitmap(array.buffers[0]->data(), array.offset, array.length,
+                                  out_bitmap, out_offset);
+    } else {
+      // Array has no bitmap but mask/replacements do, generate an all-valid bitmap
+      BitUtil::SetBitsTo(out_bitmap, out_offset, array.length, true);
+    }
+  } else {
+    BitUtil::SetBitsTo(output->buffers[0]->mutable_data(), out_offset, array.length,
+                       true);
+    output->null_count = 0;
+  }
+
+  if (replacements.is_array()) {
+    const ArrayData& array_repl = *replacements.array();
+    ReplaceWithArrayMaskImpl<Functor>(
+        array, mask, array_repl, replacements_bitmap,
+        CopyArrayBitmap{replacements_bitmap ? array_repl.buffers[0]->data() : nullptr,
+                        array_repl.offset},
+        mask_bitmap, mask_values, out_bitmap, out_values, out_offset);
+  } else {
+    const Scalar& scalar_repl = *replacements.scalar();
+    ReplaceWithArrayMaskImpl<Functor>(array, mask, scalar_repl, replacements_bitmap,
+                                      CopyScalarBitmap{scalar_repl.is_valid}, mask_bitmap,
+                                      mask_values, out_bitmap, out_values, out_offset);
+  }
+
+  if (mask.MayHaveNulls()) {
+    arrow::internal::BitmapAnd(out_bitmap, out_offset, mask.buffers[0]->data(),
+                               mask.offset, array.length, out_offset, out_bitmap);
+  }
+  return Status::OK();
+}
+
+template <typename Type, typename Enable = void>
+struct ReplaceWithMask {};
+
+template <typename Type>
+struct ReplaceWithMask<Type, enable_if_number<Type>> {
+  using T = typename TypeTraits<Type>::CType;
+
+  static void CopyData(const DataType&, uint8_t* out, const int64_t out_offset,
+                       const ArrayData& in, const int64_t in_offset,
+                       const int64_t length) {
+    const auto in_arr = in.GetValues<uint8_t>(1, (in_offset + in.offset) * sizeof(T));
+    std::memcpy(out + (out_offset * sizeof(T)), in_arr, length * sizeof(T));
+  }
+
+  static void CopyData(const DataType&, uint8_t* out, const int64_t out_offset,
+                       const Scalar& in, const int64_t in_offset, const int64_t length) {
+    T* begin = reinterpret_cast<T*>(out + (out_offset * sizeof(T)));
+    T* end = begin + length;
+    std::fill(begin, end, UnboxScalar<Type>::Unbox(in));
+  }
+
+  static Status ExecScalarMask(KernelContext* ctx, const ArrayData& array,
+                               const BooleanScalar& mask, const Datum& replacements,
+                               ArrayData* output) {
+    return ReplaceWithScalarMask(ctx, array, mask, replacements, output);
+  }
+
+  static Status ExecArrayMask(KernelContext* ctx, const ArrayData& array,
+                              const ArrayData& mask, const Datum& replacements,
+                              ArrayData* output) {
+    return ReplaceWithArrayMask<ReplaceWithMask<Type>>(ctx, array, mask, replacements,
+                                                       output);
+  }
+};
+
+template <typename Type>
+struct ReplaceWithMask<Type, enable_if_boolean<Type>> {
+  static void CopyData(const DataType&, uint8_t* out, const int64_t out_offset,
+                       const ArrayData& in, const int64_t in_offset,
+                       const int64_t length) {
+    const auto in_arr = in.GetValues<uint8_t>(1, /*absolute_offset=*/0);
+    arrow::internal::CopyBitmap(in_arr, in_offset + in.offset, length, out, out_offset);
+  }
+  static void CopyData(const DataType&, uint8_t* out, const int64_t out_offset,
+                       const Scalar& in, const int64_t in_offset, const int64_t length) {
+    BitUtil::SetBitsTo(out, out_offset, length, in.is_valid);
+  }
+
+  static Status ExecScalarMask(KernelContext* ctx, const ArrayData& array,
+                               const BooleanScalar& mask, const Datum& replacements,
+                               ArrayData* output) {
+    return ReplaceWithScalarMask(ctx, array, mask, replacements, output);
+  }
+  static Status ExecArrayMask(KernelContext* ctx, const ArrayData& array,
+                              const ArrayData& mask, const Datum& replacements,
+                              ArrayData* output) {
+    return ReplaceWithArrayMask<ReplaceWithMask<Type>>(ctx, array, mask, replacements,
+                                                       output);
+  }
+};
+
+template <typename Type>
+struct ReplaceWithMask<Type, enable_if_same<Type, FixedSizeBinaryType>> {
+  static void CopyData(const DataType& ty, uint8_t* out, const int64_t out_offset,
+                       const ArrayData& in, const int64_t in_offset,
+                       const int64_t length) {
+    const int32_t width = checked_cast<const FixedSizeBinaryType&>(ty).byte_width();
+    uint8_t* begin = out + (out_offset * width);
+    const auto in_arr = in.GetValues<uint8_t>(1, (in_offset + in.offset) * width);
+    std::memcpy(begin, in_arr, length * width);
+  }
+  static void CopyData(const DataType& ty, uint8_t* out, const int64_t out_offset,
+                       const Scalar& in, const int64_t in_offset, const int64_t length) {
+    const int32_t width = checked_cast<const FixedSizeBinaryType&>(ty).byte_width();
+    uint8_t* begin = out + (out_offset * width);
+    const auto& scalar = checked_cast<const FixedSizeBinaryScalar&>(in);
+    // Null scalar may have null value buffer
+    if (!scalar.value) return;
+    const Buffer& buffer = *scalar.value;
+    const uint8_t* value = buffer.data();
+    DCHECK_GE(buffer.size(), width);
+    for (int i = 0; i < length; i++) {
+      std::memcpy(begin, value, width);
+      begin += width;
+    }
+  }
+
+  static Status ExecScalarMask(KernelContext* ctx, const ArrayData& array,
+                               const BooleanScalar& mask, const Datum& replacements,
+                               ArrayData* output) {
+    return ReplaceWithScalarMask(ctx, array, mask, replacements, output);
+  }
+
+  static Status ExecArrayMask(KernelContext* ctx, const ArrayData& array,
+                              const ArrayData& mask, const Datum& replacements,
+                              ArrayData* output) {
+    return ReplaceWithArrayMask<ReplaceWithMask<Type>>(ctx, array, mask, replacements,
+                                                       output);
+  }
+};
+
+template <typename Type>
+struct ReplaceWithMask<Type, enable_if_decimal<Type>> {
+  using ScalarType = typename TypeTraits<Type>::ScalarType;
+  static void CopyData(const DataType& ty, uint8_t* out, const int64_t out_offset,
+                       const ArrayData& in, const int64_t in_offset,
+                       const int64_t length) {
+    const int32_t width = checked_cast<const FixedSizeBinaryType&>(ty).byte_width();
+    uint8_t* begin = out + (out_offset * width);
+    const auto in_arr = in.GetValues<uint8_t>(1, (in_offset + in.offset) * width);
+    std::memcpy(begin, in_arr, length * width);
+  }
+  static void CopyData(const DataType& ty, uint8_t* out, const int64_t out_offset,
+                       const Scalar& in, const int64_t in_offset, const int64_t length) {
+    const int32_t width = checked_cast<const FixedSizeBinaryType&>(ty).byte_width();
+    uint8_t* begin = out + (out_offset * width);
+    const auto& scalar = checked_cast<const ScalarType&>(in);
+    const auto value = scalar.value.ToBytes();
+    for (int i = 0; i < length; i++) {
+      std::memcpy(begin, value.data(), width);
+      begin += width;
+    }
+  }
+
+  static Status ExecScalarMask(KernelContext* ctx, const ArrayData& array,
+                               const BooleanScalar& mask, const Datum& replacements,
+                               ArrayData* output) {
+    return ReplaceWithScalarMask(ctx, array, mask, replacements, output);
+  }
+
+  static Status ExecArrayMask(KernelContext* ctx, const ArrayData& array,
+                              const ArrayData& mask, const Datum& replacements,
+                              ArrayData* output) {
+    return ReplaceWithArrayMask<ReplaceWithMask<Type>>(ctx, array, mask, replacements,
+                                                       output);
+  }
+};
+
+template <typename Type>
+struct ReplaceWithMask<Type, enable_if_null<Type>> {
+  static Status ExecScalarMask(KernelContext* ctx, const ArrayData& array,
+                               const BooleanScalar& mask, const Datum& replacements,
+                               ArrayData* output) {
+    *output = array;
+    return Status::OK();
+  }
+  static Status ExecArrayMask(KernelContext* ctx, const ArrayData& array,
+                              const ArrayData& mask, const Datum& replacements,
+                              ArrayData* output) {
+    *output = array;
+    return Status::OK();
+  }
+};
+
+template <typename Type>
+struct ReplaceWithMask<Type, enable_if_base_binary<Type>> {
+  using offset_type = typename Type::offset_type;
+  using BuilderType = typename TypeTraits<Type>::BuilderType;
+
+  static Status ExecScalarMask(KernelContext* ctx, const ArrayData& array,
+                               const BooleanScalar& mask, const Datum& replacements,
+                               ArrayData* output) {
+    return ReplaceWithScalarMask(ctx, array, mask, replacements, output);
+  }
+  static Status ExecArrayMask(KernelContext* ctx, const ArrayData& array,
+                              const ArrayData& mask, const Datum& replacements,
+                              ArrayData* output) {
+    BuilderType builder(array.type, ctx->memory_pool());
+    RETURN_NOT_OK(builder.Reserve(array.length));
+    RETURN_NOT_OK(builder.ReserveData(array.buffers[2]->size()));
+    int64_t source_offset = 0;
+    int64_t replacements_offset = 0;
+    RETURN_NOT_OK(VisitArrayDataInline<BooleanType>(
+        mask,
+        [&](bool replace) {
+          if (replace && replacements.is_scalar()) {
+            const Scalar& scalar = *replacements.scalar();
+            if (scalar.is_valid) {
+              RETURN_NOT_OK(builder.Append(UnboxScalar<Type>::Unbox(scalar)));
+            } else {
+              RETURN_NOT_OK(builder.AppendNull());
+            }
+          } else {
+            const ArrayData& source = replace ? *replacements.array() : array;
+            const int64_t offset = replace ? replacements_offset++ : source_offset;
+            if (!source.MayHaveNulls() ||
+                BitUtil::GetBit(source.buffers[0]->data(), source.offset + offset)) {
+              const uint8_t* data = source.buffers[2]->data();
+              const offset_type* offsets = source.GetValues<offset_type>(1);
+              const offset_type offset0 = offsets[offset];
+              const offset_type offset1 = offsets[offset + 1];
+              RETURN_NOT_OK(builder.Append(data + offset0, offset1 - offset0));
+            } else {
+              RETURN_NOT_OK(builder.AppendNull());
+            }
+          }
+          source_offset++;
+          return Status::OK();
+        },
+        [&]() {
+          RETURN_NOT_OK(builder.AppendNull());
+          source_offset++;
+          return Status::OK();
+        }));
+    std::shared_ptr<Array> temp_output;
+    RETURN_NOT_OK(builder.Finish(&temp_output));
+    *output = *temp_output->data();
+    // Builder type != logical type due to GenerateTypeAgnosticVarBinaryBase
+    output->type = array.type;
+    return Status::OK();
+  }
+};
+
+template <typename Type>
+struct ReplaceWithMaskFunctor {
+  static Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    const ArrayData& array = *batch[0].array();
+    const Datum& replacements = batch[2];
+    ArrayData* output = out->array().get();
+    output->length = array.length;
+
+    // Needed for FixedSizeBinary/parameterized types
+    if (!array.type->Equals(*replacements.type(), /*check_metadata=*/false)) {
+      return Status::Invalid("Replacements must be of same type (expected ",
+                             array.type->ToString(), " but got ",
+                             replacements.type()->ToString(), ")");
+    }
+
+    if (!replacements.is_array() && !replacements.is_scalar()) {
+      return Status::Invalid("Replacements must be array or scalar");
+    }
+
+    if (batch[1].is_scalar()) {
+      return ReplaceWithMask<Type>::ExecScalarMask(
+          ctx, array, batch[1].scalar_as<BooleanScalar>(), replacements, output);
+    }
+    const ArrayData& mask = *batch[1].array();
+    if (array.length != mask.length) {
+      return Status::Invalid("Mask must be of same length as array (expected ",
+                             array.length, " items but got ", mask.length, " items)");
+    }
+    return ReplaceWithMask<Type>::ExecArrayMask(ctx, array, mask, replacements, output);
+  }
+};
+
+}  // namespace
+
+const FunctionDoc replace_with_mask_doc(
+    "Replace items using a mask and replacement values",
+    ("Given an array and a Boolean mask (either scalar or of equal length), "
+     "along with replacement values (either scalar or array), "
+     "each element of the array for which the corresponding mask element is "
+     "true will be replaced by the next value from the replacements, "
+     "or with null if the mask is null. "
+     "Hence, for replacement arrays, len(replacements) == sum(mask == true)."),
+    {"values", "mask", "replacements"});
+
+void RegisterVectorReplace(FunctionRegistry* registry) {
+  auto func = std::make_shared<VectorFunction>("replace_with_mask", Arity::Ternary(),
+                                               &replace_with_mask_doc);
+  auto add_kernel = [&](detail::GetTypeId get_id, ArrayKernelExec exec) {
+    VectorKernel kernel;
+    kernel.can_execute_chunkwise = false;
+    if (is_fixed_width(get_id.id)) {
+      kernel.null_handling = NullHandling::type::COMPUTED_PREALLOCATE;
+    } else {
+      kernel.can_write_into_slices = false;
+      kernel.null_handling = NullHandling::type::COMPUTED_NO_PREALLOCATE;
+    }
+    kernel.mem_allocation = MemAllocation::type::PREALLOCATE;

Review comment:
       I have a feeling that we should leave all kernels as following,
   ```
   kernel.null_handling = NullHandling::type::COMPUTED_NO_PREALLOCATE;
   kernel.mem_allocation = MemAllocation::type::NO_PREALLOCATE;
   ```
   and later, change these flags. One thing I realized was,  NullHandling::COMPUTED_PREALLOCATE, and MemAllocation::PREALLOCATE introduces a lot of niche cases. It was helpful for me to test those cases using `compute::CheckScalar` test util. It checks for slicing, chunks etc for scalar functions. 
   https://github.com/apache/arrow/blob/e990d177b1f1dec962315487682f613d46be573c/cpp/src/arrow/compute/kernels/test_util.cc#L106
   But now that we are on vector functions, the semantics might change :-) 




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

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

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



[GitHub] [arrow] bkietz commented on a change in pull request #10412: ARROW-9430: [C++] Implement replace_with_mask kernel

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



##########
File path: cpp/src/arrow/compute/kernels/vector_replace.cc
##########
@@ -0,0 +1,510 @@
+// 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_scalar.h"
+#include "arrow/compute/kernels/common.h"
+#include "arrow/util/bitmap_ops.h"
+
+namespace arrow {
+namespace compute {
+namespace internal {
+
+namespace {
+
+Status ReplacementArrayTooShort(int64_t expected, int64_t actual) {
+  return Status::Invalid("Replacement array must be of appropriate length (expected ",
+                         expected, " items but got ", actual, " items)");
+}
+
+// Helper to implement replace_with kernel with scalar mask for fixed-width types,
+// using callbacks to handle both bool and byte-sized types
+Status ReplaceWithScalarMask(KernelContext* ctx, const ArrayData& array,
+                             const BooleanScalar& mask, const Datum& replacements,
+                             ArrayData* output) {
+  if (!mask.is_valid) {
+    // Output = null
+    ARROW_ASSIGN_OR_RAISE(auto replacement_array,
+                          MakeArrayOfNull(array.type, array.length, ctx->memory_pool()));
+    *output = *replacement_array->data();
+    return Status::OK();
+  }
+  if (mask.value) {
+    // Output = replacement
+    if (replacements.is_scalar()) {
+      ARROW_ASSIGN_OR_RAISE(
+          auto replacement_array,
+          MakeArrayFromScalar(*replacements.scalar(), array.length, ctx->memory_pool()));
+      *output = *replacement_array->data();
+    } else {
+      auto replacement_array = replacements.array();
+      if (replacement_array->length != array.length) {
+        return ReplacementArrayTooShort(array.length, replacement_array->length);
+      }
+      *output = *replacement_array;
+    }
+  } else {
+    // Output = input
+    *output = array;
+  }
+  return Status::OK();
+}
+
+struct CopyArrayBitmap {
+  const uint8_t* in_bitmap;
+  int64_t in_offset;
+
+  void CopyBitmap(uint8_t* out_bitmap, int64_t out_offset, int64_t offset,
+                  int64_t length) const {
+    arrow::internal::CopyBitmap(in_bitmap, in_offset + offset, length, out_bitmap,
+                                out_offset);
+  }
+
+  void SetBit(uint8_t* out_bitmap, int64_t out_offset, int64_t offset) const {
+    BitUtil::SetBitTo(out_bitmap, out_offset,
+                      BitUtil::GetBit(in_bitmap, in_offset + offset));
+  }
+};
+
+struct CopyScalarBitmap {
+  const bool is_valid;
+
+  void CopyBitmap(uint8_t* out_bitmap, int64_t out_offset, int64_t offset,
+                  int64_t length) const {
+    BitUtil::SetBitsTo(out_bitmap, out_offset, length, is_valid);
+  }
+
+  void SetBit(uint8_t* out_bitmap, int64_t out_offset, int64_t offset) const {
+    BitUtil::SetBitTo(out_bitmap, out_offset, is_valid);
+  }
+};
+
+// Helper to implement replace_with kernel with array mask for fixed-width types,
+// using callbacks to handle both bool and byte-sized types and to handle
+// scalar and array replacements
+template <typename Functor, typename Data, typename CopyBitmap>
+void ReplaceWithArrayMaskImpl(const ArrayData& array, const ArrayData& mask,
+                              const Data& replacements, bool replacements_bitmap,
+                              const CopyBitmap copy_bitmap, const uint8_t* mask_bitmap,
+                              const uint8_t* mask_values, uint8_t* out_bitmap,
+                              uint8_t* out_values, const int64_t out_offset) {
+  Functor::CopyData(*array.type, out_values, /*out_offset=*/0, array, /*in_offset=*/0,
+                    array.length);
+  arrow::internal::OptionalBinaryBitBlockCounter counter(
+      mask_values, mask.offset, mask_bitmap, mask.offset, mask.length);
+  int64_t write_offset = 0;
+  int64_t replacements_offset = 0;
+  while (write_offset < array.length) {
+    BitBlockCount block = counter.NextAndBlock();
+    if (block.AllSet()) {
+      // Copy from replacement array
+      Functor::CopyData(*array.type, out_values, out_offset + write_offset, replacements,
+                        replacements_offset, block.length);
+      if (replacements_bitmap) {
+        copy_bitmap.CopyBitmap(out_bitmap, out_offset + write_offset, replacements_offset,
+                               block.length);
+      } else if (!replacements_bitmap && out_bitmap) {
+        BitUtil::SetBitsTo(out_bitmap, out_offset + write_offset, block.length, true);
+      }
+      replacements_offset += block.length;
+    } else if (block.popcount) {
+      for (int64_t i = 0; i < block.length; ++i) {
+        if (BitUtil::GetBit(mask_values, write_offset + mask.offset + i) &&
+            (!mask_bitmap ||
+             BitUtil::GetBit(mask_bitmap, write_offset + mask.offset + i))) {
+          Functor::CopyData(*array.type, out_values, out_offset + write_offset + i,
+                            replacements, replacements_offset, /*length=*/1);
+          if (replacements_bitmap) {
+            copy_bitmap.SetBit(out_bitmap, out_offset + write_offset + i,
+                               replacements_offset);
+          }
+          replacements_offset++;
+        }
+      }
+    }
+    write_offset += block.length;
+  }
+}
+
+template <typename Functor>
+Status ReplaceWithArrayMask(KernelContext* ctx, const ArrayData& array,
+                            const ArrayData& mask, const Datum& replacements,
+                            ArrayData* output) {
+  const int64_t out_offset = output->offset;
+  uint8_t* out_bitmap = nullptr;
+  uint8_t* out_values = output->buffers[1]->mutable_data();
+  const uint8_t* mask_bitmap = mask.MayHaveNulls() ? mask.buffers[0]->data() : nullptr;
+  const uint8_t* mask_values = mask.buffers[1]->data();
+  const bool replacements_bitmap = replacements.is_array()
+                                       ? replacements.array()->MayHaveNulls()
+                                       : !replacements.scalar()->is_valid;
+  if (replacements.is_array()) {
+    // Check that we have enough replacement values
+    const int64_t replacements_length = replacements.array()->length;
+
+    arrow::internal::OptionalBinaryBitBlockCounter counter(
+        mask_values, mask.offset, mask_bitmap, mask.offset, mask.length);
+    int64_t count = 0;
+    for (int64_t offset = 0; offset < mask.length;) {
+      BitBlockCount block = counter.NextAndBlock();
+      count += block.popcount;
+      offset += block.length;
+    }
+    if (count > replacements_length) {
+      return ReplacementArrayTooShort(count, replacements_length);
+    }
+  }
+  if (array.MayHaveNulls() || mask.MayHaveNulls() || replacements_bitmap) {
+    out_bitmap = output->buffers[0]->mutable_data();
+    output->null_count = -1;
+    if (array.MayHaveNulls()) {
+      // Copy array's bitmap
+      arrow::internal::CopyBitmap(array.buffers[0]->data(), array.offset, array.length,
+                                  out_bitmap, out_offset);
+    } else {
+      // Array has no bitmap but mask/replacements do, generate an all-valid bitmap
+      BitUtil::SetBitsTo(out_bitmap, out_offset, array.length, true);
+    }
+  } else {
+    BitUtil::SetBitsTo(output->buffers[0]->mutable_data(), out_offset, array.length,
+                       true);
+    output->null_count = 0;
+  }
+
+  if (replacements.is_array()) {
+    const ArrayData& array_repl = *replacements.array();
+    ReplaceWithArrayMaskImpl<Functor>(
+        array, mask, array_repl, replacements_bitmap,
+        CopyArrayBitmap{replacements_bitmap ? array_repl.buffers[0]->data() : nullptr,
+                        array_repl.offset},
+        mask_bitmap, mask_values, out_bitmap, out_values, out_offset);
+  } else {
+    const Scalar& scalar_repl = *replacements.scalar();
+    ReplaceWithArrayMaskImpl<Functor>(array, mask, scalar_repl, replacements_bitmap,
+                                      CopyScalarBitmap{scalar_repl.is_valid}, mask_bitmap,
+                                      mask_values, out_bitmap, out_values, out_offset);
+  }
+
+  if (mask.MayHaveNulls()) {
+    arrow::internal::BitmapAnd(out_bitmap, out_offset, mask.buffers[0]->data(),
+                               mask.offset, array.length, out_offset, out_bitmap);
+  }
+  return Status::OK();
+}
+
+template <typename Type, typename Enable = void>
+struct ReplaceWithMask {};
+
+template <typename Type>
+struct ReplaceWithMask<Type, enable_if_number<Type>> {
+  using T = typename TypeTraits<Type>::CType;
+
+  static void CopyData(const DataType&, uint8_t* out, const int64_t out_offset,
+                       const ArrayData& in, const int64_t in_offset,
+                       const int64_t length) {
+    const auto in_arr = in.GetValues<uint8_t>(1, (in_offset + in.offset) * sizeof(T));
+    std::memcpy(out + (out_offset * sizeof(T)), in_arr, length * sizeof(T));
+  }
+
+  static void CopyData(const DataType&, uint8_t* out, const int64_t out_offset,
+                       const Scalar& in, const int64_t in_offset, const int64_t length) {
+    T* begin = reinterpret_cast<T*>(out + (out_offset * sizeof(T)));
+    T* end = begin + length;
+    std::fill(begin, end, UnboxScalar<Type>::Unbox(in));
+  }
+
+  static Status ExecScalarMask(KernelContext* ctx, const ArrayData& array,
+                               const BooleanScalar& mask, const Datum& replacements,
+                               ArrayData* output) {
+    return ReplaceWithScalarMask(ctx, array, mask, replacements, output);
+  }
+
+  static Status ExecArrayMask(KernelContext* ctx, const ArrayData& array,
+                              const ArrayData& mask, const Datum& replacements,
+                              ArrayData* output) {
+    return ReplaceWithArrayMask<ReplaceWithMask<Type>>(ctx, array, mask, replacements,
+                                                       output);
+  }
+};
+
+template <typename Type>
+struct ReplaceWithMask<Type, enable_if_boolean<Type>> {
+  static void CopyData(const DataType&, uint8_t* out, const int64_t out_offset,
+                       const ArrayData& in, const int64_t in_offset,
+                       const int64_t length) {
+    const auto in_arr = in.GetValues<uint8_t>(1, /*absolute_offset=*/0);
+    arrow::internal::CopyBitmap(in_arr, in_offset + in.offset, length, out, out_offset);
+  }
+  static void CopyData(const DataType&, uint8_t* out, const int64_t out_offset,
+                       const Scalar& in, const int64_t in_offset, const int64_t length) {
+    BitUtil::SetBitsTo(out, out_offset, length, in.is_valid);
+  }
+
+  static Status ExecScalarMask(KernelContext* ctx, const ArrayData& array,
+                               const BooleanScalar& mask, const Datum& replacements,
+                               ArrayData* output) {
+    return ReplaceWithScalarMask(ctx, array, mask, replacements, output);
+  }
+  static Status ExecArrayMask(KernelContext* ctx, const ArrayData& array,
+                              const ArrayData& mask, const Datum& replacements,
+                              ArrayData* output) {
+    return ReplaceWithArrayMask<ReplaceWithMask<Type>>(ctx, array, mask, replacements,
+                                                       output);
+  }
+};
+
+template <typename Type>
+struct ReplaceWithMask<Type, enable_if_same<Type, FixedSizeBinaryType>> {
+  static void CopyData(const DataType& ty, uint8_t* out, const int64_t out_offset,
+                       const ArrayData& in, const int64_t in_offset,
+                       const int64_t length) {
+    const int32_t width = checked_cast<const FixedSizeBinaryType&>(ty).byte_width();
+    uint8_t* begin = out + (out_offset * width);
+    const auto in_arr = in.GetValues<uint8_t>(1, (in_offset + in.offset) * width);
+    std::memcpy(begin, in_arr, length * width);
+  }
+  static void CopyData(const DataType& ty, uint8_t* out, const int64_t out_offset,
+                       const Scalar& in, const int64_t in_offset, const int64_t length) {
+    const int32_t width = checked_cast<const FixedSizeBinaryType&>(ty).byte_width();
+    uint8_t* begin = out + (out_offset * width);
+    const auto& scalar = checked_cast<const FixedSizeBinaryScalar&>(in);
+    // Null scalar may have null value buffer
+    if (!scalar.value) return;
+    const Buffer& buffer = *scalar.value;
+    const uint8_t* value = buffer.data();
+    DCHECK_GE(buffer.size(), width);
+    for (int i = 0; i < length; i++) {
+      std::memcpy(begin, value, width);
+      begin += width;
+    }
+  }
+
+  static Status ExecScalarMask(KernelContext* ctx, const ArrayData& array,
+                               const BooleanScalar& mask, const Datum& replacements,
+                               ArrayData* output) {
+    return ReplaceWithScalarMask(ctx, array, mask, replacements, output);
+  }
+
+  static Status ExecArrayMask(KernelContext* ctx, const ArrayData& array,
+                              const ArrayData& mask, const Datum& replacements,
+                              ArrayData* output) {
+    return ReplaceWithArrayMask<ReplaceWithMask<Type>>(ctx, array, mask, replacements,
+                                                       output);
+  }
+};
+
+template <typename Type>
+struct ReplaceWithMask<Type, enable_if_decimal<Type>> {
+  using ScalarType = typename TypeTraits<Type>::ScalarType;
+  static void CopyData(const DataType& ty, uint8_t* out, const int64_t out_offset,
+                       const ArrayData& in, const int64_t in_offset,
+                       const int64_t length) {
+    const int32_t width = checked_cast<const FixedSizeBinaryType&>(ty).byte_width();
+    uint8_t* begin = out + (out_offset * width);
+    const auto in_arr = in.GetValues<uint8_t>(1, (in_offset + in.offset) * width);
+    std::memcpy(begin, in_arr, length * width);
+  }
+  static void CopyData(const DataType& ty, uint8_t* out, const int64_t out_offset,
+                       const Scalar& in, const int64_t in_offset, const int64_t length) {
+    const int32_t width = checked_cast<const FixedSizeBinaryType&>(ty).byte_width();
+    uint8_t* begin = out + (out_offset * width);
+    const auto& scalar = checked_cast<const ScalarType&>(in);
+    const auto value = scalar.value.ToBytes();
+    for (int i = 0; i < length; i++) {
+      std::memcpy(begin, value.data(), width);
+      begin += width;
+    }
+  }
+
+  static Status ExecScalarMask(KernelContext* ctx, const ArrayData& array,
+                               const BooleanScalar& mask, const Datum& replacements,
+                               ArrayData* output) {
+    return ReplaceWithScalarMask(ctx, array, mask, replacements, output);
+  }
+
+  static Status ExecArrayMask(KernelContext* ctx, const ArrayData& array,
+                              const ArrayData& mask, const Datum& replacements,
+                              ArrayData* output) {
+    return ReplaceWithArrayMask<ReplaceWithMask<Type>>(ctx, array, mask, replacements,
+                                                       output);
+  }
+};
+
+template <typename Type>
+struct ReplaceWithMask<Type, enable_if_null<Type>> {
+  static Status ExecScalarMask(KernelContext* ctx, const ArrayData& array,
+                               const BooleanScalar& mask, const Datum& replacements,
+                               ArrayData* output) {
+    *output = array;
+    return Status::OK();
+  }
+  static Status ExecArrayMask(KernelContext* ctx, const ArrayData& array,
+                              const ArrayData& mask, const Datum& replacements,
+                              ArrayData* output) {
+    *output = array;
+    return Status::OK();
+  }
+};
+
+template <typename Type>
+struct ReplaceWithMask<Type, enable_if_base_binary<Type>> {
+  using offset_type = typename Type::offset_type;
+  using BuilderType = typename TypeTraits<Type>::BuilderType;
+
+  static Status ExecScalarMask(KernelContext* ctx, const ArrayData& array,
+                               const BooleanScalar& mask, const Datum& replacements,
+                               ArrayData* output) {
+    return ReplaceWithScalarMask(ctx, array, mask, replacements, output);
+  }
+  static Status ExecArrayMask(KernelContext* ctx, const ArrayData& array,
+                              const ArrayData& mask, const Datum& replacements,
+                              ArrayData* output) {
+    BuilderType builder(array.type, ctx->memory_pool());
+    RETURN_NOT_OK(builder.Reserve(array.length));
+    RETURN_NOT_OK(builder.ReserveData(array.buffers[2]->size()));
+    int64_t source_offset = 0;
+    int64_t replacements_offset = 0;
+    RETURN_NOT_OK(VisitArrayDataInline<BooleanType>(
+        mask,
+        [&](bool replace) {
+          if (replace && replacements.is_scalar()) {
+            const Scalar& scalar = *replacements.scalar();
+            if (scalar.is_valid) {
+              RETURN_NOT_OK(builder.Append(UnboxScalar<Type>::Unbox(scalar)));
+            } else {
+              RETURN_NOT_OK(builder.AppendNull());
+            }
+          } else {
+            const ArrayData& source = replace ? *replacements.array() : array;
+            const int64_t offset = replace ? replacements_offset++ : source_offset;
+            if (!source.MayHaveNulls() ||
+                BitUtil::GetBit(source.buffers[0]->data(), source.offset + offset)) {
+              const uint8_t* data = source.buffers[2]->data();
+              const offset_type* offsets = source.GetValues<offset_type>(1);
+              const offset_type offset0 = offsets[offset];
+              const offset_type offset1 = offsets[offset + 1];
+              RETURN_NOT_OK(builder.Append(data + offset0, offset1 - offset0));
+            } else {
+              RETURN_NOT_OK(builder.AppendNull());
+            }
+          }
+          source_offset++;
+          return Status::OK();
+        },
+        [&]() {
+          RETURN_NOT_OK(builder.AppendNull());
+          source_offset++;
+          return Status::OK();
+        }));
+    std::shared_ptr<Array> temp_output;
+    RETURN_NOT_OK(builder.Finish(&temp_output));
+    *output = *temp_output->data();
+    // Builder type != logical type due to GenerateTypeAgnosticVarBinaryBase
+    output->type = array.type;
+    return Status::OK();
+  }
+};
+
+template <typename Type>
+struct ReplaceWithMaskFunctor {
+  static Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    const ArrayData& array = *batch[0].array();
+    const Datum& replacements = batch[2];
+    ArrayData* output = out->array().get();
+    output->length = array.length;
+
+    // Needed for FixedSizeBinary/parameterized types
+    if (!array.type->Equals(*replacements.type(), /*check_metadata=*/false)) {
+      return Status::Invalid("Replacements must be of same type (expected ",
+                             array.type->ToString(), " but got ",
+                             replacements.type()->ToString(), ")");
+    }
+
+    if (!replacements.is_array() && !replacements.is_scalar()) {
+      return Status::Invalid("Replacements must be array or scalar");
+    }
+
+    if (batch[1].is_scalar()) {
+      return ReplaceWithMask<Type>::ExecScalarMask(
+          ctx, array, batch[1].scalar_as<BooleanScalar>(), replacements, output);
+    }
+    const ArrayData& mask = *batch[1].array();
+    if (array.length != mask.length) {
+      return Status::Invalid("Mask must be of same length as array (expected ",
+                             array.length, " items but got ", mask.length, " items)");
+    }
+    return ReplaceWithMask<Type>::ExecArrayMask(ctx, array, mask, replacements, output);
+  }
+};
+
+}  // namespace
+
+const FunctionDoc replace_with_mask_doc(
+    "Replace items using a mask and replacement values",
+    ("Given an array and a Boolean mask (either scalar or of equal length), "
+     "along with replacement values (either scalar or array), "
+     "each element of the array for which the corresponding mask element is "
+     "true will be replaced by the next value from the replacements, "
+     "or with null if the mask is null. "
+     "Hence, for replacement arrays, len(replacements) == sum(mask == true)."),
+    {"values", "mask", "replacements"});
+
+void RegisterVectorReplace(FunctionRegistry* registry) {
+  auto func = std::make_shared<VectorFunction>("replace_with_mask", Arity::Ternary(),
+                                               &replace_with_mask_doc);
+  auto add_kernel = [&](detail::GetTypeId get_id, ArrayKernelExec exec) {
+    VectorKernel kernel;
+    kernel.can_execute_chunkwise = false;
+    if (is_fixed_width(get_id.id)) {
+      kernel.null_handling = NullHandling::type::COMPUTED_PREALLOCATE;
+    } else {
+      kernel.can_write_into_slices = false;
+      kernel.null_handling = NullHandling::type::COMPUTED_NO_PREALLOCATE;
+    }
+    kernel.mem_allocation = MemAllocation::type::PREALLOCATE;

Review comment:
       It is unfortunate that we don't have a versatile utility for vector functions like CheckScalar. One way to verify correct writing into slices would be: run the function once to ensure output is correctly allocated/shaped/etc, then invoke the kernel directly into a slice of that output. If everything is working as it should, the kernel should simply overwrite that slice with new values, leaving values outside the slice untouched.




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

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

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



[GitHub] [arrow] lidavidm commented on a change in pull request #10412: ARROW-9430: [C++] Implement replace_with_mask kernel

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



##########
File path: cpp/src/arrow/compute/kernels/vector_replace.cc
##########
@@ -0,0 +1,510 @@
+// 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_scalar.h"
+#include "arrow/compute/kernels/common.h"
+#include "arrow/util/bitmap_ops.h"
+
+namespace arrow {
+namespace compute {
+namespace internal {
+
+namespace {
+
+Status ReplacementArrayTooShort(int64_t expected, int64_t actual) {
+  return Status::Invalid("Replacement array must be of appropriate length (expected ",
+                         expected, " items but got ", actual, " items)");
+}
+
+// Helper to implement replace_with kernel with scalar mask for fixed-width types,
+// using callbacks to handle both bool and byte-sized types
+Status ReplaceWithScalarMask(KernelContext* ctx, const ArrayData& array,
+                             const BooleanScalar& mask, const Datum& replacements,
+                             ArrayData* output) {
+  if (!mask.is_valid) {
+    // Output = null
+    ARROW_ASSIGN_OR_RAISE(auto replacement_array,
+                          MakeArrayOfNull(array.type, array.length, ctx->memory_pool()));
+    *output = *replacement_array->data();

Review comment:
       You are right, this needs to be a bit smarter.
   
   I also notice that it doesn't handle the case where len(replacements) > len(mask).




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

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

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



[GitHub] [arrow] bkietz commented on pull request #10412: ARROW-9430: [C++] Implement replace_with_mask kernel

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


   FWIW: I'd guess this should be a vector kernel. As a thought experiment: I don't think it'd ever be correct to use this function from a query expression even in its scalar form since that'd require prior knowledge of the number of set bits in the mask


-- 
This is an automated message from the 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 #10412: ARROW-9430: [C++] Implement replace_with_mask kernel

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


   Note there's some code here for handling fixed-width types that now duplicates what's in ARROW-13064/#10557. We should probably unify those at some point (after one or the other merges).


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

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

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



[GitHub] [arrow] lidavidm commented on pull request #10412: ARROW-9430: [C++] Implement replace_with_mask kernel

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


   Yup, good catch, though maybe let's decide if this is to be a vector or scalar kernel (as I also need to add Python bindings)


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

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



[GitHub] [arrow] lidavidm commented on pull request #10412: ARROW-9430: [C++] Implement override_mask kernel

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


   It's in a struct because it's a different length, yes. We could make it a vector kernel instead of a scalar kernel and then the replacements could be passed as another argument. I'm not sure which would be more useful though.
   
   For the name, replace_with_mask etc. sound good. CC @nirandaperera so he's aware for ARROW-9431 as well.


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

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



[GitHub] [arrow] jorisvandenbossche commented on pull request #10412: ARROW-9430: [C++] Implement replace_with_mask kernel

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


   Is this waiting on another reviewer? (I would like to see it included in 5.0 ;))


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

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

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



[GitHub] [arrow] lidavidm commented on a change in pull request #10412: ARROW-9430: [C++] Implement replace_with_mask kernel

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



##########
File path: cpp/src/arrow/compute/kernels/vector_replace.cc
##########
@@ -0,0 +1,495 @@
+// 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_scalar.h"
+#include "arrow/compute/kernels/common.h"
+#include "arrow/util/bitmap_ops.h"
+
+namespace arrow {
+namespace compute {
+namespace internal {
+
+namespace {
+
+Status ReplacementArrayTooShort(int64_t expected, int64_t actual) {
+  return Status::Invalid("Replacement array must be of appropriate length (expected ",
+                         expected, " items but got ", actual, " items)");
+}
+
+// Helper to implement replace_with kernel with scalar mask for fixed-width types,
+// using callbacks to handle both bool and byte-sized types
+Status ReplaceWithScalarMask(KernelContext* ctx, const ArrayData& array,
+                             const BooleanScalar& mask, const Datum& replacements,
+                             ArrayData* output) {
+  if (!mask.is_valid) {
+    // Output = null
+    ARROW_ASSIGN_OR_RAISE(auto array,
+                          MakeArrayOfNull(array.type, array.length, ctx->memory_pool()));
+    *output = *array->data();
+    return Status::OK();
+  }
+  if (mask.value) {
+    // Output = replacement
+    if (replacements.is_scalar()) {
+      ARROW_ASSIGN_OR_RAISE(
+          auto replacement_array,
+          MakeArrayFromScalar(*replacements.scalar(), array.length, ctx->memory_pool()));
+      *output = *replacement_array->data();
+    } else {
+      auto replacement_array = replacements.array();
+      if (replacement_array->length != array.length) {
+        return ReplacementArrayTooShort(array.length, replacement_array->length);
+      }
+      *output = *replacement_array;
+    }
+  } else {
+    // Output = input
+    *output = array;
+  }
+  return Status::OK();
+}
+
+// Helper to implement replace_with kernel with array mask for fixed-width types,
+// using callbacks to handle both bool and byte-sized types and to handle
+// scalar and array replacements
+template <typename Functor>
+Status ReplaceWithArrayMask(KernelContext* ctx, const ArrayData& array,
+                            const ArrayData& mask, const Datum& replacements,
+                            ArrayData* output) {
+  ARROW_ASSIGN_OR_RAISE(output->buffers[1],
+                        Functor::AllocateData(ctx, *array.type, array.length));
+
+  uint8_t* out_bitmap = nullptr;
+  uint8_t* out_values = output->buffers[1]->mutable_data();
+  const uint8_t* mask_bitmap = mask.MayHaveNulls() ? mask.buffers[0]->data() : nullptr;
+  const uint8_t* mask_values = mask.buffers[1]->data();
+  bool replacements_bitmap;
+  int64_t replacements_length;
+  if (replacements.is_array()) {
+    replacements_bitmap = replacements.array()->MayHaveNulls();
+    replacements_length = replacements.array()->length;
+  } else {
+    replacements_bitmap = !replacements.scalar()->is_valid;
+    replacements_length = std::numeric_limits<int64_t>::max();
+  }
+  if (array.MayHaveNulls() || mask.MayHaveNulls() || replacements_bitmap) {
+    ARROW_ASSIGN_OR_RAISE(output->buffers[0], ctx->AllocateBitmap(array.length));
+    out_bitmap = output->buffers[0]->mutable_data();
+    output->null_count = -1;
+    if (array.MayHaveNulls()) {
+      arrow::internal::CopyBitmap(array.buffers[0]->data(), array.offset, array.length,
+                                  out_bitmap, /*dest_offset=*/0);
+    } else {
+      std::memset(out_bitmap, 0xFF, output->buffers[0]->size());

Review comment:
       Done.




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

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

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



[GitHub] [arrow] jorisvandenbossche commented on a change in pull request #10412: ARROW-9430: [C++] Implement replace_with_mask kernel

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



##########
File path: cpp/src/arrow/compute/kernels/vector_replace.cc
##########
@@ -0,0 +1,495 @@
+// 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_scalar.h"
+#include "arrow/compute/kernels/common.h"
+#include "arrow/util/bitmap_ops.h"
+
+namespace arrow {
+namespace compute {
+namespace internal {
+
+namespace {
+
+Status ReplacementArrayTooShort(int64_t expected, int64_t actual) {
+  return Status::Invalid("Replacement array must be of appropriate length (expected ",
+                         expected, " items but got ", actual, " items)");
+}
+
+// Helper to implement replace_with kernel with scalar mask for fixed-width types,
+// using callbacks to handle both bool and byte-sized types
+Status ReplaceWithScalarMask(KernelContext* ctx, const ArrayData& array,
+                             const BooleanScalar& mask, const Datum& replacements,
+                             ArrayData* output) {
+  if (!mask.is_valid) {
+    // Output = null
+    ARROW_ASSIGN_OR_RAISE(auto array,
+                          MakeArrayOfNull(array.type, array.length, ctx->memory_pool()));
+    *output = *array->data();
+    return Status::OK();
+  }
+  if (mask.value) {
+    // Output = replacement
+    if (replacements.is_scalar()) {
+      ARROW_ASSIGN_OR_RAISE(
+          auto replacement_array,
+          MakeArrayFromScalar(*replacements.scalar(), array.length, ctx->memory_pool()));
+      *output = *replacement_array->data();
+    } else {
+      auto replacement_array = replacements.array();
+      if (replacement_array->length != array.length) {
+        return ReplacementArrayTooShort(array.length, replacement_array->length);
+      }
+      *output = *replacement_array;
+    }
+  } else {
+    // Output = input
+    *output = array;
+  }
+  return Status::OK();
+}
+
+// Helper to implement replace_with kernel with array mask for fixed-width types,
+// using callbacks to handle both bool and byte-sized types and to handle
+// scalar and array replacements
+template <typename Functor>
+Status ReplaceWithArrayMask(KernelContext* ctx, const ArrayData& array,
+                            const ArrayData& mask, const Datum& replacements,
+                            ArrayData* output) {
+  ARROW_ASSIGN_OR_RAISE(output->buffers[1],
+                        Functor::AllocateData(ctx, *array.type, array.length));
+
+  uint8_t* out_bitmap = nullptr;
+  uint8_t* out_values = output->buffers[1]->mutable_data();
+  const uint8_t* mask_bitmap = mask.MayHaveNulls() ? mask.buffers[0]->data() : nullptr;
+  const uint8_t* mask_values = mask.buffers[1]->data();
+  bool replacements_bitmap;
+  int64_t replacements_length;
+  if (replacements.is_array()) {
+    replacements_bitmap = replacements.array()->MayHaveNulls();
+    replacements_length = replacements.array()->length;
+  } else {
+    replacements_bitmap = !replacements.scalar()->is_valid;
+    replacements_length = std::numeric_limits<int64_t>::max();
+  }
+  if (array.MayHaveNulls() || mask.MayHaveNulls() || replacements_bitmap) {
+    ARROW_ASSIGN_OR_RAISE(output->buffers[0], ctx->AllocateBitmap(array.length));
+    out_bitmap = output->buffers[0]->mutable_data();
+    output->null_count = -1;
+    if (array.MayHaveNulls()) {
+      arrow::internal::CopyBitmap(array.buffers[0]->data(), array.offset, array.length,
+                                  out_bitmap, /*dest_offset=*/0);
+    } else {
+      std::memset(out_bitmap, 0xFF, output->buffers[0]->size());

Review comment:
       Maybe add a comment to mention this is setting the out_bitmap to all valid?

##########
File path: cpp/src/arrow/compute/kernels/vector_replace.cc
##########
@@ -0,0 +1,495 @@
+// 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_scalar.h"
+#include "arrow/compute/kernels/common.h"
+#include "arrow/util/bitmap_ops.h"
+
+namespace arrow {
+namespace compute {
+namespace internal {
+
+namespace {
+
+Status ReplacementArrayTooShort(int64_t expected, int64_t actual) {
+  return Status::Invalid("Replacement array must be of appropriate length (expected ",
+                         expected, " items but got ", actual, " items)");
+}
+
+// Helper to implement replace_with kernel with scalar mask for fixed-width types,
+// using callbacks to handle both bool and byte-sized types
+Status ReplaceWithScalarMask(KernelContext* ctx, const ArrayData& array,
+                             const BooleanScalar& mask, const Datum& replacements,
+                             ArrayData* output) {
+  if (!mask.is_valid) {
+    // Output = null
+    ARROW_ASSIGN_OR_RAISE(auto array,
+                          MakeArrayOfNull(array.type, array.length, ctx->memory_pool()));
+    *output = *array->data();

Review comment:
       ```suggestion
       ARROW_ASSIGN_OR_RAISE(auto replacement_array,
                             MakeArrayOfNull(array.type, array.length, ctx->memory_pool()));
       *output = *replacement_array->data();
   ```
   
   (to be consistent with below, and not confuse with the input array which uses the same variable name)

##########
File path: cpp/src/arrow/compute/kernels/vector_replace.cc
##########
@@ -0,0 +1,495 @@
+// 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_scalar.h"
+#include "arrow/compute/kernels/common.h"
+#include "arrow/util/bitmap_ops.h"
+
+namespace arrow {
+namespace compute {
+namespace internal {
+
+namespace {
+
+Status ReplacementArrayTooShort(int64_t expected, int64_t actual) {
+  return Status::Invalid("Replacement array must be of appropriate length (expected ",
+                         expected, " items but got ", actual, " items)");
+}
+
+// Helper to implement replace_with kernel with scalar mask for fixed-width types,
+// using callbacks to handle both bool and byte-sized types
+Status ReplaceWithScalarMask(KernelContext* ctx, const ArrayData& array,
+                             const BooleanScalar& mask, const Datum& replacements,
+                             ArrayData* output) {
+  if (!mask.is_valid) {
+    // Output = null
+    ARROW_ASSIGN_OR_RAISE(auto array,
+                          MakeArrayOfNull(array.type, array.length, ctx->memory_pool()));
+    *output = *array->data();
+    return Status::OK();
+  }
+  if (mask.value) {
+    // Output = replacement
+    if (replacements.is_scalar()) {
+      ARROW_ASSIGN_OR_RAISE(
+          auto replacement_array,
+          MakeArrayFromScalar(*replacements.scalar(), array.length, ctx->memory_pool()));
+      *output = *replacement_array->data();
+    } else {
+      auto replacement_array = replacements.array();
+      if (replacement_array->length != array.length) {
+        return ReplacementArrayTooShort(array.length, replacement_array->length);
+      }
+      *output = *replacement_array;
+    }
+  } else {
+    // Output = input
+    *output = array;
+  }
+  return Status::OK();
+}
+
+// Helper to implement replace_with kernel with array mask for fixed-width types,
+// using callbacks to handle both bool and byte-sized types and to handle
+// scalar and array replacements
+template <typename Functor>
+Status ReplaceWithArrayMask(KernelContext* ctx, const ArrayData& array,
+                            const ArrayData& mask, const Datum& replacements,
+                            ArrayData* output) {
+  ARROW_ASSIGN_OR_RAISE(output->buffers[1],
+                        Functor::AllocateData(ctx, *array.type, array.length));
+
+  uint8_t* out_bitmap = nullptr;
+  uint8_t* out_values = output->buffers[1]->mutable_data();
+  const uint8_t* mask_bitmap = mask.MayHaveNulls() ? mask.buffers[0]->data() : nullptr;
+  const uint8_t* mask_values = mask.buffers[1]->data();
+  bool replacements_bitmap;
+  int64_t replacements_length;
+  if (replacements.is_array()) {
+    replacements_bitmap = replacements.array()->MayHaveNulls();
+    replacements_length = replacements.array()->length;
+  } else {
+    replacements_bitmap = !replacements.scalar()->is_valid;
+    replacements_length = std::numeric_limits<int64_t>::max();
+  }
+  if (array.MayHaveNulls() || mask.MayHaveNulls() || replacements_bitmap) {
+    ARROW_ASSIGN_OR_RAISE(output->buffers[0], ctx->AllocateBitmap(array.length));
+    out_bitmap = output->buffers[0]->mutable_data();
+    output->null_count = -1;
+    if (array.MayHaveNulls()) {
+      arrow::internal::CopyBitmap(array.buffers[0]->data(), array.offset, array.length,
+                                  out_bitmap, /*dest_offset=*/0);
+    } else {
+      std::memset(out_bitmap, 0xFF, output->buffers[0]->size());
+    }
+  } else {
+    output->null_count = 0;
+  }
+  auto copy_bitmap = [&](int64_t out_offset, int64_t in_offset, int64_t length) {
+    DCHECK(out_bitmap);
+    if (replacements.is_array()) {
+      const auto& in_data = *replacements.array();
+      const auto in_bitmap = in_data.GetValues<uint8_t>(0, /*absolute_offset=*/0);
+      arrow::internal::CopyBitmap(in_bitmap, in_data.offset + in_offset, length,
+                                  out_bitmap, out_offset);
+    } else {
+      BitUtil::SetBitsTo(out_bitmap, out_offset, length, !replacements_bitmap);
+    }
+  };
+
+  Functor::CopyData(*array.type, out_values, /*out_offset=*/0, array, /*in_offset=*/0,
+                    array.length);
+  arrow::internal::BitBlockCounter value_counter(mask_values, mask.offset, mask.length);
+  arrow::internal::OptionalBitBlockCounter valid_counter(mask_bitmap, mask.offset,
+                                                         mask.length);
+  int64_t out_offset = 0;
+  int64_t replacements_offset = 0;
+  while (out_offset < array.length) {
+    BitBlockCount value_block = value_counter.NextWord();
+    BitBlockCount valid_block = valid_counter.NextWord();
+    DCHECK_EQ(value_block.length, valid_block.length);
+    if (value_block.AllSet() && valid_block.AllSet()) {
+      // Copy from replacement array
+      if (replacements_offset + valid_block.length > replacements_length) {
+        return ReplacementArrayTooShort(replacements_offset + valid_block.length,
+                                        replacements_length);
+      }
+      Functor::CopyData(*array.type, out_values, out_offset, replacements,
+                        replacements_offset, valid_block.length);
+      if (replacements_bitmap) {
+        copy_bitmap(out_offset, replacements_offset, valid_block.length);
+      } else if (!replacements_bitmap && out_bitmap) {
+        BitUtil::SetBitsTo(out_bitmap, out_offset, valid_block.length, true);
+      }
+      replacements_offset += valid_block.length;
+    } else if (value_block.NoneSet() && valid_block.AllSet()) {
+      // Do nothing
+    } else if (valid_block.NoneSet()) {
+      DCHECK(out_bitmap);
+      BitUtil::SetBitsTo(out_bitmap, out_offset, valid_block.length, false);
+    } else {
+      for (int64_t i = 0; i < valid_block.length; ++i) {
+        if (BitUtil::GetBit(mask_values, out_offset + mask.offset + i) &&
+            (!mask_bitmap ||
+             BitUtil::GetBit(mask_bitmap, out_offset + mask.offset + i))) {
+          if (replacements_offset >= replacements_length) {
+            return ReplacementArrayTooShort(replacements_offset + 1, replacements_length);
+          }
+          Functor::CopyData(*array.type, out_values, out_offset + i, replacements,
+                            replacements_offset,
+                            /*length=*/1);
+          if (replacements_bitmap) {
+            copy_bitmap(out_offset + i, replacements_offset, 1);
+          }
+          replacements_offset++;
+        }
+      }
+    }
+    out_offset += valid_block.length;
+  }
+
+  if (mask.MayHaveNulls()) {
+    arrow::internal::BitmapAnd(out_bitmap, /*left_offset=*/0, mask.buffers[0]->data(),
+                               mask.offset, array.length,
+                               /*out_offset=*/0, out_bitmap);

Review comment:
       Hasn't this already been done in the `else if (valid_block.NoneSet()) {` block above?




-- 
This is an automated message from the 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 #10412: ARROW-9430: [C++] Implement replace_with_mask kernel

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



##########
File path: cpp/src/arrow/compute/kernels/vector_replace.cc
##########
@@ -0,0 +1,510 @@
+// 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_scalar.h"
+#include "arrow/compute/kernels/common.h"
+#include "arrow/util/bitmap_ops.h"
+
+namespace arrow {
+namespace compute {
+namespace internal {
+
+namespace {
+
+Status ReplacementArrayTooShort(int64_t expected, int64_t actual) {
+  return Status::Invalid("Replacement array must be of appropriate length (expected ",
+                         expected, " items but got ", actual, " items)");
+}
+
+// Helper to implement replace_with kernel with scalar mask for fixed-width types,
+// using callbacks to handle both bool and byte-sized types
+Status ReplaceWithScalarMask(KernelContext* ctx, const ArrayData& array,
+                             const BooleanScalar& mask, const Datum& replacements,
+                             ArrayData* output) {
+  if (!mask.is_valid) {
+    // Output = null
+    ARROW_ASSIGN_OR_RAISE(auto replacement_array,
+                          MakeArrayOfNull(array.type, array.length, ctx->memory_pool()));
+    *output = *replacement_array->data();
+    return Status::OK();
+  }
+  if (mask.value) {
+    // Output = replacement
+    if (replacements.is_scalar()) {
+      ARROW_ASSIGN_OR_RAISE(
+          auto replacement_array,
+          MakeArrayFromScalar(*replacements.scalar(), array.length, ctx->memory_pool()));
+      *output = *replacement_array->data();
+    } else {
+      auto replacement_array = replacements.array();
+      if (replacement_array->length != array.length) {
+        return ReplacementArrayTooShort(array.length, replacement_array->length);
+      }
+      *output = *replacement_array;
+    }
+  } else {
+    // Output = input
+    *output = array;
+  }
+  return Status::OK();
+}
+
+struct CopyArrayBitmap {
+  const uint8_t* in_bitmap;
+  int64_t in_offset;
+
+  void CopyBitmap(uint8_t* out_bitmap, int64_t out_offset, int64_t offset,
+                  int64_t length) const {
+    arrow::internal::CopyBitmap(in_bitmap, in_offset + offset, length, out_bitmap,
+                                out_offset);
+  }
+
+  void SetBit(uint8_t* out_bitmap, int64_t out_offset, int64_t offset) const {
+    BitUtil::SetBitTo(out_bitmap, out_offset,
+                      BitUtil::GetBit(in_bitmap, in_offset + offset));
+  }
+};
+
+struct CopyScalarBitmap {
+  const bool is_valid;
+
+  void CopyBitmap(uint8_t* out_bitmap, int64_t out_offset, int64_t offset,
+                  int64_t length) const {
+    BitUtil::SetBitsTo(out_bitmap, out_offset, length, is_valid);
+  }
+
+  void SetBit(uint8_t* out_bitmap, int64_t out_offset, int64_t offset) const {
+    BitUtil::SetBitTo(out_bitmap, out_offset, is_valid);
+  }
+};
+
+// Helper to implement replace_with kernel with array mask for fixed-width types,
+// using callbacks to handle both bool and byte-sized types and to handle
+// scalar and array replacements
+template <typename Functor, typename Data, typename CopyBitmap>
+void ReplaceWithArrayMaskImpl(const ArrayData& array, const ArrayData& mask,
+                              const Data& replacements, bool replacements_bitmap,
+                              const CopyBitmap copy_bitmap, const uint8_t* mask_bitmap,

Review comment:
       In this case I intentionally made CopyBitmap itself cheaper to copy than to use as a reference - it's <= 2 words though I suppose the compiler will optimize it identically either way.
   
   Except, doing this does seem about ~5% slower:
   
   ```
   Before:
   -------------------------------------------------------------------------------------------------------
   Benchmark                                             Time             CPU   Iterations UserCounters...
   -------------------------------------------------------------------------------------------------------
   ReplaceWithMaskLowSelectivityBench/16384/0        33631 ns        33631 ns       204971 bytes_per_second=3.62975G/s
   ReplaceWithMaskLowSelectivityBench/16384/99       35018 ns        35017 ns       202363 bytes_per_second=3.46498G/s
   ReplaceWithMaskHighSelectivityBench/16384/0       77268 ns        77267 ns        90912 bytes_per_second=1.57985G/s
   ReplaceWithMaskHighSelectivityBench/16384/99      75751 ns        75750 ns        92444 bytes_per_second=1.60176G/s
   
   After:
   -------------------------------------------------------------------------------------------------------
   Benchmark                                             Time             CPU   Iterations UserCounters...
   -------------------------------------------------------------------------------------------------------
   ReplaceWithMaskLowSelectivityBench/16384/0        35512 ns        35511 ns       192582 bytes_per_second=3.43751G/s
   ReplaceWithMaskLowSelectivityBench/16384/99       36702 ns        36701 ns       191996 bytes_per_second=3.30598G/s
   ReplaceWithMaskHighSelectivityBench/16384/0       82957 ns        82956 ns        85194 bytes_per_second=1.47151G/s
   ReplaceWithMaskHighSelectivityBench/16384/99      80415 ns        80413 ns        86354 bytes_per_second=1.50887G/s
   ```




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

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

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



[GitHub] [arrow] ursabot commented on pull request #10412: ARROW-9430: [C++] Implement replace_with_mask kernel

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


   
   Supported benchmark command examples:
   
   `@ursabot benchmark help`
   
   To run all benchmarks:
   `@ursabot please benchmark`
   
   To filter benchmarks by language:
   `@ursabot please benchmark lang=Python`
   `@ursabot please benchmark lang=C++`
   `@ursabot please benchmark lang=R`
   `@ursabot please benchmark lang=Java`
   
   To filter Python and R benchmarks by name:
   `@ursabot please benchmark name=file-write`
   `@ursabot please benchmark name=file-write lang=Python`
   `@ursabot please benchmark name=file-.*`
   
   To filter C++ benchmarks by archery --suite-filter and --benchmark-filter:
   `@ursabot please benchmark command=cpp-micro --suite-filter=arrow-compute-vector-selection-benchmark --benchmark-filter=TakeStringRandomIndicesWithNulls/262144/2 --iterations=3`
   
   For other `command=cpp-micro` options, please see https://github.com/ursacomputing/benchmarks/blob/main/benchmarks/cpp_micro_benchmarks.py
   


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

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

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



[GitHub] [arrow] lidavidm commented on a change in pull request #10412: ARROW-9430: [C++] Implement replace_with_mask kernel

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



##########
File path: cpp/src/arrow/compute/kernels/vector_replace.cc
##########
@@ -0,0 +1,510 @@
+// 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_scalar.h"
+#include "arrow/compute/kernels/common.h"
+#include "arrow/util/bitmap_ops.h"
+
+namespace arrow {
+namespace compute {
+namespace internal {
+
+namespace {
+
+Status ReplacementArrayTooShort(int64_t expected, int64_t actual) {
+  return Status::Invalid("Replacement array must be of appropriate length (expected ",
+                         expected, " items but got ", actual, " items)");
+}
+
+// Helper to implement replace_with kernel with scalar mask for fixed-width types,
+// using callbacks to handle both bool and byte-sized types
+Status ReplaceWithScalarMask(KernelContext* ctx, const ArrayData& array,
+                             const BooleanScalar& mask, const Datum& replacements,
+                             ArrayData* output) {
+  if (!mask.is_valid) {
+    // Output = null
+    ARROW_ASSIGN_OR_RAISE(auto replacement_array,
+                          MakeArrayOfNull(array.type, array.length, ctx->memory_pool()));
+    *output = *replacement_array->data();

Review comment:
       Fixed.




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

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

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



[GitHub] [arrow] lidavidm commented on pull request #10412: ARROW-9430: [C++] Implement replace_with_mask kernel

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


   I think I've addressed all the feedback here.


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

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

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



[GitHub] [arrow] lidavidm commented on a change in pull request #10412: ARROW-9430: [C++] Implement replace_with_mask kernel

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



##########
File path: cpp/src/arrow/compute/kernels/vector_replace.cc
##########
@@ -0,0 +1,466 @@
+// 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_scalar.h"
+#include "arrow/compute/kernels/common.h"
+#include "arrow/util/bitmap_ops.h"
+
+namespace arrow {
+namespace compute {
+namespace internal {
+
+namespace {
+
+Status ReplacementArrayTooShort(int64_t expected, int64_t actual) {
+  return Status::Invalid("Replacement array must be of appropriate length (expected ",
+                         expected, " items but got ", actual, " items)");
+}
+
+// Helper to implement replace_with kernel with scalar mask for fixed-width types,
+// using callbacks to handle both bool and byte-sized types
+Status ReplaceWithScalarMask(KernelContext* ctx, const ArrayData& array,
+                             const BooleanScalar& mask, const Datum& replacements,
+                             ArrayData* output) {
+  if (!mask.is_valid) {
+    // Output = null
+    ARROW_ASSIGN_OR_RAISE(auto replacement_array,
+                          MakeArrayOfNull(array.type, array.length, ctx->memory_pool()));
+    *output = *replacement_array->data();
+    return Status::OK();
+  }
+  if (mask.value) {
+    // Output = replacement
+    if (replacements.is_scalar()) {
+      ARROW_ASSIGN_OR_RAISE(
+          auto replacement_array,
+          MakeArrayFromScalar(*replacements.scalar(), array.length, ctx->memory_pool()));
+      *output = *replacement_array->data();
+    } else {
+      auto replacement_array = replacements.array();
+      if (replacement_array->length != array.length) {
+        return ReplacementArrayTooShort(array.length, replacement_array->length);
+      }
+      *output = *replacement_array;
+    }
+  } else {
+    // Output = input
+    *output = array;
+  }
+  return Status::OK();
+}
+
+// Helper to implement replace_with kernel with array mask for fixed-width types,
+// using callbacks to handle both bool and byte-sized types and to handle
+// scalar and array replacements
+template <typename Functor>
+Status ReplaceWithArrayMask(KernelContext* ctx, const ArrayData& array,
+                            const ArrayData& mask, const Datum& replacements,
+                            ArrayData* output) {
+  uint8_t* out_bitmap = nullptr;
+  uint8_t* out_values = output->buffers[1]->mutable_data();
+  const uint8_t* mask_bitmap = mask.MayHaveNulls() ? mask.buffers[0]->data() : nullptr;
+  const uint8_t* mask_values = mask.buffers[1]->data();
+  bool replacements_bitmap;
+  int64_t replacements_length;
+  if (replacements.is_array()) {
+    replacements_bitmap = replacements.array()->MayHaveNulls();
+    replacements_length = replacements.array()->length;
+  } else {
+    replacements_bitmap = !replacements.scalar()->is_valid;
+    replacements_length = std::numeric_limits<int64_t>::max();
+  }
+  if (array.MayHaveNulls() || mask.MayHaveNulls() || replacements_bitmap) {
+    ARROW_ASSIGN_OR_RAISE(output->buffers[0], ctx->AllocateBitmap(array.length));
+    out_bitmap = output->buffers[0]->mutable_data();
+    output->null_count = -1;
+    if (array.MayHaveNulls()) {
+      // Copy array's bitmap
+      arrow::internal::CopyBitmap(array.buffers[0]->data(), array.offset, array.length,
+                                  out_bitmap, /*dest_offset=*/0);
+    } else {
+      // Array has no bitmap but mask/replacements do, generate an all-valid bitmap
+      std::memset(out_bitmap, 0xFF, output->buffers[0]->size());
+    }
+  } else {
+    output->null_count = 0;
+  }
+  auto copy_bitmap = [&](int64_t out_offset, int64_t in_offset, int64_t length) {
+    DCHECK(out_bitmap);
+    if (replacements.is_array()) {
+      const auto& in_data = *replacements.array();
+      const auto in_bitmap = in_data.GetValues<uint8_t>(0, /*absolute_offset=*/0);
+      arrow::internal::CopyBitmap(in_bitmap, in_data.offset + in_offset, length,
+                                  out_bitmap, out_offset);
+    } else {
+      BitUtil::SetBitsTo(out_bitmap, out_offset, length, !replacements_bitmap);
+    }
+  };
+
+  Functor::CopyData(*array.type, out_values, /*out_offset=*/0, array, /*in_offset=*/0,
+                    array.length);
+  arrow::internal::BitBlockCounter value_counter(mask_values, mask.offset, mask.length);
+  arrow::internal::OptionalBitBlockCounter valid_counter(mask_bitmap, mask.offset,
+                                                         mask.length);
+  int64_t out_offset = 0;
+  int64_t replacements_offset = 0;
+  while (out_offset < array.length) {
+    BitBlockCount value_block = value_counter.NextWord();
+    BitBlockCount valid_block = valid_counter.NextWord();
+    DCHECK_EQ(value_block.length, valid_block.length);
+    if (value_block.AllSet() && valid_block.AllSet()) {
+      // Copy from replacement array
+      if (replacements_offset + valid_block.length > replacements_length) {
+        return ReplacementArrayTooShort(replacements_offset + valid_block.length,
+                                        replacements_length);
+      }
+      Functor::CopyData(*array.type, out_values, out_offset, replacements,
+                        replacements_offset, valid_block.length);
+      if (replacements_bitmap) {
+        copy_bitmap(out_offset, replacements_offset, valid_block.length);
+      } else if (!replacements_bitmap && out_bitmap) {
+        BitUtil::SetBitsTo(out_bitmap, out_offset, valid_block.length, true);
+      }
+      replacements_offset += valid_block.length;
+    } else if ((value_block.NoneSet() && valid_block.AllSet()) || valid_block.NoneSet()) {
+      // Do nothing
+    } else {
+      for (int64_t i = 0; i < valid_block.length; ++i) {
+        if (BitUtil::GetBit(mask_values, out_offset + mask.offset + i) &&
+            (!mask_bitmap ||
+             BitUtil::GetBit(mask_bitmap, out_offset + mask.offset + i))) {
+          if (replacements_offset >= replacements_length) {

Review comment:
       We'd have to use OptionalBinaryBitBlockCounter to take into account mask and offset - so we're doing the work anyways. I'll put up a benchmark and see which might be faster. We could also ARROW_PREDICT_FALSE here and see if that helps.




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

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

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



[GitHub] [arrow] lidavidm commented on pull request #10412: ARROW-9430: [C++] Implement replace_with_mask kernel

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


   Merged, thanks. This should unblock ARROW-9431 if you do still plan to 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.

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

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



[GitHub] [arrow] nirandaperera commented on a change in pull request #10412: ARROW-9430: [C++] Implement replace_with_mask kernel

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



##########
File path: cpp/src/arrow/compute/kernels/vector_replace.cc
##########
@@ -0,0 +1,466 @@
+// 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_scalar.h"
+#include "arrow/compute/kernels/common.h"
+#include "arrow/util/bitmap_ops.h"
+
+namespace arrow {
+namespace compute {
+namespace internal {
+
+namespace {
+
+Status ReplacementArrayTooShort(int64_t expected, int64_t actual) {
+  return Status::Invalid("Replacement array must be of appropriate length (expected ",
+                         expected, " items but got ", actual, " items)");
+}
+
+// Helper to implement replace_with kernel with scalar mask for fixed-width types,
+// using callbacks to handle both bool and byte-sized types
+Status ReplaceWithScalarMask(KernelContext* ctx, const ArrayData& array,
+                             const BooleanScalar& mask, const Datum& replacements,
+                             ArrayData* output) {
+  if (!mask.is_valid) {
+    // Output = null
+    ARROW_ASSIGN_OR_RAISE(auto replacement_array,
+                          MakeArrayOfNull(array.type, array.length, ctx->memory_pool()));
+    *output = *replacement_array->data();
+    return Status::OK();
+  }
+  if (mask.value) {
+    // Output = replacement
+    if (replacements.is_scalar()) {
+      ARROW_ASSIGN_OR_RAISE(
+          auto replacement_array,
+          MakeArrayFromScalar(*replacements.scalar(), array.length, ctx->memory_pool()));
+      *output = *replacement_array->data();
+    } else {
+      auto replacement_array = replacements.array();
+      if (replacement_array->length != array.length) {
+        return ReplacementArrayTooShort(array.length, replacement_array->length);
+      }
+      *output = *replacement_array;
+    }
+  } else {
+    // Output = input
+    *output = array;
+  }
+  return Status::OK();
+}
+
+// Helper to implement replace_with kernel with array mask for fixed-width types,
+// using callbacks to handle both bool and byte-sized types and to handle
+// scalar and array replacements
+template <typename Functor>
+Status ReplaceWithArrayMask(KernelContext* ctx, const ArrayData& array,
+                            const ArrayData& mask, const Datum& replacements,
+                            ArrayData* output) {
+  uint8_t* out_bitmap = nullptr;
+  uint8_t* out_values = output->buffers[1]->mutable_data();
+  const uint8_t* mask_bitmap = mask.MayHaveNulls() ? mask.buffers[0]->data() : nullptr;
+  const uint8_t* mask_values = mask.buffers[1]->data();
+  bool replacements_bitmap;
+  int64_t replacements_length;
+  if (replacements.is_array()) {
+    replacements_bitmap = replacements.array()->MayHaveNulls();
+    replacements_length = replacements.array()->length;
+  } else {
+    replacements_bitmap = !replacements.scalar()->is_valid;
+    replacements_length = std::numeric_limits<int64_t>::max();
+  }
+  if (array.MayHaveNulls() || mask.MayHaveNulls() || replacements_bitmap) {
+    ARROW_ASSIGN_OR_RAISE(output->buffers[0], ctx->AllocateBitmap(array.length));
+    out_bitmap = output->buffers[0]->mutable_data();
+    output->null_count = -1;
+    if (array.MayHaveNulls()) {
+      // Copy array's bitmap
+      arrow::internal::CopyBitmap(array.buffers[0]->data(), array.offset, array.length,
+                                  out_bitmap, /*dest_offset=*/0);
+    } else {
+      // Array has no bitmap but mask/replacements do, generate an all-valid bitmap
+      std::memset(out_bitmap, 0xFF, output->buffers[0]->size());
+    }
+  } else {
+    output->null_count = 0;
+  }
+  auto copy_bitmap = [&](int64_t out_offset, int64_t in_offset, int64_t length) {
+    DCHECK(out_bitmap);
+    if (replacements.is_array()) {
+      const auto& in_data = *replacements.array();
+      const auto in_bitmap = in_data.GetValues<uint8_t>(0, /*absolute_offset=*/0);
+      arrow::internal::CopyBitmap(in_bitmap, in_data.offset + in_offset, length,
+                                  out_bitmap, out_offset);
+    } else {
+      BitUtil::SetBitsTo(out_bitmap, out_offset, length, !replacements_bitmap);
+    }
+  };
+
+  Functor::CopyData(*array.type, out_values, /*out_offset=*/0, array, /*in_offset=*/0,
+                    array.length);
+  arrow::internal::BitBlockCounter value_counter(mask_values, mask.offset, mask.length);
+  arrow::internal::OptionalBitBlockCounter valid_counter(mask_bitmap, mask.offset,
+                                                         mask.length);
+  int64_t out_offset = 0;
+  int64_t replacements_offset = 0;
+  while (out_offset < array.length) {
+    BitBlockCount value_block = value_counter.NextWord();
+    BitBlockCount valid_block = valid_counter.NextWord();
+    DCHECK_EQ(value_block.length, valid_block.length);
+    if (value_block.AllSet() && valid_block.AllSet()) {
+      // Copy from replacement array
+      if (replacements_offset + valid_block.length > replacements_length) {
+        return ReplacementArrayTooShort(replacements_offset + valid_block.length,
+                                        replacements_length);
+      }
+      Functor::CopyData(*array.type, out_values, out_offset, replacements,
+                        replacements_offset, valid_block.length);
+      if (replacements_bitmap) {
+        copy_bitmap(out_offset, replacements_offset, valid_block.length);
+      } else if (!replacements_bitmap && out_bitmap) {
+        BitUtil::SetBitsTo(out_bitmap, out_offset, valid_block.length, true);
+      }
+      replacements_offset += valid_block.length;
+    } else if ((value_block.NoneSet() && valid_block.AllSet()) || valid_block.NoneSet()) {
+      // Do nothing
+    } else {
+      for (int64_t i = 0; i < valid_block.length; ++i) {
+        if (BitUtil::GetBit(mask_values, out_offset + mask.offset + i) &&
+            (!mask_bitmap ||
+             BitUtil::GetBit(mask_bitmap, out_offset + mask.offset + i))) {
+          if (replacements_offset >= replacements_length) {
+            return ReplacementArrayTooShort(replacements_offset + 1, replacements_length);
+          }
+          Functor::CopyData(*array.type, out_values, out_offset + i, replacements,
+                            replacements_offset,
+                            /*length=*/1);
+          if (replacements_bitmap) {
+            copy_bitmap(out_offset + i, replacements_offset, 1);
+          }
+          replacements_offset++;
+        }
+      }
+    }
+    out_offset += valid_block.length;
+  }
+
+  if (mask.MayHaveNulls()) {
+    arrow::internal::BitmapAnd(out_bitmap, /*left_offset=*/0, mask.buffers[0]->data(),
+                               mask.offset, array.length,
+                               /*out_offset=*/0, out_bitmap);
+  }
+  return Status::OK();
+}
+
+template <typename Type, typename Enable = void>
+struct ReplaceWithMask {};
+
+template <typename Type>
+struct ReplaceWithMask<Type, enable_if_number<Type>> {
+  using T = typename TypeTraits<Type>::CType;
+
+  static void CopyData(const DataType&, uint8_t* out, const int64_t out_offset,
+                       const Datum& in, const int64_t in_offset, const int64_t length) {
+    if (in.is_array()) {
+      const auto& in_data = *in.array();
+      const auto in_arr =
+          in_data.GetValues<uint8_t>(1, (in_offset + in_data.offset) * sizeof(T));
+      std::memcpy(out + (out_offset * sizeof(T)), in_arr, length * sizeof(T));
+    } else {
+      T* begin = reinterpret_cast<T*>(out + (out_offset * sizeof(T)));
+      T* end = begin + length;
+      std::fill(begin, end, UnboxScalar<Type>::Unbox(*in.scalar()));
+    }
+  }
+
+  static Status ExecScalarMask(KernelContext* ctx, const ArrayData& array,
+                               const BooleanScalar& mask, const Datum& replacements,
+                               ArrayData* output) {
+    return ReplaceWithScalarMask(ctx, array, mask, replacements, output);
+  }
+
+  static Status ExecArrayMask(KernelContext* ctx, const ArrayData& array,
+                              const ArrayData& mask, const Datum& replacements,
+                              ArrayData* output) {
+    return ReplaceWithArrayMask<ReplaceWithMask<Type>>(ctx, array, mask, replacements,
+                                                       output);
+  }
+};
+
+template <typename Type>
+struct ReplaceWithMask<Type, enable_if_boolean<Type>> {
+  static void CopyData(const DataType&, uint8_t* out, const int64_t out_offset,
+                       const Datum& in, const int64_t in_offset, const int64_t length) {
+    if (in.is_array()) {
+      const auto& in_data = *in.array();
+      const auto in_arr = in_data.GetValues<uint8_t>(1, /*absolute_offset=*/0);
+      arrow::internal::CopyBitmap(in_arr, in_offset + in_data.offset, length, out,
+                                  out_offset);
+    } else {
+      BitUtil::SetBitsTo(out, out_offset, length, in.scalar()->is_valid);
+    }
+  }
+
+  static Status ExecScalarMask(KernelContext* ctx, const ArrayData& array,
+                               const BooleanScalar& mask, const Datum& replacements,
+                               ArrayData* output) {
+    return ReplaceWithScalarMask(ctx, array, mask, replacements, output);
+  }
+  static Status ExecArrayMask(KernelContext* ctx, const ArrayData& array,
+                              const ArrayData& mask, const Datum& replacements,
+                              ArrayData* output) {
+    return ReplaceWithArrayMask<ReplaceWithMask<Type>>(ctx, array, mask, replacements,
+                                                       output);
+  }
+};
+
+template <typename Type>
+struct ReplaceWithMask<Type, enable_if_same<Type, FixedSizeBinaryType>> {
+  static void CopyData(const DataType& ty, uint8_t* out, const int64_t out_offset,
+                       const Datum& in, const int64_t in_offset, const int64_t length) {
+    const int32_t width = checked_cast<const FixedSizeBinaryType&>(ty).byte_width();
+    uint8_t* begin = out + (out_offset * width);
+    if (in.is_array()) {
+      const auto& in_data = *in.array();
+      const auto in_arr =
+          in_data.GetValues<uint8_t>(1, (in_offset + in_data.offset) * width);
+      std::memcpy(begin, in_arr, length * width);
+    } else {
+      const FixedSizeBinaryScalar& scalar =
+          checked_cast<const FixedSizeBinaryScalar&>(*in.scalar());
+      // Null scalar may have null value buffer
+      if (!scalar.value) return;
+      const Buffer& buffer = *scalar.value;
+      const uint8_t* value = buffer.data();
+      DCHECK_GE(buffer.size(), width);
+      for (int i = 0; i < length; i++) {
+        std::memcpy(begin, value, width);
+        begin += width;
+      }
+    }
+  }
+
+  static Status ExecScalarMask(KernelContext* ctx, const ArrayData& array,
+                               const BooleanScalar& mask, const Datum& replacements,
+                               ArrayData* output) {
+    return ReplaceWithScalarMask(ctx, array, mask, replacements, output);
+  }
+
+  static Status ExecArrayMask(KernelContext* ctx, const ArrayData& array,
+                              const ArrayData& mask, const Datum& replacements,
+                              ArrayData* output) {
+    return ReplaceWithArrayMask<ReplaceWithMask<Type>>(ctx, array, mask, replacements,
+                                                       output);
+  }
+};
+
+template <typename Type>
+struct ReplaceWithMask<Type, enable_if_decimal<Type>> {
+  using ScalarType = typename TypeTraits<Type>::ScalarType;
+  static void CopyData(const DataType& ty, uint8_t* out, const int64_t out_offset,
+                       const Datum& in, const int64_t in_offset, const int64_t length) {
+    const int32_t width = checked_cast<const FixedSizeBinaryType&>(ty).byte_width();
+    uint8_t* begin = out + (out_offset * width);
+    if (in.is_array()) {
+      const auto& in_data = *in.array();
+      const auto in_arr =
+          in_data.GetValues<uint8_t>(1, (in_offset + in_data.offset) * width);
+      std::memcpy(begin, in_arr, length * width);
+    } else {
+      const ScalarType& scalar = checked_cast<const ScalarType&>(*in.scalar());
+      const auto value = scalar.value.ToBytes();
+      for (int i = 0; i < length; i++) {
+        std::memcpy(begin, value.data(), width);
+        begin += width;
+      }
+    }
+  }
+
+  static Status ExecScalarMask(KernelContext* ctx, const ArrayData& array,
+                               const BooleanScalar& mask, const Datum& replacements,
+                               ArrayData* output) {
+    return ReplaceWithScalarMask(ctx, array, mask, replacements, output);
+  }
+
+  static Status ExecArrayMask(KernelContext* ctx, const ArrayData& array,
+                              const ArrayData& mask, const Datum& replacements,
+                              ArrayData* output) {
+    return ReplaceWithArrayMask<ReplaceWithMask<Type>>(ctx, array, mask, replacements,
+                                                       output);
+  }
+};
+
+template <typename Type>
+struct ReplaceWithMask<Type, enable_if_null<Type>> {
+  static Status ExecScalarMask(KernelContext* ctx, const ArrayData& array,
+                               const BooleanScalar& mask, const Datum& replacements,
+                               ArrayData* output) {
+    *output = array;
+    return Status::OK();
+  }
+  static Status ExecArrayMask(KernelContext* ctx, const ArrayData& array,
+                              const ArrayData& mask, const Datum& replacements,
+                              ArrayData* output) {
+    *output = array;
+    return Status::OK();
+  }
+};
+
+template <typename Type>
+struct ReplaceWithMask<Type, enable_if_base_binary<Type>> {
+  using offset_type = typename Type::offset_type;
+  using BuilderType = typename TypeTraits<Type>::BuilderType;
+
+  static Status ExecScalarMask(KernelContext* ctx, const ArrayData& array,
+                               const BooleanScalar& mask, const Datum& replacements,
+                               ArrayData* output) {
+    return ReplaceWithScalarMask(ctx, array, mask, replacements, output);
+  }
+  static Status ExecArrayMask(KernelContext* ctx, const ArrayData& array,
+                              const ArrayData& mask, const Datum& replacements,
+                              ArrayData* output) {
+    BuilderType builder(array.type, ctx->memory_pool());
+    RETURN_NOT_OK(builder.Reserve(array.length));

Review comment:
       > Why +1? The builder will reserve the correct number of offsets.
   
   my bad! :sweat_smile: 




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

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

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



[GitHub] [arrow] jorisvandenbossche commented on pull request #10412: ARROW-9430: [C++] Implement replace_with_mask kernel

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


   @bkietz I think it's correct that this will probably not be used in a typical query execution context. Its main target is to allow to mimic "setitem" operations in eg pandas (`arr[mask] = val`). 


-- 
This is an automated message from the 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 #10412: ARROW-9430: [C++] Implement replace_with_mask kernel

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



##########
File path: cpp/src/arrow/compute/kernels/vector_replace.cc
##########
@@ -0,0 +1,510 @@
+// 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_scalar.h"
+#include "arrow/compute/kernels/common.h"
+#include "arrow/util/bitmap_ops.h"
+
+namespace arrow {
+namespace compute {
+namespace internal {
+
+namespace {
+
+Status ReplacementArrayTooShort(int64_t expected, int64_t actual) {
+  return Status::Invalid("Replacement array must be of appropriate length (expected ",
+                         expected, " items but got ", actual, " items)");
+}
+
+// Helper to implement replace_with kernel with scalar mask for fixed-width types,
+// using callbacks to handle both bool and byte-sized types
+Status ReplaceWithScalarMask(KernelContext* ctx, const ArrayData& array,
+                             const BooleanScalar& mask, const Datum& replacements,
+                             ArrayData* output) {
+  if (!mask.is_valid) {
+    // Output = null
+    ARROW_ASSIGN_OR_RAISE(auto replacement_array,
+                          MakeArrayOfNull(array.type, array.length, ctx->memory_pool()));
+    *output = *replacement_array->data();
+    return Status::OK();
+  }
+  if (mask.value) {
+    // Output = replacement
+    if (replacements.is_scalar()) {
+      ARROW_ASSIGN_OR_RAISE(
+          auto replacement_array,
+          MakeArrayFromScalar(*replacements.scalar(), array.length, ctx->memory_pool()));
+      *output = *replacement_array->data();
+    } else {
+      auto replacement_array = replacements.array();
+      if (replacement_array->length != array.length) {
+        return ReplacementArrayTooShort(array.length, replacement_array->length);
+      }
+      *output = *replacement_array;
+    }
+  } else {
+    // Output = input
+    *output = array;
+  }
+  return Status::OK();
+}
+
+struct CopyArrayBitmap {
+  const uint8_t* in_bitmap;
+  int64_t in_offset;
+
+  void CopyBitmap(uint8_t* out_bitmap, int64_t out_offset, int64_t offset,
+                  int64_t length) const {
+    arrow::internal::CopyBitmap(in_bitmap, in_offset + offset, length, out_bitmap,
+                                out_offset);
+  }
+
+  void SetBit(uint8_t* out_bitmap, int64_t out_offset, int64_t offset) const {
+    BitUtil::SetBitTo(out_bitmap, out_offset,
+                      BitUtil::GetBit(in_bitmap, in_offset + offset));
+  }
+};
+
+struct CopyScalarBitmap {
+  const bool is_valid;
+
+  void CopyBitmap(uint8_t* out_bitmap, int64_t out_offset, int64_t offset,
+                  int64_t length) const {
+    BitUtil::SetBitsTo(out_bitmap, out_offset, length, is_valid);
+  }
+
+  void SetBit(uint8_t* out_bitmap, int64_t out_offset, int64_t offset) const {
+    BitUtil::SetBitTo(out_bitmap, out_offset, is_valid);
+  }
+};
+
+// Helper to implement replace_with kernel with array mask for fixed-width types,
+// using callbacks to handle both bool and byte-sized types and to handle
+// scalar and array replacements
+template <typename Functor, typename Data, typename CopyBitmap>
+void ReplaceWithArrayMaskImpl(const ArrayData& array, const ArrayData& mask,
+                              const Data& replacements, bool replacements_bitmap,
+                              const CopyBitmap copy_bitmap, const uint8_t* mask_bitmap,

Review comment:
       interesting :smile: 




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

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

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



[GitHub] [arrow] lidavidm removed a comment on pull request #10412: ARROW-9430: [C++] Implement replace_with_mask kernel

Posted by GitBox <gi...@apache.org>.
lidavidm removed a comment on pull request #10412:
URL: https://github.com/apache/arrow/pull/10412#issuecomment-876500592


   @ursabot please benchmark language=C++


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

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

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



[GitHub] [arrow] bkietz commented on pull request #10412: ARROW-9430: [C++] Implement replace_with_mask kernel

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


   FWIW: I'd guess this should be a vector kernel. As a thought experiment: I don't think it'd ever be correct to use this function from a query expression even in its scalar form since that'd require prior knowledge of the number of set bits in the mask


-- 
This is an automated message from the 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 #10412: ARROW-9430: [C++] Implement replace_with_mask kernel

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


   


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

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

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



[GitHub] [arrow] lidavidm commented on a change in pull request #10412: ARROW-9430: [C++] Implement replace_with_mask kernel

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



##########
File path: cpp/src/arrow/compute/kernels/vector_replace.cc
##########
@@ -0,0 +1,466 @@
+// 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_scalar.h"
+#include "arrow/compute/kernels/common.h"
+#include "arrow/util/bitmap_ops.h"
+
+namespace arrow {
+namespace compute {
+namespace internal {
+
+namespace {
+
+Status ReplacementArrayTooShort(int64_t expected, int64_t actual) {
+  return Status::Invalid("Replacement array must be of appropriate length (expected ",
+                         expected, " items but got ", actual, " items)");
+}
+
+// Helper to implement replace_with kernel with scalar mask for fixed-width types,
+// using callbacks to handle both bool and byte-sized types
+Status ReplaceWithScalarMask(KernelContext* ctx, const ArrayData& array,
+                             const BooleanScalar& mask, const Datum& replacements,
+                             ArrayData* output) {
+  if (!mask.is_valid) {
+    // Output = null
+    ARROW_ASSIGN_OR_RAISE(auto replacement_array,
+                          MakeArrayOfNull(array.type, array.length, ctx->memory_pool()));
+    *output = *replacement_array->data();
+    return Status::OK();
+  }
+  if (mask.value) {
+    // Output = replacement
+    if (replacements.is_scalar()) {
+      ARROW_ASSIGN_OR_RAISE(
+          auto replacement_array,
+          MakeArrayFromScalar(*replacements.scalar(), array.length, ctx->memory_pool()));
+      *output = *replacement_array->data();
+    } else {
+      auto replacement_array = replacements.array();
+      if (replacement_array->length != array.length) {
+        return ReplacementArrayTooShort(array.length, replacement_array->length);
+      }
+      *output = *replacement_array;
+    }
+  } else {
+    // Output = input
+    *output = array;
+  }
+  return Status::OK();
+}
+
+// Helper to implement replace_with kernel with array mask for fixed-width types,
+// using callbacks to handle both bool and byte-sized types and to handle
+// scalar and array replacements
+template <typename Functor>
+Status ReplaceWithArrayMask(KernelContext* ctx, const ArrayData& array,
+                            const ArrayData& mask, const Datum& replacements,
+                            ArrayData* output) {
+  uint8_t* out_bitmap = nullptr;
+  uint8_t* out_values = output->buffers[1]->mutable_data();
+  const uint8_t* mask_bitmap = mask.MayHaveNulls() ? mask.buffers[0]->data() : nullptr;
+  const uint8_t* mask_values = mask.buffers[1]->data();
+  bool replacements_bitmap;
+  int64_t replacements_length;
+  if (replacements.is_array()) {
+    replacements_bitmap = replacements.array()->MayHaveNulls();
+    replacements_length = replacements.array()->length;
+  } else {
+    replacements_bitmap = !replacements.scalar()->is_valid;
+    replacements_length = std::numeric_limits<int64_t>::max();
+  }
+  if (array.MayHaveNulls() || mask.MayHaveNulls() || replacements_bitmap) {
+    ARROW_ASSIGN_OR_RAISE(output->buffers[0], ctx->AllocateBitmap(array.length));
+    out_bitmap = output->buffers[0]->mutable_data();
+    output->null_count = -1;
+    if (array.MayHaveNulls()) {
+      // Copy array's bitmap
+      arrow::internal::CopyBitmap(array.buffers[0]->data(), array.offset, array.length,
+                                  out_bitmap, /*dest_offset=*/0);
+    } else {
+      // Array has no bitmap but mask/replacements do, generate an all-valid bitmap
+      std::memset(out_bitmap, 0xFF, output->buffers[0]->size());
+    }
+  } else {
+    output->null_count = 0;
+  }
+  auto copy_bitmap = [&](int64_t out_offset, int64_t in_offset, int64_t length) {
+    DCHECK(out_bitmap);
+    if (replacements.is_array()) {
+      const auto& in_data = *replacements.array();
+      const auto in_bitmap = in_data.GetValues<uint8_t>(0, /*absolute_offset=*/0);
+      arrow::internal::CopyBitmap(in_bitmap, in_data.offset + in_offset, length,
+                                  out_bitmap, out_offset);
+    } else {
+      BitUtil::SetBitsTo(out_bitmap, out_offset, length, !replacements_bitmap);
+    }
+  };
+
+  Functor::CopyData(*array.type, out_values, /*out_offset=*/0, array, /*in_offset=*/0,
+                    array.length);
+  arrow::internal::BitBlockCounter value_counter(mask_values, mask.offset, mask.length);
+  arrow::internal::OptionalBitBlockCounter valid_counter(mask_bitmap, mask.offset,
+                                                         mask.length);
+  int64_t out_offset = 0;
+  int64_t replacements_offset = 0;
+  while (out_offset < array.length) {
+    BitBlockCount value_block = value_counter.NextWord();
+    BitBlockCount valid_block = valid_counter.NextWord();
+    DCHECK_EQ(value_block.length, valid_block.length);
+    if (value_block.AllSet() && valid_block.AllSet()) {
+      // Copy from replacement array
+      if (replacements_offset + valid_block.length > replacements_length) {
+        return ReplacementArrayTooShort(replacements_offset + valid_block.length,
+                                        replacements_length);
+      }
+      Functor::CopyData(*array.type, out_values, out_offset, replacements,
+                        replacements_offset, valid_block.length);
+      if (replacements_bitmap) {
+        copy_bitmap(out_offset, replacements_offset, valid_block.length);
+      } else if (!replacements_bitmap && out_bitmap) {
+        BitUtil::SetBitsTo(out_bitmap, out_offset, valid_block.length, true);
+      }
+      replacements_offset += valid_block.length;
+    } else if ((value_block.NoneSet() && valid_block.AllSet()) || valid_block.NoneSet()) {

Review comment:
       Ah, thanks! Right, if the mask itself isn't set, then it doesn't matter if they're valid or not.




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

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

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



[GitHub] [arrow] lidavidm commented on pull request #10412: ARROW-9430: [C++] Implement replace_with_mask kernel

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


   @bkietz any other comments?


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

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

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



[GitHub] [arrow] lidavidm commented on pull request #10412: ARROW-9430: [C++] Implement replace_with_mask kernel

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


   This is now a vector kernel, with docs, that should support any fixed-size type. However I still need to add support for binary types which I expect people would want to use.


-- 
This is an automated message from the 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 #10412: ARROW-9430: [C++] Implement override_mask kernel

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


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


-- 
This is an automated message from the 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 #10412: ARROW-9430: [C++] Implement replace_with_mask kernel

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



##########
File path: cpp/src/arrow/compute/kernels/vector_replace.cc
##########
@@ -0,0 +1,466 @@
+// 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_scalar.h"
+#include "arrow/compute/kernels/common.h"
+#include "arrow/util/bitmap_ops.h"
+
+namespace arrow {
+namespace compute {
+namespace internal {
+
+namespace {
+
+Status ReplacementArrayTooShort(int64_t expected, int64_t actual) {
+  return Status::Invalid("Replacement array must be of appropriate length (expected ",
+                         expected, " items but got ", actual, " items)");
+}
+
+// Helper to implement replace_with kernel with scalar mask for fixed-width types,
+// using callbacks to handle both bool and byte-sized types
+Status ReplaceWithScalarMask(KernelContext* ctx, const ArrayData& array,
+                             const BooleanScalar& mask, const Datum& replacements,
+                             ArrayData* output) {
+  if (!mask.is_valid) {
+    // Output = null
+    ARROW_ASSIGN_OR_RAISE(auto replacement_array,
+                          MakeArrayOfNull(array.type, array.length, ctx->memory_pool()));
+    *output = *replacement_array->data();
+    return Status::OK();
+  }
+  if (mask.value) {
+    // Output = replacement
+    if (replacements.is_scalar()) {
+      ARROW_ASSIGN_OR_RAISE(
+          auto replacement_array,
+          MakeArrayFromScalar(*replacements.scalar(), array.length, ctx->memory_pool()));
+      *output = *replacement_array->data();
+    } else {
+      auto replacement_array = replacements.array();
+      if (replacement_array->length != array.length) {
+        return ReplacementArrayTooShort(array.length, replacement_array->length);
+      }
+      *output = *replacement_array;
+    }
+  } else {
+    // Output = input
+    *output = array;
+  }
+  return Status::OK();
+}
+
+// Helper to implement replace_with kernel with array mask for fixed-width types,
+// using callbacks to handle both bool and byte-sized types and to handle
+// scalar and array replacements
+template <typename Functor>
+Status ReplaceWithArrayMask(KernelContext* ctx, const ArrayData& array,
+                            const ArrayData& mask, const Datum& replacements,
+                            ArrayData* output) {
+  uint8_t* out_bitmap = nullptr;
+  uint8_t* out_values = output->buffers[1]->mutable_data();
+  const uint8_t* mask_bitmap = mask.MayHaveNulls() ? mask.buffers[0]->data() : nullptr;
+  const uint8_t* mask_values = mask.buffers[1]->data();
+  bool replacements_bitmap;
+  int64_t replacements_length;
+  if (replacements.is_array()) {
+    replacements_bitmap = replacements.array()->MayHaveNulls();
+    replacements_length = replacements.array()->length;
+  } else {
+    replacements_bitmap = !replacements.scalar()->is_valid;
+    replacements_length = std::numeric_limits<int64_t>::max();
+  }
+  if (array.MayHaveNulls() || mask.MayHaveNulls() || replacements_bitmap) {
+    ARROW_ASSIGN_OR_RAISE(output->buffers[0], ctx->AllocateBitmap(array.length));
+    out_bitmap = output->buffers[0]->mutable_data();
+    output->null_count = -1;
+    if (array.MayHaveNulls()) {
+      // Copy array's bitmap
+      arrow::internal::CopyBitmap(array.buffers[0]->data(), array.offset, array.length,
+                                  out_bitmap, /*dest_offset=*/0);
+    } else {
+      // Array has no bitmap but mask/replacements do, generate an all-valid bitmap
+      std::memset(out_bitmap, 0xFF, output->buffers[0]->size());
+    }
+  } else {
+    output->null_count = 0;
+  }
+  auto copy_bitmap = [&](int64_t out_offset, int64_t in_offset, int64_t length) {
+    DCHECK(out_bitmap);
+    if (replacements.is_array()) {
+      const auto& in_data = *replacements.array();
+      const auto in_bitmap = in_data.GetValues<uint8_t>(0, /*absolute_offset=*/0);
+      arrow::internal::CopyBitmap(in_bitmap, in_data.offset + in_offset, length,
+                                  out_bitmap, out_offset);
+    } else {
+      BitUtil::SetBitsTo(out_bitmap, out_offset, length, !replacements_bitmap);
+    }
+  };
+
+  Functor::CopyData(*array.type, out_values, /*out_offset=*/0, array, /*in_offset=*/0,
+                    array.length);
+  arrow::internal::BitBlockCounter value_counter(mask_values, mask.offset, mask.length);

Review comment:
       Doh, yes, that's a good catch, thanks.




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

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

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



[GitHub] [arrow] lidavidm commented on a change in pull request #10412: ARROW-9430: [C++] Implement replace_with_mask kernel

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



##########
File path: cpp/src/arrow/compute/kernels/vector_replace.cc
##########
@@ -0,0 +1,495 @@
+// 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_scalar.h"
+#include "arrow/compute/kernels/common.h"
+#include "arrow/util/bitmap_ops.h"
+
+namespace arrow {
+namespace compute {
+namespace internal {
+
+namespace {
+
+Status ReplacementArrayTooShort(int64_t expected, int64_t actual) {
+  return Status::Invalid("Replacement array must be of appropriate length (expected ",
+                         expected, " items but got ", actual, " items)");
+}
+
+// Helper to implement replace_with kernel with scalar mask for fixed-width types,
+// using callbacks to handle both bool and byte-sized types
+Status ReplaceWithScalarMask(KernelContext* ctx, const ArrayData& array,
+                             const BooleanScalar& mask, const Datum& replacements,
+                             ArrayData* output) {
+  if (!mask.is_valid) {
+    // Output = null
+    ARROW_ASSIGN_OR_RAISE(auto array,
+                          MakeArrayOfNull(array.type, array.length, ctx->memory_pool()));
+    *output = *array->data();
+    return Status::OK();
+  }
+  if (mask.value) {
+    // Output = replacement
+    if (replacements.is_scalar()) {
+      ARROW_ASSIGN_OR_RAISE(
+          auto replacement_array,
+          MakeArrayFromScalar(*replacements.scalar(), array.length, ctx->memory_pool()));
+      *output = *replacement_array->data();
+    } else {
+      auto replacement_array = replacements.array();
+      if (replacement_array->length != array.length) {
+        return ReplacementArrayTooShort(array.length, replacement_array->length);
+      }
+      *output = *replacement_array;
+    }
+  } else {
+    // Output = input
+    *output = array;
+  }
+  return Status::OK();
+}
+
+// Helper to implement replace_with kernel with array mask for fixed-width types,
+// using callbacks to handle both bool and byte-sized types and to handle
+// scalar and array replacements
+template <typename Functor>
+Status ReplaceWithArrayMask(KernelContext* ctx, const ArrayData& array,
+                            const ArrayData& mask, const Datum& replacements,
+                            ArrayData* output) {
+  ARROW_ASSIGN_OR_RAISE(output->buffers[1],
+                        Functor::AllocateData(ctx, *array.type, array.length));
+
+  uint8_t* out_bitmap = nullptr;
+  uint8_t* out_values = output->buffers[1]->mutable_data();
+  const uint8_t* mask_bitmap = mask.MayHaveNulls() ? mask.buffers[0]->data() : nullptr;
+  const uint8_t* mask_values = mask.buffers[1]->data();
+  bool replacements_bitmap;
+  int64_t replacements_length;
+  if (replacements.is_array()) {
+    replacements_bitmap = replacements.array()->MayHaveNulls();
+    replacements_length = replacements.array()->length;
+  } else {
+    replacements_bitmap = !replacements.scalar()->is_valid;
+    replacements_length = std::numeric_limits<int64_t>::max();
+  }
+  if (array.MayHaveNulls() || mask.MayHaveNulls() || replacements_bitmap) {
+    ARROW_ASSIGN_OR_RAISE(output->buffers[0], ctx->AllocateBitmap(array.length));
+    out_bitmap = output->buffers[0]->mutable_data();
+    output->null_count = -1;
+    if (array.MayHaveNulls()) {
+      arrow::internal::CopyBitmap(array.buffers[0]->data(), array.offset, array.length,
+                                  out_bitmap, /*dest_offset=*/0);
+    } else {
+      std::memset(out_bitmap, 0xFF, output->buffers[0]->size());
+    }
+  } else {
+    output->null_count = 0;
+  }
+  auto copy_bitmap = [&](int64_t out_offset, int64_t in_offset, int64_t length) {
+    DCHECK(out_bitmap);
+    if (replacements.is_array()) {
+      const auto& in_data = *replacements.array();
+      const auto in_bitmap = in_data.GetValues<uint8_t>(0, /*absolute_offset=*/0);
+      arrow::internal::CopyBitmap(in_bitmap, in_data.offset + in_offset, length,
+                                  out_bitmap, out_offset);
+    } else {
+      BitUtil::SetBitsTo(out_bitmap, out_offset, length, !replacements_bitmap);
+    }
+  };
+
+  Functor::CopyData(*array.type, out_values, /*out_offset=*/0, array, /*in_offset=*/0,
+                    array.length);
+  arrow::internal::BitBlockCounter value_counter(mask_values, mask.offset, mask.length);
+  arrow::internal::OptionalBitBlockCounter valid_counter(mask_bitmap, mask.offset,
+                                                         mask.length);
+  int64_t out_offset = 0;
+  int64_t replacements_offset = 0;
+  while (out_offset < array.length) {
+    BitBlockCount value_block = value_counter.NextWord();
+    BitBlockCount valid_block = valid_counter.NextWord();
+    DCHECK_EQ(value_block.length, valid_block.length);
+    if (value_block.AllSet() && valid_block.AllSet()) {
+      // Copy from replacement array
+      if (replacements_offset + valid_block.length > replacements_length) {
+        return ReplacementArrayTooShort(replacements_offset + valid_block.length,
+                                        replacements_length);
+      }
+      Functor::CopyData(*array.type, out_values, out_offset, replacements,
+                        replacements_offset, valid_block.length);
+      if (replacements_bitmap) {
+        copy_bitmap(out_offset, replacements_offset, valid_block.length);
+      } else if (!replacements_bitmap && out_bitmap) {
+        BitUtil::SetBitsTo(out_bitmap, out_offset, valid_block.length, true);
+      }
+      replacements_offset += valid_block.length;
+    } else if (value_block.NoneSet() && valid_block.AllSet()) {
+      // Do nothing
+    } else if (valid_block.NoneSet()) {
+      DCHECK(out_bitmap);
+      BitUtil::SetBitsTo(out_bitmap, out_offset, valid_block.length, false);
+    } else {
+      for (int64_t i = 0; i < valid_block.length; ++i) {
+        if (BitUtil::GetBit(mask_values, out_offset + mask.offset + i) &&
+            (!mask_bitmap ||
+             BitUtil::GetBit(mask_bitmap, out_offset + mask.offset + i))) {
+          if (replacements_offset >= replacements_length) {
+            return ReplacementArrayTooShort(replacements_offset + 1, replacements_length);
+          }
+          Functor::CopyData(*array.type, out_values, out_offset + i, replacements,
+                            replacements_offset,
+                            /*length=*/1);
+          if (replacements_bitmap) {
+            copy_bitmap(out_offset + i, replacements_offset, 1);
+          }
+          replacements_offset++;
+        }
+      }
+    }
+    out_offset += valid_block.length;
+  }
+
+  if (mask.MayHaveNulls()) {
+    arrow::internal::BitmapAnd(out_bitmap, /*left_offset=*/0, mask.buffers[0]->data(),
+                               mask.offset, array.length,
+                               /*out_offset=*/0, out_bitmap);

Review comment:
       That branch only applies some of the time, but you are right in that there's no point doing it above since it's replicated here. (However the branch should be kept to skip those values.)




-- 
This is an automated message from the 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 #10412: ARROW-9430: [C++] Implement replace_with_mask kernel

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


   @ursabot please benchmark lang=C++


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

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

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



[GitHub] [arrow] nirandaperera commented on pull request #10412: ARROW-9430: [C++] Implement replace_with_mask kernel

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


   BTW we need to add docs to the PR I think :slightly_smiling_face: 


-- 
This is an automated message from the 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 #10412: ARROW-9430: [C++] Implement replace_with_mask kernel

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



##########
File path: cpp/src/arrow/compute/kernels/vector_replace.cc
##########
@@ -0,0 +1,466 @@
+// 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_scalar.h"
+#include "arrow/compute/kernels/common.h"
+#include "arrow/util/bitmap_ops.h"
+
+namespace arrow {
+namespace compute {
+namespace internal {
+
+namespace {
+
+Status ReplacementArrayTooShort(int64_t expected, int64_t actual) {
+  return Status::Invalid("Replacement array must be of appropriate length (expected ",
+                         expected, " items but got ", actual, " items)");
+}
+
+// Helper to implement replace_with kernel with scalar mask for fixed-width types,
+// using callbacks to handle both bool and byte-sized types
+Status ReplaceWithScalarMask(KernelContext* ctx, const ArrayData& array,
+                             const BooleanScalar& mask, const Datum& replacements,
+                             ArrayData* output) {
+  if (!mask.is_valid) {
+    // Output = null
+    ARROW_ASSIGN_OR_RAISE(auto replacement_array,
+                          MakeArrayOfNull(array.type, array.length, ctx->memory_pool()));
+    *output = *replacement_array->data();
+    return Status::OK();
+  }
+  if (mask.value) {
+    // Output = replacement
+    if (replacements.is_scalar()) {
+      ARROW_ASSIGN_OR_RAISE(
+          auto replacement_array,
+          MakeArrayFromScalar(*replacements.scalar(), array.length, ctx->memory_pool()));
+      *output = *replacement_array->data();
+    } else {
+      auto replacement_array = replacements.array();
+      if (replacement_array->length != array.length) {
+        return ReplacementArrayTooShort(array.length, replacement_array->length);
+      }
+      *output = *replacement_array;
+    }
+  } else {
+    // Output = input
+    *output = array;
+  }
+  return Status::OK();
+}
+
+// Helper to implement replace_with kernel with array mask for fixed-width types,
+// using callbacks to handle both bool and byte-sized types and to handle
+// scalar and array replacements
+template <typename Functor>
+Status ReplaceWithArrayMask(KernelContext* ctx, const ArrayData& array,
+                            const ArrayData& mask, const Datum& replacements,
+                            ArrayData* output) {
+  uint8_t* out_bitmap = nullptr;
+  uint8_t* out_values = output->buffers[1]->mutable_data();
+  const uint8_t* mask_bitmap = mask.MayHaveNulls() ? mask.buffers[0]->data() : nullptr;
+  const uint8_t* mask_values = mask.buffers[1]->data();
+  bool replacements_bitmap;
+  int64_t replacements_length;
+  if (replacements.is_array()) {
+    replacements_bitmap = replacements.array()->MayHaveNulls();
+    replacements_length = replacements.array()->length;
+  } else {
+    replacements_bitmap = !replacements.scalar()->is_valid;
+    replacements_length = std::numeric_limits<int64_t>::max();
+  }
+  if (array.MayHaveNulls() || mask.MayHaveNulls() || replacements_bitmap) {
+    ARROW_ASSIGN_OR_RAISE(output->buffers[0], ctx->AllocateBitmap(array.length));
+    out_bitmap = output->buffers[0]->mutable_data();
+    output->null_count = -1;
+    if (array.MayHaveNulls()) {
+      // Copy array's bitmap
+      arrow::internal::CopyBitmap(array.buffers[0]->data(), array.offset, array.length,
+                                  out_bitmap, /*dest_offset=*/0);
+    } else {
+      // Array has no bitmap but mask/replacements do, generate an all-valid bitmap
+      std::memset(out_bitmap, 0xFF, output->buffers[0]->size());
+    }
+  } else {
+    output->null_count = 0;
+  }
+  auto copy_bitmap = [&](int64_t out_offset, int64_t in_offset, int64_t length) {
+    DCHECK(out_bitmap);
+    if (replacements.is_array()) {
+      const auto& in_data = *replacements.array();
+      const auto in_bitmap = in_data.GetValues<uint8_t>(0, /*absolute_offset=*/0);
+      arrow::internal::CopyBitmap(in_bitmap, in_data.offset + in_offset, length,
+                                  out_bitmap, out_offset);
+    } else {
+      BitUtil::SetBitsTo(out_bitmap, out_offset, length, !replacements_bitmap);
+    }
+  };
+
+  Functor::CopyData(*array.type, out_values, /*out_offset=*/0, array, /*in_offset=*/0,
+                    array.length);
+  arrow::internal::BitBlockCounter value_counter(mask_values, mask.offset, mask.length);

Review comment:
       sorry I meant to say `OptionalBinaryBitBlockCounter`




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

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

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



[GitHub] [arrow] jorisvandenbossche commented on pull request #10412: ARROW-9430: [C++] Implement replace_with_mask kernel

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


   @bkietz do you want to take another look 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] bkietz commented on a change in pull request #10412: ARROW-9430: [C++] Implement replace_with_mask kernel

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



##########
File path: cpp/src/arrow/compute/kernels/vector_replace.cc
##########
@@ -0,0 +1,495 @@
+// 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_scalar.h"
+#include "arrow/compute/kernels/common.h"
+#include "arrow/util/bitmap_ops.h"
+
+namespace arrow {
+namespace compute {
+namespace internal {
+
+namespace {
+
+Status ReplacementArrayTooShort(int64_t expected, int64_t actual) {
+  return Status::Invalid("Replacement array must be of appropriate length (expected ",
+                         expected, " items but got ", actual, " items)");
+}
+
+// Helper to implement replace_with kernel with scalar mask for fixed-width types,
+// using callbacks to handle both bool and byte-sized types
+Status ReplaceWithScalarMask(KernelContext* ctx, const ArrayData& array,
+                             const BooleanScalar& mask, const Datum& replacements,
+                             ArrayData* output) {
+  if (!mask.is_valid) {
+    // Output = null
+    ARROW_ASSIGN_OR_RAISE(auto array,
+                          MakeArrayOfNull(array.type, array.length, ctx->memory_pool()));
+    *output = *array->data();
+    return Status::OK();
+  }
+  if (mask.value) {
+    // Output = replacement
+    if (replacements.is_scalar()) {
+      ARROW_ASSIGN_OR_RAISE(
+          auto replacement_array,
+          MakeArrayFromScalar(*replacements.scalar(), array.length, ctx->memory_pool()));
+      *output = *replacement_array->data();
+    } else {
+      auto replacement_array = replacements.array();
+      if (replacement_array->length != array.length) {
+        return ReplacementArrayTooShort(array.length, replacement_array->length);
+      }
+      *output = *replacement_array;
+    }
+  } else {
+    // Output = input
+    *output = array;
+  }
+  return Status::OK();
+}
+
+// Helper to implement replace_with kernel with array mask for fixed-width types,
+// using callbacks to handle both bool and byte-sized types and to handle
+// scalar and array replacements
+template <typename Functor>
+Status ReplaceWithArrayMask(KernelContext* ctx, const ArrayData& array,
+                            const ArrayData& mask, const Datum& replacements,
+                            ArrayData* output) {
+  ARROW_ASSIGN_OR_RAISE(output->buffers[1],
+                        Functor::AllocateData(ctx, *array.type, array.length));
+
+  uint8_t* out_bitmap = nullptr;
+  uint8_t* out_values = output->buffers[1]->mutable_data();
+  const uint8_t* mask_bitmap = mask.MayHaveNulls() ? mask.buffers[0]->data() : nullptr;
+  const uint8_t* mask_values = mask.buffers[1]->data();
+  bool replacements_bitmap;
+  int64_t replacements_length;
+  if (replacements.is_array()) {
+    replacements_bitmap = replacements.array()->MayHaveNulls();
+    replacements_length = replacements.array()->length;
+  } else {
+    replacements_bitmap = !replacements.scalar()->is_valid;
+    replacements_length = std::numeric_limits<int64_t>::max();
+  }
+  if (array.MayHaveNulls() || mask.MayHaveNulls() || replacements_bitmap) {
+    ARROW_ASSIGN_OR_RAISE(output->buffers[0], ctx->AllocateBitmap(array.length));
+    out_bitmap = output->buffers[0]->mutable_data();
+    output->null_count = -1;
+    if (array.MayHaveNulls()) {
+      arrow::internal::CopyBitmap(array.buffers[0]->data(), array.offset, array.length,
+                                  out_bitmap, /*dest_offset=*/0);
+    } else {
+      std::memset(out_bitmap, 0xFF, output->buffers[0]->size());
+    }
+  } else {
+    output->null_count = 0;
+  }
+  auto copy_bitmap = [&](int64_t out_offset, int64_t in_offset, int64_t length) {
+    DCHECK(out_bitmap);
+    if (replacements.is_array()) {
+      const auto& in_data = *replacements.array();
+      const auto in_bitmap = in_data.GetValues<uint8_t>(0, /*absolute_offset=*/0);
+      arrow::internal::CopyBitmap(in_bitmap, in_data.offset + in_offset, length,
+                                  out_bitmap, out_offset);
+    } else {
+      BitUtil::SetBitsTo(out_bitmap, out_offset, length, !replacements_bitmap);
+    }
+  };
+
+  Functor::CopyData(*array.type, out_values, /*out_offset=*/0, array, /*in_offset=*/0,
+                    array.length);
+  arrow::internal::BitBlockCounter value_counter(mask_values, mask.offset, mask.length);
+  arrow::internal::OptionalBitBlockCounter valid_counter(mask_bitmap, mask.offset,
+                                                         mask.length);
+  int64_t out_offset = 0;
+  int64_t replacements_offset = 0;
+  while (out_offset < array.length) {
+    BitBlockCount value_block = value_counter.NextWord();
+    BitBlockCount valid_block = valid_counter.NextWord();
+    DCHECK_EQ(value_block.length, valid_block.length);
+    if (value_block.AllSet() && valid_block.AllSet()) {
+      // Copy from replacement array
+      if (replacements_offset + valid_block.length > replacements_length) {
+        return ReplacementArrayTooShort(replacements_offset + valid_block.length,
+                                        replacements_length);
+      }
+      Functor::CopyData(*array.type, out_values, out_offset, replacements,
+                        replacements_offset, valid_block.length);
+      if (replacements_bitmap) {
+        copy_bitmap(out_offset, replacements_offset, valid_block.length);
+      } else if (!replacements_bitmap && out_bitmap) {
+        BitUtil::SetBitsTo(out_bitmap, out_offset, valid_block.length, true);
+      }
+      replacements_offset += valid_block.length;
+    } else if (value_block.NoneSet() && valid_block.AllSet()) {
+      // Do nothing
+    } else if (valid_block.NoneSet()) {
+      DCHECK(out_bitmap);
+      BitUtil::SetBitsTo(out_bitmap, out_offset, valid_block.length, false);
+    } else {
+      for (int64_t i = 0; i < valid_block.length; ++i) {
+        if (BitUtil::GetBit(mask_values, out_offset + mask.offset + i) &&
+            (!mask_bitmap ||
+             BitUtil::GetBit(mask_bitmap, out_offset + mask.offset + i))) {
+          if (replacements_offset >= replacements_length) {
+            return ReplacementArrayTooShort(replacements_offset + 1, replacements_length);
+          }
+          Functor::CopyData(*array.type, out_values, out_offset + i, replacements,
+                            replacements_offset,
+                            /*length=*/1);
+          if (replacements_bitmap) {
+            copy_bitmap(out_offset + i, replacements_offset, 1);
+          }
+          replacements_offset++;
+        }
+      }
+    }
+    out_offset += valid_block.length;
+  }
+
+  if (mask.MayHaveNulls()) {
+    arrow::internal::BitmapAnd(out_bitmap, /*left_offset=*/0, mask.buffers[0]->data(),
+                               mask.offset, array.length,
+                               /*out_offset=*/0, out_bitmap);
+  }
+  return Status::OK();
+}
+
+template <typename Type, typename Enable = void>
+struct ReplaceWithMask {};
+
+template <typename Type>
+struct ReplaceWithMask<Type, enable_if_number<Type>> {
+  using T = typename TypeTraits<Type>::CType;
+
+  static Result<std::shared_ptr<Buffer>> AllocateData(KernelContext* ctx, const DataType&,
+                                                      const int64_t length) {
+    return ctx->Allocate(length * sizeof(T));
+  }
+
+  static void CopyData(const DataType&, uint8_t* out, const int64_t out_offset,
+                       const Datum& in, const int64_t in_offset, const int64_t length) {
+    if (in.is_array()) {
+      const auto& in_data = *in.array();
+      const auto in_arr =
+          in_data.GetValues<uint8_t>(1, (in_offset + in_data.offset) * sizeof(T));
+      std::memcpy(out + (out_offset * sizeof(T)), in_arr, length * sizeof(T));
+    } else {
+      T* begin = reinterpret_cast<T*>(out + (out_offset * sizeof(T)));
+      T* end = begin + length;
+      std::fill(begin, end, UnboxScalar<Type>::Unbox(*in.scalar()));
+    }
+  }
+
+  static Status ExecScalarMask(KernelContext* ctx, const ArrayData& array,
+                               const BooleanScalar& mask, const Datum& replacements,
+                               ArrayData* output) {
+    return ReplaceWithScalarMask(ctx, array, mask, replacements, output);
+  }
+
+  static Status ExecArrayMask(KernelContext* ctx, const ArrayData& array,
+                              const ArrayData& mask, const Datum& replacements,
+                              ArrayData* output) {
+    return ReplaceWithArrayMask<ReplaceWithMask<Type>>(ctx, array, mask, replacements,
+                                                       output);
+  }
+};
+
+template <typename Type>
+struct ReplaceWithMask<Type, enable_if_boolean<Type>> {
+  static Result<std::shared_ptr<Buffer>> AllocateData(KernelContext* ctx, const DataType&,
+                                                      const int64_t length) {
+    return ctx->AllocateBitmap(length);
+  }
+
+  static void CopyData(const DataType&, uint8_t* out, const int64_t out_offset,
+                       const Datum& in, const int64_t in_offset, const int64_t length) {
+    if (in.is_array()) {
+      const auto& in_data = *in.array();
+      const auto in_arr = in_data.GetValues<uint8_t>(1, /*absolute_offset=*/0);
+      arrow::internal::CopyBitmap(in_arr, in_offset + in_data.offset, length, out,
+                                  out_offset);
+    } else {
+      BitUtil::SetBitsTo(out, out_offset, length, in.scalar()->is_valid);
+    }
+  }
+
+  static Status ExecScalarMask(KernelContext* ctx, const ArrayData& array,
+                               const BooleanScalar& mask, const Datum& replacements,
+                               ArrayData* output) {
+    return ReplaceWithScalarMask(ctx, array, mask, replacements, output);
+  }
+  static Status ExecArrayMask(KernelContext* ctx, const ArrayData& array,
+                              const ArrayData& mask, const Datum& replacements,
+                              ArrayData* output) {
+    return ReplaceWithArrayMask<ReplaceWithMask<Type>>(ctx, array, mask, replacements,
+                                                       output);
+  }
+};
+
+template <typename Type>
+struct ReplaceWithMask<Type, enable_if_same<Type, FixedSizeBinaryType>> {
+  static Result<std::shared_ptr<Buffer>> AllocateData(KernelContext* ctx,
+                                                      const DataType& ty,
+                                                      const int64_t length) {
+    return ctx->Allocate(length *
+                         checked_cast<const FixedSizeBinaryType&>(ty).byte_width());
+  }
+
+  static void CopyData(const DataType& ty, uint8_t* out, const int64_t out_offset,
+                       const Datum& in, const int64_t in_offset, const int64_t length) {
+    const int32_t width = checked_cast<const FixedSizeBinaryType&>(ty).byte_width();
+    uint8_t* begin = out + (out_offset * width);
+    if (in.is_array()) {
+      const auto& in_data = *in.array();
+      const auto in_arr =
+          in_data.GetValues<uint8_t>(1, (in_offset + in_data.offset) * width);
+      std::memcpy(begin, in_arr, length * width);
+    } else {
+      const FixedSizeBinaryScalar& scalar =
+          checked_cast<const FixedSizeBinaryScalar&>(*in.scalar());
+      // Null scalar may have null value buffer
+      if (!scalar.value) return;
+      const Buffer& buffer = *scalar.value;
+      const uint8_t* value = buffer.data();
+      DCHECK_GE(buffer.size(), width);
+      for (int i = 0; i < length; i++) {
+        std::memcpy(begin, value, width);
+        begin += width;
+      }
+    }
+  }
+
+  static Status ExecScalarMask(KernelContext* ctx, const ArrayData& array,
+                               const BooleanScalar& mask, const Datum& replacements,
+                               ArrayData* output) {
+    return ReplaceWithScalarMask(ctx, array, mask, replacements, output);
+  }
+
+  static Status ExecArrayMask(KernelContext* ctx, const ArrayData& array,
+                              const ArrayData& mask, const Datum& replacements,
+                              ArrayData* output) {
+    return ReplaceWithArrayMask<ReplaceWithMask<Type>>(ctx, array, mask, replacements,
+                                                       output);
+  }
+};
+
+template <typename Type>
+struct ReplaceWithMask<Type, enable_if_decimal<Type>> {
+  using ScalarType = typename TypeTraits<Type>::ScalarType;
+
+  static Result<std::shared_ptr<Buffer>> AllocateData(KernelContext* ctx,
+                                                      const DataType& ty,
+                                                      const int64_t length) {
+    return ctx->Allocate(length *
+                         checked_cast<const FixedSizeBinaryType&>(ty).byte_width());
+  }
+
+  static void CopyData(const DataType& ty, uint8_t* out, const int64_t out_offset,
+                       const Datum& in, const int64_t in_offset, const int64_t length) {
+    const int32_t width = checked_cast<const FixedSizeBinaryType&>(ty).byte_width();
+    uint8_t* begin = out + (out_offset * width);
+    if (in.is_array()) {
+      const auto& in_data = *in.array();
+      const auto in_arr =
+          in_data.GetValues<uint8_t>(1, (in_offset + in_data.offset) * width);
+      std::memcpy(begin, in_arr, length * width);
+    } else {
+      const ScalarType& scalar = checked_cast<const ScalarType&>(*in.scalar());
+      const auto value = scalar.value.ToBytes();
+      for (int i = 0; i < length; i++) {
+        std::memcpy(begin, value.data(), width);
+        begin += width;
+      }
+    }
+  }
+
+  static Status ExecScalarMask(KernelContext* ctx, const ArrayData& array,
+                               const BooleanScalar& mask, const Datum& replacements,
+                               ArrayData* output) {
+    return ReplaceWithScalarMask(ctx, array, mask, replacements, output);
+  }
+
+  static Status ExecArrayMask(KernelContext* ctx, const ArrayData& array,
+                              const ArrayData& mask, const Datum& replacements,
+                              ArrayData* output) {
+    return ReplaceWithArrayMask<ReplaceWithMask<Type>>(ctx, array, mask, replacements,
+                                                       output);
+  }
+};
+
+template <typename Type>
+struct ReplaceWithMask<Type, enable_if_null<Type>> {
+  static Status ExecScalarMask(KernelContext* ctx, const ArrayData& array,
+                               const BooleanScalar& mask, const Datum& replacements,
+                               ArrayData* output) {
+    *output = array;
+    return Status::OK();
+  }
+  static Status ExecArrayMask(KernelContext* ctx, const ArrayData& array,
+                              const ArrayData& mask, const Datum& replacements,
+                              ArrayData* output) {
+    *output = array;
+    return Status::OK();
+  }
+};
+
+template <typename Type>
+struct ReplaceWithMask<Type, enable_if_base_binary<Type>> {
+  using offset_type = typename Type::offset_type;
+  using BuilderType = typename TypeTraits<Type>::BuilderType;
+
+  static Status ExecScalarMask(KernelContext* ctx, const ArrayData& array,
+                               const BooleanScalar& mask, const Datum& replacements,
+                               ArrayData* output) {
+    return ReplaceWithScalarMask(ctx, array, mask, replacements, output);
+  }
+  static Status ExecArrayMask(KernelContext* ctx, const ArrayData& array,
+                              const ArrayData& mask, const Datum& replacements,
+                              ArrayData* output) {
+    BuilderType builder(array.type, ctx->memory_pool());
+    RETURN_NOT_OK(builder.Reserve(array.length));
+    RETURN_NOT_OK(builder.ReserveData(array.buffers[2]->size()));
+    int64_t source_offset = 0;
+    int64_t replacements_offset = 0;
+    RETURN_NOT_OK(VisitArrayDataInline<BooleanType>(
+        mask,
+        [&](bool replace) {
+          if (replace && replacements.is_scalar()) {
+            const Scalar& scalar = *replacements.scalar();
+            if (scalar.is_valid) {
+              RETURN_NOT_OK(builder.Append(UnboxScalar<Type>::Unbox(scalar)));
+            } else {
+              RETURN_NOT_OK(builder.AppendNull());
+            }
+          } else {
+            const ArrayData& source = replace ? *replacements.array() : array;
+            const int64_t offset = replace ? replacements_offset++ : source_offset;
+            if (!source.MayHaveNulls() ||
+                BitUtil::GetBit(source.buffers[0]->data(), source.offset + offset)) {
+              const uint8_t* data = source.buffers[2]->data();
+              const offset_type* offsets = source.GetValues<offset_type>(1);
+              const offset_type offset0 = offsets[offset];
+              const offset_type offset1 = offsets[offset + 1];
+              RETURN_NOT_OK(builder.Append(data + offset0, offset1 - offset0));
+            } else {
+              RETURN_NOT_OK(builder.AppendNull());
+            }
+          }
+          source_offset++;
+          return Status::OK();
+        },
+        [&]() {
+          RETURN_NOT_OK(builder.AppendNull());
+          source_offset++;
+          return Status::OK();
+        }));
+    std::shared_ptr<Array> temp_output;
+    RETURN_NOT_OK(builder.Finish(&temp_output));
+    *output = *temp_output->data();
+    // Builder type != logical type due to GenerateTypeAgnosticVarBinaryBase
+    output->type = array.type;
+    return Status::OK();
+  }
+};
+
+template <typename Type>
+struct ReplaceWithMaskFunctor {
+  static Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    const ArrayData& array = *batch[0].array();
+    const Datum& replacements = batch[2];
+    ArrayData* output = out->array().get();
+    output->length = array.length;
+
+    // Needed for FixedSizeBinary/parameterized types
+    if (!array.type->Equals(*replacements.type(), /*check_metadata=*/false)) {
+      return Status::Invalid("Replacements must be of same type (expected ",
+                             array.type->ToString(), " but got ",
+                             replacements.type()->ToString(), ")");
+    }
+
+    if (!replacements.is_array() && !replacements.is_scalar()) {
+      return Status::Invalid("Replacements must be array or scalar");
+    }
+
+    if (batch[1].is_scalar()) {
+      return ReplaceWithMask<Type>::ExecScalarMask(
+          ctx, array, batch[1].scalar_as<BooleanScalar>(), replacements, output);
+    }
+    const ArrayData& mask = *batch[1].array();
+    if (array.length != mask.length) {
+      return Status::Invalid("Mask must be of same length as array (expected ",
+                             array.length, " items but got ", mask.length, " items)");
+    }
+    return ReplaceWithMask<Type>::ExecArrayMask(ctx, array, mask, replacements, output);
+  }
+};
+
+}  // namespace
+
+const FunctionDoc replace_with_mask_doc(
+    "Replace items using a mask and replacement values",
+    ("Given an array and a Boolean mask (either scalar or of equal length), "
+     "along with replacement values (either scalar or array), "
+     "each element of the array for which the corresponding mask element is "
+     "true will be replaced by the next value from the replacements, "
+     "or with null if the mask is null. "
+     "Hence, for replacement arrays, len(replacements) == sum(mask == true)."),
+    {"values", "mask", "replacements"});
+
+void RegisterVectorReplace(FunctionRegistry* registry) {
+  auto func = std::make_shared<VectorFunction>("replace_with_mask", Arity::Ternary(),
+                                               &replace_with_mask_doc);
+  auto add_kernel = [&](const std::shared_ptr<DataType>& ty, ArrayKernelExec exec) {
+    VectorKernel kernel;
+    kernel.can_execute_chunkwise = false;
+    kernel.null_handling = NullHandling::type::COMPUTED_NO_PREALLOCATE;
+    kernel.mem_allocation = MemAllocation::type::NO_PREALLOCATE;
+    kernel.signature = KernelSignature::Make(
+        {InputType::Array(ty->id()), InputType(boolean()), InputType(ty->id())},
+        OutputType(FirstType));
+    kernel.exec = std::move(exec);
+    DCHECK_OK(func->AddKernel(std::move(kernel)));
+  };
+  auto add_primitive_kernel = [&](const std::shared_ptr<DataType>& ty) {
+    add_kernel(ty, GenerateTypeAgnosticPrimitive<ReplaceWithMaskFunctor>(*ty));
+  };
+  for (const auto& ty : NumericTypes()) {
+    add_primitive_kernel(ty);
+  }
+  for (const auto& ty : TemporalTypes()) {
+    add_primitive_kernel(ty);
+  }
+  add_primitive_kernel(null());
+  add_primitive_kernel(boolean());
+  add_primitive_kernel(day_time_interval());
+  add_primitive_kernel(month_interval());
+  add_kernel(fixed_size_binary(0), ReplaceWithMaskFunctor<FixedSizeBinaryType>::Exec);
+  add_kernel(decimal128(1, 1), ReplaceWithMaskFunctor<Decimal128Type>::Exec);
+  add_kernel(decimal256(1, 1), ReplaceWithMaskFunctor<Decimal256Type>::Exec);
+  for (const auto& ty : BaseBinaryTypes()) {
+    add_kernel(ty, GenerateTypeAgnosticVarBinaryBase<ReplaceWithMaskFunctor>(*ty));

Review comment:
       Nit:
   ```suggestion
     add_kernel(Type::FIXED_SIZE_BINARY, ReplaceWithMaskFunctor<FixedSizeBinaryType>::Exec);
     add_kernel(Type::DECIMAL128, ReplaceWithMaskFunctor<Decimal128Type>::Exec);
     add_kernel(Type::DECIMAL256, ReplaceWithMaskFunctor<Decimal256Type>::Exec);
     for (const auto& ty : BaseBinaryTypes()) {
       add_kernel(ty->id(), GenerateTypeAgnosticVarBinaryBase<ReplaceWithMaskFunctor>(*ty));
   ```




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

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



[GitHub] [arrow] lidavidm commented on pull request #10412: ARROW-9430: [C++] Implement override_mask kernel

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


   Note once #10410 goes through, this should probably be consolidated into the same file (`scalar_if_else.cc`).


-- 
This is an automated message from the 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 #10412: ARROW-9430: [C++] Implement override_mask kernel

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


   (N.B. needs some more work - trying to add a test with random data)


-- 
This is an automated message from the 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 #10412: ARROW-9430: [C++] Implement replace_with_mask kernel

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



##########
File path: cpp/src/arrow/compute/kernels/vector_replace.cc
##########
@@ -0,0 +1,510 @@
+// 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_scalar.h"
+#include "arrow/compute/kernels/common.h"
+#include "arrow/util/bitmap_ops.h"
+
+namespace arrow {
+namespace compute {
+namespace internal {
+
+namespace {
+
+Status ReplacementArrayTooShort(int64_t expected, int64_t actual) {
+  return Status::Invalid("Replacement array must be of appropriate length (expected ",
+                         expected, " items but got ", actual, " items)");
+}
+
+// Helper to implement replace_with kernel with scalar mask for fixed-width types,
+// using callbacks to handle both bool and byte-sized types
+Status ReplaceWithScalarMask(KernelContext* ctx, const ArrayData& array,
+                             const BooleanScalar& mask, const Datum& replacements,
+                             ArrayData* output) {
+  if (!mask.is_valid) {
+    // Output = null
+    ARROW_ASSIGN_OR_RAISE(auto replacement_array,
+                          MakeArrayOfNull(array.type, array.length, ctx->memory_pool()));
+    *output = *replacement_array->data();
+    return Status::OK();
+  }
+  if (mask.value) {
+    // Output = replacement
+    if (replacements.is_scalar()) {
+      ARROW_ASSIGN_OR_RAISE(
+          auto replacement_array,
+          MakeArrayFromScalar(*replacements.scalar(), array.length, ctx->memory_pool()));
+      *output = *replacement_array->data();
+    } else {
+      auto replacement_array = replacements.array();
+      if (replacement_array->length != array.length) {
+        return ReplacementArrayTooShort(array.length, replacement_array->length);
+      }
+      *output = *replacement_array;
+    }
+  } else {
+    // Output = input
+    *output = array;
+  }
+  return Status::OK();
+}
+
+struct CopyArrayBitmap {
+  const uint8_t* in_bitmap;
+  int64_t in_offset;
+
+  void CopyBitmap(uint8_t* out_bitmap, int64_t out_offset, int64_t offset,
+                  int64_t length) const {
+    arrow::internal::CopyBitmap(in_bitmap, in_offset + offset, length, out_bitmap,
+                                out_offset);
+  }
+
+  void SetBit(uint8_t* out_bitmap, int64_t out_offset, int64_t offset) const {
+    BitUtil::SetBitTo(out_bitmap, out_offset,
+                      BitUtil::GetBit(in_bitmap, in_offset + offset));
+  }
+};
+
+struct CopyScalarBitmap {
+  const bool is_valid;
+
+  void CopyBitmap(uint8_t* out_bitmap, int64_t out_offset, int64_t offset,
+                  int64_t length) const {
+    BitUtil::SetBitsTo(out_bitmap, out_offset, length, is_valid);
+  }
+
+  void SetBit(uint8_t* out_bitmap, int64_t out_offset, int64_t offset) const {
+    BitUtil::SetBitTo(out_bitmap, out_offset, is_valid);
+  }
+};
+
+// Helper to implement replace_with kernel with array mask for fixed-width types,
+// using callbacks to handle both bool and byte-sized types and to handle
+// scalar and array replacements
+template <typename Functor, typename Data, typename CopyBitmap>
+void ReplaceWithArrayMaskImpl(const ArrayData& array, const ArrayData& mask,
+                              const Data& replacements, bool replacements_bitmap,
+                              const CopyBitmap copy_bitmap, const uint8_t* mask_bitmap,
+                              const uint8_t* mask_values, uint8_t* out_bitmap,
+                              uint8_t* out_values, const int64_t out_offset) {
+  Functor::CopyData(*array.type, out_values, /*out_offset=*/0, array, /*in_offset=*/0,
+                    array.length);
+  arrow::internal::OptionalBinaryBitBlockCounter counter(
+      mask_values, mask.offset, mask_bitmap, mask.offset, mask.length);
+  int64_t write_offset = 0;
+  int64_t replacements_offset = 0;
+  while (write_offset < array.length) {
+    BitBlockCount block = counter.NextAndBlock();
+    if (block.AllSet()) {
+      // Copy from replacement array
+      Functor::CopyData(*array.type, out_values, out_offset + write_offset, replacements,
+                        replacements_offset, block.length);
+      if (replacements_bitmap) {
+        copy_bitmap.CopyBitmap(out_bitmap, out_offset + write_offset, replacements_offset,
+                               block.length);
+      } else if (!replacements_bitmap && out_bitmap) {
+        BitUtil::SetBitsTo(out_bitmap, out_offset + write_offset, block.length, true);
+      }
+      replacements_offset += block.length;
+    } else if (block.popcount) {
+      for (int64_t i = 0; i < block.length; ++i) {
+        if (BitUtil::GetBit(mask_values, write_offset + mask.offset + i) &&
+            (!mask_bitmap ||
+             BitUtil::GetBit(mask_bitmap, write_offset + mask.offset + i))) {
+          Functor::CopyData(*array.type, out_values, out_offset + write_offset + i,
+                            replacements, replacements_offset, /*length=*/1);
+          if (replacements_bitmap) {
+            copy_bitmap.SetBit(out_bitmap, out_offset + write_offset + i,
+                               replacements_offset);
+          }
+          replacements_offset++;
+        }
+      }
+    }
+    write_offset += block.length;
+  }
+}
+
+template <typename Functor>
+Status ReplaceWithArrayMask(KernelContext* ctx, const ArrayData& array,
+                            const ArrayData& mask, const Datum& replacements,
+                            ArrayData* output) {
+  const int64_t out_offset = output->offset;
+  uint8_t* out_bitmap = nullptr;
+  uint8_t* out_values = output->buffers[1]->mutable_data();
+  const uint8_t* mask_bitmap = mask.MayHaveNulls() ? mask.buffers[0]->data() : nullptr;
+  const uint8_t* mask_values = mask.buffers[1]->data();
+  const bool replacements_bitmap = replacements.is_array()
+                                       ? replacements.array()->MayHaveNulls()
+                                       : !replacements.scalar()->is_valid;
+  if (replacements.is_array()) {
+    // Check that we have enough replacement values
+    const int64_t replacements_length = replacements.array()->length;
+
+    arrow::internal::OptionalBinaryBitBlockCounter counter(
+        mask_values, mask.offset, mask_bitmap, mask.offset, mask.length);
+    int64_t count = 0;
+    for (int64_t offset = 0; offset < mask.length;) {
+      BitBlockCount block = counter.NextAndBlock();
+      count += block.popcount;
+      offset += block.length;
+    }
+    if (count > replacements_length) {
+      return ReplacementArrayTooShort(count, replacements_length);
+    }
+  }
+  if (array.MayHaveNulls() || mask.MayHaveNulls() || replacements_bitmap) {
+    out_bitmap = output->buffers[0]->mutable_data();
+    output->null_count = -1;
+    if (array.MayHaveNulls()) {
+      // Copy array's bitmap
+      arrow::internal::CopyBitmap(array.buffers[0]->data(), array.offset, array.length,
+                                  out_bitmap, out_offset);
+    } else {
+      // Array has no bitmap but mask/replacements do, generate an all-valid bitmap
+      BitUtil::SetBitsTo(out_bitmap, out_offset, array.length, true);
+    }
+  } else {
+    BitUtil::SetBitsTo(output->buffers[0]->mutable_data(), out_offset, array.length,
+                       true);
+    output->null_count = 0;
+  }
+
+  if (replacements.is_array()) {
+    const ArrayData& array_repl = *replacements.array();
+    ReplaceWithArrayMaskImpl<Functor>(
+        array, mask, array_repl, replacements_bitmap,
+        CopyArrayBitmap{replacements_bitmap ? array_repl.buffers[0]->data() : nullptr,
+                        array_repl.offset},
+        mask_bitmap, mask_values, out_bitmap, out_values, out_offset);
+  } else {
+    const Scalar& scalar_repl = *replacements.scalar();
+    ReplaceWithArrayMaskImpl<Functor>(array, mask, scalar_repl, replacements_bitmap,
+                                      CopyScalarBitmap{scalar_repl.is_valid}, mask_bitmap,
+                                      mask_values, out_bitmap, out_values, out_offset);
+  }
+
+  if (mask.MayHaveNulls()) {
+    arrow::internal::BitmapAnd(out_bitmap, out_offset, mask.buffers[0]->data(),
+                               mask.offset, array.length, out_offset, out_bitmap);
+  }
+  return Status::OK();
+}
+
+template <typename Type, typename Enable = void>
+struct ReplaceWithMask {};
+
+template <typename Type>
+struct ReplaceWithMask<Type, enable_if_number<Type>> {
+  using T = typename TypeTraits<Type>::CType;
+
+  static void CopyData(const DataType&, uint8_t* out, const int64_t out_offset,
+                       const ArrayData& in, const int64_t in_offset,
+                       const int64_t length) {
+    const auto in_arr = in.GetValues<uint8_t>(1, (in_offset + in.offset) * sizeof(T));
+    std::memcpy(out + (out_offset * sizeof(T)), in_arr, length * sizeof(T));
+  }
+
+  static void CopyData(const DataType&, uint8_t* out, const int64_t out_offset,
+                       const Scalar& in, const int64_t in_offset, const int64_t length) {
+    T* begin = reinterpret_cast<T*>(out + (out_offset * sizeof(T)));
+    T* end = begin + length;
+    std::fill(begin, end, UnboxScalar<Type>::Unbox(in));
+  }
+
+  static Status ExecScalarMask(KernelContext* ctx, const ArrayData& array,
+                               const BooleanScalar& mask, const Datum& replacements,
+                               ArrayData* output) {
+    return ReplaceWithScalarMask(ctx, array, mask, replacements, output);
+  }
+
+  static Status ExecArrayMask(KernelContext* ctx, const ArrayData& array,
+                              const ArrayData& mask, const Datum& replacements,
+                              ArrayData* output) {
+    return ReplaceWithArrayMask<ReplaceWithMask<Type>>(ctx, array, mask, replacements,
+                                                       output);
+  }
+};
+
+template <typename Type>
+struct ReplaceWithMask<Type, enable_if_boolean<Type>> {
+  static void CopyData(const DataType&, uint8_t* out, const int64_t out_offset,
+                       const ArrayData& in, const int64_t in_offset,
+                       const int64_t length) {
+    const auto in_arr = in.GetValues<uint8_t>(1, /*absolute_offset=*/0);
+    arrow::internal::CopyBitmap(in_arr, in_offset + in.offset, length, out, out_offset);
+  }
+  static void CopyData(const DataType&, uint8_t* out, const int64_t out_offset,
+                       const Scalar& in, const int64_t in_offset, const int64_t length) {
+    BitUtil::SetBitsTo(out, out_offset, length, in.is_valid);
+  }
+
+  static Status ExecScalarMask(KernelContext* ctx, const ArrayData& array,
+                               const BooleanScalar& mask, const Datum& replacements,
+                               ArrayData* output) {
+    return ReplaceWithScalarMask(ctx, array, mask, replacements, output);
+  }
+  static Status ExecArrayMask(KernelContext* ctx, const ArrayData& array,
+                              const ArrayData& mask, const Datum& replacements,
+                              ArrayData* output) {
+    return ReplaceWithArrayMask<ReplaceWithMask<Type>>(ctx, array, mask, replacements,
+                                                       output);
+  }
+};
+
+template <typename Type>
+struct ReplaceWithMask<Type, enable_if_same<Type, FixedSizeBinaryType>> {
+  static void CopyData(const DataType& ty, uint8_t* out, const int64_t out_offset,
+                       const ArrayData& in, const int64_t in_offset,
+                       const int64_t length) {
+    const int32_t width = checked_cast<const FixedSizeBinaryType&>(ty).byte_width();
+    uint8_t* begin = out + (out_offset * width);
+    const auto in_arr = in.GetValues<uint8_t>(1, (in_offset + in.offset) * width);
+    std::memcpy(begin, in_arr, length * width);
+  }
+  static void CopyData(const DataType& ty, uint8_t* out, const int64_t out_offset,
+                       const Scalar& in, const int64_t in_offset, const int64_t length) {
+    const int32_t width = checked_cast<const FixedSizeBinaryType&>(ty).byte_width();
+    uint8_t* begin = out + (out_offset * width);
+    const auto& scalar = checked_cast<const FixedSizeBinaryScalar&>(in);
+    // Null scalar may have null value buffer
+    if (!scalar.value) return;
+    const Buffer& buffer = *scalar.value;
+    const uint8_t* value = buffer.data();
+    DCHECK_GE(buffer.size(), width);
+    for (int i = 0; i < length; i++) {
+      std::memcpy(begin, value, width);
+      begin += width;
+    }
+  }
+
+  static Status ExecScalarMask(KernelContext* ctx, const ArrayData& array,
+                               const BooleanScalar& mask, const Datum& replacements,
+                               ArrayData* output) {
+    return ReplaceWithScalarMask(ctx, array, mask, replacements, output);
+  }
+
+  static Status ExecArrayMask(KernelContext* ctx, const ArrayData& array,
+                              const ArrayData& mask, const Datum& replacements,
+                              ArrayData* output) {
+    return ReplaceWithArrayMask<ReplaceWithMask<Type>>(ctx, array, mask, replacements,
+                                                       output);
+  }
+};
+
+template <typename Type>
+struct ReplaceWithMask<Type, enable_if_decimal<Type>> {
+  using ScalarType = typename TypeTraits<Type>::ScalarType;
+  static void CopyData(const DataType& ty, uint8_t* out, const int64_t out_offset,
+                       const ArrayData& in, const int64_t in_offset,
+                       const int64_t length) {
+    const int32_t width = checked_cast<const FixedSizeBinaryType&>(ty).byte_width();
+    uint8_t* begin = out + (out_offset * width);
+    const auto in_arr = in.GetValues<uint8_t>(1, (in_offset + in.offset) * width);
+    std::memcpy(begin, in_arr, length * width);
+  }
+  static void CopyData(const DataType& ty, uint8_t* out, const int64_t out_offset,
+                       const Scalar& in, const int64_t in_offset, const int64_t length) {
+    const int32_t width = checked_cast<const FixedSizeBinaryType&>(ty).byte_width();
+    uint8_t* begin = out + (out_offset * width);
+    const auto& scalar = checked_cast<const ScalarType&>(in);
+    const auto value = scalar.value.ToBytes();
+    for (int i = 0; i < length; i++) {
+      std::memcpy(begin, value.data(), width);
+      begin += width;
+    }
+  }
+
+  static Status ExecScalarMask(KernelContext* ctx, const ArrayData& array,
+                               const BooleanScalar& mask, const Datum& replacements,
+                               ArrayData* output) {
+    return ReplaceWithScalarMask(ctx, array, mask, replacements, output);
+  }
+
+  static Status ExecArrayMask(KernelContext* ctx, const ArrayData& array,
+                              const ArrayData& mask, const Datum& replacements,
+                              ArrayData* output) {
+    return ReplaceWithArrayMask<ReplaceWithMask<Type>>(ctx, array, mask, replacements,
+                                                       output);
+  }
+};
+
+template <typename Type>
+struct ReplaceWithMask<Type, enable_if_null<Type>> {
+  static Status ExecScalarMask(KernelContext* ctx, const ArrayData& array,
+                               const BooleanScalar& mask, const Datum& replacements,
+                               ArrayData* output) {
+    *output = array;
+    return Status::OK();
+  }
+  static Status ExecArrayMask(KernelContext* ctx, const ArrayData& array,
+                              const ArrayData& mask, const Datum& replacements,
+                              ArrayData* output) {
+    *output = array;
+    return Status::OK();
+  }
+};
+
+template <typename Type>
+struct ReplaceWithMask<Type, enable_if_base_binary<Type>> {
+  using offset_type = typename Type::offset_type;
+  using BuilderType = typename TypeTraits<Type>::BuilderType;
+
+  static Status ExecScalarMask(KernelContext* ctx, const ArrayData& array,
+                               const BooleanScalar& mask, const Datum& replacements,
+                               ArrayData* output) {
+    return ReplaceWithScalarMask(ctx, array, mask, replacements, output);
+  }
+  static Status ExecArrayMask(KernelContext* ctx, const ArrayData& array,
+                              const ArrayData& mask, const Datum& replacements,
+                              ArrayData* output) {
+    BuilderType builder(array.type, ctx->memory_pool());
+    RETURN_NOT_OK(builder.Reserve(array.length));
+    RETURN_NOT_OK(builder.ReserveData(array.buffers[2]->size()));
+    int64_t source_offset = 0;
+    int64_t replacements_offset = 0;
+    RETURN_NOT_OK(VisitArrayDataInline<BooleanType>(
+        mask,
+        [&](bool replace) {
+          if (replace && replacements.is_scalar()) {
+            const Scalar& scalar = *replacements.scalar();
+            if (scalar.is_valid) {
+              RETURN_NOT_OK(builder.Append(UnboxScalar<Type>::Unbox(scalar)));
+            } else {
+              RETURN_NOT_OK(builder.AppendNull());
+            }
+          } else {
+            const ArrayData& source = replace ? *replacements.array() : array;
+            const int64_t offset = replace ? replacements_offset++ : source_offset;
+            if (!source.MayHaveNulls() ||
+                BitUtil::GetBit(source.buffers[0]->data(), source.offset + offset)) {
+              const uint8_t* data = source.buffers[2]->data();
+              const offset_type* offsets = source.GetValues<offset_type>(1);
+              const offset_type offset0 = offsets[offset];
+              const offset_type offset1 = offsets[offset + 1];
+              RETURN_NOT_OK(builder.Append(data + offset0, offset1 - offset0));
+            } else {
+              RETURN_NOT_OK(builder.AppendNull());
+            }
+          }
+          source_offset++;
+          return Status::OK();
+        },
+        [&]() {
+          RETURN_NOT_OK(builder.AppendNull());
+          source_offset++;
+          return Status::OK();
+        }));
+    std::shared_ptr<Array> temp_output;
+    RETURN_NOT_OK(builder.Finish(&temp_output));
+    *output = *temp_output->data();
+    // Builder type != logical type due to GenerateTypeAgnosticVarBinaryBase
+    output->type = array.type;
+    return Status::OK();
+  }
+};
+
+template <typename Type>
+struct ReplaceWithMaskFunctor {
+  static Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    const ArrayData& array = *batch[0].array();
+    const Datum& replacements = batch[2];
+    ArrayData* output = out->array().get();
+    output->length = array.length;
+
+    // Needed for FixedSizeBinary/parameterized types
+    if (!array.type->Equals(*replacements.type(), /*check_metadata=*/false)) {
+      return Status::Invalid("Replacements must be of same type (expected ",
+                             array.type->ToString(), " but got ",
+                             replacements.type()->ToString(), ")");
+    }
+
+    if (!replacements.is_array() && !replacements.is_scalar()) {
+      return Status::Invalid("Replacements must be array or scalar");
+    }
+
+    if (batch[1].is_scalar()) {
+      return ReplaceWithMask<Type>::ExecScalarMask(
+          ctx, array, batch[1].scalar_as<BooleanScalar>(), replacements, output);
+    }
+    const ArrayData& mask = *batch[1].array();
+    if (array.length != mask.length) {
+      return Status::Invalid("Mask must be of same length as array (expected ",
+                             array.length, " items but got ", mask.length, " items)");
+    }
+    return ReplaceWithMask<Type>::ExecArrayMask(ctx, array, mask, replacements, output);
+  }
+};
+
+}  // namespace
+
+const FunctionDoc replace_with_mask_doc(
+    "Replace items using a mask and replacement values",
+    ("Given an array and a Boolean mask (either scalar or of equal length), "
+     "along with replacement values (either scalar or array), "
+     "each element of the array for which the corresponding mask element is "
+     "true will be replaced by the next value from the replacements, "
+     "or with null if the mask is null. "
+     "Hence, for replacement arrays, len(replacements) == sum(mask == true)."),
+    {"values", "mask", "replacements"});
+
+void RegisterVectorReplace(FunctionRegistry* registry) {
+  auto func = std::make_shared<VectorFunction>("replace_with_mask", Arity::Ternary(),
+                                               &replace_with_mask_doc);
+  auto add_kernel = [&](detail::GetTypeId get_id, ArrayKernelExec exec) {
+    VectorKernel kernel;
+    kernel.can_execute_chunkwise = false;
+    if (is_fixed_width(get_id.id)) {
+      kernel.null_handling = NullHandling::type::COMPUTED_PREALLOCATE;
+    } else {
+      kernel.can_write_into_slices = false;
+      kernel.null_handling = NullHandling::type::COMPUTED_NO_PREALLOCATE;
+    }
+    kernel.mem_allocation = MemAllocation::type::PREALLOCATE;

Review comment:
       Actually: I'm not sure if it even makes sense for this kernel to write into a slice, since it needs the entirety of all its arguments to execute. So if we manually force it to write into a slice of the output, it'd write different results.




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

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

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



[GitHub] [arrow] lidavidm commented on pull request #10412: ARROW-9430: [C++] Implement replace_with_mask kernel

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


   This is now a vector kernel, with docs, that should support any fixed-size type. However I still need to add support for binary types which I expect people would want to use.


-- 
This is an automated message from the 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 #10412: ARROW-9430: [C++] Implement replace_with_mask kernel

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


   Quick benchmark:
   
   ```
   -------------------------------------------------------------------------------------------------------
   Benchmark                                             Time             CPU   Iterations UserCounters...
   -------------------------------------------------------------------------------------------------------
   ReplaceWithMaskLowSelectivityBench/16384/0        90109 ns        90108 ns         7647 bytes_per_second=1.35471G/s
   ReplaceWithMaskHighSelectivityBench/16384/0      280454 ns       280453 ns         2509 bytes_per_second=445.708M/s
   ReplaceWithMaskLowSelectivityBench/16384/99       91352 ns        91352 ns         7695 bytes_per_second=1.32819G/s
   ReplaceWithMaskHighSelectivityBench/16384/99     275072 ns       275075 ns         2535 bytes_per_second=451.676M/s
   ```


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

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

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



[GitHub] [arrow] lidavidm commented on a change in pull request #10412: ARROW-9430: [C++] Implement replace_with_mask kernel

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



##########
File path: cpp/src/arrow/compute/kernels/vector_replace.cc
##########
@@ -0,0 +1,495 @@
+// 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_scalar.h"
+#include "arrow/compute/kernels/common.h"
+#include "arrow/util/bitmap_ops.h"
+
+namespace arrow {
+namespace compute {
+namespace internal {
+
+namespace {
+
+Status ReplacementArrayTooShort(int64_t expected, int64_t actual) {
+  return Status::Invalid("Replacement array must be of appropriate length (expected ",
+                         expected, " items but got ", actual, " items)");
+}
+
+// Helper to implement replace_with kernel with scalar mask for fixed-width types,
+// using callbacks to handle both bool and byte-sized types
+Status ReplaceWithScalarMask(KernelContext* ctx, const ArrayData& array,
+                             const BooleanScalar& mask, const Datum& replacements,
+                             ArrayData* output) {
+  if (!mask.is_valid) {
+    // Output = null
+    ARROW_ASSIGN_OR_RAISE(auto array,
+                          MakeArrayOfNull(array.type, array.length, ctx->memory_pool()));
+    *output = *array->data();
+    return Status::OK();
+  }
+  if (mask.value) {
+    // Output = replacement
+    if (replacements.is_scalar()) {
+      ARROW_ASSIGN_OR_RAISE(
+          auto replacement_array,
+          MakeArrayFromScalar(*replacements.scalar(), array.length, ctx->memory_pool()));
+      *output = *replacement_array->data();
+    } else {
+      auto replacement_array = replacements.array();
+      if (replacement_array->length != array.length) {
+        return ReplacementArrayTooShort(array.length, replacement_array->length);
+      }
+      *output = *replacement_array;
+    }
+  } else {
+    // Output = input
+    *output = array;
+  }
+  return Status::OK();
+}
+
+// Helper to implement replace_with kernel with array mask for fixed-width types,
+// using callbacks to handle both bool and byte-sized types and to handle
+// scalar and array replacements
+template <typename Functor>
+Status ReplaceWithArrayMask(KernelContext* ctx, const ArrayData& array,
+                            const ArrayData& mask, const Datum& replacements,
+                            ArrayData* output) {
+  ARROW_ASSIGN_OR_RAISE(output->buffers[1],
+                        Functor::AllocateData(ctx, *array.type, array.length));
+
+  uint8_t* out_bitmap = nullptr;
+  uint8_t* out_values = output->buffers[1]->mutable_data();
+  const uint8_t* mask_bitmap = mask.MayHaveNulls() ? mask.buffers[0]->data() : nullptr;
+  const uint8_t* mask_values = mask.buffers[1]->data();
+  bool replacements_bitmap;
+  int64_t replacements_length;
+  if (replacements.is_array()) {
+    replacements_bitmap = replacements.array()->MayHaveNulls();
+    replacements_length = replacements.array()->length;
+  } else {
+    replacements_bitmap = !replacements.scalar()->is_valid;
+    replacements_length = std::numeric_limits<int64_t>::max();
+  }
+  if (array.MayHaveNulls() || mask.MayHaveNulls() || replacements_bitmap) {
+    ARROW_ASSIGN_OR_RAISE(output->buffers[0], ctx->AllocateBitmap(array.length));
+    out_bitmap = output->buffers[0]->mutable_data();
+    output->null_count = -1;
+    if (array.MayHaveNulls()) {
+      arrow::internal::CopyBitmap(array.buffers[0]->data(), array.offset, array.length,
+                                  out_bitmap, /*dest_offset=*/0);
+    } else {
+      std::memset(out_bitmap, 0xFF, output->buffers[0]->size());

Review comment:
       I guess if we wanted to support can_write_into_slices then we'd need SetBitsTo here.




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

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

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



[GitHub] [arrow] jorisvandenbossche commented on pull request #10412: ARROW-9430: [C++] Implement override_mask kernel

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


   High-level question: is there a reason that the replacement values are passed through an options struct, and not as third argument? (because that is an array of a different length? But eg for "take" the indices are not in an options struct)
   
   And if we can start bike-shedding about the name .. ;) For me, "override_mask" sounds like it would replace the validity mask of the array. So if we use "override" as verb, I would at least make it "overrride_with_mask" or so. But "replace" or "set_values" might also be possible verbs (and with the different variations of this kernel, it could eg be "replace_with_mask", "replace_with_indices", "replace_with_mapping")


-- 
This is an automated message from the 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 #10412: ARROW-9430: [C++] Implement replace_with_mask kernel

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


   Final comments from anyone?


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

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

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



[GitHub] [arrow] jorisvandenbossche commented on a change in pull request #10412: ARROW-9430: [C++] Implement replace_with_mask kernel

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



##########
File path: cpp/src/arrow/compute/api_vector.h
##########
@@ -157,6 +157,23 @@ Result<std::shared_ptr<ArrayData>> GetTakeIndices(
 
 }  // namespace internal
 
+/// \brief ReplaceWithMask replaces each value in the array corresponding
+/// to a true value in the mask with the next element from `options`.

Review comment:
       ```suggestion
   /// to a true value in the mask with the next element from `replacements`.
   ```

##########
File path: cpp/src/arrow/compute/kernels/vector_replace_test.cc
##########
@@ -0,0 +1,738 @@
+// 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 <gmock/gmock.h>
+#include <gtest/gtest.h>
+
+#include "arrow/compute/api_vector.h"
+#include "arrow/compute/kernels/test_util.h"
+#include "arrow/testing/gtest_common.h"
+#include "arrow/testing/gtest_util.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/key_value_metadata.h"
+#include "arrow/util/make_unique.h"
+
+namespace arrow {
+namespace compute {
+
+using arrow::internal::checked_pointer_cast;
+
+namespace {
+template <typename T>
+enable_if_parameter_free<T, std::shared_ptr<DataType>> default_type_instance() {
+  return TypeTraits<T>::type_singleton();
+}
+template <typename T>
+enable_if_time<T, std::shared_ptr<DataType>> default_type_instance() {
+  // Time32 requires second/milli, Time64 requires nano/micro
+  if (TypeTraits<T>::bytes_required(1) == 4) {
+    return std::make_shared<T>(TimeUnit::type::SECOND);
+  } else {
+    return std::make_shared<T>(TimeUnit::type::NANO);
+  }
+}
+template <typename T>
+enable_if_timestamp<T, std::shared_ptr<DataType>> default_type_instance() {
+  return std::make_shared<T>(TimeUnit::type::SECOND);
+}
+template <typename T>
+enable_if_decimal<T, std::shared_ptr<DataType>> default_type_instance() {
+  return std::make_shared<T>(5, 2);
+}
+template <typename T>
+enable_if_parameter_free<T, std::unique_ptr<typename TypeTraits<T>::BuilderType>>
+builder_instance() {
+  return arrow::internal::make_unique<typename TypeTraits<T>::BuilderType>();
+}
+template <typename T>
+enable_if_time<T, std::unique_ptr<typename TypeTraits<T>::BuilderType>>
+builder_instance() {
+  return arrow::internal::make_unique<typename TypeTraits<T>::BuilderType>(
+      default_type_instance<T>(), default_memory_pool());
+}
+template <typename T>
+enable_if_timestamp<T, std::unique_ptr<typename TypeTraits<T>::BuilderType>>
+builder_instance() {
+  return arrow::internal::make_unique<typename TypeTraits<T>::BuilderType>(
+      default_type_instance<T>(), default_memory_pool());
+}
+template <typename T>
+enable_if_t<std::is_signed<T>::value, T> max_int_value() {
+  return static_cast<T>(
+      std::min<double>(16384.0, static_cast<double>(std::numeric_limits<T>::max())));
+}
+template <typename T>
+enable_if_t<std::is_unsigned<T>::value, T> max_int_value() {
+  return static_cast<T>(
+      std::min<double>(16384.0, static_cast<double>(std::numeric_limits<T>::max())));
+}
+}  // namespace
+
+template <typename T>
+class TestReplaceKernel : public ::testing::Test {
+ protected:
+  virtual std::shared_ptr<DataType> type() = 0;
+
+  using ReplaceFunction = std::function<Result<Datum>(const Datum&, const Datum&,
+                                                      const Datum&, ExecContext*)>;
+
+  void SetUp() override { equal_options_ = equal_options_.nans_equal(true); }
+
+  Datum mask_scalar(bool value) { return Datum(std::make_shared<BooleanScalar>(value)); }
+
+  Datum null_mask_scalar() {
+    auto scalar = std::make_shared<BooleanScalar>(true);
+    scalar->is_valid = false;
+    return Datum(std::move(scalar));
+  }
+
+  Datum scalar(const std::string& json) { return ScalarFromJSON(type(), json); }
+
+  std::shared_ptr<Array> array(const std::string& value) {
+    return ArrayFromJSON(type(), value);
+  }
+
+  std::shared_ptr<Array> mask(const std::string& value) {
+    return ArrayFromJSON(boolean(), value);
+  }
+
+  Status AssertRaises(ReplaceFunction func, const std::shared_ptr<Array>& array,
+                      const Datum& mask, const std::shared_ptr<Array>& replacements) {
+    auto result = func(array, mask, replacements, nullptr);
+    EXPECT_FALSE(result.ok());
+    return result.status();
+  }
+
+  void Assert(ReplaceFunction func, const std::shared_ptr<Array>& array,
+              const Datum& mask, Datum replacements,
+              const std::shared_ptr<Array>& expected) {
+    SCOPED_TRACE("Replacements: " + (replacements.is_array()
+                                         ? replacements.make_array()->ToString()
+                                         : replacements.scalar()->ToString()));
+    SCOPED_TRACE("Mask: " + (mask.is_array() ? mask.make_array()->ToString()
+                                             : mask.scalar()->ToString()));
+    SCOPED_TRACE("Array: " + array->ToString());
+
+    ASSERT_OK_AND_ASSIGN(auto actual, func(array, mask, replacements, nullptr));
+    ASSERT_TRUE(actual.is_array());
+    ASSERT_OK(actual.make_array()->ValidateFull());
+
+    AssertArraysApproxEqual(*expected, *actual.make_array(), /*verbose=*/true,
+                            equal_options_);
+  }
+
+  std::shared_ptr<Array> NaiveImpl(
+      const typename TypeTraits<T>::ArrayType& array, const BooleanArray& mask,
+      const typename TypeTraits<T>::ArrayType& replacements) {
+    auto length = array.length();
+    auto builder = builder_instance<T>();
+    int64_t replacement_offset = 0;
+    for (int64_t i = 0; i < length; ++i) {
+      if (mask.IsValid(i)) {
+        if (mask.Value(i)) {
+          if (replacements.IsValid(replacement_offset)) {
+            ARROW_EXPECT_OK(builder->Append(replacements.Value(replacement_offset++)));
+          } else {
+            ARROW_EXPECT_OK(builder->AppendNull());
+            replacement_offset++;
+          }
+        } else {
+          if (array.IsValid(i)) {
+            ARROW_EXPECT_OK(builder->Append(array.Value(i)));
+          } else {
+            ARROW_EXPECT_OK(builder->AppendNull());
+          }
+        }
+      } else {
+        ARROW_EXPECT_OK(builder->AppendNull());
+      }
+    }
+    EXPECT_OK_AND_ASSIGN(auto expected, builder->Finish());
+    return expected;
+  }
+
+  EqualOptions equal_options_ = EqualOptions::Defaults();
+};
+
+template <typename T>
+class TestReplaceNumeric : public TestReplaceKernel<T> {
+ protected:
+  std::shared_ptr<DataType> type() override { return default_type_instance<T>(); }
+};
+
+class TestReplaceBoolean : public TestReplaceKernel<BooleanType> {
+ protected:
+  std::shared_ptr<DataType> type() override {
+    return TypeTraits<BooleanType>::type_singleton();
+  }
+};
+
+class TestReplaceFixedSizeBinary : public TestReplaceKernel<FixedSizeBinaryType> {
+ protected:
+  std::shared_ptr<DataType> type() override { return fixed_size_binary(3); }
+};
+
+template <typename T>
+class TestReplaceDecimal : public TestReplaceKernel<T> {
+ protected:
+  std::shared_ptr<DataType> type() override { return default_type_instance<T>(); }
+};
+
+class TestReplaceDayTimeInterval : public TestReplaceKernel<DayTimeIntervalType> {
+ protected:
+  std::shared_ptr<DataType> type() override {
+    return TypeTraits<DayTimeIntervalType>::type_singleton();
+  }
+};
+
+template <typename T>
+class TestReplaceBinary : public TestReplaceKernel<T> {
+ protected:
+  std::shared_ptr<DataType> type() override { return default_type_instance<T>(); }
+};
+
+using NumericBasedTypes =
+    ::testing::Types<UInt8Type, UInt16Type, UInt32Type, UInt64Type, Int8Type, Int16Type,
+                     Int32Type, Int64Type, FloatType, DoubleType, Date32Type, Date64Type,
+                     Time32Type, Time64Type, TimestampType, MonthIntervalType>;
+
+TYPED_TEST_SUITE(TestReplaceNumeric, NumericBasedTypes);
+TYPED_TEST_SUITE(TestReplaceDecimal, DecimalArrowTypes);
+TYPED_TEST_SUITE(TestReplaceBinary, BinaryTypes);
+
+TYPED_TEST(TestReplaceNumeric, ReplaceWithMask) {
+  this->Assert(ReplaceWithMask, this->array("[]"), this->mask_scalar(false),
+               this->array("[]"), this->array("[]"));
+  this->Assert(ReplaceWithMask, this->array("[]"), this->mask_scalar(true),
+               this->array("[]"), this->array("[]"));
+  this->Assert(ReplaceWithMask, this->array("[]"), this->null_mask_scalar(),
+               this->array("[]"), this->array("[]"));
+
+  this->Assert(ReplaceWithMask, this->array("[1]"), this->mask_scalar(false),
+               this->array("[]"), this->array("[1]"));
+  this->Assert(ReplaceWithMask, this->array("[1]"), this->mask_scalar(true),
+               this->array("[0]"), this->array("[0]"));
+  this->Assert(ReplaceWithMask, this->array("[1]"), this->null_mask_scalar(),
+               this->array("[]"), this->array("[null]"));
+
+  this->Assert(ReplaceWithMask, this->array("[0, 0]"), this->mask_scalar(false),
+               this->scalar("1"), this->array("[0, 0]"));
+  this->Assert(ReplaceWithMask, this->array("[0, 0]"), this->mask_scalar(true),
+               this->scalar("1"), this->array("[1, 1]"));
+  this->Assert(ReplaceWithMask, this->array("[0, 0]"), this->mask_scalar(true),
+               this->scalar("null"), this->array("[null, null]"));
+
+  this->Assert(ReplaceWithMask, this->array("[]"), this->mask("[]"), this->array("[]"),
+               this->array("[]"));
+  this->Assert(ReplaceWithMask, this->array("[0, 1, 2, 3]"),
+               this->mask("[false, false, false, false]"), this->array("[]"),
+               this->array("[0, 1, 2, 3]"));
+  this->Assert(ReplaceWithMask, this->array("[0, 1, 2, 3]"),
+               this->mask("[true, true, true, true]"), this->array("[10, 11, 12, 13]"),
+               this->array("[10, 11, 12, 13]"));
+  this->Assert(ReplaceWithMask, this->array("[0, 1, 2, 3]"),
+               this->mask("[null, null, null, null]"), this->array("[]"),
+               this->array("[null, null, null, null]"));
+  this->Assert(ReplaceWithMask, this->array("[0, 1, 2, null]"),
+               this->mask("[false, false, false, false]"), this->array("[]"),
+               this->array("[0, 1, 2, null]"));
+  this->Assert(ReplaceWithMask, this->array("[0, 1, 2, null]"),
+               this->mask("[true, true, true, true]"), this->array("[10, 11, 12, 13]"),
+               this->array("[10, 11, 12, 13]"));
+  this->Assert(ReplaceWithMask, this->array("[0, 1, 2, null]"),
+               this->mask("[null, null, null, null]"), this->array("[]"),
+               this->array("[null, null, null, null]"));
+  this->Assert(ReplaceWithMask, this->array("[0, 1, 2, 3, 4, 5]"),
+               this->mask("[true, true, false, false, null, null]"),
+               this->array("[10, null]"), this->array("[10, null, 2, 3, null, null]"));
+  this->Assert(ReplaceWithMask, this->array("[null, null, null, null, null, null]"),
+               this->mask("[true, true, false, false, null, null]"),
+               this->array("[10, null]"),
+               this->array("[10, null, null, null, null, null]"));
+
+  this->Assert(ReplaceWithMask, this->array("[]"), this->mask("[]"), this->scalar("1"),
+               this->array("[]"));
+  this->Assert(ReplaceWithMask, this->array("[0, 1]"), this->mask("[true, true]"),
+               this->scalar("10"), this->array("[10, 10]"));
+  this->Assert(ReplaceWithMask, this->array("[0, 1]"), this->mask("[true, true]"),
+               this->scalar("null"), this->array("[null, null]"));
+  this->Assert(ReplaceWithMask, this->array("[0, 1, 2]"),
+               this->mask("[true, false, null]"), this->scalar("10"),
+               this->array("[10, 1, null]"));
+}
+
+TYPED_TEST(TestReplaceNumeric, ReplaceWithMaskRandom) {
+  using ArrayType = typename TypeTraits<TypeParam>::ArrayType;
+  using CType = typename TypeTraits<TypeParam>::CType;
+  auto ty = this->type();
+
+  random::RandomArrayGenerator rand(/*seed=*/0);
+  const int64_t length = 1023;
+  // Clamp the range because date/time types don't print well with extreme values
+  std::vector<std::string> values = {"0.01", "0"};
+  values.push_back(std::to_string(max_int_value<CType>()));
+  auto options = key_value_metadata({"null_probability", "min", "max"}, values);
+  auto array =
+      checked_pointer_cast<ArrayType>(rand.ArrayOf(*field("a", ty, options), length));
+  auto mask = checked_pointer_cast<BooleanArray>(
+      rand.ArrayOf(boolean(), length, /*null_probability=*/0.01));
+  const int64_t num_replacements = std::count_if(
+      mask->begin(), mask->end(),
+      [](util::optional<bool> value) { return value.has_value() && *value; });
+  auto replacements = checked_pointer_cast<ArrayType>(
+      rand.ArrayOf(*field("a", ty, options), num_replacements));
+  auto expected = this->NaiveImpl(*array, *mask, *replacements);
+
+  this->Assert(ReplaceWithMask, array, mask, replacements, expected);
+  for (int64_t slice = 1; slice <= 16; slice++) {
+    auto sliced_array = checked_pointer_cast<ArrayType>(array->Slice(slice, 15));
+    auto sliced_mask = checked_pointer_cast<BooleanArray>(mask->Slice(slice, 15));
+    auto new_expected = this->NaiveImpl(*sliced_array, *sliced_mask, *replacements);
+    this->Assert(ReplaceWithMask, sliced_array, sliced_mask, replacements, new_expected);
+  }
+}
+
+TYPED_TEST(TestReplaceNumeric, ReplaceWithMaskErrors) {
+  EXPECT_RAISES_WITH_MESSAGE_THAT(
+      Invalid,
+      ::testing::HasSubstr("Replacement array must be of appropriate length (expected 1 "
+                           "items but got 2 items)"),
+      this->AssertRaises(ReplaceWithMask, this->array("[1]"), this->mask_scalar(true),
+                         this->array("[0, 1]")));
+  EXPECT_RAISES_WITH_MESSAGE_THAT(
+      Invalid,
+      ::testing::HasSubstr("Replacement array must be of appropriate length (expected 2 "
+                           "items but got 1 items)"),
+      this->AssertRaises(ReplaceWithMask, this->array("[1, 2]"),
+                         this->mask("[true, true]"), this->array("[0]")));
+  EXPECT_RAISES_WITH_MESSAGE_THAT(
+      Invalid,
+      ::testing::HasSubstr("Replacement array must be of appropriate length (expected 1 "
+                           "items but got 0 items)"),
+      this->AssertRaises(ReplaceWithMask, this->array("[1, 2]"),
+                         this->mask("[true, null]"), this->array("[]")));
+  EXPECT_RAISES_WITH_MESSAGE_THAT(
+      Invalid,
+      ::testing::HasSubstr("Mask must be of same length as array (expected 2 "
+                           "items but got 0 items)"),
+      this->AssertRaises(ReplaceWithMask, this->array("[1, 2]"), this->mask("[]"),
+                         this->array("[]")));
+}
+
+TEST_F(TestReplaceBoolean, ReplaceWithMask) {
+  this->Assert(ReplaceWithMask, this->array("[]"), this->mask_scalar(false),
+               this->array("[]"), this->array("[]"));
+  this->Assert(ReplaceWithMask, this->array("[]"), this->mask_scalar(true),
+               this->array("[]"), this->array("[]"));
+  this->Assert(ReplaceWithMask, this->array("[]"), this->null_mask_scalar(),
+               this->array("[]"), this->array("[]"));
+
+  this->Assert(ReplaceWithMask, this->array("[true]"), this->mask_scalar(false),
+               this->array("[]"), this->array("[true]"));
+  this->Assert(ReplaceWithMask, this->array("[true]"), this->mask_scalar(true),
+               this->array("[false]"), this->array("[false]"));
+  this->Assert(ReplaceWithMask, this->array("[true]"), this->null_mask_scalar(),
+               this->array("[]"), this->array("[null]"));
+
+  this->Assert(ReplaceWithMask, this->array("[false, false]"), this->mask_scalar(false),
+               this->scalar("true"), this->array("[false, false]"));
+  this->Assert(ReplaceWithMask, this->array("[false, false]"), this->mask_scalar(true),
+               this->scalar("true"), this->array("[true, true]"));
+  this->Assert(ReplaceWithMask, this->array("[false, false]"), this->mask_scalar(true),
+               this->scalar("null"), this->array("[null, null]"));
+
+  this->Assert(ReplaceWithMask, this->array("[]"), this->mask("[]"), this->array("[]"),
+               this->array("[]"));
+  this->Assert(ReplaceWithMask, this->array("[true, true, true, true]"),
+               this->mask("[false, false, false, false]"), this->array("[]"),
+               this->array("[true, true, true, true]"));
+  this->Assert(ReplaceWithMask, this->array("[true, true, true, true]"),
+               this->mask("[true, true, true, true]"),
+               this->array("[false, false, false, false]"),
+               this->array("[false, false, false, false]"));
+  this->Assert(ReplaceWithMask, this->array("[true, true, true, true]"),
+               this->mask("[null, null, null, null]"), this->array("[]"),
+               this->array("[null, null, null, null]"));
+  this->Assert(ReplaceWithMask, this->array("[true, true, true, null]"),
+               this->mask("[false, false, false, false]"), this->array("[]"),
+               this->array("[true, true, true, null]"));
+  this->Assert(ReplaceWithMask, this->array("[true, true, true, null]"),
+               this->mask("[true, true, true, true]"),
+               this->array("[false, false, false, false]"),
+               this->array("[false, false, false, false]"));
+  this->Assert(ReplaceWithMask, this->array("[true, true, true, null]"),
+               this->mask("[null, null, null, null]"), this->array("[]"),
+               this->array("[null, null, null, null]"));
+  this->Assert(ReplaceWithMask, this->array("[true, true, true, true, true, true]"),
+               this->mask("[true, true, false, false, null, null]"),
+               this->array("[false, null]"),
+               this->array("[false, null, true, true, null, null]"));
+  this->Assert(ReplaceWithMask, this->array("[null, null, null, null, null, null]"),
+               this->mask("[true, true, false, false, null, null]"),
+               this->array("[false, null]"),
+               this->array("[false, null, null, null, null, null]"));
+
+  this->Assert(ReplaceWithMask, this->array("[]"), this->mask("[]"), this->scalar("true"),
+               this->array("[]"));
+  this->Assert(ReplaceWithMask, this->array("[false, false]"), this->mask("[true, true]"),
+               this->scalar("true"), this->array("[true, true]"));
+  this->Assert(ReplaceWithMask, this->array("[false, false]"), this->mask("[true, true]"),
+               this->scalar("null"), this->array("[null, null]"));
+  this->Assert(ReplaceWithMask, this->array("[false, false, false]"),
+               this->mask("[true, false, null]"), this->scalar("true"),
+               this->array("[true, false, null]"));
+}
+
+TEST_F(TestReplaceBoolean, ReplaceWithMaskErrors) {
+  EXPECT_RAISES_WITH_MESSAGE_THAT(
+      Invalid,
+      ::testing::HasSubstr("Replacement array must be of appropriate length (expected 1 "
+                           "items but got 2 items)"),
+      this->AssertRaises(ReplaceWithMask, this->array("[true]"), this->mask_scalar(true),
+                         this->array("[false, false]")));
+  EXPECT_RAISES_WITH_MESSAGE_THAT(
+      Invalid,
+      ::testing::HasSubstr("Replacement array must be of appropriate length (expected 2 "
+                           "items but got 1 items)"),
+      this->AssertRaises(ReplaceWithMask, this->array("[true, true]"),
+                         this->mask("[true, true]"), this->array("[false]")));
+  EXPECT_RAISES_WITH_MESSAGE_THAT(
+      Invalid,
+      ::testing::HasSubstr("Replacement array must be of appropriate length (expected 1 "
+                           "items but got 0 items)"),
+      this->AssertRaises(ReplaceWithMask, this->array("[true, true]"),
+                         this->mask("[true, null]"), this->array("[]")));
+  EXPECT_RAISES_WITH_MESSAGE_THAT(
+      Invalid,
+      ::testing::HasSubstr("Mask must be of same length as array (expected 2 "
+                           "items but got 0 items)"),
+      this->AssertRaises(ReplaceWithMask, this->array("[true, true]"), this->mask("[]"),
+                         this->array("[]")));
+}
+
+TEST_F(TestReplaceFixedSizeBinary, ReplaceWithMask) {
+  this->Assert(ReplaceWithMask, this->array("[]"), this->mask_scalar(false),
+               this->array("[]"), this->array("[]"));
+  this->Assert(ReplaceWithMask, this->array("[]"), this->mask_scalar(true),
+               this->array("[]"), this->array("[]"));
+  this->Assert(ReplaceWithMask, this->array("[]"), this->null_mask_scalar(),
+               this->array("[]"), this->array("[]"));
+
+  this->Assert(ReplaceWithMask, this->array(R"(["foo"])"), this->mask_scalar(false),
+               this->array("[]"), this->array(R"(["foo"])"));
+  this->Assert(ReplaceWithMask, this->array(R"(["foo"])"), this->mask_scalar(true),
+               this->array(R"(["bar"])"), this->array(R"(["bar"])"));
+  this->Assert(ReplaceWithMask, this->array(R"(["foo"])"), this->null_mask_scalar(),
+               this->array("[]"), this->array("[null]"));
+
+  this->Assert(ReplaceWithMask, this->array(R"(["foo", "bar"])"),
+               this->mask_scalar(false), this->scalar(R"("baz")"),
+               this->array(R"(["foo", "bar"])"));
+  this->Assert(ReplaceWithMask, this->array(R"(["foo", "bar"])"), this->mask_scalar(true),
+               this->scalar(R"("baz")"), this->array(R"(["baz", "baz"])"));
+  this->Assert(ReplaceWithMask, this->array(R"(["foo", "bar"])"), this->mask_scalar(true),
+               this->scalar("null"), this->array(R"([null, null])"));
+
+  this->Assert(ReplaceWithMask, this->array("[]"), this->mask("[]"), this->array("[]"),
+               this->array("[]"));
+  this->Assert(ReplaceWithMask, this->array(R"(["aaa", "bbb", "ccc", "ddd"])"),
+               this->mask("[false, false, false, false]"), this->array("[]"),
+               this->array(R"(["aaa", "bbb", "ccc", "ddd"])"));
+  this->Assert(ReplaceWithMask, this->array(R"(["aaa", "bbb", "ccc", "ddd"])"),
+               this->mask("[true, true, true, true]"),
+               this->array(R"(["eee", "fff", "ggg", "hhh"])"),
+               this->array(R"(["eee", "fff", "ggg", "hhh"])"));
+  this->Assert(ReplaceWithMask, this->array(R"(["aaa", "bbb", "ccc", "ddd"])"),
+               this->mask("[null, null, null, null]"), this->array("[]"),
+               this->array(R"([null, null, null, null])"));
+  this->Assert(ReplaceWithMask, this->array(R"(["aaa", "bbb", "ccc", null])"),
+               this->mask("[false, false, false, false]"), this->array("[]"),
+               this->array(R"(["aaa", "bbb", "ccc", null])"));
+  this->Assert(ReplaceWithMask, this->array(R"(["aaa", "bbb", "ccc", null])"),
+               this->mask("[true, true, true, true]"),
+               this->array(R"(["eee", "fff", "ggg", "hhh"])"),
+               this->array(R"(["eee", "fff", "ggg", "hhh"])"));
+  this->Assert(ReplaceWithMask, this->array(R"(["aaa", "bbb", "ccc", null])"),
+               this->mask("[null, null, null, null]"), this->array("[]"),
+               this->array(R"([null, null, null, null])"));
+  this->Assert(ReplaceWithMask,
+               this->array(R"(["aaa", "bbb", "ccc", "ddd", "eee", "fff"])"),
+               this->mask("[true, true, false, false, null, null]"),
+               this->array(R"(["ggg", null])"),
+               this->array(R"(["ggg", null, "ccc", "ddd", null, null])"));
+  this->Assert(ReplaceWithMask, this->array(R"([null, null, null, null, null, null])"),
+               this->mask("[true, true, false, false, null, null]"),
+               this->array(R"(["aaa", null])"),
+               this->array(R"(["aaa", null, null, null, null, null])"));
+
+  this->Assert(ReplaceWithMask, this->array("[]"), this->mask("[]"),
+               this->scalar(R"("zzz")"), this->array("[]"));
+  this->Assert(ReplaceWithMask, this->array(R"(["aaa", "bbb"])"),
+               this->mask("[true, true]"), this->scalar(R"("zzz")"),
+               this->array(R"(["zzz", "zzz"])"));
+  this->Assert(ReplaceWithMask, this->array(R"(["aaa", "bbb"])"),
+               this->mask("[true, true]"), this->scalar("null"),
+               this->array("[null, null]"));
+  this->Assert(ReplaceWithMask, this->array(R"(["aaa", "bbb", "ccc"])"),
+               this->mask("[true, false, null]"), this->scalar(R"("zzz")"),
+               this->array(R"(["zzz", "bbb", null])"));
+}
+
+TEST_F(TestReplaceFixedSizeBinary, ReplaceWithMaskErrors) {
+  EXPECT_RAISES_WITH_MESSAGE_THAT(
+      Invalid,
+      ::testing::AllOf(
+          ::testing::HasSubstr("Replacements must be of same type (expected "),
+          ::testing::HasSubstr(this->type()->ToString()),
+          ::testing::HasSubstr("but got fixed_size_binary[2]")),
+      this->AssertRaises(ReplaceWithMask, this->array("[]"), this->mask_scalar(true),
+                         ArrayFromJSON(fixed_size_binary(2), "[]")));
+}
+
+TYPED_TEST(TestReplaceDecimal, ReplaceWithMask) {
+  this->Assert(ReplaceWithMask, this->array("[]"), this->mask_scalar(false),
+               this->array("[]"), this->array("[]"));
+  this->Assert(ReplaceWithMask, this->array("[]"), this->mask_scalar(true),
+               this->array("[]"), this->array("[]"));
+  this->Assert(ReplaceWithMask, this->array("[]"), this->null_mask_scalar(),
+               this->array("[]"), this->array("[]"));
+
+  this->Assert(ReplaceWithMask, this->array(R"(["1.00"])"), this->mask_scalar(false),
+               this->array("[]"), this->array(R"(["1.00"])"));
+  this->Assert(ReplaceWithMask, this->array(R"(["1.00"])"), this->mask_scalar(true),
+               this->array(R"(["0.00"])"), this->array(R"(["0.00"])"));
+  this->Assert(ReplaceWithMask, this->array(R"(["1.00"])"), this->null_mask_scalar(),
+               this->array("[]"), this->array("[null]"));
+
+  this->Assert(ReplaceWithMask, this->array(R"(["0.00", "0.00"])"),
+               this->mask_scalar(false), this->scalar(R"("1.00")"),
+               this->array(R"(["0.00", "0.00"])"));
+  this->Assert(ReplaceWithMask, this->array(R"(["0.00", "0.00"])"),
+               this->mask_scalar(true), this->scalar(R"("1.00")"),
+               this->array(R"(["1.00", "1.00"])"));
+  this->Assert(ReplaceWithMask, this->array(R"(["0.00", "0.00"])"),
+               this->mask_scalar(true), this->scalar("null"),
+               this->array("[null, null]"));
+
+  this->Assert(ReplaceWithMask, this->array("[]"), this->mask("[]"), this->array("[]"),
+               this->array("[]"));
+  this->Assert(ReplaceWithMask, this->array(R"(["0.00", "1.00", "2.00", "3.00"])"),
+               this->mask("[false, false, false, false]"), this->array("[]"),
+               this->array(R"(["0.00", "1.00", "2.00", "3.00"])"));
+  this->Assert(ReplaceWithMask, this->array(R"(["0.00", "1.00", "2.00", "3.00"])"),
+               this->mask("[true, true, true, true]"),
+               this->array(R"(["10.00", "11.00", "12.00", "13.00"])"),
+               this->array(R"(["10.00", "11.00", "12.00", "13.00"])"));
+  this->Assert(ReplaceWithMask, this->array(R"(["0.00", "1.00", "2.00", "3.00"])"),
+               this->mask("[null, null, null, null]"), this->array("[]"),
+               this->array("[null, null, null, null]"));
+  this->Assert(ReplaceWithMask, this->array(R"(["0.00", "1.00", "2.00", null])"),
+               this->mask("[false, false, false, false]"), this->array("[]"),
+               this->array(R"(["0.00", "1.00", "2.00", null])"));
+  this->Assert(ReplaceWithMask, this->array(R"(["0.00", "1.00", "2.00", null])"),
+               this->mask("[true, true, true, true]"),
+               this->array(R"(["10.00", "11.00", "12.00", "13.00"])"),
+               this->array(R"(["10.00", "11.00", "12.00", "13.00"])"));
+  this->Assert(ReplaceWithMask, this->array(R"(["0.00", "1.00", "2.00", null])"),
+               this->mask("[null, null, null, null]"), this->array("[]"),
+               this->array("[null, null, null, null]"));
+  this->Assert(ReplaceWithMask,
+               this->array(R"(["0.00", "1.00", "2.00", "3.00", "4.00", "5.00"])"),
+               this->mask("[true, true, false, false, null, null]"),
+               this->array(R"(["10.00", null])"),
+               this->array(R"(["10.00", null, "2.00", "3.00", null, null])"));
+  this->Assert(ReplaceWithMask, this->array("[null, null, null, null, null, null]"),
+               this->mask("[true, true, false, false, null, null]"),
+               this->array(R"(["10.00", null])"),
+               this->array(R"(["10.00", null, null, null, null, null])"));
+
+  this->Assert(ReplaceWithMask, this->array("[]"), this->mask("[]"),
+               this->scalar(R"("1.00")"), this->array("[]"));
+  this->Assert(ReplaceWithMask, this->array(R"(["0.00", "1.00"])"),
+               this->mask("[true, true]"), this->scalar(R"("10.00")"),
+               this->array(R"(["10.00", "10.00"])"));
+  this->Assert(ReplaceWithMask, this->array(R"(["0.00", "1.00"])"),
+               this->mask("[true, true]"), this->scalar("null"),
+               this->array("[null, null]"));
+  this->Assert(ReplaceWithMask, this->array(R"(["0.00", "1.00", "2.00"])"),
+               this->mask("[true, false, null]"), this->scalar(R"("10.00")"),
+               this->array(R"(["10.00", "1.00", null])"));
+}
+
+TEST_F(TestReplaceDayTimeInterval, ReplaceWithMask) {
+  this->Assert(ReplaceWithMask, this->array("[]"), this->mask_scalar(false),
+               this->array("[]"), this->array("[]"));
+  this->Assert(ReplaceWithMask, this->array("[]"), this->mask_scalar(true),
+               this->array("[]"), this->array("[]"));
+  this->Assert(ReplaceWithMask, this->array("[]"), this->null_mask_scalar(),
+               this->array("[]"), this->array("[]"));
+
+  this->Assert(ReplaceWithMask, this->array("[[1, 2]]"), this->mask_scalar(false),
+               this->array("[]"), this->array("[[1, 2]]"));
+  this->Assert(ReplaceWithMask, this->array("[[1, 2]]"), this->mask_scalar(true),
+               this->array("[[3, 4]]"), this->array("[[3, 4]]"));
+  this->Assert(ReplaceWithMask, this->array("[[1, 2]]"), this->null_mask_scalar(),
+               this->array("[]"), this->array("[null]"));
+
+  this->Assert(ReplaceWithMask, this->array("[[1, 2], [3, 4]]"), this->mask_scalar(false),
+               this->scalar("[7, 8]"), this->array("[[1, 2], [3, 4]]"));
+  this->Assert(ReplaceWithMask, this->array("[[1, 2], [3, 4]]"), this->mask_scalar(true),
+               this->scalar("[7, 8]"), this->array("[[7, 8], [7, 8]]"));
+  this->Assert(ReplaceWithMask, this->array("[[1, 2], [3, 4]]"), this->mask_scalar(true),
+               this->scalar("null"), this->array("[null, null]"));
+
+  this->Assert(ReplaceWithMask, this->array("[]"), this->mask("[]"), this->array("[]"),
+               this->array("[]"));
+  this->Assert(ReplaceWithMask, this->array("[[1, 2], [1, 2], [1, 2], [1, 2]]"),
+               this->mask("[false, false, false, false]"), this->array("[]"),
+               this->array("[[1, 2], [1, 2], [1, 2], [1, 2]]"));
+  this->Assert(ReplaceWithMask, this->array("[[1, 2], [1, 2], [1, 2], [1, 2]]"),
+               this->mask("[true, true, true, true]"),
+               this->array("[[3, 4], [3, 4], [3, 4], [3, 4]]"),
+               this->array("[[3, 4], [3, 4], [3, 4], [3, 4]]"));
+  this->Assert(ReplaceWithMask, this->array("[[1, 2], [1, 2], [1, 2], [1, 2]]"),
+               this->mask("[null, null, null, null]"), this->array("[]"),
+               this->array("[null, null, null, null]"));
+  this->Assert(ReplaceWithMask, this->array("[[1, 2], [1, 2], [1, 2], null]"),
+               this->mask("[false, false, false, false]"), this->array("[]"),
+               this->array("[[1, 2], [1, 2], [1, 2], null]"));
+  this->Assert(ReplaceWithMask, this->array("[[1, 2], [1, 2], [1, 2], null]"),
+               this->mask("[true, true, true, true]"),
+               this->array("[[3, 4], [3, 4], [3, 4], [3, 4]]"),
+               this->array("[[3, 4], [3, 4], [3, 4], [3, 4]]"));
+  this->Assert(ReplaceWithMask, this->array("[[1, 2], [1, 2], [1, 2], null]"),
+               this->mask("[null, null, null, null]"), this->array("[]"),
+               this->array("[null, null, null, null]"));
+  this->Assert(
+      ReplaceWithMask, this->array("[[1, 2], [1, 2], [1, 2], [1, 2], [1, 2], [1, 2]]"),
+      this->mask("[true, true, false, false, null, null]"), this->array("[[3, 4], null]"),
+      this->array("[[3, 4], null, [1, 2], [1, 2], null, null]"));
+  this->Assert(ReplaceWithMask, this->array("[null, null, null, null, null, null]"),
+               this->mask("[true, true, false, false, null, null]"),
+               this->array("[[3, 4], null]"),
+               this->array("[[3, 4], null, null, null, null, null]"));
+
+  this->Assert(ReplaceWithMask, this->array("[]"), this->mask("[]"),
+               this->scalar("[7, 8]"), this->array("[]"));
+  this->Assert(ReplaceWithMask, this->array("[[1, 2], [3, 4]]"),
+               this->mask("[true, true]"), this->scalar("[7, 8]"),
+               this->array("[[7, 8], [7, 8]]"));
+  this->Assert(ReplaceWithMask, this->array("[[1, 2], [3, 4]]"),
+               this->mask("[true, true]"), this->scalar("null"),
+               this->array("[null, null]"));
+  this->Assert(ReplaceWithMask, this->array("[[1, 2], [3, 4], [5, 6]]"),
+               this->mask("[true, false, null]"), this->scalar("[7, 8]"),
+               this->array("[[7, 8], [3, 4], null]"));
+}
+
+TYPED_TEST(TestReplaceBinary, ReplaceWithMask) {

Review comment:
       If I understand correctly, this test is for the variable sized binary/string types? But the values in the test seem to only contain fixed length strings (probably copied from the TestReplaceFixedSizeBinary). Maybe introduce a few strings with varying length?

##########
File path: cpp/src/arrow/compute/kernels/vector_replace.cc
##########
@@ -0,0 +1,494 @@
+// 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_scalar.h"
+#include "arrow/compute/kernels/common.h"
+#include "arrow/util/bitmap_ops.h"
+
+namespace arrow {
+namespace compute {
+namespace internal {
+
+namespace {
+
+Status ReplacementArrayTooShort(int64_t expected, int64_t actual) {
+  return Status::Invalid("Replacement array must be of appropriate length (expected ",
+                         expected, " items but got ", actual, " items)");
+}
+
+// Helper to implement replace_with kernel with scalar mask for fixed-width types,
+// using callbacks to handle both bool and byte-sized types
+Status ReplaceWithScalarMask(KernelContext* ctx, const ArrayData& array,
+                             const BooleanScalar& mask, const Datum& replacements,
+                             ArrayData* output) {
+  if (!mask.is_valid) {
+    // Output = null
+    ARROW_ASSIGN_OR_RAISE(auto array,
+                          MakeArrayOfNull(array.type, array.length, ctx->memory_pool()));
+    *output = *array->data();
+    return Status::OK();
+  }
+  if (mask.value) {
+    // Output = replacement
+    if (replacements.is_scalar()) {
+      ARROW_ASSIGN_OR_RAISE(
+          auto replacement_array,
+          MakeArrayFromScalar(*replacements.scalar(), array.length, ctx->memory_pool()));
+      *output = *replacement_array->data();
+    } else {
+      auto replacement_array = replacements.array();
+      if (replacement_array->length != array.length) {
+        return ReplacementArrayTooShort(array.length, replacement_array->length);
+      }
+      *output = *replacement_array;
+    }
+  } else {
+    // Output = input
+    *output = array;
+  }
+  return Status::OK();
+}
+
+// Helper to implement replace_with kernel with array mask for fixed-width types,
+// using callbacks to handle both bool and byte-sized types and to handle
+// scalar and array replacements
+template <typename Functor>
+Status ReplaceWithArrayMask(KernelContext* ctx, const ArrayData& array,
+                            const ArrayData& mask, const Datum& replacements,
+                            ArrayData* output) {
+  ARROW_ASSIGN_OR_RAISE(output->buffers[1],
+                        Functor::AllocateData(ctx, *array.type, array.length));
+
+  uint8_t* out_bitmap = nullptr;
+  uint8_t* out_values = output->buffers[1]->mutable_data();
+  const uint8_t* mask_bitmap = mask.MayHaveNulls() ? mask.buffers[0]->data() : nullptr;
+  const uint8_t* mask_values = mask.buffers[1]->data();
+  bool replacements_bitmap;
+  int64_t replacements_length;
+  if (replacements.is_array()) {
+    replacements_bitmap = replacements.array()->MayHaveNulls();
+    replacements_length = replacements.array()->length;
+  } else {
+    replacements_bitmap = !replacements.scalar()->is_valid;
+    replacements_length = std::numeric_limits<int64_t>::max();
+  }
+  if (array.MayHaveNulls() || mask.MayHaveNulls() || replacements_bitmap) {
+    ARROW_ASSIGN_OR_RAISE(output->buffers[0], ctx->AllocateBitmap(array.length));
+    out_bitmap = output->buffers[0]->mutable_data();
+    output->null_count = -1;
+    if (array.MayHaveNulls()) {
+      arrow::internal::CopyBitmap(array.buffers[0]->data(), array.offset, array.length,
+                                  out_bitmap, /*dest_offset=*/0);
+    } else {
+      std::memset(out_bitmap, 0xFF, output->buffers[0]->size());
+    }
+  } else {
+    output->null_count = 0;
+  }
+  auto copy_bitmap = [&](int64_t out_offset, int64_t in_offset, int64_t length) {
+    DCHECK(out_bitmap);
+    if (replacements.is_array()) {
+      const auto& in_data = *replacements.array();
+      const auto in_bitmap = in_data.GetValues<uint8_t>(0, /*absolute_offset=*/0);
+      arrow::internal::CopyBitmap(in_bitmap, in_data.offset + in_offset, length,
+                                  out_bitmap, out_offset);
+    } else {
+      BitUtil::SetBitsTo(out_bitmap, out_offset, length, !replacements_bitmap);
+    }
+  };
+
+  Functor::CopyData(*array.type, out_values, /*out_offset=*/0, array, /*in_offset=*/0,
+                    array.length);
+  arrow::internal::BitBlockCounter value_counter(mask_values, mask.offset, mask.length);
+  arrow::internal::OptionalBitBlockCounter valid_counter(mask_bitmap, mask.offset,
+                                                         mask.length);
+  int64_t out_offset = 0;
+  int64_t replacements_offset = 0;
+  while (out_offset < array.length) {
+    BitBlockCount value_block = value_counter.NextWord();
+    BitBlockCount valid_block = valid_counter.NextWord();
+    DCHECK_EQ(value_block.length, valid_block.length);
+    if (value_block.AllSet() && valid_block.AllSet()) {
+      // Copy from replacement array
+      if (replacements_offset + valid_block.length > replacements_length) {
+        return ReplacementArrayTooShort(replacements_offset + valid_block.length,
+                                        replacements_length);
+      }
+      Functor::CopyData(*array.type, out_values, out_offset, replacements,
+                        replacements_offset, valid_block.length);
+      if (replacements_bitmap) {
+        copy_bitmap(out_offset, replacements_offset, valid_block.length);
+      } else if (!replacements_bitmap && out_bitmap) {
+        BitUtil::SetBitsTo(out_bitmap, out_offset, valid_block.length, true);
+      }
+      replacements_offset += valid_block.length;
+    } else if (value_block.NoneSet() && valid_block.AllSet()) {
+      // Do nothing
+    } else if (valid_block.NoneSet()) {
+      DCHECK(out_bitmap);
+      BitUtil::SetBitsTo(out_bitmap, out_offset, valid_block.length, false);
+    } else {
+      for (int64_t i = 0; i < valid_block.length; ++i) {
+        if (BitUtil::GetBit(mask_values, out_offset + mask.offset + i) &&
+            (!mask_bitmap ||
+             BitUtil::GetBit(mask_bitmap, out_offset + mask.offset + i))) {
+          if (replacements_offset >= replacements_length) {
+            return ReplacementArrayTooShort(replacements_offset + 1, replacements_length);
+          }
+          Functor::CopyData(*array.type, out_values, out_offset + i, replacements,
+                            replacements_offset,
+                            /*length=*/1);
+          if (replacements_bitmap) {
+            copy_bitmap(out_offset + i, replacements_offset, 1);
+          }
+          replacements_offset++;
+        }
+      }
+    }
+    out_offset += valid_block.length;
+  }
+
+  if (mask.MayHaveNulls()) {
+    arrow::internal::BitmapAnd(out_bitmap, /*left_offset=*/0, mask.buffers[0]->data(),
+                               mask.offset, array.length,
+                               /*out_offset=*/0, out_bitmap);
+  }
+  return Status::OK();
+}
+
+template <typename Type, typename Enable = void>
+struct ReplaceWithMask {};
+
+template <typename Type>
+struct ReplaceWithMask<Type, enable_if_number<Type>> {
+  using T = typename TypeTraits<Type>::CType;
+
+  static Result<std::shared_ptr<Buffer>> AllocateData(KernelContext* ctx, const DataType&,
+                                                      const int64_t length) {
+    return ctx->Allocate(length * sizeof(T));
+  }
+
+  static void CopyData(const DataType&, uint8_t* out, const int64_t out_offset,
+                       const Datum& in, const int64_t in_offset, const int64_t length) {
+    if (in.is_array()) {
+      const auto& in_data = *in.array();
+      const auto in_arr =
+          in_data.GetValues<uint8_t>(1, (in_offset + in_data.offset) * sizeof(T));
+      std::memcpy(out + (out_offset * sizeof(T)), in_arr, length * sizeof(T));
+    } else {
+      T* begin = reinterpret_cast<T*>(out + (out_offset * sizeof(T)));
+      T* end = begin + length;
+      std::fill(begin, end, UnboxScalar<Type>::Unbox(*in.scalar()));
+    }
+  }
+
+  static Status ExecScalarMask(KernelContext* ctx, const ArrayData& array,
+                               const BooleanScalar& mask, const Datum& replacements,
+                               ArrayData* output) {
+    return ReplaceWithScalarMask(ctx, array, mask, replacements, output);
+  }
+
+  static Status ExecArrayMask(KernelContext* ctx, const ArrayData& array,
+                              const ArrayData& mask, const Datum& replacements,
+                              ArrayData* output) {
+    return ReplaceWithArrayMask<ReplaceWithMask<Type>>(ctx, array, mask, replacements,
+                                                       output);
+  }
+};
+
+template <typename Type>
+struct ReplaceWithMask<Type, enable_if_boolean<Type>> {
+  static Result<std::shared_ptr<Buffer>> AllocateData(KernelContext* ctx, const DataType&,
+                                                      const int64_t length) {
+    return ctx->AllocateBitmap(length);
+  }
+
+  static void CopyData(const DataType&, uint8_t* out, const int64_t out_offset,
+                       const Datum& in, const int64_t in_offset, const int64_t length) {
+    if (in.is_array()) {
+      const auto& in_data = *in.array();
+      const auto in_arr = in_data.GetValues<uint8_t>(1, /*absolute_offset=*/0);
+      arrow::internal::CopyBitmap(in_arr, in_offset + in_data.offset, length, out,
+                                  out_offset);
+    } else {
+      BitUtil::SetBitsTo(out, out_offset, length, in.scalar()->is_valid);
+    }
+  }
+
+  static Status ExecScalarMask(KernelContext* ctx, const ArrayData& array,
+                               const BooleanScalar& mask, const Datum& replacements,
+                               ArrayData* output) {
+    return ReplaceWithScalarMask(ctx, array, mask, replacements, output);
+  }
+  static Status ExecArrayMask(KernelContext* ctx, const ArrayData& array,
+                              const ArrayData& mask, const Datum& replacements,
+                              ArrayData* output) {
+    return ReplaceWithArrayMask<ReplaceWithMask<Type>>(ctx, array, mask, replacements,
+                                                       output);
+  }
+};
+
+template <typename Type>
+struct ReplaceWithMask<Type, enable_if_same<Type, FixedSizeBinaryType>> {
+  static Result<std::shared_ptr<Buffer>> AllocateData(KernelContext* ctx,
+                                                      const DataType& ty,
+                                                      const int64_t length) {
+    return ctx->Allocate(length *
+                         checked_cast<const FixedSizeBinaryType&>(ty).byte_width());
+  }
+
+  static void CopyData(const DataType& ty, uint8_t* out, const int64_t out_offset,
+                       const Datum& in, const int64_t in_offset, const int64_t length) {
+    const int32_t width = checked_cast<const FixedSizeBinaryType&>(ty).byte_width();
+    uint8_t* begin = out + (out_offset * width);
+    if (in.is_array()) {
+      const auto& in_data = *in.array();
+      const auto in_arr =
+          in_data.GetValues<uint8_t>(1, (in_offset + in_data.offset) * width);
+      std::memcpy(begin, in_arr, length * width);
+    } else {
+      const FixedSizeBinaryScalar& scalar =
+          checked_cast<const FixedSizeBinaryScalar&>(*in.scalar());
+      // Null scalar may have null value buffer
+      if (!scalar.value) return;
+      const Buffer& buffer = *scalar.value;
+      const uint8_t* value = buffer.data();
+      DCHECK_GE(buffer.size(), width);
+      for (int i = 0; i < length; i++) {
+        std::memcpy(begin, value, width);
+        begin += width;
+      }
+    }
+  }
+
+  static Status ExecScalarMask(KernelContext* ctx, const ArrayData& array,
+                               const BooleanScalar& mask, const Datum& replacements,
+                               ArrayData* output) {
+    return ReplaceWithScalarMask(ctx, array, mask, replacements, output);
+  }
+
+  static Status ExecArrayMask(KernelContext* ctx, const ArrayData& array,
+                              const ArrayData& mask, const Datum& replacements,
+                              ArrayData* output) {
+    return ReplaceWithArrayMask<ReplaceWithMask<Type>>(ctx, array, mask, replacements,
+                                                       output);
+  }
+};
+
+template <typename Type>
+struct ReplaceWithMask<Type, enable_if_decimal<Type>> {
+  using ScalarType = typename TypeTraits<Type>::ScalarType;
+
+  static Result<std::shared_ptr<Buffer>> AllocateData(KernelContext* ctx,
+                                                      const DataType& ty,
+                                                      const int64_t length) {
+    return ctx->Allocate(length *
+                         checked_cast<const FixedSizeBinaryType&>(ty).byte_width());
+  }
+
+  static void CopyData(const DataType& ty, uint8_t* out, const int64_t out_offset,
+                       const Datum& in, const int64_t in_offset, const int64_t length) {
+    const int32_t width = checked_cast<const FixedSizeBinaryType&>(ty).byte_width();
+    uint8_t* begin = out + (out_offset * width);
+    if (in.is_array()) {
+      const auto& in_data = *in.array();
+      const auto in_arr =
+          in_data.GetValues<uint8_t>(1, (in_offset + in_data.offset) * width);
+      std::memcpy(begin, in_arr, length * width);
+    } else {
+      const ScalarType& scalar = checked_cast<const ScalarType&>(*in.scalar());
+      const auto value = scalar.value.ToBytes();
+      for (int i = 0; i < length; i++) {
+        std::memcpy(begin, value.data(), width);
+        begin += width;
+      }
+    }
+  }
+
+  static Status ExecScalarMask(KernelContext* ctx, const ArrayData& array,
+                               const BooleanScalar& mask, const Datum& replacements,
+                               ArrayData* output) {
+    return ReplaceWithScalarMask(ctx, array, mask, replacements, output);
+  }
+
+  static Status ExecArrayMask(KernelContext* ctx, const ArrayData& array,
+                              const ArrayData& mask, const Datum& replacements,
+                              ArrayData* output) {
+    return ReplaceWithArrayMask<ReplaceWithMask<Type>>(ctx, array, mask, replacements,
+                                                       output);
+  }
+};
+
+template <typename Type>
+struct ReplaceWithMask<Type, enable_if_null<Type>> {
+  static Status ExecScalarMask(KernelContext* ctx, const ArrayData& array,
+                               const BooleanScalar& mask, const Datum& replacements,
+                               ArrayData* output) {
+    *output = array;
+    return Status::OK();
+  }
+  static Status ExecArrayMask(KernelContext* ctx, const ArrayData& array,
+                              const ArrayData& mask, const Datum& replacements,
+                              ArrayData* output) {
+    *output = array;
+    return Status::OK();
+  }
+};
+
+template <typename Type>
+struct ReplaceWithMask<Type, enable_if_base_binary<Type>> {
+  using offset_type = typename Type::offset_type;
+  using BuilderType = typename TypeTraits<Type>::BuilderType;
+
+  static Status ExecScalarMask(KernelContext* ctx, const ArrayData& array,
+                               const BooleanScalar& mask, const Datum& replacements,
+                               ArrayData* output) {
+    return ReplaceWithScalarMask(ctx, array, mask, replacements, output);
+  }
+  static Status ExecArrayMask(KernelContext* ctx, const ArrayData& array,
+                              const ArrayData& mask, const Datum& replacements,
+                              ArrayData* output) {
+    BuilderType builder(array.type, ctx->memory_pool());
+    RETURN_NOT_OK(builder.Reserve(array.length));
+    RETURN_NOT_OK(builder.ReserveData(array.buffers[2]->size()));
+    int64_t source_offset = 0;
+    int64_t replacements_offset = 0;
+    RETURN_NOT_OK(VisitArrayDataInline<BooleanType>(
+        mask,
+        [&](bool replace) {
+          if (replace && replacements.is_scalar()) {
+            const Scalar& scalar = *replacements.scalar();
+            if (scalar.is_valid) {
+              RETURN_NOT_OK(builder.Append(UnboxScalar<Type>::Unbox(scalar)));
+            } else {
+              RETURN_NOT_OK(builder.AppendNull());
+            }
+          } else {
+            const ArrayData& source = replace ? *replacements.array() : array;
+            const int64_t offset = replace ? replacements_offset++ : source_offset;
+            if (!source.MayHaveNulls() ||
+                BitUtil::GetBit(source.buffers[0]->data(), source.offset + offset)) {
+              const uint8_t* data = source.buffers[2]->data();
+              const offset_type* offsets = source.GetValues<offset_type>(1);
+              const offset_type offset0 = offsets[offset];
+              const offset_type offset1 = offsets[offset + 1];
+              RETURN_NOT_OK(builder.Append(data + offset0, offset1 - offset0));
+            } else {
+              RETURN_NOT_OK(builder.AppendNull());
+            }
+          }
+          source_offset++;
+          return Status::OK();
+        },
+        [&]() {
+          RETURN_NOT_OK(builder.AppendNull());
+          source_offset++;
+          return Status::OK();
+        }));
+    std::shared_ptr<Array> temp_output;
+    RETURN_NOT_OK(builder.Finish(&temp_output));
+    *output = *temp_output->data();
+    // Builder type != logical type due to GenerateTypeAgnosticVarBinaryBase
+    output->type = array.type;
+    return Status::OK();
+  }
+};
+
+template <typename Type>
+struct ReplaceWithMaskFunctor {
+  static Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    const ArrayData& array = *batch[0].array();
+    const Datum& replacements = batch[2];
+    ArrayData* output = out->array().get();
+    output->length = array.length;
+
+    // Needed for FixedSizeBinary/parameterized types
+    if (!array.type->Equals(*replacements.type(), /*check_metadata=*/false)) {
+      return Status::Invalid("Replacements must be of same type (expected ",
+                             array.type->ToString(), " but got ",
+                             replacements.type()->ToString(), ")");
+    }
+
+    if (!replacements.is_array() && !replacements.is_scalar()) {
+      return Status::Invalid("Replacements must be array or scalar");
+    }
+
+    if (batch[1].is_scalar()) {
+      return ReplaceWithMask<Type>::ExecScalarMask(
+          ctx, array, batch[1].scalar_as<BooleanScalar>(), replacements, output);
+    }
+    const ArrayData& mask = *batch[1].array();
+    if (array.length != mask.length) {
+      return Status::Invalid("Mask must be of same length as array (expected ",
+                             array.length, " items but got ", mask.length, " items)");
+    }
+    return ReplaceWithMask<Type>::ExecArrayMask(ctx, array, mask, replacements, output);
+  }
+};
+
+}  // namespace
+
+const FunctionDoc replace_with_mask_doc(
+    "Replace items using a mask and replacement values",
+    ("Given an array and a Boolean mask (either scalar or of equal length), "
+     "along with replacement values (either scalar or array), "
+     "each corresponding element of the mask will be replaced by the next "
+     "value of the replacements (or null if the mask is null)."

Review comment:
       ```suggestion
        "value of the replacements (or null if the mask is null). "
   ```

##########
File path: cpp/src/arrow/compute/kernels/vector_replace.cc
##########
@@ -0,0 +1,494 @@
+// 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_scalar.h"
+#include "arrow/compute/kernels/common.h"
+#include "arrow/util/bitmap_ops.h"
+
+namespace arrow {
+namespace compute {
+namespace internal {
+
+namespace {
+
+Status ReplacementArrayTooShort(int64_t expected, int64_t actual) {
+  return Status::Invalid("Replacement array must be of appropriate length (expected ",
+                         expected, " items but got ", actual, " items)");
+}
+
+// Helper to implement replace_with kernel with scalar mask for fixed-width types,
+// using callbacks to handle both bool and byte-sized types
+Status ReplaceWithScalarMask(KernelContext* ctx, const ArrayData& array,
+                             const BooleanScalar& mask, const Datum& replacements,
+                             ArrayData* output) {
+  if (!mask.is_valid) {
+    // Output = null
+    ARROW_ASSIGN_OR_RAISE(auto array,
+                          MakeArrayOfNull(array.type, array.length, ctx->memory_pool()));
+    *output = *array->data();
+    return Status::OK();
+  }
+  if (mask.value) {
+    // Output = replacement
+    if (replacements.is_scalar()) {
+      ARROW_ASSIGN_OR_RAISE(
+          auto replacement_array,
+          MakeArrayFromScalar(*replacements.scalar(), array.length, ctx->memory_pool()));
+      *output = *replacement_array->data();
+    } else {
+      auto replacement_array = replacements.array();
+      if (replacement_array->length != array.length) {
+        return ReplacementArrayTooShort(array.length, replacement_array->length);
+      }
+      *output = *replacement_array;
+    }
+  } else {
+    // Output = input
+    *output = array;
+  }
+  return Status::OK();
+}
+
+// Helper to implement replace_with kernel with array mask for fixed-width types,
+// using callbacks to handle both bool and byte-sized types and to handle
+// scalar and array replacements
+template <typename Functor>
+Status ReplaceWithArrayMask(KernelContext* ctx, const ArrayData& array,
+                            const ArrayData& mask, const Datum& replacements,
+                            ArrayData* output) {
+  ARROW_ASSIGN_OR_RAISE(output->buffers[1],
+                        Functor::AllocateData(ctx, *array.type, array.length));
+
+  uint8_t* out_bitmap = nullptr;
+  uint8_t* out_values = output->buffers[1]->mutable_data();
+  const uint8_t* mask_bitmap = mask.MayHaveNulls() ? mask.buffers[0]->data() : nullptr;
+  const uint8_t* mask_values = mask.buffers[1]->data();
+  bool replacements_bitmap;
+  int64_t replacements_length;
+  if (replacements.is_array()) {
+    replacements_bitmap = replacements.array()->MayHaveNulls();
+    replacements_length = replacements.array()->length;
+  } else {
+    replacements_bitmap = !replacements.scalar()->is_valid;
+    replacements_length = std::numeric_limits<int64_t>::max();
+  }
+  if (array.MayHaveNulls() || mask.MayHaveNulls() || replacements_bitmap) {
+    ARROW_ASSIGN_OR_RAISE(output->buffers[0], ctx->AllocateBitmap(array.length));
+    out_bitmap = output->buffers[0]->mutable_data();
+    output->null_count = -1;
+    if (array.MayHaveNulls()) {
+      arrow::internal::CopyBitmap(array.buffers[0]->data(), array.offset, array.length,
+                                  out_bitmap, /*dest_offset=*/0);
+    } else {
+      std::memset(out_bitmap, 0xFF, output->buffers[0]->size());
+    }
+  } else {
+    output->null_count = 0;
+  }
+  auto copy_bitmap = [&](int64_t out_offset, int64_t in_offset, int64_t length) {
+    DCHECK(out_bitmap);
+    if (replacements.is_array()) {
+      const auto& in_data = *replacements.array();
+      const auto in_bitmap = in_data.GetValues<uint8_t>(0, /*absolute_offset=*/0);
+      arrow::internal::CopyBitmap(in_bitmap, in_data.offset + in_offset, length,
+                                  out_bitmap, out_offset);
+    } else {
+      BitUtil::SetBitsTo(out_bitmap, out_offset, length, !replacements_bitmap);
+    }
+  };
+
+  Functor::CopyData(*array.type, out_values, /*out_offset=*/0, array, /*in_offset=*/0,
+                    array.length);
+  arrow::internal::BitBlockCounter value_counter(mask_values, mask.offset, mask.length);
+  arrow::internal::OptionalBitBlockCounter valid_counter(mask_bitmap, mask.offset,
+                                                         mask.length);
+  int64_t out_offset = 0;
+  int64_t replacements_offset = 0;
+  while (out_offset < array.length) {
+    BitBlockCount value_block = value_counter.NextWord();
+    BitBlockCount valid_block = valid_counter.NextWord();
+    DCHECK_EQ(value_block.length, valid_block.length);
+    if (value_block.AllSet() && valid_block.AllSet()) {
+      // Copy from replacement array
+      if (replacements_offset + valid_block.length > replacements_length) {
+        return ReplacementArrayTooShort(replacements_offset + valid_block.length,
+                                        replacements_length);
+      }
+      Functor::CopyData(*array.type, out_values, out_offset, replacements,
+                        replacements_offset, valid_block.length);
+      if (replacements_bitmap) {
+        copy_bitmap(out_offset, replacements_offset, valid_block.length);
+      } else if (!replacements_bitmap && out_bitmap) {
+        BitUtil::SetBitsTo(out_bitmap, out_offset, valid_block.length, true);
+      }
+      replacements_offset += valid_block.length;
+    } else if (value_block.NoneSet() && valid_block.AllSet()) {
+      // Do nothing
+    } else if (valid_block.NoneSet()) {
+      DCHECK(out_bitmap);
+      BitUtil::SetBitsTo(out_bitmap, out_offset, valid_block.length, false);
+    } else {
+      for (int64_t i = 0; i < valid_block.length; ++i) {
+        if (BitUtil::GetBit(mask_values, out_offset + mask.offset + i) &&
+            (!mask_bitmap ||
+             BitUtil::GetBit(mask_bitmap, out_offset + mask.offset + i))) {
+          if (replacements_offset >= replacements_length) {
+            return ReplacementArrayTooShort(replacements_offset + 1, replacements_length);
+          }
+          Functor::CopyData(*array.type, out_values, out_offset + i, replacements,
+                            replacements_offset,
+                            /*length=*/1);
+          if (replacements_bitmap) {
+            copy_bitmap(out_offset + i, replacements_offset, 1);
+          }
+          replacements_offset++;
+        }
+      }
+    }
+    out_offset += valid_block.length;
+  }
+
+  if (mask.MayHaveNulls()) {
+    arrow::internal::BitmapAnd(out_bitmap, /*left_offset=*/0, mask.buffers[0]->data(),
+                               mask.offset, array.length,
+                               /*out_offset=*/0, out_bitmap);
+  }
+  return Status::OK();
+}
+
+template <typename Type, typename Enable = void>
+struct ReplaceWithMask {};
+
+template <typename Type>
+struct ReplaceWithMask<Type, enable_if_number<Type>> {
+  using T = typename TypeTraits<Type>::CType;
+
+  static Result<std::shared_ptr<Buffer>> AllocateData(KernelContext* ctx, const DataType&,
+                                                      const int64_t length) {
+    return ctx->Allocate(length * sizeof(T));
+  }
+
+  static void CopyData(const DataType&, uint8_t* out, const int64_t out_offset,
+                       const Datum& in, const int64_t in_offset, const int64_t length) {
+    if (in.is_array()) {
+      const auto& in_data = *in.array();
+      const auto in_arr =
+          in_data.GetValues<uint8_t>(1, (in_offset + in_data.offset) * sizeof(T));
+      std::memcpy(out + (out_offset * sizeof(T)), in_arr, length * sizeof(T));
+    } else {
+      T* begin = reinterpret_cast<T*>(out + (out_offset * sizeof(T)));
+      T* end = begin + length;
+      std::fill(begin, end, UnboxScalar<Type>::Unbox(*in.scalar()));
+    }
+  }
+
+  static Status ExecScalarMask(KernelContext* ctx, const ArrayData& array,
+                               const BooleanScalar& mask, const Datum& replacements,
+                               ArrayData* output) {
+    return ReplaceWithScalarMask(ctx, array, mask, replacements, output);
+  }
+
+  static Status ExecArrayMask(KernelContext* ctx, const ArrayData& array,
+                              const ArrayData& mask, const Datum& replacements,
+                              ArrayData* output) {
+    return ReplaceWithArrayMask<ReplaceWithMask<Type>>(ctx, array, mask, replacements,
+                                                       output);
+  }
+};
+
+template <typename Type>
+struct ReplaceWithMask<Type, enable_if_boolean<Type>> {
+  static Result<std::shared_ptr<Buffer>> AllocateData(KernelContext* ctx, const DataType&,
+                                                      const int64_t length) {
+    return ctx->AllocateBitmap(length);
+  }
+
+  static void CopyData(const DataType&, uint8_t* out, const int64_t out_offset,
+                       const Datum& in, const int64_t in_offset, const int64_t length) {
+    if (in.is_array()) {
+      const auto& in_data = *in.array();
+      const auto in_arr = in_data.GetValues<uint8_t>(1, /*absolute_offset=*/0);
+      arrow::internal::CopyBitmap(in_arr, in_offset + in_data.offset, length, out,
+                                  out_offset);
+    } else {
+      BitUtil::SetBitsTo(out, out_offset, length, in.scalar()->is_valid);
+    }
+  }
+
+  static Status ExecScalarMask(KernelContext* ctx, const ArrayData& array,
+                               const BooleanScalar& mask, const Datum& replacements,
+                               ArrayData* output) {
+    return ReplaceWithScalarMask(ctx, array, mask, replacements, output);
+  }
+  static Status ExecArrayMask(KernelContext* ctx, const ArrayData& array,
+                              const ArrayData& mask, const Datum& replacements,
+                              ArrayData* output) {
+    return ReplaceWithArrayMask<ReplaceWithMask<Type>>(ctx, array, mask, replacements,
+                                                       output);
+  }
+};
+
+template <typename Type>
+struct ReplaceWithMask<Type, enable_if_same<Type, FixedSizeBinaryType>> {
+  static Result<std::shared_ptr<Buffer>> AllocateData(KernelContext* ctx,
+                                                      const DataType& ty,
+                                                      const int64_t length) {
+    return ctx->Allocate(length *
+                         checked_cast<const FixedSizeBinaryType&>(ty).byte_width());
+  }
+
+  static void CopyData(const DataType& ty, uint8_t* out, const int64_t out_offset,
+                       const Datum& in, const int64_t in_offset, const int64_t length) {
+    const int32_t width = checked_cast<const FixedSizeBinaryType&>(ty).byte_width();
+    uint8_t* begin = out + (out_offset * width);
+    if (in.is_array()) {
+      const auto& in_data = *in.array();
+      const auto in_arr =
+          in_data.GetValues<uint8_t>(1, (in_offset + in_data.offset) * width);
+      std::memcpy(begin, in_arr, length * width);
+    } else {
+      const FixedSizeBinaryScalar& scalar =
+          checked_cast<const FixedSizeBinaryScalar&>(*in.scalar());
+      // Null scalar may have null value buffer
+      if (!scalar.value) return;
+      const Buffer& buffer = *scalar.value;
+      const uint8_t* value = buffer.data();
+      DCHECK_GE(buffer.size(), width);
+      for (int i = 0; i < length; i++) {
+        std::memcpy(begin, value, width);
+        begin += width;
+      }
+    }
+  }
+
+  static Status ExecScalarMask(KernelContext* ctx, const ArrayData& array,
+                               const BooleanScalar& mask, const Datum& replacements,
+                               ArrayData* output) {
+    return ReplaceWithScalarMask(ctx, array, mask, replacements, output);
+  }
+
+  static Status ExecArrayMask(KernelContext* ctx, const ArrayData& array,
+                              const ArrayData& mask, const Datum& replacements,
+                              ArrayData* output) {
+    return ReplaceWithArrayMask<ReplaceWithMask<Type>>(ctx, array, mask, replacements,
+                                                       output);
+  }
+};
+
+template <typename Type>
+struct ReplaceWithMask<Type, enable_if_decimal<Type>> {
+  using ScalarType = typename TypeTraits<Type>::ScalarType;
+
+  static Result<std::shared_ptr<Buffer>> AllocateData(KernelContext* ctx,
+                                                      const DataType& ty,
+                                                      const int64_t length) {
+    return ctx->Allocate(length *
+                         checked_cast<const FixedSizeBinaryType&>(ty).byte_width());
+  }
+
+  static void CopyData(const DataType& ty, uint8_t* out, const int64_t out_offset,
+                       const Datum& in, const int64_t in_offset, const int64_t length) {
+    const int32_t width = checked_cast<const FixedSizeBinaryType&>(ty).byte_width();
+    uint8_t* begin = out + (out_offset * width);
+    if (in.is_array()) {
+      const auto& in_data = *in.array();
+      const auto in_arr =
+          in_data.GetValues<uint8_t>(1, (in_offset + in_data.offset) * width);
+      std::memcpy(begin, in_arr, length * width);
+    } else {
+      const ScalarType& scalar = checked_cast<const ScalarType&>(*in.scalar());
+      const auto value = scalar.value.ToBytes();
+      for (int i = 0; i < length; i++) {
+        std::memcpy(begin, value.data(), width);
+        begin += width;
+      }
+    }
+  }
+
+  static Status ExecScalarMask(KernelContext* ctx, const ArrayData& array,
+                               const BooleanScalar& mask, const Datum& replacements,
+                               ArrayData* output) {
+    return ReplaceWithScalarMask(ctx, array, mask, replacements, output);
+  }
+
+  static Status ExecArrayMask(KernelContext* ctx, const ArrayData& array,
+                              const ArrayData& mask, const Datum& replacements,
+                              ArrayData* output) {
+    return ReplaceWithArrayMask<ReplaceWithMask<Type>>(ctx, array, mask, replacements,
+                                                       output);
+  }
+};
+
+template <typename Type>
+struct ReplaceWithMask<Type, enable_if_null<Type>> {
+  static Status ExecScalarMask(KernelContext* ctx, const ArrayData& array,
+                               const BooleanScalar& mask, const Datum& replacements,
+                               ArrayData* output) {
+    *output = array;
+    return Status::OK();
+  }
+  static Status ExecArrayMask(KernelContext* ctx, const ArrayData& array,
+                              const ArrayData& mask, const Datum& replacements,
+                              ArrayData* output) {
+    *output = array;
+    return Status::OK();
+  }
+};
+
+template <typename Type>
+struct ReplaceWithMask<Type, enable_if_base_binary<Type>> {
+  using offset_type = typename Type::offset_type;
+  using BuilderType = typename TypeTraits<Type>::BuilderType;
+
+  static Status ExecScalarMask(KernelContext* ctx, const ArrayData& array,
+                               const BooleanScalar& mask, const Datum& replacements,
+                               ArrayData* output) {
+    return ReplaceWithScalarMask(ctx, array, mask, replacements, output);
+  }
+  static Status ExecArrayMask(KernelContext* ctx, const ArrayData& array,
+                              const ArrayData& mask, const Datum& replacements,
+                              ArrayData* output) {
+    BuilderType builder(array.type, ctx->memory_pool());
+    RETURN_NOT_OK(builder.Reserve(array.length));
+    RETURN_NOT_OK(builder.ReserveData(array.buffers[2]->size()));
+    int64_t source_offset = 0;
+    int64_t replacements_offset = 0;
+    RETURN_NOT_OK(VisitArrayDataInline<BooleanType>(
+        mask,
+        [&](bool replace) {
+          if (replace && replacements.is_scalar()) {
+            const Scalar& scalar = *replacements.scalar();
+            if (scalar.is_valid) {
+              RETURN_NOT_OK(builder.Append(UnboxScalar<Type>::Unbox(scalar)));
+            } else {
+              RETURN_NOT_OK(builder.AppendNull());
+            }
+          } else {
+            const ArrayData& source = replace ? *replacements.array() : array;
+            const int64_t offset = replace ? replacements_offset++ : source_offset;
+            if (!source.MayHaveNulls() ||
+                BitUtil::GetBit(source.buffers[0]->data(), source.offset + offset)) {
+              const uint8_t* data = source.buffers[2]->data();
+              const offset_type* offsets = source.GetValues<offset_type>(1);
+              const offset_type offset0 = offsets[offset];
+              const offset_type offset1 = offsets[offset + 1];
+              RETURN_NOT_OK(builder.Append(data + offset0, offset1 - offset0));
+            } else {
+              RETURN_NOT_OK(builder.AppendNull());
+            }
+          }
+          source_offset++;
+          return Status::OK();
+        },
+        [&]() {
+          RETURN_NOT_OK(builder.AppendNull());
+          source_offset++;
+          return Status::OK();
+        }));
+    std::shared_ptr<Array> temp_output;
+    RETURN_NOT_OK(builder.Finish(&temp_output));
+    *output = *temp_output->data();
+    // Builder type != logical type due to GenerateTypeAgnosticVarBinaryBase
+    output->type = array.type;
+    return Status::OK();
+  }
+};
+
+template <typename Type>
+struct ReplaceWithMaskFunctor {
+  static Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    const ArrayData& array = *batch[0].array();
+    const Datum& replacements = batch[2];
+    ArrayData* output = out->array().get();
+    output->length = array.length;
+
+    // Needed for FixedSizeBinary/parameterized types
+    if (!array.type->Equals(*replacements.type(), /*check_metadata=*/false)) {
+      return Status::Invalid("Replacements must be of same type (expected ",
+                             array.type->ToString(), " but got ",
+                             replacements.type()->ToString(), ")");
+    }
+
+    if (!replacements.is_array() && !replacements.is_scalar()) {
+      return Status::Invalid("Replacements must be array or scalar");
+    }
+
+    if (batch[1].is_scalar()) {
+      return ReplaceWithMask<Type>::ExecScalarMask(
+          ctx, array, batch[1].scalar_as<BooleanScalar>(), replacements, output);
+    }
+    const ArrayData& mask = *batch[1].array();
+    if (array.length != mask.length) {
+      return Status::Invalid("Mask must be of same length as array (expected ",
+                             array.length, " items but got ", mask.length, " items)");
+    }
+    return ReplaceWithMask<Type>::ExecArrayMask(ctx, array, mask, replacements, output);
+  }
+};
+
+}  // namespace
+
+const FunctionDoc replace_with_mask_doc(
+    "Replace items using a mask and replacement values",
+    ("Given an array and a Boolean mask (either scalar or of equal length), "
+     "along with replacement values (either scalar or array), "
+     "each corresponding element of the mask will be replaced by the next "
+     "value of the replacements (or null if the mask is null)."
+     "Hence, for replacement arrays, len(replacements) == popcnt(mask)."),

Review comment:
       I think popcnt will not be clear for many users. Maybe something like `sum(mask == true)` ?

##########
File path: docs/source/cpp/compute.rst
##########
@@ -815,6 +815,22 @@ Associative transforms
   Each output element corresponds to a unique value in the input, along
   with the number of times this value has appeared.
 
+Replacements
+~~~~~~~~~~~~
+
+These functions create a copy of the first input with some elements
+replaced, based on the remaining inputs.
+
++-------------------+------------+-------------------------+--------------+--------------+------------------+-------------+
+| Function name     | Arity      | Input type 1            | Input type 2 | Input type 3 | Output type      | Notes       |
++===================+============+=========================+==============+==============+==================+=============+
+| replace_with_mask | Ternary    | Fixed-width, non-binary | Boolean      | Input type 1 | Input type 1     | \(1)        |

Review comment:
       Non-fixed width and binary is now also supported already?

##########
File path: cpp/src/arrow/compute/kernels/vector_replace.cc
##########
@@ -0,0 +1,494 @@
+// 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_scalar.h"
+#include "arrow/compute/kernels/common.h"
+#include "arrow/util/bitmap_ops.h"
+
+namespace arrow {
+namespace compute {
+namespace internal {
+
+namespace {
+
+Status ReplacementArrayTooShort(int64_t expected, int64_t actual) {
+  return Status::Invalid("Replacement array must be of appropriate length (expected ",
+                         expected, " items but got ", actual, " items)");
+}
+
+// Helper to implement replace_with kernel with scalar mask for fixed-width types,
+// using callbacks to handle both bool and byte-sized types
+Status ReplaceWithScalarMask(KernelContext* ctx, const ArrayData& array,
+                             const BooleanScalar& mask, const Datum& replacements,
+                             ArrayData* output) {
+  if (!mask.is_valid) {
+    // Output = null
+    ARROW_ASSIGN_OR_RAISE(auto array,
+                          MakeArrayOfNull(array.type, array.length, ctx->memory_pool()));
+    *output = *array->data();
+    return Status::OK();
+  }
+  if (mask.value) {
+    // Output = replacement
+    if (replacements.is_scalar()) {
+      ARROW_ASSIGN_OR_RAISE(
+          auto replacement_array,
+          MakeArrayFromScalar(*replacements.scalar(), array.length, ctx->memory_pool()));
+      *output = *replacement_array->data();
+    } else {
+      auto replacement_array = replacements.array();
+      if (replacement_array->length != array.length) {
+        return ReplacementArrayTooShort(array.length, replacement_array->length);
+      }
+      *output = *replacement_array;
+    }
+  } else {
+    // Output = input
+    *output = array;
+  }
+  return Status::OK();
+}
+
+// Helper to implement replace_with kernel with array mask for fixed-width types,
+// using callbacks to handle both bool and byte-sized types and to handle
+// scalar and array replacements
+template <typename Functor>
+Status ReplaceWithArrayMask(KernelContext* ctx, const ArrayData& array,
+                            const ArrayData& mask, const Datum& replacements,
+                            ArrayData* output) {
+  ARROW_ASSIGN_OR_RAISE(output->buffers[1],
+                        Functor::AllocateData(ctx, *array.type, array.length));
+
+  uint8_t* out_bitmap = nullptr;
+  uint8_t* out_values = output->buffers[1]->mutable_data();
+  const uint8_t* mask_bitmap = mask.MayHaveNulls() ? mask.buffers[0]->data() : nullptr;
+  const uint8_t* mask_values = mask.buffers[1]->data();
+  bool replacements_bitmap;
+  int64_t replacements_length;
+  if (replacements.is_array()) {
+    replacements_bitmap = replacements.array()->MayHaveNulls();
+    replacements_length = replacements.array()->length;
+  } else {
+    replacements_bitmap = !replacements.scalar()->is_valid;
+    replacements_length = std::numeric_limits<int64_t>::max();
+  }
+  if (array.MayHaveNulls() || mask.MayHaveNulls() || replacements_bitmap) {
+    ARROW_ASSIGN_OR_RAISE(output->buffers[0], ctx->AllocateBitmap(array.length));
+    out_bitmap = output->buffers[0]->mutable_data();
+    output->null_count = -1;
+    if (array.MayHaveNulls()) {
+      arrow::internal::CopyBitmap(array.buffers[0]->data(), array.offset, array.length,
+                                  out_bitmap, /*dest_offset=*/0);
+    } else {
+      std::memset(out_bitmap, 0xFF, output->buffers[0]->size());
+    }
+  } else {
+    output->null_count = 0;
+  }
+  auto copy_bitmap = [&](int64_t out_offset, int64_t in_offset, int64_t length) {
+    DCHECK(out_bitmap);
+    if (replacements.is_array()) {
+      const auto& in_data = *replacements.array();
+      const auto in_bitmap = in_data.GetValues<uint8_t>(0, /*absolute_offset=*/0);
+      arrow::internal::CopyBitmap(in_bitmap, in_data.offset + in_offset, length,
+                                  out_bitmap, out_offset);
+    } else {
+      BitUtil::SetBitsTo(out_bitmap, out_offset, length, !replacements_bitmap);
+    }
+  };
+
+  Functor::CopyData(*array.type, out_values, /*out_offset=*/0, array, /*in_offset=*/0,
+                    array.length);
+  arrow::internal::BitBlockCounter value_counter(mask_values, mask.offset, mask.length);
+  arrow::internal::OptionalBitBlockCounter valid_counter(mask_bitmap, mask.offset,
+                                                         mask.length);
+  int64_t out_offset = 0;
+  int64_t replacements_offset = 0;
+  while (out_offset < array.length) {
+    BitBlockCount value_block = value_counter.NextWord();
+    BitBlockCount valid_block = valid_counter.NextWord();
+    DCHECK_EQ(value_block.length, valid_block.length);
+    if (value_block.AllSet() && valid_block.AllSet()) {
+      // Copy from replacement array
+      if (replacements_offset + valid_block.length > replacements_length) {
+        return ReplacementArrayTooShort(replacements_offset + valid_block.length,
+                                        replacements_length);
+      }
+      Functor::CopyData(*array.type, out_values, out_offset, replacements,
+                        replacements_offset, valid_block.length);
+      if (replacements_bitmap) {
+        copy_bitmap(out_offset, replacements_offset, valid_block.length);
+      } else if (!replacements_bitmap && out_bitmap) {
+        BitUtil::SetBitsTo(out_bitmap, out_offset, valid_block.length, true);
+      }
+      replacements_offset += valid_block.length;
+    } else if (value_block.NoneSet() && valid_block.AllSet()) {
+      // Do nothing
+    } else if (valid_block.NoneSet()) {
+      DCHECK(out_bitmap);
+      BitUtil::SetBitsTo(out_bitmap, out_offset, valid_block.length, false);
+    } else {
+      for (int64_t i = 0; i < valid_block.length; ++i) {
+        if (BitUtil::GetBit(mask_values, out_offset + mask.offset + i) &&
+            (!mask_bitmap ||
+             BitUtil::GetBit(mask_bitmap, out_offset + mask.offset + i))) {
+          if (replacements_offset >= replacements_length) {
+            return ReplacementArrayTooShort(replacements_offset + 1, replacements_length);
+          }
+          Functor::CopyData(*array.type, out_values, out_offset + i, replacements,
+                            replacements_offset,
+                            /*length=*/1);
+          if (replacements_bitmap) {
+            copy_bitmap(out_offset + i, replacements_offset, 1);
+          }
+          replacements_offset++;
+        }
+      }
+    }
+    out_offset += valid_block.length;
+  }
+
+  if (mask.MayHaveNulls()) {
+    arrow::internal::BitmapAnd(out_bitmap, /*left_offset=*/0, mask.buffers[0]->data(),
+                               mask.offset, array.length,
+                               /*out_offset=*/0, out_bitmap);
+  }
+  return Status::OK();
+}
+
+template <typename Type, typename Enable = void>
+struct ReplaceWithMask {};
+
+template <typename Type>
+struct ReplaceWithMask<Type, enable_if_number<Type>> {
+  using T = typename TypeTraits<Type>::CType;
+
+  static Result<std::shared_ptr<Buffer>> AllocateData(KernelContext* ctx, const DataType&,
+                                                      const int64_t length) {
+    return ctx->Allocate(length * sizeof(T));
+  }
+
+  static void CopyData(const DataType&, uint8_t* out, const int64_t out_offset,
+                       const Datum& in, const int64_t in_offset, const int64_t length) {
+    if (in.is_array()) {
+      const auto& in_data = *in.array();
+      const auto in_arr =
+          in_data.GetValues<uint8_t>(1, (in_offset + in_data.offset) * sizeof(T));
+      std::memcpy(out + (out_offset * sizeof(T)), in_arr, length * sizeof(T));
+    } else {
+      T* begin = reinterpret_cast<T*>(out + (out_offset * sizeof(T)));
+      T* end = begin + length;
+      std::fill(begin, end, UnboxScalar<Type>::Unbox(*in.scalar()));
+    }
+  }
+
+  static Status ExecScalarMask(KernelContext* ctx, const ArrayData& array,
+                               const BooleanScalar& mask, const Datum& replacements,
+                               ArrayData* output) {
+    return ReplaceWithScalarMask(ctx, array, mask, replacements, output);
+  }
+
+  static Status ExecArrayMask(KernelContext* ctx, const ArrayData& array,
+                              const ArrayData& mask, const Datum& replacements,
+                              ArrayData* output) {
+    return ReplaceWithArrayMask<ReplaceWithMask<Type>>(ctx, array, mask, replacements,
+                                                       output);
+  }
+};
+
+template <typename Type>
+struct ReplaceWithMask<Type, enable_if_boolean<Type>> {
+  static Result<std::shared_ptr<Buffer>> AllocateData(KernelContext* ctx, const DataType&,
+                                                      const int64_t length) {
+    return ctx->AllocateBitmap(length);
+  }
+
+  static void CopyData(const DataType&, uint8_t* out, const int64_t out_offset,
+                       const Datum& in, const int64_t in_offset, const int64_t length) {
+    if (in.is_array()) {
+      const auto& in_data = *in.array();
+      const auto in_arr = in_data.GetValues<uint8_t>(1, /*absolute_offset=*/0);
+      arrow::internal::CopyBitmap(in_arr, in_offset + in_data.offset, length, out,
+                                  out_offset);
+    } else {
+      BitUtil::SetBitsTo(out, out_offset, length, in.scalar()->is_valid);
+    }
+  }
+
+  static Status ExecScalarMask(KernelContext* ctx, const ArrayData& array,
+                               const BooleanScalar& mask, const Datum& replacements,
+                               ArrayData* output) {
+    return ReplaceWithScalarMask(ctx, array, mask, replacements, output);
+  }
+  static Status ExecArrayMask(KernelContext* ctx, const ArrayData& array,
+                              const ArrayData& mask, const Datum& replacements,
+                              ArrayData* output) {
+    return ReplaceWithArrayMask<ReplaceWithMask<Type>>(ctx, array, mask, replacements,
+                                                       output);
+  }
+};
+
+template <typename Type>
+struct ReplaceWithMask<Type, enable_if_same<Type, FixedSizeBinaryType>> {
+  static Result<std::shared_ptr<Buffer>> AllocateData(KernelContext* ctx,
+                                                      const DataType& ty,
+                                                      const int64_t length) {
+    return ctx->Allocate(length *
+                         checked_cast<const FixedSizeBinaryType&>(ty).byte_width());
+  }
+
+  static void CopyData(const DataType& ty, uint8_t* out, const int64_t out_offset,
+                       const Datum& in, const int64_t in_offset, const int64_t length) {
+    const int32_t width = checked_cast<const FixedSizeBinaryType&>(ty).byte_width();
+    uint8_t* begin = out + (out_offset * width);
+    if (in.is_array()) {
+      const auto& in_data = *in.array();
+      const auto in_arr =
+          in_data.GetValues<uint8_t>(1, (in_offset + in_data.offset) * width);
+      std::memcpy(begin, in_arr, length * width);
+    } else {
+      const FixedSizeBinaryScalar& scalar =
+          checked_cast<const FixedSizeBinaryScalar&>(*in.scalar());
+      // Null scalar may have null value buffer
+      if (!scalar.value) return;
+      const Buffer& buffer = *scalar.value;
+      const uint8_t* value = buffer.data();
+      DCHECK_GE(buffer.size(), width);
+      for (int i = 0; i < length; i++) {
+        std::memcpy(begin, value, width);
+        begin += width;
+      }
+    }
+  }
+
+  static Status ExecScalarMask(KernelContext* ctx, const ArrayData& array,
+                               const BooleanScalar& mask, const Datum& replacements,
+                               ArrayData* output) {
+    return ReplaceWithScalarMask(ctx, array, mask, replacements, output);
+  }
+
+  static Status ExecArrayMask(KernelContext* ctx, const ArrayData& array,
+                              const ArrayData& mask, const Datum& replacements,
+                              ArrayData* output) {
+    return ReplaceWithArrayMask<ReplaceWithMask<Type>>(ctx, array, mask, replacements,
+                                                       output);
+  }
+};
+
+template <typename Type>
+struct ReplaceWithMask<Type, enable_if_decimal<Type>> {
+  using ScalarType = typename TypeTraits<Type>::ScalarType;
+
+  static Result<std::shared_ptr<Buffer>> AllocateData(KernelContext* ctx,
+                                                      const DataType& ty,
+                                                      const int64_t length) {
+    return ctx->Allocate(length *
+                         checked_cast<const FixedSizeBinaryType&>(ty).byte_width());
+  }
+
+  static void CopyData(const DataType& ty, uint8_t* out, const int64_t out_offset,
+                       const Datum& in, const int64_t in_offset, const int64_t length) {
+    const int32_t width = checked_cast<const FixedSizeBinaryType&>(ty).byte_width();
+    uint8_t* begin = out + (out_offset * width);
+    if (in.is_array()) {
+      const auto& in_data = *in.array();
+      const auto in_arr =
+          in_data.GetValues<uint8_t>(1, (in_offset + in_data.offset) * width);
+      std::memcpy(begin, in_arr, length * width);
+    } else {
+      const ScalarType& scalar = checked_cast<const ScalarType&>(*in.scalar());
+      const auto value = scalar.value.ToBytes();
+      for (int i = 0; i < length; i++) {
+        std::memcpy(begin, value.data(), width);
+        begin += width;
+      }
+    }
+  }
+
+  static Status ExecScalarMask(KernelContext* ctx, const ArrayData& array,
+                               const BooleanScalar& mask, const Datum& replacements,
+                               ArrayData* output) {
+    return ReplaceWithScalarMask(ctx, array, mask, replacements, output);
+  }
+
+  static Status ExecArrayMask(KernelContext* ctx, const ArrayData& array,
+                              const ArrayData& mask, const Datum& replacements,
+                              ArrayData* output) {
+    return ReplaceWithArrayMask<ReplaceWithMask<Type>>(ctx, array, mask, replacements,
+                                                       output);
+  }
+};
+
+template <typename Type>
+struct ReplaceWithMask<Type, enable_if_null<Type>> {
+  static Status ExecScalarMask(KernelContext* ctx, const ArrayData& array,
+                               const BooleanScalar& mask, const Datum& replacements,
+                               ArrayData* output) {
+    *output = array;
+    return Status::OK();
+  }
+  static Status ExecArrayMask(KernelContext* ctx, const ArrayData& array,
+                              const ArrayData& mask, const Datum& replacements,
+                              ArrayData* output) {
+    *output = array;
+    return Status::OK();
+  }
+};
+
+template <typename Type>
+struct ReplaceWithMask<Type, enable_if_base_binary<Type>> {
+  using offset_type = typename Type::offset_type;
+  using BuilderType = typename TypeTraits<Type>::BuilderType;
+
+  static Status ExecScalarMask(KernelContext* ctx, const ArrayData& array,
+                               const BooleanScalar& mask, const Datum& replacements,
+                               ArrayData* output) {
+    return ReplaceWithScalarMask(ctx, array, mask, replacements, output);
+  }
+  static Status ExecArrayMask(KernelContext* ctx, const ArrayData& array,
+                              const ArrayData& mask, const Datum& replacements,
+                              ArrayData* output) {
+    BuilderType builder(array.type, ctx->memory_pool());
+    RETURN_NOT_OK(builder.Reserve(array.length));
+    RETURN_NOT_OK(builder.ReserveData(array.buffers[2]->size()));
+    int64_t source_offset = 0;
+    int64_t replacements_offset = 0;
+    RETURN_NOT_OK(VisitArrayDataInline<BooleanType>(
+        mask,
+        [&](bool replace) {
+          if (replace && replacements.is_scalar()) {
+            const Scalar& scalar = *replacements.scalar();
+            if (scalar.is_valid) {
+              RETURN_NOT_OK(builder.Append(UnboxScalar<Type>::Unbox(scalar)));
+            } else {
+              RETURN_NOT_OK(builder.AppendNull());
+            }
+          } else {
+            const ArrayData& source = replace ? *replacements.array() : array;
+            const int64_t offset = replace ? replacements_offset++ : source_offset;
+            if (!source.MayHaveNulls() ||
+                BitUtil::GetBit(source.buffers[0]->data(), source.offset + offset)) {
+              const uint8_t* data = source.buffers[2]->data();
+              const offset_type* offsets = source.GetValues<offset_type>(1);
+              const offset_type offset0 = offsets[offset];
+              const offset_type offset1 = offsets[offset + 1];
+              RETURN_NOT_OK(builder.Append(data + offset0, offset1 - offset0));
+            } else {
+              RETURN_NOT_OK(builder.AppendNull());
+            }
+          }
+          source_offset++;
+          return Status::OK();
+        },
+        [&]() {
+          RETURN_NOT_OK(builder.AppendNull());
+          source_offset++;
+          return Status::OK();
+        }));
+    std::shared_ptr<Array> temp_output;
+    RETURN_NOT_OK(builder.Finish(&temp_output));
+    *output = *temp_output->data();
+    // Builder type != logical type due to GenerateTypeAgnosticVarBinaryBase
+    output->type = array.type;
+    return Status::OK();
+  }
+};
+
+template <typename Type>
+struct ReplaceWithMaskFunctor {
+  static Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    const ArrayData& array = *batch[0].array();
+    const Datum& replacements = batch[2];
+    ArrayData* output = out->array().get();
+    output->length = array.length;
+
+    // Needed for FixedSizeBinary/parameterized types
+    if (!array.type->Equals(*replacements.type(), /*check_metadata=*/false)) {
+      return Status::Invalid("Replacements must be of same type (expected ",
+                             array.type->ToString(), " but got ",
+                             replacements.type()->ToString(), ")");
+    }
+
+    if (!replacements.is_array() && !replacements.is_scalar()) {
+      return Status::Invalid("Replacements must be array or scalar");
+    }
+
+    if (batch[1].is_scalar()) {
+      return ReplaceWithMask<Type>::ExecScalarMask(
+          ctx, array, batch[1].scalar_as<BooleanScalar>(), replacements, output);
+    }
+    const ArrayData& mask = *batch[1].array();
+    if (array.length != mask.length) {
+      return Status::Invalid("Mask must be of same length as array (expected ",
+                             array.length, " items but got ", mask.length, " items)");
+    }
+    return ReplaceWithMask<Type>::ExecArrayMask(ctx, array, mask, replacements, output);
+  }
+};
+
+}  // namespace
+
+const FunctionDoc replace_with_mask_doc(
+    "Replace items using a mask and replacement values",
+    ("Given an array and a Boolean mask (either scalar or of equal length), "
+     "along with replacement values (either scalar or array), "
+     "each corresponding element of the mask will be replaced by the next "

Review comment:
       ```suggestion
        "each corresponding true value of the mask will be replaced by the next "
   ```
   
   ?
   




-- 
This is an automated message from the 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 #10412: ARROW-9430: [C++] Implement replace_with_mask kernel

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


   And now we're about 2-3x faster!
   
   ```
   -------------------------------------------------------------------------------------------------------
   Benchmark                                             Time             CPU   Iterations UserCounters...
   -------------------------------------------------------------------------------------------------------
   ReplaceWithMaskLowSelectivityBench/16384/0        44072 ns        44072 ns        16258 bytes_per_second=2.76981G/s
   ReplaceWithMaskLowSelectivityBench/16384/99       44797 ns        44797 ns        15796 bytes_per_second=2.70848G/s
   ReplaceWithMaskHighSelectivityBench/16384/0       92927 ns        92926 ns         7377 bytes_per_second=1.31362G/s
   ReplaceWithMaskHighSelectivityBench/16384/99      90865 ns        90865 ns         7657 bytes_per_second=1.33531G/s
   ```


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

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

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



[GitHub] [arrow] lidavidm commented on pull request #10412: ARROW-9430: [C++] Implement replace_with_mask kernel

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


   I removed `ReplaceWithOptions` when I made this a vector kernel, so I fixed up the docs - thanks for catching that.


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

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



[GitHub] [arrow] nirandaperera commented on pull request #10412: ARROW-9430: [C++] Implement override_mask kernel

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


   @lidavidm @jorisvandenbossche @bkietz I'm also thinking about how to handle different length'd arrays for ARROW-9431. Like David said, compute infrastructure guarantees that all arrays passed to the function are of the same length. If we are going ahead with the `Options` approach, for ARROW-9431, it would be a unary function and other 2 arrays in the options.
   So should we make ARROW-9430 and ARROW-9431 vector kernels instead?


-- 
This is an automated message from the 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 #10412: ARROW-9430: [C++] Implement replace_with_mask kernel

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


   Nice to see this one merged, thanks all!


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

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

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



[GitHub] [arrow] lidavidm commented on pull request #10412: ARROW-9430: [C++] Implement replace_with_mask kernel

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


   Performance here looks highly variable - the benchmark seems to be one of the bimodal ones; we either run at ~1.3 G/s or ~1.7 G/s when repeatedly running the benchmark.
   
   As written, this makes use of rather generic code like CopyBitmap, which hurts us quite a bit in terms of performance. Specializing the cases for array and scalar replacements and using direct methods like SetBitTo looks to help performance quite a bit, so I'll give that a swing.


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

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

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



[GitHub] [arrow] nirandaperera commented on a change in pull request #10412: ARROW-9430: [C++] Implement replace_with_mask kernel

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



##########
File path: cpp/src/arrow/compute/kernels/vector_replace.cc
##########
@@ -0,0 +1,466 @@
+// 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_scalar.h"
+#include "arrow/compute/kernels/common.h"
+#include "arrow/util/bitmap_ops.h"
+
+namespace arrow {
+namespace compute {
+namespace internal {
+
+namespace {
+
+Status ReplacementArrayTooShort(int64_t expected, int64_t actual) {
+  return Status::Invalid("Replacement array must be of appropriate length (expected ",
+                         expected, " items but got ", actual, " items)");
+}
+
+// Helper to implement replace_with kernel with scalar mask for fixed-width types,
+// using callbacks to handle both bool and byte-sized types
+Status ReplaceWithScalarMask(KernelContext* ctx, const ArrayData& array,
+                             const BooleanScalar& mask, const Datum& replacements,
+                             ArrayData* output) {
+  if (!mask.is_valid) {
+    // Output = null
+    ARROW_ASSIGN_OR_RAISE(auto replacement_array,
+                          MakeArrayOfNull(array.type, array.length, ctx->memory_pool()));
+    *output = *replacement_array->data();
+    return Status::OK();
+  }
+  if (mask.value) {
+    // Output = replacement
+    if (replacements.is_scalar()) {
+      ARROW_ASSIGN_OR_RAISE(
+          auto replacement_array,
+          MakeArrayFromScalar(*replacements.scalar(), array.length, ctx->memory_pool()));
+      *output = *replacement_array->data();
+    } else {
+      auto replacement_array = replacements.array();
+      if (replacement_array->length != array.length) {
+        return ReplacementArrayTooShort(array.length, replacement_array->length);
+      }
+      *output = *replacement_array;
+    }
+  } else {
+    // Output = input
+    *output = array;
+  }
+  return Status::OK();
+}
+
+// Helper to implement replace_with kernel with array mask for fixed-width types,
+// using callbacks to handle both bool and byte-sized types and to handle
+// scalar and array replacements
+template <typename Functor>
+Status ReplaceWithArrayMask(KernelContext* ctx, const ArrayData& array,
+                            const ArrayData& mask, const Datum& replacements,
+                            ArrayData* output) {
+  uint8_t* out_bitmap = nullptr;
+  uint8_t* out_values = output->buffers[1]->mutable_data();
+  const uint8_t* mask_bitmap = mask.MayHaveNulls() ? mask.buffers[0]->data() : nullptr;
+  const uint8_t* mask_values = mask.buffers[1]->data();
+  bool replacements_bitmap;
+  int64_t replacements_length;
+  if (replacements.is_array()) {
+    replacements_bitmap = replacements.array()->MayHaveNulls();
+    replacements_length = replacements.array()->length;
+  } else {
+    replacements_bitmap = !replacements.scalar()->is_valid;
+    replacements_length = std::numeric_limits<int64_t>::max();
+  }
+  if (array.MayHaveNulls() || mask.MayHaveNulls() || replacements_bitmap) {
+    ARROW_ASSIGN_OR_RAISE(output->buffers[0], ctx->AllocateBitmap(array.length));
+    out_bitmap = output->buffers[0]->mutable_data();
+    output->null_count = -1;
+    if (array.MayHaveNulls()) {
+      // Copy array's bitmap
+      arrow::internal::CopyBitmap(array.buffers[0]->data(), array.offset, array.length,
+                                  out_bitmap, /*dest_offset=*/0);
+    } else {
+      // Array has no bitmap but mask/replacements do, generate an all-valid bitmap
+      std::memset(out_bitmap, 0xFF, output->buffers[0]->size());
+    }
+  } else {
+    output->null_count = 0;
+  }
+  auto copy_bitmap = [&](int64_t out_offset, int64_t in_offset, int64_t length) {
+    DCHECK(out_bitmap);
+    if (replacements.is_array()) {
+      const auto& in_data = *replacements.array();
+      const auto in_bitmap = in_data.GetValues<uint8_t>(0, /*absolute_offset=*/0);
+      arrow::internal::CopyBitmap(in_bitmap, in_data.offset + in_offset, length,
+                                  out_bitmap, out_offset);
+    } else {
+      BitUtil::SetBitsTo(out_bitmap, out_offset, length, !replacements_bitmap);
+    }
+  };
+
+  Functor::CopyData(*array.type, out_values, /*out_offset=*/0, array, /*in_offset=*/0,
+                    array.length);
+  arrow::internal::BitBlockCounter value_counter(mask_values, mask.offset, mask.length);
+  arrow::internal::OptionalBitBlockCounter valid_counter(mask_bitmap, mask.offset,
+                                                         mask.length);
+  int64_t out_offset = 0;
+  int64_t replacements_offset = 0;
+  while (out_offset < array.length) {
+    BitBlockCount value_block = value_counter.NextWord();
+    BitBlockCount valid_block = valid_counter.NextWord();
+    DCHECK_EQ(value_block.length, valid_block.length);
+    if (value_block.AllSet() && valid_block.AllSet()) {
+      // Copy from replacement array
+      if (replacements_offset + valid_block.length > replacements_length) {
+        return ReplacementArrayTooShort(replacements_offset + valid_block.length,
+                                        replacements_length);
+      }
+      Functor::CopyData(*array.type, out_values, out_offset, replacements,
+                        replacements_offset, valid_block.length);
+      if (replacements_bitmap) {
+        copy_bitmap(out_offset, replacements_offset, valid_block.length);
+      } else if (!replacements_bitmap && out_bitmap) {
+        BitUtil::SetBitsTo(out_bitmap, out_offset, valid_block.length, true);
+      }
+      replacements_offset += valid_block.length;
+    } else if ((value_block.NoneSet() && valid_block.AllSet()) || valid_block.NoneSet()) {

Review comment:
       nit. I think this simplifies a bit. 
   ```
   A.B + ~B = (A+~B).(B+~B) = A+~B --> value_block.NoneSet() || valid_block.NoneSet()
   ```

##########
File path: cpp/src/arrow/compute/kernels/vector_replace.cc
##########
@@ -0,0 +1,466 @@
+// 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_scalar.h"
+#include "arrow/compute/kernels/common.h"
+#include "arrow/util/bitmap_ops.h"
+
+namespace arrow {
+namespace compute {
+namespace internal {
+
+namespace {
+
+Status ReplacementArrayTooShort(int64_t expected, int64_t actual) {
+  return Status::Invalid("Replacement array must be of appropriate length (expected ",
+                         expected, " items but got ", actual, " items)");
+}
+
+// Helper to implement replace_with kernel with scalar mask for fixed-width types,
+// using callbacks to handle both bool and byte-sized types
+Status ReplaceWithScalarMask(KernelContext* ctx, const ArrayData& array,
+                             const BooleanScalar& mask, const Datum& replacements,
+                             ArrayData* output) {
+  if (!mask.is_valid) {
+    // Output = null
+    ARROW_ASSIGN_OR_RAISE(auto replacement_array,
+                          MakeArrayOfNull(array.type, array.length, ctx->memory_pool()));
+    *output = *replacement_array->data();
+    return Status::OK();
+  }
+  if (mask.value) {
+    // Output = replacement
+    if (replacements.is_scalar()) {
+      ARROW_ASSIGN_OR_RAISE(
+          auto replacement_array,
+          MakeArrayFromScalar(*replacements.scalar(), array.length, ctx->memory_pool()));
+      *output = *replacement_array->data();
+    } else {
+      auto replacement_array = replacements.array();
+      if (replacement_array->length != array.length) {
+        return ReplacementArrayTooShort(array.length, replacement_array->length);
+      }
+      *output = *replacement_array;
+    }
+  } else {
+    // Output = input
+    *output = array;
+  }
+  return Status::OK();
+}
+
+// Helper to implement replace_with kernel with array mask for fixed-width types,
+// using callbacks to handle both bool and byte-sized types and to handle
+// scalar and array replacements
+template <typename Functor>
+Status ReplaceWithArrayMask(KernelContext* ctx, const ArrayData& array,
+                            const ArrayData& mask, const Datum& replacements,
+                            ArrayData* output) {
+  uint8_t* out_bitmap = nullptr;
+  uint8_t* out_values = output->buffers[1]->mutable_data();
+  const uint8_t* mask_bitmap = mask.MayHaveNulls() ? mask.buffers[0]->data() : nullptr;
+  const uint8_t* mask_values = mask.buffers[1]->data();
+  bool replacements_bitmap;
+  int64_t replacements_length;
+  if (replacements.is_array()) {
+    replacements_bitmap = replacements.array()->MayHaveNulls();
+    replacements_length = replacements.array()->length;
+  } else {
+    replacements_bitmap = !replacements.scalar()->is_valid;
+    replacements_length = std::numeric_limits<int64_t>::max();
+  }
+  if (array.MayHaveNulls() || mask.MayHaveNulls() || replacements_bitmap) {
+    ARROW_ASSIGN_OR_RAISE(output->buffers[0], ctx->AllocateBitmap(array.length));
+    out_bitmap = output->buffers[0]->mutable_data();
+    output->null_count = -1;
+    if (array.MayHaveNulls()) {
+      // Copy array's bitmap
+      arrow::internal::CopyBitmap(array.buffers[0]->data(), array.offset, array.length,
+                                  out_bitmap, /*dest_offset=*/0);
+    } else {
+      // Array has no bitmap but mask/replacements do, generate an all-valid bitmap
+      std::memset(out_bitmap, 0xFF, output->buffers[0]->size());
+    }
+  } else {
+    output->null_count = 0;
+  }
+  auto copy_bitmap = [&](int64_t out_offset, int64_t in_offset, int64_t length) {
+    DCHECK(out_bitmap);
+    if (replacements.is_array()) {
+      const auto& in_data = *replacements.array();
+      const auto in_bitmap = in_data.GetValues<uint8_t>(0, /*absolute_offset=*/0);
+      arrow::internal::CopyBitmap(in_bitmap, in_data.offset + in_offset, length,
+                                  out_bitmap, out_offset);
+    } else {
+      BitUtil::SetBitsTo(out_bitmap, out_offset, length, !replacements_bitmap);
+    }
+  };
+
+  Functor::CopyData(*array.type, out_values, /*out_offset=*/0, array, /*in_offset=*/0,
+                    array.length);
+  arrow::internal::BitBlockCounter value_counter(mask_values, mask.offset, mask.length);
+  arrow::internal::OptionalBitBlockCounter valid_counter(mask_bitmap, mask.offset,
+                                                         mask.length);
+  int64_t out_offset = 0;
+  int64_t replacements_offset = 0;
+  while (out_offset < array.length) {
+    BitBlockCount value_block = value_counter.NextWord();
+    BitBlockCount valid_block = valid_counter.NextWord();
+    DCHECK_EQ(value_block.length, valid_block.length);
+    if (value_block.AllSet() && valid_block.AllSet()) {
+      // Copy from replacement array
+      if (replacements_offset + valid_block.length > replacements_length) {
+        return ReplacementArrayTooShort(replacements_offset + valid_block.length,
+                                        replacements_length);
+      }
+      Functor::CopyData(*array.type, out_values, out_offset, replacements,
+                        replacements_offset, valid_block.length);
+      if (replacements_bitmap) {
+        copy_bitmap(out_offset, replacements_offset, valid_block.length);
+      } else if (!replacements_bitmap && out_bitmap) {
+        BitUtil::SetBitsTo(out_bitmap, out_offset, valid_block.length, true);
+      }
+      replacements_offset += valid_block.length;
+    } else if ((value_block.NoneSet() && valid_block.AllSet()) || valid_block.NoneSet()) {
+      // Do nothing
+    } else {
+      for (int64_t i = 0; i < valid_block.length; ++i) {
+        if (BitUtil::GetBit(mask_values, out_offset + mask.offset + i) &&
+            (!mask_bitmap ||
+             BitUtil::GetBit(mask_bitmap, out_offset + mask.offset + i))) {
+          if (replacements_offset >= replacements_length) {

Review comment:
       can't we pre-calculate true count in `mask` and validate it with the `replacements.length()` before hand, so that we don't have to do this check for every block? :thinking: 

##########
File path: cpp/src/arrow/compute/kernels/vector_replace.cc
##########
@@ -0,0 +1,466 @@
+// 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_scalar.h"
+#include "arrow/compute/kernels/common.h"
+#include "arrow/util/bitmap_ops.h"
+
+namespace arrow {
+namespace compute {
+namespace internal {
+
+namespace {
+
+Status ReplacementArrayTooShort(int64_t expected, int64_t actual) {
+  return Status::Invalid("Replacement array must be of appropriate length (expected ",
+                         expected, " items but got ", actual, " items)");
+}
+
+// Helper to implement replace_with kernel with scalar mask for fixed-width types,
+// using callbacks to handle both bool and byte-sized types
+Status ReplaceWithScalarMask(KernelContext* ctx, const ArrayData& array,
+                             const BooleanScalar& mask, const Datum& replacements,
+                             ArrayData* output) {
+  if (!mask.is_valid) {
+    // Output = null
+    ARROW_ASSIGN_OR_RAISE(auto replacement_array,
+                          MakeArrayOfNull(array.type, array.length, ctx->memory_pool()));
+    *output = *replacement_array->data();
+    return Status::OK();
+  }
+  if (mask.value) {
+    // Output = replacement
+    if (replacements.is_scalar()) {
+      ARROW_ASSIGN_OR_RAISE(
+          auto replacement_array,
+          MakeArrayFromScalar(*replacements.scalar(), array.length, ctx->memory_pool()));
+      *output = *replacement_array->data();
+    } else {
+      auto replacement_array = replacements.array();
+      if (replacement_array->length != array.length) {
+        return ReplacementArrayTooShort(array.length, replacement_array->length);
+      }
+      *output = *replacement_array;
+    }
+  } else {
+    // Output = input
+    *output = array;
+  }
+  return Status::OK();
+}
+
+// Helper to implement replace_with kernel with array mask for fixed-width types,
+// using callbacks to handle both bool and byte-sized types and to handle
+// scalar and array replacements
+template <typename Functor>
+Status ReplaceWithArrayMask(KernelContext* ctx, const ArrayData& array,
+                            const ArrayData& mask, const Datum& replacements,
+                            ArrayData* output) {
+  uint8_t* out_bitmap = nullptr;
+  uint8_t* out_values = output->buffers[1]->mutable_data();
+  const uint8_t* mask_bitmap = mask.MayHaveNulls() ? mask.buffers[0]->data() : nullptr;
+  const uint8_t* mask_values = mask.buffers[1]->data();
+  bool replacements_bitmap;
+  int64_t replacements_length;
+  if (replacements.is_array()) {
+    replacements_bitmap = replacements.array()->MayHaveNulls();
+    replacements_length = replacements.array()->length;
+  } else {
+    replacements_bitmap = !replacements.scalar()->is_valid;
+    replacements_length = std::numeric_limits<int64_t>::max();
+  }
+  if (array.MayHaveNulls() || mask.MayHaveNulls() || replacements_bitmap) {
+    ARROW_ASSIGN_OR_RAISE(output->buffers[0], ctx->AllocateBitmap(array.length));
+    out_bitmap = output->buffers[0]->mutable_data();
+    output->null_count = -1;
+    if (array.MayHaveNulls()) {
+      // Copy array's bitmap
+      arrow::internal::CopyBitmap(array.buffers[0]->data(), array.offset, array.length,
+                                  out_bitmap, /*dest_offset=*/0);
+    } else {
+      // Array has no bitmap but mask/replacements do, generate an all-valid bitmap
+      std::memset(out_bitmap, 0xFF, output->buffers[0]->size());
+    }
+  } else {
+    output->null_count = 0;
+  }
+  auto copy_bitmap = [&](int64_t out_offset, int64_t in_offset, int64_t length) {
+    DCHECK(out_bitmap);
+    if (replacements.is_array()) {
+      const auto& in_data = *replacements.array();
+      const auto in_bitmap = in_data.GetValues<uint8_t>(0, /*absolute_offset=*/0);
+      arrow::internal::CopyBitmap(in_bitmap, in_data.offset + in_offset, length,
+                                  out_bitmap, out_offset);
+    } else {
+      BitUtil::SetBitsTo(out_bitmap, out_offset, length, !replacements_bitmap);
+    }
+  };
+
+  Functor::CopyData(*array.type, out_values, /*out_offset=*/0, array, /*in_offset=*/0,
+                    array.length);
+  arrow::internal::BitBlockCounter value_counter(mask_values, mask.offset, mask.length);

Review comment:
       can't we use `OptionalBitBlockCounter` here? :thinking: 

##########
File path: cpp/src/arrow/compute/kernels/vector_replace.cc
##########
@@ -0,0 +1,495 @@
+// 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_scalar.h"
+#include "arrow/compute/kernels/common.h"
+#include "arrow/util/bitmap_ops.h"
+
+namespace arrow {
+namespace compute {
+namespace internal {
+
+namespace {
+
+Status ReplacementArrayTooShort(int64_t expected, int64_t actual) {
+  return Status::Invalid("Replacement array must be of appropriate length (expected ",
+                         expected, " items but got ", actual, " items)");
+}
+
+// Helper to implement replace_with kernel with scalar mask for fixed-width types,
+// using callbacks to handle both bool and byte-sized types
+Status ReplaceWithScalarMask(KernelContext* ctx, const ArrayData& array,
+                             const BooleanScalar& mask, const Datum& replacements,
+                             ArrayData* output) {
+  if (!mask.is_valid) {
+    // Output = null
+    ARROW_ASSIGN_OR_RAISE(auto array,
+                          MakeArrayOfNull(array.type, array.length, ctx->memory_pool()));
+    *output = *array->data();
+    return Status::OK();
+  }
+  if (mask.value) {
+    // Output = replacement
+    if (replacements.is_scalar()) {
+      ARROW_ASSIGN_OR_RAISE(
+          auto replacement_array,
+          MakeArrayFromScalar(*replacements.scalar(), array.length, ctx->memory_pool()));
+      *output = *replacement_array->data();
+    } else {
+      auto replacement_array = replacements.array();
+      if (replacement_array->length != array.length) {
+        return ReplacementArrayTooShort(array.length, replacement_array->length);
+      }
+      *output = *replacement_array;
+    }
+  } else {
+    // Output = input
+    *output = array;
+  }
+  return Status::OK();
+}
+
+// Helper to implement replace_with kernel with array mask for fixed-width types,
+// using callbacks to handle both bool and byte-sized types and to handle
+// scalar and array replacements
+template <typename Functor>
+Status ReplaceWithArrayMask(KernelContext* ctx, const ArrayData& array,
+                            const ArrayData& mask, const Datum& replacements,
+                            ArrayData* output) {
+  ARROW_ASSIGN_OR_RAISE(output->buffers[1],
+                        Functor::AllocateData(ctx, *array.type, array.length));
+
+  uint8_t* out_bitmap = nullptr;
+  uint8_t* out_values = output->buffers[1]->mutable_data();
+  const uint8_t* mask_bitmap = mask.MayHaveNulls() ? mask.buffers[0]->data() : nullptr;
+  const uint8_t* mask_values = mask.buffers[1]->data();
+  bool replacements_bitmap;
+  int64_t replacements_length;
+  if (replacements.is_array()) {
+    replacements_bitmap = replacements.array()->MayHaveNulls();
+    replacements_length = replacements.array()->length;
+  } else {
+    replacements_bitmap = !replacements.scalar()->is_valid;
+    replacements_length = std::numeric_limits<int64_t>::max();
+  }
+  if (array.MayHaveNulls() || mask.MayHaveNulls() || replacements_bitmap) {
+    ARROW_ASSIGN_OR_RAISE(output->buffers[0], ctx->AllocateBitmap(array.length));
+    out_bitmap = output->buffers[0]->mutable_data();
+    output->null_count = -1;
+    if (array.MayHaveNulls()) {
+      arrow::internal::CopyBitmap(array.buffers[0]->data(), array.offset, array.length,
+                                  out_bitmap, /*dest_offset=*/0);
+    } else {
+      std::memset(out_bitmap, 0xFF, output->buffers[0]->size());

Review comment:
       @lidavidm I think it will be better to use `BitUtil::SetBitsTo/SetBitmap` here. It would more precisely set values upto the [`offset`, `offset+length`).

##########
File path: cpp/src/arrow/compute/kernels/vector_replace.cc
##########
@@ -0,0 +1,466 @@
+// 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_scalar.h"
+#include "arrow/compute/kernels/common.h"
+#include "arrow/util/bitmap_ops.h"
+
+namespace arrow {
+namespace compute {
+namespace internal {
+
+namespace {
+
+Status ReplacementArrayTooShort(int64_t expected, int64_t actual) {
+  return Status::Invalid("Replacement array must be of appropriate length (expected ",
+                         expected, " items but got ", actual, " items)");
+}
+
+// Helper to implement replace_with kernel with scalar mask for fixed-width types,
+// using callbacks to handle both bool and byte-sized types
+Status ReplaceWithScalarMask(KernelContext* ctx, const ArrayData& array,
+                             const BooleanScalar& mask, const Datum& replacements,
+                             ArrayData* output) {
+  if (!mask.is_valid) {
+    // Output = null
+    ARROW_ASSIGN_OR_RAISE(auto replacement_array,
+                          MakeArrayOfNull(array.type, array.length, ctx->memory_pool()));
+    *output = *replacement_array->data();
+    return Status::OK();
+  }
+  if (mask.value) {
+    // Output = replacement
+    if (replacements.is_scalar()) {
+      ARROW_ASSIGN_OR_RAISE(
+          auto replacement_array,
+          MakeArrayFromScalar(*replacements.scalar(), array.length, ctx->memory_pool()));
+      *output = *replacement_array->data();
+    } else {
+      auto replacement_array = replacements.array();
+      if (replacement_array->length != array.length) {
+        return ReplacementArrayTooShort(array.length, replacement_array->length);
+      }
+      *output = *replacement_array;
+    }
+  } else {
+    // Output = input
+    *output = array;
+  }
+  return Status::OK();
+}
+
+// Helper to implement replace_with kernel with array mask for fixed-width types,
+// using callbacks to handle both bool and byte-sized types and to handle
+// scalar and array replacements
+template <typename Functor>
+Status ReplaceWithArrayMask(KernelContext* ctx, const ArrayData& array,
+                            const ArrayData& mask, const Datum& replacements,
+                            ArrayData* output) {
+  uint8_t* out_bitmap = nullptr;
+  uint8_t* out_values = output->buffers[1]->mutable_data();
+  const uint8_t* mask_bitmap = mask.MayHaveNulls() ? mask.buffers[0]->data() : nullptr;
+  const uint8_t* mask_values = mask.buffers[1]->data();
+  bool replacements_bitmap;
+  int64_t replacements_length;
+  if (replacements.is_array()) {
+    replacements_bitmap = replacements.array()->MayHaveNulls();
+    replacements_length = replacements.array()->length;
+  } else {
+    replacements_bitmap = !replacements.scalar()->is_valid;
+    replacements_length = std::numeric_limits<int64_t>::max();
+  }
+  if (array.MayHaveNulls() || mask.MayHaveNulls() || replacements_bitmap) {
+    ARROW_ASSIGN_OR_RAISE(output->buffers[0], ctx->AllocateBitmap(array.length));
+    out_bitmap = output->buffers[0]->mutable_data();
+    output->null_count = -1;
+    if (array.MayHaveNulls()) {
+      // Copy array's bitmap
+      arrow::internal::CopyBitmap(array.buffers[0]->data(), array.offset, array.length,
+                                  out_bitmap, /*dest_offset=*/0);
+    } else {
+      // Array has no bitmap but mask/replacements do, generate an all-valid bitmap
+      std::memset(out_bitmap, 0xFF, output->buffers[0]->size());
+    }
+  } else {
+    output->null_count = 0;
+  }
+  auto copy_bitmap = [&](int64_t out_offset, int64_t in_offset, int64_t length) {
+    DCHECK(out_bitmap);
+    if (replacements.is_array()) {
+      const auto& in_data = *replacements.array();
+      const auto in_bitmap = in_data.GetValues<uint8_t>(0, /*absolute_offset=*/0);
+      arrow::internal::CopyBitmap(in_bitmap, in_data.offset + in_offset, length,
+                                  out_bitmap, out_offset);
+    } else {
+      BitUtil::SetBitsTo(out_bitmap, out_offset, length, !replacements_bitmap);
+    }
+  };
+
+  Functor::CopyData(*array.type, out_values, /*out_offset=*/0, array, /*in_offset=*/0,
+                    array.length);
+  arrow::internal::BitBlockCounter value_counter(mask_values, mask.offset, mask.length);
+  arrow::internal::OptionalBitBlockCounter valid_counter(mask_bitmap, mask.offset,
+                                                         mask.length);
+  int64_t out_offset = 0;
+  int64_t replacements_offset = 0;
+  while (out_offset < array.length) {
+    BitBlockCount value_block = value_counter.NextWord();
+    BitBlockCount valid_block = valid_counter.NextWord();
+    DCHECK_EQ(value_block.length, valid_block.length);
+    if (value_block.AllSet() && valid_block.AllSet()) {
+      // Copy from replacement array
+      if (replacements_offset + valid_block.length > replacements_length) {
+        return ReplacementArrayTooShort(replacements_offset + valid_block.length,
+                                        replacements_length);
+      }
+      Functor::CopyData(*array.type, out_values, out_offset, replacements,
+                        replacements_offset, valid_block.length);
+      if (replacements_bitmap) {
+        copy_bitmap(out_offset, replacements_offset, valid_block.length);
+      } else if (!replacements_bitmap && out_bitmap) {
+        BitUtil::SetBitsTo(out_bitmap, out_offset, valid_block.length, true);
+      }
+      replacements_offset += valid_block.length;
+    } else if ((value_block.NoneSet() && valid_block.AllSet()) || valid_block.NoneSet()) {
+      // Do nothing
+    } else {
+      for (int64_t i = 0; i < valid_block.length; ++i) {
+        if (BitUtil::GetBit(mask_values, out_offset + mask.offset + i) &&
+            (!mask_bitmap ||
+             BitUtil::GetBit(mask_bitmap, out_offset + mask.offset + i))) {
+          if (replacements_offset >= replacements_length) {
+            return ReplacementArrayTooShort(replacements_offset + 1, replacements_length);
+          }
+          Functor::CopyData(*array.type, out_values, out_offset + i, replacements,
+                            replacements_offset,
+                            /*length=*/1);
+          if (replacements_bitmap) {
+            copy_bitmap(out_offset + i, replacements_offset, 1);
+          }
+          replacements_offset++;
+        }
+      }
+    }
+    out_offset += valid_block.length;
+  }
+
+  if (mask.MayHaveNulls()) {
+    arrow::internal::BitmapAnd(out_bitmap, /*left_offset=*/0, mask.buffers[0]->data(),
+                               mask.offset, array.length,
+                               /*out_offset=*/0, out_bitmap);
+  }
+  return Status::OK();
+}
+
+template <typename Type, typename Enable = void>
+struct ReplaceWithMask {};
+
+template <typename Type>
+struct ReplaceWithMask<Type, enable_if_number<Type>> {
+  using T = typename TypeTraits<Type>::CType;
+
+  static void CopyData(const DataType&, uint8_t* out, const int64_t out_offset,
+                       const Datum& in, const int64_t in_offset, const int64_t length) {
+    if (in.is_array()) {
+      const auto& in_data = *in.array();
+      const auto in_arr =
+          in_data.GetValues<uint8_t>(1, (in_offset + in_data.offset) * sizeof(T));
+      std::memcpy(out + (out_offset * sizeof(T)), in_arr, length * sizeof(T));
+    } else {
+      T* begin = reinterpret_cast<T*>(out + (out_offset * sizeof(T)));
+      T* end = begin + length;
+      std::fill(begin, end, UnboxScalar<Type>::Unbox(*in.scalar()));
+    }
+  }
+
+  static Status ExecScalarMask(KernelContext* ctx, const ArrayData& array,
+                               const BooleanScalar& mask, const Datum& replacements,
+                               ArrayData* output) {
+    return ReplaceWithScalarMask(ctx, array, mask, replacements, output);
+  }
+
+  static Status ExecArrayMask(KernelContext* ctx, const ArrayData& array,
+                              const ArrayData& mask, const Datum& replacements,
+                              ArrayData* output) {
+    return ReplaceWithArrayMask<ReplaceWithMask<Type>>(ctx, array, mask, replacements,
+                                                       output);
+  }
+};
+
+template <typename Type>
+struct ReplaceWithMask<Type, enable_if_boolean<Type>> {
+  static void CopyData(const DataType&, uint8_t* out, const int64_t out_offset,
+                       const Datum& in, const int64_t in_offset, const int64_t length) {
+    if (in.is_array()) {
+      const auto& in_data = *in.array();
+      const auto in_arr = in_data.GetValues<uint8_t>(1, /*absolute_offset=*/0);
+      arrow::internal::CopyBitmap(in_arr, in_offset + in_data.offset, length, out,
+                                  out_offset);
+    } else {
+      BitUtil::SetBitsTo(out, out_offset, length, in.scalar()->is_valid);
+    }
+  }
+
+  static Status ExecScalarMask(KernelContext* ctx, const ArrayData& array,
+                               const BooleanScalar& mask, const Datum& replacements,
+                               ArrayData* output) {
+    return ReplaceWithScalarMask(ctx, array, mask, replacements, output);
+  }
+  static Status ExecArrayMask(KernelContext* ctx, const ArrayData& array,
+                              const ArrayData& mask, const Datum& replacements,
+                              ArrayData* output) {
+    return ReplaceWithArrayMask<ReplaceWithMask<Type>>(ctx, array, mask, replacements,
+                                                       output);
+  }
+};
+
+template <typename Type>
+struct ReplaceWithMask<Type, enable_if_same<Type, FixedSizeBinaryType>> {
+  static void CopyData(const DataType& ty, uint8_t* out, const int64_t out_offset,
+                       const Datum& in, const int64_t in_offset, const int64_t length) {
+    const int32_t width = checked_cast<const FixedSizeBinaryType&>(ty).byte_width();
+    uint8_t* begin = out + (out_offset * width);
+    if (in.is_array()) {
+      const auto& in_data = *in.array();
+      const auto in_arr =
+          in_data.GetValues<uint8_t>(1, (in_offset + in_data.offset) * width);
+      std::memcpy(begin, in_arr, length * width);
+    } else {
+      const FixedSizeBinaryScalar& scalar =
+          checked_cast<const FixedSizeBinaryScalar&>(*in.scalar());
+      // Null scalar may have null value buffer
+      if (!scalar.value) return;
+      const Buffer& buffer = *scalar.value;
+      const uint8_t* value = buffer.data();
+      DCHECK_GE(buffer.size(), width);
+      for (int i = 0; i < length; i++) {
+        std::memcpy(begin, value, width);
+        begin += width;
+      }
+    }
+  }
+
+  static Status ExecScalarMask(KernelContext* ctx, const ArrayData& array,
+                               const BooleanScalar& mask, const Datum& replacements,
+                               ArrayData* output) {
+    return ReplaceWithScalarMask(ctx, array, mask, replacements, output);
+  }
+
+  static Status ExecArrayMask(KernelContext* ctx, const ArrayData& array,
+                              const ArrayData& mask, const Datum& replacements,
+                              ArrayData* output) {
+    return ReplaceWithArrayMask<ReplaceWithMask<Type>>(ctx, array, mask, replacements,
+                                                       output);
+  }
+};
+
+template <typename Type>
+struct ReplaceWithMask<Type, enable_if_decimal<Type>> {
+  using ScalarType = typename TypeTraits<Type>::ScalarType;
+  static void CopyData(const DataType& ty, uint8_t* out, const int64_t out_offset,
+                       const Datum& in, const int64_t in_offset, const int64_t length) {
+    const int32_t width = checked_cast<const FixedSizeBinaryType&>(ty).byte_width();
+    uint8_t* begin = out + (out_offset * width);
+    if (in.is_array()) {
+      const auto& in_data = *in.array();
+      const auto in_arr =
+          in_data.GetValues<uint8_t>(1, (in_offset + in_data.offset) * width);
+      std::memcpy(begin, in_arr, length * width);
+    } else {
+      const ScalarType& scalar = checked_cast<const ScalarType&>(*in.scalar());
+      const auto value = scalar.value.ToBytes();
+      for (int i = 0; i < length; i++) {
+        std::memcpy(begin, value.data(), width);
+        begin += width;
+      }
+    }
+  }
+
+  static Status ExecScalarMask(KernelContext* ctx, const ArrayData& array,
+                               const BooleanScalar& mask, const Datum& replacements,
+                               ArrayData* output) {
+    return ReplaceWithScalarMask(ctx, array, mask, replacements, output);
+  }
+
+  static Status ExecArrayMask(KernelContext* ctx, const ArrayData& array,
+                              const ArrayData& mask, const Datum& replacements,
+                              ArrayData* output) {
+    return ReplaceWithArrayMask<ReplaceWithMask<Type>>(ctx, array, mask, replacements,
+                                                       output);
+  }
+};
+
+template <typename Type>
+struct ReplaceWithMask<Type, enable_if_null<Type>> {
+  static Status ExecScalarMask(KernelContext* ctx, const ArrayData& array,
+                               const BooleanScalar& mask, const Datum& replacements,
+                               ArrayData* output) {
+    *output = array;
+    return Status::OK();
+  }
+  static Status ExecArrayMask(KernelContext* ctx, const ArrayData& array,
+                              const ArrayData& mask, const Datum& replacements,
+                              ArrayData* output) {
+    *output = array;
+    return Status::OK();
+  }
+};
+
+template <typename Type>
+struct ReplaceWithMask<Type, enable_if_base_binary<Type>> {
+  using offset_type = typename Type::offset_type;
+  using BuilderType = typename TypeTraits<Type>::BuilderType;
+
+  static Status ExecScalarMask(KernelContext* ctx, const ArrayData& array,
+                               const BooleanScalar& mask, const Datum& replacements,
+                               ArrayData* output) {
+    return ReplaceWithScalarMask(ctx, array, mask, replacements, output);
+  }
+  static Status ExecArrayMask(KernelContext* ctx, const ArrayData& array,
+                              const ArrayData& mask, const Datum& replacements,
+                              ArrayData* output) {
+    BuilderType builder(array.type, ctx->memory_pool());
+    RETURN_NOT_OK(builder.Reserve(array.length));

Review comment:
       nit. `array.length + 1`? :thinking: 




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

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

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



[GitHub] [arrow] lidavidm commented on a change in pull request #10412: ARROW-9430: [C++] Implement replace_with_mask kernel

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



##########
File path: cpp/src/arrow/compute/kernels/vector_replace.cc
##########
@@ -0,0 +1,466 @@
+// 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_scalar.h"
+#include "arrow/compute/kernels/common.h"
+#include "arrow/util/bitmap_ops.h"
+
+namespace arrow {
+namespace compute {
+namespace internal {
+
+namespace {
+
+Status ReplacementArrayTooShort(int64_t expected, int64_t actual) {
+  return Status::Invalid("Replacement array must be of appropriate length (expected ",
+                         expected, " items but got ", actual, " items)");
+}
+
+// Helper to implement replace_with kernel with scalar mask for fixed-width types,
+// using callbacks to handle both bool and byte-sized types
+Status ReplaceWithScalarMask(KernelContext* ctx, const ArrayData& array,
+                             const BooleanScalar& mask, const Datum& replacements,
+                             ArrayData* output) {
+  if (!mask.is_valid) {
+    // Output = null
+    ARROW_ASSIGN_OR_RAISE(auto replacement_array,
+                          MakeArrayOfNull(array.type, array.length, ctx->memory_pool()));
+    *output = *replacement_array->data();
+    return Status::OK();
+  }
+  if (mask.value) {
+    // Output = replacement
+    if (replacements.is_scalar()) {
+      ARROW_ASSIGN_OR_RAISE(
+          auto replacement_array,
+          MakeArrayFromScalar(*replacements.scalar(), array.length, ctx->memory_pool()));
+      *output = *replacement_array->data();
+    } else {
+      auto replacement_array = replacements.array();
+      if (replacement_array->length != array.length) {
+        return ReplacementArrayTooShort(array.length, replacement_array->length);
+      }
+      *output = *replacement_array;
+    }
+  } else {
+    // Output = input
+    *output = array;
+  }
+  return Status::OK();
+}
+
+// Helper to implement replace_with kernel with array mask for fixed-width types,
+// using callbacks to handle both bool and byte-sized types and to handle
+// scalar and array replacements
+template <typename Functor>
+Status ReplaceWithArrayMask(KernelContext* ctx, const ArrayData& array,
+                            const ArrayData& mask, const Datum& replacements,
+                            ArrayData* output) {
+  uint8_t* out_bitmap = nullptr;
+  uint8_t* out_values = output->buffers[1]->mutable_data();
+  const uint8_t* mask_bitmap = mask.MayHaveNulls() ? mask.buffers[0]->data() : nullptr;
+  const uint8_t* mask_values = mask.buffers[1]->data();
+  bool replacements_bitmap;
+  int64_t replacements_length;
+  if (replacements.is_array()) {
+    replacements_bitmap = replacements.array()->MayHaveNulls();
+    replacements_length = replacements.array()->length;
+  } else {
+    replacements_bitmap = !replacements.scalar()->is_valid;
+    replacements_length = std::numeric_limits<int64_t>::max();
+  }
+  if (array.MayHaveNulls() || mask.MayHaveNulls() || replacements_bitmap) {
+    ARROW_ASSIGN_OR_RAISE(output->buffers[0], ctx->AllocateBitmap(array.length));
+    out_bitmap = output->buffers[0]->mutable_data();
+    output->null_count = -1;
+    if (array.MayHaveNulls()) {
+      // Copy array's bitmap
+      arrow::internal::CopyBitmap(array.buffers[0]->data(), array.offset, array.length,
+                                  out_bitmap, /*dest_offset=*/0);
+    } else {
+      // Array has no bitmap but mask/replacements do, generate an all-valid bitmap
+      std::memset(out_bitmap, 0xFF, output->buffers[0]->size());
+    }
+  } else {
+    output->null_count = 0;
+  }
+  auto copy_bitmap = [&](int64_t out_offset, int64_t in_offset, int64_t length) {
+    DCHECK(out_bitmap);
+    if (replacements.is_array()) {
+      const auto& in_data = *replacements.array();
+      const auto in_bitmap = in_data.GetValues<uint8_t>(0, /*absolute_offset=*/0);
+      arrow::internal::CopyBitmap(in_bitmap, in_data.offset + in_offset, length,
+                                  out_bitmap, out_offset);
+    } else {
+      BitUtil::SetBitsTo(out_bitmap, out_offset, length, !replacements_bitmap);
+    }
+  };
+
+  Functor::CopyData(*array.type, out_values, /*out_offset=*/0, array, /*in_offset=*/0,
+                    array.length);
+  arrow::internal::BitBlockCounter value_counter(mask_values, mask.offset, mask.length);

Review comment:
       mask_values is never null, it's the values buffer of the mask argument.

##########
File path: cpp/src/arrow/compute/kernels/vector_replace.cc
##########
@@ -0,0 +1,466 @@
+// 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_scalar.h"
+#include "arrow/compute/kernels/common.h"
+#include "arrow/util/bitmap_ops.h"
+
+namespace arrow {
+namespace compute {
+namespace internal {
+
+namespace {
+
+Status ReplacementArrayTooShort(int64_t expected, int64_t actual) {
+  return Status::Invalid("Replacement array must be of appropriate length (expected ",
+                         expected, " items but got ", actual, " items)");
+}
+
+// Helper to implement replace_with kernel with scalar mask for fixed-width types,
+// using callbacks to handle both bool and byte-sized types
+Status ReplaceWithScalarMask(KernelContext* ctx, const ArrayData& array,
+                             const BooleanScalar& mask, const Datum& replacements,
+                             ArrayData* output) {
+  if (!mask.is_valid) {
+    // Output = null
+    ARROW_ASSIGN_OR_RAISE(auto replacement_array,
+                          MakeArrayOfNull(array.type, array.length, ctx->memory_pool()));
+    *output = *replacement_array->data();
+    return Status::OK();
+  }
+  if (mask.value) {
+    // Output = replacement
+    if (replacements.is_scalar()) {
+      ARROW_ASSIGN_OR_RAISE(
+          auto replacement_array,
+          MakeArrayFromScalar(*replacements.scalar(), array.length, ctx->memory_pool()));
+      *output = *replacement_array->data();
+    } else {
+      auto replacement_array = replacements.array();
+      if (replacement_array->length != array.length) {
+        return ReplacementArrayTooShort(array.length, replacement_array->length);
+      }
+      *output = *replacement_array;
+    }
+  } else {
+    // Output = input
+    *output = array;
+  }
+  return Status::OK();
+}
+
+// Helper to implement replace_with kernel with array mask for fixed-width types,
+// using callbacks to handle both bool and byte-sized types and to handle
+// scalar and array replacements
+template <typename Functor>
+Status ReplaceWithArrayMask(KernelContext* ctx, const ArrayData& array,
+                            const ArrayData& mask, const Datum& replacements,
+                            ArrayData* output) {
+  uint8_t* out_bitmap = nullptr;
+  uint8_t* out_values = output->buffers[1]->mutable_data();
+  const uint8_t* mask_bitmap = mask.MayHaveNulls() ? mask.buffers[0]->data() : nullptr;
+  const uint8_t* mask_values = mask.buffers[1]->data();
+  bool replacements_bitmap;
+  int64_t replacements_length;
+  if (replacements.is_array()) {
+    replacements_bitmap = replacements.array()->MayHaveNulls();
+    replacements_length = replacements.array()->length;
+  } else {
+    replacements_bitmap = !replacements.scalar()->is_valid;
+    replacements_length = std::numeric_limits<int64_t>::max();
+  }
+  if (array.MayHaveNulls() || mask.MayHaveNulls() || replacements_bitmap) {
+    ARROW_ASSIGN_OR_RAISE(output->buffers[0], ctx->AllocateBitmap(array.length));
+    out_bitmap = output->buffers[0]->mutable_data();
+    output->null_count = -1;
+    if (array.MayHaveNulls()) {
+      // Copy array's bitmap
+      arrow::internal::CopyBitmap(array.buffers[0]->data(), array.offset, array.length,
+                                  out_bitmap, /*dest_offset=*/0);
+    } else {
+      // Array has no bitmap but mask/replacements do, generate an all-valid bitmap
+      std::memset(out_bitmap, 0xFF, output->buffers[0]->size());
+    }
+  } else {
+    output->null_count = 0;
+  }
+  auto copy_bitmap = [&](int64_t out_offset, int64_t in_offset, int64_t length) {
+    DCHECK(out_bitmap);
+    if (replacements.is_array()) {
+      const auto& in_data = *replacements.array();
+      const auto in_bitmap = in_data.GetValues<uint8_t>(0, /*absolute_offset=*/0);
+      arrow::internal::CopyBitmap(in_bitmap, in_data.offset + in_offset, length,
+                                  out_bitmap, out_offset);
+    } else {
+      BitUtil::SetBitsTo(out_bitmap, out_offset, length, !replacements_bitmap);
+    }
+  };
+
+  Functor::CopyData(*array.type, out_values, /*out_offset=*/0, array, /*in_offset=*/0,
+                    array.length);
+  arrow::internal::BitBlockCounter value_counter(mask_values, mask.offset, mask.length);
+  arrow::internal::OptionalBitBlockCounter valid_counter(mask_bitmap, mask.offset,
+                                                         mask.length);
+  int64_t out_offset = 0;
+  int64_t replacements_offset = 0;
+  while (out_offset < array.length) {
+    BitBlockCount value_block = value_counter.NextWord();
+    BitBlockCount valid_block = valid_counter.NextWord();
+    DCHECK_EQ(value_block.length, valid_block.length);
+    if (value_block.AllSet() && valid_block.AllSet()) {
+      // Copy from replacement array
+      if (replacements_offset + valid_block.length > replacements_length) {
+        return ReplacementArrayTooShort(replacements_offset + valid_block.length,
+                                        replacements_length);
+      }
+      Functor::CopyData(*array.type, out_values, out_offset, replacements,
+                        replacements_offset, valid_block.length);
+      if (replacements_bitmap) {
+        copy_bitmap(out_offset, replacements_offset, valid_block.length);
+      } else if (!replacements_bitmap && out_bitmap) {
+        BitUtil::SetBitsTo(out_bitmap, out_offset, valid_block.length, true);
+      }
+      replacements_offset += valid_block.length;
+    } else if ((value_block.NoneSet() && valid_block.AllSet()) || valid_block.NoneSet()) {
+      // Do nothing
+    } else {
+      for (int64_t i = 0; i < valid_block.length; ++i) {
+        if (BitUtil::GetBit(mask_values, out_offset + mask.offset + i) &&
+            (!mask_bitmap ||
+             BitUtil::GetBit(mask_bitmap, out_offset + mask.offset + i))) {
+          if (replacements_offset >= replacements_length) {
+            return ReplacementArrayTooShort(replacements_offset + 1, replacements_length);
+          }
+          Functor::CopyData(*array.type, out_values, out_offset + i, replacements,
+                            replacements_offset,
+                            /*length=*/1);
+          if (replacements_bitmap) {
+            copy_bitmap(out_offset + i, replacements_offset, 1);
+          }
+          replacements_offset++;
+        }
+      }
+    }
+    out_offset += valid_block.length;
+  }
+
+  if (mask.MayHaveNulls()) {
+    arrow::internal::BitmapAnd(out_bitmap, /*left_offset=*/0, mask.buffers[0]->data(),
+                               mask.offset, array.length,
+                               /*out_offset=*/0, out_bitmap);
+  }
+  return Status::OK();
+}
+
+template <typename Type, typename Enable = void>
+struct ReplaceWithMask {};
+
+template <typename Type>
+struct ReplaceWithMask<Type, enable_if_number<Type>> {
+  using T = typename TypeTraits<Type>::CType;
+
+  static void CopyData(const DataType&, uint8_t* out, const int64_t out_offset,
+                       const Datum& in, const int64_t in_offset, const int64_t length) {
+    if (in.is_array()) {
+      const auto& in_data = *in.array();
+      const auto in_arr =
+          in_data.GetValues<uint8_t>(1, (in_offset + in_data.offset) * sizeof(T));
+      std::memcpy(out + (out_offset * sizeof(T)), in_arr, length * sizeof(T));
+    } else {
+      T* begin = reinterpret_cast<T*>(out + (out_offset * sizeof(T)));
+      T* end = begin + length;
+      std::fill(begin, end, UnboxScalar<Type>::Unbox(*in.scalar()));
+    }
+  }
+
+  static Status ExecScalarMask(KernelContext* ctx, const ArrayData& array,
+                               const BooleanScalar& mask, const Datum& replacements,
+                               ArrayData* output) {
+    return ReplaceWithScalarMask(ctx, array, mask, replacements, output);
+  }
+
+  static Status ExecArrayMask(KernelContext* ctx, const ArrayData& array,
+                              const ArrayData& mask, const Datum& replacements,
+                              ArrayData* output) {
+    return ReplaceWithArrayMask<ReplaceWithMask<Type>>(ctx, array, mask, replacements,
+                                                       output);
+  }
+};
+
+template <typename Type>
+struct ReplaceWithMask<Type, enable_if_boolean<Type>> {
+  static void CopyData(const DataType&, uint8_t* out, const int64_t out_offset,
+                       const Datum& in, const int64_t in_offset, const int64_t length) {
+    if (in.is_array()) {
+      const auto& in_data = *in.array();
+      const auto in_arr = in_data.GetValues<uint8_t>(1, /*absolute_offset=*/0);
+      arrow::internal::CopyBitmap(in_arr, in_offset + in_data.offset, length, out,
+                                  out_offset);
+    } else {
+      BitUtil::SetBitsTo(out, out_offset, length, in.scalar()->is_valid);
+    }
+  }
+
+  static Status ExecScalarMask(KernelContext* ctx, const ArrayData& array,
+                               const BooleanScalar& mask, const Datum& replacements,
+                               ArrayData* output) {
+    return ReplaceWithScalarMask(ctx, array, mask, replacements, output);
+  }
+  static Status ExecArrayMask(KernelContext* ctx, const ArrayData& array,
+                              const ArrayData& mask, const Datum& replacements,
+                              ArrayData* output) {
+    return ReplaceWithArrayMask<ReplaceWithMask<Type>>(ctx, array, mask, replacements,
+                                                       output);
+  }
+};
+
+template <typename Type>
+struct ReplaceWithMask<Type, enable_if_same<Type, FixedSizeBinaryType>> {
+  static void CopyData(const DataType& ty, uint8_t* out, const int64_t out_offset,
+                       const Datum& in, const int64_t in_offset, const int64_t length) {
+    const int32_t width = checked_cast<const FixedSizeBinaryType&>(ty).byte_width();
+    uint8_t* begin = out + (out_offset * width);
+    if (in.is_array()) {
+      const auto& in_data = *in.array();
+      const auto in_arr =
+          in_data.GetValues<uint8_t>(1, (in_offset + in_data.offset) * width);
+      std::memcpy(begin, in_arr, length * width);
+    } else {
+      const FixedSizeBinaryScalar& scalar =
+          checked_cast<const FixedSizeBinaryScalar&>(*in.scalar());
+      // Null scalar may have null value buffer
+      if (!scalar.value) return;
+      const Buffer& buffer = *scalar.value;
+      const uint8_t* value = buffer.data();
+      DCHECK_GE(buffer.size(), width);
+      for (int i = 0; i < length; i++) {
+        std::memcpy(begin, value, width);
+        begin += width;
+      }
+    }
+  }
+
+  static Status ExecScalarMask(KernelContext* ctx, const ArrayData& array,
+                               const BooleanScalar& mask, const Datum& replacements,
+                               ArrayData* output) {
+    return ReplaceWithScalarMask(ctx, array, mask, replacements, output);
+  }
+
+  static Status ExecArrayMask(KernelContext* ctx, const ArrayData& array,
+                              const ArrayData& mask, const Datum& replacements,
+                              ArrayData* output) {
+    return ReplaceWithArrayMask<ReplaceWithMask<Type>>(ctx, array, mask, replacements,
+                                                       output);
+  }
+};
+
+template <typename Type>
+struct ReplaceWithMask<Type, enable_if_decimal<Type>> {
+  using ScalarType = typename TypeTraits<Type>::ScalarType;
+  static void CopyData(const DataType& ty, uint8_t* out, const int64_t out_offset,
+                       const Datum& in, const int64_t in_offset, const int64_t length) {
+    const int32_t width = checked_cast<const FixedSizeBinaryType&>(ty).byte_width();
+    uint8_t* begin = out + (out_offset * width);
+    if (in.is_array()) {
+      const auto& in_data = *in.array();
+      const auto in_arr =
+          in_data.GetValues<uint8_t>(1, (in_offset + in_data.offset) * width);
+      std::memcpy(begin, in_arr, length * width);
+    } else {
+      const ScalarType& scalar = checked_cast<const ScalarType&>(*in.scalar());
+      const auto value = scalar.value.ToBytes();
+      for (int i = 0; i < length; i++) {
+        std::memcpy(begin, value.data(), width);
+        begin += width;
+      }
+    }
+  }
+
+  static Status ExecScalarMask(KernelContext* ctx, const ArrayData& array,
+                               const BooleanScalar& mask, const Datum& replacements,
+                               ArrayData* output) {
+    return ReplaceWithScalarMask(ctx, array, mask, replacements, output);
+  }
+
+  static Status ExecArrayMask(KernelContext* ctx, const ArrayData& array,
+                              const ArrayData& mask, const Datum& replacements,
+                              ArrayData* output) {
+    return ReplaceWithArrayMask<ReplaceWithMask<Type>>(ctx, array, mask, replacements,
+                                                       output);
+  }
+};
+
+template <typename Type>
+struct ReplaceWithMask<Type, enable_if_null<Type>> {
+  static Status ExecScalarMask(KernelContext* ctx, const ArrayData& array,
+                               const BooleanScalar& mask, const Datum& replacements,
+                               ArrayData* output) {
+    *output = array;
+    return Status::OK();
+  }
+  static Status ExecArrayMask(KernelContext* ctx, const ArrayData& array,
+                              const ArrayData& mask, const Datum& replacements,
+                              ArrayData* output) {
+    *output = array;
+    return Status::OK();
+  }
+};
+
+template <typename Type>
+struct ReplaceWithMask<Type, enable_if_base_binary<Type>> {
+  using offset_type = typename Type::offset_type;
+  using BuilderType = typename TypeTraits<Type>::BuilderType;
+
+  static Status ExecScalarMask(KernelContext* ctx, const ArrayData& array,
+                               const BooleanScalar& mask, const Datum& replacements,
+                               ArrayData* output) {
+    return ReplaceWithScalarMask(ctx, array, mask, replacements, output);
+  }
+  static Status ExecArrayMask(KernelContext* ctx, const ArrayData& array,
+                              const ArrayData& mask, const Datum& replacements,
+                              ArrayData* output) {
+    BuilderType builder(array.type, ctx->memory_pool());
+    RETURN_NOT_OK(builder.Reserve(array.length));

Review comment:
       Why +1? The builder will reserve the correct number of offsets.

##########
File path: cpp/src/arrow/compute/kernels/vector_replace.cc
##########
@@ -0,0 +1,495 @@
+// 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_scalar.h"
+#include "arrow/compute/kernels/common.h"
+#include "arrow/util/bitmap_ops.h"
+
+namespace arrow {
+namespace compute {
+namespace internal {
+
+namespace {
+
+Status ReplacementArrayTooShort(int64_t expected, int64_t actual) {
+  return Status::Invalid("Replacement array must be of appropriate length (expected ",
+                         expected, " items but got ", actual, " items)");
+}
+
+// Helper to implement replace_with kernel with scalar mask for fixed-width types,
+// using callbacks to handle both bool and byte-sized types
+Status ReplaceWithScalarMask(KernelContext* ctx, const ArrayData& array,
+                             const BooleanScalar& mask, const Datum& replacements,
+                             ArrayData* output) {
+  if (!mask.is_valid) {
+    // Output = null
+    ARROW_ASSIGN_OR_RAISE(auto array,
+                          MakeArrayOfNull(array.type, array.length, ctx->memory_pool()));
+    *output = *array->data();
+    return Status::OK();
+  }
+  if (mask.value) {
+    // Output = replacement
+    if (replacements.is_scalar()) {
+      ARROW_ASSIGN_OR_RAISE(
+          auto replacement_array,
+          MakeArrayFromScalar(*replacements.scalar(), array.length, ctx->memory_pool()));
+      *output = *replacement_array->data();
+    } else {
+      auto replacement_array = replacements.array();
+      if (replacement_array->length != array.length) {
+        return ReplacementArrayTooShort(array.length, replacement_array->length);
+      }
+      *output = *replacement_array;
+    }
+  } else {
+    // Output = input
+    *output = array;
+  }
+  return Status::OK();
+}
+
+// Helper to implement replace_with kernel with array mask for fixed-width types,
+// using callbacks to handle both bool and byte-sized types and to handle
+// scalar and array replacements
+template <typename Functor>
+Status ReplaceWithArrayMask(KernelContext* ctx, const ArrayData& array,
+                            const ArrayData& mask, const Datum& replacements,
+                            ArrayData* output) {
+  ARROW_ASSIGN_OR_RAISE(output->buffers[1],
+                        Functor::AllocateData(ctx, *array.type, array.length));
+
+  uint8_t* out_bitmap = nullptr;
+  uint8_t* out_values = output->buffers[1]->mutable_data();
+  const uint8_t* mask_bitmap = mask.MayHaveNulls() ? mask.buffers[0]->data() : nullptr;
+  const uint8_t* mask_values = mask.buffers[1]->data();
+  bool replacements_bitmap;
+  int64_t replacements_length;
+  if (replacements.is_array()) {
+    replacements_bitmap = replacements.array()->MayHaveNulls();
+    replacements_length = replacements.array()->length;
+  } else {
+    replacements_bitmap = !replacements.scalar()->is_valid;
+    replacements_length = std::numeric_limits<int64_t>::max();
+  }
+  if (array.MayHaveNulls() || mask.MayHaveNulls() || replacements_bitmap) {
+    ARROW_ASSIGN_OR_RAISE(output->buffers[0], ctx->AllocateBitmap(array.length));
+    out_bitmap = output->buffers[0]->mutable_data();
+    output->null_count = -1;
+    if (array.MayHaveNulls()) {
+      arrow::internal::CopyBitmap(array.buffers[0]->data(), array.offset, array.length,
+                                  out_bitmap, /*dest_offset=*/0);
+    } else {
+      std::memset(out_bitmap, 0xFF, output->buffers[0]->size());

Review comment:
       We might over-allocate a bit and we should make sure any such bytes are initialized. Also I'd guess it's faster to just bit-blit a constant value over a buffer rather than try to set bits precisely. (SetBitsTo i s quite a bit more complicated.)




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

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

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



[GitHub] [arrow] lidavidm commented on a change in pull request #10412: ARROW-9430: [C++] Implement replace_with_mask kernel

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



##########
File path: cpp/src/arrow/compute/kernels/vector_replace.cc
##########
@@ -0,0 +1,494 @@
+// 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_scalar.h"
+#include "arrow/compute/kernels/common.h"
+#include "arrow/util/bitmap_ops.h"
+
+namespace arrow {
+namespace compute {
+namespace internal {
+
+namespace {
+
+Status ReplacementArrayTooShort(int64_t expected, int64_t actual) {
+  return Status::Invalid("Replacement array must be of appropriate length (expected ",
+                         expected, " items but got ", actual, " items)");
+}
+
+// Helper to implement replace_with kernel with scalar mask for fixed-width types,
+// using callbacks to handle both bool and byte-sized types
+Status ReplaceWithScalarMask(KernelContext* ctx, const ArrayData& array,
+                             const BooleanScalar& mask, const Datum& replacements,
+                             ArrayData* output) {
+  if (!mask.is_valid) {
+    // Output = null
+    ARROW_ASSIGN_OR_RAISE(auto array,
+                          MakeArrayOfNull(array.type, array.length, ctx->memory_pool()));
+    *output = *array->data();
+    return Status::OK();
+  }
+  if (mask.value) {
+    // Output = replacement
+    if (replacements.is_scalar()) {
+      ARROW_ASSIGN_OR_RAISE(
+          auto replacement_array,
+          MakeArrayFromScalar(*replacements.scalar(), array.length, ctx->memory_pool()));
+      *output = *replacement_array->data();
+    } else {
+      auto replacement_array = replacements.array();
+      if (replacement_array->length != array.length) {
+        return ReplacementArrayTooShort(array.length, replacement_array->length);
+      }
+      *output = *replacement_array;
+    }
+  } else {
+    // Output = input
+    *output = array;
+  }
+  return Status::OK();
+}
+
+// Helper to implement replace_with kernel with array mask for fixed-width types,
+// using callbacks to handle both bool and byte-sized types and to handle
+// scalar and array replacements
+template <typename Functor>
+Status ReplaceWithArrayMask(KernelContext* ctx, const ArrayData& array,
+                            const ArrayData& mask, const Datum& replacements,
+                            ArrayData* output) {
+  ARROW_ASSIGN_OR_RAISE(output->buffers[1],
+                        Functor::AllocateData(ctx, *array.type, array.length));
+
+  uint8_t* out_bitmap = nullptr;
+  uint8_t* out_values = output->buffers[1]->mutable_data();
+  const uint8_t* mask_bitmap = mask.MayHaveNulls() ? mask.buffers[0]->data() : nullptr;
+  const uint8_t* mask_values = mask.buffers[1]->data();
+  bool replacements_bitmap;
+  int64_t replacements_length;
+  if (replacements.is_array()) {
+    replacements_bitmap = replacements.array()->MayHaveNulls();
+    replacements_length = replacements.array()->length;
+  } else {
+    replacements_bitmap = !replacements.scalar()->is_valid;
+    replacements_length = std::numeric_limits<int64_t>::max();
+  }
+  if (array.MayHaveNulls() || mask.MayHaveNulls() || replacements_bitmap) {
+    ARROW_ASSIGN_OR_RAISE(output->buffers[0], ctx->AllocateBitmap(array.length));
+    out_bitmap = output->buffers[0]->mutable_data();
+    output->null_count = -1;
+    if (array.MayHaveNulls()) {
+      arrow::internal::CopyBitmap(array.buffers[0]->data(), array.offset, array.length,
+                                  out_bitmap, /*dest_offset=*/0);
+    } else {
+      std::memset(out_bitmap, 0xFF, output->buffers[0]->size());
+    }
+  } else {
+    output->null_count = 0;
+  }
+  auto copy_bitmap = [&](int64_t out_offset, int64_t in_offset, int64_t length) {
+    DCHECK(out_bitmap);
+    if (replacements.is_array()) {
+      const auto& in_data = *replacements.array();
+      const auto in_bitmap = in_data.GetValues<uint8_t>(0, /*absolute_offset=*/0);
+      arrow::internal::CopyBitmap(in_bitmap, in_data.offset + in_offset, length,
+                                  out_bitmap, out_offset);
+    } else {
+      BitUtil::SetBitsTo(out_bitmap, out_offset, length, !replacements_bitmap);
+    }
+  };
+
+  Functor::CopyData(*array.type, out_values, /*out_offset=*/0, array, /*in_offset=*/0,
+                    array.length);
+  arrow::internal::BitBlockCounter value_counter(mask_values, mask.offset, mask.length);
+  arrow::internal::OptionalBitBlockCounter valid_counter(mask_bitmap, mask.offset,
+                                                         mask.length);
+  int64_t out_offset = 0;
+  int64_t replacements_offset = 0;
+  while (out_offset < array.length) {
+    BitBlockCount value_block = value_counter.NextWord();
+    BitBlockCount valid_block = valid_counter.NextWord();
+    DCHECK_EQ(value_block.length, valid_block.length);
+    if (value_block.AllSet() && valid_block.AllSet()) {
+      // Copy from replacement array
+      if (replacements_offset + valid_block.length > replacements_length) {
+        return ReplacementArrayTooShort(replacements_offset + valid_block.length,
+                                        replacements_length);
+      }
+      Functor::CopyData(*array.type, out_values, out_offset, replacements,
+                        replacements_offset, valid_block.length);
+      if (replacements_bitmap) {
+        copy_bitmap(out_offset, replacements_offset, valid_block.length);
+      } else if (!replacements_bitmap && out_bitmap) {
+        BitUtil::SetBitsTo(out_bitmap, out_offset, valid_block.length, true);
+      }
+      replacements_offset += valid_block.length;
+    } else if (value_block.NoneSet() && valid_block.AllSet()) {
+      // Do nothing
+    } else if (valid_block.NoneSet()) {
+      DCHECK(out_bitmap);
+      BitUtil::SetBitsTo(out_bitmap, out_offset, valid_block.length, false);
+    } else {
+      for (int64_t i = 0; i < valid_block.length; ++i) {
+        if (BitUtil::GetBit(mask_values, out_offset + mask.offset + i) &&
+            (!mask_bitmap ||
+             BitUtil::GetBit(mask_bitmap, out_offset + mask.offset + i))) {
+          if (replacements_offset >= replacements_length) {
+            return ReplacementArrayTooShort(replacements_offset + 1, replacements_length);
+          }
+          Functor::CopyData(*array.type, out_values, out_offset + i, replacements,
+                            replacements_offset,
+                            /*length=*/1);
+          if (replacements_bitmap) {
+            copy_bitmap(out_offset + i, replacements_offset, 1);
+          }
+          replacements_offset++;
+        }
+      }
+    }
+    out_offset += valid_block.length;
+  }
+
+  if (mask.MayHaveNulls()) {
+    arrow::internal::BitmapAnd(out_bitmap, /*left_offset=*/0, mask.buffers[0]->data(),
+                               mask.offset, array.length,
+                               /*out_offset=*/0, out_bitmap);
+  }
+  return Status::OK();
+}
+
+template <typename Type, typename Enable = void>
+struct ReplaceWithMask {};
+
+template <typename Type>
+struct ReplaceWithMask<Type, enable_if_number<Type>> {
+  using T = typename TypeTraits<Type>::CType;
+
+  static Result<std::shared_ptr<Buffer>> AllocateData(KernelContext* ctx, const DataType&,
+                                                      const int64_t length) {
+    return ctx->Allocate(length * sizeof(T));
+  }
+
+  static void CopyData(const DataType&, uint8_t* out, const int64_t out_offset,
+                       const Datum& in, const int64_t in_offset, const int64_t length) {
+    if (in.is_array()) {
+      const auto& in_data = *in.array();
+      const auto in_arr =
+          in_data.GetValues<uint8_t>(1, (in_offset + in_data.offset) * sizeof(T));
+      std::memcpy(out + (out_offset * sizeof(T)), in_arr, length * sizeof(T));
+    } else {
+      T* begin = reinterpret_cast<T*>(out + (out_offset * sizeof(T)));
+      T* end = begin + length;
+      std::fill(begin, end, UnboxScalar<Type>::Unbox(*in.scalar()));
+    }
+  }
+
+  static Status ExecScalarMask(KernelContext* ctx, const ArrayData& array,
+                               const BooleanScalar& mask, const Datum& replacements,
+                               ArrayData* output) {
+    return ReplaceWithScalarMask(ctx, array, mask, replacements, output);
+  }
+
+  static Status ExecArrayMask(KernelContext* ctx, const ArrayData& array,
+                              const ArrayData& mask, const Datum& replacements,
+                              ArrayData* output) {
+    return ReplaceWithArrayMask<ReplaceWithMask<Type>>(ctx, array, mask, replacements,
+                                                       output);
+  }
+};
+
+template <typename Type>
+struct ReplaceWithMask<Type, enable_if_boolean<Type>> {
+  static Result<std::shared_ptr<Buffer>> AllocateData(KernelContext* ctx, const DataType&,
+                                                      const int64_t length) {
+    return ctx->AllocateBitmap(length);
+  }
+
+  static void CopyData(const DataType&, uint8_t* out, const int64_t out_offset,
+                       const Datum& in, const int64_t in_offset, const int64_t length) {
+    if (in.is_array()) {
+      const auto& in_data = *in.array();
+      const auto in_arr = in_data.GetValues<uint8_t>(1, /*absolute_offset=*/0);
+      arrow::internal::CopyBitmap(in_arr, in_offset + in_data.offset, length, out,
+                                  out_offset);
+    } else {
+      BitUtil::SetBitsTo(out, out_offset, length, in.scalar()->is_valid);
+    }
+  }
+
+  static Status ExecScalarMask(KernelContext* ctx, const ArrayData& array,
+                               const BooleanScalar& mask, const Datum& replacements,
+                               ArrayData* output) {
+    return ReplaceWithScalarMask(ctx, array, mask, replacements, output);
+  }
+  static Status ExecArrayMask(KernelContext* ctx, const ArrayData& array,
+                              const ArrayData& mask, const Datum& replacements,
+                              ArrayData* output) {
+    return ReplaceWithArrayMask<ReplaceWithMask<Type>>(ctx, array, mask, replacements,
+                                                       output);
+  }
+};
+
+template <typename Type>
+struct ReplaceWithMask<Type, enable_if_same<Type, FixedSizeBinaryType>> {
+  static Result<std::shared_ptr<Buffer>> AllocateData(KernelContext* ctx,
+                                                      const DataType& ty,
+                                                      const int64_t length) {
+    return ctx->Allocate(length *
+                         checked_cast<const FixedSizeBinaryType&>(ty).byte_width());
+  }
+
+  static void CopyData(const DataType& ty, uint8_t* out, const int64_t out_offset,
+                       const Datum& in, const int64_t in_offset, const int64_t length) {
+    const int32_t width = checked_cast<const FixedSizeBinaryType&>(ty).byte_width();
+    uint8_t* begin = out + (out_offset * width);
+    if (in.is_array()) {
+      const auto& in_data = *in.array();
+      const auto in_arr =
+          in_data.GetValues<uint8_t>(1, (in_offset + in_data.offset) * width);
+      std::memcpy(begin, in_arr, length * width);
+    } else {
+      const FixedSizeBinaryScalar& scalar =
+          checked_cast<const FixedSizeBinaryScalar&>(*in.scalar());
+      // Null scalar may have null value buffer
+      if (!scalar.value) return;
+      const Buffer& buffer = *scalar.value;
+      const uint8_t* value = buffer.data();
+      DCHECK_GE(buffer.size(), width);
+      for (int i = 0; i < length; i++) {
+        std::memcpy(begin, value, width);
+        begin += width;
+      }
+    }
+  }
+
+  static Status ExecScalarMask(KernelContext* ctx, const ArrayData& array,
+                               const BooleanScalar& mask, const Datum& replacements,
+                               ArrayData* output) {
+    return ReplaceWithScalarMask(ctx, array, mask, replacements, output);
+  }
+
+  static Status ExecArrayMask(KernelContext* ctx, const ArrayData& array,
+                              const ArrayData& mask, const Datum& replacements,
+                              ArrayData* output) {
+    return ReplaceWithArrayMask<ReplaceWithMask<Type>>(ctx, array, mask, replacements,
+                                                       output);
+  }
+};
+
+template <typename Type>
+struct ReplaceWithMask<Type, enable_if_decimal<Type>> {
+  using ScalarType = typename TypeTraits<Type>::ScalarType;
+
+  static Result<std::shared_ptr<Buffer>> AllocateData(KernelContext* ctx,
+                                                      const DataType& ty,
+                                                      const int64_t length) {
+    return ctx->Allocate(length *
+                         checked_cast<const FixedSizeBinaryType&>(ty).byte_width());
+  }
+
+  static void CopyData(const DataType& ty, uint8_t* out, const int64_t out_offset,
+                       const Datum& in, const int64_t in_offset, const int64_t length) {
+    const int32_t width = checked_cast<const FixedSizeBinaryType&>(ty).byte_width();
+    uint8_t* begin = out + (out_offset * width);
+    if (in.is_array()) {
+      const auto& in_data = *in.array();
+      const auto in_arr =
+          in_data.GetValues<uint8_t>(1, (in_offset + in_data.offset) * width);
+      std::memcpy(begin, in_arr, length * width);
+    } else {
+      const ScalarType& scalar = checked_cast<const ScalarType&>(*in.scalar());
+      const auto value = scalar.value.ToBytes();
+      for (int i = 0; i < length; i++) {
+        std::memcpy(begin, value.data(), width);
+        begin += width;
+      }
+    }
+  }
+
+  static Status ExecScalarMask(KernelContext* ctx, const ArrayData& array,
+                               const BooleanScalar& mask, const Datum& replacements,
+                               ArrayData* output) {
+    return ReplaceWithScalarMask(ctx, array, mask, replacements, output);
+  }
+
+  static Status ExecArrayMask(KernelContext* ctx, const ArrayData& array,
+                              const ArrayData& mask, const Datum& replacements,
+                              ArrayData* output) {
+    return ReplaceWithArrayMask<ReplaceWithMask<Type>>(ctx, array, mask, replacements,
+                                                       output);
+  }
+};
+
+template <typename Type>
+struct ReplaceWithMask<Type, enable_if_null<Type>> {
+  static Status ExecScalarMask(KernelContext* ctx, const ArrayData& array,
+                               const BooleanScalar& mask, const Datum& replacements,
+                               ArrayData* output) {
+    *output = array;
+    return Status::OK();
+  }
+  static Status ExecArrayMask(KernelContext* ctx, const ArrayData& array,
+                              const ArrayData& mask, const Datum& replacements,
+                              ArrayData* output) {
+    *output = array;
+    return Status::OK();
+  }
+};
+
+template <typename Type>
+struct ReplaceWithMask<Type, enable_if_base_binary<Type>> {
+  using offset_type = typename Type::offset_type;
+  using BuilderType = typename TypeTraits<Type>::BuilderType;
+
+  static Status ExecScalarMask(KernelContext* ctx, const ArrayData& array,
+                               const BooleanScalar& mask, const Datum& replacements,
+                               ArrayData* output) {
+    return ReplaceWithScalarMask(ctx, array, mask, replacements, output);
+  }
+  static Status ExecArrayMask(KernelContext* ctx, const ArrayData& array,
+                              const ArrayData& mask, const Datum& replacements,
+                              ArrayData* output) {
+    BuilderType builder(array.type, ctx->memory_pool());
+    RETURN_NOT_OK(builder.Reserve(array.length));
+    RETURN_NOT_OK(builder.ReserveData(array.buffers[2]->size()));
+    int64_t source_offset = 0;
+    int64_t replacements_offset = 0;
+    RETURN_NOT_OK(VisitArrayDataInline<BooleanType>(
+        mask,
+        [&](bool replace) {
+          if (replace && replacements.is_scalar()) {
+            const Scalar& scalar = *replacements.scalar();
+            if (scalar.is_valid) {
+              RETURN_NOT_OK(builder.Append(UnboxScalar<Type>::Unbox(scalar)));
+            } else {
+              RETURN_NOT_OK(builder.AppendNull());
+            }
+          } else {
+            const ArrayData& source = replace ? *replacements.array() : array;
+            const int64_t offset = replace ? replacements_offset++ : source_offset;
+            if (!source.MayHaveNulls() ||
+                BitUtil::GetBit(source.buffers[0]->data(), source.offset + offset)) {
+              const uint8_t* data = source.buffers[2]->data();
+              const offset_type* offsets = source.GetValues<offset_type>(1);
+              const offset_type offset0 = offsets[offset];
+              const offset_type offset1 = offsets[offset + 1];
+              RETURN_NOT_OK(builder.Append(data + offset0, offset1 - offset0));
+            } else {
+              RETURN_NOT_OK(builder.AppendNull());
+            }
+          }
+          source_offset++;
+          return Status::OK();
+        },
+        [&]() {
+          RETURN_NOT_OK(builder.AppendNull());
+          source_offset++;
+          return Status::OK();
+        }));
+    std::shared_ptr<Array> temp_output;
+    RETURN_NOT_OK(builder.Finish(&temp_output));
+    *output = *temp_output->data();
+    // Builder type != logical type due to GenerateTypeAgnosticVarBinaryBase
+    output->type = array.type;
+    return Status::OK();
+  }
+};
+
+template <typename Type>
+struct ReplaceWithMaskFunctor {
+  static Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    const ArrayData& array = *batch[0].array();
+    const Datum& replacements = batch[2];
+    ArrayData* output = out->array().get();
+    output->length = array.length;
+
+    // Needed for FixedSizeBinary/parameterized types
+    if (!array.type->Equals(*replacements.type(), /*check_metadata=*/false)) {
+      return Status::Invalid("Replacements must be of same type (expected ",
+                             array.type->ToString(), " but got ",
+                             replacements.type()->ToString(), ")");
+    }
+
+    if (!replacements.is_array() && !replacements.is_scalar()) {
+      return Status::Invalid("Replacements must be array or scalar");
+    }
+
+    if (batch[1].is_scalar()) {
+      return ReplaceWithMask<Type>::ExecScalarMask(
+          ctx, array, batch[1].scalar_as<BooleanScalar>(), replacements, output);
+    }
+    const ArrayData& mask = *batch[1].array();
+    if (array.length != mask.length) {
+      return Status::Invalid("Mask must be of same length as array (expected ",
+                             array.length, " items but got ", mask.length, " items)");
+    }
+    return ReplaceWithMask<Type>::ExecArrayMask(ctx, array, mask, replacements, output);
+  }
+};
+
+}  // namespace
+
+const FunctionDoc replace_with_mask_doc(
+    "Replace items using a mask and replacement values",
+    ("Given an array and a Boolean mask (either scalar or of equal length), "
+     "along with replacement values (either scalar or array), "
+     "each corresponding element of the mask will be replaced by the next "

Review comment:
       This made no sense on a re-read so I reworded the docs 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] lidavidm commented on pull request #10412: ARROW-9430: [C++] Implement replace_with_mask kernel

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


   @ursabot please benchmark language=C++


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

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

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



[GitHub] [arrow] nirandaperera commented on a change in pull request #10412: ARROW-9430: [C++] Implement replace_with_mask kernel

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



##########
File path: cpp/src/arrow/compute/kernels/vector_replace.cc
##########
@@ -0,0 +1,495 @@
+// 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_scalar.h"
+#include "arrow/compute/kernels/common.h"
+#include "arrow/util/bitmap_ops.h"
+
+namespace arrow {
+namespace compute {
+namespace internal {
+
+namespace {
+
+Status ReplacementArrayTooShort(int64_t expected, int64_t actual) {
+  return Status::Invalid("Replacement array must be of appropriate length (expected ",
+                         expected, " items but got ", actual, " items)");
+}
+
+// Helper to implement replace_with kernel with scalar mask for fixed-width types,
+// using callbacks to handle both bool and byte-sized types
+Status ReplaceWithScalarMask(KernelContext* ctx, const ArrayData& array,
+                             const BooleanScalar& mask, const Datum& replacements,
+                             ArrayData* output) {
+  if (!mask.is_valid) {
+    // Output = null
+    ARROW_ASSIGN_OR_RAISE(auto array,
+                          MakeArrayOfNull(array.type, array.length, ctx->memory_pool()));
+    *output = *array->data();
+    return Status::OK();
+  }
+  if (mask.value) {
+    // Output = replacement
+    if (replacements.is_scalar()) {
+      ARROW_ASSIGN_OR_RAISE(
+          auto replacement_array,
+          MakeArrayFromScalar(*replacements.scalar(), array.length, ctx->memory_pool()));
+      *output = *replacement_array->data();
+    } else {
+      auto replacement_array = replacements.array();
+      if (replacement_array->length != array.length) {
+        return ReplacementArrayTooShort(array.length, replacement_array->length);
+      }
+      *output = *replacement_array;
+    }
+  } else {
+    // Output = input
+    *output = array;
+  }
+  return Status::OK();
+}
+
+// Helper to implement replace_with kernel with array mask for fixed-width types,
+// using callbacks to handle both bool and byte-sized types and to handle
+// scalar and array replacements
+template <typename Functor>
+Status ReplaceWithArrayMask(KernelContext* ctx, const ArrayData& array,
+                            const ArrayData& mask, const Datum& replacements,
+                            ArrayData* output) {
+  ARROW_ASSIGN_OR_RAISE(output->buffers[1],
+                        Functor::AllocateData(ctx, *array.type, array.length));
+
+  uint8_t* out_bitmap = nullptr;
+  uint8_t* out_values = output->buffers[1]->mutable_data();
+  const uint8_t* mask_bitmap = mask.MayHaveNulls() ? mask.buffers[0]->data() : nullptr;
+  const uint8_t* mask_values = mask.buffers[1]->data();
+  bool replacements_bitmap;
+  int64_t replacements_length;
+  if (replacements.is_array()) {
+    replacements_bitmap = replacements.array()->MayHaveNulls();
+    replacements_length = replacements.array()->length;
+  } else {
+    replacements_bitmap = !replacements.scalar()->is_valid;
+    replacements_length = std::numeric_limits<int64_t>::max();
+  }
+  if (array.MayHaveNulls() || mask.MayHaveNulls() || replacements_bitmap) {
+    ARROW_ASSIGN_OR_RAISE(output->buffers[0], ctx->AllocateBitmap(array.length));
+    out_bitmap = output->buffers[0]->mutable_data();
+    output->null_count = -1;
+    if (array.MayHaveNulls()) {
+      arrow::internal::CopyBitmap(array.buffers[0]->data(), array.offset, array.length,
+                                  out_bitmap, /*dest_offset=*/0);
+    } else {
+      std::memset(out_bitmap, 0xFF, output->buffers[0]->size());

Review comment:
       Yes, that would prevent stepping on other slice's 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.

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

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



[GitHub] [arrow] nirandaperera commented on a change in pull request #10412: ARROW-9430: [C++] Implement replace_with_mask kernel

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



##########
File path: cpp/src/arrow/compute/kernels/vector_replace.cc
##########
@@ -0,0 +1,510 @@
+// 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_scalar.h"
+#include "arrow/compute/kernels/common.h"
+#include "arrow/util/bitmap_ops.h"
+
+namespace arrow {
+namespace compute {
+namespace internal {
+
+namespace {
+
+Status ReplacementArrayTooShort(int64_t expected, int64_t actual) {
+  return Status::Invalid("Replacement array must be of appropriate length (expected ",
+                         expected, " items but got ", actual, " items)");
+}
+
+// Helper to implement replace_with kernel with scalar mask for fixed-width types,
+// using callbacks to handle both bool and byte-sized types
+Status ReplaceWithScalarMask(KernelContext* ctx, const ArrayData& array,
+                             const BooleanScalar& mask, const Datum& replacements,
+                             ArrayData* output) {
+  if (!mask.is_valid) {
+    // Output = null
+    ARROW_ASSIGN_OR_RAISE(auto replacement_array,
+                          MakeArrayOfNull(array.type, array.length, ctx->memory_pool()));
+    *output = *replacement_array->data();
+    return Status::OK();
+  }
+  if (mask.value) {
+    // Output = replacement
+    if (replacements.is_scalar()) {
+      ARROW_ASSIGN_OR_RAISE(
+          auto replacement_array,
+          MakeArrayFromScalar(*replacements.scalar(), array.length, ctx->memory_pool()));
+      *output = *replacement_array->data();
+    } else {
+      auto replacement_array = replacements.array();
+      if (replacement_array->length != array.length) {
+        return ReplacementArrayTooShort(array.length, replacement_array->length);
+      }
+      *output = *replacement_array;
+    }
+  } else {
+    // Output = input
+    *output = array;
+  }
+  return Status::OK();
+}
+
+struct CopyArrayBitmap {
+  const uint8_t* in_bitmap;
+  int64_t in_offset;
+
+  void CopyBitmap(uint8_t* out_bitmap, int64_t out_offset, int64_t offset,
+                  int64_t length) const {
+    arrow::internal::CopyBitmap(in_bitmap, in_offset + offset, length, out_bitmap,
+                                out_offset);
+  }
+
+  void SetBit(uint8_t* out_bitmap, int64_t out_offset, int64_t offset) const {
+    BitUtil::SetBitTo(out_bitmap, out_offset,
+                      BitUtil::GetBit(in_bitmap, in_offset + offset));
+  }
+};
+
+struct CopyScalarBitmap {
+  const bool is_valid;
+
+  void CopyBitmap(uint8_t* out_bitmap, int64_t out_offset, int64_t offset,
+                  int64_t length) const {
+    BitUtil::SetBitsTo(out_bitmap, out_offset, length, is_valid);
+  }
+
+  void SetBit(uint8_t* out_bitmap, int64_t out_offset, int64_t offset) const {
+    BitUtil::SetBitTo(out_bitmap, out_offset, is_valid);
+  }
+};
+
+// Helper to implement replace_with kernel with array mask for fixed-width types,
+// using callbacks to handle both bool and byte-sized types and to handle
+// scalar and array replacements
+template <typename Functor, typename Data, typename CopyBitmap>
+void ReplaceWithArrayMaskImpl(const ArrayData& array, const ArrayData& mask,
+                              const Data& replacements, bool replacements_bitmap,
+                              const CopyBitmap copy_bitmap, const uint8_t* mask_bitmap,
+                              const uint8_t* mask_values, uint8_t* out_bitmap,
+                              uint8_t* out_values, const int64_t out_offset) {
+  Functor::CopyData(*array.type, out_values, /*out_offset=*/0, array, /*in_offset=*/0,
+                    array.length);
+  arrow::internal::OptionalBinaryBitBlockCounter counter(
+      mask_values, mask.offset, mask_bitmap, mask.offset, mask.length);
+  int64_t write_offset = 0;
+  int64_t replacements_offset = 0;
+  while (write_offset < array.length) {
+    BitBlockCount block = counter.NextAndBlock();
+    if (block.AllSet()) {
+      // Copy from replacement array
+      Functor::CopyData(*array.type, out_values, out_offset + write_offset, replacements,
+                        replacements_offset, block.length);
+      if (replacements_bitmap) {
+        copy_bitmap.CopyBitmap(out_bitmap, out_offset + write_offset, replacements_offset,
+                               block.length);
+      } else if (!replacements_bitmap && out_bitmap) {
+        BitUtil::SetBitsTo(out_bitmap, out_offset + write_offset, block.length, true);
+      }
+      replacements_offset += block.length;
+    } else if (block.popcount) {
+      for (int64_t i = 0; i < block.length; ++i) {
+        if (BitUtil::GetBit(mask_values, write_offset + mask.offset + i) &&
+            (!mask_bitmap ||
+             BitUtil::GetBit(mask_bitmap, write_offset + mask.offset + i))) {
+          Functor::CopyData(*array.type, out_values, out_offset + write_offset + i,
+                            replacements, replacements_offset, /*length=*/1);
+          if (replacements_bitmap) {
+            copy_bitmap.SetBit(out_bitmap, out_offset + write_offset + i,
+                               replacements_offset);
+          }
+          replacements_offset++;
+        }
+      }
+    }
+    write_offset += block.length;
+  }
+}
+
+template <typename Functor>
+Status ReplaceWithArrayMask(KernelContext* ctx, const ArrayData& array,
+                            const ArrayData& mask, const Datum& replacements,
+                            ArrayData* output) {
+  const int64_t out_offset = output->offset;
+  uint8_t* out_bitmap = nullptr;
+  uint8_t* out_values = output->buffers[1]->mutable_data();
+  const uint8_t* mask_bitmap = mask.MayHaveNulls() ? mask.buffers[0]->data() : nullptr;
+  const uint8_t* mask_values = mask.buffers[1]->data();
+  const bool replacements_bitmap = replacements.is_array()
+                                       ? replacements.array()->MayHaveNulls()
+                                       : !replacements.scalar()->is_valid;
+  if (replacements.is_array()) {
+    // Check that we have enough replacement values
+    const int64_t replacements_length = replacements.array()->length;
+
+    arrow::internal::OptionalBinaryBitBlockCounter counter(
+        mask_values, mask.offset, mask_bitmap, mask.offset, mask.length);
+    int64_t count = 0;
+    for (int64_t offset = 0; offset < mask.length;) {
+      BitBlockCount block = counter.NextAndBlock();
+      count += block.popcount;
+      offset += block.length;
+    }
+    if (count > replacements_length) {
+      return ReplacementArrayTooShort(count, replacements_length);
+    }
+  }
+  if (array.MayHaveNulls() || mask.MayHaveNulls() || replacements_bitmap) {
+    out_bitmap = output->buffers[0]->mutable_data();
+    output->null_count = -1;
+    if (array.MayHaveNulls()) {
+      // Copy array's bitmap
+      arrow::internal::CopyBitmap(array.buffers[0]->data(), array.offset, array.length,
+                                  out_bitmap, out_offset);
+    } else {
+      // Array has no bitmap but mask/replacements do, generate an all-valid bitmap
+      BitUtil::SetBitsTo(out_bitmap, out_offset, array.length, true);
+    }
+  } else {
+    BitUtil::SetBitsTo(output->buffers[0]->mutable_data(), out_offset, array.length,
+                       true);
+    output->null_count = 0;
+  }
+
+  if (replacements.is_array()) {
+    const ArrayData& array_repl = *replacements.array();
+    ReplaceWithArrayMaskImpl<Functor>(
+        array, mask, array_repl, replacements_bitmap,
+        CopyArrayBitmap{replacements_bitmap ? array_repl.buffers[0]->data() : nullptr,
+                        array_repl.offset},
+        mask_bitmap, mask_values, out_bitmap, out_values, out_offset);
+  } else {
+    const Scalar& scalar_repl = *replacements.scalar();
+    ReplaceWithArrayMaskImpl<Functor>(array, mask, scalar_repl, replacements_bitmap,
+                                      CopyScalarBitmap{scalar_repl.is_valid}, mask_bitmap,
+                                      mask_values, out_bitmap, out_values, out_offset);
+  }
+
+  if (mask.MayHaveNulls()) {
+    arrow::internal::BitmapAnd(out_bitmap, out_offset, mask.buffers[0]->data(),
+                               mask.offset, array.length, out_offset, out_bitmap);
+  }
+  return Status::OK();
+}
+
+template <typename Type, typename Enable = void>
+struct ReplaceWithMask {};
+
+template <typename Type>
+struct ReplaceWithMask<Type, enable_if_number<Type>> {
+  using T = typename TypeTraits<Type>::CType;
+
+  static void CopyData(const DataType&, uint8_t* out, const int64_t out_offset,
+                       const ArrayData& in, const int64_t in_offset,
+                       const int64_t length) {
+    const auto in_arr = in.GetValues<uint8_t>(1, (in_offset + in.offset) * sizeof(T));
+    std::memcpy(out + (out_offset * sizeof(T)), in_arr, length * sizeof(T));
+  }
+
+  static void CopyData(const DataType&, uint8_t* out, const int64_t out_offset,
+                       const Scalar& in, const int64_t in_offset, const int64_t length) {
+    T* begin = reinterpret_cast<T*>(out + (out_offset * sizeof(T)));
+    T* end = begin + length;
+    std::fill(begin, end, UnboxScalar<Type>::Unbox(in));
+  }
+
+  static Status ExecScalarMask(KernelContext* ctx, const ArrayData& array,
+                               const BooleanScalar& mask, const Datum& replacements,
+                               ArrayData* output) {
+    return ReplaceWithScalarMask(ctx, array, mask, replacements, output);
+  }
+
+  static Status ExecArrayMask(KernelContext* ctx, const ArrayData& array,
+                              const ArrayData& mask, const Datum& replacements,
+                              ArrayData* output) {
+    return ReplaceWithArrayMask<ReplaceWithMask<Type>>(ctx, array, mask, replacements,
+                                                       output);
+  }
+};
+
+template <typename Type>
+struct ReplaceWithMask<Type, enable_if_boolean<Type>> {
+  static void CopyData(const DataType&, uint8_t* out, const int64_t out_offset,
+                       const ArrayData& in, const int64_t in_offset,
+                       const int64_t length) {
+    const auto in_arr = in.GetValues<uint8_t>(1, /*absolute_offset=*/0);
+    arrow::internal::CopyBitmap(in_arr, in_offset + in.offset, length, out, out_offset);
+  }
+  static void CopyData(const DataType&, uint8_t* out, const int64_t out_offset,
+                       const Scalar& in, const int64_t in_offset, const int64_t length) {
+    BitUtil::SetBitsTo(out, out_offset, length, in.is_valid);
+  }
+
+  static Status ExecScalarMask(KernelContext* ctx, const ArrayData& array,
+                               const BooleanScalar& mask, const Datum& replacements,
+                               ArrayData* output) {
+    return ReplaceWithScalarMask(ctx, array, mask, replacements, output);
+  }
+  static Status ExecArrayMask(KernelContext* ctx, const ArrayData& array,
+                              const ArrayData& mask, const Datum& replacements,
+                              ArrayData* output) {
+    return ReplaceWithArrayMask<ReplaceWithMask<Type>>(ctx, array, mask, replacements,
+                                                       output);
+  }
+};
+
+template <typename Type>
+struct ReplaceWithMask<Type, enable_if_same<Type, FixedSizeBinaryType>> {
+  static void CopyData(const DataType& ty, uint8_t* out, const int64_t out_offset,
+                       const ArrayData& in, const int64_t in_offset,
+                       const int64_t length) {
+    const int32_t width = checked_cast<const FixedSizeBinaryType&>(ty).byte_width();
+    uint8_t* begin = out + (out_offset * width);
+    const auto in_arr = in.GetValues<uint8_t>(1, (in_offset + in.offset) * width);
+    std::memcpy(begin, in_arr, length * width);
+  }
+  static void CopyData(const DataType& ty, uint8_t* out, const int64_t out_offset,
+                       const Scalar& in, const int64_t in_offset, const int64_t length) {
+    const int32_t width = checked_cast<const FixedSizeBinaryType&>(ty).byte_width();
+    uint8_t* begin = out + (out_offset * width);
+    const auto& scalar = checked_cast<const FixedSizeBinaryScalar&>(in);
+    // Null scalar may have null value buffer
+    if (!scalar.value) return;
+    const Buffer& buffer = *scalar.value;
+    const uint8_t* value = buffer.data();
+    DCHECK_GE(buffer.size(), width);
+    for (int i = 0; i < length; i++) {
+      std::memcpy(begin, value, width);
+      begin += width;
+    }
+  }
+
+  static Status ExecScalarMask(KernelContext* ctx, const ArrayData& array,
+                               const BooleanScalar& mask, const Datum& replacements,
+                               ArrayData* output) {
+    return ReplaceWithScalarMask(ctx, array, mask, replacements, output);
+  }
+
+  static Status ExecArrayMask(KernelContext* ctx, const ArrayData& array,
+                              const ArrayData& mask, const Datum& replacements,
+                              ArrayData* output) {
+    return ReplaceWithArrayMask<ReplaceWithMask<Type>>(ctx, array, mask, replacements,
+                                                       output);
+  }
+};
+
+template <typename Type>
+struct ReplaceWithMask<Type, enable_if_decimal<Type>> {
+  using ScalarType = typename TypeTraits<Type>::ScalarType;
+  static void CopyData(const DataType& ty, uint8_t* out, const int64_t out_offset,
+                       const ArrayData& in, const int64_t in_offset,
+                       const int64_t length) {
+    const int32_t width = checked_cast<const FixedSizeBinaryType&>(ty).byte_width();
+    uint8_t* begin = out + (out_offset * width);
+    const auto in_arr = in.GetValues<uint8_t>(1, (in_offset + in.offset) * width);
+    std::memcpy(begin, in_arr, length * width);
+  }
+  static void CopyData(const DataType& ty, uint8_t* out, const int64_t out_offset,
+                       const Scalar& in, const int64_t in_offset, const int64_t length) {
+    const int32_t width = checked_cast<const FixedSizeBinaryType&>(ty).byte_width();
+    uint8_t* begin = out + (out_offset * width);
+    const auto& scalar = checked_cast<const ScalarType&>(in);
+    const auto value = scalar.value.ToBytes();
+    for (int i = 0; i < length; i++) {
+      std::memcpy(begin, value.data(), width);
+      begin += width;
+    }
+  }
+
+  static Status ExecScalarMask(KernelContext* ctx, const ArrayData& array,
+                               const BooleanScalar& mask, const Datum& replacements,
+                               ArrayData* output) {
+    return ReplaceWithScalarMask(ctx, array, mask, replacements, output);
+  }
+
+  static Status ExecArrayMask(KernelContext* ctx, const ArrayData& array,
+                              const ArrayData& mask, const Datum& replacements,
+                              ArrayData* output) {
+    return ReplaceWithArrayMask<ReplaceWithMask<Type>>(ctx, array, mask, replacements,
+                                                       output);
+  }
+};
+
+template <typename Type>
+struct ReplaceWithMask<Type, enable_if_null<Type>> {
+  static Status ExecScalarMask(KernelContext* ctx, const ArrayData& array,
+                               const BooleanScalar& mask, const Datum& replacements,
+                               ArrayData* output) {
+    *output = array;
+    return Status::OK();
+  }
+  static Status ExecArrayMask(KernelContext* ctx, const ArrayData& array,
+                              const ArrayData& mask, const Datum& replacements,
+                              ArrayData* output) {
+    *output = array;
+    return Status::OK();
+  }
+};
+
+template <typename Type>
+struct ReplaceWithMask<Type, enable_if_base_binary<Type>> {
+  using offset_type = typename Type::offset_type;
+  using BuilderType = typename TypeTraits<Type>::BuilderType;
+
+  static Status ExecScalarMask(KernelContext* ctx, const ArrayData& array,
+                               const BooleanScalar& mask, const Datum& replacements,
+                               ArrayData* output) {
+    return ReplaceWithScalarMask(ctx, array, mask, replacements, output);
+  }
+  static Status ExecArrayMask(KernelContext* ctx, const ArrayData& array,
+                              const ArrayData& mask, const Datum& replacements,
+                              ArrayData* output) {
+    BuilderType builder(array.type, ctx->memory_pool());
+    RETURN_NOT_OK(builder.Reserve(array.length));
+    RETURN_NOT_OK(builder.ReserveData(array.buffers[2]->size()));
+    int64_t source_offset = 0;
+    int64_t replacements_offset = 0;
+    RETURN_NOT_OK(VisitArrayDataInline<BooleanType>(
+        mask,
+        [&](bool replace) {
+          if (replace && replacements.is_scalar()) {
+            const Scalar& scalar = *replacements.scalar();
+            if (scalar.is_valid) {
+              RETURN_NOT_OK(builder.Append(UnboxScalar<Type>::Unbox(scalar)));
+            } else {
+              RETURN_NOT_OK(builder.AppendNull());
+            }
+          } else {
+            const ArrayData& source = replace ? *replacements.array() : array;
+            const int64_t offset = replace ? replacements_offset++ : source_offset;
+            if (!source.MayHaveNulls() ||
+                BitUtil::GetBit(source.buffers[0]->data(), source.offset + offset)) {
+              const uint8_t* data = source.buffers[2]->data();
+              const offset_type* offsets = source.GetValues<offset_type>(1);
+              const offset_type offset0 = offsets[offset];
+              const offset_type offset1 = offsets[offset + 1];
+              RETURN_NOT_OK(builder.Append(data + offset0, offset1 - offset0));
+            } else {
+              RETURN_NOT_OK(builder.AppendNull());
+            }
+          }
+          source_offset++;
+          return Status::OK();
+        },
+        [&]() {
+          RETURN_NOT_OK(builder.AppendNull());
+          source_offset++;
+          return Status::OK();
+        }));
+    std::shared_ptr<Array> temp_output;
+    RETURN_NOT_OK(builder.Finish(&temp_output));
+    *output = *temp_output->data();
+    // Builder type != logical type due to GenerateTypeAgnosticVarBinaryBase
+    output->type = array.type;
+    return Status::OK();
+  }
+};
+
+template <typename Type>
+struct ReplaceWithMaskFunctor {
+  static Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    const ArrayData& array = *batch[0].array();
+    const Datum& replacements = batch[2];
+    ArrayData* output = out->array().get();
+    output->length = array.length;
+
+    // Needed for FixedSizeBinary/parameterized types
+    if (!array.type->Equals(*replacements.type(), /*check_metadata=*/false)) {
+      return Status::Invalid("Replacements must be of same type (expected ",
+                             array.type->ToString(), " but got ",
+                             replacements.type()->ToString(), ")");
+    }
+
+    if (!replacements.is_array() && !replacements.is_scalar()) {
+      return Status::Invalid("Replacements must be array or scalar");
+    }
+
+    if (batch[1].is_scalar()) {
+      return ReplaceWithMask<Type>::ExecScalarMask(
+          ctx, array, batch[1].scalar_as<BooleanScalar>(), replacements, output);
+    }
+    const ArrayData& mask = *batch[1].array();
+    if (array.length != mask.length) {
+      return Status::Invalid("Mask must be of same length as array (expected ",
+                             array.length, " items but got ", mask.length, " items)");
+    }
+    return ReplaceWithMask<Type>::ExecArrayMask(ctx, array, mask, replacements, output);
+  }
+};
+
+}  // namespace
+
+const FunctionDoc replace_with_mask_doc(
+    "Replace items using a mask and replacement values",
+    ("Given an array and a Boolean mask (either scalar or of equal length), "
+     "along with replacement values (either scalar or array), "
+     "each element of the array for which the corresponding mask element is "
+     "true will be replaced by the next value from the replacements, "
+     "or with null if the mask is null. "
+     "Hence, for replacement arrays, len(replacements) == sum(mask == true)."),
+    {"values", "mask", "replacements"});
+
+void RegisterVectorReplace(FunctionRegistry* registry) {
+  auto func = std::make_shared<VectorFunction>("replace_with_mask", Arity::Ternary(),
+                                               &replace_with_mask_doc);
+  auto add_kernel = [&](detail::GetTypeId get_id, ArrayKernelExec exec) {
+    VectorKernel kernel;
+    kernel.can_execute_chunkwise = false;
+    if (is_fixed_width(get_id.id)) {
+      kernel.null_handling = NullHandling::type::COMPUTED_PREALLOCATE;
+    } else {
+      kernel.can_write_into_slices = false;
+      kernel.null_handling = NullHandling::type::COMPUTED_NO_PREALLOCATE;
+    }
+    kernel.mem_allocation = MemAllocation::type::PREALLOCATE;

Review comment:
       @bkietz WDYT?




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

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

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



[GitHub] [arrow] lidavidm commented on pull request #10412: ARROW-9430: [C++] Implement replace_with_mask kernel

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


   The `ReplaceWithMaskRandom` tests do test sliced arrays. (Unlike the scalar tests we can't test slices just by slicing the given example inputs, unfortunately.)


-- 
This is an automated message from the 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 #10412: ARROW-9430: [C++] Implement replace_with_mask kernel

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


   > I think I've addressed all the feedback here.
   
   I'm +1 for this!  


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

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

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



[GitHub] [arrow] ursabot edited a comment on pull request #10412: ARROW-9430: [C++] Implement replace_with_mask kernel

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


   Benchmark runs are scheduled for baseline = 7eea2f53a1002552bbb87db5611e75c15b88b504 and contender = f79438d96c42b7728c3f9860aadad545cc5ac483. Results will be available as each benchmark for each run completes.
   Conbench compare runs links:
   [Skipped :warning: Provided benchmark filters do not have any benchmark groups to be executed on ec2-t3-xlarge-us-east-2] [ec2-t3-xlarge-us-east-2 (mimalloc)](https://conbench.ursa.dev/compare/runs/35393eacbb0b4a7fafc94d16ccfa1a9d...978ddda346754d86b47bbb88aa93094d/)
   [Skipped :warning: Only ['Python', 'R'] langs are supported on ursa-i9-9960x] [ursa-i9-9960x (mimalloc)](https://conbench.ursa.dev/compare/runs/478d6e32c067446fbde6959e6945e2f2...00fa3be8f88e4ac4a5fbadb861abfe5a/)
   [Finished :arrow_down:0.0% :arrow_up:0.0%] [ursa-thinkcentre-m75q (mimalloc)](https://conbench.ursa.dev/compare/runs/6b1fb3141e074ec0ba6a80cfcc4e068e...7b5b67d940d742d283fe96c295d10e67/)
   Supported benchmarks:
   ursa-i9-9960x: langs = Python, R
   ursa-thinkcentre-m75q: langs = C++, Java
   ec2-t3-xlarge-us-east-2: cloud = True
   


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

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

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



[GitHub] [arrow] ursabot commented on pull request #10412: ARROW-9430: [C++] Implement replace_with_mask kernel

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


   Benchmark runs are scheduled for baseline = 7eea2f53a1002552bbb87db5611e75c15b88b504 and contender = f79438d96c42b7728c3f9860aadad545cc5ac483. Results will be available as each benchmark for each run completes.
   Conbench compare runs links:
   [Skipped :warning: Provided benchmark filters do not have any benchmark groups to be executed on ec2-t3-xlarge-us-east-2] [ec2-t3-xlarge-us-east-2 (mimalloc)](https://conbench.ursa.dev/compare/runs/35393eacbb0b4a7fafc94d16ccfa1a9d...978ddda346754d86b47bbb88aa93094d/)
   [Skipped :warning: Only ['Python', 'R'] langs are supported on ursa-i9-9960x] [ursa-i9-9960x (mimalloc)](https://conbench.ursa.dev/compare/runs/478d6e32c067446fbde6959e6945e2f2...00fa3be8f88e4ac4a5fbadb861abfe5a/)
   [Scheduled] [ursa-thinkcentre-m75q (mimalloc)](https://conbench.ursa.dev/compare/runs/6b1fb3141e074ec0ba6a80cfcc4e068e...7b5b67d940d742d283fe96c295d10e67/)
   Supported benchmarks:
   ursa-i9-9960x: langs = Python, R
   ursa-thinkcentre-m75q: langs = C++, Java
   ec2-t3-xlarge-us-east-2: cloud = True
   


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

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

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



[GitHub] [arrow] lidavidm commented on pull request #10412: ARROW-9430: [C++] Implement replace_with_mask kernel

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


   @bkietz @jorisvandenbossche I know y'all are busy, but any other comments? Once this is in, @nirandaperera can get started on ARROW-9431 on top of this


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

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



[GitHub] [arrow] lidavidm commented on pull request #10412: ARROW-9430: [C++] Implement replace_with_mask kernel

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


   I've cleaned things up here. Once this and any of the scalar select/choose/etc. kernels land I can go back and consolidate the helpers used between these kernels.


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

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

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