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

[GitHub] [arrow] wesm opened a new pull request #7442: ARROW-9075: [C++] Optimized Filter implementation: faster performance + compilation, smaller code size

wesm opened a new pull request #7442:
URL: https://github.com/apache/arrow/pull/7442


   NOTE: the diff is artificially larger due to some code rearranging (that was necessitated because of how some data selection code is shared between the Take and Filter implementations).
   
   Summary:
   
   * Filter is now 1.5-6x faster across the board, most notably on primitive types with high selectivity filters. The BitBlockCounters do a lot of the heavy lifting in that case but even in the worst case scenario when the block counters never encounter a "full" block, this is still consistently faster.
   * Total -O3 code size for **both** Take and Filter is now about 600KB. That's down from about 8MB total prior to this patch and ARROW-5760
   
   Some incidental changes:
   * Implemented a fast conversion from boolean filter to take indices (aka "selection vector"),  `compute::internal::GetTakeIndices`. I have also altered the implementation of filtering a record batch to use this, which should be faster (it would be good to have some benchmarks to confirm this). 
   * Various expansions to the BitBlockCounter classes that I needed to support this work
   * Fixed a bug ARROW-9142 with RandomArrayGenerator::Boolean. The probability parameter was being interpreted as the probability of a false value rather than the probability of a true. IIUC with Bernoulli distributions, the probability specified is P(X = 1) not P(X = 0). Please someone confirm 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] wesm edited a comment on pull request #7442: ARROW-9075: [C++] Optimized Filter implementation: faster performance + compilation, smaller code size

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


   To show some simple numbers to show the perf before and after in Python, this example has a high selectivity (all but one value selected) and low selectivity filter (1/100 and 1/1000):
   
   ```
   import numpy as np
   import pandas as pd
   import pyarrow as pa
   import pyarrow.compute as pc
   
   string_values = pa.array([pd.util.testing.rands(16)
                             for i in range(10000)] * 100)
   double_values = pa.array(np.random.randn(1000000))
   
   all_but_one = np.ones(len(string_values), dtype=bool)
   all_but_one[500000] = False
   
   one_in_100 = np.array(np.random.binomial(1, 0.01, size=1000000), dtype=bool)
   one_in_1000 = np.array(np.random.binomial(1, 0.001, size=1000000), dtype=bool)
   ```
   
   before:
   
   ```
   In [2]: timeit pc.filter(double_values, one_in_100)                                                                                                
   2.06 ms ± 41.3 µs per loop (mean ± std. dev. of 7 runs, 100 loops each)
   
   In [3]: timeit pc.filter(double_values, one_in_1000)                                                                                               
   1.82 ms ± 3.69 µs per loop (mean ± std. dev. of 7 runs, 1000 loops each)
   
   In [4]: timeit pc.filter(double_values, all_but_one)                                                                                               
   5.75 ms ± 15.9 µs per loop (mean ± std. dev. of 7 runs, 100 loops each)
   
   In [5]: timeit pc.filter(string_values, one_in_100)                                                                                                
   2.23 ms ± 14.2 µs per loop (mean ± std. dev. of 7 runs, 100 loops each)
   
   In [6]: timeit pc.filter(string_values, one_in_1000)                                                                                               
   1.85 ms ± 3.92 µs per loop (mean ± std. dev. of 7 runs, 1000 loops each)
   
   In [7]: timeit pc.filter(string_values, all_but_one)                                                                                               
   11.6 ms ± 183 µs per loop (mean ± std. dev. of 7 runs, 100 loops each)
   ```
   
   after
   
   ```
   In [4]: timeit pc.filter(double_values, one_in_100)                                               
   1.1 ms ± 7.03 µs per loop (mean ± std. dev. of 7 runs, 1000 loops each)
   
   In [5]: timeit pc.filter(double_values, one_in_1000)
   531 µs ± 8.52 µs per loop (mean ± std. dev. of 7 runs, 1000 loops each)
   
   In [7]: timeit pc.filter(double_values, all_but_one)                                              
   1.83 ms ± 7.36 µs per loop (mean ± std. dev. of 7 runs, 1000 loops each)
   
   In [10]: timeit pc.filter(string_values, one_in_100)                                                                                               
   1.28 ms ± 3.16 µs per loop (mean ± std. dev. of 7 runs, 1000 loops each)
   
   In [11]: timeit pc.filter(string_values, one_in_1000)                                                                                              
   561 µs ± 1.69 µs per loop (mean ± std. dev. of 7 runs, 1000 loops each)
   
   In [12]: timeit pc.filter(string_values, all_but_one)                                                                                              
   6.66 ms ± 34.1 µs per loop (mean ± std. dev. of 7 runs, 100 loops each)
   ```
   
   EDIT: updated benchmarks for low-selectivity optimization


----------------------------------------------------------------
This is an automated message from the 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] wesm commented on pull request #7442: ARROW-9075: [C++] Optimized Filter implementation: faster performance + compilation, smaller code size

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


   +1. Thanks all for the 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.

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



[GitHub] [arrow] wesm commented on pull request #7442: ARROW-9075: [C++] Optimized Filter implementation: faster performance + compilation, smaller code size

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


   So these "readability" improvements made performance worse so I'll revert them


----------------------------------------------------------------
This is an automated message from the 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] wesm commented on pull request #7442: ARROW-9075: [C++] Optimized Filter implementation: faster performance + compilation, smaller code size

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


   The string perf regressions are mostly for the cases where 99.9% of the values are selected. I'll take a closer look at this to see what can be done. The varbinary case is so important that we might want to create a specialized implementation for it


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

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



[GitHub] [arrow] wesm commented on pull request #7442: ARROW-9075: [C++] Optimized Filter implementation: faster performance + compilation, smaller code size

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


   To show some simple numbers to show the perf before and after in Python, this example has a high selectivity (all but one value selected) and low selectivity filter (only 1% of values selected): 
   
   ```
   import numpy as np
   import pandas as pd
   import pyarrow as pa
   import pyarrow.compute as pc
   
   string_values = pa.array([pd.util.testing.rands(16)
                             for i in range(10000)] * 100)
   double_values = pa.array(np.random.randn(1000000))
   
   all_but_one = np.ones(len(string_values), dtype=bool)
   all_but_one[500000] = False
   
   only_1pct = np.array(np.random.binomial(1, 0.01, size=1000000), dtype=bool)
   ```
   
   before:
   
   ```
   In [4]: timeit pc.filter(double_values, only_1pct)                                                
   2.01 ms ± 7.89 µs per loop (mean ± std. dev. of 7 runs, 100 loops each)
   
   In [5]: timeit pc.filter(double_values, all_but_one)                                              
   5.74 ms ± 17.6 µs per loop (mean ± std. dev. of 7 runs, 100 loops each)
   
   In [6]: timeit pc.filter(string_values, only_1pct)                                                
   2.21 ms ± 6.87 µs per loop (mean ± std. dev. of 7 runs, 100 loops each)
   
   In [7]: timeit pc.filter(string_values, all_but_one)                                              
   11.4 ms ± 142 µs per loop (mean ± std. dev. of 7 runs, 100 loops each)
   ```
   
   after
   
   ```
   In [29]: timeit pc.filter(double_values, only_1pct)                                               
   1.43 ms ± 3.79 µs per loop (mean ± std. dev. of 7 runs, 1000 loops each)
   
   In [30]: timeit pc.filter(double_values, all_but_one)                                             
   1.81 ms ± 20.6 µs per loop (mean ± std. dev. of 7 runs, 1000 loops each)
   
   In [31]: timeit pc.filter(string_values, only_1pct)                                               
   1.57 ms ± 4.06 µs per loop (mean ± std. dev. of 7 runs, 1000 loops each)
   
   In [32]: timeit pc.filter(string_values, all_but_one)                                             
   6.66 ms ± 39.7 µs per loop (mean ± std. dev. of 7 runs, 100 loops each)
   ```


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

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



[GitHub] [arrow] wesm commented on a change in pull request #7442: ARROW-9075: [C++] Optimized Filter implementation: faster performance + compilation, smaller code size

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



##########
File path: cpp/src/arrow/compute/kernels/util_internal.h
##########
@@ -0,0 +1,50 @@
+// 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.
+
+#pragma once
+
+#include <cstdint>
+
+#include "arrow/buffer.h"
+
+namespace arrow {
+namespace compute {
+namespace internal {
+
+// An internal data structure for unpacking a primitive argument to pass to a
+// kernel implementation
+struct PrimitiveArg {
+  const uint8_t* is_valid;
+  const uint8_t* data;

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.

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



[GitHub] [arrow] emkornfield commented on a change in pull request #7442: ARROW-9075: [C++] Optimized Filter implementation: faster performance + compilation, smaller code size

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



##########
File path: cpp/src/arrow/util/bit_block_counter.h
##########
@@ -33,11 +33,50 @@ class Buffer;
 
 namespace internal {
 
+namespace detail {
+
+// These templates are here to help with unit tests
+
+template <typename T>
+struct BitBlockAnd {
+  static T Call(T left, T right) { return left & right; }
+};
+
+template <>
+struct BitBlockAnd<bool> {
+  static bool Call(bool left, bool right) { return left && right; }
+};
+
+template <typename T>
+struct BitBlockOr {
+  static T Call(T left, T right) { return left | right; }
+};
+
+template <>
+struct BitBlockOr<bool> {
+  static bool Call(bool left, bool right) { return left || right; }
+};
+
+template <typename T>
+struct BitBlockOrNot {
+  static T Call(T left, T right) { return left | ~right; }
+};
+
+template <>
+struct BitBlockOrNot<bool> {
+  static bool Call(bool left, bool right) { return left || !right; }
+};
+
+}  // namespace detail
+
 /// \brief Return value from bit block counters: the total number of bits and
 /// the number of set bits.
 struct BitBlockCount {
   int16_t length;
   int16_t popcount;
+
+  bool IsEmpty() const { return this->popcount == 0; }

Review comment:
       Nit: NoneSet and AllSet might be better names
   
   I could interpret empty to be length equal to zero (by the way to have a mtehod for this condition 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] wesm commented on a change in pull request #7442: ARROW-9075: [C++] Optimized Filter implementation: faster performance + compilation, smaller code size

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



##########
File path: cpp/src/arrow/compute/kernels/vector_selection.cc
##########
@@ -0,0 +1,1816 @@
+// 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 <algorithm>
+#include <cstring>
+#include <limits>
+
+#include "arrow/array/array_base.h"
+#include "arrow/array/array_binary.h"
+#include "arrow/array/array_dict.h"
+#include "arrow/array/array_nested.h"
+#include "arrow/array/builder_primitive.h"
+#include "arrow/array/concatenate.h"
+#include "arrow/buffer_builder.h"
+#include "arrow/compute/api_vector.h"
+#include "arrow/compute/kernels/common.h"
+#include "arrow/compute/kernels/util_internal.h"
+#include "arrow/extension_type.h"
+#include "arrow/record_batch.h"
+#include "arrow/result.h"
+#include "arrow/util/bit_block_counter.h"
+#include "arrow/util/bit_util.h"
+#include "arrow/util/bitmap_ops.h"
+#include "arrow/util/bitmap_reader.h"
+#include "arrow/util/int_util.h"
+
+namespace arrow {
+
+using internal::BinaryBitBlockCounter;
+using internal::BitBlockCount;
+using internal::BitBlockCounter;
+using internal::BitmapReader;
+using internal::CopyBitmap;
+using internal::GetArrayView;
+using internal::IndexBoundsCheck;
+using internal::OptionalBitBlockCounter;
+using internal::OptionalBitIndexer;
+
+namespace compute {
+namespace internal {
+
+int64_t GetFilterOutputSize(const ArrayData& filter,
+                            FilterOptions::NullSelectionBehavior null_selection) {
+  int64_t output_size = 0;
+  int64_t position = 0;
+  if (filter.GetNullCount() > 0) {
+    const uint8_t* filter_is_valid = filter.buffers[0]->data();
+    BinaryBitBlockCounter bit_counter(filter.buffers[1]->data(), filter.offset,
+                                      filter_is_valid, filter.offset, filter.length);
+    if (null_selection == FilterOptions::EMIT_NULL) {
+      while (position < filter.length) {
+        BitBlockCount block = bit_counter.NextOrNotWord();
+        output_size += block.popcount;
+        position += block.length;
+      }
+    } else {
+      while (position < filter.length) {
+        BitBlockCount block = bit_counter.NextAndWord();
+        output_size += block.popcount;
+        position += block.length;
+      }
+    }
+  } else {
+    // The filter has no nulls, so we plow through its data as fast as
+    // possible.
+    BitBlockCounter bit_counter(filter.buffers[1]->data(), filter.offset, filter.length);
+    while (position < filter.length) {
+      BitBlockCount block = bit_counter.NextFourWords();
+      output_size += block.popcount;
+      position += block.length;
+    }
+  }
+  return output_size;
+}
+
+template <typename IndexType>
+Result<std::shared_ptr<ArrayData>> GetTakeIndicesImpl(
+    const ArrayData& filter, FilterOptions::NullSelectionBehavior null_selection,
+    MemoryPool* memory_pool) {
+  using T = typename IndexType::c_type;
+  typename TypeTraits<IndexType>::BuilderType builder(memory_pool);
+
+  const uint8_t* filter_data = filter.buffers[1]->data();
+  BitBlockCounter data_counter(filter_data, filter.offset, filter.length);
+
+  // The position relative to the start of the filter
+  T position = 0;
+
+  // The current position taking the filter offset into account
+  int64_t position_with_offset = filter.offset;
+  if (filter.GetNullCount() > 0) {
+    // The filter has nulls, so we scan the validity bitmap and the filter data
+    // bitmap together, branching on the null selection type.
+    const uint8_t* filter_is_valid = filter.buffers[0]->data();
+
+    // To count blocks whether filter_data[i] || !filter_is_valid[i]
+    BinaryBitBlockCounter filter_counter(filter_data, filter.offset, filter_is_valid,
+                                         filter.offset, filter.length);
+    if (null_selection == FilterOptions::DROP) {
+      while (position < filter.length) {
+        BitBlockCount and_block = filter_counter.NextAndWord();
+        RETURN_NOT_OK(builder.Reserve(and_block.popcount));
+        if (and_block.AllSet()) {
+          // All the values are selected and non-null
+          for (int64_t i = 0; i < and_block.length; ++i) {
+            builder.UnsafeAppend(position++);
+          }
+          position_with_offset += and_block.length;
+        } else {
+          // Some of the values are false or null
+          for (int64_t i = 0; i < and_block.length; ++i) {
+            if (BitUtil::GetBit(filter_is_valid, position_with_offset) &&
+                BitUtil::GetBit(filter_data, position_with_offset)) {
+              builder.UnsafeAppend(position);
+            }
+            ++position;
+            ++position_with_offset;
+          }
+        }
+      }
+    } else {
+      BitBlockCounter is_valid_counter(filter_is_valid, filter.offset, filter.length);
+      while (position < filter.length) {
+        // true OR NOT valid
+        BitBlockCount or_not_block = filter_counter.NextOrNotWord();
+        RETURN_NOT_OK(builder.Reserve(or_not_block.popcount));
+
+        // If the values are all valid and the or_not_block is full, then we
+        // can infer that all the values are true and skip the bit checking
+        BitBlockCount is_valid_block = is_valid_counter.NextWord();
+
+        if (or_not_block.AllSet() && is_valid_block.AllSet()) {
+          // All the values are selected and non-null
+          for (int64_t i = 0; i < or_not_block.length; ++i) {
+            builder.UnsafeAppend(position++);
+          }
+          position_with_offset += or_not_block.length;
+        } else {
+          // Some of the values are false or null
+          for (int64_t i = 0; i < or_not_block.length; ++i) {
+            if (BitUtil::GetBit(filter_is_valid, position_with_offset)) {
+              if (BitUtil::GetBit(filter_data, position_with_offset)) {
+                builder.UnsafeAppend(position);
+              }
+            } else {
+              // Null slot, so append a null
+              builder.UnsafeAppendNull();
+            }
+            ++position;
+            ++position_with_offset;
+          }
+        }
+      }
+    }
+  } else {
+    // The filter has no nulls, so we need only look for true values
+    BitBlockCount current_block = data_counter.NextWord();
+    while (position < filter.length) {
+      if (current_block.AllSet()) {
+        int64_t run_length = 0;
+
+        // If we've found a all-true block, then we scan forward until we find
+        // a block that has some false values (or we reach the end
+        while (current_block.length > 0 && current_block.AllSet()) {
+          run_length += current_block.length;
+          current_block = data_counter.NextWord();
+        }
+
+        // Append the consecutive run of indices
+        RETURN_NOT_OK(builder.Reserve(run_length));
+        for (int64_t i = 0; i < run_length; ++i) {
+          builder.UnsafeAppend(position++);
+        }
+        position_with_offset += run_length;
+      } else {
+        // Must do bitchecking on the current block
+        RETURN_NOT_OK(builder.Reserve(current_block.popcount));
+        for (int64_t i = 0; i < current_block.length; ++i) {
+          if (BitUtil::GetBit(filter_data, position_with_offset)) {
+            builder.UnsafeAppend(position);
+          }
+          ++position;
+          ++position_with_offset;
+        }
+        current_block = data_counter.NextWord();
+      }
+    }
+  }
+  std::shared_ptr<ArrayData> result;
+  RETURN_NOT_OK(builder.FinishInternal(&result));
+  return result;
+}
+
+Result<std::shared_ptr<ArrayData>> GetTakeIndices(
+    const ArrayData& filter, FilterOptions::NullSelectionBehavior null_selection,
+    MemoryPool* memory_pool) {
+  DCHECK_EQ(filter.type->id(), Type::BOOL);
+  if (filter.length <= std::numeric_limits<uint16_t>::max()) {
+    return GetTakeIndicesImpl<UInt16Type>(filter, null_selection, memory_pool);
+  } else if (filter.length <= std::numeric_limits<uint32_t>::max()) {
+    return GetTakeIndicesImpl<UInt32Type>(filter, null_selection, memory_pool);
+  } else {
+    // Arrays over 4 billion elements, not especially likely.
+    return Status::NotImplemented(
+        "Filter length exceeds UINT32_MAX, "
+        "consider a different strategy for selecting elements");
+  }
+}
+
+namespace {
+
+template <typename ArrowType>
+struct GetCType {
+  using type = typename ArrowType::c_type;
+};
+
+// We want uint8_t for boolean instead of bool
+template <>
+struct GetCType<BooleanType> {
+  using type = uint8_t;
+};
+
+using FilterState = OptionsWrapper<FilterOptions>;
+using TakeState = OptionsWrapper<TakeOptions>;
+
+Status PreallocateData(KernelContext* ctx, int64_t length, int bit_width, Datum* out) {
+  // Preallocate memory
+  ArrayData* out_arr = out->mutable_array();
+  out_arr->length = length;
+  out_arr->buffers.resize(2);
+
+  ARROW_ASSIGN_OR_RAISE(out_arr->buffers[0], ctx->AllocateBitmap(length));
+  if (bit_width == 1) {
+    ARROW_ASSIGN_OR_RAISE(out_arr->buffers[1], ctx->AllocateBitmap(length));
+  } else {
+    ARROW_ASSIGN_OR_RAISE(out_arr->buffers[1], ctx->Allocate(length * bit_width / 8));
+  }
+  return Status::OK();
+}
+
+// ----------------------------------------------------------------------
+// Implement optimized take for primitive types from boolean to 1/2/4/8-byte
+// C-type based types. Use common implementation for every byte width and only
+// generate code for unsigned integer indices, since after boundschecking to
+// check for negative numbers in the indices we can safely reinterpret_cast
+// signed integers as unsigned.
+
+/// \brief The Take implementation for primitive (fixed-width) types does not
+/// use the logical Arrow type but rather the physical C type. This way we
+/// only generate one take function for each byte width.
+///
+/// This function assumes that the indices have been boundschecked.
+template <typename IndexCType, typename ValueCType>
+struct PrimitiveTakeImpl {
+  static void Exec(const PrimitiveArg& values, const PrimitiveArg& indices,
+                   Datum* out_datum) {
+    auto values_data = reinterpret_cast<const ValueCType*>(values.data);
+    auto values_is_valid = values.is_valid;
+    auto values_offset = values.offset;
+
+    auto indices_data = reinterpret_cast<const IndexCType*>(indices.data);
+    auto indices_is_valid = indices.is_valid;
+    auto indices_offset = indices.offset;
+
+    ArrayData* out_arr = out_datum->mutable_array();
+    auto out = out_arr->GetMutableValues<ValueCType>(1);
+    auto out_is_valid = out_arr->buffers[0]->mutable_data();
+    auto out_offset = out_arr->offset;
+
+    // If either the values or indices have nulls, we preemptively zero out the
+    // out validity bitmap so that we don't have to use ClearBit in each
+    // iteration for nulls.
+    if (values.null_count > 0 || indices.null_count > 0) {
+      BitUtil::SetBitsTo(out_is_valid, out_offset, indices.length, false);
+    }
+
+    OptionalBitBlockCounter indices_bit_counter(indices_is_valid, indices_offset,
+                                                indices.length);
+    int64_t position = 0;
+    int64_t valid_count = 0;
+    while (position < indices.length) {
+      BitBlockCount block = indices_bit_counter.NextBlock();
+      if (values.null_count == 0) {
+        // Values are never null, so things are easier
+        valid_count += block.popcount;
+        if (block.popcount == block.length) {
+          // Fastest path: neither values nor index nulls
+          BitUtil::SetBitsTo(out_is_valid, out_offset + position, block.length, true);
+          for (int64_t i = 0; i < block.length; ++i) {
+            out[position] = values_data[indices_data[position]];
+            ++position;
+          }
+        } else if (block.popcount > 0) {
+          // Slow path: some indices but not all are null
+          for (int64_t i = 0; i < block.length; ++i) {
+            if (BitUtil::GetBit(indices_is_valid, indices_offset + position)) {
+              // index is not null
+              BitUtil::SetBit(out_is_valid, out_offset + position);
+              out[position] = values_data[indices_data[position]];
+            } else {
+              out[position] = ValueCType{};
+            }
+            ++position;
+          }
+        } else {
+          memset(out + position, 0, sizeof(ValueCType) * block.length);
+          position += block.length;
+        }
+      } else {
+        // Values have nulls, so we must do random access into the values bitmap
+        if (block.popcount == block.length) {
+          // Faster path: indices are not null but values may be
+          for (int64_t i = 0; i < block.length; ++i) {
+            if (BitUtil::GetBit(values_is_valid,
+                                values_offset + indices_data[position])) {
+              // value is not null
+              out[position] = values_data[indices_data[position]];
+              BitUtil::SetBit(out_is_valid, out_offset + position);
+              ++valid_count;
+            } else {
+              out[position] = ValueCType{};
+            }
+            ++position;
+          }
+        } else if (block.popcount > 0) {
+          // Slow path: some but not all indices are null. Since we are doing
+          // random access in general we have to check the value nullness one by
+          // one.
+          for (int64_t i = 0; i < block.length; ++i) {
+            if (BitUtil::GetBit(indices_is_valid, indices_offset + position) &&
+                BitUtil::GetBit(values_is_valid,
+                                values_offset + indices_data[position])) {
+              // index is not null && value is not null
+              out[position] = values_data[indices_data[position]];
+              BitUtil::SetBit(out_is_valid, out_offset + position);
+              ++valid_count;
+            } else {
+              out[position] = ValueCType{};
+            }
+            ++position;
+          }
+        } else {
+          memset(out + position, 0, sizeof(ValueCType) * block.length);
+          position += block.length;
+        }
+      }
+    }
+    out_arr->null_count = out_arr->length - valid_count;
+  }
+};
+
+template <typename IndexCType>
+struct BooleanTakeImpl {
+  static void Exec(const PrimitiveArg& values, const PrimitiveArg& indices,
+                   Datum* out_datum) {
+    const uint8_t* values_data = values.data;
+    auto values_is_valid = values.is_valid;
+    auto values_offset = values.offset;
+
+    auto indices_data = reinterpret_cast<const IndexCType*>(indices.data);
+    auto indices_is_valid = indices.is_valid;
+    auto indices_offset = indices.offset;
+
+    ArrayData* out_arr = out_datum->mutable_array();
+    auto out = out_arr->buffers[1]->mutable_data();
+    auto out_is_valid = out_arr->buffers[0]->mutable_data();
+    auto out_offset = out_arr->offset;
+
+    // If either the values or indices have nulls, we preemptively zero out the
+    // out validity bitmap so that we don't have to use ClearBit in each
+    // iteration for nulls.
+    if (values.null_count > 0 || indices.null_count > 0) {
+      BitUtil::SetBitsTo(out_is_valid, out_offset, indices.length, false);
+    }
+    // Avoid uninitialized data in values array
+    BitUtil::SetBitsTo(out, out_offset, indices.length, false);
+
+    auto PlaceDataBit = [&](int64_t loc, IndexCType index) {
+      BitUtil::SetBitTo(out, out_offset + loc,
+                        BitUtil::GetBit(values_data, values_offset + index));
+    };
+
+    OptionalBitBlockCounter indices_bit_counter(indices_is_valid, indices_offset,
+                                                indices.length);
+    int64_t position = 0;
+    int64_t valid_count = 0;
+    while (position < indices.length) {
+      BitBlockCount block = indices_bit_counter.NextBlock();
+      if (values.null_count == 0) {
+        // Values are never null, so things are easier
+        valid_count += block.popcount;
+        if (block.popcount == block.length) {
+          // Fastest path: neither values nor index nulls
+          BitUtil::SetBitsTo(out_is_valid, out_offset + position, block.length, true);
+          for (int64_t i = 0; i < block.length; ++i) {
+            PlaceDataBit(position, indices_data[position]);
+            ++position;
+          }
+        } else if (block.popcount > 0) {
+          // Slow path: some but not all indices are null
+          for (int64_t i = 0; i < block.length; ++i) {
+            if (BitUtil::GetBit(indices_is_valid, indices_offset + position)) {
+              // index is not null
+              BitUtil::SetBit(out_is_valid, out_offset + position);
+              PlaceDataBit(position, indices_data[position]);
+            }
+            ++position;
+          }
+        } else {
+          position += block.length;
+        }
+      } else {
+        // Values have nulls, so we must do random access into the values bitmap
+        if (block.popcount == block.length) {
+          // Faster path: indices are not null but values may be
+          for (int64_t i = 0; i < block.length; ++i) {
+            if (BitUtil::GetBit(values_is_valid,
+                                values_offset + indices_data[position])) {
+              // value is not null
+              BitUtil::SetBit(out_is_valid, out_offset + position);
+              PlaceDataBit(position, indices_data[position]);
+              ++valid_count;
+            }
+            ++position;
+          }
+        } else if (block.popcount > 0) {
+          // Slow path: some but not all indices are null. Since we are doing
+          // random access in general we have to check the value nullness one by
+          // one.
+          for (int64_t i = 0; i < block.length; ++i) {
+            if (BitUtil::GetBit(indices_is_valid, indices_offset + position)) {
+              // index is not null
+              if (BitUtil::GetBit(values_is_valid,
+                                  values_offset + indices_data[position])) {
+                // value is not null
+                PlaceDataBit(position, indices_data[position]);
+                BitUtil::SetBit(out_is_valid, out_offset + position);
+                ++valid_count;
+              }
+            }
+            ++position;
+          }
+        } else {
+          position += block.length;
+        }
+      }
+    }
+    out_arr->null_count = out_arr->length - valid_count;
+  }
+};
+
+template <template <typename...> class TakeImpl, typename... Args>
+void TakeIndexDispatch(const PrimitiveArg& values, const PrimitiveArg& indices,
+                       Datum* out) {
+  // With the simplifying assumption that boundschecking has taken place
+  // already at a higher level, we can now assume that the index values are all
+  // non-negative. Thus, we can interpret signed integers as unsigned and avoid
+  // having to generate double the amount of binary code to handle each integer
+  // width.
+  switch (indices.bit_width) {
+    case 8:
+      return TakeImpl<uint8_t, Args...>::Exec(values, indices, out);
+    case 16:
+      return TakeImpl<uint16_t, Args...>::Exec(values, indices, out);
+    case 32:
+      return TakeImpl<uint32_t, Args...>::Exec(values, indices, out);
+    case 64:
+      return TakeImpl<uint64_t, Args...>::Exec(values, indices, out);
+    default:
+      DCHECK(false) << "Invalid indices byte width";
+      break;
+  }
+}
+
+void PrimitiveTake(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+  const auto& state = checked_cast<const TakeState&>(*ctx->state());
+  if (state.options.boundscheck) {
+    KERNEL_RETURN_IF_ERROR(ctx, IndexBoundsCheck(*batch[1].array(), batch[0].length()));
+  }
+
+  PrimitiveArg values = GetPrimitiveArg(*batch[0].array());
+  PrimitiveArg indices = GetPrimitiveArg(*batch[1].array());
+  KERNEL_RETURN_IF_ERROR(ctx,
+                         PreallocateData(ctx, indices.length, values.bit_width, out));
+  switch (values.bit_width) {
+    case 1:
+      return TakeIndexDispatch<BooleanTakeImpl>(values, indices, out);
+    case 8:
+      return TakeIndexDispatch<PrimitiveTakeImpl, int8_t>(values, indices, out);
+    case 16:
+      return TakeIndexDispatch<PrimitiveTakeImpl, int16_t>(values, indices, out);
+    case 32:
+      return TakeIndexDispatch<PrimitiveTakeImpl, int32_t>(values, indices, out);
+    case 64:
+      return TakeIndexDispatch<PrimitiveTakeImpl, int64_t>(values, indices, out);
+    default:
+      DCHECK(false) << "Invalid values byte width";
+      break;
+  }
+}
+
+// ----------------------------------------------------------------------
+// Optimized and streamlined filter for primitive types
+
+// Use either BitBlockCounter or BinaryBitBlockCounter to quickly scan filter a
+// word at a time for the DROP selection type.
+class DropNullCounter {
+ public:
+  // validity bitmap may be null
+  DropNullCounter(const uint8_t* validity, const uint8_t* data, int64_t offset,
+                  int64_t length)
+      : data_counter_(data, offset, length),
+        data_and_validity_counter_(data, offset, validity, offset, length),
+        has_validity_(validity != nullptr) {}
+
+  BitBlockCount NextBlock() {
+    if (has_validity_) {
+      // filter is true AND not null
+      return data_and_validity_counter_.NextAndWord();
+    } else {
+      return data_counter_.NextWord();
+    }
+  }
+
+ private:
+  // For when just data is present, but no validity bitmap
+  BitBlockCounter data_counter_;
+
+  // For when both validity bitmap and data are present
+  BinaryBitBlockCounter data_and_validity_counter_;
+  bool has_validity_;
+};
+
+/// \brief The Filter implementation for primitive (fixed-width) types does not
+/// use the logical Arrow type but rather then physical C type. This way we
+/// only generate one take function for each byte width. We use the same
+/// implementation here for boolean and fixed-byte-size inputs with some
+/// template specialization.
+template <typename ArrowType>
+class PrimitiveFilterImpl {
+ public:
+  using T = typename GetCType<ArrowType>::type;
+
+  PrimitiveFilterImpl(const PrimitiveArg& values, const PrimitiveArg& filter,
+                      FilterOptions::NullSelectionBehavior null_selection,
+                      Datum* out_datum)
+      : values_is_valid_(values.is_valid),
+        values_data_(reinterpret_cast<const T*>(values.data)),
+        values_null_count_(values.null_count),
+        values_offset_(values.offset),
+        values_length_(values.length),
+        filter_is_valid_(filter.is_valid),
+        filter_data_(filter.data),
+        filter_null_count_(filter.null_count),
+        filter_offset_(filter.offset),
+        null_selection_(null_selection) {
+    ArrayData* out_arr = out_datum->mutable_array();
+    out_is_valid_ = out_arr->buffers[0]->mutable_data();
+    out_data_ = reinterpret_cast<T*>(out_arr->buffers[1]->mutable_data());
+    out_offset_ = out_arr->offset;
+    out_length_ = out_arr->length;
+    out_position_ = 0;
+  }
+
+  void ExecNonNull() {
+    // The result is all not-null
+    BitUtil::SetBitsTo(out_is_valid_, out_offset_ + out_position_, out_length_, true);
+
+    // Fast filter when values and filter are not null
+    // Bit counters used for both null_selection behaviors
+    BitBlockCounter filter_counter(filter_data_, filter_offset_, values_length_);
+
+    int64_t in_position = 0;
+    BitBlockCount current_block = filter_counter.NextWord();
+    while (in_position < values_length_) {
+      if (current_block.AllSet()) {
+        int64_t run_length = 0;
+        // If we've found a all-true block, then we scan forward until we find
+        // a block that has some false values (or we reach the end
+        while (current_block.length > 0 && current_block.AllSet()) {
+          run_length += current_block.length;
+          current_block = filter_counter.NextWord();
+        }
+        WriteValueSegment(in_position, run_length);
+        in_position += run_length;
+      } else if (current_block.NoneSet()) {
+        // Nothing selected
+        in_position += current_block.length;
+        current_block = filter_counter.NextWord();
+      } else {
+        // Some values selected
+        for (int64_t i = 0; i < current_block.length; ++i) {
+          if (BitUtil::GetBit(filter_data_, filter_offset_ + in_position)) {
+            WriteValue(in_position);
+          }
+          ++in_position;
+        }
+        current_block = filter_counter.NextWord();
+      }
+    }
+  }
+
+  void Exec() {
+    if (filter_null_count_ == 0 && values_null_count_ == 0) {
+      return ExecNonNull();
+    }
+
+    // Bit counters used for both null_selection behaviors
+    DropNullCounter drop_null_counter(filter_is_valid_, filter_data_, filter_offset_,
+                                      values_length_);
+    OptionalBitBlockCounter data_counter(values_is_valid_, values_offset_,
+                                         values_length_);
+    OptionalBitBlockCounter filter_valid_counter(filter_is_valid_, filter_offset_,
+                                                 values_length_);
+
+    auto WriteNotNull = [&](int64_t index) {
+      BitUtil::SetBit(out_is_valid_, out_offset_ + out_position_);
+      // Increments out_position_
+      WriteValue(index);
+    };
+
+    auto WriteMaybeNull = [&](int64_t index) {
+      BitUtil::SetBitTo(out_is_valid_, out_offset_ + out_position_,
+                        BitUtil::GetBit(values_is_valid_, values_offset_ + index));
+      // Increments out_position_
+      WriteValue(index);
+    };
+
+    int64_t in_position = 0;
+    while (in_position < values_length_) {
+      BitBlockCount filter_block = drop_null_counter.NextBlock();
+      BitBlockCount filter_valid_block = filter_valid_counter.NextWord();
+      BitBlockCount data_block = data_counter.NextWord();
+      if (filter_block.AllSet() && data_block.AllSet()) {
+        // Fastest path: all values in block are included and not null
+        BitUtil::SetBitsTo(out_is_valid_, out_offset_ + out_position_,
+                           filter_block.length, true);
+        WriteValueSegment(in_position, filter_block.length);
+        in_position += filter_block.length;
+      } else if (filter_block.AllSet()) {
+        // Faster: all values are selected, but some values are null
+        // Batch copy bits from values validity bitmap to output validity bitmap
+        CopyBitmap(values_is_valid_, values_offset_ + in_position, filter_block.length,
+                   out_is_valid_, out_offset_ + out_position_);
+        WriteValueSegment(in_position, filter_block.length);
+        in_position += filter_block.length;
+      } else if (filter_block.NoneSet() && null_selection_ == FilterOptions::DROP) {
+        // For this exceedingly common case in low-selectivity filters we can
+        // skip further analysis of the data and move on to the next block.
+        in_position += filter_block.length;
+      } else {
+        // Some filter values are false or null
+        if (data_block.AllSet()) {
+          // No values are null
+          if (filter_valid_block.AllSet()) {
+            // Filter is non-null but some values are false
+            for (int64_t i = 0; i < filter_block.length; ++i) {
+              if (BitUtil::GetBit(filter_data_, filter_offset_ + in_position)) {
+                WriteNotNull(in_position);
+              }
+              ++in_position;
+            }
+          } else if (null_selection_ == FilterOptions::DROP) {
+            // If any values are selected, they ARE NOT null
+            for (int64_t i = 0; i < filter_block.length; ++i) {
+              if (BitUtil::GetBit(filter_is_valid_, filter_offset_ + in_position) &&
+                  BitUtil::GetBit(filter_data_, filter_offset_ + in_position)) {
+                WriteNotNull(in_position);
+              }
+              ++in_position;
+            }
+          } else {  // null_selection == FilterOptions::EMIT_NULL
+            // Data values in this block are not null
+            for (int64_t i = 0; i < filter_block.length; ++i) {
+              const bool is_valid =
+                  BitUtil::GetBit(filter_is_valid_, filter_offset_ + in_position);
+              if (is_valid &&
+                  BitUtil::GetBit(filter_data_, filter_offset_ + in_position)) {
+                // Filter slot is non-null and set
+                WriteNotNull(in_position);
+              } else if (!is_valid) {
+                // Filter slot is null, so we have a null in the output
+                BitUtil::ClearBit(out_is_valid_, out_offset_ + out_position_);
+                WriteNull();
+              }
+              ++in_position;
+            }
+          }
+        } else {  // !data_block.AllSet()
+          // Some values are null
+          if (filter_valid_block.AllSet()) {
+            // Filter is non-null but some values are false
+            for (int64_t i = 0; i < filter_block.length; ++i) {
+              if (BitUtil::GetBit(filter_data_, filter_offset_ + in_position)) {
+                WriteMaybeNull(in_position);
+              }
+              ++in_position;
+            }
+          } else if (null_selection_ == FilterOptions::DROP) {
+            // If any values are selected, they ARE NOT null
+            for (int64_t i = 0; i < filter_block.length; ++i) {
+              if (BitUtil::GetBit(filter_is_valid_, filter_offset_ + in_position) &&
+                  BitUtil::GetBit(filter_data_, filter_offset_ + in_position)) {
+                WriteMaybeNull(in_position);
+              }
+              ++in_position;
+            }
+          } else {  // null_selection == FilterOptions::EMIT_NULL
+            // Data values in this block are not null
+            for (int64_t i = 0; i < filter_block.length; ++i) {
+              const bool is_valid =
+                  BitUtil::GetBit(filter_is_valid_, filter_offset_ + in_position);
+              if (is_valid &&
+                  BitUtil::GetBit(filter_data_, filter_offset_ + in_position)) {
+                // Filter slot is non-null and set
+                WriteMaybeNull(in_position);
+              } else if (!is_valid) {
+                // Filter slot is null, so we have a null in the output
+                BitUtil::ClearBit(out_is_valid_, out_offset_ + out_position_);
+                WriteNull();
+              }
+              ++in_position;
+            }
+          }
+        }
+      }  // !filter_block.AllSet()
+    }    // while(in_position < values_length_)
+  }
+
+  // Write the next out_position given the selected in_position for the input
+  // data and advance out_position
+  void WriteValue(int64_t in_position) {
+    out_data_[out_position_++] = values_data_[in_position];
+  }
+
+  void WriteValueSegment(int64_t in_start, int64_t length) {
+    std::memcpy(out_data_ + out_position_, values_data_ + in_start, length * sizeof(T));
+    out_position_ += length;
+  }
+
+  void WriteNull() {
+    // Zero the memory
+    out_data_[out_position_++] = T{};
+  }
+
+ private:
+  const uint8_t* values_is_valid_;
+  const T* values_data_;
+  int64_t values_null_count_;
+  int64_t values_offset_;
+  int64_t values_length_;
+  const uint8_t* filter_is_valid_;
+  const uint8_t* filter_data_;
+  int64_t filter_null_count_;
+  int64_t filter_offset_;
+  FilterOptions::NullSelectionBehavior null_selection_;
+  uint8_t* out_is_valid_;
+  T* out_data_;
+  int64_t out_offset_;
+  int64_t out_length_;
+  int64_t out_position_;
+};
+
+template <>
+inline void PrimitiveFilterImpl<BooleanType>::WriteValue(int64_t in_position) {
+  BitUtil::SetBitTo(out_data_, out_offset_ + out_position_++,
+                    BitUtil::GetBit(values_data_, values_offset_ + in_position));
+}
+
+template <>
+inline void PrimitiveFilterImpl<BooleanType>::WriteValueSegment(int64_t in_start,
+                                                                int64_t length) {
+  CopyBitmap(values_data_, values_offset_ + in_start, length, out_data_,
+             out_offset_ + out_position_);
+  out_position_ += length;
+}
+
+template <>
+inline void PrimitiveFilterImpl<BooleanType>::WriteNull() {
+  // Zero the bit
+  BitUtil::ClearBit(out_data_, out_offset_ + out_position_++);
+}
+
+void PrimitiveFilter(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+  const auto& state = checked_cast<const FilterState&>(*ctx->state());
+  PrimitiveArg values = GetPrimitiveArg(*batch[0].array());
+  PrimitiveArg filter = GetPrimitiveArg(*batch[1].array());
+  FilterOptions::NullSelectionBehavior null_selection =
+      state.options.null_selection_behavior;
+
+  int64_t output_length = GetFilterOutputSize(*batch[1].array(), null_selection);
+  KERNEL_RETURN_IF_ERROR(ctx, PreallocateData(ctx, output_length, values.bit_width, out));
+
+  // The output precomputed null count is unknown except in the narrow
+  // condition that all the values are non-null and the filter will not cause
+  // any new nulls to be created.
+  if (values.null_count == 0 &&
+      (null_selection == FilterOptions::DROP || filter.null_count == 0)) {
+    out->mutable_array()->null_count = 0;
+  } else {
+    out->mutable_array()->null_count = kUnknownNullCount;
+  }
+  switch (values.bit_width) {
+    case 1:
+      return PrimitiveFilterImpl<BooleanType>(values, filter, null_selection, out).Exec();
+    case 8:
+      return PrimitiveFilterImpl<UInt8Type>(values, filter, null_selection, out).Exec();
+    case 16:
+      return PrimitiveFilterImpl<UInt16Type>(values, filter, null_selection, out).Exec();
+    case 32:
+      return PrimitiveFilterImpl<UInt32Type>(values, filter, null_selection, out).Exec();
+    case 64:
+      return PrimitiveFilterImpl<UInt64Type>(values, filter, null_selection, out).Exec();
+    default:
+      DCHECK(false) << "Invalid values bit width";
+      break;
+  }
+}
+
+// ----------------------------------------------------------------------
+// Null take and filter
+
+void NullTake(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+  const auto& state = checked_cast<const TakeState&>(*ctx->state());
+  if (state.options.boundscheck) {
+    KERNEL_RETURN_IF_ERROR(ctx, IndexBoundsCheck(*batch[1].array(), batch[0].length()));
+  }
+  out->value = std::make_shared<NullArray>(batch.length)->data();
+}
+
+void NullFilter(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+  const auto& state = checked_cast<const FilterState&>(*ctx->state());
+  int64_t output_length =
+      GetFilterOutputSize(*batch[1].array(), state.options.null_selection_behavior);
+  out->value = std::make_shared<NullArray>(output_length)->data();
+}
+
+// ----------------------------------------------------------------------
+// Dictionary take and filter
+
+void DictionaryTake(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+  const auto& state = checked_cast<const TakeState&>(*ctx->state());
+  DictionaryArray values(batch[0].array());
+  Datum result;
+  KERNEL_RETURN_IF_ERROR(
+      ctx, Take(Datum(values.indices()), batch[1], state.options, ctx->exec_context())
+               .Value(&result));
+  DictionaryArray taken_values(values.type(), result.make_array(), values.dictionary());
+  out->value = taken_values.data();
+}
+
+void DictionaryFilter(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+  const auto& state = checked_cast<const FilterState&>(*ctx->state());
+  DictionaryArray dict_values(batch[0].array());
+  Datum result;
+  KERNEL_RETURN_IF_ERROR(ctx, Filter(Datum(dict_values.indices()), batch[1].array(),
+                                     state.options, ctx->exec_context())
+                                  .Value(&result));
+  DictionaryArray filtered_values(dict_values.type(), result.make_array(),
+                                  dict_values.dictionary());
+  out->value = filtered_values.data();
+}
+
+// ----------------------------------------------------------------------
+// Extension take and filter
+
+void ExtensionTake(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+  const auto& state = checked_cast<const TakeState&>(*ctx->state());
+  ExtensionArray values(batch[0].array());
+  Datum result;
+  KERNEL_RETURN_IF_ERROR(
+      ctx, Take(Datum(values.storage()), batch[1], state.options, ctx->exec_context())
+               .Value(&result));
+  ExtensionArray taken_values(values.type(), result.make_array());
+  out->value = taken_values.data();
+}
+
+void ExtensionFilter(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+  const auto& state = checked_cast<const FilterState&>(*ctx->state());
+  ExtensionArray ext_values(batch[0].array());
+  Datum result;
+  KERNEL_RETURN_IF_ERROR(ctx, Filter(Datum(ext_values.storage()), batch[1].array(),
+                                     state.options, ctx->exec_context())
+                                  .Value(&result));
+  ExtensionArray filtered_values(ext_values.type(), result.make_array());
+  out->value = filtered_values.data();
+}
+
+// ----------------------------------------------------------------------
+// Implement take for other data types where there is less performance
+// sensitivity by visiting the selected indices.
+
+// Use CRTP to dispatch to type-specific processing of take indices for each
+// unsigned integer type.
+template <typename Impl, typename Type>
+struct Selection {
+  using ValuesArrayType = typename TypeTraits<Type>::ArrayType;
+
+  // Forwards the generic value visitors to the take index visitor template
+  template <typename IndexCType>
+  struct TakeAdapter {
+    static constexpr bool is_take = true;
+
+    Impl* impl;
+    explicit TakeAdapter(Impl* impl) : impl(impl) {}
+    template <typename ValidVisitor, typename NullVisitor>
+    Status Generate(ValidVisitor&& visit_valid, NullVisitor&& visit_null) {
+      return impl->template VisitTake<IndexCType>(std::forward<ValidVisitor>(visit_valid),
+                                                  std::forward<NullVisitor>(visit_null));
+    }
+  };
+
+  // Forwards the generic value visitors to the VisitFilter template
+  struct FilterAdapter {
+    static constexpr bool is_take = false;
+
+    Impl* impl;
+    explicit FilterAdapter(Impl* impl) : impl(impl) {}
+    template <typename ValidVisitor, typename NullVisitor>
+    Status Generate(ValidVisitor&& visit_valid, NullVisitor&& visit_null) {
+      return impl->VisitFilter(std::forward<ValidVisitor>(visit_valid),
+                               std::forward<NullVisitor>(visit_null));
+    }
+  };
+
+  KernelContext* ctx;
+  std::shared_ptr<ArrayData> values;
+  std::shared_ptr<ArrayData> selection;
+  int64_t output_length;
+  ArrayData* out;
+  TypedBufferBuilder<bool> validity_builder;
+
+  Selection(KernelContext* ctx, const ExecBatch& batch, int64_t output_length, Datum* out)
+      : ctx(ctx),
+        values(batch[0].array()),
+        selection(batch[1].array()),
+        output_length(output_length),
+        out(out->mutable_array()),
+        validity_builder(ctx->memory_pool()) {}
+
+  virtual ~Selection() = default;
+
+  Status FinishCommon() {
+    out->buffers.resize(values->buffers.size());
+    out->length = validity_builder.length();
+    out->null_count = validity_builder.false_count();
+    return validity_builder.Finish(&out->buffers[0]);
+  }
+
+  template <typename IndexCType, typename ValidVisitor, typename NullVisitor>
+  Status VisitTake(ValidVisitor&& visit_valid, NullVisitor&& visit_null) {
+    const auto indices_values = selection->GetValues<IndexCType>(1);
+    const uint8_t* is_valid = GetValidityBitmap(*selection);
+    OptionalBitIndexer indices_is_valid(selection->buffers[0], selection->offset);
+    OptionalBitIndexer values_is_valid(values->buffers[0], values->offset);
+    const bool values_have_nulls = (values->GetNullCount() > 0);
+
+    OptionalBitBlockCounter bit_counter(is_valid, selection->offset, selection->length);
+    int64_t position = 0;
+    while (position < selection->length) {
+      BitBlockCount block = bit_counter.NextBlock();
+      const bool indices_have_nulls = block.popcount < block.length;
+      if (!indices_have_nulls && !values_have_nulls) {
+        // Fastest path, neither indices nor values have nulls
+        validity_builder.UnsafeAppend(block.length, true);
+        for (int64_t i = 0; i < block.length; ++i) {
+          RETURN_NOT_OK(visit_valid(indices_values[position++]));
+        }
+      } else if (block.popcount > 0) {
+        // Since we have to branch on whether the indices are null or not, we
+        // combine the "non-null indices block but some values null" and
+        // "some-null indices block but values non-null" into a single loop.
+        for (int64_t i = 0; i < block.length; ++i) {
+          if ((!indices_have_nulls || indices_is_valid[position]) &&
+              values_is_valid[indices_values[position]]) {
+            validity_builder.UnsafeAppend(true);
+            RETURN_NOT_OK(visit_valid(indices_values[position]));
+          } else {
+            validity_builder.UnsafeAppend(false);
+            RETURN_NOT_OK(visit_null());
+          }
+          ++position;
+        }
+      } else {
+        // The whole block is null
+        validity_builder.UnsafeAppend(block.length, false);
+        for (int64_t i = 0; i < block.length; ++i) {
+          RETURN_NOT_OK(visit_null());
+        }
+        position += block.length;
+      }
+    }
+    return Status::OK();
+  }
+
+  // We use the NullVisitor both for "selected" nulls as well as "emitted"
+  // nulls coming from the filter when using FilterOptions::EMIT_NULL
+  template <typename ValidVisitor, typename NullVisitor>
+  Status VisitFilter(ValidVisitor&& visit_valid, NullVisitor&& visit_null) {
+    const auto& state = checked_cast<const FilterState&>(*ctx->state());
+    auto null_selection = state.options.null_selection_behavior;
+
+    const auto filter_data = selection->buffers[1]->data();
+
+    const uint8_t* filter_is_valid = GetValidityBitmap(*selection);
+    const int64_t filter_offset = selection->offset;
+    OptionalBitIndexer values_is_valid(values->buffers[0], values->offset);
+
+    // We use 3 block counters for fast scanning of the filter
+    //
+    // * values_valid_counter: for values null/not-null
+    // * filter_valid_counter: for filter null/not-null
+    // * filter_counter: for filter true/false
+    OptionalBitBlockCounter values_valid_counter(GetValidityBitmap(*values),
+                                                 values->offset, values->length);
+    OptionalBitBlockCounter filter_valid_counter(filter_is_valid, filter_offset,
+                                                 selection->length);
+    BitBlockCounter filter_counter(filter_data, filter_offset, selection->length);
+    int64_t in_position = 0;
+
+    auto AppendNotNull = [&](int64_t index) -> Status {
+      validity_builder.UnsafeAppend(true);
+      return visit_valid(index);
+    };
+
+    auto AppendNull = [&]() -> Status {
+      validity_builder.UnsafeAppend(false);
+      return visit_null();
+    };
+
+    auto AppendMaybeNull = [&](int64_t index) -> Status {
+      if (values_is_valid[index]) {
+        return AppendNotNull(index);
+      } else {
+        return AppendNull();
+      }
+    };
+
+    while (in_position < selection->length) {
+      BitBlockCount filter_valid_block = filter_valid_counter.NextWord();
+      BitBlockCount values_valid_block = values_valid_counter.NextWord();
+      BitBlockCount filter_block = filter_counter.NextWord();
+      if (filter_block.NoneSet() && null_selection == FilterOptions::DROP) {
+        // For this exceedingly common case in low-selectivity filters we can
+        // skip further analysis of the data and move on to the next block.
+        in_position += filter_block.length;
+      } else if (filter_valid_block.AllSet()) {
+        // Simpler path: no filter values are null
+        if (filter_block.AllSet()) {
+          // Fastest path: filter values are all true and not null
+          if (values_valid_block.AllSet()) {
+            // The values aren't null either
+            validity_builder.UnsafeAppend(filter_block.length, true);
+            for (int64_t i = 0; i < filter_block.length; ++i) {
+              RETURN_NOT_OK(visit_valid(in_position++));
+            }
+          } else {
+            // Some of the values in this block are null
+            for (int64_t i = 0; i < filter_block.length; ++i) {
+              RETURN_NOT_OK(AppendMaybeNull(in_position++));
+            }
+          }
+        } else {  // !filter_block.AllSet()
+          // Some of the filter values are false, but all not null
+          if (values_valid_block.AllSet()) {
+            // All the values are not-null, so we can skip null checking for
+            // them
+            for (int64_t i = 0; i < filter_block.length; ++i) {
+              if (BitUtil::GetBit(filter_data, filter_offset + in_position)) {
+                RETURN_NOT_OK(AppendNotNull(in_position));
+              }
+              ++in_position;
+            }
+          } else {
+            // Some of the values in the block are null, so we have to check
+            // each one
+            for (int64_t i = 0; i < filter_block.length; ++i) {
+              if (BitUtil::GetBit(filter_data, filter_offset + in_position)) {
+                RETURN_NOT_OK(AppendMaybeNull(in_position));
+              }
+              ++in_position;
+            }
+          }
+        }
+      } else {  // !filter_valid_block.AllSet()
+        // Some of the filter values are null, so we have to handle the DROP
+        // versus EMIT_NULL null selection behavior.
+        if (null_selection == FilterOptions::DROP) {

Review comment:
       I'm extracting some code into lambdas and doing this transform




----------------------------------------------------------------
This is an automated message from the 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] wesm commented on a change in pull request #7442: ARROW-9075: [C++] Optimized Filter implementation: faster performance + compilation, smaller code size

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



##########
File path: cpp/src/arrow/testing/random.cc
##########
@@ -84,7 +84,7 @@ std::shared_ptr<Array> RandomArrayGenerator::Boolean(int64_t size, double probab
 
   BufferVector buffers{2};
   // Need 2 distinct generators such that probabilities are not shared.
-  GenOpt value_gen(seed(), 0, 1, probability);
+  GenOpt value_gen(seed(), 0, 1, 1 - probability);

Review comment:
       @pitrou or @fsaintjacques could you please confirm this change




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

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



[GitHub] [arrow] wesm commented on a change in pull request #7442: ARROW-9075: [C++] Optimized Filter implementation: faster performance + compilation, smaller code size

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



##########
File path: cpp/src/arrow/compute/kernels/vector_selection.cc
##########
@@ -0,0 +1,1816 @@
+// 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 <algorithm>
+#include <cstring>
+#include <limits>
+
+#include "arrow/array/array_base.h"
+#include "arrow/array/array_binary.h"
+#include "arrow/array/array_dict.h"
+#include "arrow/array/array_nested.h"
+#include "arrow/array/builder_primitive.h"
+#include "arrow/array/concatenate.h"
+#include "arrow/buffer_builder.h"
+#include "arrow/compute/api_vector.h"
+#include "arrow/compute/kernels/common.h"
+#include "arrow/compute/kernels/util_internal.h"
+#include "arrow/extension_type.h"
+#include "arrow/record_batch.h"
+#include "arrow/result.h"
+#include "arrow/util/bit_block_counter.h"
+#include "arrow/util/bit_util.h"
+#include "arrow/util/bitmap_ops.h"
+#include "arrow/util/bitmap_reader.h"
+#include "arrow/util/int_util.h"
+
+namespace arrow {
+
+using internal::BinaryBitBlockCounter;
+using internal::BitBlockCount;
+using internal::BitBlockCounter;
+using internal::BitmapReader;
+using internal::CopyBitmap;
+using internal::GetArrayView;
+using internal::IndexBoundsCheck;
+using internal::OptionalBitBlockCounter;
+using internal::OptionalBitIndexer;
+
+namespace compute {
+namespace internal {
+
+int64_t GetFilterOutputSize(const ArrayData& filter,
+                            FilterOptions::NullSelectionBehavior null_selection) {
+  int64_t output_size = 0;
+  int64_t position = 0;
+  if (filter.GetNullCount() > 0) {
+    const uint8_t* filter_is_valid = filter.buffers[0]->data();
+    BinaryBitBlockCounter bit_counter(filter.buffers[1]->data(), filter.offset,
+                                      filter_is_valid, filter.offset, filter.length);
+    if (null_selection == FilterOptions::EMIT_NULL) {
+      while (position < filter.length) {
+        BitBlockCount block = bit_counter.NextOrNotWord();
+        output_size += block.popcount;
+        position += block.length;
+      }
+    } else {
+      while (position < filter.length) {
+        BitBlockCount block = bit_counter.NextAndWord();
+        output_size += block.popcount;
+        position += block.length;
+      }
+    }
+  } else {
+    // The filter has no nulls, so we plow through its data as fast as
+    // possible.
+    BitBlockCounter bit_counter(filter.buffers[1]->data(), filter.offset, filter.length);
+    while (position < filter.length) {
+      BitBlockCount block = bit_counter.NextFourWords();
+      output_size += block.popcount;
+      position += block.length;
+    }
+  }
+  return output_size;
+}
+
+template <typename IndexType>
+Result<std::shared_ptr<ArrayData>> GetTakeIndicesImpl(
+    const ArrayData& filter, FilterOptions::NullSelectionBehavior null_selection,
+    MemoryPool* memory_pool) {
+  using T = typename IndexType::c_type;
+  typename TypeTraits<IndexType>::BuilderType builder(memory_pool);
+
+  const uint8_t* filter_data = filter.buffers[1]->data();
+  BitBlockCounter data_counter(filter_data, filter.offset, filter.length);
+
+  // The position relative to the start of the filter
+  T position = 0;
+
+  // The current position taking the filter offset into account
+  int64_t position_with_offset = filter.offset;
+  if (filter.GetNullCount() > 0) {
+    // The filter has nulls, so we scan the validity bitmap and the filter data
+    // bitmap together, branching on the null selection type.
+    const uint8_t* filter_is_valid = filter.buffers[0]->data();
+
+    // To count blocks whether filter_data[i] || !filter_is_valid[i]
+    BinaryBitBlockCounter filter_counter(filter_data, filter.offset, filter_is_valid,
+                                         filter.offset, filter.length);
+    if (null_selection == FilterOptions::DROP) {
+      while (position < filter.length) {
+        BitBlockCount and_block = filter_counter.NextAndWord();
+        RETURN_NOT_OK(builder.Reserve(and_block.popcount));
+        if (and_block.AllSet()) {
+          // All the values are selected and non-null
+          for (int64_t i = 0; i < and_block.length; ++i) {
+            builder.UnsafeAppend(position++);
+          }
+          position_with_offset += and_block.length;
+        } else {
+          // Some of the values are false or null
+          for (int64_t i = 0; i < and_block.length; ++i) {
+            if (BitUtil::GetBit(filter_is_valid, position_with_offset) &&
+                BitUtil::GetBit(filter_data, position_with_offset)) {
+              builder.UnsafeAppend(position);
+            }
+            ++position;
+            ++position_with_offset;
+          }
+        }
+      }
+    } else {
+      BitBlockCounter is_valid_counter(filter_is_valid, filter.offset, filter.length);
+      while (position < filter.length) {
+        // true OR NOT valid
+        BitBlockCount or_not_block = filter_counter.NextOrNotWord();
+        RETURN_NOT_OK(builder.Reserve(or_not_block.popcount));
+
+        // If the values are all valid and the or_not_block is full, then we
+        // can infer that all the values are true and skip the bit checking
+        BitBlockCount is_valid_block = is_valid_counter.NextWord();
+
+        if (or_not_block.AllSet() && is_valid_block.AllSet()) {
+          // All the values are selected and non-null
+          for (int64_t i = 0; i < or_not_block.length; ++i) {
+            builder.UnsafeAppend(position++);
+          }
+          position_with_offset += or_not_block.length;
+        } else {
+          // Some of the values are false or null
+          for (int64_t i = 0; i < or_not_block.length; ++i) {
+            if (BitUtil::GetBit(filter_is_valid, position_with_offset)) {
+              if (BitUtil::GetBit(filter_data, position_with_offset)) {
+                builder.UnsafeAppend(position);
+              }
+            } else {
+              // Null slot, so append a null
+              builder.UnsafeAppendNull();
+            }
+            ++position;
+            ++position_with_offset;
+          }
+        }
+      }
+    }
+  } else {
+    // The filter has no nulls, so we need only look for true values
+    BitBlockCount current_block = data_counter.NextWord();
+    while (position < filter.length) {
+      if (current_block.AllSet()) {
+        int64_t run_length = 0;
+
+        // If we've found a all-true block, then we scan forward until we find
+        // a block that has some false values (or we reach the end
+        while (current_block.length > 0 && current_block.AllSet()) {
+          run_length += current_block.length;
+          current_block = data_counter.NextWord();
+        }
+
+        // Append the consecutive run of indices
+        RETURN_NOT_OK(builder.Reserve(run_length));
+        for (int64_t i = 0; i < run_length; ++i) {
+          builder.UnsafeAppend(position++);
+        }
+        position_with_offset += run_length;
+      } else {
+        // Must do bitchecking on the current block
+        RETURN_NOT_OK(builder.Reserve(current_block.popcount));
+        for (int64_t i = 0; i < current_block.length; ++i) {
+          if (BitUtil::GetBit(filter_data, position_with_offset)) {
+            builder.UnsafeAppend(position);
+          }
+          ++position;
+          ++position_with_offset;
+        }
+        current_block = data_counter.NextWord();
+      }
+    }
+  }
+  std::shared_ptr<ArrayData> result;
+  RETURN_NOT_OK(builder.FinishInternal(&result));
+  return result;
+}
+
+Result<std::shared_ptr<ArrayData>> GetTakeIndices(
+    const ArrayData& filter, FilterOptions::NullSelectionBehavior null_selection,
+    MemoryPool* memory_pool) {
+  DCHECK_EQ(filter.type->id(), Type::BOOL);
+  if (filter.length <= std::numeric_limits<uint16_t>::max()) {
+    return GetTakeIndicesImpl<UInt16Type>(filter, null_selection, memory_pool);
+  } else if (filter.length <= std::numeric_limits<uint32_t>::max()) {
+    return GetTakeIndicesImpl<UInt32Type>(filter, null_selection, memory_pool);
+  } else {
+    // Arrays over 4 billion elements, not especially likely.
+    return Status::NotImplemented(
+        "Filter length exceeds UINT32_MAX, "
+        "consider a different strategy for selecting elements");
+  }
+}
+
+namespace {
+
+template <typename ArrowType>
+struct GetCType {
+  using type = typename ArrowType::c_type;
+};
+
+// We want uint8_t for boolean instead of bool
+template <>
+struct GetCType<BooleanType> {
+  using type = uint8_t;
+};
+
+using FilterState = OptionsWrapper<FilterOptions>;
+using TakeState = OptionsWrapper<TakeOptions>;
+
+Status PreallocateData(KernelContext* ctx, int64_t length, int bit_width, Datum* out) {
+  // Preallocate memory
+  ArrayData* out_arr = out->mutable_array();
+  out_arr->length = length;
+  out_arr->buffers.resize(2);
+
+  ARROW_ASSIGN_OR_RAISE(out_arr->buffers[0], ctx->AllocateBitmap(length));
+  if (bit_width == 1) {
+    ARROW_ASSIGN_OR_RAISE(out_arr->buffers[1], ctx->AllocateBitmap(length));
+  } else {
+    ARROW_ASSIGN_OR_RAISE(out_arr->buffers[1], ctx->Allocate(length * bit_width / 8));
+  }
+  return Status::OK();
+}
+
+// ----------------------------------------------------------------------
+// Implement optimized take for primitive types from boolean to 1/2/4/8-byte
+// C-type based types. Use common implementation for every byte width and only
+// generate code for unsigned integer indices, since after boundschecking to
+// check for negative numbers in the indices we can safely reinterpret_cast
+// signed integers as unsigned.
+
+/// \brief The Take implementation for primitive (fixed-width) types does not
+/// use the logical Arrow type but rather the physical C type. This way we
+/// only generate one take function for each byte width.
+///
+/// This function assumes that the indices have been boundschecked.
+template <typename IndexCType, typename ValueCType>
+struct PrimitiveTakeImpl {
+  static void Exec(const PrimitiveArg& values, const PrimitiveArg& indices,
+                   Datum* out_datum) {
+    auto values_data = reinterpret_cast<const ValueCType*>(values.data);
+    auto values_is_valid = values.is_valid;
+    auto values_offset = values.offset;
+
+    auto indices_data = reinterpret_cast<const IndexCType*>(indices.data);
+    auto indices_is_valid = indices.is_valid;
+    auto indices_offset = indices.offset;
+
+    ArrayData* out_arr = out_datum->mutable_array();
+    auto out = out_arr->GetMutableValues<ValueCType>(1);
+    auto out_is_valid = out_arr->buffers[0]->mutable_data();
+    auto out_offset = out_arr->offset;
+
+    // If either the values or indices have nulls, we preemptively zero out the
+    // out validity bitmap so that we don't have to use ClearBit in each
+    // iteration for nulls.
+    if (values.null_count > 0 || indices.null_count > 0) {
+      BitUtil::SetBitsTo(out_is_valid, out_offset, indices.length, false);
+    }
+
+    OptionalBitBlockCounter indices_bit_counter(indices_is_valid, indices_offset,
+                                                indices.length);
+    int64_t position = 0;
+    int64_t valid_count = 0;
+    while (position < indices.length) {
+      BitBlockCount block = indices_bit_counter.NextBlock();
+      if (values.null_count == 0) {
+        // Values are never null, so things are easier
+        valid_count += block.popcount;
+        if (block.popcount == block.length) {
+          // Fastest path: neither values nor index nulls
+          BitUtil::SetBitsTo(out_is_valid, out_offset + position, block.length, true);
+          for (int64_t i = 0; i < block.length; ++i) {
+            out[position] = values_data[indices_data[position]];
+            ++position;
+          }
+        } else if (block.popcount > 0) {
+          // Slow path: some indices but not all are null
+          for (int64_t i = 0; i < block.length; ++i) {
+            if (BitUtil::GetBit(indices_is_valid, indices_offset + position)) {
+              // index is not null
+              BitUtil::SetBit(out_is_valid, out_offset + position);
+              out[position] = values_data[indices_data[position]];
+            } else {
+              out[position] = ValueCType{};
+            }
+            ++position;
+          }
+        } else {
+          memset(out + position, 0, sizeof(ValueCType) * block.length);
+          position += block.length;
+        }
+      } else {
+        // Values have nulls, so we must do random access into the values bitmap
+        if (block.popcount == block.length) {
+          // Faster path: indices are not null but values may be
+          for (int64_t i = 0; i < block.length; ++i) {
+            if (BitUtil::GetBit(values_is_valid,
+                                values_offset + indices_data[position])) {
+              // value is not null
+              out[position] = values_data[indices_data[position]];
+              BitUtil::SetBit(out_is_valid, out_offset + position);
+              ++valid_count;
+            } else {
+              out[position] = ValueCType{};
+            }
+            ++position;
+          }
+        } else if (block.popcount > 0) {
+          // Slow path: some but not all indices are null. Since we are doing
+          // random access in general we have to check the value nullness one by
+          // one.
+          for (int64_t i = 0; i < block.length; ++i) {
+            if (BitUtil::GetBit(indices_is_valid, indices_offset + position) &&
+                BitUtil::GetBit(values_is_valid,
+                                values_offset + indices_data[position])) {
+              // index is not null && value is not null
+              out[position] = values_data[indices_data[position]];
+              BitUtil::SetBit(out_is_valid, out_offset + position);
+              ++valid_count;
+            } else {
+              out[position] = ValueCType{};
+            }
+            ++position;
+          }
+        } else {
+          memset(out + position, 0, sizeof(ValueCType) * block.length);
+          position += block.length;
+        }
+      }
+    }
+    out_arr->null_count = out_arr->length - valid_count;
+  }
+};
+
+template <typename IndexCType>
+struct BooleanTakeImpl {
+  static void Exec(const PrimitiveArg& values, const PrimitiveArg& indices,
+                   Datum* out_datum) {
+    const uint8_t* values_data = values.data;
+    auto values_is_valid = values.is_valid;
+    auto values_offset = values.offset;
+
+    auto indices_data = reinterpret_cast<const IndexCType*>(indices.data);
+    auto indices_is_valid = indices.is_valid;
+    auto indices_offset = indices.offset;
+
+    ArrayData* out_arr = out_datum->mutable_array();
+    auto out = out_arr->buffers[1]->mutable_data();
+    auto out_is_valid = out_arr->buffers[0]->mutable_data();
+    auto out_offset = out_arr->offset;
+
+    // If either the values or indices have nulls, we preemptively zero out the
+    // out validity bitmap so that we don't have to use ClearBit in each
+    // iteration for nulls.
+    if (values.null_count > 0 || indices.null_count > 0) {
+      BitUtil::SetBitsTo(out_is_valid, out_offset, indices.length, false);
+    }
+    // Avoid uninitialized data in values array
+    BitUtil::SetBitsTo(out, out_offset, indices.length, false);
+
+    auto PlaceDataBit = [&](int64_t loc, IndexCType index) {
+      BitUtil::SetBitTo(out, out_offset + loc,
+                        BitUtil::GetBit(values_data, values_offset + index));
+    };
+
+    OptionalBitBlockCounter indices_bit_counter(indices_is_valid, indices_offset,
+                                                indices.length);
+    int64_t position = 0;
+    int64_t valid_count = 0;
+    while (position < indices.length) {
+      BitBlockCount block = indices_bit_counter.NextBlock();
+      if (values.null_count == 0) {
+        // Values are never null, so things are easier
+        valid_count += block.popcount;
+        if (block.popcount == block.length) {
+          // Fastest path: neither values nor index nulls
+          BitUtil::SetBitsTo(out_is_valid, out_offset + position, block.length, true);
+          for (int64_t i = 0; i < block.length; ++i) {
+            PlaceDataBit(position, indices_data[position]);
+            ++position;
+          }
+        } else if (block.popcount > 0) {
+          // Slow path: some but not all indices are null
+          for (int64_t i = 0; i < block.length; ++i) {
+            if (BitUtil::GetBit(indices_is_valid, indices_offset + position)) {
+              // index is not null
+              BitUtil::SetBit(out_is_valid, out_offset + position);
+              PlaceDataBit(position, indices_data[position]);
+            }
+            ++position;
+          }
+        } else {
+          position += block.length;
+        }
+      } else {
+        // Values have nulls, so we must do random access into the values bitmap
+        if (block.popcount == block.length) {
+          // Faster path: indices are not null but values may be
+          for (int64_t i = 0; i < block.length; ++i) {
+            if (BitUtil::GetBit(values_is_valid,
+                                values_offset + indices_data[position])) {
+              // value is not null
+              BitUtil::SetBit(out_is_valid, out_offset + position);
+              PlaceDataBit(position, indices_data[position]);
+              ++valid_count;
+            }
+            ++position;
+          }
+        } else if (block.popcount > 0) {
+          // Slow path: some but not all indices are null. Since we are doing
+          // random access in general we have to check the value nullness one by
+          // one.
+          for (int64_t i = 0; i < block.length; ++i) {
+            if (BitUtil::GetBit(indices_is_valid, indices_offset + position)) {
+              // index is not null
+              if (BitUtil::GetBit(values_is_valid,
+                                  values_offset + indices_data[position])) {
+                // value is not null
+                PlaceDataBit(position, indices_data[position]);
+                BitUtil::SetBit(out_is_valid, out_offset + position);
+                ++valid_count;
+              }
+            }
+            ++position;
+          }
+        } else {
+          position += block.length;
+        }
+      }
+    }
+    out_arr->null_count = out_arr->length - valid_count;
+  }
+};
+
+template <template <typename...> class TakeImpl, typename... Args>
+void TakeIndexDispatch(const PrimitiveArg& values, const PrimitiveArg& indices,
+                       Datum* out) {
+  // With the simplifying assumption that boundschecking has taken place
+  // already at a higher level, we can now assume that the index values are all
+  // non-negative. Thus, we can interpret signed integers as unsigned and avoid
+  // having to generate double the amount of binary code to handle each integer
+  // width.
+  switch (indices.bit_width) {
+    case 8:
+      return TakeImpl<uint8_t, Args...>::Exec(values, indices, out);
+    case 16:
+      return TakeImpl<uint16_t, Args...>::Exec(values, indices, out);
+    case 32:
+      return TakeImpl<uint32_t, Args...>::Exec(values, indices, out);
+    case 64:
+      return TakeImpl<uint64_t, Args...>::Exec(values, indices, out);
+    default:
+      DCHECK(false) << "Invalid indices byte width";
+      break;
+  }
+}
+
+void PrimitiveTake(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+  const auto& state = checked_cast<const TakeState&>(*ctx->state());
+  if (state.options.boundscheck) {
+    KERNEL_RETURN_IF_ERROR(ctx, IndexBoundsCheck(*batch[1].array(), batch[0].length()));
+  }
+
+  PrimitiveArg values = GetPrimitiveArg(*batch[0].array());
+  PrimitiveArg indices = GetPrimitiveArg(*batch[1].array());
+  KERNEL_RETURN_IF_ERROR(ctx,
+                         PreallocateData(ctx, indices.length, values.bit_width, out));
+  switch (values.bit_width) {
+    case 1:
+      return TakeIndexDispatch<BooleanTakeImpl>(values, indices, out);
+    case 8:
+      return TakeIndexDispatch<PrimitiveTakeImpl, int8_t>(values, indices, out);
+    case 16:
+      return TakeIndexDispatch<PrimitiveTakeImpl, int16_t>(values, indices, out);
+    case 32:
+      return TakeIndexDispatch<PrimitiveTakeImpl, int32_t>(values, indices, out);
+    case 64:
+      return TakeIndexDispatch<PrimitiveTakeImpl, int64_t>(values, indices, out);
+    default:
+      DCHECK(false) << "Invalid values byte width";
+      break;
+  }
+}
+
+// ----------------------------------------------------------------------
+// Optimized and streamlined filter for primitive types
+
+// Use either BitBlockCounter or BinaryBitBlockCounter to quickly scan filter a
+// word at a time for the DROP selection type.
+class DropNullCounter {
+ public:
+  // validity bitmap may be null
+  DropNullCounter(const uint8_t* validity, const uint8_t* data, int64_t offset,
+                  int64_t length)
+      : data_counter_(data, offset, length),
+        data_and_validity_counter_(data, offset, validity, offset, length),
+        has_validity_(validity != nullptr) {}
+
+  BitBlockCount NextBlock() {
+    if (has_validity_) {
+      // filter is true AND not null
+      return data_and_validity_counter_.NextAndWord();
+    } else {
+      return data_counter_.NextWord();
+    }
+  }
+
+ private:
+  // For when just data is present, but no validity bitmap
+  BitBlockCounter data_counter_;
+
+  // For when both validity bitmap and data are present
+  BinaryBitBlockCounter data_and_validity_counter_;
+  bool has_validity_;
+};
+
+/// \brief The Filter implementation for primitive (fixed-width) types does not
+/// use the logical Arrow type but rather then physical C type. This way we
+/// only generate one take function for each byte width. We use the same
+/// implementation here for boolean and fixed-byte-size inputs with some
+/// template specialization.
+template <typename ArrowType>
+class PrimitiveFilterImpl {
+ public:
+  using T = typename GetCType<ArrowType>::type;
+
+  PrimitiveFilterImpl(const PrimitiveArg& values, const PrimitiveArg& filter,
+                      FilterOptions::NullSelectionBehavior null_selection,
+                      Datum* out_datum)
+      : values_is_valid_(values.is_valid),
+        values_data_(reinterpret_cast<const T*>(values.data)),
+        values_null_count_(values.null_count),
+        values_offset_(values.offset),
+        values_length_(values.length),
+        filter_is_valid_(filter.is_valid),
+        filter_data_(filter.data),
+        filter_null_count_(filter.null_count),
+        filter_offset_(filter.offset),
+        null_selection_(null_selection) {
+    ArrayData* out_arr = out_datum->mutable_array();
+    out_is_valid_ = out_arr->buffers[0]->mutable_data();
+    out_data_ = reinterpret_cast<T*>(out_arr->buffers[1]->mutable_data());
+    out_offset_ = out_arr->offset;
+    out_length_ = out_arr->length;
+    out_position_ = 0;
+  }
+
+  void ExecNonNull() {
+    // The result is all not-null
+    BitUtil::SetBitsTo(out_is_valid_, out_offset_ + out_position_, out_length_, true);
+
+    // Fast filter when values and filter are not null
+    // Bit counters used for both null_selection behaviors
+    BitBlockCounter filter_counter(filter_data_, filter_offset_, values_length_);
+
+    int64_t in_position = 0;
+    BitBlockCount current_block = filter_counter.NextWord();
+    while (in_position < values_length_) {
+      if (current_block.AllSet()) {
+        int64_t run_length = 0;
+        // If we've found a all-true block, then we scan forward until we find
+        // a block that has some false values (or we reach the end
+        while (current_block.length > 0 && current_block.AllSet()) {
+          run_length += current_block.length;
+          current_block = filter_counter.NextWord();
+        }
+        WriteValueSegment(in_position, run_length);
+        in_position += run_length;
+      } else if (current_block.NoneSet()) {
+        // Nothing selected
+        in_position += current_block.length;
+        current_block = filter_counter.NextWord();
+      } else {
+        // Some values selected
+        for (int64_t i = 0; i < current_block.length; ++i) {
+          if (BitUtil::GetBit(filter_data_, filter_offset_ + in_position)) {
+            WriteValue(in_position);
+          }
+          ++in_position;
+        }
+        current_block = filter_counter.NextWord();
+      }
+    }
+  }
+
+  void Exec() {
+    if (filter_null_count_ == 0 && values_null_count_ == 0) {
+      return ExecNonNull();
+    }
+
+    // Bit counters used for both null_selection behaviors
+    DropNullCounter drop_null_counter(filter_is_valid_, filter_data_, filter_offset_,
+                                      values_length_);
+    OptionalBitBlockCounter data_counter(values_is_valid_, values_offset_,
+                                         values_length_);
+    OptionalBitBlockCounter filter_valid_counter(filter_is_valid_, filter_offset_,
+                                                 values_length_);
+
+    auto WriteNotNull = [&](int64_t index) {
+      BitUtil::SetBit(out_is_valid_, out_offset_ + out_position_);
+      // Increments out_position_
+      WriteValue(index);
+    };
+
+    auto WriteMaybeNull = [&](int64_t index) {
+      BitUtil::SetBitTo(out_is_valid_, out_offset_ + out_position_,
+                        BitUtil::GetBit(values_is_valid_, values_offset_ + index));
+      // Increments out_position_
+      WriteValue(index);
+    };
+
+    int64_t in_position = 0;
+    while (in_position < values_length_) {

Review comment:
       Well, I'm not going to refactor it in this PR. Getting it to be correct and fast was time consuming. 




----------------------------------------------------------------
This is an automated message from the 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] wesm closed pull request #7442: ARROW-9075: [C++] Optimized Filter implementation: faster performance + compilation, smaller code size

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


   


----------------------------------------------------------------
This is an automated message from the 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] wesm commented on pull request #7442: ARROW-9075: [C++] Optimized Filter implementation: faster performance + compilation, smaller code size

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


   @buildbot benchmark --help


----------------------------------------------------------------
This is an automated message from the 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] wesm commented on a change in pull request #7442: ARROW-9075: [C++] Optimized Filter implementation: faster performance + compilation, smaller code size

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



##########
File path: cpp/src/arrow/compute/kernels/vector_selection.cc
##########
@@ -0,0 +1,1758 @@
+// 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 <algorithm>
+#include <cstring>
+#include <limits>
+
+#include "arrow/array/array_base.h"
+#include "arrow/array/array_binary.h"
+#include "arrow/array/array_dict.h"
+#include "arrow/array/array_nested.h"
+#include "arrow/array/builder_primitive.h"
+#include "arrow/array/concatenate.h"
+#include "arrow/buffer_builder.h"
+#include "arrow/compute/api_vector.h"
+#include "arrow/compute/kernels/common.h"
+#include "arrow/compute/kernels/util_internal.h"
+#include "arrow/extension_type.h"
+#include "arrow/record_batch.h"
+#include "arrow/result.h"
+#include "arrow/util/bit_block_counter.h"
+#include "arrow/util/bit_util.h"
+#include "arrow/util/bitmap_ops.h"
+#include "arrow/util/bitmap_reader.h"
+#include "arrow/util/int_util.h"
+
+namespace arrow {
+
+using internal::BinaryBitBlockCounter;
+using internal::BitBlockCount;
+using internal::BitBlockCounter;
+using internal::BitmapReader;
+using internal::CopyBitmap;
+using internal::GetArrayView;
+using internal::IndexBoundsCheck;
+using internal::OptionalBitBlockCounter;
+using internal::OptionalBitIndexer;
+
+namespace compute {
+namespace internal {
+
+int64_t GetFilterOutputSize(const ArrayData& filter,
+                            FilterOptions::NullSelectionBehavior null_selection) {
+  int64_t output_size = 0;
+  int64_t position = 0;
+  if (filter.GetNullCount() > 0) {
+    const uint8_t* filter_is_valid = filter.buffers[0]->data();
+    BinaryBitBlockCounter bit_counter(filter.buffers[1]->data(), filter.offset,
+                                      filter_is_valid, filter.offset, filter.length);
+    if (null_selection == FilterOptions::EMIT_NULL) {
+      while (position < filter.length) {
+        BitBlockCount block = bit_counter.NextOrNotWord();
+        output_size += block.popcount;
+        position += block.length;
+      }
+    } else {
+      while (position < filter.length) {
+        BitBlockCount block = bit_counter.NextAndWord();
+        output_size += block.popcount;
+        position += block.length;
+      }
+    }
+  } else {
+    // The filter has no nulls, so we plow through its data as fast as
+    // possible.
+    BitBlockCounter bit_counter(filter.buffers[1]->data(), filter.offset, filter.length);
+    while (position < filter.length) {
+      BitBlockCount block = bit_counter.NextFourWords();
+      output_size += block.popcount;
+      position += block.length;
+    }
+  }
+  return output_size;
+}
+
+template <typename IndexType>
+Result<std::shared_ptr<ArrayData>> GetTakeIndicesImpl(
+    const ArrayData& filter, FilterOptions::NullSelectionBehavior null_selection,
+    MemoryPool* memory_pool) {
+  using T = typename IndexType::c_type;
+  typename TypeTraits<IndexType>::BuilderType builder(memory_pool);
+
+  const uint8_t* filter_data = filter.buffers[1]->data();
+  BitBlockCounter data_counter(filter_data, filter.offset, filter.length);
+
+  // The position relative to the start of the filter
+  T position = 0;
+
+  // The current position taking the filter offset into account
+  int64_t position_with_offset = filter.offset;
+  if (filter.GetNullCount() > 0) {
+    // The filter has nulls, so we scan the validity bitmap and the filter data
+    // bitmap together, branching on the null selection type.
+    const uint8_t* filter_is_valid = filter.buffers[0]->data();
+
+    // To count blocks whether filter_data[i] || !filter_is_valid[i]
+    BinaryBitBlockCounter filter_counter(filter_data, filter.offset, filter_is_valid,
+                                         filter.offset, filter.length);
+    if (null_selection == FilterOptions::DROP) {
+      while (position < filter.length) {
+        BitBlockCount and_block = filter_counter.NextAndWord();
+        RETURN_NOT_OK(builder.Reserve(and_block.popcount));
+        if (and_block.IsFull()) {
+          // All the values are selected and non-null
+          for (int64_t i = 0; i < and_block.length; ++i) {
+            builder.UnsafeAppend(position++);
+          }
+          position_with_offset += and_block.length;
+        } else {
+          // Some of the values are false or null
+          for (int64_t i = 0; i < and_block.length; ++i) {
+            if (BitUtil::GetBit(filter_is_valid, position_with_offset) &&
+                BitUtil::GetBit(filter_data, position_with_offset)) {
+              builder.UnsafeAppend(position);
+            }
+            ++position;
+            ++position_with_offset;
+          }
+        }
+      }
+    } else {
+      BitBlockCounter is_valid_counter(filter_is_valid, filter.offset, filter.length);
+      while (position < filter.length) {
+        // true OR NOT valid
+        BitBlockCount or_not_block = filter_counter.NextOrNotWord();
+        RETURN_NOT_OK(builder.Reserve(or_not_block.popcount));
+
+        // If the values are all valid and the or_not_block is full, then we
+        // can infer that all the values are true and skip the bit checking
+        BitBlockCount is_valid_block = is_valid_counter.NextWord();
+
+        if (or_not_block.IsFull() && is_valid_block.IsFull()) {
+          // All the values are selected and non-null
+          for (int64_t i = 0; i < or_not_block.length; ++i) {
+            builder.UnsafeAppend(position++);
+          }
+          position_with_offset += or_not_block.length;
+        } else {
+          // Some of the values are false or null
+          for (int64_t i = 0; i < or_not_block.length; ++i) {
+            if (BitUtil::GetBit(filter_is_valid, position_with_offset)) {
+              if (BitUtil::GetBit(filter_data, position_with_offset)) {
+                builder.UnsafeAppend(position);
+              }
+            } else {
+              // Null slot, so append a null
+              builder.UnsafeAppendNull();
+            }
+            ++position;
+            ++position_with_offset;
+          }
+        }
+      }
+    }
+  } else {
+    // The filter has no nulls, so we need only look for true values
+    BitBlockCount current_block = data_counter.NextWord();
+    while (position < filter.length) {
+      if (current_block.IsFull()) {
+        int64_t run_length = 0;
+
+        // If we've found a all-true block, then we scan forward until we find
+        // a block that has some false values (or we reach the end
+        while (current_block.length > 0 && current_block.IsFull()) {
+          run_length += current_block.length;
+          current_block = data_counter.NextWord();
+        }
+
+        // Append the consecutive run of indices
+        RETURN_NOT_OK(builder.Reserve(run_length));
+        for (int64_t i = 0; i < run_length; ++i) {
+          builder.UnsafeAppend(position++);
+        }
+        position_with_offset += run_length;
+      } else {
+        // Must do bitchecking on the current block
+        RETURN_NOT_OK(builder.Reserve(current_block.popcount));
+        for (int64_t i = 0; i < current_block.length; ++i) {
+          if (BitUtil::GetBit(filter_data, position_with_offset)) {
+            builder.UnsafeAppend(position);
+          }
+          ++position;
+          ++position_with_offset;
+        }
+        current_block = data_counter.NextWord();
+      }
+    }
+  }
+  std::shared_ptr<ArrayData> result;
+  RETURN_NOT_OK(builder.FinishInternal(&result));
+  return result;
+}
+
+Result<std::shared_ptr<ArrayData>> GetTakeIndices(
+    const ArrayData& filter, FilterOptions::NullSelectionBehavior null_selection,
+    MemoryPool* memory_pool) {
+  DCHECK_EQ(filter.type->id(), Type::BOOL);
+  if (filter.length <= std::numeric_limits<uint16_t>::max()) {
+    return GetTakeIndicesImpl<UInt16Type>(filter, null_selection, memory_pool);
+  } else if (filter.length <= std::numeric_limits<uint32_t>::max()) {
+    return GetTakeIndicesImpl<UInt32Type>(filter, null_selection, memory_pool);
+  } else {
+    // Arrays over 4 billion elements, not especially likely.
+    return Status::NotImplemented(
+        "Filter length exceeds UINT32_MAX, "
+        "consider a different strategy for selecting elements");
+  }
+}
+
+namespace {
+
+template <typename ArrowType>
+struct GetCType {
+  using type = typename ArrowType::c_type;
+};
+
+// We want uint8_t for boolean instead of bool
+template <>
+struct GetCType<BooleanType> {
+  using type = uint8_t;
+};
+
+using FilterState = OptionsWrapper<FilterOptions>;
+using TakeState = OptionsWrapper<TakeOptions>;
+
+Status PreallocateData(KernelContext* ctx, int64_t length, int bit_width, Datum* out) {
+  // Preallocate memory
+  ArrayData* out_arr = out->mutable_array();
+  out_arr->length = length;
+  out_arr->buffers.resize(2);
+
+  ARROW_ASSIGN_OR_RAISE(out_arr->buffers[0], ctx->AllocateBitmap(length));
+  if (bit_width == 1) {
+    ARROW_ASSIGN_OR_RAISE(out_arr->buffers[1], ctx->AllocateBitmap(length));
+  } else {
+    ARROW_ASSIGN_OR_RAISE(out_arr->buffers[1], ctx->Allocate(length * bit_width / 8));
+  }
+  return Status::OK();
+}
+
+// ----------------------------------------------------------------------
+// Implement optimized take for primitive types from boolean to 1/2/4/8-byte
+// C-type based types. Use common implementation for every byte width and only
+// generate code for unsigned integer indices, since after boundschecking to
+// check for negative numbers in the indices we can safely reinterpret_cast
+// signed integers as unsigned.
+
+/// \brief The Take implementation for primitive (fixed-width) types does not
+/// use the logical Arrow type but rather the physical C type. This way we
+/// only generate one take function for each byte width.
+///
+/// This function assumes that the indices have been boundschecked.
+template <typename IndexCType, typename ValueCType>
+struct PrimitiveTakeImpl {
+  static void Exec(const PrimitiveArg& values, const PrimitiveArg& indices,
+                   Datum* out_datum) {
+    auto values_data = reinterpret_cast<const ValueCType*>(values.data);
+    auto values_is_valid = values.is_valid;
+    auto values_offset = values.offset;
+
+    auto indices_data = reinterpret_cast<const IndexCType*>(indices.data);
+    auto indices_is_valid = indices.is_valid;
+    auto indices_offset = indices.offset;
+
+    ArrayData* out_arr = out_datum->mutable_array();
+    auto out = out_arr->GetMutableValues<ValueCType>(1);
+    auto out_is_valid = out_arr->buffers[0]->mutable_data();
+    auto out_offset = out_arr->offset;
+
+    // If either the values or indices have nulls, we preemptively zero out the
+    // out validity bitmap so that we don't have to use ClearBit in each
+    // iteration for nulls.
+    if (values.null_count > 0 || indices.null_count > 0) {
+      BitUtil::SetBitsTo(out_is_valid, out_offset, indices.length, false);
+    }
+
+    OptionalBitBlockCounter indices_bit_counter(indices_is_valid, indices_offset,
+                                                indices.length);
+    int64_t position = 0;
+    int64_t valid_count = 0;
+    while (position < indices.length) {
+      BitBlockCount block = indices_bit_counter.NextBlock();
+      if (values.null_count == 0) {
+        // Values are never null, so things are easier
+        valid_count += block.popcount;
+        if (block.popcount == block.length) {
+          // Fastest path: neither values nor index nulls
+          BitUtil::SetBitsTo(out_is_valid, out_offset + position, block.length, true);
+          for (int64_t i = 0; i < block.length; ++i) {
+            out[position] = values_data[indices_data[position]];
+            ++position;
+          }
+        } else if (block.popcount > 0) {
+          // Slow path: some indices but not all are null
+          for (int64_t i = 0; i < block.length; ++i) {
+            if (BitUtil::GetBit(indices_is_valid, indices_offset + position)) {
+              // index is not null
+              BitUtil::SetBit(out_is_valid, out_offset + position);
+              out[position] = values_data[indices_data[position]];
+            } else {
+              out[position] = ValueCType{};
+            }
+            ++position;
+          }
+        } else {
+          memset(out + position, 0, sizeof(ValueCType) * block.length);
+          position += block.length;
+        }
+      } else {
+        // Values have nulls, so we must do random access into the values bitmap
+        if (block.popcount == block.length) {
+          // Faster path: indices are not null but values may be
+          for (int64_t i = 0; i < block.length; ++i) {
+            if (BitUtil::GetBit(values_is_valid,
+                                values_offset + indices_data[position])) {
+              // value is not null
+              out[position] = values_data[indices_data[position]];
+              BitUtil::SetBit(out_is_valid, out_offset + position);
+              ++valid_count;
+            } else {
+              out[position] = ValueCType{};
+            }
+            ++position;
+          }
+        } else if (block.popcount > 0) {
+          // Slow path: some but not all indices are null. Since we are doing
+          // random access in general we have to check the value nullness one by
+          // one.
+          for (int64_t i = 0; i < block.length; ++i) {
+            if (BitUtil::GetBit(indices_is_valid, indices_offset + position) &&
+                BitUtil::GetBit(values_is_valid,
+                                values_offset + indices_data[position])) {
+              // index is not null && value is not null
+              out[position] = values_data[indices_data[position]];
+              BitUtil::SetBit(out_is_valid, out_offset + position);
+              ++valid_count;
+            } else {
+              out[position] = ValueCType{};
+            }
+            ++position;
+          }
+        } else {
+          memset(out + position, 0, sizeof(ValueCType) * block.length);
+          position += block.length;
+        }
+      }
+    }
+    out_arr->null_count = out_arr->length - valid_count;
+  }
+};
+
+template <typename IndexCType>
+struct BooleanTakeImpl {
+  static void Exec(const PrimitiveArg& values, const PrimitiveArg& indices,
+                   Datum* out_datum) {
+    const uint8_t* values_data = values.data;
+    auto values_is_valid = values.is_valid;
+    auto values_offset = values.offset;
+
+    auto indices_data = reinterpret_cast<const IndexCType*>(indices.data);
+    auto indices_is_valid = indices.is_valid;
+    auto indices_offset = indices.offset;
+
+    ArrayData* out_arr = out_datum->mutable_array();
+    auto out = out_arr->buffers[1]->mutable_data();
+    auto out_is_valid = out_arr->buffers[0]->mutable_data();
+    auto out_offset = out_arr->offset;
+
+    // If either the values or indices have nulls, we preemptively zero out the
+    // out validity bitmap so that we don't have to use ClearBit in each
+    // iteration for nulls.
+    if (values.null_count > 0 || indices.null_count > 0) {
+      BitUtil::SetBitsTo(out_is_valid, out_offset, indices.length, false);
+    }
+    // Avoid uninitialized data in values array
+    BitUtil::SetBitsTo(out, out_offset, indices.length, false);
+
+    auto PlaceDataBit = [&](int64_t loc, IndexCType index) {
+      BitUtil::SetBitTo(out, out_offset + loc,
+                        BitUtil::GetBit(values_data, values_offset + index));
+    };
+
+    OptionalBitBlockCounter indices_bit_counter(indices_is_valid, indices_offset,
+                                                indices.length);
+    int64_t position = 0;
+    int64_t valid_count = 0;
+    while (position < indices.length) {
+      BitBlockCount block = indices_bit_counter.NextBlock();
+      if (values.null_count == 0) {
+        // Values are never null, so things are easier
+        valid_count += block.popcount;
+        if (block.popcount == block.length) {
+          // Fastest path: neither values nor index nulls
+          BitUtil::SetBitsTo(out_is_valid, out_offset + position, block.length, true);
+          for (int64_t i = 0; i < block.length; ++i) {
+            PlaceDataBit(position, indices_data[position]);
+            ++position;
+          }
+        } else if (block.popcount > 0) {
+          // Slow path: some but not all indices are null
+          for (int64_t i = 0; i < block.length; ++i) {
+            if (BitUtil::GetBit(indices_is_valid, indices_offset + position)) {
+              // index is not null
+              BitUtil::SetBit(out_is_valid, out_offset + position);
+              PlaceDataBit(position, indices_data[position]);
+            }
+            ++position;
+          }
+        } else {
+          position += block.length;
+        }
+      } else {
+        // Values have nulls, so we must do random access into the values bitmap
+        if (block.popcount == block.length) {
+          // Faster path: indices are not null but values may be
+          for (int64_t i = 0; i < block.length; ++i) {
+            if (BitUtil::GetBit(values_is_valid,
+                                values_offset + indices_data[position])) {
+              // value is not null
+              BitUtil::SetBit(out_is_valid, out_offset + position);
+              PlaceDataBit(position, indices_data[position]);
+              ++valid_count;
+            }
+            ++position;
+          }
+        } else if (block.popcount > 0) {
+          // Slow path: some but not all indices are null. Since we are doing
+          // random access in general we have to check the value nullness one by
+          // one.
+          for (int64_t i = 0; i < block.length; ++i) {
+            if (BitUtil::GetBit(indices_is_valid, indices_offset + position)) {
+              // index is not null
+              if (BitUtil::GetBit(values_is_valid,
+                                  values_offset + indices_data[position])) {
+                // value is not null
+                PlaceDataBit(position, indices_data[position]);
+                BitUtil::SetBit(out_is_valid, out_offset + position);
+                ++valid_count;
+              }
+            }
+            ++position;
+          }
+        } else {
+          position += block.length;
+        }
+      }
+    }
+    out_arr->null_count = out_arr->length - valid_count;
+  }
+};
+
+template <template <typename...> class TakeImpl, typename... Args>
+void TakeIndexDispatch(const PrimitiveArg& values, const PrimitiveArg& indices,
+                       Datum* out) {
+  // With the simplifying assumption that boundschecking has taken place
+  // already at a higher level, we can now assume that the index values are all
+  // non-negative. Thus, we can interpret signed integers as unsigned and avoid
+  // having to generate double the amount of binary code to handle each integer
+  // width.
+  switch (indices.bit_width) {
+    case 8:
+      return TakeImpl<uint8_t, Args...>::Exec(values, indices, out);
+    case 16:
+      return TakeImpl<uint16_t, Args...>::Exec(values, indices, out);
+    case 32:
+      return TakeImpl<uint32_t, Args...>::Exec(values, indices, out);
+    case 64:
+      return TakeImpl<uint64_t, Args...>::Exec(values, indices, out);
+    default:
+      DCHECK(false) << "Invalid indices byte width";
+      break;
+  }
+}
+
+void PrimitiveTake(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+  const auto& state = checked_cast<const TakeState&>(*ctx->state());
+  if (state.options.boundscheck) {
+    KERNEL_RETURN_IF_ERROR(ctx, IndexBoundsCheck(*batch[1].array(), batch[0].length()));
+  }
+
+  PrimitiveArg values = GetPrimitiveArg(*batch[0].array());
+  PrimitiveArg indices = GetPrimitiveArg(*batch[1].array());
+  KERNEL_RETURN_IF_ERROR(ctx,
+                         PreallocateData(ctx, indices.length, values.bit_width, out));
+  switch (values.bit_width) {
+    case 1:
+      return TakeIndexDispatch<BooleanTakeImpl>(values, indices, out);
+    case 8:
+      return TakeIndexDispatch<PrimitiveTakeImpl, int8_t>(values, indices, out);
+    case 16:
+      return TakeIndexDispatch<PrimitiveTakeImpl, int16_t>(values, indices, out);
+    case 32:
+      return TakeIndexDispatch<PrimitiveTakeImpl, int32_t>(values, indices, out);
+    case 64:
+      return TakeIndexDispatch<PrimitiveTakeImpl, int64_t>(values, indices, out);
+    default:
+      DCHECK(false) << "Invalid values byte width";
+      break;
+  }
+}
+
+// ----------------------------------------------------------------------
+// Optimized and streamlined filter for primitive types
+
+// Use either BitBlockCounter or BinaryBitBlockCounter to quickly scan filter a
+// word at a time for the DROP selection type.
+class DropNullCounter {
+ public:
+  // validity bitmap may be null
+  DropNullCounter(const uint8_t* validity, const uint8_t* data, int64_t offset,
+                  int64_t length)
+      : data_counter_(data, offset, length),
+        data_and_validity_counter_(data, offset, validity, offset, length),
+        has_validity_(validity != nullptr) {}
+
+  BitBlockCount NextBlock() {
+    if (has_validity_) {
+      // filter is true AND not null
+      return data_and_validity_counter_.NextAndWord();
+    } else {
+      return data_counter_.NextWord();
+    }
+  }
+
+ private:
+  // For when just data is present, but no validity bitmap
+  BitBlockCounter data_counter_;
+
+  // For when both validity bitmap and data are present
+  BinaryBitBlockCounter data_and_validity_counter_;
+  bool has_validity_;
+};
+
+/// \brief The Filter implementation for primitive (fixed-width) types does not
+/// use the logical Arrow type but rather then physical C type. This way we
+/// only generate one take function for each byte width. We use the same
+/// implementation here for boolean and fixed-byte-size inputs with some
+/// template specialization.
+template <typename ArrowType>
+class PrimitiveFilterImpl {
+ public:
+  using T = typename GetCType<ArrowType>::type;
+
+  PrimitiveFilterImpl(const PrimitiveArg& values, const PrimitiveArg& filter,
+                      FilterOptions::NullSelectionBehavior null_selection,
+                      Datum* out_datum)
+      : values_is_valid_(values.is_valid),
+        values_data_(reinterpret_cast<const T*>(values.data)),
+        values_offset_(values.offset),
+        values_length_(values.length),
+        filter_is_valid_(filter.is_valid),
+        filter_data_(filter.data),
+        filter_offset_(filter.offset),
+        null_selection_(null_selection) {
+    ArrayData* out_arr = out_datum->mutable_array();
+    out_is_valid_ = out_arr->buffers[0]->mutable_data();
+    out_data_ = reinterpret_cast<T*>(out_arr->buffers[1]->mutable_data());
+    out_offset_ = out_arr->offset;
+    out_position_ = 0;
+  }
+
+  void Exec() {
+    // Bit counters used for both null_selection behaviors
+    DropNullCounter drop_null_counter(filter_is_valid_, filter_data_, filter_offset_,
+                                      values_length_);
+    OptionalBitBlockCounter data_counter(values_is_valid_, values_offset_,
+                                         values_length_);
+    OptionalBitBlockCounter filter_valid_counter(filter_is_valid_, filter_offset_,
+                                                 values_length_);
+
+    auto WriteNotNull = [&](int64_t index) {
+      BitUtil::SetBit(out_is_valid_, out_offset_ + out_position_);
+      // Increments out_position_
+      WriteValue(index);
+    };
+
+    auto WriteMaybeNull = [&](int64_t index) {
+      BitUtil::SetBitTo(out_is_valid_, out_offset_ + out_position_,
+                        BitUtil::GetBit(values_is_valid_, values_offset_ + index));
+      // Increments out_position_
+      WriteValue(index);
+    };
+
+    int64_t in_position = 0;
+    while (in_position < values_length_) {
+      BitBlockCount filter_block = drop_null_counter.NextBlock();
+      BitBlockCount filter_valid_block = filter_valid_counter.NextWord();
+      BitBlockCount data_block = data_counter.NextWord();
+      if (filter_block.IsFull() && data_block.IsFull()) {
+        // Fastest path: all values in block are included and not null
+        BitUtil::SetBitsTo(out_is_valid_, out_offset_ + out_position_,
+                           filter_block.length, true);
+        for (int64_t i = 0; i < filter_block.length; ++i) {
+          WriteValue(in_position++);
+        }
+      } else if (filter_block.IsFull()) {
+        // Faster: all values are selected, but some values are null
+        // Batch copy bits from values validity bitmap to output validity bitmap
+        CopyBitmap(values_is_valid_, values_offset_ + in_position, filter_block.length,
+                   out_is_valid_, out_offset_ + out_position_);
+        for (int64_t i = 0; i < filter_block.length; ++i) {
+          // Copy the value in the null slots regardless
+          WriteValue(in_position++);
+        }
+      } else if (filter_block.IsEmpty() && null_selection_ == FilterOptions::DROP) {
+        // For this exceedingly common case in low-selectivity filters we can
+        // skip further analysis of the data and move on to the next block.
+        in_position += filter_block.length;
+      } else {
+        // Some filter values are false or null
+        if (data_block.IsFull()) {
+          // No values are null
+          if (filter_valid_block.IsFull()) {
+            // Filter is non-null but some values are false
+            for (int64_t i = 0; i < filter_block.length; ++i) {
+              if (BitUtil::GetBit(filter_data_, filter_offset_ + in_position)) {
+                WriteNotNull(in_position);
+              }
+              ++in_position;
+            }
+          } else if (null_selection_ == FilterOptions::DROP) {
+            // If any values are selected, they ARE NOT null
+            for (int64_t i = 0; i < filter_block.length; ++i) {
+              if (BitUtil::GetBit(filter_is_valid_, filter_offset_ + in_position) &&
+                  BitUtil::GetBit(filter_data_, filter_offset_ + in_position)) {
+                WriteNotNull(in_position);
+              }
+              ++in_position;
+            }
+          } else {  // null_selection == FilterOptions::EMIT_NULL
+            // Data values in this block are not null
+            for (int64_t i = 0; i < filter_block.length; ++i) {
+              const bool is_valid =
+                  BitUtil::GetBit(filter_is_valid_, filter_offset_ + in_position);
+              if (is_valid &&
+                  BitUtil::GetBit(filter_data_, filter_offset_ + in_position)) {
+                // Filter slot is non-null and set
+                WriteNotNull(in_position);
+              } else if (!is_valid) {
+                // Filter slot is null, so we have a null in the output
+                BitUtil::ClearBit(out_is_valid_, out_offset_ + out_position_);
+                WriteNull();
+              }
+              ++in_position;
+            }
+          }
+        } else {  // !data_block.IsFull()
+          // Some values are null
+          if (filter_valid_block.IsFull()) {
+            // Filter is non-null but some values are false
+            for (int64_t i = 0; i < filter_block.length; ++i) {
+              if (BitUtil::GetBit(filter_data_, filter_offset_ + in_position)) {
+                WriteMaybeNull(in_position);
+              }
+              ++in_position;
+            }
+          } else if (null_selection_ == FilterOptions::DROP) {
+            // If any values are selected, they ARE NOT null
+            for (int64_t i = 0; i < filter_block.length; ++i) {
+              if (BitUtil::GetBit(filter_is_valid_, filter_offset_ + in_position) &&
+                  BitUtil::GetBit(filter_data_, filter_offset_ + in_position)) {
+                WriteMaybeNull(in_position);
+              }
+              ++in_position;
+            }
+          } else {  // null_selection == FilterOptions::EMIT_NULL
+            // Data values in this block are not null
+            for (int64_t i = 0; i < filter_block.length; ++i) {
+              const bool is_valid =
+                  BitUtil::GetBit(filter_is_valid_, filter_offset_ + in_position);
+              if (is_valid &&
+                  BitUtil::GetBit(filter_data_, filter_offset_ + in_position)) {
+                // Filter slot is non-null and set
+                WriteMaybeNull(in_position);
+              } else if (!is_valid) {
+                // Filter slot is null, so we have a null in the output
+                BitUtil::ClearBit(out_is_valid_, out_offset_ + out_position_);
+                WriteNull();
+              }
+              ++in_position;
+            }
+          }
+        }
+      }  // !filter_block.IsFull()
+    }    // while(in_position < values_length_)
+  }
+
+  // Write the next out_position given the selected in_position for the input
+  // data and advance out_position
+  void WriteValue(int64_t in_position) {
+    out_data_[out_position_++] = values_data_[in_position];
+  }
+
+  void WriteNull() {
+    // Zero the memory
+    out_data_[out_position_++] = T{};
+  }
+
+ private:
+  const uint8_t* values_is_valid_;
+  const T* values_data_;
+  int64_t values_offset_;
+  int64_t values_length_;
+  const uint8_t* filter_is_valid_;
+  const uint8_t* filter_data_;
+  int64_t filter_offset_;
+  FilterOptions::NullSelectionBehavior null_selection_;
+  uint8_t* out_is_valid_;
+  T* out_data_;
+  int64_t out_offset_;
+  int64_t out_position_;
+};
+
+template <>
+inline void PrimitiveFilterImpl<BooleanType>::WriteValue(int64_t in_position) {
+  BitUtil::SetBitTo(out_data_, out_offset_ + out_position_++,
+                    BitUtil::GetBit(values_data_, values_offset_ + in_position));
+}
+
+template <>
+inline void PrimitiveFilterImpl<BooleanType>::WriteNull() {
+  // Zero the bit
+  BitUtil::ClearBit(out_data_, out_offset_ + out_position_++);
+}
+
+void PrimitiveFilter(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+  const auto& state = checked_cast<const FilterState&>(*ctx->state());
+  PrimitiveArg values = GetPrimitiveArg(*batch[0].array());
+  PrimitiveArg filter = GetPrimitiveArg(*batch[1].array());
+  FilterOptions::NullSelectionBehavior null_selection =
+      state.options.null_selection_behavior;
+
+  int64_t output_length = GetFilterOutputSize(*batch[1].array(), null_selection);
+  KERNEL_RETURN_IF_ERROR(ctx, PreallocateData(ctx, output_length, values.bit_width, out));
+
+  // The output precomputed null count is unknown except in the narrow
+  // condition that all the values are non-null and the filter will not cause
+  // any new nulls to be created.
+  if (values.null_count == 0 &&
+      (null_selection == FilterOptions::DROP || filter.null_count == 0)) {
+    out->mutable_array()->null_count = 0;
+  } else {
+    out->mutable_array()->null_count = kUnknownNullCount;
+  }
+  switch (values.bit_width) {
+    case 1:
+      return PrimitiveFilterImpl<BooleanType>(values, filter, null_selection, out).Exec();
+    case 8:
+      return PrimitiveFilterImpl<UInt8Type>(values, filter, null_selection, out).Exec();
+    case 16:
+      return PrimitiveFilterImpl<UInt16Type>(values, filter, null_selection, out).Exec();
+    case 32:
+      return PrimitiveFilterImpl<UInt32Type>(values, filter, null_selection, out).Exec();
+    case 64:
+      return PrimitiveFilterImpl<UInt64Type>(values, filter, null_selection, out).Exec();
+    default:
+      DCHECK(false) << "Invalid values bit width";
+      break;
+  }
+}
+
+// ----------------------------------------------------------------------
+// Null take and filter
+
+void NullTake(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+  const auto& state = checked_cast<const TakeState&>(*ctx->state());
+  if (state.options.boundscheck) {
+    KERNEL_RETURN_IF_ERROR(ctx, IndexBoundsCheck(*batch[1].array(), batch[0].length()));
+  }
+  out->value = std::make_shared<NullArray>(batch.length)->data();
+}
+
+void NullFilter(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+  const auto& state = checked_cast<const FilterState&>(*ctx->state());
+  int64_t output_length =
+      GetFilterOutputSize(*batch[1].array(), state.options.null_selection_behavior);
+  out->value = std::make_shared<NullArray>(output_length)->data();
+}
+
+// ----------------------------------------------------------------------
+// Dictionary take and filter
+
+void DictionaryTake(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+  const auto& state = checked_cast<const TakeState&>(*ctx->state());
+  DictionaryArray values(batch[0].array());
+  Datum result;
+  KERNEL_RETURN_IF_ERROR(
+      ctx, Take(Datum(values.indices()), batch[1], state.options, ctx->exec_context())
+               .Value(&result));
+  DictionaryArray taken_values(values.type(), result.make_array(), values.dictionary());
+  out->value = taken_values.data();
+}
+
+void DictionaryFilter(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+  const auto& state = checked_cast<const FilterState&>(*ctx->state());
+  DictionaryArray dict_values(batch[0].array());
+  Datum result;
+  KERNEL_RETURN_IF_ERROR(ctx, Filter(Datum(dict_values.indices()), batch[1].array(),
+                                     state.options, ctx->exec_context())
+                                  .Value(&result));
+  DictionaryArray filtered_values(dict_values.type(), result.make_array(),
+                                  dict_values.dictionary());
+  out->value = filtered_values.data();
+}
+
+// ----------------------------------------------------------------------
+// Extension take and filter
+
+void ExtensionTake(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+  const auto& state = checked_cast<const TakeState&>(*ctx->state());
+  ExtensionArray values(batch[0].array());
+  Datum result;
+  KERNEL_RETURN_IF_ERROR(
+      ctx, Take(Datum(values.storage()), batch[1], state.options, ctx->exec_context())
+               .Value(&result));
+  ExtensionArray taken_values(values.type(), result.make_array());
+  out->value = taken_values.data();
+}
+
+void ExtensionFilter(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+  const auto& state = checked_cast<const FilterState&>(*ctx->state());
+  ExtensionArray ext_values(batch[0].array());
+  Datum result;
+  KERNEL_RETURN_IF_ERROR(ctx, Filter(Datum(ext_values.storage()), batch[1].array(),
+                                     state.options, ctx->exec_context())
+                                  .Value(&result));
+  ExtensionArray filtered_values(ext_values.type(), result.make_array());
+  out->value = filtered_values.data();
+}
+
+// ----------------------------------------------------------------------
+// Implement take for other data types where there is less performance
+// sensitivity by visiting the selected indices.
+
+// Use CRTP to dispatch to type-specific processing of take indices for each
+// unsigned integer type.
+template <typename Impl, typename Type>
+struct Selection {
+  using ValuesArrayType = typename TypeTraits<Type>::ArrayType;
+
+  // Forwards the generic value visitors to the take index visitor template
+  template <typename IndexCType>
+  struct TakeAdapter {
+    static constexpr bool is_take = true;
+
+    Impl* impl;
+    explicit TakeAdapter(Impl* impl) : impl(impl) {}
+    template <typename ValidVisitor, typename NullVisitor>
+    Status Generate(ValidVisitor&& visit_valid, NullVisitor&& visit_null) {
+      return impl->template VisitTake<IndexCType>(std::forward<ValidVisitor>(visit_valid),
+                                                  std::forward<NullVisitor>(visit_null));
+    }
+  };
+
+  // Forwards the generic value visitors to the VisitFilter template
+  struct FilterAdapter {
+    static constexpr bool is_take = false;
+
+    Impl* impl;
+    explicit FilterAdapter(Impl* impl) : impl(impl) {}
+    template <typename ValidVisitor, typename NullVisitor>
+    Status Generate(ValidVisitor&& visit_valid, NullVisitor&& visit_null) {
+      return impl->VisitFilter(std::forward<ValidVisitor>(visit_valid),
+                               std::forward<NullVisitor>(visit_null));
+    }
+  };
+
+  KernelContext* ctx;
+  std::shared_ptr<ArrayData> values;
+  std::shared_ptr<ArrayData> selection;
+  int64_t output_length;
+  ArrayData* out;
+  TypedBufferBuilder<bool> validity_builder;
+
+  Selection(KernelContext* ctx, const ExecBatch& batch, int64_t output_length, Datum* out)
+      : ctx(ctx),
+        values(batch[0].array()),
+        selection(batch[1].array()),
+        output_length(output_length),
+        out(out->mutable_array()),
+        validity_builder(ctx->memory_pool()) {}
+
+  virtual ~Selection() = default;
+
+  Status FinishCommon() {
+    out->buffers.resize(values->buffers.size());
+    out->length = validity_builder.length();
+    out->null_count = validity_builder.false_count();
+    return validity_builder.Finish(&out->buffers[0]);
+  }
+
+  template <typename IndexCType, typename ValidVisitor, typename NullVisitor>
+  Status VisitTake(ValidVisitor&& visit_valid, NullVisitor&& visit_null) {
+    const auto indices_values = selection->GetValues<IndexCType>(1);
+    const uint8_t* is_valid = GetValidityBitmap(*selection);
+    OptionalBitIndexer indices_is_valid(selection->buffers[0], selection->offset);
+    OptionalBitIndexer values_is_valid(values->buffers[0], values->offset);
+    const bool values_have_nulls = (values->GetNullCount() > 0);
+
+    OptionalBitBlockCounter bit_counter(is_valid, selection->offset, selection->length);
+    int64_t position = 0;
+    while (position < selection->length) {
+      BitBlockCount block = bit_counter.NextBlock();
+      const bool indices_have_nulls = block.popcount < block.length;
+      if (!indices_have_nulls && !values_have_nulls) {
+        // Fastest path, neither indices nor values have nulls
+        validity_builder.UnsafeAppend(block.length, true);
+        for (int64_t i = 0; i < block.length; ++i) {
+          RETURN_NOT_OK(visit_valid(indices_values[position++]));
+        }
+      } else if (block.popcount > 0) {
+        // Since we have to branch on whether the indices are null or not, we
+        // combine the "non-null indices block but some values null" and
+        // "some-null indices block but values non-null" into a single loop.
+        for (int64_t i = 0; i < block.length; ++i) {
+          if ((!indices_have_nulls || indices_is_valid[position]) &&
+              values_is_valid[indices_values[position]]) {
+            validity_builder.UnsafeAppend(true);
+            RETURN_NOT_OK(visit_valid(indices_values[position]));
+          } else {
+            validity_builder.UnsafeAppend(false);
+            RETURN_NOT_OK(visit_null());
+          }
+          ++position;
+        }
+      } else {
+        // The whole block is null
+        validity_builder.UnsafeAppend(block.length, false);
+        for (int64_t i = 0; i < block.length; ++i) {
+          RETURN_NOT_OK(visit_null());
+        }
+        position += block.length;
+      }
+    }
+    return Status::OK();
+  }
+
+  // We use the NullVisitor both for "selected" nulls as well as "emitted"
+  // nulls coming from the filter when using FilterOptions::EMIT_NULL
+  template <typename ValidVisitor, typename NullVisitor>
+  Status VisitFilter(ValidVisitor&& visit_valid, NullVisitor&& visit_null) {
+    const auto& state = checked_cast<const FilterState&>(*ctx->state());
+    auto null_selection = state.options.null_selection_behavior;
+
+    const auto filter_data = selection->buffers[1]->data();
+
+    const uint8_t* filter_is_valid = GetValidityBitmap(*selection);
+    const int64_t filter_offset = selection->offset;
+    OptionalBitIndexer values_is_valid(values->buffers[0], values->offset);
+
+    // We use 3 block counters for fast scanning of the filter
+    //
+    // * values_valid_counter: for values null/not-null
+    // * filter_valid_counter: for filter null/not-null
+    // * filter_counter: for filter true/false
+    OptionalBitBlockCounter values_valid_counter(GetValidityBitmap(*values),
+                                                 values->offset, values->length);
+    OptionalBitBlockCounter filter_valid_counter(filter_is_valid, filter_offset,
+                                                 selection->length);
+    BitBlockCounter filter_counter(filter_data, filter_offset, selection->length);
+    int64_t in_position = 0;
+
+    auto AppendNotNull = [&](int64_t index) -> Status {
+      validity_builder.UnsafeAppend(true);
+      return visit_valid(index);
+    };
+
+    auto AppendNull = [&]() -> Status {
+      validity_builder.UnsafeAppend(false);
+      return visit_null();
+    };
+
+    auto AppendMaybeNull = [&](int64_t index) -> Status {
+      if (values_is_valid[index]) {
+        return AppendNotNull(index);
+      } else {
+        return AppendNull();
+      }
+    };
+
+    while (in_position < selection->length) {
+      BitBlockCount filter_valid_block = filter_valid_counter.NextWord();
+      BitBlockCount values_valid_block = values_valid_counter.NextWord();
+      BitBlockCount filter_block = filter_counter.NextWord();
+      if (filter_block.IsEmpty() && null_selection == FilterOptions::DROP) {
+        // For this exceedingly common case in low-selectivity filters we can
+        // skip further analysis of the data and move on to the next block.
+        in_position += filter_block.length;
+      } else if (filter_valid_block.IsFull()) {
+        // Simpler path: no filter values are null
+        if (filter_block.IsFull()) {
+          // Fastest path: filter values are all true and not null
+          if (values_valid_block.IsFull()) {
+            // The values aren't null either
+            validity_builder.UnsafeAppend(filter_block.length, true);
+            for (int64_t i = 0; i < filter_block.length; ++i) {
+              RETURN_NOT_OK(visit_valid(in_position++));
+            }
+          } else {
+            // Some of the values in this block are null
+            for (int64_t i = 0; i < filter_block.length; ++i) {
+              RETURN_NOT_OK(AppendMaybeNull(in_position++));
+            }
+          }
+        } else {  // !filter_block.IsFull()
+          // Some of the filter values are false, but all not null
+          if (values_valid_block.IsFull()) {
+            // All the values are not-null, so we can skip null checking for
+            // them
+            for (int64_t i = 0; i < filter_block.length; ++i) {
+              if (BitUtil::GetBit(filter_data, filter_offset + in_position)) {
+                RETURN_NOT_OK(AppendNotNull(in_position));
+              }
+              ++in_position;
+            }
+          } else {
+            // Some of the values in the block are null, so we have to check
+            // each one
+            for (int64_t i = 0; i < filter_block.length; ++i) {
+              if (BitUtil::GetBit(filter_data, filter_offset + in_position)) {
+                RETURN_NOT_OK(AppendMaybeNull(in_position));
+              }
+              ++in_position;
+            }
+          }
+        }
+      } else {  // !filter_valid_block.IsFull()
+        // Some of the filter values are null, so we have to handle the DROP
+        // versus EMIT_NULL null selection behavior.
+        if (null_selection == FilterOptions::DROP) {
+          // Filter null values are treated as false.
+          for (int64_t i = 0; i < filter_block.length; ++i) {
+            if (BitUtil::GetBit(filter_is_valid, filter_offset + in_position) &&
+                BitUtil::GetBit(filter_data, filter_offset + in_position)) {
+              RETURN_NOT_OK(AppendMaybeNull(in_position));
+            }
+            ++in_position;
+          }
+        } else {
+          // Filter null values are appended to output as null whether the
+          // value in the corresponding slot is valid or not
+          for (int64_t i = 0; i < filter_block.length; ++i) {
+            const bool filter_not_null =
+                BitUtil::GetBit(filter_is_valid, filter_offset + in_position);
+            if (filter_not_null &&
+                BitUtil::GetBit(filter_data, filter_offset + in_position)) {
+              RETURN_NOT_OK(AppendMaybeNull(in_position));
+            } else if (!filter_not_null) {
+              // EMIT_NULL case
+              RETURN_NOT_OK(AppendNull());
+            }
+            ++in_position;
+          }
+        }
+      }
+    }
+    return Status::OK();
+  }
+
+  virtual Status Init() { return Status::OK(); }
+
+  // Implementation specific finish logic
+  virtual Status Finish() = 0;
+
+  Status ExecTake() {
+    RETURN_NOT_OK(this->validity_builder.Reserve(output_length));
+    RETURN_NOT_OK(Init());
+    int index_width =
+        checked_cast<const FixedWidthType&>(*this->selection->type).bit_width() / 8;
+
+    // CTRP dispatch here
+    switch (index_width) {
+      case 1: {
+        Status s =
+            static_cast<Impl*>(this)->template GenerateOutput<TakeAdapter<uint8_t>>();
+        RETURN_NOT_OK(s);
+      } break;
+      case 2: {
+        Status s =
+            static_cast<Impl*>(this)->template GenerateOutput<TakeAdapter<uint16_t>>();
+        RETURN_NOT_OK(s);
+      } break;
+      case 4: {
+        Status s =
+            static_cast<Impl*>(this)->template GenerateOutput<TakeAdapter<uint32_t>>();
+        RETURN_NOT_OK(s);
+      } break;
+      case 8: {
+        Status s =
+            static_cast<Impl*>(this)->template GenerateOutput<TakeAdapter<uint64_t>>();
+        RETURN_NOT_OK(s);
+      } break;
+      default:
+        DCHECK(false) << "Invalid index width";
+        break;
+    }
+    RETURN_NOT_OK(this->FinishCommon());
+    return Finish();
+  }
+
+  Status ExecFilter() {
+    RETURN_NOT_OK(this->validity_builder.Reserve(output_length));
+    RETURN_NOT_OK(Init());
+    // CRTP dispatch
+    Status s = static_cast<Impl*>(this)->template GenerateOutput<FilterAdapter>();
+    RETURN_NOT_OK(s);
+    RETURN_NOT_OK(this->FinishCommon());
+    return Finish();
+  }
+};
+
+#define LIFT_BASE_MEMBERS()                               \
+  using ValuesArrayType = typename Base::ValuesArrayType; \
+  using Base::ctx;                                        \
+  using Base::values;                                     \
+  using Base::selection;                                  \
+  using Base::output_length;                              \
+  using Base::out;                                        \
+  using Base::validity_builder
+
+static inline Status VisitNoop() { return Status::OK(); }
+
+// A take implementation for 32-bit and 64-bit variable binary types. Common
+// generated kernels are shared between Binary/String and
+// LargeBinary/LargeString
+template <typename Type>
+struct VarBinaryImpl : public Selection<VarBinaryImpl<Type>, Type> {
+  using offset_type = typename Type::offset_type;
+
+  using Base = Selection<VarBinaryImpl<Type>, Type>;
+  LIFT_BASE_MEMBERS();
+
+  std::shared_ptr<ArrayData> values_as_binary;
+  TypedBufferBuilder<offset_type> offset_builder;
+  TypedBufferBuilder<uint8_t> data_builder;
+
+  static constexpr int64_t kOffsetLimit = std::numeric_limits<offset_type>::max() - 1;
+
+  VarBinaryImpl(KernelContext* ctx, const ExecBatch& batch, int64_t output_length,
+                Datum* out)
+      : Base(ctx, batch, output_length, out),
+        offset_builder(ctx->memory_pool()),
+        data_builder(ctx->memory_pool()) {}
+
+  template <typename Adapter>
+  Status GenerateOutput() {
+    ValuesArrayType typed_values(this->values_as_binary);
+
+    // Presize the data builder with a rough estimate of the required data size
+    if (values->length > 0) {
+      const double mean_value_length =
+          (typed_values.total_values_length() / static_cast<double>(values->length));
+
+      // TODO: See if possible to reduce output_length for take/filter cases
+      // where there are nulls in the selection array
+      RETURN_NOT_OK(
+          data_builder.Reserve(static_cast<int64_t>(mean_value_length * output_length)));
+    }
+    int64_t space_available = data_builder.capacity();
+
+    const offset_type* raw_offsets = typed_values.raw_value_offsets();
+    const uint8_t* raw_data = typed_values.raw_data();
+
+    offset_type offset = 0;
+    Adapter adapter(this);
+    RETURN_NOT_OK(adapter.Generate(
+        [&](int64_t index) {
+          offset_builder.UnsafeAppend(offset);
+          offset_type val_offset = raw_offsets[index];
+          offset_type val_size = raw_offsets[index + 1] - val_offset;
+
+          // Use static property to prune this code from the filter path in
+          // optimized builds
+          if (Adapter::is_take &&
+              ARROW_PREDICT_FALSE(static_cast<int64_t>(offset) +
+                                  static_cast<int64_t>(val_size)) > kOffsetLimit) {
+            return Status::Invalid("Take operation overflowed binary array capacity");
+          }
+          offset += val_size;
+          if (ARROW_PREDICT_FALSE(val_size > space_available)) {
+            RETURN_NOT_OK(data_builder.Reserve(val_size));
+            space_available = data_builder.capacity() - data_builder.length();
+          }
+          data_builder.UnsafeAppend(raw_data + val_offset, val_size);
+          space_available -= val_size;
+          return Status::OK();
+        },
+        [&]() {
+          offset_builder.UnsafeAppend(offset);
+          return Status::OK();
+        }));
+    offset_builder.UnsafeAppend(offset);
+    return Status::OK();
+  }
+
+  Status Init() override {
+    ARROW_ASSIGN_OR_RAISE(this->values_as_binary,
+                          GetArrayView(this->values, TypeTraits<Type>::type_singleton()));
+    return offset_builder.Reserve(output_length + 1);
+  }
+
+  Status Finish() override {
+    RETURN_NOT_OK(offset_builder.Finish(&out->buffers[1]));
+    return data_builder.Finish(&out->buffers[2]);
+  }
+};
+
+struct FSBImpl : public Selection<FSBImpl, FixedSizeBinaryType> {
+  using Base = Selection<FSBImpl, FixedSizeBinaryType>;
+  LIFT_BASE_MEMBERS();
+
+  TypedBufferBuilder<uint8_t> data_builder;
+
+  FSBImpl(KernelContext* ctx, const ExecBatch& batch, int64_t output_length, Datum* out)
+      : Base(ctx, batch, output_length, out), data_builder(ctx->memory_pool()) {}
+
+  template <typename Adapter>
+  Status GenerateOutput() {
+    FixedSizeBinaryArray typed_values(this->values);
+    int32_t value_size = typed_values.byte_width();
+
+    RETURN_NOT_OK(data_builder.Reserve(value_size * output_length));
+    Adapter adapter(this);
+    return adapter.Generate(
+        [&](int64_t index) {
+          auto val = typed_values.GetView(index);
+          data_builder.UnsafeAppend(reinterpret_cast<const uint8_t*>(val.data()),
+                                    value_size);
+          return Status::OK();
+        },
+        [&]() {
+          data_builder.UnsafeAppend(value_size, static_cast<uint8_t>(0x00));
+          return Status::OK();
+        });
+  }
+
+  Status Finish() override { return data_builder.Finish(&out->buffers[1]); }
+};
+
+template <typename Type>
+struct ListImpl : public Selection<ListImpl<Type>, Type> {
+  using offset_type = typename Type::offset_type;
+
+  using Base = Selection<ListImpl<Type>, Type>;
+  LIFT_BASE_MEMBERS();
+
+  TypedBufferBuilder<offset_type> offset_builder;
+  typename TypeTraits<Type>::OffsetBuilderType child_index_builder;
+
+  ListImpl(KernelContext* ctx, const ExecBatch& batch, int64_t output_length, Datum* out)
+      : Base(ctx, batch, output_length, out),
+        offset_builder(ctx->memory_pool()),
+        child_index_builder(ctx->memory_pool()) {}
+
+  template <typename Adapter>
+  Status GenerateOutput() {
+    ValuesArrayType typed_values(this->values);
+
+    // TODO presize child_index_builder with a similar heuristic as VarBinaryImpl
+
+    offset_type offset = 0;
+    Adapter adapter(this);
+    RETURN_NOT_OK(adapter.Generate(
+        [&](int64_t index) {
+          offset_builder.UnsafeAppend(offset);
+          offset_type value_offset = typed_values.value_offset(index);
+          offset_type value_length = typed_values.value_length(index);
+          offset += value_length;
+          RETURN_NOT_OK(child_index_builder.Reserve(value_length));
+          for (offset_type j = value_offset; j < value_offset + value_length; ++j) {
+            child_index_builder.UnsafeAppend(j);
+          }
+          return Status::OK();
+        },
+        [&]() {
+          offset_builder.UnsafeAppend(offset);
+          return Status::OK();
+        }));
+    offset_builder.UnsafeAppend(offset);
+    return Status::OK();
+  }
+
+  Status Init() override {
+    RETURN_NOT_OK(offset_builder.Reserve(output_length + 1));
+    return Status::OK();
+  }
+
+  Status Finish() override {
+    std::shared_ptr<Array> child_indices;
+    RETURN_NOT_OK(child_index_builder.Finish(&child_indices));
+
+    ValuesArrayType typed_values(this->values);
+
+    // No need to boundscheck the child values indices
+    ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Array> taken_child,
+                          Take(*typed_values.values(), *child_indices,
+                               TakeOptions::NoBoundsCheck(), ctx->exec_context()));
+    RETURN_NOT_OK(offset_builder.Finish(&out->buffers[1]));
+    out->child_data = {taken_child->data()};
+    return Status::OK();
+  }
+};
+
+struct FSLImpl : public Selection<FSLImpl, FixedSizeListType> {
+  Int64Builder child_index_builder;
+
+  using Base = Selection<FSLImpl, FixedSizeListType>;
+  LIFT_BASE_MEMBERS();
+
+  FSLImpl(KernelContext* ctx, const ExecBatch& batch, int64_t output_length, Datum* out)
+      : Base(ctx, batch, output_length, out), child_index_builder(ctx->memory_pool()) {}
+
+  template <typename Adapter>
+  Status GenerateOutput() {
+    ValuesArrayType typed_values(this->values);
+    int32_t list_size = typed_values.list_type()->list_size();
+
+    /// We must take list_size elements even for null elements of
+    /// indices.
+    RETURN_NOT_OK(child_index_builder.Reserve(output_length * list_size));
+
+    Adapter adapter(this);
+    return adapter.Generate(
+        [&](int64_t index) {
+          int64_t offset = index * list_size;
+          for (int64_t j = offset; j < offset + list_size; ++j) {
+            child_index_builder.UnsafeAppend(j);
+          }
+          return Status::OK();
+        },
+        [&]() { return child_index_builder.AppendNulls(list_size); });
+  }
+
+  Status Finish() override {
+    std::shared_ptr<Array> child_indices;
+    RETURN_NOT_OK(child_index_builder.Finish(&child_indices));
+
+    ValuesArrayType typed_values(this->values);
+
+    // No need to boundscheck the child values indices
+    ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Array> taken_child,
+                          Take(*typed_values.values(), *child_indices,
+                               TakeOptions::NoBoundsCheck(), ctx->exec_context()));
+    out->child_data = {taken_child->data()};
+    return Status::OK();
+  }
+};
+
+// ----------------------------------------------------------------------
+// Struct selection implementations
+
+// We need a slightly different approach for StructType. For Take, we can
+// invoke Take on each struct field's data with boundschecking disabled. For
+// Filter on the other hand, if we natively call Filter on each field, then the

Review comment:
       typo: naively




----------------------------------------------------------------
This is an automated message from the 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] ursabot commented on pull request #7442: ARROW-9075: [C++] Optimized Filter implementation: faster performance + compilation, smaller code size

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


   [AMD64 Ubuntu 18.04 C++ Benchmark (#112952)](https://ci.ursalabs.org/#builders/73/builds/81) builder has been succeeded.
   
   Revision: f50b39e54c50e8a53606eda486c88e6ec51d7006
   
   ```diff
     =======================================  ===============  ================  ========
     benchmark                                baseline         contender         change
     =======================================  ===============  ================  ========
   - FilterFSLInt64FilterNoNulls/262144/14    5.457 GiB/sec    4.398 GiB/sec     -19.404%
     FilterStringFilterWithNulls/262144/4     642.405 MiB/sec  677.920 MiB/sec   5.528%
   - FilterFSLInt64FilterNoNulls/262144/7     463.992 MiB/sec  378.391 MiB/sec   -18.449%
     FilterFSLInt64FilterWithNulls/262144/6   333.996 MiB/sec  320.327 MiB/sec   -4.093%
   - FilterFSLInt64FilterWithNulls/262144/1   516.189 MiB/sec  459.926 MiB/sec   -10.900%
   - FilterStringFilterNoNulls/262144/4       681.504 MiB/sec  595.788 MiB/sec   -12.577%
   - FilterFSLInt64FilterNoNulls/262144/8     5.889 GiB/sec    4.675 GiB/sec     -20.610%
   - FilterInt64FilterWithNulls/262144/10     606.960 MiB/sec  547.973 MiB/sec   -9.718%
   - FilterInt64FilterNoNulls/262144/7        638.264 MiB/sec  568.923 MiB/sec   -10.864%
     FilterStringFilterWithNulls/262144/6     431.474 MiB/sec  484.077 MiB/sec   12.191%
   - FilterStringFilterNoNulls/262144/14      1.245 GiB/sec    1008.386 MiB/sec  -20.893%
     FilterFSLInt64FilterWithNulls/262144/11  4.239 GiB/sec    4.029 GiB/sec     -4.954%
   - FilterStringFilterNoNulls/262144/8       10.899 GiB/sec   8.494 GiB/sec     -22.064%
   - FilterFSLInt64FilterNoNulls/262144/4     515.626 MiB/sec  406.426 MiB/sec   -21.178%
     FilterInt64FilterNoNulls/262144/6        3.697 GiB/sec    3.525 GiB/sec     -4.664%
     FilterInt64FilterNoNulls/262144/8        6.829 GiB/sec    6.809 GiB/sec     -0.301%
   - FilterFSLInt64FilterNoNulls/262144/2     6.453 GiB/sec    4.950 GiB/sec     -23.289%
   - FilterInt64FilterWithNulls/262144/13     606.984 MiB/sec  548.948 MiB/sec   -9.561%
   - FilterStringFilterNoNulls/262144/1       707.132 MiB/sec  609.027 MiB/sec   -13.874%
     FilterStringFilterWithNulls/262144/3     436.301 MiB/sec  488.825 MiB/sec   12.038%
     FilterStringFilterWithNulls/262144/1     616.105 MiB/sec  675.493 MiB/sec   9.639%
     FilterStringFilterNoNulls/262144/3       548.660 MiB/sec  533.539 MiB/sec   -2.756%
   - FilterFSLInt64FilterNoNulls/262144/9     268.363 MiB/sec  250.359 MiB/sec   -6.709%
   - FilterStringFilterNoNulls/262144/13      89.995 MiB/sec   76.326 MiB/sec    -15.189%
     FilterStringFilterWithNulls/262144/12    71.366 MiB/sec   82.415 MiB/sec    15.483%
     FilterInt64FilterNoNulls/262144/9        3.209 GiB/sec    3.114 GiB/sec     -2.971%
     FilterFSLInt64FilterWithNulls/262144/9   288.819 MiB/sec  276.679 MiB/sec   -4.203%
     FilterStringFilterNoNulls/262144/12      66.141 MiB/sec   65.509 MiB/sec    -0.956%
   - FilterFSLInt64FilterWithNulls/262144/4   474.907 MiB/sec  429.013 MiB/sec   -9.664%
   - FilterInt64FilterWithNulls/262144/1      651.659 MiB/sec  556.258 MiB/sec   -14.640%
     FilterStringFilterWithNulls/262144/14    911.019 MiB/sec  871.756 MiB/sec   -4.310%
   - FilterInt64FilterNoNulls/262144/4        675.941 MiB/sec  569.448 MiB/sec   -15.755%
   - FilterFSLInt64FilterNoNulls/262144/13    352.227 MiB/sec  307.638 MiB/sec   -12.659%
     FilterInt64FilterWithNulls/262144/5      5.129 GiB/sec    4.921 GiB/sec     -4.068%
   - FilterFSLInt64FilterWithNulls/262144/14  4.168 GiB/sec    3.909 GiB/sec     -6.200%
     FilterStringFilterWithNulls/262144/9     396.156 MiB/sec  442.591 MiB/sec   11.721%
   - FilterFSLInt64FilterNoNulls/262144/3     554.664 MiB/sec  464.787 MiB/sec   -16.204%
   - FilterStringFilterNoNulls/262144/2       11.394 GiB/sec   8.924 GiB/sec     -21.683%
   - FilterStringFilterWithNulls/262144/8     8.856 GiB/sec    8.075 GiB/sec     -8.825%
   - FilterFSLInt64FilterNoNulls/262144/10    389.368 MiB/sec  333.033 MiB/sec   -14.468%
   - FilterFSLInt64FilterNoNulls/262144/11    5.587 GiB/sec    4.507 GiB/sec     -19.338%
     FilterStringFilterWithNulls/262144/10    580.314 MiB/sec  612.106 MiB/sec   5.478%
   - FilterFSLInt64FilterNoNulls/262144/5     6.032 GiB/sec    4.717 GiB/sec     -21.802%
   - FilterFSLInt64FilterNoNulls/262144/0     725.211 MiB/sec  565.535 MiB/sec   -22.018%
   - FilterInt64FilterNoNulls/262144/3        4.266 GiB/sec    3.855 GiB/sec     -9.641%
   - FilterInt64FilterWithNulls/262144/12     549.159 MiB/sec  499.761 MiB/sec   -8.995%
   - FilterInt64FilterWithNulls/262144/0      622.810 MiB/sec  497.075 MiB/sec   -20.188%
   - FilterInt64FilterNoNulls/262144/1        1.021 GiB/sec    980.686 MiB/sec   -6.230%
   - FilterFSLInt64FilterWithNulls/262144/0   399.890 MiB/sec  375.677 MiB/sec   -6.055%
   - FilterFSLInt64FilterWithNulls/262144/2   4.497 GiB/sec    4.233 GiB/sec     -5.880%
   - FilterFSLInt64FilterNoNulls/262144/1     564.700 MiB/sec  431.560 MiB/sec   -23.577%
   - FilterInt64FilterWithNulls/262144/9      549.832 MiB/sec  499.657 MiB/sec   -9.125%
   - FilterInt64FilterWithNulls/262144/7      625.701 MiB/sec  550.091 MiB/sec   -12.084%
     FilterInt64FilterNoNulls/262144/14       6.386 GiB/sec    6.901 GiB/sec     8.073%
     FilterInt64FilterWithNulls/262144/8      5.034 GiB/sec    4.958 GiB/sec     -1.517%
     FilterInt64FilterNoNulls/262144/12       3.215 GiB/sec    3.131 GiB/sec     -2.607%
     FilterStringFilterNoNulls/262144/0       560.832 MiB/sec  545.275 MiB/sec   -2.774%
   - FilterStringFilterNoNulls/262144/7       641.313 MiB/sec  582.952 MiB/sec   -9.100%
   - FilterInt64FilterWithNulls/262144/3      615.558 MiB/sec  496.003 MiB/sec   -19.422%
   - FilterStringFilterNoNulls/262144/10      578.560 MiB/sec  506.085 MiB/sec   -12.527%
     FilterInt64FilterWithNulls/262144/14     4.934 GiB/sec    4.873 GiB/sec     -1.228%
     FilterInt64FilterNoNulls/262144/5        7.145 GiB/sec    6.863 GiB/sec     -3.945%
     FilterStringFilterWithNulls/262144/7     632.496 MiB/sec  669.411 MiB/sec   5.836%
     FilterInt64FilterWithNulls/262144/11     4.937 GiB/sec    4.860 GiB/sec     -1.544%
   - FilterStringFilterWithNulls/262144/5     9.095 GiB/sec    8.275 GiB/sec     -9.015%
     FilterStringFilterNoNulls/262144/6       483.482 MiB/sec  470.273 MiB/sec   -2.732%
   - FilterFSLInt64FilterWithNulls/262144/7   464.358 MiB/sec  418.157 MiB/sec   -9.949%
   - FilterStringFilterNoNulls/262144/11      10.039 GiB/sec   7.873 GiB/sec     -21.572%
     FilterInt64FilterNoNulls/262144/11       6.389 GiB/sec    6.942 GiB/sec     8.664%
   - FilterFSLInt64FilterNoNulls/262144/6     400.926 MiB/sec  355.070 MiB/sec   -11.437%
   - FilterStringFilterNoNulls/262144/5       10.942 GiB/sec   8.621 GiB/sec     -21.211%
     FilterInt64FilterNoNulls/262144/2        7.901 GiB/sec    7.942 GiB/sec     0.526%
   - FilterFSLInt64FilterWithNulls/262144/13  387.523 MiB/sec  354.145 MiB/sec   -8.613%
   - FilterInt64FilterNoNulls/262144/10       635.634 MiB/sec  574.368 MiB/sec   -9.639%
   - FilterStringFilterWithNulls/262144/11    8.363 GiB/sec    7.663 GiB/sec     -8.365%
   - FilterInt64FilterWithNulls/262144/4      644.733 MiB/sec  554.689 MiB/sec   -13.966%
   - FilterInt64FilterWithNulls/262144/2      5.308 GiB/sec    4.950 GiB/sec     -6.739%
   - FilterInt64FilterWithNulls/262144/6      582.743 MiB/sec  494.561 MiB/sec   -15.132%
     FilterFSLInt64FilterWithNulls/262144/5   4.299 GiB/sec    4.094 GiB/sec     -4.757%
     FilterInt64FilterNoNulls/262144/0        7.685 GiB/sec    8.021 GiB/sec     4.371%
   - FilterInt64FilterNoNulls/262144/13       634.999 MiB/sec  574.211 MiB/sec   -9.573%
   - FilterStringFilterWithNulls/262144/2     9.478 GiB/sec    8.593 GiB/sec     -9.337%
     FilterFSLInt64FilterWithNulls/262144/8   4.256 GiB/sec    4.060 GiB/sec     -4.609%
   - FilterFSLInt64FilterWithNulls/262144/10  422.316 MiB/sec  380.968 MiB/sec   -9.791%
     FilterStringFilterNoNulls/262144/9       383.197 MiB/sec  374.020 MiB/sec   -2.395%
   - FilterFSLInt64FilterNoNulls/262144/12    242.820 MiB/sec  227.762 MiB/sec   -6.201%
     FilterStringFilterWithNulls/262144/0     429.008 MiB/sec  493.378 MiB/sec   15.004%
   - FilterFSLInt64FilterWithNulls/262144/12  267.881 MiB/sec  249.827 MiB/sec   -6.739%
     FilterFSLInt64FilterWithNulls/262144/3   349.988 MiB/sec  337.076 MiB/sec   -3.689%
     FilterStringFilterWithNulls/262144/13    90.911 MiB/sec   97.476 MiB/sec    7.222%
     =======================================  ===============  ================  ========
   ```


----------------------------------------------------------------
This is an automated message from the 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] wesm commented on a change in pull request #7442: ARROW-9075: [C++] Optimized Filter implementation: faster performance + compilation, smaller code size

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



##########
File path: cpp/src/arrow/util/bit_block_counter.h
##########
@@ -33,11 +33,50 @@ class Buffer;
 
 namespace internal {
 
+namespace detail {
+
+// These templates are here to help with unit tests
+
+template <typename T>
+struct BitBlockAnd {
+  static T Call(T left, T right) { return left & right; }
+};
+
+template <>
+struct BitBlockAnd<bool> {
+  static bool Call(bool left, bool right) { return left && right; }
+};
+
+template <typename T>
+struct BitBlockOr {
+  static T Call(T left, T right) { return left | right; }
+};
+
+template <>
+struct BitBlockOr<bool> {
+  static bool Call(bool left, bool right) { return left || right; }
+};
+
+template <typename T>
+struct BitBlockOrNot {
+  static T Call(T left, T right) { return left | ~right; }
+};
+
+template <>
+struct BitBlockOrNot<bool> {
+  static bool Call(bool left, bool right) { return left || !right; }
+};
+
+}  // namespace detail
+
 /// \brief Return value from bit block counters: the total number of bits and
 /// the number of set bits.
 struct BitBlockCount {
   int16_t length;
   int16_t popcount;
+
+  bool IsEmpty() const { return this->popcount == 0; }

Review comment:
       Good point. I'll rename them




----------------------------------------------------------------
This is an automated message from the 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] wesm commented on a change in pull request #7442: ARROW-9075: [C++] Optimized Filter implementation: faster performance + compilation, smaller code size

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



##########
File path: cpp/src/arrow/testing/random.cc
##########
@@ -84,7 +84,7 @@ std::shared_ptr<Array> RandomArrayGenerator::Boolean(int64_t size, double probab
 
   BufferVector buffers{2};
   // Need 2 distinct generators such that probabilities are not shared.
-  GenOpt value_gen(seed(), 0, 1, probability);
+  GenOpt value_gen(seed(), 0, 1, 1 - probability);

Review comment:
       Done, and renamed the parameters to be clear that it's the "true" probaility




----------------------------------------------------------------
This is an automated message from the 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] wesm commented on pull request #7442: ARROW-9075: [C++] Optimized Filter implementation: faster performance + compilation, smaller code size

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


   @ursabot benchmark --benchmark-filter=Filter c4f425768


----------------------------------------------------------------
This is an automated message from the 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] wesm commented on a change in pull request #7442: ARROW-9075: [C++] Optimized Filter implementation: faster performance + compilation, smaller code size

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



##########
File path: cpp/src/arrow/compute/kernels/vector_selection_test.cc
##########
@@ -0,0 +1,1637 @@
+// 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 <algorithm>
+#include <iostream>
+#include <limits>
+#include <memory>
+#include <string>
+#include <utility>
+#include <vector>
+
+#include "arrow/compute/api.h"
+#include "arrow/compute/kernels/test_util.h"
+#include "arrow/table.h"
+#include "arrow/testing/gtest_common.h"
+#include "arrow/testing/gtest_util.h"
+#include "arrow/testing/random.h"
+#include "arrow/testing/util.h"
+
+namespace arrow {
+
+using internal::checked_cast;
+using internal::checked_pointer_cast;
+using util::string_view;
+
+namespace compute {
+
+// ----------------------------------------------------------------------
+// Some random data generation helpers
+
+template <typename Type>
+std::shared_ptr<Array> RandomNumeric(int64_t length, double null_probability,
+                                     random::RandomArrayGenerator* rng) {
+  return rng->Numeric<Type>(length, 0, 127, null_probability);
+}
+
+std::shared_ptr<Array> RandomBoolean(int64_t length, double null_probability,
+                                     random::RandomArrayGenerator* rng) {
+  return rng->Boolean(length, 0.5, null_probability);
+}
+
+std::shared_ptr<Array> RandomString(int64_t length, double null_probability,
+                                    random::RandomArrayGenerator* rng) {
+  return rng->String(length, 0, 32, null_probability);
+}
+
+std::shared_ptr<Array> RandomLargeString(int64_t length, double null_probability,
+                                         random::RandomArrayGenerator* rng) {
+  return rng->LargeString(length, 0, 32, null_probability);
+}
+
+std::shared_ptr<Array> RandomFixedSizeBinary(int64_t length, double null_probability,
+                                             random::RandomArrayGenerator* rng) {
+  const int32_t value_size = 16;
+  int64_t data_nbytes = length * value_size;
+  std::shared_ptr<Buffer> data = *AllocateBuffer(data_nbytes);
+  random_bytes(data_nbytes, /*seed=*/0, data->mutable_data());
+  auto validity = rng->Boolean(length, 1 - null_probability);
+
+  // Assemble the data for a FixedSizeBinaryArray
+  auto values_data = std::make_shared<ArrayData>(fixed_size_binary(value_size), length);
+  values_data->buffers = {validity->data()->buffers[1], data};
+  return MakeArray(values_data);
+}
+
+// ----------------------------------------------------------------------
+
+TEST(GetTakeIndices, Basics) {
+  auto CheckCase = [&](const std::string& filter_json, const std::string& indices_json,
+                       FilterOptions::NullSelectionBehavior null_selection,
+                       const std::shared_ptr<DataType>& indices_type = uint16()) {
+    auto filter = ArrayFromJSON(boolean(), filter_json);
+    auto expected_indices = ArrayFromJSON(indices_type, indices_json);
+    ASSERT_OK_AND_ASSIGN(auto indices,
+                         internal::GetTakeIndices(*filter->data(), null_selection));
+    AssertArraysEqual(*expected_indices, *MakeArray(indices), /*verbose=*/true);
+  };
+
+  // Drop null cases
+  CheckCase("[]", "[]", FilterOptions::DROP);
+  CheckCase("[null]", "[]", FilterOptions::DROP);
+  CheckCase("[null, false, true, true, false, true]", "[2, 3, 5]", FilterOptions::DROP);
+
+  // Emit null cases
+  CheckCase("[]", "[]", FilterOptions::EMIT_NULL);
+  CheckCase("[null]", "[null]", FilterOptions::EMIT_NULL);
+  CheckCase("[null, false, true, true]", "[null, 2, 3]", FilterOptions::EMIT_NULL);
+}
+
+// TODO: Add slicing
+
+template <typename IndexArrayType>
+void CheckGetTakeIndicesCase(const Array& untyped_filter) {
+  const auto& filter = checked_cast<const BooleanArray&>(untyped_filter);
+  ASSERT_OK_AND_ASSIGN(std::shared_ptr<ArrayData> drop_indices,
+                       internal::GetTakeIndices(*filter.data(), FilterOptions::DROP));
+  // Verify DROP indices
+  {
+    IndexArrayType indices(drop_indices);
+    int64_t out_position = 0;
+    for (int64_t i = 0; i < filter.length(); ++i) {
+      if (filter.IsValid(i)) {
+        if (filter.Value(i)) {
+          ASSERT_EQ(indices.Value(out_position), i);
+          ++out_position;
+        }
+      }
+    }
+    // Check that the end length agrees with the output of GetFilterOutputSize
+    ASSERT_EQ(out_position,
+              internal::GetFilterOutputSize(*filter.data(), FilterOptions::DROP));
+  }
+
+  ASSERT_OK_AND_ASSIGN(
+      std::shared_ptr<ArrayData> emit_indices,
+      internal::GetTakeIndices(*filter.data(), FilterOptions::EMIT_NULL));
+
+  // Verify EMIT_NULL indices
+  {
+    IndexArrayType indices(emit_indices);
+    int64_t out_position = 0;
+    for (int64_t i = 0; i < filter.length(); ++i) {
+      if (filter.IsValid(i)) {
+        if (filter.Value(i)) {
+          ASSERT_EQ(indices.Value(out_position), i);
+          ++out_position;
+        }
+      } else {
+        ASSERT_TRUE(indices.IsNull(out_position));
+        ++out_position;
+      }
+    }
+
+    // Check that the end length agrees with the output of GetFilterOutputSize
+    ASSERT_EQ(out_position,
+              internal::GetFilterOutputSize(*filter.data(), FilterOptions::EMIT_NULL));
+  }
+}
+
+TEST(GetTakeIndices, RandomlyGenerated) {
+  random::RandomArrayGenerator rng(kRandomSeed);
+
+  const int64_t length = 5000;

Review comment:
       OK




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

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



[GitHub] [arrow] github-actions[bot] commented on pull request #7442: ARROW-9075: [C++] Optimized Filter implementation: faster performance + compilation, smaller code size

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


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


----------------------------------------------------------------
This is an automated message from the 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] wesm commented on pull request #7442: ARROW-9075: [C++] Optimized Filter implementation: faster performance + compilation, smaller code size

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


   @ursabot benchmark --benchmark-filter=Filter 66df3d0


----------------------------------------------------------------
This is an automated message from the 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] wesm commented on pull request #7442: ARROW-9075: [C++] Optimized Filter implementation: faster performance + compilation, smaller code size

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


   Something weird with the commit history, I'm not sure those benchmarks are right. I'll rebase things again and rerun


----------------------------------------------------------------
This is an automated message from the 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] ursabot commented on pull request #7442: ARROW-9075: [C++] Optimized Filter implementation: faster performance + compilation, smaller code size

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


   [AMD64 Ubuntu 18.04 C++ Benchmark (#113048)](https://ci.ursalabs.org/#builders/73/builds/83) builder has been succeeded.
   
   Revision: 54bb83848d391477c5ded222fd4401acbe08c6c7
   
   ```diff
     =======================================  ===============  ===============  ========
     benchmark                                baseline         contender        change
     =======================================  ===============  ===============  ========
     FilterStringFilterWithNulls/262144/9     395.928 MiB/sec  397.664 MiB/sec  0.439%
     FilterInt64FilterWithNulls/262144/0      621.828 MiB/sec  613.884 MiB/sec  -1.277%
     FilterStringFilterWithNulls/262144/10    578.179 MiB/sec  577.449 MiB/sec  -0.126%
     FilterFSLInt64FilterWithNulls/262144/14  4.068 GiB/sec    4.018 GiB/sec    -1.247%
     FilterInt64FilterWithNulls/262144/13     604.515 MiB/sec  575.481 MiB/sec  -4.803%
     FilterFSLInt64FilterNoNulls/262144/13    350.875 MiB/sec  355.061 MiB/sec  1.193%
     FilterStringFilterWithNulls/262144/0     441.188 MiB/sec  442.379 MiB/sec  0.270%
     FilterInt64FilterWithNulls/262144/7      623.569 MiB/sec  594.423 MiB/sec  -4.674%
     FilterStringFilterWithNulls/262144/12    73.925 MiB/sec   73.930 MiB/sec   0.007%
     FilterStringFilterNoNulls/262144/3       548.889 MiB/sec  548.269 MiB/sec  -0.113%
     FilterInt64FilterNoNulls/262144/0        7.942 GiB/sec    8.079 GiB/sec    1.727%
     FilterInt64FilterNoNulls/262144/6        3.827 GiB/sec    3.725 GiB/sec    -2.665%
     FilterStringFilterWithNulls/262144/2     9.138 GiB/sec    9.205 GiB/sec    0.726%
     FilterFSLInt64FilterWithNulls/262144/13  385.938 MiB/sec  370.599 MiB/sec  -3.975%
     FilterInt64FilterWithNulls/262144/9      549.281 MiB/sec  542.112 MiB/sec  -1.305%
     FilterInt64FilterWithNulls/262144/2      5.253 GiB/sec    5.047 GiB/sec    -3.918%
     FilterFSLInt64FilterNoNulls/262144/5     5.778 GiB/sec    5.676 GiB/sec    -1.761%
     FilterStringFilterNoNulls/262144/1       711.705 MiB/sec  697.941 MiB/sec  -1.934%
     FilterStringFilterNoNulls/262144/0       560.111 MiB/sec  560.315 MiB/sec  0.036%
     FilterStringFilterWithNulls/262144/5     8.773 GiB/sec    8.976 GiB/sec    2.318%
     FilterInt64FilterWithNulls/262144/11     4.863 GiB/sec    4.942 GiB/sec    1.631%
     FilterFSLInt64FilterWithNulls/262144/11  4.145 GiB/sec    4.089 GiB/sec    -1.362%
     FilterInt64FilterNoNulls/262144/2        7.854 GiB/sec    7.609 GiB/sec    -3.117%
     FilterStringFilterNoNulls/262144/11      9.751 GiB/sec    9.565 GiB/sec    -1.904%
     FilterStringFilterNoNulls/262144/7       641.570 MiB/sec  650.710 MiB/sec  1.425%
     FilterStringFilterWithNulls/262144/3     435.185 MiB/sec  436.932 MiB/sec  0.401%
     FilterFSLInt64FilterNoNulls/262144/14    5.202 GiB/sec    5.302 GiB/sec    1.915%
     FilterInt64FilterNoNulls/262144/4        674.907 MiB/sec  654.585 MiB/sec  -3.011%
     FilterInt64FilterNoNulls/262144/5        7.023 GiB/sec    6.971 GiB/sec    -0.741%
     FilterInt64FilterWithNulls/262144/12     548.203 MiB/sec  542.909 MiB/sec  -0.966%
     FilterFSLInt64FilterNoNulls/262144/10    387.772 MiB/sec  390.564 MiB/sec  0.720%
     FilterInt64FilterWithNulls/262144/8      4.951 GiB/sec    5.094 GiB/sec    2.880%
     FilterStringFilterNoNulls/262144/13      90.750 MiB/sec   91.694 MiB/sec   1.040%
     FilterFSLInt64FilterWithNulls/262144/12  230.292 MiB/sec  263.113 MiB/sec  14.252%
     FilterStringFilterNoNulls/262144/12      70.772 MiB/sec   70.740 MiB/sec   -0.044%
     FilterStringFilterWithNulls/262144/14    927.254 MiB/sec  925.791 MiB/sec  -0.158%
     FilterStringFilterNoNulls/262144/5       10.587 GiB/sec   10.322 GiB/sec   -2.509%
     FilterFSLInt64FilterNoNulls/262144/3     551.473 MiB/sec  556.816 MiB/sec  0.969%
     FilterInt64FilterNoNulls/262144/14       6.302 GiB/sec    6.848 GiB/sec    8.656%
     FilterInt64FilterWithNulls/262144/14     4.804 GiB/sec    4.945 GiB/sec    2.933%
     FilterStringFilterNoNulls/262144/14      1.257 GiB/sec    1.247 GiB/sec    -0.814%
     FilterFSLInt64FilterNoNulls/262144/6     399.266 MiB/sec  402.455 MiB/sec  0.799%
     FilterInt64FilterWithNulls/262144/5      5.037 GiB/sec    4.954 GiB/sec    -1.645%
     FilterFSLInt64FilterNoNulls/262144/8     5.576 GiB/sec    5.576 GiB/sec    -0.004%
     FilterFSLInt64FilterNoNulls/262144/7     462.231 MiB/sec  456.668 MiB/sec  -1.203%
     FilterFSLInt64FilterNoNulls/262144/11    5.377 GiB/sec    5.381 GiB/sec    0.082%
     FilterStringFilterNoNulls/262144/6       487.645 MiB/sec  487.464 MiB/sec  -0.037%
     FilterStringFilterNoNulls/262144/4       687.214 MiB/sec  678.019 MiB/sec  -1.338%
     FilterFSLInt64FilterWithNulls/262144/9   287.916 MiB/sec  285.805 MiB/sec  -0.733%
     FilterInt64FilterNoNulls/262144/9        3.245 GiB/sec    3.126 GiB/sec    -3.683%
     FilterFSLInt64FilterWithNulls/262144/1   514.149 MiB/sec  501.235 MiB/sec  -2.512%
     FilterInt64FilterNoNulls/262144/11       6.304 GiB/sec    6.838 GiB/sec    8.471%
     FilterInt64FilterWithNulls/262144/4      642.597 MiB/sec  617.492 MiB/sec  -3.907%
     FilterFSLInt64FilterNoNulls/262144/0     723.263 MiB/sec  719.475 MiB/sec  -0.524%
     FilterFSLInt64FilterWithNulls/262144/2   4.335 GiB/sec    4.281 GiB/sec    -1.228%
     FilterStringFilterWithNulls/262144/8     8.635 GiB/sec    8.847 GiB/sec    2.451%
     FilterFSLInt64FilterWithNulls/262144/4   473.024 MiB/sec  457.711 MiB/sec  -3.237%
     FilterStringFilterWithNulls/262144/4     637.237 MiB/sec  646.187 MiB/sec  1.405%
     FilterStringFilterWithNulls/262144/6     430.118 MiB/sec  433.059 MiB/sec  0.684%
     FilterStringFilterNoNulls/262144/10      572.254 MiB/sec  573.892 MiB/sec  0.286%
     FilterStringFilterWithNulls/262144/1     644.800 MiB/sec  644.056 MiB/sec  -0.115%
     FilterStringFilterWithNulls/262144/7     635.644 MiB/sec  640.796 MiB/sec  0.810%
     FilterInt64FilterWithNulls/262144/6      581.863 MiB/sec  575.886 MiB/sec  -1.027%
     FilterFSLInt64FilterNoNulls/262144/4     513.508 MiB/sec  499.319 MiB/sec  -2.763%
     FilterInt64FilterNoNulls/262144/13       632.203 MiB/sec  613.689 MiB/sec  -2.928%
     FilterStringFilterNoNulls/262144/8       10.491 GiB/sec   10.181 GiB/sec   -2.953%
     FilterFSLInt64FilterNoNulls/262144/1     563.147 MiB/sec  540.663 MiB/sec  -3.993%
     FilterFSLInt64FilterNoNulls/262144/9     267.226 MiB/sec  269.194 MiB/sec  0.736%
     FilterFSLInt64FilterWithNulls/262144/10  420.329 MiB/sec  405.197 MiB/sec  -3.600%
   - FilterInt64FilterNoNulls/262144/1        1.022 GiB/sec    922.850 MiB/sec  -11.845%
     FilterInt64FilterNoNulls/262144/7        652.709 MiB/sec  631.526 MiB/sec  -3.245%
     FilterStringFilterNoNulls/262144/2       11.144 GiB/sec   10.843 GiB/sec   -2.698%
     FilterStringFilterWithNulls/262144/13    91.231 MiB/sec   91.638 MiB/sec   0.446%
     FilterInt64FilterNoNulls/262144/12       3.242 GiB/sec    3.112 GiB/sec    -4.024%
     FilterFSLInt64FilterNoNulls/262144/12    242.297 MiB/sec  242.607 MiB/sec  0.128%
     FilterFSLInt64FilterNoNulls/262144/2     6.165 GiB/sec    6.062 GiB/sec    -1.679%
     FilterFSLInt64FilterWithNulls/262144/6   331.566 MiB/sec  332.386 MiB/sec  0.247%
     FilterInt64FilterWithNulls/262144/1      648.702 MiB/sec  622.712 MiB/sec  -4.006%
     FilterFSLInt64FilterWithNulls/262144/5   4.123 GiB/sec    4.122 GiB/sec    -0.014%
     FilterFSLInt64FilterWithNulls/262144/0   399.262 MiB/sec  398.338 MiB/sec  -0.231%
     FilterFSLInt64FilterWithNulls/262144/3   347.643 MiB/sec  349.930 MiB/sec  0.658%
     FilterInt64FilterNoNulls/262144/3        4.312 GiB/sec    4.291 GiB/sec    -0.478%
     FilterStringFilterWithNulls/262144/11    8.207 GiB/sec    8.348 GiB/sec    1.720%
     FilterStringFilterNoNulls/262144/9       391.780 MiB/sec  391.367 MiB/sec  -0.106%
     FilterFSLInt64FilterWithNulls/262144/8   4.142 GiB/sec    4.103 GiB/sec    -0.926%
     FilterInt64FilterNoNulls/262144/8        6.703 GiB/sec    6.908 GiB/sec    3.063%
     FilterInt64FilterWithNulls/262144/10     604.595 MiB/sec  575.671 MiB/sec  -4.784%
     FilterFSLInt64FilterWithNulls/262144/7   461.693 MiB/sec  447.411 MiB/sec  -3.093%
     FilterInt64FilterNoNulls/262144/10       632.128 MiB/sec  614.452 MiB/sec  -2.796%
     FilterInt64FilterWithNulls/262144/3      613.629 MiB/sec  607.939 MiB/sec  -0.927%
     =======================================  ===============  ===============  ========
   ```


----------------------------------------------------------------
This is an automated message from the 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] wesm commented on pull request #7442: ARROW-9075: [C++] Optimized Filter implementation: faster performance + compilation, smaller code size

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


   I think I improved some of the readability problems and addressed the other comments. I'd like to merge this soon once CI is creen


----------------------------------------------------------------
This is an automated message from the 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] wesm commented on a change in pull request #7442: ARROW-9075: [C++] Optimized Filter implementation: faster performance + compilation, smaller code size

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



##########
File path: cpp/src/arrow/compute/kernels/vector_selection.cc
##########
@@ -0,0 +1,1816 @@
+// 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 <algorithm>
+#include <cstring>
+#include <limits>
+
+#include "arrow/array/array_base.h"
+#include "arrow/array/array_binary.h"
+#include "arrow/array/array_dict.h"
+#include "arrow/array/array_nested.h"
+#include "arrow/array/builder_primitive.h"
+#include "arrow/array/concatenate.h"
+#include "arrow/buffer_builder.h"
+#include "arrow/compute/api_vector.h"
+#include "arrow/compute/kernels/common.h"
+#include "arrow/compute/kernels/util_internal.h"
+#include "arrow/extension_type.h"
+#include "arrow/record_batch.h"
+#include "arrow/result.h"
+#include "arrow/util/bit_block_counter.h"
+#include "arrow/util/bit_util.h"
+#include "arrow/util/bitmap_ops.h"
+#include "arrow/util/bitmap_reader.h"
+#include "arrow/util/int_util.h"
+
+namespace arrow {
+
+using internal::BinaryBitBlockCounter;
+using internal::BitBlockCount;
+using internal::BitBlockCounter;
+using internal::BitmapReader;
+using internal::CopyBitmap;
+using internal::GetArrayView;
+using internal::IndexBoundsCheck;
+using internal::OptionalBitBlockCounter;
+using internal::OptionalBitIndexer;
+
+namespace compute {
+namespace internal {
+
+int64_t GetFilterOutputSize(const ArrayData& filter,
+                            FilterOptions::NullSelectionBehavior null_selection) {
+  int64_t output_size = 0;
+  int64_t position = 0;
+  if (filter.GetNullCount() > 0) {
+    const uint8_t* filter_is_valid = filter.buffers[0]->data();
+    BinaryBitBlockCounter bit_counter(filter.buffers[1]->data(), filter.offset,
+                                      filter_is_valid, filter.offset, filter.length);
+    if (null_selection == FilterOptions::EMIT_NULL) {
+      while (position < filter.length) {
+        BitBlockCount block = bit_counter.NextOrNotWord();
+        output_size += block.popcount;
+        position += block.length;
+      }
+    } else {
+      while (position < filter.length) {
+        BitBlockCount block = bit_counter.NextAndWord();
+        output_size += block.popcount;
+        position += block.length;
+      }
+    }
+  } else {
+    // The filter has no nulls, so we plow through its data as fast as
+    // possible.
+    BitBlockCounter bit_counter(filter.buffers[1]->data(), filter.offset, filter.length);
+    while (position < filter.length) {
+      BitBlockCount block = bit_counter.NextFourWords();
+      output_size += block.popcount;
+      position += block.length;
+    }
+  }
+  return output_size;
+}
+
+template <typename IndexType>
+Result<std::shared_ptr<ArrayData>> GetTakeIndicesImpl(
+    const ArrayData& filter, FilterOptions::NullSelectionBehavior null_selection,
+    MemoryPool* memory_pool) {
+  using T = typename IndexType::c_type;
+  typename TypeTraits<IndexType>::BuilderType builder(memory_pool);
+
+  const uint8_t* filter_data = filter.buffers[1]->data();
+  BitBlockCounter data_counter(filter_data, filter.offset, filter.length);
+
+  // The position relative to the start of the filter
+  T position = 0;
+
+  // The current position taking the filter offset into account
+  int64_t position_with_offset = filter.offset;
+  if (filter.GetNullCount() > 0) {
+    // The filter has nulls, so we scan the validity bitmap and the filter data
+    // bitmap together, branching on the null selection type.
+    const uint8_t* filter_is_valid = filter.buffers[0]->data();
+
+    // To count blocks whether filter_data[i] || !filter_is_valid[i]
+    BinaryBitBlockCounter filter_counter(filter_data, filter.offset, filter_is_valid,
+                                         filter.offset, filter.length);
+    if (null_selection == FilterOptions::DROP) {
+      while (position < filter.length) {
+        BitBlockCount and_block = filter_counter.NextAndWord();
+        RETURN_NOT_OK(builder.Reserve(and_block.popcount));
+        if (and_block.AllSet()) {
+          // All the values are selected and non-null
+          for (int64_t i = 0; i < and_block.length; ++i) {
+            builder.UnsafeAppend(position++);
+          }
+          position_with_offset += and_block.length;
+        } else {
+          // Some of the values are false or null
+          for (int64_t i = 0; i < and_block.length; ++i) {
+            if (BitUtil::GetBit(filter_is_valid, position_with_offset) &&
+                BitUtil::GetBit(filter_data, position_with_offset)) {
+              builder.UnsafeAppend(position);
+            }
+            ++position;
+            ++position_with_offset;
+          }
+        }
+      }
+    } else {
+      BitBlockCounter is_valid_counter(filter_is_valid, filter.offset, filter.length);
+      while (position < filter.length) {
+        // true OR NOT valid
+        BitBlockCount or_not_block = filter_counter.NextOrNotWord();
+        RETURN_NOT_OK(builder.Reserve(or_not_block.popcount));
+
+        // If the values are all valid and the or_not_block is full, then we
+        // can infer that all the values are true and skip the bit checking
+        BitBlockCount is_valid_block = is_valid_counter.NextWord();
+
+        if (or_not_block.AllSet() && is_valid_block.AllSet()) {
+          // All the values are selected and non-null
+          for (int64_t i = 0; i < or_not_block.length; ++i) {
+            builder.UnsafeAppend(position++);
+          }
+          position_with_offset += or_not_block.length;
+        } else {
+          // Some of the values are false or null
+          for (int64_t i = 0; i < or_not_block.length; ++i) {
+            if (BitUtil::GetBit(filter_is_valid, position_with_offset)) {
+              if (BitUtil::GetBit(filter_data, position_with_offset)) {
+                builder.UnsafeAppend(position);
+              }
+            } else {
+              // Null slot, so append a null
+              builder.UnsafeAppendNull();
+            }
+            ++position;
+            ++position_with_offset;
+          }
+        }
+      }
+    }
+  } else {
+    // The filter has no nulls, so we need only look for true values
+    BitBlockCount current_block = data_counter.NextWord();
+    while (position < filter.length) {
+      if (current_block.AllSet()) {
+        int64_t run_length = 0;
+
+        // If we've found a all-true block, then we scan forward until we find
+        // a block that has some false values (or we reach the end
+        while (current_block.length > 0 && current_block.AllSet()) {
+          run_length += current_block.length;
+          current_block = data_counter.NextWord();
+        }
+
+        // Append the consecutive run of indices
+        RETURN_NOT_OK(builder.Reserve(run_length));
+        for (int64_t i = 0; i < run_length; ++i) {
+          builder.UnsafeAppend(position++);
+        }
+        position_with_offset += run_length;
+      } else {
+        // Must do bitchecking on the current block

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.

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



[GitHub] [arrow] wesm commented on pull request #7442: ARROW-9075: [C++] Optimized Filter implementation: faster performance + compilation, smaller code size

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


   @ursabot benchmark --benchmark-filter=Filter 04006ff


----------------------------------------------------------------
This is an automated message from the 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] ursabot commented on pull request #7442: ARROW-9075: [C++] Optimized Filter implementation: faster performance + compilation, smaller code size

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


   [AMD64 Ubuntu 18.04 C++ Benchmark (#112487)](https://ci.ursalabs.org/#builders/73/builds/75) builder has been succeeded.
   
   Revision: 31a66630f6bcb9a3f74912da7d31ac2412e97184
   
   ```diff
     =======================================  ===============  ===============  =========
     benchmark                                baseline         contender        change
     =======================================  ===============  ===============  =========
     FilterInt64FilterWithNulls/262144/3      563.800 MiB/sec  576.625 MiB/sec  2.275%
   - FilterStringFilterWithNulls/262144/3     498.174 MiB/sec  434.196 MiB/sec  -12.842%
     FilterFSLInt64FilterNoNulls/262144/9     158.897 MiB/sec  268.195 MiB/sec  68.785%
     FilterInt64FilterNoNulls/262144/14       2.793 GiB/sec    6.554 GiB/sec    134.709%
     FilterFSLInt64FilterNoNulls/262144/11    2.356 GiB/sec    5.386 GiB/sec    128.589%
     FilterStringFilterNoNulls/262144/2       4.937 GiB/sec    10.996 GiB/sec   122.715%
     FilterFSLInt64FilterWithNulls/262144/5   1.590 GiB/sec    4.193 GiB/sec    163.732%
     FilterInt64FilterWithNulls/262144/12     519.932 MiB/sec  496.829 MiB/sec  -4.443%
     FilterInt64FilterNoNulls/262144/0        669.365 MiB/sec  7.541 GiB/sec    1053.558%
     FilterFSLInt64FilterNoNulls/262144/1     268.027 MiB/sec  560.837 MiB/sec  109.246%
     FilterStringFilterNoNulls/262144/6       488.692 MiB/sec  481.827 MiB/sec  -1.405%
     FilterInt64FilterNoNulls/262144/8        2.735 GiB/sec    6.313 GiB/sec    130.810%
     FilterInt64FilterNoNulls/262144/5        2.809 GiB/sec    6.018 GiB/sec    114.267%
   - FilterStringFilterWithNulls/262144/12    84.168 MiB/sec   70.410 MiB/sec   -16.346%
     FilterFSLInt64FilterNoNulls/262144/0     169.867 MiB/sec  718.594 MiB/sec  323.035%
     FilterStringFilterWithNulls/262144/14    355.644 MiB/sec  878.914 MiB/sec  147.133%
     FilterStringFilterWithNulls/262144/2     3.338 GiB/sec    8.903 GiB/sec    166.736%
     FilterFSLInt64FilterWithNulls/262144/1   263.151 MiB/sec  512.905 MiB/sec  94.909%
     FilterFSLInt64FilterNoNulls/262144/14    2.395 GiB/sec    5.212 GiB/sec    117.604%
     FilterInt64FilterWithNulls/262144/11     1.729 GiB/sec    4.684 GiB/sec    170.948%
     FilterInt64FilterNoNulls/262144/9        566.051 MiB/sec  3.083 GiB/sec    457.794%
   - FilterStringFilterWithNulls/262144/10    619.724 MiB/sec  578.798 MiB/sec  -6.604%
     FilterInt64FilterWithNulls/262144/1      541.616 MiB/sec  558.958 MiB/sec  3.202%
     FilterFSLInt64FilterWithNulls/262144/14  1.596 GiB/sec    4.061 GiB/sec    154.454%
     FilterFSLInt64FilterWithNulls/262144/0   170.064 MiB/sec  398.738 MiB/sec  134.464%
     FilterInt64FilterWithNulls/262144/2      1.739 GiB/sec    4.883 GiB/sec    180.721%
     FilterInt64FilterWithNulls/262144/4      528.271 MiB/sec  555.772 MiB/sec  5.206%
     FilterFSLInt64FilterNoNulls/262144/2     2.383 GiB/sec    6.074 GiB/sec    154.832%
     FilterInt64FilterNoNulls/262144/4        584.370 MiB/sec  579.728 MiB/sec  -0.794%
     FilterInt64FilterNoNulls/262144/12       575.177 MiB/sec  3.023 GiB/sec    438.268%
   - FilterStringFilterWithNulls/262144/9     459.179 MiB/sec  394.515 MiB/sec  -14.083%
     FilterStringFilterNoNulls/262144/5       4.936 GiB/sec    10.562 GiB/sec   113.987%
     FilterInt64FilterNoNulls/262144/2        2.838 GiB/sec    7.390 GiB/sec    160.374%
     FilterFSLInt64FilterNoNulls/262144/7     261.996 MiB/sec  464.922 MiB/sec  77.454%
     FilterStringFilterNoNulls/262144/14      580.305 MiB/sec  1.253 GiB/sec    121.158%
     FilterFSLInt64FilterWithNulls/262144/13  249.426 MiB/sec  386.982 MiB/sec  55.149%
   - FilterInt64FilterWithNulls/262144/9      530.774 MiB/sec  497.368 MiB/sec  -6.294%
     FilterStringFilterWithNulls/262144/8     3.270 GiB/sec    8.467 GiB/sec    158.943%
     FilterFSLInt64FilterNoNulls/262144/10    257.812 MiB/sec  390.196 MiB/sec  51.349%
   - FilterStringFilterNoNulls/262144/13      98.039 MiB/sec   90.475 MiB/sec   -7.716%
     FilterInt64FilterWithNulls/262144/8      1.737 GiB/sec    4.652 GiB/sec    167.790%
     FilterFSLInt64FilterWithNulls/262144/3   167.057 MiB/sec  351.817 MiB/sec  110.597%
   - FilterStringFilterWithNulls/262144/6     494.580 MiB/sec  429.801 MiB/sec  -13.098%
     FilterFSLInt64FilterWithNulls/262144/12  165.174 MiB/sec  262.176 MiB/sec  58.728%
     FilterInt64FilterWithNulls/262144/7      526.592 MiB/sec  541.187 MiB/sec  2.772%
     FilterStringFilterNoNulls/262144/11      4.531 GiB/sec    9.652 GiB/sec    113.006%
     FilterStringFilterWithNulls/262144/1     662.260 MiB/sec  633.359 MiB/sec  -4.364%
     FilterStringFilterWithNulls/262144/4     670.467 MiB/sec  644.877 MiB/sec  -3.817%
     FilterStringFilterNoNulls/262144/0       503.582 MiB/sec  550.304 MiB/sec  9.278%
   - FilterStringFilterNoNulls/262144/9       443.066 MiB/sec  390.416 MiB/sec  -11.883%
     FilterFSLInt64FilterNoNulls/262144/13    251.747 MiB/sec  351.809 MiB/sec  39.747%
     FilterInt64FilterNoNulls/262144/11       2.788 GiB/sec    6.687 GiB/sec    139.878%
   - FilterInt64FilterWithNulls/262144/0      620.421 MiB/sec  585.692 MiB/sec  -5.598%
     FilterFSLInt64FilterWithNulls/262144/8   1.593 GiB/sec    4.155 GiB/sec    160.783%
   - FilterStringFilterNoNulls/262144/7       692.942 MiB/sec  654.463 MiB/sec  -5.553%
     FilterStringFilterNoNulls/262144/8       4.900 GiB/sec    10.519 GiB/sec   114.694%
     FilterInt64FilterWithNulls/262144/10     510.602 MiB/sec  527.612 MiB/sec  3.331%
     FilterFSLInt64FilterNoNulls/262144/3     159.401 MiB/sec  555.494 MiB/sec  248.487%
     FilterFSLInt64FilterNoNulls/262144/6     162.294 MiB/sec  399.907 MiB/sec  146.410%
   - FilterStringFilterWithNulls/262144/0     517.359 MiB/sec  439.657 MiB/sec  -15.019%
     FilterInt64FilterWithNulls/262144/13     502.220 MiB/sec  527.971 MiB/sec  5.128%
     FilterStringFilterWithNulls/262144/7     666.386 MiB/sec  638.254 MiB/sec  -4.221%
     FilterInt64FilterNoNulls/262144/6        603.261 MiB/sec  3.473 GiB/sec    489.518%
     FilterStringFilterWithNulls/262144/11    2.994 GiB/sec    8.094 GiB/sec    170.304%
     FilterFSLInt64FilterWithNulls/262144/6   165.225 MiB/sec  335.017 MiB/sec  102.765%
     FilterFSLInt64FilterWithNulls/262144/7   257.333 MiB/sec  466.760 MiB/sec  81.383%
     FilterInt64FilterNoNulls/262144/7        583.317 MiB/sec  564.896 MiB/sec  -3.158%
     FilterStringFilterNoNulls/262144/4       691.530 MiB/sec  699.221 MiB/sec  1.112%
     FilterFSLInt64FilterWithNulls/262144/11  1.592 GiB/sec    4.057 GiB/sec    154.837%
   - FilterStringFilterNoNulls/262144/12      88.970 MiB/sec   70.067 MiB/sec   -21.246%
     FilterInt64FilterNoNulls/262144/10       562.254 MiB/sec  545.802 MiB/sec  -2.926%
     FilterInt64FilterWithNulls/262144/14     1.738 GiB/sec    4.747 GiB/sec    173.077%
     FilterFSLInt64FilterWithNulls/262144/2   1.570 GiB/sec    4.295 GiB/sec    173.597%
     FilterInt64FilterNoNulls/262144/13       558.715 MiB/sec  554.622 MiB/sec  -0.733%
     FilterInt64FilterWithNulls/262144/6      561.253 MiB/sec  537.786 MiB/sec  -4.181%
     FilterStringFilterWithNulls/262144/13    91.370 MiB/sec   89.650 MiB/sec   -1.882%
     FilterFSLInt64FilterNoNulls/262144/12    153.042 MiB/sec  241.416 MiB/sec  57.745%
     FilterFSLInt64FilterNoNulls/262144/5     2.414 GiB/sec    5.672 GiB/sec    134.917%
     FilterFSLInt64FilterNoNulls/262144/8     2.377 GiB/sec    5.541 GiB/sec    133.082%
   - FilterStringFilterNoNulls/262144/10      632.556 MiB/sec  572.816 MiB/sec  -9.444%
     FilterFSLInt64FilterWithNulls/262144/9   166.869 MiB/sec  288.049 MiB/sec  72.620%
     FilterInt64FilterNoNulls/262144/1        599.855 MiB/sec  912.146 MiB/sec  52.061%
     FilterStringFilterWithNulls/262144/5     3.295 GiB/sec    8.587 GiB/sec    160.574%
     FilterFSLInt64FilterNoNulls/262144/4     263.896 MiB/sec  514.836 MiB/sec  95.091%
     FilterFSLInt64FilterWithNulls/262144/4   258.744 MiB/sec  477.042 MiB/sec  84.369%
     FilterInt64FilterWithNulls/262144/5      1.735 GiB/sec    4.728 GiB/sec    172.542%
     FilterStringFilterNoNulls/262144/3       495.135 MiB/sec  539.178 MiB/sec  8.895%
     FilterInt64FilterNoNulls/262144/3        611.978 MiB/sec  3.929 GiB/sec    557.402%
     FilterFSLInt64FilterWithNulls/262144/10  255.156 MiB/sec  417.072 MiB/sec  63.458%
     FilterStringFilterNoNulls/262144/1       714.448 MiB/sec  713.457 MiB/sec  -0.139%
     =======================================  ===============  ===============  =========
   ```


----------------------------------------------------------------
This is an automated message from the 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] wesm edited a comment on pull request #7442: ARROW-9075: [C++] Optimized Filter implementation: faster performance + compilation, smaller code size

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


   I implemented some other optimizations, especially for the case where neither values nor filter contain nulls. I'm working on updated benchmarks
   
   Updated benchmarks: https://gist.github.com/wesm/ad07cec1613b6327926dfe1d95e7f4f0/revisions?diff=split


----------------------------------------------------------------
This is an automated message from the 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] wesm commented on a change in pull request #7442: ARROW-9075: [C++] Optimized Filter implementation: faster performance + compilation, smaller code size

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



##########
File path: cpp/src/arrow/compute/kernels/vector_selection.cc
##########
@@ -0,0 +1,1816 @@
+// 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 <algorithm>
+#include <cstring>
+#include <limits>
+
+#include "arrow/array/array_base.h"
+#include "arrow/array/array_binary.h"
+#include "arrow/array/array_dict.h"
+#include "arrow/array/array_nested.h"
+#include "arrow/array/builder_primitive.h"
+#include "arrow/array/concatenate.h"
+#include "arrow/buffer_builder.h"
+#include "arrow/compute/api_vector.h"
+#include "arrow/compute/kernels/common.h"
+#include "arrow/compute/kernels/util_internal.h"
+#include "arrow/extension_type.h"
+#include "arrow/record_batch.h"
+#include "arrow/result.h"
+#include "arrow/util/bit_block_counter.h"
+#include "arrow/util/bit_util.h"
+#include "arrow/util/bitmap_ops.h"
+#include "arrow/util/bitmap_reader.h"
+#include "arrow/util/int_util.h"
+
+namespace arrow {
+
+using internal::BinaryBitBlockCounter;
+using internal::BitBlockCount;
+using internal::BitBlockCounter;
+using internal::BitmapReader;
+using internal::CopyBitmap;
+using internal::GetArrayView;
+using internal::IndexBoundsCheck;
+using internal::OptionalBitBlockCounter;
+using internal::OptionalBitIndexer;
+
+namespace compute {
+namespace internal {
+
+int64_t GetFilterOutputSize(const ArrayData& filter,
+                            FilterOptions::NullSelectionBehavior null_selection) {
+  int64_t output_size = 0;
+  int64_t position = 0;
+  if (filter.GetNullCount() > 0) {
+    const uint8_t* filter_is_valid = filter.buffers[0]->data();
+    BinaryBitBlockCounter bit_counter(filter.buffers[1]->data(), filter.offset,
+                                      filter_is_valid, filter.offset, filter.length);
+    if (null_selection == FilterOptions::EMIT_NULL) {
+      while (position < filter.length) {
+        BitBlockCount block = bit_counter.NextOrNotWord();
+        output_size += block.popcount;
+        position += block.length;
+      }
+    } else {
+      while (position < filter.length) {
+        BitBlockCount block = bit_counter.NextAndWord();
+        output_size += block.popcount;
+        position += block.length;
+      }
+    }
+  } else {
+    // The filter has no nulls, so we plow through its data as fast as
+    // possible.
+    BitBlockCounter bit_counter(filter.buffers[1]->data(), filter.offset, filter.length);
+    while (position < filter.length) {
+      BitBlockCount block = bit_counter.NextFourWords();
+      output_size += block.popcount;
+      position += block.length;
+    }
+  }
+  return output_size;
+}
+
+template <typename IndexType>
+Result<std::shared_ptr<ArrayData>> GetTakeIndicesImpl(
+    const ArrayData& filter, FilterOptions::NullSelectionBehavior null_selection,
+    MemoryPool* memory_pool) {
+  using T = typename IndexType::c_type;
+  typename TypeTraits<IndexType>::BuilderType builder(memory_pool);
+
+  const uint8_t* filter_data = filter.buffers[1]->data();
+  BitBlockCounter data_counter(filter_data, filter.offset, filter.length);
+
+  // The position relative to the start of the filter
+  T position = 0;
+
+  // The current position taking the filter offset into account
+  int64_t position_with_offset = filter.offset;
+  if (filter.GetNullCount() > 0) {

Review comment:
       Someone can do it as follow up (and keeping an eye on the benchmarks to avoid perf regressions), it doesn't feel like a good use of my time when there are so many things to do for 1.0.0. 




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

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



[GitHub] [arrow] pitrou commented on a change in pull request #7442: ARROW-9075: [C++] Optimized Filter implementation: faster performance + compilation, smaller code size

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



##########
File path: cpp/src/arrow/compute/kernels/util_internal.h
##########
@@ -0,0 +1,50 @@
+// 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.
+
+#pragma once
+
+#include <cstdint>
+
+#include "arrow/buffer.h"
+
+namespace arrow {
+namespace compute {
+namespace internal {
+
+// An internal data structure for unpacking a primitive argument to pass to a
+// kernel implementation
+struct PrimitiveArg {
+  const uint8_t* is_valid;
+  const uint8_t* data;

Review comment:
       Should note that this takes the offset into account only if bit_width is > 1.

##########
File path: cpp/src/arrow/compute/kernels/vector_selection.cc
##########
@@ -0,0 +1,1816 @@
+// 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 <algorithm>
+#include <cstring>
+#include <limits>
+
+#include "arrow/array/array_base.h"
+#include "arrow/array/array_binary.h"
+#include "arrow/array/array_dict.h"
+#include "arrow/array/array_nested.h"
+#include "arrow/array/builder_primitive.h"
+#include "arrow/array/concatenate.h"
+#include "arrow/buffer_builder.h"
+#include "arrow/compute/api_vector.h"
+#include "arrow/compute/kernels/common.h"
+#include "arrow/compute/kernels/util_internal.h"
+#include "arrow/extension_type.h"
+#include "arrow/record_batch.h"
+#include "arrow/result.h"
+#include "arrow/util/bit_block_counter.h"
+#include "arrow/util/bit_util.h"
+#include "arrow/util/bitmap_ops.h"
+#include "arrow/util/bitmap_reader.h"
+#include "arrow/util/int_util.h"
+
+namespace arrow {
+
+using internal::BinaryBitBlockCounter;
+using internal::BitBlockCount;
+using internal::BitBlockCounter;
+using internal::BitmapReader;
+using internal::CopyBitmap;
+using internal::GetArrayView;
+using internal::IndexBoundsCheck;
+using internal::OptionalBitBlockCounter;
+using internal::OptionalBitIndexer;
+
+namespace compute {
+namespace internal {
+
+int64_t GetFilterOutputSize(const ArrayData& filter,
+                            FilterOptions::NullSelectionBehavior null_selection) {
+  int64_t output_size = 0;
+  int64_t position = 0;
+  if (filter.GetNullCount() > 0) {
+    const uint8_t* filter_is_valid = filter.buffers[0]->data();
+    BinaryBitBlockCounter bit_counter(filter.buffers[1]->data(), filter.offset,
+                                      filter_is_valid, filter.offset, filter.length);
+    if (null_selection == FilterOptions::EMIT_NULL) {
+      while (position < filter.length) {
+        BitBlockCount block = bit_counter.NextOrNotWord();
+        output_size += block.popcount;
+        position += block.length;
+      }
+    } else {
+      while (position < filter.length) {
+        BitBlockCount block = bit_counter.NextAndWord();
+        output_size += block.popcount;
+        position += block.length;
+      }
+    }
+  } else {
+    // The filter has no nulls, so we plow through its data as fast as
+    // possible.
+    BitBlockCounter bit_counter(filter.buffers[1]->data(), filter.offset, filter.length);

Review comment:
       Why not use `CountSetBits`?

##########
File path: cpp/src/arrow/compute/kernels/vector_selection.cc
##########
@@ -0,0 +1,1816 @@
+// 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 <algorithm>
+#include <cstring>
+#include <limits>
+
+#include "arrow/array/array_base.h"
+#include "arrow/array/array_binary.h"
+#include "arrow/array/array_dict.h"
+#include "arrow/array/array_nested.h"
+#include "arrow/array/builder_primitive.h"
+#include "arrow/array/concatenate.h"
+#include "arrow/buffer_builder.h"
+#include "arrow/compute/api_vector.h"
+#include "arrow/compute/kernels/common.h"
+#include "arrow/compute/kernels/util_internal.h"
+#include "arrow/extension_type.h"
+#include "arrow/record_batch.h"
+#include "arrow/result.h"
+#include "arrow/util/bit_block_counter.h"
+#include "arrow/util/bit_util.h"
+#include "arrow/util/bitmap_ops.h"
+#include "arrow/util/bitmap_reader.h"
+#include "arrow/util/int_util.h"
+
+namespace arrow {
+
+using internal::BinaryBitBlockCounter;
+using internal::BitBlockCount;
+using internal::BitBlockCounter;
+using internal::BitmapReader;
+using internal::CopyBitmap;
+using internal::GetArrayView;
+using internal::IndexBoundsCheck;
+using internal::OptionalBitBlockCounter;
+using internal::OptionalBitIndexer;
+
+namespace compute {
+namespace internal {
+
+int64_t GetFilterOutputSize(const ArrayData& filter,
+                            FilterOptions::NullSelectionBehavior null_selection) {
+  int64_t output_size = 0;
+  int64_t position = 0;
+  if (filter.GetNullCount() > 0) {
+    const uint8_t* filter_is_valid = filter.buffers[0]->data();
+    BinaryBitBlockCounter bit_counter(filter.buffers[1]->data(), filter.offset,
+                                      filter_is_valid, filter.offset, filter.length);
+    if (null_selection == FilterOptions::EMIT_NULL) {
+      while (position < filter.length) {
+        BitBlockCount block = bit_counter.NextOrNotWord();
+        output_size += block.popcount;
+        position += block.length;
+      }
+    } else {
+      while (position < filter.length) {
+        BitBlockCount block = bit_counter.NextAndWord();
+        output_size += block.popcount;
+        position += block.length;
+      }
+    }
+  } else {
+    // The filter has no nulls, so we plow through its data as fast as
+    // possible.
+    BitBlockCounter bit_counter(filter.buffers[1]->data(), filter.offset, filter.length);
+    while (position < filter.length) {
+      BitBlockCount block = bit_counter.NextFourWords();
+      output_size += block.popcount;
+      position += block.length;
+    }
+  }
+  return output_size;
+}
+
+template <typename IndexType>
+Result<std::shared_ptr<ArrayData>> GetTakeIndicesImpl(
+    const ArrayData& filter, FilterOptions::NullSelectionBehavior null_selection,
+    MemoryPool* memory_pool) {
+  using T = typename IndexType::c_type;
+  typename TypeTraits<IndexType>::BuilderType builder(memory_pool);
+
+  const uint8_t* filter_data = filter.buffers[1]->data();
+  BitBlockCounter data_counter(filter_data, filter.offset, filter.length);
+
+  // The position relative to the start of the filter
+  T position = 0;
+
+  // The current position taking the filter offset into account
+  int64_t position_with_offset = filter.offset;
+  if (filter.GetNullCount() > 0) {
+    // The filter has nulls, so we scan the validity bitmap and the filter data
+    // bitmap together, branching on the null selection type.
+    const uint8_t* filter_is_valid = filter.buffers[0]->data();
+
+    // To count blocks whether filter_data[i] || !filter_is_valid[i]
+    BinaryBitBlockCounter filter_counter(filter_data, filter.offset, filter_is_valid,
+                                         filter.offset, filter.length);
+    if (null_selection == FilterOptions::DROP) {
+      while (position < filter.length) {
+        BitBlockCount and_block = filter_counter.NextAndWord();
+        RETURN_NOT_OK(builder.Reserve(and_block.popcount));
+        if (and_block.AllSet()) {
+          // All the values are selected and non-null
+          for (int64_t i = 0; i < and_block.length; ++i) {
+            builder.UnsafeAppend(position++);
+          }
+          position_with_offset += and_block.length;
+        } else {
+          // Some of the values are false or null
+          for (int64_t i = 0; i < and_block.length; ++i) {
+            if (BitUtil::GetBit(filter_is_valid, position_with_offset) &&
+                BitUtil::GetBit(filter_data, position_with_offset)) {
+              builder.UnsafeAppend(position);
+            }
+            ++position;
+            ++position_with_offset;
+          }
+        }
+      }
+    } else {
+      BitBlockCounter is_valid_counter(filter_is_valid, filter.offset, filter.length);
+      while (position < filter.length) {
+        // true OR NOT valid
+        BitBlockCount or_not_block = filter_counter.NextOrNotWord();
+        RETURN_NOT_OK(builder.Reserve(or_not_block.popcount));
+
+        // If the values are all valid and the or_not_block is full, then we
+        // can infer that all the values are true and skip the bit checking
+        BitBlockCount is_valid_block = is_valid_counter.NextWord();
+
+        if (or_not_block.AllSet() && is_valid_block.AllSet()) {
+          // All the values are selected and non-null
+          for (int64_t i = 0; i < or_not_block.length; ++i) {
+            builder.UnsafeAppend(position++);
+          }
+          position_with_offset += or_not_block.length;
+        } else {
+          // Some of the values are false or null
+          for (int64_t i = 0; i < or_not_block.length; ++i) {
+            if (BitUtil::GetBit(filter_is_valid, position_with_offset)) {
+              if (BitUtil::GetBit(filter_data, position_with_offset)) {
+                builder.UnsafeAppend(position);
+              }
+            } else {
+              // Null slot, so append a null
+              builder.UnsafeAppendNull();
+            }
+            ++position;
+            ++position_with_offset;
+          }
+        }
+      }
+    }
+  } else {
+    // The filter has no nulls, so we need only look for true values
+    BitBlockCount current_block = data_counter.NextWord();
+    while (position < filter.length) {
+      if (current_block.AllSet()) {
+        int64_t run_length = 0;
+
+        // If we've found a all-true block, then we scan forward until we find
+        // a block that has some false values (or we reach the end
+        while (current_block.length > 0 && current_block.AllSet()) {
+          run_length += current_block.length;
+          current_block = data_counter.NextWord();
+        }
+
+        // Append the consecutive run of indices
+        RETURN_NOT_OK(builder.Reserve(run_length));
+        for (int64_t i = 0; i < run_length; ++i) {
+          builder.UnsafeAppend(position++);
+        }
+        position_with_offset += run_length;
+      } else {
+        // Must do bitchecking on the current block
+        RETURN_NOT_OK(builder.Reserve(current_block.popcount));
+        for (int64_t i = 0; i < current_block.length; ++i) {
+          if (BitUtil::GetBit(filter_data, position_with_offset)) {
+            builder.UnsafeAppend(position);
+          }
+          ++position;
+          ++position_with_offset;
+        }
+        current_block = data_counter.NextWord();
+      }
+    }
+  }
+  std::shared_ptr<ArrayData> result;
+  RETURN_NOT_OK(builder.FinishInternal(&result));
+  return result;
+}
+
+Result<std::shared_ptr<ArrayData>> GetTakeIndices(
+    const ArrayData& filter, FilterOptions::NullSelectionBehavior null_selection,
+    MemoryPool* memory_pool) {
+  DCHECK_EQ(filter.type->id(), Type::BOOL);
+  if (filter.length <= std::numeric_limits<uint16_t>::max()) {
+    return GetTakeIndicesImpl<UInt16Type>(filter, null_selection, memory_pool);
+  } else if (filter.length <= std::numeric_limits<uint32_t>::max()) {
+    return GetTakeIndicesImpl<UInt32Type>(filter, null_selection, memory_pool);
+  } else {
+    // Arrays over 4 billion elements, not especially likely.
+    return Status::NotImplemented(
+        "Filter length exceeds UINT32_MAX, "
+        "consider a different strategy for selecting elements");
+  }
+}
+
+namespace {
+
+template <typename ArrowType>
+struct GetCType {
+  using type = typename ArrowType::c_type;
+};
+
+// We want uint8_t for boolean instead of bool
+template <>
+struct GetCType<BooleanType> {
+  using type = uint8_t;
+};
+
+using FilterState = OptionsWrapper<FilterOptions>;
+using TakeState = OptionsWrapper<TakeOptions>;
+
+Status PreallocateData(KernelContext* ctx, int64_t length, int bit_width, Datum* out) {
+  // Preallocate memory
+  ArrayData* out_arr = out->mutable_array();
+  out_arr->length = length;
+  out_arr->buffers.resize(2);
+
+  ARROW_ASSIGN_OR_RAISE(out_arr->buffers[0], ctx->AllocateBitmap(length));
+  if (bit_width == 1) {
+    ARROW_ASSIGN_OR_RAISE(out_arr->buffers[1], ctx->AllocateBitmap(length));
+  } else {
+    ARROW_ASSIGN_OR_RAISE(out_arr->buffers[1], ctx->Allocate(length * bit_width / 8));
+  }
+  return Status::OK();
+}
+
+// ----------------------------------------------------------------------
+// Implement optimized take for primitive types from boolean to 1/2/4/8-byte
+// C-type based types. Use common implementation for every byte width and only
+// generate code for unsigned integer indices, since after boundschecking to
+// check for negative numbers in the indices we can safely reinterpret_cast
+// signed integers as unsigned.
+
+/// \brief The Take implementation for primitive (fixed-width) types does not
+/// use the logical Arrow type but rather the physical C type. This way we
+/// only generate one take function for each byte width.
+///
+/// This function assumes that the indices have been boundschecked.
+template <typename IndexCType, typename ValueCType>
+struct PrimitiveTakeImpl {
+  static void Exec(const PrimitiveArg& values, const PrimitiveArg& indices,
+                   Datum* out_datum) {
+    auto values_data = reinterpret_cast<const ValueCType*>(values.data);
+    auto values_is_valid = values.is_valid;
+    auto values_offset = values.offset;
+
+    auto indices_data = reinterpret_cast<const IndexCType*>(indices.data);
+    auto indices_is_valid = indices.is_valid;
+    auto indices_offset = indices.offset;
+
+    ArrayData* out_arr = out_datum->mutable_array();
+    auto out = out_arr->GetMutableValues<ValueCType>(1);
+    auto out_is_valid = out_arr->buffers[0]->mutable_data();
+    auto out_offset = out_arr->offset;
+
+    // If either the values or indices have nulls, we preemptively zero out the
+    // out validity bitmap so that we don't have to use ClearBit in each
+    // iteration for nulls.
+    if (values.null_count > 0 || indices.null_count > 0) {
+      BitUtil::SetBitsTo(out_is_valid, out_offset, indices.length, false);
+    }
+
+    OptionalBitBlockCounter indices_bit_counter(indices_is_valid, indices_offset,
+                                                indices.length);
+    int64_t position = 0;
+    int64_t valid_count = 0;
+    while (position < indices.length) {
+      BitBlockCount block = indices_bit_counter.NextBlock();
+      if (values.null_count == 0) {
+        // Values are never null, so things are easier
+        valid_count += block.popcount;
+        if (block.popcount == block.length) {
+          // Fastest path: neither values nor index nulls
+          BitUtil::SetBitsTo(out_is_valid, out_offset + position, block.length, true);
+          for (int64_t i = 0; i < block.length; ++i) {
+            out[position] = values_data[indices_data[position]];
+            ++position;
+          }
+        } else if (block.popcount > 0) {
+          // Slow path: some indices but not all are null
+          for (int64_t i = 0; i < block.length; ++i) {
+            if (BitUtil::GetBit(indices_is_valid, indices_offset + position)) {
+              // index is not null
+              BitUtil::SetBit(out_is_valid, out_offset + position);
+              out[position] = values_data[indices_data[position]];
+            } else {
+              out[position] = ValueCType{};
+            }
+            ++position;
+          }
+        } else {
+          memset(out + position, 0, sizeof(ValueCType) * block.length);
+          position += block.length;
+        }
+      } else {
+        // Values have nulls, so we must do random access into the values bitmap
+        if (block.popcount == block.length) {
+          // Faster path: indices are not null but values may be
+          for (int64_t i = 0; i < block.length; ++i) {
+            if (BitUtil::GetBit(values_is_valid,
+                                values_offset + indices_data[position])) {
+              // value is not null
+              out[position] = values_data[indices_data[position]];
+              BitUtil::SetBit(out_is_valid, out_offset + position);
+              ++valid_count;
+            } else {
+              out[position] = ValueCType{};
+            }
+            ++position;
+          }
+        } else if (block.popcount > 0) {
+          // Slow path: some but not all indices are null. Since we are doing
+          // random access in general we have to check the value nullness one by
+          // one.
+          for (int64_t i = 0; i < block.length; ++i) {
+            if (BitUtil::GetBit(indices_is_valid, indices_offset + position) &&
+                BitUtil::GetBit(values_is_valid,
+                                values_offset + indices_data[position])) {
+              // index is not null && value is not null
+              out[position] = values_data[indices_data[position]];
+              BitUtil::SetBit(out_is_valid, out_offset + position);
+              ++valid_count;
+            } else {
+              out[position] = ValueCType{};
+            }
+            ++position;
+          }
+        } else {
+          memset(out + position, 0, sizeof(ValueCType) * block.length);
+          position += block.length;
+        }
+      }
+    }
+    out_arr->null_count = out_arr->length - valid_count;
+  }
+};
+
+template <typename IndexCType>
+struct BooleanTakeImpl {
+  static void Exec(const PrimitiveArg& values, const PrimitiveArg& indices,
+                   Datum* out_datum) {
+    const uint8_t* values_data = values.data;
+    auto values_is_valid = values.is_valid;
+    auto values_offset = values.offset;
+
+    auto indices_data = reinterpret_cast<const IndexCType*>(indices.data);
+    auto indices_is_valid = indices.is_valid;
+    auto indices_offset = indices.offset;
+
+    ArrayData* out_arr = out_datum->mutable_array();
+    auto out = out_arr->buffers[1]->mutable_data();
+    auto out_is_valid = out_arr->buffers[0]->mutable_data();
+    auto out_offset = out_arr->offset;
+
+    // If either the values or indices have nulls, we preemptively zero out the
+    // out validity bitmap so that we don't have to use ClearBit in each
+    // iteration for nulls.
+    if (values.null_count > 0 || indices.null_count > 0) {
+      BitUtil::SetBitsTo(out_is_valid, out_offset, indices.length, false);
+    }
+    // Avoid uninitialized data in values array
+    BitUtil::SetBitsTo(out, out_offset, indices.length, false);
+
+    auto PlaceDataBit = [&](int64_t loc, IndexCType index) {
+      BitUtil::SetBitTo(out, out_offset + loc,
+                        BitUtil::GetBit(values_data, values_offset + index));
+    };
+
+    OptionalBitBlockCounter indices_bit_counter(indices_is_valid, indices_offset,
+                                                indices.length);
+    int64_t position = 0;
+    int64_t valid_count = 0;
+    while (position < indices.length) {
+      BitBlockCount block = indices_bit_counter.NextBlock();
+      if (values.null_count == 0) {
+        // Values are never null, so things are easier
+        valid_count += block.popcount;
+        if (block.popcount == block.length) {
+          // Fastest path: neither values nor index nulls
+          BitUtil::SetBitsTo(out_is_valid, out_offset + position, block.length, true);
+          for (int64_t i = 0; i < block.length; ++i) {
+            PlaceDataBit(position, indices_data[position]);
+            ++position;
+          }
+        } else if (block.popcount > 0) {
+          // Slow path: some but not all indices are null
+          for (int64_t i = 0; i < block.length; ++i) {
+            if (BitUtil::GetBit(indices_is_valid, indices_offset + position)) {
+              // index is not null
+              BitUtil::SetBit(out_is_valid, out_offset + position);
+              PlaceDataBit(position, indices_data[position]);
+            }
+            ++position;
+          }
+        } else {
+          position += block.length;
+        }
+      } else {
+        // Values have nulls, so we must do random access into the values bitmap
+        if (block.popcount == block.length) {
+          // Faster path: indices are not null but values may be
+          for (int64_t i = 0; i < block.length; ++i) {
+            if (BitUtil::GetBit(values_is_valid,
+                                values_offset + indices_data[position])) {
+              // value is not null
+              BitUtil::SetBit(out_is_valid, out_offset + position);
+              PlaceDataBit(position, indices_data[position]);
+              ++valid_count;
+            }
+            ++position;
+          }
+        } else if (block.popcount > 0) {
+          // Slow path: some but not all indices are null. Since we are doing
+          // random access in general we have to check the value nullness one by
+          // one.
+          for (int64_t i = 0; i < block.length; ++i) {
+            if (BitUtil::GetBit(indices_is_valid, indices_offset + position)) {
+              // index is not null
+              if (BitUtil::GetBit(values_is_valid,
+                                  values_offset + indices_data[position])) {
+                // value is not null
+                PlaceDataBit(position, indices_data[position]);
+                BitUtil::SetBit(out_is_valid, out_offset + position);
+                ++valid_count;
+              }
+            }
+            ++position;
+          }
+        } else {
+          position += block.length;
+        }
+      }
+    }
+    out_arr->null_count = out_arr->length - valid_count;
+  }
+};
+
+template <template <typename...> class TakeImpl, typename... Args>
+void TakeIndexDispatch(const PrimitiveArg& values, const PrimitiveArg& indices,
+                       Datum* out) {
+  // With the simplifying assumption that boundschecking has taken place
+  // already at a higher level, we can now assume that the index values are all
+  // non-negative. Thus, we can interpret signed integers as unsigned and avoid
+  // having to generate double the amount of binary code to handle each integer
+  // width.
+  switch (indices.bit_width) {
+    case 8:
+      return TakeImpl<uint8_t, Args...>::Exec(values, indices, out);
+    case 16:
+      return TakeImpl<uint16_t, Args...>::Exec(values, indices, out);
+    case 32:
+      return TakeImpl<uint32_t, Args...>::Exec(values, indices, out);
+    case 64:
+      return TakeImpl<uint64_t, Args...>::Exec(values, indices, out);
+    default:
+      DCHECK(false) << "Invalid indices byte width";
+      break;
+  }
+}
+
+void PrimitiveTake(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+  const auto& state = checked_cast<const TakeState&>(*ctx->state());
+  if (state.options.boundscheck) {
+    KERNEL_RETURN_IF_ERROR(ctx, IndexBoundsCheck(*batch[1].array(), batch[0].length()));
+  }
+
+  PrimitiveArg values = GetPrimitiveArg(*batch[0].array());
+  PrimitiveArg indices = GetPrimitiveArg(*batch[1].array());
+  KERNEL_RETURN_IF_ERROR(ctx,
+                         PreallocateData(ctx, indices.length, values.bit_width, out));
+  switch (values.bit_width) {
+    case 1:
+      return TakeIndexDispatch<BooleanTakeImpl>(values, indices, out);
+    case 8:
+      return TakeIndexDispatch<PrimitiveTakeImpl, int8_t>(values, indices, out);
+    case 16:
+      return TakeIndexDispatch<PrimitiveTakeImpl, int16_t>(values, indices, out);
+    case 32:
+      return TakeIndexDispatch<PrimitiveTakeImpl, int32_t>(values, indices, out);
+    case 64:
+      return TakeIndexDispatch<PrimitiveTakeImpl, int64_t>(values, indices, out);
+    default:
+      DCHECK(false) << "Invalid values byte width";
+      break;
+  }
+}
+
+// ----------------------------------------------------------------------
+// Optimized and streamlined filter for primitive types
+
+// Use either BitBlockCounter or BinaryBitBlockCounter to quickly scan filter a
+// word at a time for the DROP selection type.
+class DropNullCounter {
+ public:
+  // validity bitmap may be null
+  DropNullCounter(const uint8_t* validity, const uint8_t* data, int64_t offset,
+                  int64_t length)
+      : data_counter_(data, offset, length),
+        data_and_validity_counter_(data, offset, validity, offset, length),
+        has_validity_(validity != nullptr) {}
+
+  BitBlockCount NextBlock() {
+    if (has_validity_) {
+      // filter is true AND not null
+      return data_and_validity_counter_.NextAndWord();
+    } else {
+      return data_counter_.NextWord();
+    }
+  }
+
+ private:
+  // For when just data is present, but no validity bitmap
+  BitBlockCounter data_counter_;
+
+  // For when both validity bitmap and data are present
+  BinaryBitBlockCounter data_and_validity_counter_;
+  bool has_validity_;

Review comment:
       Can probably make this `const`.

##########
File path: cpp/src/arrow/compute/kernels/vector_selection.cc
##########
@@ -0,0 +1,1816 @@
+// 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 <algorithm>
+#include <cstring>
+#include <limits>
+
+#include "arrow/array/array_base.h"
+#include "arrow/array/array_binary.h"
+#include "arrow/array/array_dict.h"
+#include "arrow/array/array_nested.h"
+#include "arrow/array/builder_primitive.h"
+#include "arrow/array/concatenate.h"
+#include "arrow/buffer_builder.h"
+#include "arrow/compute/api_vector.h"
+#include "arrow/compute/kernels/common.h"
+#include "arrow/compute/kernels/util_internal.h"
+#include "arrow/extension_type.h"
+#include "arrow/record_batch.h"
+#include "arrow/result.h"
+#include "arrow/util/bit_block_counter.h"
+#include "arrow/util/bit_util.h"
+#include "arrow/util/bitmap_ops.h"
+#include "arrow/util/bitmap_reader.h"
+#include "arrow/util/int_util.h"
+
+namespace arrow {
+
+using internal::BinaryBitBlockCounter;
+using internal::BitBlockCount;
+using internal::BitBlockCounter;
+using internal::BitmapReader;
+using internal::CopyBitmap;
+using internal::GetArrayView;
+using internal::IndexBoundsCheck;
+using internal::OptionalBitBlockCounter;
+using internal::OptionalBitIndexer;
+
+namespace compute {
+namespace internal {
+
+int64_t GetFilterOutputSize(const ArrayData& filter,
+                            FilterOptions::NullSelectionBehavior null_selection) {
+  int64_t output_size = 0;
+  int64_t position = 0;
+  if (filter.GetNullCount() > 0) {
+    const uint8_t* filter_is_valid = filter.buffers[0]->data();
+    BinaryBitBlockCounter bit_counter(filter.buffers[1]->data(), filter.offset,
+                                      filter_is_valid, filter.offset, filter.length);
+    if (null_selection == FilterOptions::EMIT_NULL) {
+      while (position < filter.length) {
+        BitBlockCount block = bit_counter.NextOrNotWord();
+        output_size += block.popcount;
+        position += block.length;
+      }
+    } else {
+      while (position < filter.length) {
+        BitBlockCount block = bit_counter.NextAndWord();
+        output_size += block.popcount;
+        position += block.length;
+      }
+    }
+  } else {
+    // The filter has no nulls, so we plow through its data as fast as
+    // possible.
+    BitBlockCounter bit_counter(filter.buffers[1]->data(), filter.offset, filter.length);
+    while (position < filter.length) {
+      BitBlockCount block = bit_counter.NextFourWords();
+      output_size += block.popcount;
+      position += block.length;
+    }
+  }
+  return output_size;
+}
+
+template <typename IndexType>
+Result<std::shared_ptr<ArrayData>> GetTakeIndicesImpl(
+    const ArrayData& filter, FilterOptions::NullSelectionBehavior null_selection,
+    MemoryPool* memory_pool) {
+  using T = typename IndexType::c_type;
+  typename TypeTraits<IndexType>::BuilderType builder(memory_pool);
+
+  const uint8_t* filter_data = filter.buffers[1]->data();
+  BitBlockCounter data_counter(filter_data, filter.offset, filter.length);
+
+  // The position relative to the start of the filter
+  T position = 0;
+
+  // The current position taking the filter offset into account
+  int64_t position_with_offset = filter.offset;
+  if (filter.GetNullCount() > 0) {
+    // The filter has nulls, so we scan the validity bitmap and the filter data
+    // bitmap together, branching on the null selection type.
+    const uint8_t* filter_is_valid = filter.buffers[0]->data();
+
+    // To count blocks whether filter_data[i] || !filter_is_valid[i]
+    BinaryBitBlockCounter filter_counter(filter_data, filter.offset, filter_is_valid,
+                                         filter.offset, filter.length);
+    if (null_selection == FilterOptions::DROP) {
+      while (position < filter.length) {
+        BitBlockCount and_block = filter_counter.NextAndWord();
+        RETURN_NOT_OK(builder.Reserve(and_block.popcount));
+        if (and_block.AllSet()) {
+          // All the values are selected and non-null
+          for (int64_t i = 0; i < and_block.length; ++i) {
+            builder.UnsafeAppend(position++);
+          }
+          position_with_offset += and_block.length;
+        } else {
+          // Some of the values are false or null
+          for (int64_t i = 0; i < and_block.length; ++i) {
+            if (BitUtil::GetBit(filter_is_valid, position_with_offset) &&
+                BitUtil::GetBit(filter_data, position_with_offset)) {
+              builder.UnsafeAppend(position);
+            }
+            ++position;
+            ++position_with_offset;
+          }
+        }
+      }
+    } else {
+      BitBlockCounter is_valid_counter(filter_is_valid, filter.offset, filter.length);
+      while (position < filter.length) {
+        // true OR NOT valid
+        BitBlockCount or_not_block = filter_counter.NextOrNotWord();
+        RETURN_NOT_OK(builder.Reserve(or_not_block.popcount));
+
+        // If the values are all valid and the or_not_block is full, then we
+        // can infer that all the values are true and skip the bit checking
+        BitBlockCount is_valid_block = is_valid_counter.NextWord();
+
+        if (or_not_block.AllSet() && is_valid_block.AllSet()) {
+          // All the values are selected and non-null
+          for (int64_t i = 0; i < or_not_block.length; ++i) {
+            builder.UnsafeAppend(position++);
+          }
+          position_with_offset += or_not_block.length;
+        } else {
+          // Some of the values are false or null
+          for (int64_t i = 0; i < or_not_block.length; ++i) {
+            if (BitUtil::GetBit(filter_is_valid, position_with_offset)) {
+              if (BitUtil::GetBit(filter_data, position_with_offset)) {
+                builder.UnsafeAppend(position);
+              }
+            } else {
+              // Null slot, so append a null
+              builder.UnsafeAppendNull();
+            }
+            ++position;
+            ++position_with_offset;
+          }
+        }
+      }
+    }
+  } else {
+    // The filter has no nulls, so we need only look for true values
+    BitBlockCount current_block = data_counter.NextWord();
+    while (position < filter.length) {
+      if (current_block.AllSet()) {
+        int64_t run_length = 0;
+
+        // If we've found a all-true block, then we scan forward until we find
+        // a block that has some false values (or we reach the end
+        while (current_block.length > 0 && current_block.AllSet()) {
+          run_length += current_block.length;
+          current_block = data_counter.NextWord();
+        }
+
+        // Append the consecutive run of indices
+        RETURN_NOT_OK(builder.Reserve(run_length));
+        for (int64_t i = 0; i < run_length; ++i) {
+          builder.UnsafeAppend(position++);
+        }
+        position_with_offset += run_length;
+      } else {
+        // Must do bitchecking on the current block
+        RETURN_NOT_OK(builder.Reserve(current_block.popcount));
+        for (int64_t i = 0; i < current_block.length; ++i) {
+          if (BitUtil::GetBit(filter_data, position_with_offset)) {
+            builder.UnsafeAppend(position);
+          }
+          ++position;
+          ++position_with_offset;
+        }
+        current_block = data_counter.NextWord();
+      }
+    }
+  }
+  std::shared_ptr<ArrayData> result;
+  RETURN_NOT_OK(builder.FinishInternal(&result));
+  return result;
+}
+
+Result<std::shared_ptr<ArrayData>> GetTakeIndices(
+    const ArrayData& filter, FilterOptions::NullSelectionBehavior null_selection,
+    MemoryPool* memory_pool) {
+  DCHECK_EQ(filter.type->id(), Type::BOOL);
+  if (filter.length <= std::numeric_limits<uint16_t>::max()) {
+    return GetTakeIndicesImpl<UInt16Type>(filter, null_selection, memory_pool);
+  } else if (filter.length <= std::numeric_limits<uint32_t>::max()) {
+    return GetTakeIndicesImpl<UInt32Type>(filter, null_selection, memory_pool);
+  } else {
+    // Arrays over 4 billion elements, not especially likely.
+    return Status::NotImplemented(
+        "Filter length exceeds UINT32_MAX, "
+        "consider a different strategy for selecting elements");
+  }
+}
+
+namespace {
+
+template <typename ArrowType>
+struct GetCType {
+  using type = typename ArrowType::c_type;
+};
+
+// We want uint8_t for boolean instead of bool
+template <>
+struct GetCType<BooleanType> {
+  using type = uint8_t;
+};
+
+using FilterState = OptionsWrapper<FilterOptions>;
+using TakeState = OptionsWrapper<TakeOptions>;
+
+Status PreallocateData(KernelContext* ctx, int64_t length, int bit_width, Datum* out) {
+  // Preallocate memory
+  ArrayData* out_arr = out->mutable_array();
+  out_arr->length = length;
+  out_arr->buffers.resize(2);
+
+  ARROW_ASSIGN_OR_RAISE(out_arr->buffers[0], ctx->AllocateBitmap(length));
+  if (bit_width == 1) {
+    ARROW_ASSIGN_OR_RAISE(out_arr->buffers[1], ctx->AllocateBitmap(length));
+  } else {
+    ARROW_ASSIGN_OR_RAISE(out_arr->buffers[1], ctx->Allocate(length * bit_width / 8));
+  }
+  return Status::OK();
+}
+
+// ----------------------------------------------------------------------
+// Implement optimized take for primitive types from boolean to 1/2/4/8-byte
+// C-type based types. Use common implementation for every byte width and only
+// generate code for unsigned integer indices, since after boundschecking to
+// check for negative numbers in the indices we can safely reinterpret_cast
+// signed integers as unsigned.
+
+/// \brief The Take implementation for primitive (fixed-width) types does not
+/// use the logical Arrow type but rather the physical C type. This way we
+/// only generate one take function for each byte width.
+///
+/// This function assumes that the indices have been boundschecked.
+template <typename IndexCType, typename ValueCType>
+struct PrimitiveTakeImpl {
+  static void Exec(const PrimitiveArg& values, const PrimitiveArg& indices,
+                   Datum* out_datum) {
+    auto values_data = reinterpret_cast<const ValueCType*>(values.data);
+    auto values_is_valid = values.is_valid;
+    auto values_offset = values.offset;
+
+    auto indices_data = reinterpret_cast<const IndexCType*>(indices.data);
+    auto indices_is_valid = indices.is_valid;
+    auto indices_offset = indices.offset;
+
+    ArrayData* out_arr = out_datum->mutable_array();
+    auto out = out_arr->GetMutableValues<ValueCType>(1);
+    auto out_is_valid = out_arr->buffers[0]->mutable_data();
+    auto out_offset = out_arr->offset;
+
+    // If either the values or indices have nulls, we preemptively zero out the
+    // out validity bitmap so that we don't have to use ClearBit in each
+    // iteration for nulls.
+    if (values.null_count > 0 || indices.null_count > 0) {
+      BitUtil::SetBitsTo(out_is_valid, out_offset, indices.length, false);
+    }
+
+    OptionalBitBlockCounter indices_bit_counter(indices_is_valid, indices_offset,
+                                                indices.length);
+    int64_t position = 0;
+    int64_t valid_count = 0;
+    while (position < indices.length) {
+      BitBlockCount block = indices_bit_counter.NextBlock();
+      if (values.null_count == 0) {
+        // Values are never null, so things are easier
+        valid_count += block.popcount;
+        if (block.popcount == block.length) {
+          // Fastest path: neither values nor index nulls
+          BitUtil::SetBitsTo(out_is_valid, out_offset + position, block.length, true);
+          for (int64_t i = 0; i < block.length; ++i) {
+            out[position] = values_data[indices_data[position]];
+            ++position;
+          }
+        } else if (block.popcount > 0) {
+          // Slow path: some indices but not all are null
+          for (int64_t i = 0; i < block.length; ++i) {
+            if (BitUtil::GetBit(indices_is_valid, indices_offset + position)) {
+              // index is not null
+              BitUtil::SetBit(out_is_valid, out_offset + position);
+              out[position] = values_data[indices_data[position]];
+            } else {
+              out[position] = ValueCType{};
+            }
+            ++position;
+          }
+        } else {
+          memset(out + position, 0, sizeof(ValueCType) * block.length);
+          position += block.length;
+        }
+      } else {
+        // Values have nulls, so we must do random access into the values bitmap
+        if (block.popcount == block.length) {
+          // Faster path: indices are not null but values may be
+          for (int64_t i = 0; i < block.length; ++i) {
+            if (BitUtil::GetBit(values_is_valid,
+                                values_offset + indices_data[position])) {
+              // value is not null
+              out[position] = values_data[indices_data[position]];
+              BitUtil::SetBit(out_is_valid, out_offset + position);
+              ++valid_count;
+            } else {
+              out[position] = ValueCType{};
+            }
+            ++position;
+          }
+        } else if (block.popcount > 0) {
+          // Slow path: some but not all indices are null. Since we are doing
+          // random access in general we have to check the value nullness one by
+          // one.
+          for (int64_t i = 0; i < block.length; ++i) {
+            if (BitUtil::GetBit(indices_is_valid, indices_offset + position) &&
+                BitUtil::GetBit(values_is_valid,
+                                values_offset + indices_data[position])) {
+              // index is not null && value is not null
+              out[position] = values_data[indices_data[position]];
+              BitUtil::SetBit(out_is_valid, out_offset + position);
+              ++valid_count;
+            } else {
+              out[position] = ValueCType{};
+            }
+            ++position;
+          }
+        } else {
+          memset(out + position, 0, sizeof(ValueCType) * block.length);
+          position += block.length;
+        }
+      }
+    }
+    out_arr->null_count = out_arr->length - valid_count;
+  }
+};
+
+template <typename IndexCType>
+struct BooleanTakeImpl {
+  static void Exec(const PrimitiveArg& values, const PrimitiveArg& indices,
+                   Datum* out_datum) {
+    const uint8_t* values_data = values.data;
+    auto values_is_valid = values.is_valid;
+    auto values_offset = values.offset;
+
+    auto indices_data = reinterpret_cast<const IndexCType*>(indices.data);
+    auto indices_is_valid = indices.is_valid;
+    auto indices_offset = indices.offset;
+
+    ArrayData* out_arr = out_datum->mutable_array();
+    auto out = out_arr->buffers[1]->mutable_data();
+    auto out_is_valid = out_arr->buffers[0]->mutable_data();
+    auto out_offset = out_arr->offset;
+
+    // If either the values or indices have nulls, we preemptively zero out the
+    // out validity bitmap so that we don't have to use ClearBit in each
+    // iteration for nulls.
+    if (values.null_count > 0 || indices.null_count > 0) {
+      BitUtil::SetBitsTo(out_is_valid, out_offset, indices.length, false);
+    }
+    // Avoid uninitialized data in values array
+    BitUtil::SetBitsTo(out, out_offset, indices.length, false);
+
+    auto PlaceDataBit = [&](int64_t loc, IndexCType index) {
+      BitUtil::SetBitTo(out, out_offset + loc,
+                        BitUtil::GetBit(values_data, values_offset + index));
+    };
+
+    OptionalBitBlockCounter indices_bit_counter(indices_is_valid, indices_offset,
+                                                indices.length);
+    int64_t position = 0;
+    int64_t valid_count = 0;
+    while (position < indices.length) {
+      BitBlockCount block = indices_bit_counter.NextBlock();
+      if (values.null_count == 0) {
+        // Values are never null, so things are easier
+        valid_count += block.popcount;
+        if (block.popcount == block.length) {
+          // Fastest path: neither values nor index nulls
+          BitUtil::SetBitsTo(out_is_valid, out_offset + position, block.length, true);
+          for (int64_t i = 0; i < block.length; ++i) {
+            PlaceDataBit(position, indices_data[position]);
+            ++position;
+          }
+        } else if (block.popcount > 0) {
+          // Slow path: some but not all indices are null
+          for (int64_t i = 0; i < block.length; ++i) {
+            if (BitUtil::GetBit(indices_is_valid, indices_offset + position)) {
+              // index is not null
+              BitUtil::SetBit(out_is_valid, out_offset + position);
+              PlaceDataBit(position, indices_data[position]);
+            }
+            ++position;
+          }
+        } else {
+          position += block.length;
+        }
+      } else {
+        // Values have nulls, so we must do random access into the values bitmap
+        if (block.popcount == block.length) {
+          // Faster path: indices are not null but values may be
+          for (int64_t i = 0; i < block.length; ++i) {
+            if (BitUtil::GetBit(values_is_valid,
+                                values_offset + indices_data[position])) {
+              // value is not null
+              BitUtil::SetBit(out_is_valid, out_offset + position);
+              PlaceDataBit(position, indices_data[position]);
+              ++valid_count;
+            }
+            ++position;
+          }
+        } else if (block.popcount > 0) {
+          // Slow path: some but not all indices are null. Since we are doing
+          // random access in general we have to check the value nullness one by
+          // one.
+          for (int64_t i = 0; i < block.length; ++i) {
+            if (BitUtil::GetBit(indices_is_valid, indices_offset + position)) {
+              // index is not null
+              if (BitUtil::GetBit(values_is_valid,
+                                  values_offset + indices_data[position])) {
+                // value is not null
+                PlaceDataBit(position, indices_data[position]);
+                BitUtil::SetBit(out_is_valid, out_offset + position);
+                ++valid_count;
+              }
+            }
+            ++position;
+          }
+        } else {
+          position += block.length;
+        }
+      }
+    }
+    out_arr->null_count = out_arr->length - valid_count;
+  }
+};
+
+template <template <typename...> class TakeImpl, typename... Args>
+void TakeIndexDispatch(const PrimitiveArg& values, const PrimitiveArg& indices,
+                       Datum* out) {
+  // With the simplifying assumption that boundschecking has taken place
+  // already at a higher level, we can now assume that the index values are all
+  // non-negative. Thus, we can interpret signed integers as unsigned and avoid
+  // having to generate double the amount of binary code to handle each integer
+  // width.
+  switch (indices.bit_width) {
+    case 8:
+      return TakeImpl<uint8_t, Args...>::Exec(values, indices, out);
+    case 16:
+      return TakeImpl<uint16_t, Args...>::Exec(values, indices, out);
+    case 32:
+      return TakeImpl<uint32_t, Args...>::Exec(values, indices, out);
+    case 64:
+      return TakeImpl<uint64_t, Args...>::Exec(values, indices, out);
+    default:
+      DCHECK(false) << "Invalid indices byte width";
+      break;
+  }
+}
+
+void PrimitiveTake(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+  const auto& state = checked_cast<const TakeState&>(*ctx->state());
+  if (state.options.boundscheck) {
+    KERNEL_RETURN_IF_ERROR(ctx, IndexBoundsCheck(*batch[1].array(), batch[0].length()));
+  }
+
+  PrimitiveArg values = GetPrimitiveArg(*batch[0].array());
+  PrimitiveArg indices = GetPrimitiveArg(*batch[1].array());
+  KERNEL_RETURN_IF_ERROR(ctx,
+                         PreallocateData(ctx, indices.length, values.bit_width, out));
+  switch (values.bit_width) {
+    case 1:
+      return TakeIndexDispatch<BooleanTakeImpl>(values, indices, out);
+    case 8:
+      return TakeIndexDispatch<PrimitiveTakeImpl, int8_t>(values, indices, out);
+    case 16:
+      return TakeIndexDispatch<PrimitiveTakeImpl, int16_t>(values, indices, out);
+    case 32:
+      return TakeIndexDispatch<PrimitiveTakeImpl, int32_t>(values, indices, out);
+    case 64:
+      return TakeIndexDispatch<PrimitiveTakeImpl, int64_t>(values, indices, out);
+    default:
+      DCHECK(false) << "Invalid values byte width";
+      break;
+  }
+}
+
+// ----------------------------------------------------------------------
+// Optimized and streamlined filter for primitive types
+
+// Use either BitBlockCounter or BinaryBitBlockCounter to quickly scan filter a
+// word at a time for the DROP selection type.
+class DropNullCounter {
+ public:
+  // validity bitmap may be null
+  DropNullCounter(const uint8_t* validity, const uint8_t* data, int64_t offset,
+                  int64_t length)
+      : data_counter_(data, offset, length),
+        data_and_validity_counter_(data, offset, validity, offset, length),
+        has_validity_(validity != nullptr) {}
+
+  BitBlockCount NextBlock() {
+    if (has_validity_) {
+      // filter is true AND not null
+      return data_and_validity_counter_.NextAndWord();
+    } else {
+      return data_counter_.NextWord();
+    }
+  }
+
+ private:
+  // For when just data is present, but no validity bitmap
+  BitBlockCounter data_counter_;
+
+  // For when both validity bitmap and data are present
+  BinaryBitBlockCounter data_and_validity_counter_;
+  bool has_validity_;
+};
+
+/// \brief The Filter implementation for primitive (fixed-width) types does not
+/// use the logical Arrow type but rather then physical C type. This way we
+/// only generate one take function for each byte width. We use the same
+/// implementation here for boolean and fixed-byte-size inputs with some
+/// template specialization.
+template <typename ArrowType>
+class PrimitiveFilterImpl {
+ public:
+  using T = typename GetCType<ArrowType>::type;
+
+  PrimitiveFilterImpl(const PrimitiveArg& values, const PrimitiveArg& filter,
+                      FilterOptions::NullSelectionBehavior null_selection,
+                      Datum* out_datum)
+      : values_is_valid_(values.is_valid),
+        values_data_(reinterpret_cast<const T*>(values.data)),
+        values_null_count_(values.null_count),
+        values_offset_(values.offset),
+        values_length_(values.length),
+        filter_is_valid_(filter.is_valid),
+        filter_data_(filter.data),
+        filter_null_count_(filter.null_count),
+        filter_offset_(filter.offset),
+        null_selection_(null_selection) {
+    ArrayData* out_arr = out_datum->mutable_array();
+    out_is_valid_ = out_arr->buffers[0]->mutable_data();
+    out_data_ = reinterpret_cast<T*>(out_arr->buffers[1]->mutable_data());
+    out_offset_ = out_arr->offset;
+    out_length_ = out_arr->length;
+    out_position_ = 0;
+  }
+
+  void ExecNonNull() {
+    // The result is all not-null
+    BitUtil::SetBitsTo(out_is_valid_, out_offset_ + out_position_, out_length_, true);

Review comment:
       Shouldn't we just drop the null bitmap in this case? e.g. store `out_null_count_ = 0`.

##########
File path: cpp/src/arrow/testing/random.cc
##########
@@ -84,7 +84,7 @@ std::shared_ptr<Array> RandomArrayGenerator::Boolean(int64_t size, double probab
 
   BufferVector buffers{2};
   // Need 2 distinct generators such that probabilities are not shared.
-  GenOpt value_gen(seed(), 0, 1, probability);
+  GenOpt value_gen(seed(), 0, 1, 1 - probability);

Review comment:
       Need a comment at least.

##########
File path: cpp/src/arrow/compute/kernels/vector_selection.cc
##########
@@ -0,0 +1,1816 @@
+// 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 <algorithm>
+#include <cstring>
+#include <limits>
+
+#include "arrow/array/array_base.h"
+#include "arrow/array/array_binary.h"
+#include "arrow/array/array_dict.h"
+#include "arrow/array/array_nested.h"
+#include "arrow/array/builder_primitive.h"
+#include "arrow/array/concatenate.h"
+#include "arrow/buffer_builder.h"
+#include "arrow/compute/api_vector.h"
+#include "arrow/compute/kernels/common.h"
+#include "arrow/compute/kernels/util_internal.h"
+#include "arrow/extension_type.h"
+#include "arrow/record_batch.h"
+#include "arrow/result.h"
+#include "arrow/util/bit_block_counter.h"
+#include "arrow/util/bit_util.h"
+#include "arrow/util/bitmap_ops.h"
+#include "arrow/util/bitmap_reader.h"
+#include "arrow/util/int_util.h"
+
+namespace arrow {
+
+using internal::BinaryBitBlockCounter;
+using internal::BitBlockCount;
+using internal::BitBlockCounter;
+using internal::BitmapReader;
+using internal::CopyBitmap;
+using internal::GetArrayView;
+using internal::IndexBoundsCheck;
+using internal::OptionalBitBlockCounter;
+using internal::OptionalBitIndexer;
+
+namespace compute {
+namespace internal {
+
+int64_t GetFilterOutputSize(const ArrayData& filter,
+                            FilterOptions::NullSelectionBehavior null_selection) {
+  int64_t output_size = 0;
+  int64_t position = 0;
+  if (filter.GetNullCount() > 0) {
+    const uint8_t* filter_is_valid = filter.buffers[0]->data();
+    BinaryBitBlockCounter bit_counter(filter.buffers[1]->data(), filter.offset,

Review comment:
       You can probably use `Bitmap::VisitWords` instead.

##########
File path: cpp/src/arrow/compute/kernels/vector_selection.cc
##########
@@ -0,0 +1,1816 @@
+// 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 <algorithm>
+#include <cstring>
+#include <limits>
+
+#include "arrow/array/array_base.h"
+#include "arrow/array/array_binary.h"
+#include "arrow/array/array_dict.h"
+#include "arrow/array/array_nested.h"
+#include "arrow/array/builder_primitive.h"
+#include "arrow/array/concatenate.h"
+#include "arrow/buffer_builder.h"
+#include "arrow/compute/api_vector.h"
+#include "arrow/compute/kernels/common.h"
+#include "arrow/compute/kernels/util_internal.h"
+#include "arrow/extension_type.h"
+#include "arrow/record_batch.h"
+#include "arrow/result.h"
+#include "arrow/util/bit_block_counter.h"
+#include "arrow/util/bit_util.h"
+#include "arrow/util/bitmap_ops.h"
+#include "arrow/util/bitmap_reader.h"
+#include "arrow/util/int_util.h"
+
+namespace arrow {
+
+using internal::BinaryBitBlockCounter;
+using internal::BitBlockCount;
+using internal::BitBlockCounter;
+using internal::BitmapReader;
+using internal::CopyBitmap;
+using internal::GetArrayView;
+using internal::IndexBoundsCheck;
+using internal::OptionalBitBlockCounter;
+using internal::OptionalBitIndexer;
+
+namespace compute {
+namespace internal {
+
+int64_t GetFilterOutputSize(const ArrayData& filter,
+                            FilterOptions::NullSelectionBehavior null_selection) {
+  int64_t output_size = 0;
+  int64_t position = 0;
+  if (filter.GetNullCount() > 0) {
+    const uint8_t* filter_is_valid = filter.buffers[0]->data();
+    BinaryBitBlockCounter bit_counter(filter.buffers[1]->data(), filter.offset,
+                                      filter_is_valid, filter.offset, filter.length);
+    if (null_selection == FilterOptions::EMIT_NULL) {
+      while (position < filter.length) {
+        BitBlockCount block = bit_counter.NextOrNotWord();
+        output_size += block.popcount;
+        position += block.length;
+      }
+    } else {
+      while (position < filter.length) {
+        BitBlockCount block = bit_counter.NextAndWord();
+        output_size += block.popcount;
+        position += block.length;
+      }
+    }
+  } else {
+    // The filter has no nulls, so we plow through its data as fast as
+    // possible.
+    BitBlockCounter bit_counter(filter.buffers[1]->data(), filter.offset, filter.length);
+    while (position < filter.length) {
+      BitBlockCount block = bit_counter.NextFourWords();
+      output_size += block.popcount;
+      position += block.length;
+    }
+  }
+  return output_size;
+}
+
+template <typename IndexType>
+Result<std::shared_ptr<ArrayData>> GetTakeIndicesImpl(
+    const ArrayData& filter, FilterOptions::NullSelectionBehavior null_selection,
+    MemoryPool* memory_pool) {
+  using T = typename IndexType::c_type;
+  typename TypeTraits<IndexType>::BuilderType builder(memory_pool);
+
+  const uint8_t* filter_data = filter.buffers[1]->data();
+  BitBlockCounter data_counter(filter_data, filter.offset, filter.length);
+
+  // The position relative to the start of the filter
+  T position = 0;
+
+  // The current position taking the filter offset into account
+  int64_t position_with_offset = filter.offset;
+  if (filter.GetNullCount() > 0) {
+    // The filter has nulls, so we scan the validity bitmap and the filter data
+    // bitmap together, branching on the null selection type.
+    const uint8_t* filter_is_valid = filter.buffers[0]->data();
+
+    // To count blocks whether filter_data[i] || !filter_is_valid[i]
+    BinaryBitBlockCounter filter_counter(filter_data, filter.offset, filter_is_valid,
+                                         filter.offset, filter.length);
+    if (null_selection == FilterOptions::DROP) {
+      while (position < filter.length) {
+        BitBlockCount and_block = filter_counter.NextAndWord();
+        RETURN_NOT_OK(builder.Reserve(and_block.popcount));
+        if (and_block.AllSet()) {
+          // All the values are selected and non-null
+          for (int64_t i = 0; i < and_block.length; ++i) {
+            builder.UnsafeAppend(position++);
+          }
+          position_with_offset += and_block.length;
+        } else {
+          // Some of the values are false or null
+          for (int64_t i = 0; i < and_block.length; ++i) {
+            if (BitUtil::GetBit(filter_is_valid, position_with_offset) &&
+                BitUtil::GetBit(filter_data, position_with_offset)) {
+              builder.UnsafeAppend(position);
+            }
+            ++position;
+            ++position_with_offset;
+          }
+        }
+      }
+    } else {
+      BitBlockCounter is_valid_counter(filter_is_valid, filter.offset, filter.length);
+      while (position < filter.length) {
+        // true OR NOT valid
+        BitBlockCount or_not_block = filter_counter.NextOrNotWord();
+        RETURN_NOT_OK(builder.Reserve(or_not_block.popcount));
+
+        // If the values are all valid and the or_not_block is full, then we
+        // can infer that all the values are true and skip the bit checking
+        BitBlockCount is_valid_block = is_valid_counter.NextWord();
+
+        if (or_not_block.AllSet() && is_valid_block.AllSet()) {
+          // All the values are selected and non-null
+          for (int64_t i = 0; i < or_not_block.length; ++i) {
+            builder.UnsafeAppend(position++);
+          }
+          position_with_offset += or_not_block.length;
+        } else {
+          // Some of the values are false or null
+          for (int64_t i = 0; i < or_not_block.length; ++i) {
+            if (BitUtil::GetBit(filter_is_valid, position_with_offset)) {
+              if (BitUtil::GetBit(filter_data, position_with_offset)) {
+                builder.UnsafeAppend(position);
+              }
+            } else {
+              // Null slot, so append a null
+              builder.UnsafeAppendNull();
+            }
+            ++position;
+            ++position_with_offset;
+          }
+        }
+      }
+    }
+  } else {
+    // The filter has no nulls, so we need only look for true values
+    BitBlockCount current_block = data_counter.NextWord();
+    while (position < filter.length) {
+      if (current_block.AllSet()) {
+        int64_t run_length = 0;
+
+        // If we've found a all-true block, then we scan forward until we find
+        // a block that has some false values (or we reach the end
+        while (current_block.length > 0 && current_block.AllSet()) {
+          run_length += current_block.length;
+          current_block = data_counter.NextWord();
+        }
+
+        // Append the consecutive run of indices
+        RETURN_NOT_OK(builder.Reserve(run_length));
+        for (int64_t i = 0; i < run_length; ++i) {
+          builder.UnsafeAppend(position++);
+        }
+        position_with_offset += run_length;
+      } else {
+        // Must do bitchecking on the current block

Review comment:
       Can't you short-circuit this is popcount is 0?

##########
File path: cpp/src/arrow/compute/kernels/vector_selection.cc
##########
@@ -0,0 +1,1816 @@
+// 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 <algorithm>
+#include <cstring>
+#include <limits>
+
+#include "arrow/array/array_base.h"
+#include "arrow/array/array_binary.h"
+#include "arrow/array/array_dict.h"
+#include "arrow/array/array_nested.h"
+#include "arrow/array/builder_primitive.h"
+#include "arrow/array/concatenate.h"
+#include "arrow/buffer_builder.h"
+#include "arrow/compute/api_vector.h"
+#include "arrow/compute/kernels/common.h"
+#include "arrow/compute/kernels/util_internal.h"
+#include "arrow/extension_type.h"
+#include "arrow/record_batch.h"
+#include "arrow/result.h"
+#include "arrow/util/bit_block_counter.h"
+#include "arrow/util/bit_util.h"
+#include "arrow/util/bitmap_ops.h"
+#include "arrow/util/bitmap_reader.h"
+#include "arrow/util/int_util.h"
+
+namespace arrow {
+
+using internal::BinaryBitBlockCounter;
+using internal::BitBlockCount;
+using internal::BitBlockCounter;
+using internal::BitmapReader;
+using internal::CopyBitmap;
+using internal::GetArrayView;
+using internal::IndexBoundsCheck;
+using internal::OptionalBitBlockCounter;
+using internal::OptionalBitIndexer;
+
+namespace compute {
+namespace internal {
+
+int64_t GetFilterOutputSize(const ArrayData& filter,
+                            FilterOptions::NullSelectionBehavior null_selection) {
+  int64_t output_size = 0;
+  int64_t position = 0;
+  if (filter.GetNullCount() > 0) {
+    const uint8_t* filter_is_valid = filter.buffers[0]->data();
+    BinaryBitBlockCounter bit_counter(filter.buffers[1]->data(), filter.offset,
+                                      filter_is_valid, filter.offset, filter.length);
+    if (null_selection == FilterOptions::EMIT_NULL) {
+      while (position < filter.length) {
+        BitBlockCount block = bit_counter.NextOrNotWord();
+        output_size += block.popcount;
+        position += block.length;
+      }
+    } else {
+      while (position < filter.length) {
+        BitBlockCount block = bit_counter.NextAndWord();
+        output_size += block.popcount;
+        position += block.length;
+      }
+    }
+  } else {
+    // The filter has no nulls, so we plow through its data as fast as
+    // possible.
+    BitBlockCounter bit_counter(filter.buffers[1]->data(), filter.offset, filter.length);
+    while (position < filter.length) {
+      BitBlockCount block = bit_counter.NextFourWords();
+      output_size += block.popcount;
+      position += block.length;
+    }
+  }
+  return output_size;
+}
+
+template <typename IndexType>
+Result<std::shared_ptr<ArrayData>> GetTakeIndicesImpl(
+    const ArrayData& filter, FilterOptions::NullSelectionBehavior null_selection,
+    MemoryPool* memory_pool) {
+  using T = typename IndexType::c_type;
+  typename TypeTraits<IndexType>::BuilderType builder(memory_pool);
+
+  const uint8_t* filter_data = filter.buffers[1]->data();
+  BitBlockCounter data_counter(filter_data, filter.offset, filter.length);
+
+  // The position relative to the start of the filter
+  T position = 0;
+
+  // The current position taking the filter offset into account
+  int64_t position_with_offset = filter.offset;
+  if (filter.GetNullCount() > 0) {
+    // The filter has nulls, so we scan the validity bitmap and the filter data
+    // bitmap together, branching on the null selection type.
+    const uint8_t* filter_is_valid = filter.buffers[0]->data();
+
+    // To count blocks whether filter_data[i] || !filter_is_valid[i]
+    BinaryBitBlockCounter filter_counter(filter_data, filter.offset, filter_is_valid,
+                                         filter.offset, filter.length);
+    if (null_selection == FilterOptions::DROP) {
+      while (position < filter.length) {
+        BitBlockCount and_block = filter_counter.NextAndWord();
+        RETURN_NOT_OK(builder.Reserve(and_block.popcount));
+        if (and_block.AllSet()) {
+          // All the values are selected and non-null
+          for (int64_t i = 0; i < and_block.length; ++i) {
+            builder.UnsafeAppend(position++);
+          }
+          position_with_offset += and_block.length;
+        } else {
+          // Some of the values are false or null

Review comment:
       Shouldn't you short-circult this if popcount is 0?

##########
File path: cpp/src/arrow/compute/kernels/vector_selection.cc
##########
@@ -0,0 +1,1816 @@
+// 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 <algorithm>
+#include <cstring>
+#include <limits>
+
+#include "arrow/array/array_base.h"
+#include "arrow/array/array_binary.h"
+#include "arrow/array/array_dict.h"
+#include "arrow/array/array_nested.h"
+#include "arrow/array/builder_primitive.h"
+#include "arrow/array/concatenate.h"
+#include "arrow/buffer_builder.h"
+#include "arrow/compute/api_vector.h"
+#include "arrow/compute/kernels/common.h"
+#include "arrow/compute/kernels/util_internal.h"
+#include "arrow/extension_type.h"
+#include "arrow/record_batch.h"
+#include "arrow/result.h"
+#include "arrow/util/bit_block_counter.h"
+#include "arrow/util/bit_util.h"
+#include "arrow/util/bitmap_ops.h"
+#include "arrow/util/bitmap_reader.h"
+#include "arrow/util/int_util.h"
+
+namespace arrow {
+
+using internal::BinaryBitBlockCounter;
+using internal::BitBlockCount;
+using internal::BitBlockCounter;
+using internal::BitmapReader;
+using internal::CopyBitmap;
+using internal::GetArrayView;
+using internal::IndexBoundsCheck;
+using internal::OptionalBitBlockCounter;
+using internal::OptionalBitIndexer;
+
+namespace compute {
+namespace internal {
+
+int64_t GetFilterOutputSize(const ArrayData& filter,
+                            FilterOptions::NullSelectionBehavior null_selection) {
+  int64_t output_size = 0;
+  int64_t position = 0;
+  if (filter.GetNullCount() > 0) {
+    const uint8_t* filter_is_valid = filter.buffers[0]->data();
+    BinaryBitBlockCounter bit_counter(filter.buffers[1]->data(), filter.offset,
+                                      filter_is_valid, filter.offset, filter.length);
+    if (null_selection == FilterOptions::EMIT_NULL) {
+      while (position < filter.length) {
+        BitBlockCount block = bit_counter.NextOrNotWord();
+        output_size += block.popcount;
+        position += block.length;
+      }
+    } else {
+      while (position < filter.length) {
+        BitBlockCount block = bit_counter.NextAndWord();
+        output_size += block.popcount;
+        position += block.length;
+      }
+    }
+  } else {
+    // The filter has no nulls, so we plow through its data as fast as
+    // possible.
+    BitBlockCounter bit_counter(filter.buffers[1]->data(), filter.offset, filter.length);
+    while (position < filter.length) {
+      BitBlockCount block = bit_counter.NextFourWords();
+      output_size += block.popcount;
+      position += block.length;
+    }
+  }
+  return output_size;
+}
+
+template <typename IndexType>
+Result<std::shared_ptr<ArrayData>> GetTakeIndicesImpl(
+    const ArrayData& filter, FilterOptions::NullSelectionBehavior null_selection,
+    MemoryPool* memory_pool) {
+  using T = typename IndexType::c_type;
+  typename TypeTraits<IndexType>::BuilderType builder(memory_pool);
+
+  const uint8_t* filter_data = filter.buffers[1]->data();
+  BitBlockCounter data_counter(filter_data, filter.offset, filter.length);
+
+  // The position relative to the start of the filter
+  T position = 0;
+
+  // The current position taking the filter offset into account
+  int64_t position_with_offset = filter.offset;
+  if (filter.GetNullCount() > 0) {
+    // The filter has nulls, so we scan the validity bitmap and the filter data
+    // bitmap together, branching on the null selection type.
+    const uint8_t* filter_is_valid = filter.buffers[0]->data();
+
+    // To count blocks whether filter_data[i] || !filter_is_valid[i]
+    BinaryBitBlockCounter filter_counter(filter_data, filter.offset, filter_is_valid,
+                                         filter.offset, filter.length);
+    if (null_selection == FilterOptions::DROP) {
+      while (position < filter.length) {
+        BitBlockCount and_block = filter_counter.NextAndWord();
+        RETURN_NOT_OK(builder.Reserve(and_block.popcount));
+        if (and_block.AllSet()) {
+          // All the values are selected and non-null
+          for (int64_t i = 0; i < and_block.length; ++i) {
+            builder.UnsafeAppend(position++);
+          }
+          position_with_offset += and_block.length;
+        } else {
+          // Some of the values are false or null
+          for (int64_t i = 0; i < and_block.length; ++i) {
+            if (BitUtil::GetBit(filter_is_valid, position_with_offset) &&
+                BitUtil::GetBit(filter_data, position_with_offset)) {
+              builder.UnsafeAppend(position);
+            }
+            ++position;
+            ++position_with_offset;
+          }
+        }
+      }
+    } else {
+      BitBlockCounter is_valid_counter(filter_is_valid, filter.offset, filter.length);
+      while (position < filter.length) {
+        // true OR NOT valid
+        BitBlockCount or_not_block = filter_counter.NextOrNotWord();
+        RETURN_NOT_OK(builder.Reserve(or_not_block.popcount));
+
+        // If the values are all valid and the or_not_block is full, then we
+        // can infer that all the values are true and skip the bit checking
+        BitBlockCount is_valid_block = is_valid_counter.NextWord();
+
+        if (or_not_block.AllSet() && is_valid_block.AllSet()) {
+          // All the values are selected and non-null
+          for (int64_t i = 0; i < or_not_block.length; ++i) {
+            builder.UnsafeAppend(position++);
+          }
+          position_with_offset += or_not_block.length;
+        } else {
+          // Some of the values are false or null
+          for (int64_t i = 0; i < or_not_block.length; ++i) {
+            if (BitUtil::GetBit(filter_is_valid, position_with_offset)) {
+              if (BitUtil::GetBit(filter_data, position_with_offset)) {
+                builder.UnsafeAppend(position);
+              }
+            } else {
+              // Null slot, so append a null
+              builder.UnsafeAppendNull();
+            }
+            ++position;
+            ++position_with_offset;
+          }
+        }
+      }
+    }
+  } else {
+    // The filter has no nulls, so we need only look for true values
+    BitBlockCount current_block = data_counter.NextWord();
+    while (position < filter.length) {
+      if (current_block.AllSet()) {
+        int64_t run_length = 0;
+
+        // If we've found a all-true block, then we scan forward until we find
+        // a block that has some false values (or we reach the end
+        while (current_block.length > 0 && current_block.AllSet()) {
+          run_length += current_block.length;
+          current_block = data_counter.NextWord();
+        }
+
+        // Append the consecutive run of indices
+        RETURN_NOT_OK(builder.Reserve(run_length));
+        for (int64_t i = 0; i < run_length; ++i) {
+          builder.UnsafeAppend(position++);
+        }
+        position_with_offset += run_length;
+      } else {
+        // Must do bitchecking on the current block
+        RETURN_NOT_OK(builder.Reserve(current_block.popcount));
+        for (int64_t i = 0; i < current_block.length; ++i) {
+          if (BitUtil::GetBit(filter_data, position_with_offset)) {
+            builder.UnsafeAppend(position);
+          }
+          ++position;
+          ++position_with_offset;
+        }
+        current_block = data_counter.NextWord();
+      }
+    }
+  }
+  std::shared_ptr<ArrayData> result;
+  RETURN_NOT_OK(builder.FinishInternal(&result));
+  return result;
+}
+
+Result<std::shared_ptr<ArrayData>> GetTakeIndices(
+    const ArrayData& filter, FilterOptions::NullSelectionBehavior null_selection,
+    MemoryPool* memory_pool) {
+  DCHECK_EQ(filter.type->id(), Type::BOOL);
+  if (filter.length <= std::numeric_limits<uint16_t>::max()) {
+    return GetTakeIndicesImpl<UInt16Type>(filter, null_selection, memory_pool);
+  } else if (filter.length <= std::numeric_limits<uint32_t>::max()) {
+    return GetTakeIndicesImpl<UInt32Type>(filter, null_selection, memory_pool);
+  } else {
+    // Arrays over 4 billion elements, not especially likely.
+    return Status::NotImplemented(
+        "Filter length exceeds UINT32_MAX, "
+        "consider a different strategy for selecting elements");
+  }
+}
+
+namespace {
+
+template <typename ArrowType>
+struct GetCType {
+  using type = typename ArrowType::c_type;
+};
+
+// We want uint8_t for boolean instead of bool
+template <>
+struct GetCType<BooleanType> {
+  using type = uint8_t;
+};
+
+using FilterState = OptionsWrapper<FilterOptions>;
+using TakeState = OptionsWrapper<TakeOptions>;
+
+Status PreallocateData(KernelContext* ctx, int64_t length, int bit_width, Datum* out) {
+  // Preallocate memory
+  ArrayData* out_arr = out->mutable_array();
+  out_arr->length = length;
+  out_arr->buffers.resize(2);
+
+  ARROW_ASSIGN_OR_RAISE(out_arr->buffers[0], ctx->AllocateBitmap(length));
+  if (bit_width == 1) {
+    ARROW_ASSIGN_OR_RAISE(out_arr->buffers[1], ctx->AllocateBitmap(length));
+  } else {
+    ARROW_ASSIGN_OR_RAISE(out_arr->buffers[1], ctx->Allocate(length * bit_width / 8));
+  }
+  return Status::OK();
+}
+
+// ----------------------------------------------------------------------
+// Implement optimized take for primitive types from boolean to 1/2/4/8-byte
+// C-type based types. Use common implementation for every byte width and only
+// generate code for unsigned integer indices, since after boundschecking to
+// check for negative numbers in the indices we can safely reinterpret_cast
+// signed integers as unsigned.
+
+/// \brief The Take implementation for primitive (fixed-width) types does not
+/// use the logical Arrow type but rather the physical C type. This way we
+/// only generate one take function for each byte width.
+///
+/// This function assumes that the indices have been boundschecked.
+template <typename IndexCType, typename ValueCType>
+struct PrimitiveTakeImpl {
+  static void Exec(const PrimitiveArg& values, const PrimitiveArg& indices,
+                   Datum* out_datum) {
+    auto values_data = reinterpret_cast<const ValueCType*>(values.data);
+    auto values_is_valid = values.is_valid;
+    auto values_offset = values.offset;
+
+    auto indices_data = reinterpret_cast<const IndexCType*>(indices.data);
+    auto indices_is_valid = indices.is_valid;
+    auto indices_offset = indices.offset;
+
+    ArrayData* out_arr = out_datum->mutable_array();
+    auto out = out_arr->GetMutableValues<ValueCType>(1);
+    auto out_is_valid = out_arr->buffers[0]->mutable_data();
+    auto out_offset = out_arr->offset;
+
+    // If either the values or indices have nulls, we preemptively zero out the
+    // out validity bitmap so that we don't have to use ClearBit in each
+    // iteration for nulls.
+    if (values.null_count > 0 || indices.null_count > 0) {
+      BitUtil::SetBitsTo(out_is_valid, out_offset, indices.length, false);
+    }
+
+    OptionalBitBlockCounter indices_bit_counter(indices_is_valid, indices_offset,
+                                                indices.length);
+    int64_t position = 0;
+    int64_t valid_count = 0;
+    while (position < indices.length) {
+      BitBlockCount block = indices_bit_counter.NextBlock();
+      if (values.null_count == 0) {
+        // Values are never null, so things are easier
+        valid_count += block.popcount;
+        if (block.popcount == block.length) {
+          // Fastest path: neither values nor index nulls
+          BitUtil::SetBitsTo(out_is_valid, out_offset + position, block.length, true);
+          for (int64_t i = 0; i < block.length; ++i) {
+            out[position] = values_data[indices_data[position]];
+            ++position;
+          }
+        } else if (block.popcount > 0) {
+          // Slow path: some indices but not all are null
+          for (int64_t i = 0; i < block.length; ++i) {
+            if (BitUtil::GetBit(indices_is_valid, indices_offset + position)) {
+              // index is not null
+              BitUtil::SetBit(out_is_valid, out_offset + position);
+              out[position] = values_data[indices_data[position]];
+            } else {
+              out[position] = ValueCType{};
+            }
+            ++position;
+          }
+        } else {
+          memset(out + position, 0, sizeof(ValueCType) * block.length);
+          position += block.length;
+        }
+      } else {
+        // Values have nulls, so we must do random access into the values bitmap
+        if (block.popcount == block.length) {
+          // Faster path: indices are not null but values may be
+          for (int64_t i = 0; i < block.length; ++i) {
+            if (BitUtil::GetBit(values_is_valid,
+                                values_offset + indices_data[position])) {
+              // value is not null
+              out[position] = values_data[indices_data[position]];
+              BitUtil::SetBit(out_is_valid, out_offset + position);
+              ++valid_count;
+            } else {
+              out[position] = ValueCType{};
+            }
+            ++position;
+          }
+        } else if (block.popcount > 0) {
+          // Slow path: some but not all indices are null. Since we are doing
+          // random access in general we have to check the value nullness one by
+          // one.
+          for (int64_t i = 0; i < block.length; ++i) {
+            if (BitUtil::GetBit(indices_is_valid, indices_offset + position) &&
+                BitUtil::GetBit(values_is_valid,
+                                values_offset + indices_data[position])) {
+              // index is not null && value is not null
+              out[position] = values_data[indices_data[position]];
+              BitUtil::SetBit(out_is_valid, out_offset + position);
+              ++valid_count;
+            } else {
+              out[position] = ValueCType{};
+            }
+            ++position;
+          }
+        } else {
+          memset(out + position, 0, sizeof(ValueCType) * block.length);
+          position += block.length;
+        }
+      }
+    }
+    out_arr->null_count = out_arr->length - valid_count;
+  }
+};
+
+template <typename IndexCType>
+struct BooleanTakeImpl {
+  static void Exec(const PrimitiveArg& values, const PrimitiveArg& indices,
+                   Datum* out_datum) {
+    const uint8_t* values_data = values.data;
+    auto values_is_valid = values.is_valid;
+    auto values_offset = values.offset;
+
+    auto indices_data = reinterpret_cast<const IndexCType*>(indices.data);
+    auto indices_is_valid = indices.is_valid;
+    auto indices_offset = indices.offset;
+
+    ArrayData* out_arr = out_datum->mutable_array();
+    auto out = out_arr->buffers[1]->mutable_data();
+    auto out_is_valid = out_arr->buffers[0]->mutable_data();
+    auto out_offset = out_arr->offset;
+
+    // If either the values or indices have nulls, we preemptively zero out the
+    // out validity bitmap so that we don't have to use ClearBit in each
+    // iteration for nulls.
+    if (values.null_count > 0 || indices.null_count > 0) {
+      BitUtil::SetBitsTo(out_is_valid, out_offset, indices.length, false);
+    }
+    // Avoid uninitialized data in values array
+    BitUtil::SetBitsTo(out, out_offset, indices.length, false);
+
+    auto PlaceDataBit = [&](int64_t loc, IndexCType index) {
+      BitUtil::SetBitTo(out, out_offset + loc,
+                        BitUtil::GetBit(values_data, values_offset + index));
+    };
+
+    OptionalBitBlockCounter indices_bit_counter(indices_is_valid, indices_offset,
+                                                indices.length);
+    int64_t position = 0;
+    int64_t valid_count = 0;
+    while (position < indices.length) {
+      BitBlockCount block = indices_bit_counter.NextBlock();
+      if (values.null_count == 0) {
+        // Values are never null, so things are easier
+        valid_count += block.popcount;
+        if (block.popcount == block.length) {
+          // Fastest path: neither values nor index nulls
+          BitUtil::SetBitsTo(out_is_valid, out_offset + position, block.length, true);
+          for (int64_t i = 0; i < block.length; ++i) {
+            PlaceDataBit(position, indices_data[position]);
+            ++position;
+          }
+        } else if (block.popcount > 0) {
+          // Slow path: some but not all indices are null
+          for (int64_t i = 0; i < block.length; ++i) {
+            if (BitUtil::GetBit(indices_is_valid, indices_offset + position)) {
+              // index is not null
+              BitUtil::SetBit(out_is_valid, out_offset + position);
+              PlaceDataBit(position, indices_data[position]);
+            }
+            ++position;
+          }
+        } else {
+          position += block.length;
+        }
+      } else {
+        // Values have nulls, so we must do random access into the values bitmap
+        if (block.popcount == block.length) {
+          // Faster path: indices are not null but values may be
+          for (int64_t i = 0; i < block.length; ++i) {
+            if (BitUtil::GetBit(values_is_valid,
+                                values_offset + indices_data[position])) {
+              // value is not null
+              BitUtil::SetBit(out_is_valid, out_offset + position);
+              PlaceDataBit(position, indices_data[position]);
+              ++valid_count;
+            }
+            ++position;
+          }
+        } else if (block.popcount > 0) {
+          // Slow path: some but not all indices are null. Since we are doing
+          // random access in general we have to check the value nullness one by
+          // one.
+          for (int64_t i = 0; i < block.length; ++i) {
+            if (BitUtil::GetBit(indices_is_valid, indices_offset + position)) {
+              // index is not null
+              if (BitUtil::GetBit(values_is_valid,
+                                  values_offset + indices_data[position])) {
+                // value is not null
+                PlaceDataBit(position, indices_data[position]);
+                BitUtil::SetBit(out_is_valid, out_offset + position);
+                ++valid_count;
+              }
+            }
+            ++position;
+          }
+        } else {
+          position += block.length;
+        }
+      }
+    }
+    out_arr->null_count = out_arr->length - valid_count;
+  }
+};
+
+template <template <typename...> class TakeImpl, typename... Args>
+void TakeIndexDispatch(const PrimitiveArg& values, const PrimitiveArg& indices,
+                       Datum* out) {
+  // With the simplifying assumption that boundschecking has taken place
+  // already at a higher level, we can now assume that the index values are all
+  // non-negative. Thus, we can interpret signed integers as unsigned and avoid
+  // having to generate double the amount of binary code to handle each integer
+  // width.
+  switch (indices.bit_width) {
+    case 8:
+      return TakeImpl<uint8_t, Args...>::Exec(values, indices, out);
+    case 16:
+      return TakeImpl<uint16_t, Args...>::Exec(values, indices, out);
+    case 32:
+      return TakeImpl<uint32_t, Args...>::Exec(values, indices, out);
+    case 64:
+      return TakeImpl<uint64_t, Args...>::Exec(values, indices, out);
+    default:
+      DCHECK(false) << "Invalid indices byte width";
+      break;
+  }
+}
+
+void PrimitiveTake(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+  const auto& state = checked_cast<const TakeState&>(*ctx->state());
+  if (state.options.boundscheck) {
+    KERNEL_RETURN_IF_ERROR(ctx, IndexBoundsCheck(*batch[1].array(), batch[0].length()));
+  }
+
+  PrimitiveArg values = GetPrimitiveArg(*batch[0].array());
+  PrimitiveArg indices = GetPrimitiveArg(*batch[1].array());
+  KERNEL_RETURN_IF_ERROR(ctx,
+                         PreallocateData(ctx, indices.length, values.bit_width, out));
+  switch (values.bit_width) {
+    case 1:
+      return TakeIndexDispatch<BooleanTakeImpl>(values, indices, out);
+    case 8:
+      return TakeIndexDispatch<PrimitiveTakeImpl, int8_t>(values, indices, out);
+    case 16:
+      return TakeIndexDispatch<PrimitiveTakeImpl, int16_t>(values, indices, out);
+    case 32:
+      return TakeIndexDispatch<PrimitiveTakeImpl, int32_t>(values, indices, out);
+    case 64:
+      return TakeIndexDispatch<PrimitiveTakeImpl, int64_t>(values, indices, out);
+    default:
+      DCHECK(false) << "Invalid values byte width";
+      break;
+  }
+}
+
+// ----------------------------------------------------------------------
+// Optimized and streamlined filter for primitive types
+
+// Use either BitBlockCounter or BinaryBitBlockCounter to quickly scan filter a
+// word at a time for the DROP selection type.
+class DropNullCounter {
+ public:
+  // validity bitmap may be null
+  DropNullCounter(const uint8_t* validity, const uint8_t* data, int64_t offset,
+                  int64_t length)
+      : data_counter_(data, offset, length),
+        data_and_validity_counter_(data, offset, validity, offset, length),
+        has_validity_(validity != nullptr) {}
+
+  BitBlockCount NextBlock() {
+    if (has_validity_) {
+      // filter is true AND not null
+      return data_and_validity_counter_.NextAndWord();
+    } else {
+      return data_counter_.NextWord();
+    }
+  }
+
+ private:
+  // For when just data is present, but no validity bitmap
+  BitBlockCounter data_counter_;
+
+  // For when both validity bitmap and data are present
+  BinaryBitBlockCounter data_and_validity_counter_;
+  bool has_validity_;
+};
+
+/// \brief The Filter implementation for primitive (fixed-width) types does not
+/// use the logical Arrow type but rather then physical C type. This way we

Review comment:
       "the physical C type"




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

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



[GitHub] [arrow] ursabot commented on pull request #7442: ARROW-9075: [C++] Optimized Filter implementation: faster performance + compilation, smaller code size

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


   ```
   Usage: @ursabot benchmark [OPTIONS] [<baseline>]
   
     Run the benchmark suite in comparison mode.
   
     This command will run the benchmark suite for tip of the branch commit
     against `<baseline>` (or master if not provided).
   
     Examples:
   
     # Run the all the benchmarks
     @ursabot benchmark
   
     # Compare only benchmarks where the name matches the /^Sum/ regex
     @ursabot benchmark --benchmark-filter=^Sum
   
     # Compare only benchmarks where the suite matches the /compute-/ regex.
     # A suite is the C++ binary.
     @ursabot benchmark --suite-filter=compute-
   
     # Sometimes a new optimization requires the addition of new benchmarks to
     # quantify the performance increase. When doing this be sure to add the
     # benchmark in a separate commit before introducing the optimization.
     #
     # Note that specifying the baseline is the only way to compare using a new
     # benchmark, since master does not contain the new benchmark and no
     # comparison is possible.
     #
     # The following command compares the results of matching benchmarks,
     # compiling against HEAD and the provided baseline commit, e.g. eaf8302.
     # You can use this to quantify the performance improvement of new
     # optimizations or to check for regressions.
     @ursabot benchmark --benchmark-filter=MyBenchmark eaf8302
   
   Options:
     --suite-filter <regex>      Regex filtering benchmark suites.
     --benchmark-filter <regex>  Regex filtering benchmarks.
     --help                      Show this message and exit.
   ```


----------------------------------------------------------------
This is an automated message from the 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] wesm commented on a change in pull request #7442: ARROW-9075: [C++] Optimized Filter implementation: faster performance + compilation, smaller code size

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



##########
File path: cpp/src/arrow/compute/kernels/vector_selection.cc
##########
@@ -0,0 +1,1816 @@
+// 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 <algorithm>
+#include <cstring>
+#include <limits>
+
+#include "arrow/array/array_base.h"
+#include "arrow/array/array_binary.h"
+#include "arrow/array/array_dict.h"
+#include "arrow/array/array_nested.h"
+#include "arrow/array/builder_primitive.h"
+#include "arrow/array/concatenate.h"
+#include "arrow/buffer_builder.h"
+#include "arrow/compute/api_vector.h"
+#include "arrow/compute/kernels/common.h"
+#include "arrow/compute/kernels/util_internal.h"
+#include "arrow/extension_type.h"
+#include "arrow/record_batch.h"
+#include "arrow/result.h"
+#include "arrow/util/bit_block_counter.h"
+#include "arrow/util/bit_util.h"
+#include "arrow/util/bitmap_ops.h"
+#include "arrow/util/bitmap_reader.h"
+#include "arrow/util/int_util.h"
+
+namespace arrow {
+
+using internal::BinaryBitBlockCounter;
+using internal::BitBlockCount;
+using internal::BitBlockCounter;
+using internal::BitmapReader;
+using internal::CopyBitmap;
+using internal::GetArrayView;
+using internal::IndexBoundsCheck;
+using internal::OptionalBitBlockCounter;
+using internal::OptionalBitIndexer;
+
+namespace compute {
+namespace internal {
+
+int64_t GetFilterOutputSize(const ArrayData& filter,
+                            FilterOptions::NullSelectionBehavior null_selection) {
+  int64_t output_size = 0;
+  int64_t position = 0;
+  if (filter.GetNullCount() > 0) {
+    const uint8_t* filter_is_valid = filter.buffers[0]->data();
+    BinaryBitBlockCounter bit_counter(filter.buffers[1]->data(), filter.offset,
+                                      filter_is_valid, filter.offset, filter.length);
+    if (null_selection == FilterOptions::EMIT_NULL) {
+      while (position < filter.length) {
+        BitBlockCount block = bit_counter.NextOrNotWord();
+        output_size += block.popcount;
+        position += block.length;
+      }
+    } else {
+      while (position < filter.length) {
+        BitBlockCount block = bit_counter.NextAndWord();
+        output_size += block.popcount;
+        position += block.length;
+      }
+    }
+  } else {
+    // The filter has no nulls, so we plow through its data as fast as
+    // possible.
+    BitBlockCounter bit_counter(filter.buffers[1]->data(), filter.offset, filter.length);

Review comment:
       Changing to use CountSetBits, which should be faster




----------------------------------------------------------------
This is an automated message from the 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] ursabot commented on pull request #7442: ARROW-9075: [C++] Optimized Filter implementation: faster performance + compilation, smaller code size

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


   [AMD64 Ubuntu 18.04 C++ Benchmark (#112989)](https://ci.ursalabs.org/#builders/73/builds/82) builder has been succeeded.
   
   Revision: 21227cc7530e59a481f7e3c0aae8d351b4226e9d
   
   ```diff
     =======================================  ===============  ================  ========
     benchmark                                baseline         contender         change
     =======================================  ===============  ================  ========
   - FilterStringFilterNoNulls/262144/7       637.909 MiB/sec  572.355 MiB/sec   -10.276%
   - FilterStringFilterNoNulls/262144/8       10.897 GiB/sec   8.711 GiB/sec     -20.057%
     FilterStringFilterNoNulls/262144/6       485.775 MiB/sec  476.410 MiB/sec   -1.928%
     FilterStringFilterWithNulls/262144/4     649.558 MiB/sec  677.796 MiB/sec   4.347%
     FilterInt64FilterNoNulls/262144/9        3.212 GiB/sec    3.264 GiB/sec     1.612%
   - FilterFSLInt64FilterNoNulls/262144/13    351.877 MiB/sec  308.073 MiB/sec   -12.449%
   - FilterFSLInt64FilterNoNulls/262144/10    389.471 MiB/sec  333.418 MiB/sec   -14.392%
   - FilterInt64FilterNoNulls/262144/4        668.729 MiB/sec  625.199 MiB/sec   -6.509%
     FilterFSLInt64FilterWithNulls/262144/9   287.988 MiB/sec  276.495 MiB/sec   -3.991%
   - FilterStringFilterWithNulls/262144/2     9.441 GiB/sec    8.793 GiB/sec     -6.865%
     FilterStringFilterWithNulls/262144/12    73.855 MiB/sec   82.463 MiB/sec    11.656%
   - FilterFSLInt64FilterNoNulls/262144/5     6.091 GiB/sec    4.403 GiB/sec     -27.714%
   - FilterFSLInt64FilterNoNulls/262144/3     550.519 MiB/sec  463.959 MiB/sec   -15.723%
     FilterInt64FilterNoNulls/262144/2        7.988 GiB/sec    7.976 GiB/sec     -0.147%
   - FilterStringFilterNoNulls/262144/4       700.795 MiB/sec  605.189 MiB/sec   -13.643%
   - FilterFSLInt64FilterWithNulls/262144/1   516.544 MiB/sec  460.521 MiB/sec   -10.846%
   - FilterStringFilterWithNulls/262144/8     8.877 GiB/sec    8.364 GiB/sec     -5.779%
   - FilterFSLInt64FilterWithNulls/262144/3   350.123 MiB/sec  329.103 MiB/sec   -6.004%
     FilterStringFilterWithNulls/262144/3     435.836 MiB/sec  494.167 MiB/sec   13.384%
     FilterInt64FilterNoNulls/262144/10       630.544 MiB/sec  628.104 MiB/sec   -0.387%
   - FilterStringFilterNoNulls/262144/5       11.014 GiB/sec   8.788 GiB/sec     -20.216%
     FilterInt64FilterNoNulls/262144/3        4.263 GiB/sec    4.181 GiB/sec     -1.936%
     FilterInt64FilterWithNulls/262144/1      635.637 MiB/sec  615.015 MiB/sec   -3.244%
     FilterStringFilterWithNulls/262144/7     638.645 MiB/sec  678.465 MiB/sec   6.235%
   - FilterFSLInt64FilterNoNulls/262144/2     6.506 GiB/sec    5.012 GiB/sec     -22.975%
   - FilterFSLInt64FilterNoNulls/262144/0     729.854 MiB/sec  569.623 MiB/sec   -21.954%
     FilterInt64FilterNoNulls/262144/5        6.946 GiB/sec    6.899 GiB/sec     -0.674%
     FilterInt64FilterWithNulls/262144/12     545.763 MiB/sec  547.657 MiB/sec   0.347%
     FilterStringFilterNoNulls/262144/9       383.858 MiB/sec  377.178 MiB/sec   -1.740%
   - FilterFSLInt64FilterNoNulls/262144/8     5.825 GiB/sec    4.702 GiB/sec     -19.289%
     FilterInt64FilterNoNulls/262144/13       632.053 MiB/sec  633.157 MiB/sec   0.175%
     FilterInt64FilterNoNulls/262144/1        1.020 GiB/sec    1.022 GiB/sec     0.239%
   - FilterFSLInt64FilterNoNulls/262144/12    242.197 MiB/sec  228.152 MiB/sec   -5.799%
     FilterInt64FilterWithNulls/262144/4      640.980 MiB/sec  614.192 MiB/sec   -4.179%
     FilterInt64FilterWithNulls/262144/8      4.967 GiB/sec    5.071 GiB/sec     2.102%
   - FilterFSLInt64FilterWithNulls/262144/0   396.373 MiB/sec  374.388 MiB/sec   -5.546%
     FilterInt64FilterWithNulls/262144/11     4.934 GiB/sec    4.997 GiB/sec     1.282%
   - FilterFSLInt64FilterNoNulls/262144/14    5.435 GiB/sec    4.459 GiB/sec     -17.946%
     FilterInt64FilterNoNulls/262144/12       3.255 GiB/sec    3.185 GiB/sec     -2.144%
     FilterStringFilterWithNulls/262144/1     638.704 MiB/sec  690.413 MiB/sec   8.096%
   - FilterStringFilterNoNulls/262144/2       11.411 GiB/sec   9.040 GiB/sec     -20.778%
     FilterInt64FilterWithNulls/262144/6      582.753 MiB/sec  554.462 MiB/sec   -4.855%
     FilterStringFilterWithNulls/262144/10    586.149 MiB/sec  616.404 MiB/sec   5.162%
     FilterInt64FilterNoNulls/262144/0        7.653 GiB/sec    7.971 GiB/sec     4.146%
     FilterInt64FilterWithNulls/262144/13     590.396 MiB/sec  607.816 MiB/sec   2.951%
   - FilterStringFilterNoNulls/262144/14      1.254 GiB/sec    1011.778 MiB/sec  -21.233%
   - FilterFSLInt64FilterWithNulls/262144/4   474.573 MiB/sec  428.073 MiB/sec   -9.798%
     FilterInt64FilterWithNulls/262144/2      5.245 GiB/sec    5.072 GiB/sec     -3.310%
   - FilterStringFilterWithNulls/262144/11    8.381 GiB/sec    7.793 GiB/sec     -7.006%
     FilterFSLInt64FilterWithNulls/262144/14  4.065 GiB/sec    3.917 GiB/sec     -3.648%
   - FilterFSLInt64FilterNoNulls/262144/1     566.516 MiB/sec  432.124 MiB/sec   -23.723%
     FilterStringFilterWithNulls/262144/6     431.308 MiB/sec  489.475 MiB/sec   13.486%
   - FilterFSLInt64FilterNoNulls/262144/9     267.636 MiB/sec  250.549 MiB/sec   -6.385%
   - FilterFSLInt64FilterWithNulls/262144/2   4.505 GiB/sec    4.244 GiB/sec     -5.789%
   - FilterStringFilterNoNulls/262144/1       699.807 MiB/sec  605.175 MiB/sec   -13.523%
     FilterInt64FilterWithNulls/262144/14     4.914 GiB/sec    4.970 GiB/sec     1.141%
   - FilterStringFilterNoNulls/262144/11      9.990 GiB/sec    7.988 GiB/sec     -20.035%
   - FilterStringFilterNoNulls/262144/12      70.677 MiB/sec   65.603 MiB/sec    -7.180%
     FilterStringFilterWithNulls/262144/9     395.814 MiB/sec  447.434 MiB/sec   13.042%
     FilterFSLInt64FilterWithNulls/262144/6   333.780 MiB/sec  319.575 MiB/sec   -4.256%
     FilterFSLInt64FilterWithNulls/262144/8   4.263 GiB/sec    4.091 GiB/sec     -4.021%
     FilterInt64FilterNoNulls/262144/14       6.414 GiB/sec    6.933 GiB/sec     8.095%
     FilterStringFilterWithNulls/262144/0     441.849 MiB/sec  496.266 MiB/sec   12.316%
     FilterInt64FilterNoNulls/262144/11       6.411 GiB/sec    6.874 GiB/sec     7.218%
   - FilterInt64FilterNoNulls/262144/7        648.036 MiB/sec  547.011 MiB/sec   -15.589%
   - FilterFSLInt64FilterWithNulls/262144/10  419.063 MiB/sec  381.681 MiB/sec   -8.920%
   - FilterFSLInt64FilterWithNulls/262144/13  386.755 MiB/sec  353.726 MiB/sec   -8.540%
     FilterInt64FilterNoNulls/262144/8        6.724 GiB/sec    7.073 GiB/sec     5.190%
     FilterInt64FilterWithNulls/262144/9      545.560 MiB/sec  545.449 MiB/sec   -0.020%
   - FilterStringFilterNoNulls/262144/10      575.809 MiB/sec  507.681 MiB/sec   -11.832%
   - FilterStringFilterWithNulls/262144/5     9.154 GiB/sec    8.428 GiB/sec     -7.931%
     FilterStringFilterNoNulls/262144/0       519.896 MiB/sec  554.802 MiB/sec   6.714%
     FilterFSLInt64FilterWithNulls/262144/5   4.294 GiB/sec    4.126 GiB/sec     -3.911%
   - FilterFSLInt64FilterNoNulls/262144/7     463.085 MiB/sec  378.577 MiB/sec   -18.249%
     FilterFSLInt64FilterWithNulls/262144/11  4.245 GiB/sec    4.061 GiB/sec     -4.333%
     FilterStringFilterNoNulls/262144/3       544.102 MiB/sec  542.846 MiB/sec   -0.231%
   - FilterInt64FilterWithNulls/262144/0      617.474 MiB/sec  560.813 MiB/sec   -9.176%
     FilterInt64FilterWithNulls/262144/7      619.732 MiB/sec  609.068 MiB/sec   -1.721%
     FilterStringFilterWithNulls/262144/13    91.185 MiB/sec   97.530 MiB/sec    6.958%
   - FilterStringFilterWithNulls/262144/14    929.857 MiB/sec  874.512 MiB/sec   -5.952%
   - FilterInt64FilterWithNulls/262144/3      604.918 MiB/sec  560.882 MiB/sec   -7.280%
   - FilterFSLInt64FilterNoNulls/262144/4     514.014 MiB/sec  411.713 MiB/sec   -19.902%
   - FilterFSLInt64FilterWithNulls/262144/7   463.921 MiB/sec  417.320 MiB/sec   -10.045%
   - FilterFSLInt64FilterWithNulls/262144/12  267.697 MiB/sec  247.408 MiB/sec   -7.579%
   - FilterFSLInt64FilterNoNulls/262144/11    5.632 GiB/sec    4.533 GiB/sec     -19.515%
   - FilterStringFilterNoNulls/262144/13      90.578 MiB/sec   76.367 MiB/sec    -15.690%
     FilterInt64FilterNoNulls/262144/6        3.709 GiB/sec    3.680 GiB/sec     -0.786%
     FilterInt64FilterWithNulls/262144/5      5.115 GiB/sec    4.997 GiB/sec     -2.309%
     FilterInt64FilterWithNulls/262144/10     604.161 MiB/sec  607.760 MiB/sec   0.596%
   - FilterFSLInt64FilterNoNulls/262144/6     389.763 MiB/sec  354.969 MiB/sec   -8.927%
     =======================================  ===============  ================  ========
   ```


----------------------------------------------------------------
This is an automated message from the 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] fsaintjacques commented on a change in pull request #7442: ARROW-9075: [C++] Optimized Filter implementation: faster performance + compilation, smaller code size

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



##########
File path: cpp/src/arrow/compute/kernels/vector_selection_test.cc
##########
@@ -0,0 +1,1637 @@
+// 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 <algorithm>
+#include <iostream>
+#include <limits>
+#include <memory>
+#include <string>
+#include <utility>
+#include <vector>
+
+#include "arrow/compute/api.h"
+#include "arrow/compute/kernels/test_util.h"
+#include "arrow/table.h"
+#include "arrow/testing/gtest_common.h"
+#include "arrow/testing/gtest_util.h"
+#include "arrow/testing/random.h"
+#include "arrow/testing/util.h"
+
+namespace arrow {
+
+using internal::checked_cast;
+using internal::checked_pointer_cast;
+using util::string_view;
+
+namespace compute {
+
+// ----------------------------------------------------------------------
+// Some random data generation helpers
+
+template <typename Type>
+std::shared_ptr<Array> RandomNumeric(int64_t length, double null_probability,
+                                     random::RandomArrayGenerator* rng) {
+  return rng->Numeric<Type>(length, 0, 127, null_probability);
+}
+
+std::shared_ptr<Array> RandomBoolean(int64_t length, double null_probability,
+                                     random::RandomArrayGenerator* rng) {
+  return rng->Boolean(length, 0.5, null_probability);
+}
+
+std::shared_ptr<Array> RandomString(int64_t length, double null_probability,
+                                    random::RandomArrayGenerator* rng) {
+  return rng->String(length, 0, 32, null_probability);
+}
+
+std::shared_ptr<Array> RandomLargeString(int64_t length, double null_probability,
+                                         random::RandomArrayGenerator* rng) {
+  return rng->LargeString(length, 0, 32, null_probability);
+}
+
+std::shared_ptr<Array> RandomFixedSizeBinary(int64_t length, double null_probability,
+                                             random::RandomArrayGenerator* rng) {
+  const int32_t value_size = 16;
+  int64_t data_nbytes = length * value_size;
+  std::shared_ptr<Buffer> data = *AllocateBuffer(data_nbytes);
+  random_bytes(data_nbytes, /*seed=*/0, data->mutable_data());
+  auto validity = rng->Boolean(length, 1 - null_probability);
+
+  // Assemble the data for a FixedSizeBinaryArray
+  auto values_data = std::make_shared<ArrayData>(fixed_size_binary(value_size), length);
+  values_data->buffers = {validity->data()->buffers[1], data};
+  return MakeArray(values_data);
+}
+
+// ----------------------------------------------------------------------
+
+TEST(GetTakeIndices, Basics) {
+  auto CheckCase = [&](const std::string& filter_json, const std::string& indices_json,
+                       FilterOptions::NullSelectionBehavior null_selection,
+                       const std::shared_ptr<DataType>& indices_type = uint16()) {
+    auto filter = ArrayFromJSON(boolean(), filter_json);
+    auto expected_indices = ArrayFromJSON(indices_type, indices_json);
+    ASSERT_OK_AND_ASSIGN(auto indices,
+                         internal::GetTakeIndices(*filter->data(), null_selection));
+    AssertArraysEqual(*expected_indices, *MakeArray(indices), /*verbose=*/true);
+  };
+
+  // Drop null cases
+  CheckCase("[]", "[]", FilterOptions::DROP);
+  CheckCase("[null]", "[]", FilterOptions::DROP);
+  CheckCase("[null, false, true, true, false, true]", "[2, 3, 5]", FilterOptions::DROP);
+
+  // Emit null cases
+  CheckCase("[]", "[]", FilterOptions::EMIT_NULL);
+  CheckCase("[null]", "[null]", FilterOptions::EMIT_NULL);
+  CheckCase("[null, false, true, true]", "[null, 2, 3]", FilterOptions::EMIT_NULL);
+}
+
+// TODO: Add slicing
+
+template <typename IndexArrayType>
+void CheckGetTakeIndicesCase(const Array& untyped_filter) {
+  const auto& filter = checked_cast<const BooleanArray&>(untyped_filter);
+  ASSERT_OK_AND_ASSIGN(std::shared_ptr<ArrayData> drop_indices,
+                       internal::GetTakeIndices(*filter.data(), FilterOptions::DROP));
+  // Verify DROP indices
+  {
+    IndexArrayType indices(drop_indices);
+    int64_t out_position = 0;
+    for (int64_t i = 0; i < filter.length(); ++i) {
+      if (filter.IsValid(i)) {
+        if (filter.Value(i)) {
+          ASSERT_EQ(indices.Value(out_position), i);
+          ++out_position;
+        }
+      }
+    }
+    // Check that the end length agrees with the output of GetFilterOutputSize
+    ASSERT_EQ(out_position,
+              internal::GetFilterOutputSize(*filter.data(), FilterOptions::DROP));
+  }
+
+  ASSERT_OK_AND_ASSIGN(
+      std::shared_ptr<ArrayData> emit_indices,
+      internal::GetTakeIndices(*filter.data(), FilterOptions::EMIT_NULL));
+
+  // Verify EMIT_NULL indices
+  {
+    IndexArrayType indices(emit_indices);
+    int64_t out_position = 0;
+    for (int64_t i = 0; i < filter.length(); ++i) {
+      if (filter.IsValid(i)) {
+        if (filter.Value(i)) {
+          ASSERT_EQ(indices.Value(out_position), i);
+          ++out_position;
+        }
+      } else {
+        ASSERT_TRUE(indices.IsNull(out_position));
+        ++out_position;
+      }
+    }
+
+    // Check that the end length agrees with the output of GetFilterOutputSize
+    ASSERT_EQ(out_position,
+              internal::GetFilterOutputSize(*filter.data(), FilterOptions::EMIT_NULL));
+  }
+}
+
+TEST(GetTakeIndices, RandomlyGenerated) {
+  random::RandomArrayGenerator rng(kRandomSeed);
+
+  const int64_t length = 5000;

Review comment:
       I think it would be safer to also test length of modulo the size of a block ± 1 to test the loop exit conditions.

##########
File path: cpp/src/arrow/testing/random.cc
##########
@@ -84,7 +84,7 @@ std::shared_ptr<Array> RandomArrayGenerator::Boolean(int64_t size, double probab
 
   BufferVector buffers{2};
   // Need 2 distinct generators such that probabilities are not shared.
-  GenOpt value_gen(seed(), 0, 1, probability);
+  GenOpt value_gen(seed(), 0, 1, 1 - probability);

Review comment:
       Confirmed. I find this a bit awkward. We probably want to make `GenerateBitmap` use the probability as-is (not applying `1-p`, and refactor such that `null_gen(..., 1 - null_probability);`. I suspect that this reversed logic is the artefact of multiple refactoring.

##########
File path: cpp/src/arrow/compute/kernels/util_internal.cc
##########
@@ -0,0 +1,61 @@
+// 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/kernels/util_internal.h"
+
+#include <cstdint>
+
+#include "arrow/array/data.h"
+#include "arrow/type.h"
+#include "arrow/util/checked_cast.h"
+
+namespace arrow {
+
+using internal::checked_cast;
+
+namespace compute {
+namespace internal {
+
+const uint8_t* GetValidityBitmap(const ArrayData& data) {
+  const uint8_t* bitmap = nullptr;
+  if (data.buffers[0]) {
+    bitmap = data.buffers[0]->data();
+  }
+  return bitmap;
+}
+
+int GetBitWidth(const DataType& type) {
+  return checked_cast<const FixedWidthType&>(type).bit_width();
+}
+
+PrimitiveArg GetPrimitiveArg(const ArrayData& arr) {
+  PrimitiveArg arg;
+  arg.is_valid = GetValidityBitmap(arr);
+  arg.data = arr.buffers[1]->data();
+  arg.bit_width = GetBitWidth(*arr.type);
+  arg.offset = arr.offset;
+  arg.length = arr.length;
+  if (arg.bit_width > 1) {
+    arg.data += arr.offset * arg.bit_width / 8;
+  }
+  arg.null_count = arr.GetNullCount();

Review comment:
       Do you want this to be always computed?

##########
File path: cpp/src/arrow/compute/kernels/vector_selection.cc
##########
@@ -0,0 +1,1816 @@
+// 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 <algorithm>
+#include <cstring>
+#include <limits>
+
+#include "arrow/array/array_base.h"
+#include "arrow/array/array_binary.h"
+#include "arrow/array/array_dict.h"
+#include "arrow/array/array_nested.h"
+#include "arrow/array/builder_primitive.h"
+#include "arrow/array/concatenate.h"
+#include "arrow/buffer_builder.h"
+#include "arrow/compute/api_vector.h"
+#include "arrow/compute/kernels/common.h"
+#include "arrow/compute/kernels/util_internal.h"
+#include "arrow/extension_type.h"
+#include "arrow/record_batch.h"
+#include "arrow/result.h"
+#include "arrow/util/bit_block_counter.h"
+#include "arrow/util/bit_util.h"
+#include "arrow/util/bitmap_ops.h"
+#include "arrow/util/bitmap_reader.h"
+#include "arrow/util/int_util.h"
+
+namespace arrow {
+
+using internal::BinaryBitBlockCounter;
+using internal::BitBlockCount;
+using internal::BitBlockCounter;
+using internal::BitmapReader;
+using internal::CopyBitmap;
+using internal::GetArrayView;
+using internal::IndexBoundsCheck;
+using internal::OptionalBitBlockCounter;
+using internal::OptionalBitIndexer;
+
+namespace compute {
+namespace internal {
+
+int64_t GetFilterOutputSize(const ArrayData& filter,
+                            FilterOptions::NullSelectionBehavior null_selection) {
+  int64_t output_size = 0;
+  int64_t position = 0;
+  if (filter.GetNullCount() > 0) {
+    const uint8_t* filter_is_valid = filter.buffers[0]->data();
+    BinaryBitBlockCounter bit_counter(filter.buffers[1]->data(), filter.offset,
+                                      filter_is_valid, filter.offset, filter.length);
+    if (null_selection == FilterOptions::EMIT_NULL) {
+      while (position < filter.length) {
+        BitBlockCount block = bit_counter.NextOrNotWord();
+        output_size += block.popcount;
+        position += block.length;
+      }
+    } else {
+      while (position < filter.length) {
+        BitBlockCount block = bit_counter.NextAndWord();
+        output_size += block.popcount;
+        position += block.length;
+      }
+    }
+  } else {
+    // The filter has no nulls, so we plow through its data as fast as
+    // possible.
+    BitBlockCounter bit_counter(filter.buffers[1]->data(), filter.offset, filter.length);
+    while (position < filter.length) {
+      BitBlockCount block = bit_counter.NextFourWords();
+      output_size += block.popcount;
+      position += block.length;
+    }
+  }
+  return output_size;
+}
+
+template <typename IndexType>
+Result<std::shared_ptr<ArrayData>> GetTakeIndicesImpl(
+    const ArrayData& filter, FilterOptions::NullSelectionBehavior null_selection,
+    MemoryPool* memory_pool) {
+  using T = typename IndexType::c_type;
+  typename TypeTraits<IndexType>::BuilderType builder(memory_pool);
+
+  const uint8_t* filter_data = filter.buffers[1]->data();
+  BitBlockCounter data_counter(filter_data, filter.offset, filter.length);
+
+  // The position relative to the start of the filter
+  T position = 0;
+
+  // The current position taking the filter offset into account
+  int64_t position_with_offset = filter.offset;
+  if (filter.GetNullCount() > 0) {
+    // The filter has nulls, so we scan the validity bitmap and the filter data
+    // bitmap together, branching on the null selection type.
+    const uint8_t* filter_is_valid = filter.buffers[0]->data();
+
+    // To count blocks whether filter_data[i] || !filter_is_valid[i]
+    BinaryBitBlockCounter filter_counter(filter_data, filter.offset, filter_is_valid,
+                                         filter.offset, filter.length);
+    if (null_selection == FilterOptions::DROP) {
+      while (position < filter.length) {
+        BitBlockCount and_block = filter_counter.NextAndWord();
+        RETURN_NOT_OK(builder.Reserve(and_block.popcount));
+        if (and_block.AllSet()) {
+          // All the values are selected and non-null
+          for (int64_t i = 0; i < and_block.length; ++i) {
+            builder.UnsafeAppend(position++);
+          }
+          position_with_offset += and_block.length;
+        } else {
+          // Some of the values are false or null
+          for (int64_t i = 0; i < and_block.length; ++i) {
+            if (BitUtil::GetBit(filter_is_valid, position_with_offset) &&
+                BitUtil::GetBit(filter_data, position_with_offset)) {
+              builder.UnsafeAppend(position);
+            }
+            ++position;
+            ++position_with_offset;
+          }
+        }
+      }
+    } else {
+      BitBlockCounter is_valid_counter(filter_is_valid, filter.offset, filter.length);
+      while (position < filter.length) {
+        // true OR NOT valid
+        BitBlockCount or_not_block = filter_counter.NextOrNotWord();
+        RETURN_NOT_OK(builder.Reserve(or_not_block.popcount));
+
+        // If the values are all valid and the or_not_block is full, then we
+        // can infer that all the values are true and skip the bit checking
+        BitBlockCount is_valid_block = is_valid_counter.NextWord();
+
+        if (or_not_block.AllSet() && is_valid_block.AllSet()) {
+          // All the values are selected and non-null
+          for (int64_t i = 0; i < or_not_block.length; ++i) {
+            builder.UnsafeAppend(position++);
+          }
+          position_with_offset += or_not_block.length;
+        } else {
+          // Some of the values are false or null
+          for (int64_t i = 0; i < or_not_block.length; ++i) {
+            if (BitUtil::GetBit(filter_is_valid, position_with_offset)) {
+              if (BitUtil::GetBit(filter_data, position_with_offset)) {
+                builder.UnsafeAppend(position);
+              }
+            } else {
+              // Null slot, so append a null
+              builder.UnsafeAppendNull();
+            }
+            ++position;
+            ++position_with_offset;
+          }
+        }
+      }
+    }
+  } else {
+    // The filter has no nulls, so we need only look for true values
+    BitBlockCount current_block = data_counter.NextWord();
+    while (position < filter.length) {
+      if (current_block.AllSet()) {
+        int64_t run_length = 0;
+
+        // If we've found a all-true block, then we scan forward until we find
+        // a block that has some false values (or we reach the end
+        while (current_block.length > 0 && current_block.AllSet()) {
+          run_length += current_block.length;
+          current_block = data_counter.NextWord();
+        }
+
+        // Append the consecutive run of indices
+        RETURN_NOT_OK(builder.Reserve(run_length));
+        for (int64_t i = 0; i < run_length; ++i) {
+          builder.UnsafeAppend(position++);
+        }
+        position_with_offset += run_length;
+      } else {
+        // Must do bitchecking on the current block
+        RETURN_NOT_OK(builder.Reserve(current_block.popcount));
+        for (int64_t i = 0; i < current_block.length; ++i) {
+          if (BitUtil::GetBit(filter_data, position_with_offset)) {
+            builder.UnsafeAppend(position);
+          }
+          ++position;
+          ++position_with_offset;
+        }
+        current_block = data_counter.NextWord();
+      }
+    }
+  }
+  std::shared_ptr<ArrayData> result;
+  RETURN_NOT_OK(builder.FinishInternal(&result));
+  return result;
+}
+
+Result<std::shared_ptr<ArrayData>> GetTakeIndices(
+    const ArrayData& filter, FilterOptions::NullSelectionBehavior null_selection,
+    MemoryPool* memory_pool) {
+  DCHECK_EQ(filter.type->id(), Type::BOOL);
+  if (filter.length <= std::numeric_limits<uint16_t>::max()) {
+    return GetTakeIndicesImpl<UInt16Type>(filter, null_selection, memory_pool);
+  } else if (filter.length <= std::numeric_limits<uint32_t>::max()) {
+    return GetTakeIndicesImpl<UInt32Type>(filter, null_selection, memory_pool);
+  } else {
+    // Arrays over 4 billion elements, not especially likely.
+    return Status::NotImplemented(
+        "Filter length exceeds UINT32_MAX, "
+        "consider a different strategy for selecting elements");
+  }
+}
+
+namespace {
+
+template <typename ArrowType>
+struct GetCType {
+  using type = typename ArrowType::c_type;
+};
+
+// We want uint8_t for boolean instead of bool
+template <>
+struct GetCType<BooleanType> {
+  using type = uint8_t;
+};
+
+using FilterState = OptionsWrapper<FilterOptions>;
+using TakeState = OptionsWrapper<TakeOptions>;
+
+Status PreallocateData(KernelContext* ctx, int64_t length, int bit_width, Datum* out) {
+  // Preallocate memory
+  ArrayData* out_arr = out->mutable_array();
+  out_arr->length = length;
+  out_arr->buffers.resize(2);
+
+  ARROW_ASSIGN_OR_RAISE(out_arr->buffers[0], ctx->AllocateBitmap(length));
+  if (bit_width == 1) {
+    ARROW_ASSIGN_OR_RAISE(out_arr->buffers[1], ctx->AllocateBitmap(length));
+  } else {
+    ARROW_ASSIGN_OR_RAISE(out_arr->buffers[1], ctx->Allocate(length * bit_width / 8));
+  }
+  return Status::OK();
+}
+
+// ----------------------------------------------------------------------
+// Implement optimized take for primitive types from boolean to 1/2/4/8-byte
+// C-type based types. Use common implementation for every byte width and only
+// generate code for unsigned integer indices, since after boundschecking to
+// check for negative numbers in the indices we can safely reinterpret_cast
+// signed integers as unsigned.
+
+/// \brief The Take implementation for primitive (fixed-width) types does not
+/// use the logical Arrow type but rather the physical C type. This way we
+/// only generate one take function for each byte width.
+///
+/// This function assumes that the indices have been boundschecked.
+template <typename IndexCType, typename ValueCType>
+struct PrimitiveTakeImpl {
+  static void Exec(const PrimitiveArg& values, const PrimitiveArg& indices,
+                   Datum* out_datum) {
+    auto values_data = reinterpret_cast<const ValueCType*>(values.data);
+    auto values_is_valid = values.is_valid;
+    auto values_offset = values.offset;
+
+    auto indices_data = reinterpret_cast<const IndexCType*>(indices.data);
+    auto indices_is_valid = indices.is_valid;
+    auto indices_offset = indices.offset;
+
+    ArrayData* out_arr = out_datum->mutable_array();
+    auto out = out_arr->GetMutableValues<ValueCType>(1);
+    auto out_is_valid = out_arr->buffers[0]->mutable_data();
+    auto out_offset = out_arr->offset;
+
+    // If either the values or indices have nulls, we preemptively zero out the
+    // out validity bitmap so that we don't have to use ClearBit in each
+    // iteration for nulls.
+    if (values.null_count > 0 || indices.null_count > 0) {
+      BitUtil::SetBitsTo(out_is_valid, out_offset, indices.length, false);
+    }
+
+    OptionalBitBlockCounter indices_bit_counter(indices_is_valid, indices_offset,
+                                                indices.length);
+    int64_t position = 0;
+    int64_t valid_count = 0;
+    while (position < indices.length) {
+      BitBlockCount block = indices_bit_counter.NextBlock();
+      if (values.null_count == 0) {
+        // Values are never null, so things are easier
+        valid_count += block.popcount;
+        if (block.popcount == block.length) {
+          // Fastest path: neither values nor index nulls
+          BitUtil::SetBitsTo(out_is_valid, out_offset + position, block.length, true);
+          for (int64_t i = 0; i < block.length; ++i) {
+            out[position] = values_data[indices_data[position]];
+            ++position;
+          }
+        } else if (block.popcount > 0) {
+          // Slow path: some indices but not all are null
+          for (int64_t i = 0; i < block.length; ++i) {
+            if (BitUtil::GetBit(indices_is_valid, indices_offset + position)) {
+              // index is not null
+              BitUtil::SetBit(out_is_valid, out_offset + position);
+              out[position] = values_data[indices_data[position]];
+            } else {
+              out[position] = ValueCType{};
+            }
+            ++position;
+          }
+        } else {
+          memset(out + position, 0, sizeof(ValueCType) * block.length);
+          position += block.length;
+        }
+      } else {
+        // Values have nulls, so we must do random access into the values bitmap
+        if (block.popcount == block.length) {
+          // Faster path: indices are not null but values may be
+          for (int64_t i = 0; i < block.length; ++i) {
+            if (BitUtil::GetBit(values_is_valid,
+                                values_offset + indices_data[position])) {
+              // value is not null
+              out[position] = values_data[indices_data[position]];
+              BitUtil::SetBit(out_is_valid, out_offset + position);
+              ++valid_count;
+            } else {
+              out[position] = ValueCType{};
+            }
+            ++position;
+          }
+        } else if (block.popcount > 0) {
+          // Slow path: some but not all indices are null. Since we are doing
+          // random access in general we have to check the value nullness one by
+          // one.
+          for (int64_t i = 0; i < block.length; ++i) {
+            if (BitUtil::GetBit(indices_is_valid, indices_offset + position) &&
+                BitUtil::GetBit(values_is_valid,
+                                values_offset + indices_data[position])) {
+              // index is not null && value is not null
+              out[position] = values_data[indices_data[position]];
+              BitUtil::SetBit(out_is_valid, out_offset + position);
+              ++valid_count;
+            } else {
+              out[position] = ValueCType{};
+            }
+            ++position;
+          }
+        } else {
+          memset(out + position, 0, sizeof(ValueCType) * block.length);
+          position += block.length;
+        }
+      }
+    }
+    out_arr->null_count = out_arr->length - valid_count;
+  }
+};
+
+template <typename IndexCType>
+struct BooleanTakeImpl {
+  static void Exec(const PrimitiveArg& values, const PrimitiveArg& indices,
+                   Datum* out_datum) {
+    const uint8_t* values_data = values.data;
+    auto values_is_valid = values.is_valid;
+    auto values_offset = values.offset;
+
+    auto indices_data = reinterpret_cast<const IndexCType*>(indices.data);
+    auto indices_is_valid = indices.is_valid;
+    auto indices_offset = indices.offset;
+
+    ArrayData* out_arr = out_datum->mutable_array();
+    auto out = out_arr->buffers[1]->mutable_data();
+    auto out_is_valid = out_arr->buffers[0]->mutable_data();
+    auto out_offset = out_arr->offset;
+
+    // If either the values or indices have nulls, we preemptively zero out the
+    // out validity bitmap so that we don't have to use ClearBit in each
+    // iteration for nulls.
+    if (values.null_count > 0 || indices.null_count > 0) {
+      BitUtil::SetBitsTo(out_is_valid, out_offset, indices.length, false);
+    }
+    // Avoid uninitialized data in values array
+    BitUtil::SetBitsTo(out, out_offset, indices.length, false);
+
+    auto PlaceDataBit = [&](int64_t loc, IndexCType index) {
+      BitUtil::SetBitTo(out, out_offset + loc,
+                        BitUtil::GetBit(values_data, values_offset + index));
+    };
+
+    OptionalBitBlockCounter indices_bit_counter(indices_is_valid, indices_offset,
+                                                indices.length);
+    int64_t position = 0;
+    int64_t valid_count = 0;
+    while (position < indices.length) {
+      BitBlockCount block = indices_bit_counter.NextBlock();
+      if (values.null_count == 0) {
+        // Values are never null, so things are easier
+        valid_count += block.popcount;
+        if (block.popcount == block.length) {
+          // Fastest path: neither values nor index nulls
+          BitUtil::SetBitsTo(out_is_valid, out_offset + position, block.length, true);
+          for (int64_t i = 0; i < block.length; ++i) {
+            PlaceDataBit(position, indices_data[position]);
+            ++position;
+          }
+        } else if (block.popcount > 0) {
+          // Slow path: some but not all indices are null
+          for (int64_t i = 0; i < block.length; ++i) {
+            if (BitUtil::GetBit(indices_is_valid, indices_offset + position)) {
+              // index is not null
+              BitUtil::SetBit(out_is_valid, out_offset + position);
+              PlaceDataBit(position, indices_data[position]);
+            }
+            ++position;
+          }
+        } else {
+          position += block.length;
+        }
+      } else {
+        // Values have nulls, so we must do random access into the values bitmap
+        if (block.popcount == block.length) {
+          // Faster path: indices are not null but values may be
+          for (int64_t i = 0; i < block.length; ++i) {
+            if (BitUtil::GetBit(values_is_valid,
+                                values_offset + indices_data[position])) {
+              // value is not null
+              BitUtil::SetBit(out_is_valid, out_offset + position);
+              PlaceDataBit(position, indices_data[position]);
+              ++valid_count;
+            }
+            ++position;
+          }
+        } else if (block.popcount > 0) {
+          // Slow path: some but not all indices are null. Since we are doing
+          // random access in general we have to check the value nullness one by
+          // one.
+          for (int64_t i = 0; i < block.length; ++i) {
+            if (BitUtil::GetBit(indices_is_valid, indices_offset + position)) {
+              // index is not null
+              if (BitUtil::GetBit(values_is_valid,
+                                  values_offset + indices_data[position])) {
+                // value is not null
+                PlaceDataBit(position, indices_data[position]);
+                BitUtil::SetBit(out_is_valid, out_offset + position);
+                ++valid_count;
+              }
+            }
+            ++position;
+          }
+        } else {
+          position += block.length;
+        }
+      }
+    }
+    out_arr->null_count = out_arr->length - valid_count;
+  }
+};
+
+template <template <typename...> class TakeImpl, typename... Args>
+void TakeIndexDispatch(const PrimitiveArg& values, const PrimitiveArg& indices,
+                       Datum* out) {
+  // With the simplifying assumption that boundschecking has taken place
+  // already at a higher level, we can now assume that the index values are all
+  // non-negative. Thus, we can interpret signed integers as unsigned and avoid
+  // having to generate double the amount of binary code to handle each integer
+  // width.
+  switch (indices.bit_width) {
+    case 8:
+      return TakeImpl<uint8_t, Args...>::Exec(values, indices, out);
+    case 16:
+      return TakeImpl<uint16_t, Args...>::Exec(values, indices, out);
+    case 32:
+      return TakeImpl<uint32_t, Args...>::Exec(values, indices, out);
+    case 64:
+      return TakeImpl<uint64_t, Args...>::Exec(values, indices, out);
+    default:
+      DCHECK(false) << "Invalid indices byte width";
+      break;
+  }
+}
+
+void PrimitiveTake(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+  const auto& state = checked_cast<const TakeState&>(*ctx->state());
+  if (state.options.boundscheck) {
+    KERNEL_RETURN_IF_ERROR(ctx, IndexBoundsCheck(*batch[1].array(), batch[0].length()));
+  }
+
+  PrimitiveArg values = GetPrimitiveArg(*batch[0].array());
+  PrimitiveArg indices = GetPrimitiveArg(*batch[1].array());
+  KERNEL_RETURN_IF_ERROR(ctx,
+                         PreallocateData(ctx, indices.length, values.bit_width, out));
+  switch (values.bit_width) {
+    case 1:
+      return TakeIndexDispatch<BooleanTakeImpl>(values, indices, out);
+    case 8:
+      return TakeIndexDispatch<PrimitiveTakeImpl, int8_t>(values, indices, out);
+    case 16:
+      return TakeIndexDispatch<PrimitiveTakeImpl, int16_t>(values, indices, out);
+    case 32:
+      return TakeIndexDispatch<PrimitiveTakeImpl, int32_t>(values, indices, out);
+    case 64:
+      return TakeIndexDispatch<PrimitiveTakeImpl, int64_t>(values, indices, out);
+    default:
+      DCHECK(false) << "Invalid values byte width";
+      break;
+  }
+}
+
+// ----------------------------------------------------------------------
+// Optimized and streamlined filter for primitive types
+
+// Use either BitBlockCounter or BinaryBitBlockCounter to quickly scan filter a
+// word at a time for the DROP selection type.
+class DropNullCounter {
+ public:
+  // validity bitmap may be null
+  DropNullCounter(const uint8_t* validity, const uint8_t* data, int64_t offset,
+                  int64_t length)
+      : data_counter_(data, offset, length),
+        data_and_validity_counter_(data, offset, validity, offset, length),
+        has_validity_(validity != nullptr) {}
+
+  BitBlockCount NextBlock() {
+    if (has_validity_) {
+      // filter is true AND not null
+      return data_and_validity_counter_.NextAndWord();
+    } else {
+      return data_counter_.NextWord();
+    }
+  }
+
+ private:
+  // For when just data is present, but no validity bitmap
+  BitBlockCounter data_counter_;
+
+  // For when both validity bitmap and data are present
+  BinaryBitBlockCounter data_and_validity_counter_;
+  bool has_validity_;
+};
+
+/// \brief The Filter implementation for primitive (fixed-width) types does not
+/// use the logical Arrow type but rather then physical C type. This way we
+/// only generate one take function for each byte width. We use the same
+/// implementation here for boolean and fixed-byte-size inputs with some
+/// template specialization.
+template <typename ArrowType>
+class PrimitiveFilterImpl {
+ public:
+  using T = typename GetCType<ArrowType>::type;
+
+  PrimitiveFilterImpl(const PrimitiveArg& values, const PrimitiveArg& filter,
+                      FilterOptions::NullSelectionBehavior null_selection,
+                      Datum* out_datum)
+      : values_is_valid_(values.is_valid),
+        values_data_(reinterpret_cast<const T*>(values.data)),
+        values_null_count_(values.null_count),
+        values_offset_(values.offset),
+        values_length_(values.length),
+        filter_is_valid_(filter.is_valid),
+        filter_data_(filter.data),
+        filter_null_count_(filter.null_count),
+        filter_offset_(filter.offset),
+        null_selection_(null_selection) {
+    ArrayData* out_arr = out_datum->mutable_array();
+    out_is_valid_ = out_arr->buffers[0]->mutable_data();
+    out_data_ = reinterpret_cast<T*>(out_arr->buffers[1]->mutable_data());
+    out_offset_ = out_arr->offset;
+    out_length_ = out_arr->length;
+    out_position_ = 0;
+  }
+
+  void ExecNonNull() {
+    // The result is all not-null
+    BitUtil::SetBitsTo(out_is_valid_, out_offset_ + out_position_, out_length_, true);
+
+    // Fast filter when values and filter are not null
+    // Bit counters used for both null_selection behaviors
+    BitBlockCounter filter_counter(filter_data_, filter_offset_, values_length_);
+
+    int64_t in_position = 0;
+    BitBlockCount current_block = filter_counter.NextWord();
+    while (in_position < values_length_) {
+      if (current_block.AllSet()) {
+        int64_t run_length = 0;
+        // If we've found a all-true block, then we scan forward until we find

Review comment:
       For reference, you essentially re-implemented a very close form of EWAH bitmap encoding.

##########
File path: cpp/src/arrow/compute/kernels/vector_selection.cc
##########
@@ -0,0 +1,1816 @@
+// 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 <algorithm>
+#include <cstring>
+#include <limits>
+
+#include "arrow/array/array_base.h"
+#include "arrow/array/array_binary.h"
+#include "arrow/array/array_dict.h"
+#include "arrow/array/array_nested.h"
+#include "arrow/array/builder_primitive.h"
+#include "arrow/array/concatenate.h"
+#include "arrow/buffer_builder.h"
+#include "arrow/compute/api_vector.h"
+#include "arrow/compute/kernels/common.h"
+#include "arrow/compute/kernels/util_internal.h"
+#include "arrow/extension_type.h"
+#include "arrow/record_batch.h"
+#include "arrow/result.h"
+#include "arrow/util/bit_block_counter.h"
+#include "arrow/util/bit_util.h"
+#include "arrow/util/bitmap_ops.h"
+#include "arrow/util/bitmap_reader.h"
+#include "arrow/util/int_util.h"
+
+namespace arrow {
+
+using internal::BinaryBitBlockCounter;
+using internal::BitBlockCount;
+using internal::BitBlockCounter;
+using internal::BitmapReader;
+using internal::CopyBitmap;
+using internal::GetArrayView;
+using internal::IndexBoundsCheck;
+using internal::OptionalBitBlockCounter;
+using internal::OptionalBitIndexer;
+
+namespace compute {
+namespace internal {
+
+int64_t GetFilterOutputSize(const ArrayData& filter,
+                            FilterOptions::NullSelectionBehavior null_selection) {
+  int64_t output_size = 0;
+  int64_t position = 0;
+  if (filter.GetNullCount() > 0) {
+    const uint8_t* filter_is_valid = filter.buffers[0]->data();
+    BinaryBitBlockCounter bit_counter(filter.buffers[1]->data(), filter.offset,
+                                      filter_is_valid, filter.offset, filter.length);
+    if (null_selection == FilterOptions::EMIT_NULL) {
+      while (position < filter.length) {
+        BitBlockCount block = bit_counter.NextOrNotWord();
+        output_size += block.popcount;
+        position += block.length;
+      }
+    } else {
+      while (position < filter.length) {
+        BitBlockCount block = bit_counter.NextAndWord();
+        output_size += block.popcount;
+        position += block.length;
+      }
+    }
+  } else {
+    // The filter has no nulls, so we plow through its data as fast as
+    // possible.
+    BitBlockCounter bit_counter(filter.buffers[1]->data(), filter.offset, filter.length);
+    while (position < filter.length) {
+      BitBlockCount block = bit_counter.NextFourWords();
+      output_size += block.popcount;
+      position += block.length;
+    }
+  }
+  return output_size;
+}
+
+template <typename IndexType>
+Result<std::shared_ptr<ArrayData>> GetTakeIndicesImpl(
+    const ArrayData& filter, FilterOptions::NullSelectionBehavior null_selection,
+    MemoryPool* memory_pool) {
+  using T = typename IndexType::c_type;
+  typename TypeTraits<IndexType>::BuilderType builder(memory_pool);
+
+  const uint8_t* filter_data = filter.buffers[1]->data();
+  BitBlockCounter data_counter(filter_data, filter.offset, filter.length);
+
+  // The position relative to the start of the filter
+  T position = 0;
+
+  // The current position taking the filter offset into account
+  int64_t position_with_offset = filter.offset;
+  if (filter.GetNullCount() > 0) {
+    // The filter has nulls, so we scan the validity bitmap and the filter data
+    // bitmap together, branching on the null selection type.
+    const uint8_t* filter_is_valid = filter.buffers[0]->data();
+
+    // To count blocks whether filter_data[i] || !filter_is_valid[i]
+    BinaryBitBlockCounter filter_counter(filter_data, filter.offset, filter_is_valid,
+                                         filter.offset, filter.length);
+    if (null_selection == FilterOptions::DROP) {
+      while (position < filter.length) {
+        BitBlockCount and_block = filter_counter.NextAndWord();
+        RETURN_NOT_OK(builder.Reserve(and_block.popcount));
+        if (and_block.AllSet()) {
+          // All the values are selected and non-null
+          for (int64_t i = 0; i < and_block.length; ++i) {
+            builder.UnsafeAppend(position++);
+          }
+          position_with_offset += and_block.length;
+        } else {
+          // Some of the values are false or null
+          for (int64_t i = 0; i < and_block.length; ++i) {
+            if (BitUtil::GetBit(filter_is_valid, position_with_offset) &&
+                BitUtil::GetBit(filter_data, position_with_offset)) {
+              builder.UnsafeAppend(position);
+            }
+            ++position;
+            ++position_with_offset;
+          }
+        }
+      }
+    } else {
+      BitBlockCounter is_valid_counter(filter_is_valid, filter.offset, filter.length);
+      while (position < filter.length) {
+        // true OR NOT valid
+        BitBlockCount or_not_block = filter_counter.NextOrNotWord();
+        RETURN_NOT_OK(builder.Reserve(or_not_block.popcount));
+
+        // If the values are all valid and the or_not_block is full, then we
+        // can infer that all the values are true and skip the bit checking
+        BitBlockCount is_valid_block = is_valid_counter.NextWord();
+
+        if (or_not_block.AllSet() && is_valid_block.AllSet()) {
+          // All the values are selected and non-null
+          for (int64_t i = 0; i < or_not_block.length; ++i) {
+            builder.UnsafeAppend(position++);
+          }
+          position_with_offset += or_not_block.length;
+        } else {
+          // Some of the values are false or null
+          for (int64_t i = 0; i < or_not_block.length; ++i) {
+            if (BitUtil::GetBit(filter_is_valid, position_with_offset)) {
+              if (BitUtil::GetBit(filter_data, position_with_offset)) {
+                builder.UnsafeAppend(position);
+              }
+            } else {
+              // Null slot, so append a null
+              builder.UnsafeAppendNull();
+            }
+            ++position;
+            ++position_with_offset;
+          }
+        }
+      }
+    }
+  } else {
+    // The filter has no nulls, so we need only look for true values
+    BitBlockCount current_block = data_counter.NextWord();
+    while (position < filter.length) {
+      if (current_block.AllSet()) {
+        int64_t run_length = 0;
+
+        // If we've found a all-true block, then we scan forward until we find
+        // a block that has some false values (or we reach the end
+        while (current_block.length > 0 && current_block.AllSet()) {
+          run_length += current_block.length;
+          current_block = data_counter.NextWord();
+        }
+
+        // Append the consecutive run of indices
+        RETURN_NOT_OK(builder.Reserve(run_length));
+        for (int64_t i = 0; i < run_length; ++i) {
+          builder.UnsafeAppend(position++);
+        }
+        position_with_offset += run_length;
+      } else {
+        // Must do bitchecking on the current block
+        RETURN_NOT_OK(builder.Reserve(current_block.popcount));
+        for (int64_t i = 0; i < current_block.length; ++i) {
+          if (BitUtil::GetBit(filter_data, position_with_offset)) {
+            builder.UnsafeAppend(position);
+          }
+          ++position;
+          ++position_with_offset;
+        }
+        current_block = data_counter.NextWord();
+      }
+    }
+  }
+  std::shared_ptr<ArrayData> result;
+  RETURN_NOT_OK(builder.FinishInternal(&result));
+  return result;
+}
+
+Result<std::shared_ptr<ArrayData>> GetTakeIndices(
+    const ArrayData& filter, FilterOptions::NullSelectionBehavior null_selection,
+    MemoryPool* memory_pool) {
+  DCHECK_EQ(filter.type->id(), Type::BOOL);
+  if (filter.length <= std::numeric_limits<uint16_t>::max()) {
+    return GetTakeIndicesImpl<UInt16Type>(filter, null_selection, memory_pool);
+  } else if (filter.length <= std::numeric_limits<uint32_t>::max()) {
+    return GetTakeIndicesImpl<UInt32Type>(filter, null_selection, memory_pool);
+  } else {
+    // Arrays over 4 billion elements, not especially likely.
+    return Status::NotImplemented(
+        "Filter length exceeds UINT32_MAX, "
+        "consider a different strategy for selecting elements");
+  }
+}
+
+namespace {
+
+template <typename ArrowType>
+struct GetCType {
+  using type = typename ArrowType::c_type;
+};
+
+// We want uint8_t for boolean instead of bool
+template <>
+struct GetCType<BooleanType> {
+  using type = uint8_t;
+};
+
+using FilterState = OptionsWrapper<FilterOptions>;
+using TakeState = OptionsWrapper<TakeOptions>;
+
+Status PreallocateData(KernelContext* ctx, int64_t length, int bit_width, Datum* out) {
+  // Preallocate memory
+  ArrayData* out_arr = out->mutable_array();
+  out_arr->length = length;
+  out_arr->buffers.resize(2);
+
+  ARROW_ASSIGN_OR_RAISE(out_arr->buffers[0], ctx->AllocateBitmap(length));
+  if (bit_width == 1) {
+    ARROW_ASSIGN_OR_RAISE(out_arr->buffers[1], ctx->AllocateBitmap(length));
+  } else {
+    ARROW_ASSIGN_OR_RAISE(out_arr->buffers[1], ctx->Allocate(length * bit_width / 8));
+  }
+  return Status::OK();
+}
+
+// ----------------------------------------------------------------------
+// Implement optimized take for primitive types from boolean to 1/2/4/8-byte
+// C-type based types. Use common implementation for every byte width and only
+// generate code for unsigned integer indices, since after boundschecking to
+// check for negative numbers in the indices we can safely reinterpret_cast
+// signed integers as unsigned.
+
+/// \brief The Take implementation for primitive (fixed-width) types does not
+/// use the logical Arrow type but rather the physical C type. This way we
+/// only generate one take function for each byte width.
+///
+/// This function assumes that the indices have been boundschecked.
+template <typename IndexCType, typename ValueCType>
+struct PrimitiveTakeImpl {
+  static void Exec(const PrimitiveArg& values, const PrimitiveArg& indices,
+                   Datum* out_datum) {
+    auto values_data = reinterpret_cast<const ValueCType*>(values.data);
+    auto values_is_valid = values.is_valid;
+    auto values_offset = values.offset;
+
+    auto indices_data = reinterpret_cast<const IndexCType*>(indices.data);
+    auto indices_is_valid = indices.is_valid;
+    auto indices_offset = indices.offset;
+
+    ArrayData* out_arr = out_datum->mutable_array();
+    auto out = out_arr->GetMutableValues<ValueCType>(1);
+    auto out_is_valid = out_arr->buffers[0]->mutable_data();
+    auto out_offset = out_arr->offset;
+
+    // If either the values or indices have nulls, we preemptively zero out the
+    // out validity bitmap so that we don't have to use ClearBit in each
+    // iteration for nulls.
+    if (values.null_count > 0 || indices.null_count > 0) {
+      BitUtil::SetBitsTo(out_is_valid, out_offset, indices.length, false);
+    }
+
+    OptionalBitBlockCounter indices_bit_counter(indices_is_valid, indices_offset,
+                                                indices.length);
+    int64_t position = 0;
+    int64_t valid_count = 0;
+    while (position < indices.length) {
+      BitBlockCount block = indices_bit_counter.NextBlock();
+      if (values.null_count == 0) {
+        // Values are never null, so things are easier
+        valid_count += block.popcount;
+        if (block.popcount == block.length) {
+          // Fastest path: neither values nor index nulls
+          BitUtil::SetBitsTo(out_is_valid, out_offset + position, block.length, true);
+          for (int64_t i = 0; i < block.length; ++i) {
+            out[position] = values_data[indices_data[position]];
+            ++position;
+          }
+        } else if (block.popcount > 0) {
+          // Slow path: some indices but not all are null
+          for (int64_t i = 0; i < block.length; ++i) {
+            if (BitUtil::GetBit(indices_is_valid, indices_offset + position)) {
+              // index is not null
+              BitUtil::SetBit(out_is_valid, out_offset + position);
+              out[position] = values_data[indices_data[position]];
+            } else {
+              out[position] = ValueCType{};
+            }
+            ++position;
+          }
+        } else {
+          memset(out + position, 0, sizeof(ValueCType) * block.length);
+          position += block.length;
+        }
+      } else {
+        // Values have nulls, so we must do random access into the values bitmap
+        if (block.popcount == block.length) {
+          // Faster path: indices are not null but values may be
+          for (int64_t i = 0; i < block.length; ++i) {
+            if (BitUtil::GetBit(values_is_valid,
+                                values_offset + indices_data[position])) {
+              // value is not null
+              out[position] = values_data[indices_data[position]];
+              BitUtil::SetBit(out_is_valid, out_offset + position);
+              ++valid_count;
+            } else {
+              out[position] = ValueCType{};
+            }
+            ++position;
+          }
+        } else if (block.popcount > 0) {
+          // Slow path: some but not all indices are null. Since we are doing
+          // random access in general we have to check the value nullness one by
+          // one.
+          for (int64_t i = 0; i < block.length; ++i) {
+            if (BitUtil::GetBit(indices_is_valid, indices_offset + position) &&
+                BitUtil::GetBit(values_is_valid,
+                                values_offset + indices_data[position])) {
+              // index is not null && value is not null
+              out[position] = values_data[indices_data[position]];
+              BitUtil::SetBit(out_is_valid, out_offset + position);
+              ++valid_count;
+            } else {
+              out[position] = ValueCType{};
+            }
+            ++position;
+          }
+        } else {
+          memset(out + position, 0, sizeof(ValueCType) * block.length);
+          position += block.length;
+        }
+      }
+    }
+    out_arr->null_count = out_arr->length - valid_count;
+  }
+};
+
+template <typename IndexCType>
+struct BooleanTakeImpl {
+  static void Exec(const PrimitiveArg& values, const PrimitiveArg& indices,
+                   Datum* out_datum) {
+    const uint8_t* values_data = values.data;
+    auto values_is_valid = values.is_valid;
+    auto values_offset = values.offset;
+
+    auto indices_data = reinterpret_cast<const IndexCType*>(indices.data);
+    auto indices_is_valid = indices.is_valid;
+    auto indices_offset = indices.offset;
+
+    ArrayData* out_arr = out_datum->mutable_array();
+    auto out = out_arr->buffers[1]->mutable_data();
+    auto out_is_valid = out_arr->buffers[0]->mutable_data();
+    auto out_offset = out_arr->offset;
+
+    // If either the values or indices have nulls, we preemptively zero out the
+    // out validity bitmap so that we don't have to use ClearBit in each
+    // iteration for nulls.
+    if (values.null_count > 0 || indices.null_count > 0) {
+      BitUtil::SetBitsTo(out_is_valid, out_offset, indices.length, false);
+    }
+    // Avoid uninitialized data in values array
+    BitUtil::SetBitsTo(out, out_offset, indices.length, false);
+
+    auto PlaceDataBit = [&](int64_t loc, IndexCType index) {
+      BitUtil::SetBitTo(out, out_offset + loc,
+                        BitUtil::GetBit(values_data, values_offset + index));
+    };
+
+    OptionalBitBlockCounter indices_bit_counter(indices_is_valid, indices_offset,
+                                                indices.length);
+    int64_t position = 0;
+    int64_t valid_count = 0;
+    while (position < indices.length) {
+      BitBlockCount block = indices_bit_counter.NextBlock();
+      if (values.null_count == 0) {
+        // Values are never null, so things are easier
+        valid_count += block.popcount;
+        if (block.popcount == block.length) {
+          // Fastest path: neither values nor index nulls
+          BitUtil::SetBitsTo(out_is_valid, out_offset + position, block.length, true);
+          for (int64_t i = 0; i < block.length; ++i) {
+            PlaceDataBit(position, indices_data[position]);
+            ++position;
+          }
+        } else if (block.popcount > 0) {
+          // Slow path: some but not all indices are null
+          for (int64_t i = 0; i < block.length; ++i) {
+            if (BitUtil::GetBit(indices_is_valid, indices_offset + position)) {
+              // index is not null
+              BitUtil::SetBit(out_is_valid, out_offset + position);
+              PlaceDataBit(position, indices_data[position]);
+            }
+            ++position;
+          }
+        } else {
+          position += block.length;
+        }
+      } else {
+        // Values have nulls, so we must do random access into the values bitmap
+        if (block.popcount == block.length) {
+          // Faster path: indices are not null but values may be
+          for (int64_t i = 0; i < block.length; ++i) {
+            if (BitUtil::GetBit(values_is_valid,
+                                values_offset + indices_data[position])) {
+              // value is not null
+              BitUtil::SetBit(out_is_valid, out_offset + position);
+              PlaceDataBit(position, indices_data[position]);
+              ++valid_count;
+            }
+            ++position;
+          }
+        } else if (block.popcount > 0) {
+          // Slow path: some but not all indices are null. Since we are doing
+          // random access in general we have to check the value nullness one by
+          // one.
+          for (int64_t i = 0; i < block.length; ++i) {
+            if (BitUtil::GetBit(indices_is_valid, indices_offset + position)) {
+              // index is not null
+              if (BitUtil::GetBit(values_is_valid,
+                                  values_offset + indices_data[position])) {
+                // value is not null
+                PlaceDataBit(position, indices_data[position]);
+                BitUtil::SetBit(out_is_valid, out_offset + position);
+                ++valid_count;
+              }
+            }
+            ++position;
+          }
+        } else {
+          position += block.length;
+        }
+      }
+    }
+    out_arr->null_count = out_arr->length - valid_count;
+  }
+};
+
+template <template <typename...> class TakeImpl, typename... Args>
+void TakeIndexDispatch(const PrimitiveArg& values, const PrimitiveArg& indices,
+                       Datum* out) {
+  // With the simplifying assumption that boundschecking has taken place
+  // already at a higher level, we can now assume that the index values are all
+  // non-negative. Thus, we can interpret signed integers as unsigned and avoid
+  // having to generate double the amount of binary code to handle each integer
+  // width.
+  switch (indices.bit_width) {
+    case 8:
+      return TakeImpl<uint8_t, Args...>::Exec(values, indices, out);
+    case 16:
+      return TakeImpl<uint16_t, Args...>::Exec(values, indices, out);
+    case 32:
+      return TakeImpl<uint32_t, Args...>::Exec(values, indices, out);
+    case 64:
+      return TakeImpl<uint64_t, Args...>::Exec(values, indices, out);
+    default:
+      DCHECK(false) << "Invalid indices byte width";
+      break;
+  }
+}
+
+void PrimitiveTake(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+  const auto& state = checked_cast<const TakeState&>(*ctx->state());
+  if (state.options.boundscheck) {
+    KERNEL_RETURN_IF_ERROR(ctx, IndexBoundsCheck(*batch[1].array(), batch[0].length()));
+  }
+
+  PrimitiveArg values = GetPrimitiveArg(*batch[0].array());
+  PrimitiveArg indices = GetPrimitiveArg(*batch[1].array());
+  KERNEL_RETURN_IF_ERROR(ctx,
+                         PreallocateData(ctx, indices.length, values.bit_width, out));
+  switch (values.bit_width) {
+    case 1:
+      return TakeIndexDispatch<BooleanTakeImpl>(values, indices, out);
+    case 8:
+      return TakeIndexDispatch<PrimitiveTakeImpl, int8_t>(values, indices, out);
+    case 16:
+      return TakeIndexDispatch<PrimitiveTakeImpl, int16_t>(values, indices, out);
+    case 32:
+      return TakeIndexDispatch<PrimitiveTakeImpl, int32_t>(values, indices, out);
+    case 64:
+      return TakeIndexDispatch<PrimitiveTakeImpl, int64_t>(values, indices, out);
+    default:
+      DCHECK(false) << "Invalid values byte width";
+      break;
+  }
+}
+
+// ----------------------------------------------------------------------
+// Optimized and streamlined filter for primitive types
+
+// Use either BitBlockCounter or BinaryBitBlockCounter to quickly scan filter a
+// word at a time for the DROP selection type.
+class DropNullCounter {
+ public:
+  // validity bitmap may be null
+  DropNullCounter(const uint8_t* validity, const uint8_t* data, int64_t offset,
+                  int64_t length)
+      : data_counter_(data, offset, length),
+        data_and_validity_counter_(data, offset, validity, offset, length),
+        has_validity_(validity != nullptr) {}
+
+  BitBlockCount NextBlock() {
+    if (has_validity_) {
+      // filter is true AND not null
+      return data_and_validity_counter_.NextAndWord();
+    } else {
+      return data_counter_.NextWord();
+    }
+  }
+
+ private:
+  // For when just data is present, but no validity bitmap
+  BitBlockCounter data_counter_;
+
+  // For when both validity bitmap and data are present
+  BinaryBitBlockCounter data_and_validity_counter_;
+  bool has_validity_;
+};
+
+/// \brief The Filter implementation for primitive (fixed-width) types does not
+/// use the logical Arrow type but rather then physical C type. This way we
+/// only generate one take function for each byte width. We use the same
+/// implementation here for boolean and fixed-byte-size inputs with some
+/// template specialization.
+template <typename ArrowType>
+class PrimitiveFilterImpl {
+ public:
+  using T = typename GetCType<ArrowType>::type;
+
+  PrimitiveFilterImpl(const PrimitiveArg& values, const PrimitiveArg& filter,
+                      FilterOptions::NullSelectionBehavior null_selection,
+                      Datum* out_datum)
+      : values_is_valid_(values.is_valid),
+        values_data_(reinterpret_cast<const T*>(values.data)),
+        values_null_count_(values.null_count),
+        values_offset_(values.offset),
+        values_length_(values.length),
+        filter_is_valid_(filter.is_valid),
+        filter_data_(filter.data),
+        filter_null_count_(filter.null_count),
+        filter_offset_(filter.offset),
+        null_selection_(null_selection) {
+    ArrayData* out_arr = out_datum->mutable_array();
+    out_is_valid_ = out_arr->buffers[0]->mutable_data();
+    out_data_ = reinterpret_cast<T*>(out_arr->buffers[1]->mutable_data());
+    out_offset_ = out_arr->offset;
+    out_length_ = out_arr->length;
+    out_position_ = 0;
+  }
+
+  void ExecNonNull() {
+    // The result is all not-null
+    BitUtil::SetBitsTo(out_is_valid_, out_offset_ + out_position_, out_length_, true);
+
+    // Fast filter when values and filter are not null
+    // Bit counters used for both null_selection behaviors
+    BitBlockCounter filter_counter(filter_data_, filter_offset_, values_length_);
+
+    int64_t in_position = 0;
+    BitBlockCount current_block = filter_counter.NextWord();
+    while (in_position < values_length_) {
+      if (current_block.AllSet()) {
+        int64_t run_length = 0;
+        // If we've found a all-true block, then we scan forward until we find
+        // a block that has some false values (or we reach the end
+        while (current_block.length > 0 && current_block.AllSet()) {
+          run_length += current_block.length;
+          current_block = filter_counter.NextWord();
+        }
+        WriteValueSegment(in_position, run_length);
+        in_position += run_length;
+      } else if (current_block.NoneSet()) {
+        // Nothing selected
+        in_position += current_block.length;
+        current_block = filter_counter.NextWord();
+      } else {
+        // Some values selected
+        for (int64_t i = 0; i < current_block.length; ++i) {
+          if (BitUtil::GetBit(filter_data_, filter_offset_ + in_position)) {
+            WriteValue(in_position);
+          }
+          ++in_position;
+        }
+        current_block = filter_counter.NextWord();
+      }
+    }
+  }
+
+  void Exec() {
+    if (filter_null_count_ == 0 && values_null_count_ == 0) {
+      return ExecNonNull();
+    }
+
+    // Bit counters used for both null_selection behaviors
+    DropNullCounter drop_null_counter(filter_is_valid_, filter_data_, filter_offset_,
+                                      values_length_);
+    OptionalBitBlockCounter data_counter(values_is_valid_, values_offset_,
+                                         values_length_);
+    OptionalBitBlockCounter filter_valid_counter(filter_is_valid_, filter_offset_,
+                                                 values_length_);
+
+    auto WriteNotNull = [&](int64_t index) {
+      BitUtil::SetBit(out_is_valid_, out_offset_ + out_position_);
+      // Increments out_position_
+      WriteValue(index);
+    };
+
+    auto WriteMaybeNull = [&](int64_t index) {
+      BitUtil::SetBitTo(out_is_valid_, out_offset_ + out_position_,
+                        BitUtil::GetBit(values_is_valid_, values_offset_ + index));
+      // Increments out_position_
+      WriteValue(index);
+    };
+
+    int64_t in_position = 0;
+    while (in_position < values_length_) {

Review comment:
       This is very hard to read and review, the nesting is deep and tracking of loops invariant is almost impossible.




----------------------------------------------------------------
This is an automated message from the 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] wesm commented on a change in pull request #7442: ARROW-9075: [C++] Optimized Filter implementation: faster performance + compilation, smaller code size

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



##########
File path: cpp/src/arrow/compute/kernels/util_internal.cc
##########
@@ -0,0 +1,61 @@
+// 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/kernels/util_internal.h"
+
+#include <cstdint>
+
+#include "arrow/array/data.h"
+#include "arrow/type.h"
+#include "arrow/util/checked_cast.h"
+
+namespace arrow {
+
+using internal::checked_cast;
+
+namespace compute {
+namespace internal {
+
+const uint8_t* GetValidityBitmap(const ArrayData& data) {
+  const uint8_t* bitmap = nullptr;
+  if (data.buffers[0]) {
+    bitmap = data.buffers[0]->data();
+  }
+  return bitmap;
+}
+
+int GetBitWidth(const DataType& type) {
+  return checked_cast<const FixedWidthType&>(type).bit_width();
+}
+
+PrimitiveArg GetPrimitiveArg(const ArrayData& arr) {
+  PrimitiveArg arg;
+  arg.is_valid = GetValidityBitmap(arr);
+  arg.data = arr.buffers[1]->data();
+  arg.bit_width = GetBitWidth(*arr.type);
+  arg.offset = arr.offset;
+  arg.length = arr.length;
+  if (arg.bit_width > 1) {
+    arg.data += arr.offset * arg.bit_width / 8;
+  }
+  arg.null_count = arr.GetNullCount();

Review comment:
       I think now that I've fixed the problem in https://github.com/apache/arrow/commit/37c9804784325502bf47b651252c39bdcf3e03a9 I don't need to compute it here




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

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



[GitHub] [arrow] bkietz commented on a change in pull request #7442: ARROW-9075: [C++] Optimized Filter implementation: faster performance + compilation, smaller code size

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



##########
File path: cpp/src/arrow/compute/api_vector.h
##########
@@ -64,6 +67,24 @@ Result<Datum> Filter(const Datum& values, const Datum& filter,
                      const FilterOptions& options = FilterOptions::Defaults(),
                      ExecContext* ctx = NULLPTR);
 
+namespace internal {
+
+// These internal functions are implemented in kernels/vector_selection.cc
+
+/// \brief Return the number of selected indices in the boolean filter
+ARROW_EXPORT
+int64_t GetFilterOutputSize(const ArrayData& filter,

Review comment:
       This should probably be extracted as a ScalarFunction named popcount or so (follow up)

##########
File path: cpp/src/arrow/compute/kernels/vector_selection.cc
##########
@@ -0,0 +1,1816 @@
+// 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 <algorithm>
+#include <cstring>
+#include <limits>
+
+#include "arrow/array/array_base.h"
+#include "arrow/array/array_binary.h"
+#include "arrow/array/array_dict.h"
+#include "arrow/array/array_nested.h"
+#include "arrow/array/builder_primitive.h"
+#include "arrow/array/concatenate.h"
+#include "arrow/buffer_builder.h"
+#include "arrow/compute/api_vector.h"
+#include "arrow/compute/kernels/common.h"
+#include "arrow/compute/kernels/util_internal.h"
+#include "arrow/extension_type.h"
+#include "arrow/record_batch.h"
+#include "arrow/result.h"
+#include "arrow/util/bit_block_counter.h"
+#include "arrow/util/bit_util.h"
+#include "arrow/util/bitmap_ops.h"
+#include "arrow/util/bitmap_reader.h"
+#include "arrow/util/int_util.h"
+
+namespace arrow {
+
+using internal::BinaryBitBlockCounter;
+using internal::BitBlockCount;
+using internal::BitBlockCounter;
+using internal::BitmapReader;
+using internal::CopyBitmap;
+using internal::GetArrayView;
+using internal::IndexBoundsCheck;
+using internal::OptionalBitBlockCounter;
+using internal::OptionalBitIndexer;
+
+namespace compute {
+namespace internal {
+
+int64_t GetFilterOutputSize(const ArrayData& filter,
+                            FilterOptions::NullSelectionBehavior null_selection) {
+  int64_t output_size = 0;
+  int64_t position = 0;
+  if (filter.GetNullCount() > 0) {
+    const uint8_t* filter_is_valid = filter.buffers[0]->data();
+    BinaryBitBlockCounter bit_counter(filter.buffers[1]->data(), filter.offset,
+                                      filter_is_valid, filter.offset, filter.length);
+    if (null_selection == FilterOptions::EMIT_NULL) {
+      while (position < filter.length) {
+        BitBlockCount block = bit_counter.NextOrNotWord();
+        output_size += block.popcount;
+        position += block.length;
+      }
+    } else {
+      while (position < filter.length) {
+        BitBlockCount block = bit_counter.NextAndWord();
+        output_size += block.popcount;
+        position += block.length;
+      }
+    }
+  } else {
+    // The filter has no nulls, so we plow through its data as fast as
+    // possible.
+    BitBlockCounter bit_counter(filter.buffers[1]->data(), filter.offset, filter.length);

Review comment:
       If this is faster than `CountSetBits`, the latter should probably be rewritten with `BitBlockCounter`

##########
File path: cpp/src/arrow/compute/kernels/vector_selection.cc
##########
@@ -0,0 +1,1816 @@
+// 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 <algorithm>
+#include <cstring>
+#include <limits>
+
+#include "arrow/array/array_base.h"
+#include "arrow/array/array_binary.h"
+#include "arrow/array/array_dict.h"
+#include "arrow/array/array_nested.h"
+#include "arrow/array/builder_primitive.h"
+#include "arrow/array/concatenate.h"
+#include "arrow/buffer_builder.h"
+#include "arrow/compute/api_vector.h"
+#include "arrow/compute/kernels/common.h"
+#include "arrow/compute/kernels/util_internal.h"
+#include "arrow/extension_type.h"
+#include "arrow/record_batch.h"
+#include "arrow/result.h"
+#include "arrow/util/bit_block_counter.h"
+#include "arrow/util/bit_util.h"
+#include "arrow/util/bitmap_ops.h"
+#include "arrow/util/bitmap_reader.h"
+#include "arrow/util/int_util.h"
+
+namespace arrow {
+
+using internal::BinaryBitBlockCounter;
+using internal::BitBlockCount;
+using internal::BitBlockCounter;
+using internal::BitmapReader;
+using internal::CopyBitmap;
+using internal::GetArrayView;
+using internal::IndexBoundsCheck;
+using internal::OptionalBitBlockCounter;
+using internal::OptionalBitIndexer;
+
+namespace compute {
+namespace internal {
+
+int64_t GetFilterOutputSize(const ArrayData& filter,
+                            FilterOptions::NullSelectionBehavior null_selection) {
+  int64_t output_size = 0;
+  int64_t position = 0;
+  if (filter.GetNullCount() > 0) {
+    const uint8_t* filter_is_valid = filter.buffers[0]->data();
+    BinaryBitBlockCounter bit_counter(filter.buffers[1]->data(), filter.offset,
+                                      filter_is_valid, filter.offset, filter.length);
+    if (null_selection == FilterOptions::EMIT_NULL) {
+      while (position < filter.length) {
+        BitBlockCount block = bit_counter.NextOrNotWord();
+        output_size += block.popcount;
+        position += block.length;
+      }
+    } else {
+      while (position < filter.length) {
+        BitBlockCount block = bit_counter.NextAndWord();
+        output_size += block.popcount;
+        position += block.length;
+      }
+    }
+  } else {
+    // The filter has no nulls, so we plow through its data as fast as
+    // possible.
+    BitBlockCounter bit_counter(filter.buffers[1]->data(), filter.offset, filter.length);
+    while (position < filter.length) {
+      BitBlockCount block = bit_counter.NextFourWords();
+      output_size += block.popcount;
+      position += block.length;
+    }
+  }
+  return output_size;
+}
+
+template <typename IndexType>
+Result<std::shared_ptr<ArrayData>> GetTakeIndicesImpl(
+    const ArrayData& filter, FilterOptions::NullSelectionBehavior null_selection,
+    MemoryPool* memory_pool) {
+  using T = typename IndexType::c_type;
+  typename TypeTraits<IndexType>::BuilderType builder(memory_pool);
+
+  const uint8_t* filter_data = filter.buffers[1]->data();
+  BitBlockCounter data_counter(filter_data, filter.offset, filter.length);
+
+  // The position relative to the start of the filter
+  T position = 0;
+
+  // The current position taking the filter offset into account
+  int64_t position_with_offset = filter.offset;
+  if (filter.GetNullCount() > 0) {
+    // The filter has nulls, so we scan the validity bitmap and the filter data
+    // bitmap together, branching on the null selection type.
+    const uint8_t* filter_is_valid = filter.buffers[0]->data();
+
+    // To count blocks whether filter_data[i] || !filter_is_valid[i]
+    BinaryBitBlockCounter filter_counter(filter_data, filter.offset, filter_is_valid,
+                                         filter.offset, filter.length);
+    if (null_selection == FilterOptions::DROP) {
+      while (position < filter.length) {
+        BitBlockCount and_block = filter_counter.NextAndWord();
+        RETURN_NOT_OK(builder.Reserve(and_block.popcount));
+        if (and_block.AllSet()) {
+          // All the values are selected and non-null
+          for (int64_t i = 0; i < and_block.length; ++i) {
+            builder.UnsafeAppend(position++);
+          }
+          position_with_offset += and_block.length;
+        } else {
+          // Some of the values are false or null
+          for (int64_t i = 0; i < and_block.length; ++i) {
+            if (BitUtil::GetBit(filter_is_valid, position_with_offset) &&
+                BitUtil::GetBit(filter_data, position_with_offset)) {
+              builder.UnsafeAppend(position);
+            }
+            ++position;
+            ++position_with_offset;
+          }
+        }
+      }
+    } else {
+      BitBlockCounter is_valid_counter(filter_is_valid, filter.offset, filter.length);
+      while (position < filter.length) {
+        // true OR NOT valid
+        BitBlockCount or_not_block = filter_counter.NextOrNotWord();
+        RETURN_NOT_OK(builder.Reserve(or_not_block.popcount));
+
+        // If the values are all valid and the or_not_block is full, then we
+        // can infer that all the values are true and skip the bit checking
+        BitBlockCount is_valid_block = is_valid_counter.NextWord();
+
+        if (or_not_block.AllSet() && is_valid_block.AllSet()) {
+          // All the values are selected and non-null
+          for (int64_t i = 0; i < or_not_block.length; ++i) {
+            builder.UnsafeAppend(position++);
+          }
+          position_with_offset += or_not_block.length;
+        } else {
+          // Some of the values are false or null
+          for (int64_t i = 0; i < or_not_block.length; ++i) {
+            if (BitUtil::GetBit(filter_is_valid, position_with_offset)) {
+              if (BitUtil::GetBit(filter_data, position_with_offset)) {
+                builder.UnsafeAppend(position);
+              }
+            } else {
+              // Null slot, so append a null
+              builder.UnsafeAppendNull();
+            }
+            ++position;
+            ++position_with_offset;
+          }
+        }
+      }
+    }
+  } else {
+    // The filter has no nulls, so we need only look for true values
+    BitBlockCount current_block = data_counter.NextWord();
+    while (position < filter.length) {
+      if (current_block.AllSet()) {
+        int64_t run_length = 0;
+
+        // If we've found a all-true block, then we scan forward until we find
+        // a block that has some false values (or we reach the end

Review comment:
       ```suggestion
           // a block that has some false values (or we reach the end)
   ```

##########
File path: cpp/src/arrow/compute/kernels/vector_selection.cc
##########
@@ -0,0 +1,1816 @@
+// 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 <algorithm>
+#include <cstring>
+#include <limits>
+
+#include "arrow/array/array_base.h"
+#include "arrow/array/array_binary.h"
+#include "arrow/array/array_dict.h"
+#include "arrow/array/array_nested.h"
+#include "arrow/array/builder_primitive.h"
+#include "arrow/array/concatenate.h"
+#include "arrow/buffer_builder.h"
+#include "arrow/compute/api_vector.h"
+#include "arrow/compute/kernels/common.h"
+#include "arrow/compute/kernels/util_internal.h"
+#include "arrow/extension_type.h"
+#include "arrow/record_batch.h"
+#include "arrow/result.h"
+#include "arrow/util/bit_block_counter.h"
+#include "arrow/util/bit_util.h"
+#include "arrow/util/bitmap_ops.h"
+#include "arrow/util/bitmap_reader.h"
+#include "arrow/util/int_util.h"
+
+namespace arrow {
+
+using internal::BinaryBitBlockCounter;
+using internal::BitBlockCount;
+using internal::BitBlockCounter;
+using internal::BitmapReader;
+using internal::CopyBitmap;
+using internal::GetArrayView;
+using internal::IndexBoundsCheck;
+using internal::OptionalBitBlockCounter;
+using internal::OptionalBitIndexer;
+
+namespace compute {
+namespace internal {
+
+int64_t GetFilterOutputSize(const ArrayData& filter,
+                            FilterOptions::NullSelectionBehavior null_selection) {
+  int64_t output_size = 0;
+  int64_t position = 0;
+  if (filter.GetNullCount() > 0) {
+    const uint8_t* filter_is_valid = filter.buffers[0]->data();
+    BinaryBitBlockCounter bit_counter(filter.buffers[1]->data(), filter.offset,
+                                      filter_is_valid, filter.offset, filter.length);
+    if (null_selection == FilterOptions::EMIT_NULL) {
+      while (position < filter.length) {
+        BitBlockCount block = bit_counter.NextOrNotWord();
+        output_size += block.popcount;
+        position += block.length;
+      }
+    } else {
+      while (position < filter.length) {
+        BitBlockCount block = bit_counter.NextAndWord();
+        output_size += block.popcount;
+        position += block.length;
+      }
+    }
+  } else {
+    // The filter has no nulls, so we plow through its data as fast as
+    // possible.
+    BitBlockCounter bit_counter(filter.buffers[1]->data(), filter.offset, filter.length);
+    while (position < filter.length) {
+      BitBlockCount block = bit_counter.NextFourWords();
+      output_size += block.popcount;
+      position += block.length;
+    }
+  }
+  return output_size;
+}
+
+template <typename IndexType>
+Result<std::shared_ptr<ArrayData>> GetTakeIndicesImpl(
+    const ArrayData& filter, FilterOptions::NullSelectionBehavior null_selection,
+    MemoryPool* memory_pool) {
+  using T = typename IndexType::c_type;
+  typename TypeTraits<IndexType>::BuilderType builder(memory_pool);
+
+  const uint8_t* filter_data = filter.buffers[1]->data();
+  BitBlockCounter data_counter(filter_data, filter.offset, filter.length);
+
+  // The position relative to the start of the filter
+  T position = 0;
+
+  // The current position taking the filter offset into account
+  int64_t position_with_offset = filter.offset;
+  if (filter.GetNullCount() > 0) {
+    // The filter has nulls, so we scan the validity bitmap and the filter data
+    // bitmap together, branching on the null selection type.
+    const uint8_t* filter_is_valid = filter.buffers[0]->data();
+
+    // To count blocks whether filter_data[i] || !filter_is_valid[i]
+    BinaryBitBlockCounter filter_counter(filter_data, filter.offset, filter_is_valid,
+                                         filter.offset, filter.length);
+    if (null_selection == FilterOptions::DROP) {
+      while (position < filter.length) {
+        BitBlockCount and_block = filter_counter.NextAndWord();
+        RETURN_NOT_OK(builder.Reserve(and_block.popcount));
+        if (and_block.AllSet()) {
+          // All the values are selected and non-null
+          for (int64_t i = 0; i < and_block.length; ++i) {
+            builder.UnsafeAppend(position++);
+          }
+          position_with_offset += and_block.length;
+        } else {
+          // Some of the values are false or null
+          for (int64_t i = 0; i < and_block.length; ++i) {
+            if (BitUtil::GetBit(filter_is_valid, position_with_offset) &&
+                BitUtil::GetBit(filter_data, position_with_offset)) {
+              builder.UnsafeAppend(position);
+            }
+            ++position;
+            ++position_with_offset;
+          }
+        }
+      }
+    } else {
+      BitBlockCounter is_valid_counter(filter_is_valid, filter.offset, filter.length);
+      while (position < filter.length) {
+        // true OR NOT valid
+        BitBlockCount or_not_block = filter_counter.NextOrNotWord();
+        RETURN_NOT_OK(builder.Reserve(or_not_block.popcount));
+
+        // If the values are all valid and the or_not_block is full, then we
+        // can infer that all the values are true and skip the bit checking
+        BitBlockCount is_valid_block = is_valid_counter.NextWord();
+
+        if (or_not_block.AllSet() && is_valid_block.AllSet()) {
+          // All the values are selected and non-null
+          for (int64_t i = 0; i < or_not_block.length; ++i) {
+            builder.UnsafeAppend(position++);
+          }
+          position_with_offset += or_not_block.length;
+        } else {
+          // Some of the values are false or null
+          for (int64_t i = 0; i < or_not_block.length; ++i) {
+            if (BitUtil::GetBit(filter_is_valid, position_with_offset)) {
+              if (BitUtil::GetBit(filter_data, position_with_offset)) {
+                builder.UnsafeAppend(position);
+              }
+            } else {
+              // Null slot, so append a null
+              builder.UnsafeAppendNull();
+            }
+            ++position;
+            ++position_with_offset;
+          }
+        }
+      }
+    }
+  } else {
+    // The filter has no nulls, so we need only look for true values
+    BitBlockCount current_block = data_counter.NextWord();
+    while (position < filter.length) {
+      if (current_block.AllSet()) {
+        int64_t run_length = 0;
+
+        // If we've found a all-true block, then we scan forward until we find
+        // a block that has some false values (or we reach the end
+        while (current_block.length > 0 && current_block.AllSet()) {
+          run_length += current_block.length;
+          current_block = data_counter.NextWord();
+        }
+
+        // Append the consecutive run of indices
+        RETURN_NOT_OK(builder.Reserve(run_length));
+        for (int64_t i = 0; i < run_length; ++i) {
+          builder.UnsafeAppend(position++);
+        }
+        position_with_offset += run_length;
+      } else {
+        // Must do bitchecking on the current block
+        RETURN_NOT_OK(builder.Reserve(current_block.popcount));
+        for (int64_t i = 0; i < current_block.length; ++i) {
+          if (BitUtil::GetBit(filter_data, position_with_offset)) {
+            builder.UnsafeAppend(position);
+          }
+          ++position;
+          ++position_with_offset;
+        }
+        current_block = data_counter.NextWord();
+      }
+    }
+  }
+  std::shared_ptr<ArrayData> result;
+  RETURN_NOT_OK(builder.FinishInternal(&result));
+  return result;
+}
+
+Result<std::shared_ptr<ArrayData>> GetTakeIndices(
+    const ArrayData& filter, FilterOptions::NullSelectionBehavior null_selection,
+    MemoryPool* memory_pool) {
+  DCHECK_EQ(filter.type->id(), Type::BOOL);
+  if (filter.length <= std::numeric_limits<uint16_t>::max()) {
+    return GetTakeIndicesImpl<UInt16Type>(filter, null_selection, memory_pool);
+  } else if (filter.length <= std::numeric_limits<uint32_t>::max()) {
+    return GetTakeIndicesImpl<UInt32Type>(filter, null_selection, memory_pool);
+  } else {
+    // Arrays over 4 billion elements, not especially likely.
+    return Status::NotImplemented(
+        "Filter length exceeds UINT32_MAX, "
+        "consider a different strategy for selecting elements");
+  }
+}
+
+namespace {
+
+template <typename ArrowType>
+struct GetCType {
+  using type = typename ArrowType::c_type;
+};
+
+// We want uint8_t for boolean instead of bool
+template <>
+struct GetCType<BooleanType> {
+  using type = uint8_t;
+};
+
+using FilterState = OptionsWrapper<FilterOptions>;
+using TakeState = OptionsWrapper<TakeOptions>;
+
+Status PreallocateData(KernelContext* ctx, int64_t length, int bit_width, Datum* out) {
+  // Preallocate memory
+  ArrayData* out_arr = out->mutable_array();
+  out_arr->length = length;
+  out_arr->buffers.resize(2);
+
+  ARROW_ASSIGN_OR_RAISE(out_arr->buffers[0], ctx->AllocateBitmap(length));
+  if (bit_width == 1) {
+    ARROW_ASSIGN_OR_RAISE(out_arr->buffers[1], ctx->AllocateBitmap(length));
+  } else {
+    ARROW_ASSIGN_OR_RAISE(out_arr->buffers[1], ctx->Allocate(length * bit_width / 8));
+  }
+  return Status::OK();
+}
+
+// ----------------------------------------------------------------------
+// Implement optimized take for primitive types from boolean to 1/2/4/8-byte
+// C-type based types. Use common implementation for every byte width and only
+// generate code for unsigned integer indices, since after boundschecking to
+// check for negative numbers in the indices we can safely reinterpret_cast
+// signed integers as unsigned.
+
+/// \brief The Take implementation for primitive (fixed-width) types does not
+/// use the logical Arrow type but rather the physical C type. This way we
+/// only generate one take function for each byte width.
+///
+/// This function assumes that the indices have been boundschecked.
+template <typename IndexCType, typename ValueCType>
+struct PrimitiveTakeImpl {
+  static void Exec(const PrimitiveArg& values, const PrimitiveArg& indices,
+                   Datum* out_datum) {
+    auto values_data = reinterpret_cast<const ValueCType*>(values.data);
+    auto values_is_valid = values.is_valid;
+    auto values_offset = values.offset;
+
+    auto indices_data = reinterpret_cast<const IndexCType*>(indices.data);
+    auto indices_is_valid = indices.is_valid;
+    auto indices_offset = indices.offset;
+
+    ArrayData* out_arr = out_datum->mutable_array();
+    auto out = out_arr->GetMutableValues<ValueCType>(1);
+    auto out_is_valid = out_arr->buffers[0]->mutable_data();
+    auto out_offset = out_arr->offset;
+
+    // If either the values or indices have nulls, we preemptively zero out the
+    // out validity bitmap so that we don't have to use ClearBit in each
+    // iteration for nulls.
+    if (values.null_count > 0 || indices.null_count > 0) {
+      BitUtil::SetBitsTo(out_is_valid, out_offset, indices.length, false);
+    }
+
+    OptionalBitBlockCounter indices_bit_counter(indices_is_valid, indices_offset,
+                                                indices.length);
+    int64_t position = 0;
+    int64_t valid_count = 0;
+    while (position < indices.length) {
+      BitBlockCount block = indices_bit_counter.NextBlock();
+      if (values.null_count == 0) {
+        // Values are never null, so things are easier
+        valid_count += block.popcount;
+        if (block.popcount == block.length) {
+          // Fastest path: neither values nor index nulls
+          BitUtil::SetBitsTo(out_is_valid, out_offset + position, block.length, true);
+          for (int64_t i = 0; i < block.length; ++i) {
+            out[position] = values_data[indices_data[position]];
+            ++position;
+          }
+        } else if (block.popcount > 0) {
+          // Slow path: some indices but not all are null
+          for (int64_t i = 0; i < block.length; ++i) {
+            if (BitUtil::GetBit(indices_is_valid, indices_offset + position)) {
+              // index is not null
+              BitUtil::SetBit(out_is_valid, out_offset + position);
+              out[position] = values_data[indices_data[position]];
+            } else {
+              out[position] = ValueCType{};
+            }
+            ++position;
+          }
+        } else {
+          memset(out + position, 0, sizeof(ValueCType) * block.length);
+          position += block.length;
+        }
+      } else {
+        // Values have nulls, so we must do random access into the values bitmap
+        if (block.popcount == block.length) {
+          // Faster path: indices are not null but values may be
+          for (int64_t i = 0; i < block.length; ++i) {
+            if (BitUtil::GetBit(values_is_valid,
+                                values_offset + indices_data[position])) {
+              // value is not null
+              out[position] = values_data[indices_data[position]];
+              BitUtil::SetBit(out_is_valid, out_offset + position);
+              ++valid_count;
+            } else {
+              out[position] = ValueCType{};
+            }
+            ++position;
+          }
+        } else if (block.popcount > 0) {
+          // Slow path: some but not all indices are null. Since we are doing
+          // random access in general we have to check the value nullness one by
+          // one.
+          for (int64_t i = 0; i < block.length; ++i) {
+            if (BitUtil::GetBit(indices_is_valid, indices_offset + position) &&
+                BitUtil::GetBit(values_is_valid,
+                                values_offset + indices_data[position])) {
+              // index is not null && value is not null
+              out[position] = values_data[indices_data[position]];
+              BitUtil::SetBit(out_is_valid, out_offset + position);
+              ++valid_count;
+            } else {
+              out[position] = ValueCType{};
+            }
+            ++position;
+          }
+        } else {
+          memset(out + position, 0, sizeof(ValueCType) * block.length);
+          position += block.length;
+        }
+      }
+    }
+    out_arr->null_count = out_arr->length - valid_count;
+  }
+};
+
+template <typename IndexCType>
+struct BooleanTakeImpl {
+  static void Exec(const PrimitiveArg& values, const PrimitiveArg& indices,
+                   Datum* out_datum) {
+    const uint8_t* values_data = values.data;
+    auto values_is_valid = values.is_valid;
+    auto values_offset = values.offset;
+
+    auto indices_data = reinterpret_cast<const IndexCType*>(indices.data);
+    auto indices_is_valid = indices.is_valid;
+    auto indices_offset = indices.offset;
+
+    ArrayData* out_arr = out_datum->mutable_array();
+    auto out = out_arr->buffers[1]->mutable_data();
+    auto out_is_valid = out_arr->buffers[0]->mutable_data();
+    auto out_offset = out_arr->offset;
+
+    // If either the values or indices have nulls, we preemptively zero out the
+    // out validity bitmap so that we don't have to use ClearBit in each
+    // iteration for nulls.
+    if (values.null_count > 0 || indices.null_count > 0) {
+      BitUtil::SetBitsTo(out_is_valid, out_offset, indices.length, false);
+    }
+    // Avoid uninitialized data in values array
+    BitUtil::SetBitsTo(out, out_offset, indices.length, false);
+
+    auto PlaceDataBit = [&](int64_t loc, IndexCType index) {
+      BitUtil::SetBitTo(out, out_offset + loc,
+                        BitUtil::GetBit(values_data, values_offset + index));
+    };
+
+    OptionalBitBlockCounter indices_bit_counter(indices_is_valid, indices_offset,
+                                                indices.length);
+    int64_t position = 0;
+    int64_t valid_count = 0;
+    while (position < indices.length) {
+      BitBlockCount block = indices_bit_counter.NextBlock();
+      if (values.null_count == 0) {
+        // Values are never null, so things are easier
+        valid_count += block.popcount;
+        if (block.popcount == block.length) {
+          // Fastest path: neither values nor index nulls
+          BitUtil::SetBitsTo(out_is_valid, out_offset + position, block.length, true);
+          for (int64_t i = 0; i < block.length; ++i) {
+            PlaceDataBit(position, indices_data[position]);
+            ++position;
+          }
+        } else if (block.popcount > 0) {
+          // Slow path: some but not all indices are null
+          for (int64_t i = 0; i < block.length; ++i) {
+            if (BitUtil::GetBit(indices_is_valid, indices_offset + position)) {
+              // index is not null
+              BitUtil::SetBit(out_is_valid, out_offset + position);
+              PlaceDataBit(position, indices_data[position]);
+            }
+            ++position;
+          }
+        } else {
+          position += block.length;
+        }
+      } else {
+        // Values have nulls, so we must do random access into the values bitmap
+        if (block.popcount == block.length) {
+          // Faster path: indices are not null but values may be
+          for (int64_t i = 0; i < block.length; ++i) {
+            if (BitUtil::GetBit(values_is_valid,
+                                values_offset + indices_data[position])) {
+              // value is not null
+              BitUtil::SetBit(out_is_valid, out_offset + position);
+              PlaceDataBit(position, indices_data[position]);
+              ++valid_count;
+            }
+            ++position;
+          }
+        } else if (block.popcount > 0) {
+          // Slow path: some but not all indices are null. Since we are doing
+          // random access in general we have to check the value nullness one by
+          // one.
+          for (int64_t i = 0; i < block.length; ++i) {
+            if (BitUtil::GetBit(indices_is_valid, indices_offset + position)) {
+              // index is not null
+              if (BitUtil::GetBit(values_is_valid,
+                                  values_offset + indices_data[position])) {
+                // value is not null
+                PlaceDataBit(position, indices_data[position]);
+                BitUtil::SetBit(out_is_valid, out_offset + position);
+                ++valid_count;
+              }
+            }
+            ++position;
+          }
+        } else {
+          position += block.length;
+        }
+      }
+    }
+    out_arr->null_count = out_arr->length - valid_count;
+  }
+};
+
+template <template <typename...> class TakeImpl, typename... Args>
+void TakeIndexDispatch(const PrimitiveArg& values, const PrimitiveArg& indices,
+                       Datum* out) {
+  // With the simplifying assumption that boundschecking has taken place
+  // already at a higher level, we can now assume that the index values are all
+  // non-negative. Thus, we can interpret signed integers as unsigned and avoid
+  // having to generate double the amount of binary code to handle each integer
+  // width.
+  switch (indices.bit_width) {
+    case 8:
+      return TakeImpl<uint8_t, Args...>::Exec(values, indices, out);
+    case 16:
+      return TakeImpl<uint16_t, Args...>::Exec(values, indices, out);
+    case 32:
+      return TakeImpl<uint32_t, Args...>::Exec(values, indices, out);
+    case 64:
+      return TakeImpl<uint64_t, Args...>::Exec(values, indices, out);
+    default:
+      DCHECK(false) << "Invalid indices byte width";
+      break;
+  }
+}
+
+void PrimitiveTake(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+  const auto& state = checked_cast<const TakeState&>(*ctx->state());
+  if (state.options.boundscheck) {
+    KERNEL_RETURN_IF_ERROR(ctx, IndexBoundsCheck(*batch[1].array(), batch[0].length()));
+  }
+
+  PrimitiveArg values = GetPrimitiveArg(*batch[0].array());
+  PrimitiveArg indices = GetPrimitiveArg(*batch[1].array());
+  KERNEL_RETURN_IF_ERROR(ctx,
+                         PreallocateData(ctx, indices.length, values.bit_width, out));
+  switch (values.bit_width) {
+    case 1:
+      return TakeIndexDispatch<BooleanTakeImpl>(values, indices, out);
+    case 8:
+      return TakeIndexDispatch<PrimitiveTakeImpl, int8_t>(values, indices, out);
+    case 16:
+      return TakeIndexDispatch<PrimitiveTakeImpl, int16_t>(values, indices, out);
+    case 32:
+      return TakeIndexDispatch<PrimitiveTakeImpl, int32_t>(values, indices, out);
+    case 64:
+      return TakeIndexDispatch<PrimitiveTakeImpl, int64_t>(values, indices, out);
+    default:
+      DCHECK(false) << "Invalid values byte width";
+      break;
+  }
+}
+
+// ----------------------------------------------------------------------
+// Optimized and streamlined filter for primitive types
+
+// Use either BitBlockCounter or BinaryBitBlockCounter to quickly scan filter a
+// word at a time for the DROP selection type.
+class DropNullCounter {
+ public:
+  // validity bitmap may be null
+  DropNullCounter(const uint8_t* validity, const uint8_t* data, int64_t offset,
+                  int64_t length)
+      : data_counter_(data, offset, length),
+        data_and_validity_counter_(data, offset, validity, offset, length),
+        has_validity_(validity != nullptr) {}
+
+  BitBlockCount NextBlock() {
+    if (has_validity_) {
+      // filter is true AND not null
+      return data_and_validity_counter_.NextAndWord();
+    } else {
+      return data_counter_.NextWord();
+    }
+  }
+
+ private:
+  // For when just data is present, but no validity bitmap
+  BitBlockCounter data_counter_;
+
+  // For when both validity bitmap and data are present
+  BinaryBitBlockCounter data_and_validity_counter_;
+  bool has_validity_;
+};
+
+/// \brief The Filter implementation for primitive (fixed-width) types does not
+/// use the logical Arrow type but rather then physical C type. This way we
+/// only generate one take function for each byte width. We use the same
+/// implementation here for boolean and fixed-byte-size inputs with some
+/// template specialization.
+template <typename ArrowType>
+class PrimitiveFilterImpl {
+ public:
+  using T = typename GetCType<ArrowType>::type;
+
+  PrimitiveFilterImpl(const PrimitiveArg& values, const PrimitiveArg& filter,
+                      FilterOptions::NullSelectionBehavior null_selection,
+                      Datum* out_datum)
+      : values_is_valid_(values.is_valid),
+        values_data_(reinterpret_cast<const T*>(values.data)),
+        values_null_count_(values.null_count),
+        values_offset_(values.offset),
+        values_length_(values.length),
+        filter_is_valid_(filter.is_valid),
+        filter_data_(filter.data),
+        filter_null_count_(filter.null_count),
+        filter_offset_(filter.offset),
+        null_selection_(null_selection) {
+    ArrayData* out_arr = out_datum->mutable_array();
+    out_is_valid_ = out_arr->buffers[0]->mutable_data();
+    out_data_ = reinterpret_cast<T*>(out_arr->buffers[1]->mutable_data());
+    out_offset_ = out_arr->offset;
+    out_length_ = out_arr->length;
+    out_position_ = 0;
+  }
+
+  void ExecNonNull() {
+    // The result is all not-null
+    BitUtil::SetBitsTo(out_is_valid_, out_offset_ + out_position_, out_length_, true);
+
+    // Fast filter when values and filter are not null
+    // Bit counters used for both null_selection behaviors
+    BitBlockCounter filter_counter(filter_data_, filter_offset_, values_length_);
+
+    int64_t in_position = 0;
+    BitBlockCount current_block = filter_counter.NextWord();
+    while (in_position < values_length_) {
+      if (current_block.AllSet()) {
+        int64_t run_length = 0;
+        // If we've found a all-true block, then we scan forward until we find
+        // a block that has some false values (or we reach the end
+        while (current_block.length > 0 && current_block.AllSet()) {
+          run_length += current_block.length;
+          current_block = filter_counter.NextWord();
+        }
+        WriteValueSegment(in_position, run_length);
+        in_position += run_length;
+      } else if (current_block.NoneSet()) {
+        // Nothing selected
+        in_position += current_block.length;
+        current_block = filter_counter.NextWord();
+      } else {
+        // Some values selected
+        for (int64_t i = 0; i < current_block.length; ++i) {
+          if (BitUtil::GetBit(filter_data_, filter_offset_ + in_position)) {
+            WriteValue(in_position);
+          }
+          ++in_position;
+        }
+        current_block = filter_counter.NextWord();
+      }
+    }
+  }
+
+  void Exec() {
+    if (filter_null_count_ == 0 && values_null_count_ == 0) {
+      return ExecNonNull();
+    }
+
+    // Bit counters used for both null_selection behaviors
+    DropNullCounter drop_null_counter(filter_is_valid_, filter_data_, filter_offset_,
+                                      values_length_);
+    OptionalBitBlockCounter data_counter(values_is_valid_, values_offset_,
+                                         values_length_);
+    OptionalBitBlockCounter filter_valid_counter(filter_is_valid_, filter_offset_,
+                                                 values_length_);
+
+    auto WriteNotNull = [&](int64_t index) {
+      BitUtil::SetBit(out_is_valid_, out_offset_ + out_position_);
+      // Increments out_position_
+      WriteValue(index);
+    };
+
+    auto WriteMaybeNull = [&](int64_t index) {
+      BitUtil::SetBitTo(out_is_valid_, out_offset_ + out_position_,
+                        BitUtil::GetBit(values_is_valid_, values_offset_ + index));
+      // Increments out_position_
+      WriteValue(index);
+    };
+
+    int64_t in_position = 0;
+    while (in_position < values_length_) {
+      BitBlockCount filter_block = drop_null_counter.NextBlock();
+      BitBlockCount filter_valid_block = filter_valid_counter.NextWord();
+      BitBlockCount data_block = data_counter.NextWord();
+      if (filter_block.AllSet() && data_block.AllSet()) {
+        // Fastest path: all values in block are included and not null
+        BitUtil::SetBitsTo(out_is_valid_, out_offset_ + out_position_,
+                           filter_block.length, true);
+        WriteValueSegment(in_position, filter_block.length);
+        in_position += filter_block.length;
+      } else if (filter_block.AllSet()) {
+        // Faster: all values are selected, but some values are null
+        // Batch copy bits from values validity bitmap to output validity bitmap
+        CopyBitmap(values_is_valid_, values_offset_ + in_position, filter_block.length,
+                   out_is_valid_, out_offset_ + out_position_);
+        WriteValueSegment(in_position, filter_block.length);
+        in_position += filter_block.length;
+      } else if (filter_block.NoneSet() && null_selection_ == FilterOptions::DROP) {
+        // For this exceedingly common case in low-selectivity filters we can
+        // skip further analysis of the data and move on to the next block.
+        in_position += filter_block.length;
+      } else {
+        // Some filter values are false or null
+        if (data_block.AllSet()) {
+          // No values are null
+          if (filter_valid_block.AllSet()) {
+            // Filter is non-null but some values are false
+            for (int64_t i = 0; i < filter_block.length; ++i) {
+              if (BitUtil::GetBit(filter_data_, filter_offset_ + in_position)) {
+                WriteNotNull(in_position);
+              }
+              ++in_position;
+            }
+          } else if (null_selection_ == FilterOptions::DROP) {
+            // If any values are selected, they ARE NOT null
+            for (int64_t i = 0; i < filter_block.length; ++i) {
+              if (BitUtil::GetBit(filter_is_valid_, filter_offset_ + in_position) &&
+                  BitUtil::GetBit(filter_data_, filter_offset_ + in_position)) {
+                WriteNotNull(in_position);
+              }
+              ++in_position;
+            }
+          } else {  // null_selection == FilterOptions::EMIT_NULL
+            // Data values in this block are not null
+            for (int64_t i = 0; i < filter_block.length; ++i) {
+              const bool is_valid =
+                  BitUtil::GetBit(filter_is_valid_, filter_offset_ + in_position);
+              if (is_valid &&
+                  BitUtil::GetBit(filter_data_, filter_offset_ + in_position)) {
+                // Filter slot is non-null and set
+                WriteNotNull(in_position);
+              } else if (!is_valid) {
+                // Filter slot is null, so we have a null in the output
+                BitUtil::ClearBit(out_is_valid_, out_offset_ + out_position_);
+                WriteNull();
+              }
+              ++in_position;
+            }
+          }
+        } else {  // !data_block.AllSet()
+          // Some values are null
+          if (filter_valid_block.AllSet()) {
+            // Filter is non-null but some values are false
+            for (int64_t i = 0; i < filter_block.length; ++i) {
+              if (BitUtil::GetBit(filter_data_, filter_offset_ + in_position)) {
+                WriteMaybeNull(in_position);
+              }
+              ++in_position;
+            }
+          } else if (null_selection_ == FilterOptions::DROP) {
+            // If any values are selected, they ARE NOT null
+            for (int64_t i = 0; i < filter_block.length; ++i) {
+              if (BitUtil::GetBit(filter_is_valid_, filter_offset_ + in_position) &&
+                  BitUtil::GetBit(filter_data_, filter_offset_ + in_position)) {
+                WriteMaybeNull(in_position);
+              }
+              ++in_position;
+            }
+          } else {  // null_selection == FilterOptions::EMIT_NULL
+            // Data values in this block are not null
+            for (int64_t i = 0; i < filter_block.length; ++i) {
+              const bool is_valid =
+                  BitUtil::GetBit(filter_is_valid_, filter_offset_ + in_position);
+              if (is_valid &&
+                  BitUtil::GetBit(filter_data_, filter_offset_ + in_position)) {
+                // Filter slot is non-null and set
+                WriteMaybeNull(in_position);
+              } else if (!is_valid) {
+                // Filter slot is null, so we have a null in the output
+                BitUtil::ClearBit(out_is_valid_, out_offset_ + out_position_);
+                WriteNull();
+              }
+              ++in_position;
+            }
+          }
+        }
+      }  // !filter_block.AllSet()
+    }    // while(in_position < values_length_)
+  }
+
+  // Write the next out_position given the selected in_position for the input
+  // data and advance out_position
+  void WriteValue(int64_t in_position) {
+    out_data_[out_position_++] = values_data_[in_position];
+  }
+
+  void WriteValueSegment(int64_t in_start, int64_t length) {
+    std::memcpy(out_data_ + out_position_, values_data_ + in_start, length * sizeof(T));
+    out_position_ += length;
+  }
+
+  void WriteNull() {
+    // Zero the memory
+    out_data_[out_position_++] = T{};
+  }
+
+ private:
+  const uint8_t* values_is_valid_;
+  const T* values_data_;
+  int64_t values_null_count_;
+  int64_t values_offset_;
+  int64_t values_length_;
+  const uint8_t* filter_is_valid_;
+  const uint8_t* filter_data_;
+  int64_t filter_null_count_;
+  int64_t filter_offset_;
+  FilterOptions::NullSelectionBehavior null_selection_;
+  uint8_t* out_is_valid_;
+  T* out_data_;
+  int64_t out_offset_;
+  int64_t out_length_;
+  int64_t out_position_;
+};
+
+template <>
+inline void PrimitiveFilterImpl<BooleanType>::WriteValue(int64_t in_position) {
+  BitUtil::SetBitTo(out_data_, out_offset_ + out_position_++,
+                    BitUtil::GetBit(values_data_, values_offset_ + in_position));
+}
+
+template <>
+inline void PrimitiveFilterImpl<BooleanType>::WriteValueSegment(int64_t in_start,
+                                                                int64_t length) {
+  CopyBitmap(values_data_, values_offset_ + in_start, length, out_data_,
+             out_offset_ + out_position_);
+  out_position_ += length;
+}
+
+template <>
+inline void PrimitiveFilterImpl<BooleanType>::WriteNull() {
+  // Zero the bit
+  BitUtil::ClearBit(out_data_, out_offset_ + out_position_++);
+}
+
+void PrimitiveFilter(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+  const auto& state = checked_cast<const FilterState&>(*ctx->state());
+  PrimitiveArg values = GetPrimitiveArg(*batch[0].array());
+  PrimitiveArg filter = GetPrimitiveArg(*batch[1].array());
+  FilterOptions::NullSelectionBehavior null_selection =
+      state.options.null_selection_behavior;
+
+  int64_t output_length = GetFilterOutputSize(*batch[1].array(), null_selection);
+  KERNEL_RETURN_IF_ERROR(ctx, PreallocateData(ctx, output_length, values.bit_width, out));
+
+  // The output precomputed null count is unknown except in the narrow
+  // condition that all the values are non-null and the filter will not cause
+  // any new nulls to be created.
+  if (values.null_count == 0 &&
+      (null_selection == FilterOptions::DROP || filter.null_count == 0)) {
+    out->mutable_array()->null_count = 0;
+  } else {
+    out->mutable_array()->null_count = kUnknownNullCount;
+  }
+  switch (values.bit_width) {
+    case 1:
+      return PrimitiveFilterImpl<BooleanType>(values, filter, null_selection, out).Exec();
+    case 8:
+      return PrimitiveFilterImpl<UInt8Type>(values, filter, null_selection, out).Exec();
+    case 16:
+      return PrimitiveFilterImpl<UInt16Type>(values, filter, null_selection, out).Exec();
+    case 32:
+      return PrimitiveFilterImpl<UInt32Type>(values, filter, null_selection, out).Exec();
+    case 64:
+      return PrimitiveFilterImpl<UInt64Type>(values, filter, null_selection, out).Exec();
+    default:
+      DCHECK(false) << "Invalid values bit width";
+      break;
+  }
+}
+
+// ----------------------------------------------------------------------
+// Null take and filter
+
+void NullTake(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+  const auto& state = checked_cast<const TakeState&>(*ctx->state());
+  if (state.options.boundscheck) {
+    KERNEL_RETURN_IF_ERROR(ctx, IndexBoundsCheck(*batch[1].array(), batch[0].length()));
+  }
+  out->value = std::make_shared<NullArray>(batch.length)->data();
+}
+
+void NullFilter(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+  const auto& state = checked_cast<const FilterState&>(*ctx->state());
+  int64_t output_length =
+      GetFilterOutputSize(*batch[1].array(), state.options.null_selection_behavior);
+  out->value = std::make_shared<NullArray>(output_length)->data();
+}
+
+// ----------------------------------------------------------------------
+// Dictionary take and filter
+
+void DictionaryTake(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+  const auto& state = checked_cast<const TakeState&>(*ctx->state());
+  DictionaryArray values(batch[0].array());
+  Datum result;
+  KERNEL_RETURN_IF_ERROR(
+      ctx, Take(Datum(values.indices()), batch[1], state.options, ctx->exec_context())
+               .Value(&result));
+  DictionaryArray taken_values(values.type(), result.make_array(), values.dictionary());
+  out->value = taken_values.data();
+}
+
+void DictionaryFilter(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+  const auto& state = checked_cast<const FilterState&>(*ctx->state());
+  DictionaryArray dict_values(batch[0].array());
+  Datum result;
+  KERNEL_RETURN_IF_ERROR(ctx, Filter(Datum(dict_values.indices()), batch[1].array(),
+                                     state.options, ctx->exec_context())
+                                  .Value(&result));
+  DictionaryArray filtered_values(dict_values.type(), result.make_array(),
+                                  dict_values.dictionary());
+  out->value = filtered_values.data();
+}
+
+// ----------------------------------------------------------------------
+// Extension take and filter
+
+void ExtensionTake(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+  const auto& state = checked_cast<const TakeState&>(*ctx->state());
+  ExtensionArray values(batch[0].array());
+  Datum result;
+  KERNEL_RETURN_IF_ERROR(
+      ctx, Take(Datum(values.storage()), batch[1], state.options, ctx->exec_context())
+               .Value(&result));
+  ExtensionArray taken_values(values.type(), result.make_array());
+  out->value = taken_values.data();
+}
+
+void ExtensionFilter(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+  const auto& state = checked_cast<const FilterState&>(*ctx->state());
+  ExtensionArray ext_values(batch[0].array());
+  Datum result;
+  KERNEL_RETURN_IF_ERROR(ctx, Filter(Datum(ext_values.storage()), batch[1].array(),
+                                     state.options, ctx->exec_context())
+                                  .Value(&result));
+  ExtensionArray filtered_values(ext_values.type(), result.make_array());
+  out->value = filtered_values.data();
+}
+
+// ----------------------------------------------------------------------
+// Implement take for other data types where there is less performance
+// sensitivity by visiting the selected indices.
+
+// Use CRTP to dispatch to type-specific processing of take indices for each
+// unsigned integer type.
+template <typename Impl, typename Type>
+struct Selection {
+  using ValuesArrayType = typename TypeTraits<Type>::ArrayType;
+
+  // Forwards the generic value visitors to the take index visitor template
+  template <typename IndexCType>
+  struct TakeAdapter {
+    static constexpr bool is_take = true;
+
+    Impl* impl;
+    explicit TakeAdapter(Impl* impl) : impl(impl) {}
+    template <typename ValidVisitor, typename NullVisitor>
+    Status Generate(ValidVisitor&& visit_valid, NullVisitor&& visit_null) {
+      return impl->template VisitTake<IndexCType>(std::forward<ValidVisitor>(visit_valid),
+                                                  std::forward<NullVisitor>(visit_null));
+    }
+  };
+
+  // Forwards the generic value visitors to the VisitFilter template
+  struct FilterAdapter {
+    static constexpr bool is_take = false;
+
+    Impl* impl;
+    explicit FilterAdapter(Impl* impl) : impl(impl) {}
+    template <typename ValidVisitor, typename NullVisitor>
+    Status Generate(ValidVisitor&& visit_valid, NullVisitor&& visit_null) {
+      return impl->VisitFilter(std::forward<ValidVisitor>(visit_valid),
+                               std::forward<NullVisitor>(visit_null));
+    }
+  };
+
+  KernelContext* ctx;
+  std::shared_ptr<ArrayData> values;
+  std::shared_ptr<ArrayData> selection;
+  int64_t output_length;
+  ArrayData* out;
+  TypedBufferBuilder<bool> validity_builder;
+
+  Selection(KernelContext* ctx, const ExecBatch& batch, int64_t output_length, Datum* out)
+      : ctx(ctx),
+        values(batch[0].array()),
+        selection(batch[1].array()),
+        output_length(output_length),
+        out(out->mutable_array()),
+        validity_builder(ctx->memory_pool()) {}
+
+  virtual ~Selection() = default;
+
+  Status FinishCommon() {
+    out->buffers.resize(values->buffers.size());
+    out->length = validity_builder.length();
+    out->null_count = validity_builder.false_count();
+    return validity_builder.Finish(&out->buffers[0]);
+  }
+
+  template <typename IndexCType, typename ValidVisitor, typename NullVisitor>
+  Status VisitTake(ValidVisitor&& visit_valid, NullVisitor&& visit_null) {
+    const auto indices_values = selection->GetValues<IndexCType>(1);
+    const uint8_t* is_valid = GetValidityBitmap(*selection);
+    OptionalBitIndexer indices_is_valid(selection->buffers[0], selection->offset);
+    OptionalBitIndexer values_is_valid(values->buffers[0], values->offset);
+    const bool values_have_nulls = (values->GetNullCount() > 0);
+
+    OptionalBitBlockCounter bit_counter(is_valid, selection->offset, selection->length);
+    int64_t position = 0;
+    while (position < selection->length) {
+      BitBlockCount block = bit_counter.NextBlock();
+      const bool indices_have_nulls = block.popcount < block.length;
+      if (!indices_have_nulls && !values_have_nulls) {
+        // Fastest path, neither indices nor values have nulls
+        validity_builder.UnsafeAppend(block.length, true);
+        for (int64_t i = 0; i < block.length; ++i) {
+          RETURN_NOT_OK(visit_valid(indices_values[position++]));
+        }
+      } else if (block.popcount > 0) {
+        // Since we have to branch on whether the indices are null or not, we
+        // combine the "non-null indices block but some values null" and
+        // "some-null indices block but values non-null" into a single loop.
+        for (int64_t i = 0; i < block.length; ++i) {
+          if ((!indices_have_nulls || indices_is_valid[position]) &&
+              values_is_valid[indices_values[position]]) {
+            validity_builder.UnsafeAppend(true);
+            RETURN_NOT_OK(visit_valid(indices_values[position]));
+          } else {
+            validity_builder.UnsafeAppend(false);
+            RETURN_NOT_OK(visit_null());
+          }
+          ++position;
+        }
+      } else {
+        // The whole block is null
+        validity_builder.UnsafeAppend(block.length, false);
+        for (int64_t i = 0; i < block.length; ++i) {
+          RETURN_NOT_OK(visit_null());
+        }
+        position += block.length;
+      }
+    }
+    return Status::OK();
+  }
+
+  // We use the NullVisitor both for "selected" nulls as well as "emitted"
+  // nulls coming from the filter when using FilterOptions::EMIT_NULL
+  template <typename ValidVisitor, typename NullVisitor>
+  Status VisitFilter(ValidVisitor&& visit_valid, NullVisitor&& visit_null) {
+    const auto& state = checked_cast<const FilterState&>(*ctx->state());
+    auto null_selection = state.options.null_selection_behavior;
+
+    const auto filter_data = selection->buffers[1]->data();
+
+    const uint8_t* filter_is_valid = GetValidityBitmap(*selection);
+    const int64_t filter_offset = selection->offset;
+    OptionalBitIndexer values_is_valid(values->buffers[0], values->offset);
+
+    // We use 3 block counters for fast scanning of the filter
+    //
+    // * values_valid_counter: for values null/not-null
+    // * filter_valid_counter: for filter null/not-null
+    // * filter_counter: for filter true/false
+    OptionalBitBlockCounter values_valid_counter(GetValidityBitmap(*values),
+                                                 values->offset, values->length);
+    OptionalBitBlockCounter filter_valid_counter(filter_is_valid, filter_offset,
+                                                 selection->length);
+    BitBlockCounter filter_counter(filter_data, filter_offset, selection->length);
+    int64_t in_position = 0;
+
+    auto AppendNotNull = [&](int64_t index) -> Status {
+      validity_builder.UnsafeAppend(true);
+      return visit_valid(index);
+    };
+
+    auto AppendNull = [&]() -> Status {
+      validity_builder.UnsafeAppend(false);
+      return visit_null();
+    };
+
+    auto AppendMaybeNull = [&](int64_t index) -> Status {
+      if (values_is_valid[index]) {
+        return AppendNotNull(index);
+      } else {
+        return AppendNull();
+      }
+    };
+
+    while (in_position < selection->length) {
+      BitBlockCount filter_valid_block = filter_valid_counter.NextWord();
+      BitBlockCount values_valid_block = values_valid_counter.NextWord();
+      BitBlockCount filter_block = filter_counter.NextWord();
+      if (filter_block.NoneSet() && null_selection == FilterOptions::DROP) {
+        // For this exceedingly common case in low-selectivity filters we can
+        // skip further analysis of the data and move on to the next block.
+        in_position += filter_block.length;
+      } else if (filter_valid_block.AllSet()) {
+        // Simpler path: no filter values are null
+        if (filter_block.AllSet()) {
+          // Fastest path: filter values are all true and not null
+          if (values_valid_block.AllSet()) {
+            // The values aren't null either
+            validity_builder.UnsafeAppend(filter_block.length, true);
+            for (int64_t i = 0; i < filter_block.length; ++i) {
+              RETURN_NOT_OK(visit_valid(in_position++));
+            }
+          } else {
+            // Some of the values in this block are null
+            for (int64_t i = 0; i < filter_block.length; ++i) {
+              RETURN_NOT_OK(AppendMaybeNull(in_position++));
+            }
+          }
+        } else {  // !filter_block.AllSet()
+          // Some of the filter values are false, but all not null
+          if (values_valid_block.AllSet()) {
+            // All the values are not-null, so we can skip null checking for
+            // them
+            for (int64_t i = 0; i < filter_block.length; ++i) {
+              if (BitUtil::GetBit(filter_data, filter_offset + in_position)) {
+                RETURN_NOT_OK(AppendNotNull(in_position));
+              }
+              ++in_position;
+            }
+          } else {
+            // Some of the values in the block are null, so we have to check
+            // each one
+            for (int64_t i = 0; i < filter_block.length; ++i) {
+              if (BitUtil::GetBit(filter_data, filter_offset + in_position)) {
+                RETURN_NOT_OK(AppendMaybeNull(in_position));
+              }
+              ++in_position;
+            }
+          }
+        }
+      } else {  // !filter_valid_block.AllSet()
+        // Some of the filter values are null, so we have to handle the DROP
+        // versus EMIT_NULL null selection behavior.
+        if (null_selection == FilterOptions::DROP) {

Review comment:
       Shouldn't this check be outside the loop?

##########
File path: cpp/src/arrow/util/bit_block_counter.h
##########
@@ -132,7 +187,16 @@ class ARROW_EXPORT BinaryBitBlockCounter {
   /// blocks in subsequent invocations.
   BitBlockCount NextAndWord();
 
+  /// \brief Computes "x | y" block for each available run of bits.
+  BitBlockCount NextOrWord();
+
+  /// \brief Computes "x | !y" block for each available run of bits.

Review comment:
       ```suggestion
     /// \brief Computes "x | ~y" block for each available run of bits.
   ```

##########
File path: cpp/src/arrow/compute/kernels/vector_selection.cc
##########
@@ -0,0 +1,1816 @@
+// 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 <algorithm>
+#include <cstring>
+#include <limits>
+
+#include "arrow/array/array_base.h"
+#include "arrow/array/array_binary.h"
+#include "arrow/array/array_dict.h"
+#include "arrow/array/array_nested.h"
+#include "arrow/array/builder_primitive.h"
+#include "arrow/array/concatenate.h"
+#include "arrow/buffer_builder.h"
+#include "arrow/compute/api_vector.h"
+#include "arrow/compute/kernels/common.h"
+#include "arrow/compute/kernels/util_internal.h"
+#include "arrow/extension_type.h"
+#include "arrow/record_batch.h"
+#include "arrow/result.h"
+#include "arrow/util/bit_block_counter.h"
+#include "arrow/util/bit_util.h"
+#include "arrow/util/bitmap_ops.h"
+#include "arrow/util/bitmap_reader.h"
+#include "arrow/util/int_util.h"
+
+namespace arrow {
+
+using internal::BinaryBitBlockCounter;
+using internal::BitBlockCount;
+using internal::BitBlockCounter;
+using internal::BitmapReader;
+using internal::CopyBitmap;
+using internal::GetArrayView;
+using internal::IndexBoundsCheck;
+using internal::OptionalBitBlockCounter;
+using internal::OptionalBitIndexer;
+
+namespace compute {
+namespace internal {
+
+int64_t GetFilterOutputSize(const ArrayData& filter,
+                            FilterOptions::NullSelectionBehavior null_selection) {
+  int64_t output_size = 0;
+  int64_t position = 0;
+  if (filter.GetNullCount() > 0) {
+    const uint8_t* filter_is_valid = filter.buffers[0]->data();
+    BinaryBitBlockCounter bit_counter(filter.buffers[1]->data(), filter.offset,
+                                      filter_is_valid, filter.offset, filter.length);
+    if (null_selection == FilterOptions::EMIT_NULL) {
+      while (position < filter.length) {
+        BitBlockCount block = bit_counter.NextOrNotWord();
+        output_size += block.popcount;
+        position += block.length;
+      }
+    } else {
+      while (position < filter.length) {
+        BitBlockCount block = bit_counter.NextAndWord();
+        output_size += block.popcount;
+        position += block.length;
+      }
+    }
+  } else {
+    // The filter has no nulls, so we plow through its data as fast as
+    // possible.
+    BitBlockCounter bit_counter(filter.buffers[1]->data(), filter.offset, filter.length);
+    while (position < filter.length) {
+      BitBlockCount block = bit_counter.NextFourWords();
+      output_size += block.popcount;
+      position += block.length;
+    }
+  }
+  return output_size;
+}
+
+template <typename IndexType>
+Result<std::shared_ptr<ArrayData>> GetTakeIndicesImpl(
+    const ArrayData& filter, FilterOptions::NullSelectionBehavior null_selection,
+    MemoryPool* memory_pool) {
+  using T = typename IndexType::c_type;
+  typename TypeTraits<IndexType>::BuilderType builder(memory_pool);
+
+  const uint8_t* filter_data = filter.buffers[1]->data();
+  BitBlockCounter data_counter(filter_data, filter.offset, filter.length);
+
+  // The position relative to the start of the filter
+  T position = 0;
+
+  // The current position taking the filter offset into account
+  int64_t position_with_offset = filter.offset;
+  if (filter.GetNullCount() > 0) {

Review comment:
       These massive if/else blocks are hard to read, could you factor them into different functions?

##########
File path: cpp/src/arrow/compute/kernels/vector_selection.cc
##########
@@ -0,0 +1,1816 @@
+// 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 <algorithm>
+#include <cstring>
+#include <limits>
+
+#include "arrow/array/array_base.h"
+#include "arrow/array/array_binary.h"
+#include "arrow/array/array_dict.h"
+#include "arrow/array/array_nested.h"
+#include "arrow/array/builder_primitive.h"
+#include "arrow/array/concatenate.h"
+#include "arrow/buffer_builder.h"
+#include "arrow/compute/api_vector.h"
+#include "arrow/compute/kernels/common.h"
+#include "arrow/compute/kernels/util_internal.h"
+#include "arrow/extension_type.h"
+#include "arrow/record_batch.h"
+#include "arrow/result.h"
+#include "arrow/util/bit_block_counter.h"
+#include "arrow/util/bit_util.h"
+#include "arrow/util/bitmap_ops.h"
+#include "arrow/util/bitmap_reader.h"
+#include "arrow/util/int_util.h"
+
+namespace arrow {
+
+using internal::BinaryBitBlockCounter;
+using internal::BitBlockCount;
+using internal::BitBlockCounter;
+using internal::BitmapReader;
+using internal::CopyBitmap;
+using internal::GetArrayView;
+using internal::IndexBoundsCheck;
+using internal::OptionalBitBlockCounter;
+using internal::OptionalBitIndexer;
+
+namespace compute {
+namespace internal {
+
+int64_t GetFilterOutputSize(const ArrayData& filter,
+                            FilterOptions::NullSelectionBehavior null_selection) {
+  int64_t output_size = 0;
+  int64_t position = 0;
+  if (filter.GetNullCount() > 0) {
+    const uint8_t* filter_is_valid = filter.buffers[0]->data();
+    BinaryBitBlockCounter bit_counter(filter.buffers[1]->data(), filter.offset,
+                                      filter_is_valid, filter.offset, filter.length);
+    if (null_selection == FilterOptions::EMIT_NULL) {
+      while (position < filter.length) {
+        BitBlockCount block = bit_counter.NextOrNotWord();
+        output_size += block.popcount;
+        position += block.length;
+      }
+    } else {
+      while (position < filter.length) {
+        BitBlockCount block = bit_counter.NextAndWord();
+        output_size += block.popcount;
+        position += block.length;
+      }
+    }
+  } else {
+    // The filter has no nulls, so we plow through its data as fast as
+    // possible.
+    BitBlockCounter bit_counter(filter.buffers[1]->data(), filter.offset, filter.length);
+    while (position < filter.length) {
+      BitBlockCount block = bit_counter.NextFourWords();
+      output_size += block.popcount;
+      position += block.length;
+    }
+  }
+  return output_size;
+}
+
+template <typename IndexType>
+Result<std::shared_ptr<ArrayData>> GetTakeIndicesImpl(
+    const ArrayData& filter, FilterOptions::NullSelectionBehavior null_selection,
+    MemoryPool* memory_pool) {
+  using T = typename IndexType::c_type;
+  typename TypeTraits<IndexType>::BuilderType builder(memory_pool);
+
+  const uint8_t* filter_data = filter.buffers[1]->data();
+  BitBlockCounter data_counter(filter_data, filter.offset, filter.length);
+
+  // The position relative to the start of the filter
+  T position = 0;
+
+  // The current position taking the filter offset into account
+  int64_t position_with_offset = filter.offset;
+  if (filter.GetNullCount() > 0) {
+    // The filter has nulls, so we scan the validity bitmap and the filter data
+    // bitmap together, branching on the null selection type.
+    const uint8_t* filter_is_valid = filter.buffers[0]->data();
+
+    // To count blocks whether filter_data[i] || !filter_is_valid[i]
+    BinaryBitBlockCounter filter_counter(filter_data, filter.offset, filter_is_valid,
+                                         filter.offset, filter.length);
+    if (null_selection == FilterOptions::DROP) {
+      while (position < filter.length) {
+        BitBlockCount and_block = filter_counter.NextAndWord();
+        RETURN_NOT_OK(builder.Reserve(and_block.popcount));
+        if (and_block.AllSet()) {
+          // All the values are selected and non-null
+          for (int64_t i = 0; i < and_block.length; ++i) {
+            builder.UnsafeAppend(position++);
+          }
+          position_with_offset += and_block.length;
+        } else {
+          // Some of the values are false or null
+          for (int64_t i = 0; i < and_block.length; ++i) {
+            if (BitUtil::GetBit(filter_is_valid, position_with_offset) &&
+                BitUtil::GetBit(filter_data, position_with_offset)) {
+              builder.UnsafeAppend(position);
+            }
+            ++position;
+            ++position_with_offset;
+          }
+        }
+      }
+    } else {
+      BitBlockCounter is_valid_counter(filter_is_valid, filter.offset, filter.length);
+      while (position < filter.length) {
+        // true OR NOT valid
+        BitBlockCount or_not_block = filter_counter.NextOrNotWord();
+        RETURN_NOT_OK(builder.Reserve(or_not_block.popcount));
+
+        // If the values are all valid and the or_not_block is full, then we
+        // can infer that all the values are true and skip the bit checking
+        BitBlockCount is_valid_block = is_valid_counter.NextWord();
+
+        if (or_not_block.AllSet() && is_valid_block.AllSet()) {
+          // All the values are selected and non-null
+          for (int64_t i = 0; i < or_not_block.length; ++i) {
+            builder.UnsafeAppend(position++);
+          }
+          position_with_offset += or_not_block.length;
+        } else {
+          // Some of the values are false or null
+          for (int64_t i = 0; i < or_not_block.length; ++i) {
+            if (BitUtil::GetBit(filter_is_valid, position_with_offset)) {
+              if (BitUtil::GetBit(filter_data, position_with_offset)) {
+                builder.UnsafeAppend(position);
+              }
+            } else {
+              // Null slot, so append a null
+              builder.UnsafeAppendNull();
+            }
+            ++position;
+            ++position_with_offset;
+          }
+        }
+      }
+    }
+  } else {
+    // The filter has no nulls, so we need only look for true values
+    BitBlockCount current_block = data_counter.NextWord();
+    while (position < filter.length) {
+      if (current_block.AllSet()) {
+        int64_t run_length = 0;
+
+        // If we've found a all-true block, then we scan forward until we find
+        // a block that has some false values (or we reach the end
+        while (current_block.length > 0 && current_block.AllSet()) {
+          run_length += current_block.length;
+          current_block = data_counter.NextWord();
+        }
+
+        // Append the consecutive run of indices
+        RETURN_NOT_OK(builder.Reserve(run_length));
+        for (int64_t i = 0; i < run_length; ++i) {
+          builder.UnsafeAppend(position++);
+        }
+        position_with_offset += run_length;
+      } else {
+        // Must do bitchecking on the current block
+        RETURN_NOT_OK(builder.Reserve(current_block.popcount));
+        for (int64_t i = 0; i < current_block.length; ++i) {
+          if (BitUtil::GetBit(filter_data, position_with_offset)) {
+            builder.UnsafeAppend(position);
+          }
+          ++position;
+          ++position_with_offset;
+        }
+        current_block = data_counter.NextWord();
+      }
+    }
+  }
+  std::shared_ptr<ArrayData> result;
+  RETURN_NOT_OK(builder.FinishInternal(&result));
+  return result;
+}
+
+Result<std::shared_ptr<ArrayData>> GetTakeIndices(
+    const ArrayData& filter, FilterOptions::NullSelectionBehavior null_selection,
+    MemoryPool* memory_pool) {
+  DCHECK_EQ(filter.type->id(), Type::BOOL);
+  if (filter.length <= std::numeric_limits<uint16_t>::max()) {
+    return GetTakeIndicesImpl<UInt16Type>(filter, null_selection, memory_pool);
+  } else if (filter.length <= std::numeric_limits<uint32_t>::max()) {
+    return GetTakeIndicesImpl<UInt32Type>(filter, null_selection, memory_pool);
+  } else {
+    // Arrays over 4 billion elements, not especially likely.
+    return Status::NotImplemented(
+        "Filter length exceeds UINT32_MAX, "
+        "consider a different strategy for selecting elements");
+  }
+}
+
+namespace {
+
+template <typename ArrowType>
+struct GetCType {
+  using type = typename ArrowType::c_type;
+};
+
+// We want uint8_t for boolean instead of bool
+template <>
+struct GetCType<BooleanType> {
+  using type = uint8_t;
+};
+
+using FilterState = OptionsWrapper<FilterOptions>;
+using TakeState = OptionsWrapper<TakeOptions>;
+
+Status PreallocateData(KernelContext* ctx, int64_t length, int bit_width, Datum* out) {
+  // Preallocate memory
+  ArrayData* out_arr = out->mutable_array();
+  out_arr->length = length;
+  out_arr->buffers.resize(2);
+
+  ARROW_ASSIGN_OR_RAISE(out_arr->buffers[0], ctx->AllocateBitmap(length));
+  if (bit_width == 1) {
+    ARROW_ASSIGN_OR_RAISE(out_arr->buffers[1], ctx->AllocateBitmap(length));
+  } else {
+    ARROW_ASSIGN_OR_RAISE(out_arr->buffers[1], ctx->Allocate(length * bit_width / 8));
+  }
+  return Status::OK();
+}
+
+// ----------------------------------------------------------------------
+// Implement optimized take for primitive types from boolean to 1/2/4/8-byte
+// C-type based types. Use common implementation for every byte width and only
+// generate code for unsigned integer indices, since after boundschecking to
+// check for negative numbers in the indices we can safely reinterpret_cast
+// signed integers as unsigned.
+
+/// \brief The Take implementation for primitive (fixed-width) types does not
+/// use the logical Arrow type but rather the physical C type. This way we
+/// only generate one take function for each byte width.
+///
+/// This function assumes that the indices have been boundschecked.
+template <typename IndexCType, typename ValueCType>
+struct PrimitiveTakeImpl {
+  static void Exec(const PrimitiveArg& values, const PrimitiveArg& indices,
+                   Datum* out_datum) {
+    auto values_data = reinterpret_cast<const ValueCType*>(values.data);
+    auto values_is_valid = values.is_valid;
+    auto values_offset = values.offset;
+
+    auto indices_data = reinterpret_cast<const IndexCType*>(indices.data);
+    auto indices_is_valid = indices.is_valid;
+    auto indices_offset = indices.offset;
+
+    ArrayData* out_arr = out_datum->mutable_array();
+    auto out = out_arr->GetMutableValues<ValueCType>(1);
+    auto out_is_valid = out_arr->buffers[0]->mutable_data();
+    auto out_offset = out_arr->offset;
+
+    // If either the values or indices have nulls, we preemptively zero out the
+    // out validity bitmap so that we don't have to use ClearBit in each
+    // iteration for nulls.
+    if (values.null_count > 0 || indices.null_count > 0) {
+      BitUtil::SetBitsTo(out_is_valid, out_offset, indices.length, false);
+    }
+
+    OptionalBitBlockCounter indices_bit_counter(indices_is_valid, indices_offset,
+                                                indices.length);
+    int64_t position = 0;
+    int64_t valid_count = 0;
+    while (position < indices.length) {
+      BitBlockCount block = indices_bit_counter.NextBlock();
+      if (values.null_count == 0) {
+        // Values are never null, so things are easier
+        valid_count += block.popcount;
+        if (block.popcount == block.length) {
+          // Fastest path: neither values nor index nulls
+          BitUtil::SetBitsTo(out_is_valid, out_offset + position, block.length, true);
+          for (int64_t i = 0; i < block.length; ++i) {
+            out[position] = values_data[indices_data[position]];
+            ++position;
+          }
+        } else if (block.popcount > 0) {
+          // Slow path: some indices but not all are null
+          for (int64_t i = 0; i < block.length; ++i) {
+            if (BitUtil::GetBit(indices_is_valid, indices_offset + position)) {
+              // index is not null
+              BitUtil::SetBit(out_is_valid, out_offset + position);
+              out[position] = values_data[indices_data[position]];
+            } else {
+              out[position] = ValueCType{};
+            }
+            ++position;
+          }
+        } else {
+          memset(out + position, 0, sizeof(ValueCType) * block.length);
+          position += block.length;
+        }
+      } else {
+        // Values have nulls, so we must do random access into the values bitmap
+        if (block.popcount == block.length) {
+          // Faster path: indices are not null but values may be
+          for (int64_t i = 0; i < block.length; ++i) {
+            if (BitUtil::GetBit(values_is_valid,
+                                values_offset + indices_data[position])) {
+              // value is not null
+              out[position] = values_data[indices_data[position]];
+              BitUtil::SetBit(out_is_valid, out_offset + position);
+              ++valid_count;
+            } else {
+              out[position] = ValueCType{};
+            }
+            ++position;
+          }
+        } else if (block.popcount > 0) {
+          // Slow path: some but not all indices are null. Since we are doing
+          // random access in general we have to check the value nullness one by
+          // one.
+          for (int64_t i = 0; i < block.length; ++i) {
+            if (BitUtil::GetBit(indices_is_valid, indices_offset + position) &&
+                BitUtil::GetBit(values_is_valid,
+                                values_offset + indices_data[position])) {
+              // index is not null && value is not null
+              out[position] = values_data[indices_data[position]];
+              BitUtil::SetBit(out_is_valid, out_offset + position);
+              ++valid_count;
+            } else {
+              out[position] = ValueCType{};
+            }
+            ++position;
+          }
+        } else {
+          memset(out + position, 0, sizeof(ValueCType) * block.length);
+          position += block.length;
+        }
+      }
+    }
+    out_arr->null_count = out_arr->length - valid_count;
+  }
+};
+
+template <typename IndexCType>
+struct BooleanTakeImpl {
+  static void Exec(const PrimitiveArg& values, const PrimitiveArg& indices,
+                   Datum* out_datum) {
+    const uint8_t* values_data = values.data;
+    auto values_is_valid = values.is_valid;
+    auto values_offset = values.offset;
+
+    auto indices_data = reinterpret_cast<const IndexCType*>(indices.data);
+    auto indices_is_valid = indices.is_valid;
+    auto indices_offset = indices.offset;
+
+    ArrayData* out_arr = out_datum->mutable_array();
+    auto out = out_arr->buffers[1]->mutable_data();
+    auto out_is_valid = out_arr->buffers[0]->mutable_data();
+    auto out_offset = out_arr->offset;
+
+    // If either the values or indices have nulls, we preemptively zero out the
+    // out validity bitmap so that we don't have to use ClearBit in each
+    // iteration for nulls.
+    if (values.null_count > 0 || indices.null_count > 0) {
+      BitUtil::SetBitsTo(out_is_valid, out_offset, indices.length, false);
+    }
+    // Avoid uninitialized data in values array
+    BitUtil::SetBitsTo(out, out_offset, indices.length, false);
+
+    auto PlaceDataBit = [&](int64_t loc, IndexCType index) {
+      BitUtil::SetBitTo(out, out_offset + loc,
+                        BitUtil::GetBit(values_data, values_offset + index));
+    };
+
+    OptionalBitBlockCounter indices_bit_counter(indices_is_valid, indices_offset,
+                                                indices.length);
+    int64_t position = 0;
+    int64_t valid_count = 0;
+    while (position < indices.length) {
+      BitBlockCount block = indices_bit_counter.NextBlock();
+      if (values.null_count == 0) {
+        // Values are never null, so things are easier
+        valid_count += block.popcount;
+        if (block.popcount == block.length) {
+          // Fastest path: neither values nor index nulls
+          BitUtil::SetBitsTo(out_is_valid, out_offset + position, block.length, true);
+          for (int64_t i = 0; i < block.length; ++i) {
+            PlaceDataBit(position, indices_data[position]);
+            ++position;
+          }
+        } else if (block.popcount > 0) {
+          // Slow path: some but not all indices are null
+          for (int64_t i = 0; i < block.length; ++i) {
+            if (BitUtil::GetBit(indices_is_valid, indices_offset + position)) {
+              // index is not null
+              BitUtil::SetBit(out_is_valid, out_offset + position);
+              PlaceDataBit(position, indices_data[position]);
+            }
+            ++position;
+          }
+        } else {
+          position += block.length;
+        }
+      } else {
+        // Values have nulls, so we must do random access into the values bitmap
+        if (block.popcount == block.length) {
+          // Faster path: indices are not null but values may be
+          for (int64_t i = 0; i < block.length; ++i) {
+            if (BitUtil::GetBit(values_is_valid,
+                                values_offset + indices_data[position])) {
+              // value is not null
+              BitUtil::SetBit(out_is_valid, out_offset + position);
+              PlaceDataBit(position, indices_data[position]);
+              ++valid_count;
+            }
+            ++position;
+          }
+        } else if (block.popcount > 0) {
+          // Slow path: some but not all indices are null. Since we are doing
+          // random access in general we have to check the value nullness one by
+          // one.
+          for (int64_t i = 0; i < block.length; ++i) {
+            if (BitUtil::GetBit(indices_is_valid, indices_offset + position)) {
+              // index is not null
+              if (BitUtil::GetBit(values_is_valid,
+                                  values_offset + indices_data[position])) {
+                // value is not null
+                PlaceDataBit(position, indices_data[position]);
+                BitUtil::SetBit(out_is_valid, out_offset + position);
+                ++valid_count;
+              }
+            }
+            ++position;
+          }
+        } else {
+          position += block.length;
+        }
+      }
+    }
+    out_arr->null_count = out_arr->length - valid_count;
+  }
+};
+
+template <template <typename...> class TakeImpl, typename... Args>
+void TakeIndexDispatch(const PrimitiveArg& values, const PrimitiveArg& indices,
+                       Datum* out) {
+  // With the simplifying assumption that boundschecking has taken place
+  // already at a higher level, we can now assume that the index values are all
+  // non-negative. Thus, we can interpret signed integers as unsigned and avoid
+  // having to generate double the amount of binary code to handle each integer
+  // width.
+  switch (indices.bit_width) {
+    case 8:
+      return TakeImpl<uint8_t, Args...>::Exec(values, indices, out);
+    case 16:
+      return TakeImpl<uint16_t, Args...>::Exec(values, indices, out);
+    case 32:
+      return TakeImpl<uint32_t, Args...>::Exec(values, indices, out);
+    case 64:
+      return TakeImpl<uint64_t, Args...>::Exec(values, indices, out);
+    default:
+      DCHECK(false) << "Invalid indices byte width";
+      break;
+  }
+}
+
+void PrimitiveTake(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+  const auto& state = checked_cast<const TakeState&>(*ctx->state());
+  if (state.options.boundscheck) {
+    KERNEL_RETURN_IF_ERROR(ctx, IndexBoundsCheck(*batch[1].array(), batch[0].length()));
+  }
+
+  PrimitiveArg values = GetPrimitiveArg(*batch[0].array());
+  PrimitiveArg indices = GetPrimitiveArg(*batch[1].array());
+  KERNEL_RETURN_IF_ERROR(ctx,
+                         PreallocateData(ctx, indices.length, values.bit_width, out));
+  switch (values.bit_width) {
+    case 1:
+      return TakeIndexDispatch<BooleanTakeImpl>(values, indices, out);
+    case 8:
+      return TakeIndexDispatch<PrimitiveTakeImpl, int8_t>(values, indices, out);
+    case 16:
+      return TakeIndexDispatch<PrimitiveTakeImpl, int16_t>(values, indices, out);
+    case 32:
+      return TakeIndexDispatch<PrimitiveTakeImpl, int32_t>(values, indices, out);
+    case 64:
+      return TakeIndexDispatch<PrimitiveTakeImpl, int64_t>(values, indices, out);
+    default:
+      DCHECK(false) << "Invalid values byte width";
+      break;
+  }
+}
+
+// ----------------------------------------------------------------------
+// Optimized and streamlined filter for primitive types
+
+// Use either BitBlockCounter or BinaryBitBlockCounter to quickly scan filter a
+// word at a time for the DROP selection type.
+class DropNullCounter {
+ public:
+  // validity bitmap may be null
+  DropNullCounter(const uint8_t* validity, const uint8_t* data, int64_t offset,
+                  int64_t length)
+      : data_counter_(data, offset, length),
+        data_and_validity_counter_(data, offset, validity, offset, length),
+        has_validity_(validity != nullptr) {}
+
+  BitBlockCount NextBlock() {
+    if (has_validity_) {
+      // filter is true AND not null
+      return data_and_validity_counter_.NextAndWord();
+    } else {
+      return data_counter_.NextWord();
+    }
+  }
+
+ private:
+  // For when just data is present, but no validity bitmap
+  BitBlockCounter data_counter_;
+
+  // For when both validity bitmap and data are present
+  BinaryBitBlockCounter data_and_validity_counter_;
+  bool has_validity_;
+};
+
+/// \brief The Filter implementation for primitive (fixed-width) types does not
+/// use the logical Arrow type but rather then physical C type. This way we
+/// only generate one take function for each byte width. We use the same
+/// implementation here for boolean and fixed-byte-size inputs with some
+/// template specialization.
+template <typename ArrowType>
+class PrimitiveFilterImpl {
+ public:
+  using T = typename GetCType<ArrowType>::type;

Review comment:
       Since this is fairly simple and only used here, it'd be more readable to inline this logic
   ```suggestion
     using T = typename std::conditional<std::is_same<ArrowType, BooleanType>::value,
         uint8_t,
         typename ArrowType::c_type>::type;
   ```




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

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



[GitHub] [arrow] wesm edited a comment on pull request #7442: ARROW-9075: [C++] Optimized Filter implementation: faster performance + compilation, smaller code size

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


   I think I improved some of the readability problems and addressed the other comments. I'd like to merge this soon once CI is green


----------------------------------------------------------------
This is an automated message from the 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] wesm commented on a change in pull request #7442: ARROW-9075: [C++] Optimized Filter implementation: faster performance + compilation, smaller code size

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



##########
File path: cpp/src/arrow/compute/kernels/vector_selection.cc
##########
@@ -0,0 +1,1816 @@
+// 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 <algorithm>
+#include <cstring>
+#include <limits>
+
+#include "arrow/array/array_base.h"
+#include "arrow/array/array_binary.h"
+#include "arrow/array/array_dict.h"
+#include "arrow/array/array_nested.h"
+#include "arrow/array/builder_primitive.h"
+#include "arrow/array/concatenate.h"
+#include "arrow/buffer_builder.h"
+#include "arrow/compute/api_vector.h"
+#include "arrow/compute/kernels/common.h"
+#include "arrow/compute/kernels/util_internal.h"
+#include "arrow/extension_type.h"
+#include "arrow/record_batch.h"
+#include "arrow/result.h"
+#include "arrow/util/bit_block_counter.h"
+#include "arrow/util/bit_util.h"
+#include "arrow/util/bitmap_ops.h"
+#include "arrow/util/bitmap_reader.h"
+#include "arrow/util/int_util.h"
+
+namespace arrow {
+
+using internal::BinaryBitBlockCounter;
+using internal::BitBlockCount;
+using internal::BitBlockCounter;
+using internal::BitmapReader;
+using internal::CopyBitmap;
+using internal::GetArrayView;
+using internal::IndexBoundsCheck;
+using internal::OptionalBitBlockCounter;
+using internal::OptionalBitIndexer;
+
+namespace compute {
+namespace internal {
+
+int64_t GetFilterOutputSize(const ArrayData& filter,
+                            FilterOptions::NullSelectionBehavior null_selection) {
+  int64_t output_size = 0;
+  int64_t position = 0;
+  if (filter.GetNullCount() > 0) {
+    const uint8_t* filter_is_valid = filter.buffers[0]->data();
+    BinaryBitBlockCounter bit_counter(filter.buffers[1]->data(), filter.offset,
+                                      filter_is_valid, filter.offset, filter.length);
+    if (null_selection == FilterOptions::EMIT_NULL) {
+      while (position < filter.length) {
+        BitBlockCount block = bit_counter.NextOrNotWord();
+        output_size += block.popcount;
+        position += block.length;
+      }
+    } else {
+      while (position < filter.length) {
+        BitBlockCount block = bit_counter.NextAndWord();
+        output_size += block.popcount;
+        position += block.length;
+      }
+    }
+  } else {
+    // The filter has no nulls, so we plow through its data as fast as
+    // possible.
+    BitBlockCounter bit_counter(filter.buffers[1]->data(), filter.offset, filter.length);
+    while (position < filter.length) {
+      BitBlockCount block = bit_counter.NextFourWords();
+      output_size += block.popcount;
+      position += block.length;
+    }
+  }
+  return output_size;
+}
+
+template <typename IndexType>
+Result<std::shared_ptr<ArrayData>> GetTakeIndicesImpl(
+    const ArrayData& filter, FilterOptions::NullSelectionBehavior null_selection,
+    MemoryPool* memory_pool) {
+  using T = typename IndexType::c_type;
+  typename TypeTraits<IndexType>::BuilderType builder(memory_pool);
+
+  const uint8_t* filter_data = filter.buffers[1]->data();
+  BitBlockCounter data_counter(filter_data, filter.offset, filter.length);
+
+  // The position relative to the start of the filter
+  T position = 0;
+
+  // The current position taking the filter offset into account
+  int64_t position_with_offset = filter.offset;
+  if (filter.GetNullCount() > 0) {
+    // The filter has nulls, so we scan the validity bitmap and the filter data
+    // bitmap together, branching on the null selection type.
+    const uint8_t* filter_is_valid = filter.buffers[0]->data();
+
+    // To count blocks whether filter_data[i] || !filter_is_valid[i]
+    BinaryBitBlockCounter filter_counter(filter_data, filter.offset, filter_is_valid,
+                                         filter.offset, filter.length);
+    if (null_selection == FilterOptions::DROP) {
+      while (position < filter.length) {
+        BitBlockCount and_block = filter_counter.NextAndWord();
+        RETURN_NOT_OK(builder.Reserve(and_block.popcount));
+        if (and_block.AllSet()) {
+          // All the values are selected and non-null
+          for (int64_t i = 0; i < and_block.length; ++i) {
+            builder.UnsafeAppend(position++);
+          }
+          position_with_offset += and_block.length;
+        } else {
+          // Some of the values are false or null

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.

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



[GitHub] [arrow] wesm commented on pull request #7442: ARROW-9075: [C++] Optimized Filter implementation: faster performance + compilation, smaller code size

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


   I implemented some other optimizations, especially for the case where neither values nor filter contain nulls. I'm working on updated benchmarks


----------------------------------------------------------------
This is an automated message from the 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] wesm edited a comment on pull request #7442: ARROW-9075: [C++] Optimized Filter implementation: faster performance + compilation, smaller code size

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


   @ursabot benchmark --help


----------------------------------------------------------------
This is an automated message from the 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] wesm edited a comment on pull request #7442: ARROW-9075: [C++] Optimized Filter implementation: faster performance + compilation, smaller code size

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


   Here's benchmark runs on my machine
   
   * BEFORE: https://gist.github.com/wesm/857a3179e7dbc928d3325b1e7f687086
   * AFTER: https://gist.github.com/wesm/ad07cec1613b6327926dfe1d95e7f4f0
   
   **If you want to benchmark yourself, please use this branch for the "before":** https://github.com/wesm/arrow/tree/ARROW-9075-comparison. It contains the RandomArrayGenerator::Boolean change and some other changes to the benchmarks without which the results will be non-comparable


----------------------------------------------------------------
This is an automated message from the 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] wesm commented on pull request #7442: ARROW-9075: [C++] Optimized Filter implementation: faster performance + compilation, smaller code size

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


   @ursabot benchmark --benchmark-filter=Filter 04006ff


----------------------------------------------------------------
This is an automated message from the 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] wesm commented on pull request #7442: ARROW-9075: [C++] Optimized Filter implementation: faster performance + compilation, smaller code size

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


   The RTools 4.0 build is spurious. This is ready for review


----------------------------------------------------------------
This is an automated message from the 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] wesm commented on a change in pull request #7442: ARROW-9075: [C++] Optimized Filter implementation: faster performance + compilation, smaller code size

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



##########
File path: cpp/src/arrow/compute/kernels/vector_selection.cc
##########
@@ -0,0 +1,1816 @@
+// 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 <algorithm>
+#include <cstring>
+#include <limits>
+
+#include "arrow/array/array_base.h"
+#include "arrow/array/array_binary.h"
+#include "arrow/array/array_dict.h"
+#include "arrow/array/array_nested.h"
+#include "arrow/array/builder_primitive.h"
+#include "arrow/array/concatenate.h"
+#include "arrow/buffer_builder.h"
+#include "arrow/compute/api_vector.h"
+#include "arrow/compute/kernels/common.h"
+#include "arrow/compute/kernels/util_internal.h"
+#include "arrow/extension_type.h"
+#include "arrow/record_batch.h"
+#include "arrow/result.h"
+#include "arrow/util/bit_block_counter.h"
+#include "arrow/util/bit_util.h"
+#include "arrow/util/bitmap_ops.h"
+#include "arrow/util/bitmap_reader.h"
+#include "arrow/util/int_util.h"
+
+namespace arrow {
+
+using internal::BinaryBitBlockCounter;
+using internal::BitBlockCount;
+using internal::BitBlockCounter;
+using internal::BitmapReader;
+using internal::CopyBitmap;
+using internal::GetArrayView;
+using internal::IndexBoundsCheck;
+using internal::OptionalBitBlockCounter;
+using internal::OptionalBitIndexer;
+
+namespace compute {
+namespace internal {
+
+int64_t GetFilterOutputSize(const ArrayData& filter,
+                            FilterOptions::NullSelectionBehavior null_selection) {
+  int64_t output_size = 0;
+  int64_t position = 0;
+  if (filter.GetNullCount() > 0) {
+    const uint8_t* filter_is_valid = filter.buffers[0]->data();
+    BinaryBitBlockCounter bit_counter(filter.buffers[1]->data(), filter.offset,

Review comment:
       Maybe. Let's leave this for follow up because I would need some benchmarks to be comfortable doing 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] pitrou commented on pull request #7442: ARROW-9075: [C++] Optimized Filter implementation: faster performance + compilation, smaller code size

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


   Everything is much faster here, including string filtering.


----------------------------------------------------------------
This is an automated message from the 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] wesm commented on pull request #7442: ARROW-9075: [C++] Optimized Filter implementation: faster performance + compilation, smaller code size

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


   I'll have to deal with the string optimization in a follow up PR, so I'm going to leave this for review as is. It would be good to get this merged sooner rather than later


----------------------------------------------------------------
This is an automated message from the 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] wesm edited a comment on pull request #7442: ARROW-9075: [C++] Optimized Filter implementation: faster performance + compilation, smaller code size

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


   I'll have to deal with the string optimization in a follow up PR, so I'm going to leave this for review as is. It would be good to get this merged sooner rather than later.
   
   EDIT: opened https://issues.apache.org/jira/browse/ARROW-9152


----------------------------------------------------------------
This is an automated message from the 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] wesm commented on pull request #7442: ARROW-9075: [C++] Optimized Filter implementation: faster performance + compilation, smaller code size

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


   I found some issues in the Python benchmarks I posted before. Here's the updated setup and current numbers
   
   setup (I was including the cost of converting NumPy booleans to Arrow booleans in the prior results). I also added a "worst case scenario" where 50% of values are not selected
   
   ```
   import numpy as np
   import pandas as pd
   import pyarrow as pa
   import pyarrow.compute as pc
   
   string_values = pa.array([pd.util.testing.rands(16)
                             for i in range(10000)] * 100)
   double_values = pa.array(np.random.randn(1000000))
   
   all_but_one = np.ones(len(string_values), dtype=bool)
   all_but_one[500000] = False
   
   one_in_2 = np.array(np.random.binomial(1, 0.50, size=1000000), dtype=bool)
   one_in_100 = np.array(np.random.binomial(1, 0.01, size=1000000), dtype=bool)
   one_in_1000 = np.array(np.random.binomial(1, 0.001, size=1000000), dtype=bool)
   
   all_but_one = pa.array(all_but_one)
   one_in_2 = pa.array(one_in_2)
   one_in_100 = pa.array(one_in_100)
   one_in_1000 = pa.array(one_in_1000)
   ```
   
   before:
   
   ```
   In [2]: timeit pc.filter(double_values, all_but_one)                                                                                                                                           
   5.15 ms ± 26.2 µs per loop (mean ± std. dev. of 7 runs, 100 loops each)
   
   In [3]: timeit pc.filter(double_values, one_in_100)                                                                                                                                            
   1.45 ms ± 8.44 µs per loop (mean ± std. dev. of 7 runs, 1000 loops each)
   
   In [4]: timeit pc.filter(double_values, one_in_1000)                                                                                                                                           
   1.37 ms ± 8.25 µs per loop (mean ± std. dev. of 7 runs, 1000 loops each)
   
   In [5]: timeit pc.filter(double_values, one_in_2)                                                                                                                                              
   7.08 ms ± 108 µs per loop (mean ± std. dev. of 7 runs, 100 loops each)
   
   In [6]: timeit pc.filter(string_values, all_but_one)                                                                                                                                           
   11 ms ± 204 µs per loop (mean ± std. dev. of 7 runs, 100 loops each)
   
   In [7]: timeit pc.filter(string_values, one_in_100)                                                                                                                                            
   1.64 ms ± 9.58 µs per loop (mean ± std. dev. of 7 runs, 1000 loops each)
   
   In [8]: timeit pc.filter(string_values, one_in_1000)                                                                                                                                           
   1.45 ms ± 4.21 µs per loop (mean ± std. dev. of 7 runs, 1000 loops each)
   
   In [9]: timeit pc.filter(string_values, one_in_2)                                                                                                                                              
   11.4 ms ± 117 µs per loop (mean ± std. dev. of 7 runs, 100 loops each)
   ```
   
   after:
   
   ```
   In [2]: timeit pc.filter(double_values, all_but_one)                                                                                                                                           
   370 µs ± 2.69 µs per loop (mean ± std. dev. of 7 runs, 1000 loops each)
   
   In [3]: timeit pc.filter(double_values, one_in_100)                                                                                                                                            
   645 µs ± 3.82 µs per loop (mean ± std. dev. of 7 runs, 1000 loops each)
   
   In [4]: timeit pc.filter(double_values, one_in_1000)                                                                                                                                           
   124 µs ± 1.51 µs per loop (mean ± std. dev. of 7 runs, 10000 loops each)
   
   In [6]: timeit pc.filter(double_values, one_in_2)                                                                                                                                              
   5.11 ms ± 38.2 µs per loop (mean ± std. dev. of 7 runs, 100 loops each)
   
   In [8]: timeit pc.filter(string_values, all_but_one)                                                                                                                                           
   6.51 ms ± 21.3 µs per loop (mean ± std. dev. of 7 runs, 100 loops each)
   
   In [9]: timeit pc.filter(string_values, one_in_100)                                                                                                                                            
   680 µs ± 3.63 µs per loop (mean ± std. dev. of 7 runs, 1000 loops each)
   
   In [10]: timeit pc.filter(string_values, one_in_1000)                                                                                                                                          
   188 µs ± 849 ns per loop (mean ± std. dev. of 7 runs, 10000 loops each)
   
   In [11]: timeit pc.filter(string_values, one_in_2)                                                                                                                                             
   7.73 ms ± 63.3 µs per loop (mean ± std. dev. of 7 runs, 100 loops each)
   ```


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

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



[GitHub] [arrow] wesm commented on a change in pull request #7442: ARROW-9075: [C++] Optimized Filter implementation: faster performance + compilation, smaller code size

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



##########
File path: cpp/src/arrow/compute/api_vector.h
##########
@@ -64,6 +67,24 @@ Result<Datum> Filter(const Datum& values, const Datum& filter,
                      const FilterOptions& options = FilterOptions::Defaults(),
                      ExecContext* ctx = NULLPTR);
 
+namespace internal {
+
+// These internal functions are implemented in kernels/vector_selection.cc
+
+/// \brief Return the number of selected indices in the boolean filter
+ARROW_EXPORT
+int64_t GetFilterOutputSize(const ArrayData& filter,

Review comment:
       OK




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

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



[GitHub] [arrow] wesm commented on a change in pull request #7442: ARROW-9075: [C++] Optimized Filter implementation: faster performance + compilation, smaller code size

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



##########
File path: cpp/src/arrow/compute/kernels/vector_selection.cc
##########
@@ -0,0 +1,1816 @@
+// 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 <algorithm>
+#include <cstring>
+#include <limits>
+
+#include "arrow/array/array_base.h"
+#include "arrow/array/array_binary.h"
+#include "arrow/array/array_dict.h"
+#include "arrow/array/array_nested.h"
+#include "arrow/array/builder_primitive.h"
+#include "arrow/array/concatenate.h"
+#include "arrow/buffer_builder.h"
+#include "arrow/compute/api_vector.h"
+#include "arrow/compute/kernels/common.h"
+#include "arrow/compute/kernels/util_internal.h"
+#include "arrow/extension_type.h"
+#include "arrow/record_batch.h"
+#include "arrow/result.h"
+#include "arrow/util/bit_block_counter.h"
+#include "arrow/util/bit_util.h"
+#include "arrow/util/bitmap_ops.h"
+#include "arrow/util/bitmap_reader.h"
+#include "arrow/util/int_util.h"
+
+namespace arrow {
+
+using internal::BinaryBitBlockCounter;
+using internal::BitBlockCount;
+using internal::BitBlockCounter;
+using internal::BitmapReader;
+using internal::CopyBitmap;
+using internal::GetArrayView;
+using internal::IndexBoundsCheck;
+using internal::OptionalBitBlockCounter;
+using internal::OptionalBitIndexer;
+
+namespace compute {
+namespace internal {
+
+int64_t GetFilterOutputSize(const ArrayData& filter,
+                            FilterOptions::NullSelectionBehavior null_selection) {
+  int64_t output_size = 0;
+  int64_t position = 0;
+  if (filter.GetNullCount() > 0) {
+    const uint8_t* filter_is_valid = filter.buffers[0]->data();
+    BinaryBitBlockCounter bit_counter(filter.buffers[1]->data(), filter.offset,
+                                      filter_is_valid, filter.offset, filter.length);
+    if (null_selection == FilterOptions::EMIT_NULL) {
+      while (position < filter.length) {
+        BitBlockCount block = bit_counter.NextOrNotWord();
+        output_size += block.popcount;
+        position += block.length;
+      }
+    } else {
+      while (position < filter.length) {
+        BitBlockCount block = bit_counter.NextAndWord();
+        output_size += block.popcount;
+        position += block.length;
+      }
+    }
+  } else {
+    // The filter has no nulls, so we plow through its data as fast as
+    // possible.
+    BitBlockCounter bit_counter(filter.buffers[1]->data(), filter.offset, filter.length);
+    while (position < filter.length) {
+      BitBlockCount block = bit_counter.NextFourWords();
+      output_size += block.popcount;
+      position += block.length;
+    }
+  }
+  return output_size;
+}
+
+template <typename IndexType>
+Result<std::shared_ptr<ArrayData>> GetTakeIndicesImpl(
+    const ArrayData& filter, FilterOptions::NullSelectionBehavior null_selection,
+    MemoryPool* memory_pool) {
+  using T = typename IndexType::c_type;
+  typename TypeTraits<IndexType>::BuilderType builder(memory_pool);
+
+  const uint8_t* filter_data = filter.buffers[1]->data();
+  BitBlockCounter data_counter(filter_data, filter.offset, filter.length);
+
+  // The position relative to the start of the filter
+  T position = 0;
+
+  // The current position taking the filter offset into account
+  int64_t position_with_offset = filter.offset;
+  if (filter.GetNullCount() > 0) {
+    // The filter has nulls, so we scan the validity bitmap and the filter data
+    // bitmap together, branching on the null selection type.
+    const uint8_t* filter_is_valid = filter.buffers[0]->data();
+
+    // To count blocks whether filter_data[i] || !filter_is_valid[i]
+    BinaryBitBlockCounter filter_counter(filter_data, filter.offset, filter_is_valid,
+                                         filter.offset, filter.length);
+    if (null_selection == FilterOptions::DROP) {
+      while (position < filter.length) {
+        BitBlockCount and_block = filter_counter.NextAndWord();
+        RETURN_NOT_OK(builder.Reserve(and_block.popcount));
+        if (and_block.AllSet()) {
+          // All the values are selected and non-null
+          for (int64_t i = 0; i < and_block.length; ++i) {
+            builder.UnsafeAppend(position++);
+          }
+          position_with_offset += and_block.length;
+        } else {
+          // Some of the values are false or null
+          for (int64_t i = 0; i < and_block.length; ++i) {
+            if (BitUtil::GetBit(filter_is_valid, position_with_offset) &&
+                BitUtil::GetBit(filter_data, position_with_offset)) {
+              builder.UnsafeAppend(position);
+            }
+            ++position;
+            ++position_with_offset;
+          }
+        }
+      }
+    } else {
+      BitBlockCounter is_valid_counter(filter_is_valid, filter.offset, filter.length);
+      while (position < filter.length) {
+        // true OR NOT valid
+        BitBlockCount or_not_block = filter_counter.NextOrNotWord();
+        RETURN_NOT_OK(builder.Reserve(or_not_block.popcount));
+
+        // If the values are all valid and the or_not_block is full, then we
+        // can infer that all the values are true and skip the bit checking
+        BitBlockCount is_valid_block = is_valid_counter.NextWord();
+
+        if (or_not_block.AllSet() && is_valid_block.AllSet()) {
+          // All the values are selected and non-null
+          for (int64_t i = 0; i < or_not_block.length; ++i) {
+            builder.UnsafeAppend(position++);
+          }
+          position_with_offset += or_not_block.length;
+        } else {
+          // Some of the values are false or null
+          for (int64_t i = 0; i < or_not_block.length; ++i) {
+            if (BitUtil::GetBit(filter_is_valid, position_with_offset)) {
+              if (BitUtil::GetBit(filter_data, position_with_offset)) {
+                builder.UnsafeAppend(position);
+              }
+            } else {
+              // Null slot, so append a null
+              builder.UnsafeAppendNull();
+            }
+            ++position;
+            ++position_with_offset;
+          }
+        }
+      }
+    }
+  } else {
+    // The filter has no nulls, so we need only look for true values
+    BitBlockCount current_block = data_counter.NextWord();
+    while (position < filter.length) {
+      if (current_block.AllSet()) {
+        int64_t run_length = 0;
+
+        // If we've found a all-true block, then we scan forward until we find
+        // a block that has some false values (or we reach the end
+        while (current_block.length > 0 && current_block.AllSet()) {
+          run_length += current_block.length;
+          current_block = data_counter.NextWord();
+        }
+
+        // Append the consecutive run of indices
+        RETURN_NOT_OK(builder.Reserve(run_length));
+        for (int64_t i = 0; i < run_length; ++i) {
+          builder.UnsafeAppend(position++);
+        }
+        position_with_offset += run_length;
+      } else {
+        // Must do bitchecking on the current block
+        RETURN_NOT_OK(builder.Reserve(current_block.popcount));
+        for (int64_t i = 0; i < current_block.length; ++i) {
+          if (BitUtil::GetBit(filter_data, position_with_offset)) {
+            builder.UnsafeAppend(position);
+          }
+          ++position;
+          ++position_with_offset;
+        }
+        current_block = data_counter.NextWord();
+      }
+    }
+  }
+  std::shared_ptr<ArrayData> result;
+  RETURN_NOT_OK(builder.FinishInternal(&result));
+  return result;
+}
+
+Result<std::shared_ptr<ArrayData>> GetTakeIndices(
+    const ArrayData& filter, FilterOptions::NullSelectionBehavior null_selection,
+    MemoryPool* memory_pool) {
+  DCHECK_EQ(filter.type->id(), Type::BOOL);
+  if (filter.length <= std::numeric_limits<uint16_t>::max()) {
+    return GetTakeIndicesImpl<UInt16Type>(filter, null_selection, memory_pool);
+  } else if (filter.length <= std::numeric_limits<uint32_t>::max()) {
+    return GetTakeIndicesImpl<UInt32Type>(filter, null_selection, memory_pool);
+  } else {
+    // Arrays over 4 billion elements, not especially likely.
+    return Status::NotImplemented(
+        "Filter length exceeds UINT32_MAX, "
+        "consider a different strategy for selecting elements");
+  }
+}
+
+namespace {
+
+template <typename ArrowType>
+struct GetCType {
+  using type = typename ArrowType::c_type;
+};
+
+// We want uint8_t for boolean instead of bool
+template <>
+struct GetCType<BooleanType> {
+  using type = uint8_t;
+};
+
+using FilterState = OptionsWrapper<FilterOptions>;
+using TakeState = OptionsWrapper<TakeOptions>;
+
+Status PreallocateData(KernelContext* ctx, int64_t length, int bit_width, Datum* out) {
+  // Preallocate memory
+  ArrayData* out_arr = out->mutable_array();
+  out_arr->length = length;
+  out_arr->buffers.resize(2);
+
+  ARROW_ASSIGN_OR_RAISE(out_arr->buffers[0], ctx->AllocateBitmap(length));
+  if (bit_width == 1) {
+    ARROW_ASSIGN_OR_RAISE(out_arr->buffers[1], ctx->AllocateBitmap(length));
+  } else {
+    ARROW_ASSIGN_OR_RAISE(out_arr->buffers[1], ctx->Allocate(length * bit_width / 8));
+  }
+  return Status::OK();
+}
+
+// ----------------------------------------------------------------------
+// Implement optimized take for primitive types from boolean to 1/2/4/8-byte
+// C-type based types. Use common implementation for every byte width and only
+// generate code for unsigned integer indices, since after boundschecking to
+// check for negative numbers in the indices we can safely reinterpret_cast
+// signed integers as unsigned.
+
+/// \brief The Take implementation for primitive (fixed-width) types does not
+/// use the logical Arrow type but rather the physical C type. This way we
+/// only generate one take function for each byte width.
+///
+/// This function assumes that the indices have been boundschecked.
+template <typename IndexCType, typename ValueCType>
+struct PrimitiveTakeImpl {
+  static void Exec(const PrimitiveArg& values, const PrimitiveArg& indices,
+                   Datum* out_datum) {
+    auto values_data = reinterpret_cast<const ValueCType*>(values.data);
+    auto values_is_valid = values.is_valid;
+    auto values_offset = values.offset;
+
+    auto indices_data = reinterpret_cast<const IndexCType*>(indices.data);
+    auto indices_is_valid = indices.is_valid;
+    auto indices_offset = indices.offset;
+
+    ArrayData* out_arr = out_datum->mutable_array();
+    auto out = out_arr->GetMutableValues<ValueCType>(1);
+    auto out_is_valid = out_arr->buffers[0]->mutable_data();
+    auto out_offset = out_arr->offset;
+
+    // If either the values or indices have nulls, we preemptively zero out the
+    // out validity bitmap so that we don't have to use ClearBit in each
+    // iteration for nulls.
+    if (values.null_count > 0 || indices.null_count > 0) {
+      BitUtil::SetBitsTo(out_is_valid, out_offset, indices.length, false);
+    }
+
+    OptionalBitBlockCounter indices_bit_counter(indices_is_valid, indices_offset,
+                                                indices.length);
+    int64_t position = 0;
+    int64_t valid_count = 0;
+    while (position < indices.length) {
+      BitBlockCount block = indices_bit_counter.NextBlock();
+      if (values.null_count == 0) {
+        // Values are never null, so things are easier
+        valid_count += block.popcount;
+        if (block.popcount == block.length) {
+          // Fastest path: neither values nor index nulls
+          BitUtil::SetBitsTo(out_is_valid, out_offset + position, block.length, true);
+          for (int64_t i = 0; i < block.length; ++i) {
+            out[position] = values_data[indices_data[position]];
+            ++position;
+          }
+        } else if (block.popcount > 0) {
+          // Slow path: some indices but not all are null
+          for (int64_t i = 0; i < block.length; ++i) {
+            if (BitUtil::GetBit(indices_is_valid, indices_offset + position)) {
+              // index is not null
+              BitUtil::SetBit(out_is_valid, out_offset + position);
+              out[position] = values_data[indices_data[position]];
+            } else {
+              out[position] = ValueCType{};
+            }
+            ++position;
+          }
+        } else {
+          memset(out + position, 0, sizeof(ValueCType) * block.length);
+          position += block.length;
+        }
+      } else {
+        // Values have nulls, so we must do random access into the values bitmap
+        if (block.popcount == block.length) {
+          // Faster path: indices are not null but values may be
+          for (int64_t i = 0; i < block.length; ++i) {
+            if (BitUtil::GetBit(values_is_valid,
+                                values_offset + indices_data[position])) {
+              // value is not null
+              out[position] = values_data[indices_data[position]];
+              BitUtil::SetBit(out_is_valid, out_offset + position);
+              ++valid_count;
+            } else {
+              out[position] = ValueCType{};
+            }
+            ++position;
+          }
+        } else if (block.popcount > 0) {
+          // Slow path: some but not all indices are null. Since we are doing
+          // random access in general we have to check the value nullness one by
+          // one.
+          for (int64_t i = 0; i < block.length; ++i) {
+            if (BitUtil::GetBit(indices_is_valid, indices_offset + position) &&
+                BitUtil::GetBit(values_is_valid,
+                                values_offset + indices_data[position])) {
+              // index is not null && value is not null
+              out[position] = values_data[indices_data[position]];
+              BitUtil::SetBit(out_is_valid, out_offset + position);
+              ++valid_count;
+            } else {
+              out[position] = ValueCType{};
+            }
+            ++position;
+          }
+        } else {
+          memset(out + position, 0, sizeof(ValueCType) * block.length);
+          position += block.length;
+        }
+      }
+    }
+    out_arr->null_count = out_arr->length - valid_count;
+  }
+};
+
+template <typename IndexCType>
+struct BooleanTakeImpl {
+  static void Exec(const PrimitiveArg& values, const PrimitiveArg& indices,
+                   Datum* out_datum) {
+    const uint8_t* values_data = values.data;
+    auto values_is_valid = values.is_valid;
+    auto values_offset = values.offset;
+
+    auto indices_data = reinterpret_cast<const IndexCType*>(indices.data);
+    auto indices_is_valid = indices.is_valid;
+    auto indices_offset = indices.offset;
+
+    ArrayData* out_arr = out_datum->mutable_array();
+    auto out = out_arr->buffers[1]->mutable_data();
+    auto out_is_valid = out_arr->buffers[0]->mutable_data();
+    auto out_offset = out_arr->offset;
+
+    // If either the values or indices have nulls, we preemptively zero out the
+    // out validity bitmap so that we don't have to use ClearBit in each
+    // iteration for nulls.
+    if (values.null_count > 0 || indices.null_count > 0) {
+      BitUtil::SetBitsTo(out_is_valid, out_offset, indices.length, false);
+    }
+    // Avoid uninitialized data in values array
+    BitUtil::SetBitsTo(out, out_offset, indices.length, false);
+
+    auto PlaceDataBit = [&](int64_t loc, IndexCType index) {
+      BitUtil::SetBitTo(out, out_offset + loc,
+                        BitUtil::GetBit(values_data, values_offset + index));
+    };
+
+    OptionalBitBlockCounter indices_bit_counter(indices_is_valid, indices_offset,
+                                                indices.length);
+    int64_t position = 0;
+    int64_t valid_count = 0;
+    while (position < indices.length) {
+      BitBlockCount block = indices_bit_counter.NextBlock();
+      if (values.null_count == 0) {
+        // Values are never null, so things are easier
+        valid_count += block.popcount;
+        if (block.popcount == block.length) {
+          // Fastest path: neither values nor index nulls
+          BitUtil::SetBitsTo(out_is_valid, out_offset + position, block.length, true);
+          for (int64_t i = 0; i < block.length; ++i) {
+            PlaceDataBit(position, indices_data[position]);
+            ++position;
+          }
+        } else if (block.popcount > 0) {
+          // Slow path: some but not all indices are null
+          for (int64_t i = 0; i < block.length; ++i) {
+            if (BitUtil::GetBit(indices_is_valid, indices_offset + position)) {
+              // index is not null
+              BitUtil::SetBit(out_is_valid, out_offset + position);
+              PlaceDataBit(position, indices_data[position]);
+            }
+            ++position;
+          }
+        } else {
+          position += block.length;
+        }
+      } else {
+        // Values have nulls, so we must do random access into the values bitmap
+        if (block.popcount == block.length) {
+          // Faster path: indices are not null but values may be
+          for (int64_t i = 0; i < block.length; ++i) {
+            if (BitUtil::GetBit(values_is_valid,
+                                values_offset + indices_data[position])) {
+              // value is not null
+              BitUtil::SetBit(out_is_valid, out_offset + position);
+              PlaceDataBit(position, indices_data[position]);
+              ++valid_count;
+            }
+            ++position;
+          }
+        } else if (block.popcount > 0) {
+          // Slow path: some but not all indices are null. Since we are doing
+          // random access in general we have to check the value nullness one by
+          // one.
+          for (int64_t i = 0; i < block.length; ++i) {
+            if (BitUtil::GetBit(indices_is_valid, indices_offset + position)) {
+              // index is not null
+              if (BitUtil::GetBit(values_is_valid,
+                                  values_offset + indices_data[position])) {
+                // value is not null
+                PlaceDataBit(position, indices_data[position]);
+                BitUtil::SetBit(out_is_valid, out_offset + position);
+                ++valid_count;
+              }
+            }
+            ++position;
+          }
+        } else {
+          position += block.length;
+        }
+      }
+    }
+    out_arr->null_count = out_arr->length - valid_count;
+  }
+};
+
+template <template <typename...> class TakeImpl, typename... Args>
+void TakeIndexDispatch(const PrimitiveArg& values, const PrimitiveArg& indices,
+                       Datum* out) {
+  // With the simplifying assumption that boundschecking has taken place
+  // already at a higher level, we can now assume that the index values are all
+  // non-negative. Thus, we can interpret signed integers as unsigned and avoid
+  // having to generate double the amount of binary code to handle each integer
+  // width.
+  switch (indices.bit_width) {
+    case 8:
+      return TakeImpl<uint8_t, Args...>::Exec(values, indices, out);
+    case 16:
+      return TakeImpl<uint16_t, Args...>::Exec(values, indices, out);
+    case 32:
+      return TakeImpl<uint32_t, Args...>::Exec(values, indices, out);
+    case 64:
+      return TakeImpl<uint64_t, Args...>::Exec(values, indices, out);
+    default:
+      DCHECK(false) << "Invalid indices byte width";
+      break;
+  }
+}
+
+void PrimitiveTake(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+  const auto& state = checked_cast<const TakeState&>(*ctx->state());
+  if (state.options.boundscheck) {
+    KERNEL_RETURN_IF_ERROR(ctx, IndexBoundsCheck(*batch[1].array(), batch[0].length()));
+  }
+
+  PrimitiveArg values = GetPrimitiveArg(*batch[0].array());
+  PrimitiveArg indices = GetPrimitiveArg(*batch[1].array());
+  KERNEL_RETURN_IF_ERROR(ctx,
+                         PreallocateData(ctx, indices.length, values.bit_width, out));
+  switch (values.bit_width) {
+    case 1:
+      return TakeIndexDispatch<BooleanTakeImpl>(values, indices, out);
+    case 8:
+      return TakeIndexDispatch<PrimitiveTakeImpl, int8_t>(values, indices, out);
+    case 16:
+      return TakeIndexDispatch<PrimitiveTakeImpl, int16_t>(values, indices, out);
+    case 32:
+      return TakeIndexDispatch<PrimitiveTakeImpl, int32_t>(values, indices, out);
+    case 64:
+      return TakeIndexDispatch<PrimitiveTakeImpl, int64_t>(values, indices, out);
+    default:
+      DCHECK(false) << "Invalid values byte width";
+      break;
+  }
+}
+
+// ----------------------------------------------------------------------
+// Optimized and streamlined filter for primitive types
+
+// Use either BitBlockCounter or BinaryBitBlockCounter to quickly scan filter a
+// word at a time for the DROP selection type.
+class DropNullCounter {
+ public:
+  // validity bitmap may be null
+  DropNullCounter(const uint8_t* validity, const uint8_t* data, int64_t offset,
+                  int64_t length)
+      : data_counter_(data, offset, length),
+        data_and_validity_counter_(data, offset, validity, offset, length),
+        has_validity_(validity != nullptr) {}
+
+  BitBlockCount NextBlock() {
+    if (has_validity_) {
+      // filter is true AND not null
+      return data_and_validity_counter_.NextAndWord();
+    } else {
+      return data_counter_.NextWord();
+    }
+  }
+
+ private:
+  // For when just data is present, but no validity bitmap
+  BitBlockCounter data_counter_;
+
+  // For when both validity bitmap and data are present
+  BinaryBitBlockCounter data_and_validity_counter_;
+  bool has_validity_;
+};
+
+/// \brief The Filter implementation for primitive (fixed-width) types does not
+/// use the logical Arrow type but rather then physical C type. This way we
+/// only generate one take function for each byte width. We use the same
+/// implementation here for boolean and fixed-byte-size inputs with some
+/// template specialization.
+template <typename ArrowType>
+class PrimitiveFilterImpl {
+ public:
+  using T = typename GetCType<ArrowType>::type;
+
+  PrimitiveFilterImpl(const PrimitiveArg& values, const PrimitiveArg& filter,
+                      FilterOptions::NullSelectionBehavior null_selection,
+                      Datum* out_datum)
+      : values_is_valid_(values.is_valid),
+        values_data_(reinterpret_cast<const T*>(values.data)),
+        values_null_count_(values.null_count),
+        values_offset_(values.offset),
+        values_length_(values.length),
+        filter_is_valid_(filter.is_valid),
+        filter_data_(filter.data),
+        filter_null_count_(filter.null_count),
+        filter_offset_(filter.offset),
+        null_selection_(null_selection) {
+    ArrayData* out_arr = out_datum->mutable_array();
+    out_is_valid_ = out_arr->buffers[0]->mutable_data();
+    out_data_ = reinterpret_cast<T*>(out_arr->buffers[1]->mutable_data());
+    out_offset_ = out_arr->offset;
+    out_length_ = out_arr->length;
+    out_position_ = 0;
+  }
+
+  void ExecNonNull() {
+    // The result is all not-null
+    BitUtil::SetBitsTo(out_is_valid_, out_offset_ + out_position_, out_length_, true);

Review comment:
       Good point, fixing




----------------------------------------------------------------
This is an automated message from the 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] wesm commented on pull request #7442: ARROW-9075: [C++] Optimized Filter implementation: faster performance + compilation, smaller code size

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


   Here's benchmark runs on my machine
   
   * BEFORE: https://gist.github.com/wesm/857a3179e7dbc928d3325b1e7f687086
   * AFTER: https://gist.github.com/wesm/ad07cec1613b6327926dfe1d95e7f4f0
   
   IF YOU WANT TO BENCHMARK YOURSELF, PLEASE USE THIS BRANCH https://github.com/wesm/arrow/tree/ARROW-9075-comparison. It contains the RandomArrayGenerator::Boolean change and some other changes to the benchmarks without which the results will be non-comparable


----------------------------------------------------------------
This is an automated message from the 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] wesm commented on pull request #7442: ARROW-9075: [C++] Optimized Filter implementation: faster performance + compilation, smaller code size

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


   True. I think for binary-based types we need to implement bulk-block-appends. It's beyond the scope of this PR -- I will take a brief look to see if there's anything dumb (like messing up the preallocation) that I did that's making things slower


----------------------------------------------------------------
This is an automated message from the 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] fsaintjacques commented on pull request #7442: ARROW-9075: [C++] Optimized Filter implementation: faster performance + compilation, smaller code size

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


   Still, a 10% decrease for string is highly tolerable for a 50-150% increase for all other types.


----------------------------------------------------------------
This is an automated message from the 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