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/04 22:21:26 UTC

[GitHub] [arrow] wesm opened a new pull request #7352: ARROW-9032: [C++] Split up arrow/util/bit_util.h into multiple header files

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


   There is a lot of code in bit_util.h that is seldom used compared to certain common utilities like `BitUtil::BytesForBits`. This moves everything outside of the `BitUtil` namespace to different headers. You can see by the frequency of includes that this makes sense so that compilation units that only need some simple bit utilities are not including a lot of header code that they never use
   
   ```
   $ grep -R bit_util.h ../src/ | wc -l
   68
   $ grep -R bitmap.h ../src/ | wc -l
   7
   $ grep -R bitmap_ops.h ../src/ | wc -l
   15
   $ grep -R bitmap_reader.h ../src/ | wc -l
   9
   $ grep -R bitmap_writer.h ../src/ | wc -l
   6
   ```
   
   This doesn't seem to affect aggregate compilation time very much but at minimum makes the code easier to navigate (in my opinion, at least). 


----------------------------------------------------------------
This is an automated message from the 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 #7352: ARROW-9032: [C++] Split up arrow/util/bit_util.h into multiple header files

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


   As far as I'm concerned this can be merged as soon as the build is green and someone signs off on 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 #7352: ARROW-9032: [C++] Split up arrow/util/bit_util.h into multiple header files

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


   OK. I will get the build passing here and work on some dependent patches until you or someone can have a chance to 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 pull request #7352: ARROW-9032: [C++] Split up arrow/util/bit_util.h into multiple header files

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


   If this PR in principle is not objectionable would it be OK if I merged it once the build is green and then I can rebase all the PRs that it causes conflicts with?


----------------------------------------------------------------
This is an automated message from the 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 #7352: ARROW-9032: [C++] Split up arrow/util/bit_util.h into multiple header files

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


   Will merge this once CI is passing


----------------------------------------------------------------
This is an automated message from the 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 pull request #7352: ARROW-9032: [C++] Split up arrow/util/bit_util.h into multiple header files

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


   @wesm I would appreciate if we could merge https://github.com/apache/arrow/pull/6985 first or if you prefer you take over the rebase 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] github-actions[bot] commented on pull request #7352: ARROW-9032: [C++] Split up arrow/util/bit_util.h into multiple header files

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


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


----------------------------------------------------------------
This is an automated message from the 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 #7352: ARROW-9032: [C++] Split up arrow/util/bit_util.h into multiple header files

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



##########
File path: cpp/src/arrow/util/bitmap_ops.h
##########
@@ -0,0 +1,228 @@
+// 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 <memory>
+
+#include "arrow/buffer.h"
+#include "arrow/memory_pool.h"
+#include "arrow/result.h"
+#include "arrow/util/bit_util.h"
+#include "arrow/util/bitmap_reader.h"
+#include "arrow/util/visibility.h"
+
+namespace arrow {
+namespace internal {
+
+// A function that visits each bit in a bitmap and calls a visitor function with a
+// boolean representation of that bit. This is intended to be analogous to
+// GenerateBits.
+template <class Visitor>
+void VisitBits(const uint8_t* bitmap, int64_t start_offset, int64_t length,

Review comment:
       it might pay to separate VisitBits method into its own header.




----------------------------------------------------------------
This is an automated message from the 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 #7352: ARROW-9032: [C++] Split up arrow/util/bit_util.h into multiple header files

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



##########
File path: cpp/src/arrow/util/bitmap_ops.h
##########
@@ -0,0 +1,228 @@
+// 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 <memory>
+
+#include "arrow/buffer.h"
+#include "arrow/memory_pool.h"
+#include "arrow/result.h"
+#include "arrow/util/bit_util.h"
+#include "arrow/util/bitmap_reader.h"
+#include "arrow/util/visibility.h"
+
+namespace arrow {
+namespace internal {
+
+// A function that visits each bit in a bitmap and calls a visitor function with a
+// boolean representation of that bit. This is intended to be analogous to
+// GenerateBits.
+template <class Visitor>
+void VisitBits(const uint8_t* bitmap, int64_t start_offset, int64_t length,

Review comment:
       yes, good idea




----------------------------------------------------------------
This is an automated message from the 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 #7352: ARROW-9032: [C++] Split up arrow/util/bit_util.h into multiple header files

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



##########
File path: cpp/src/arrow/util/bitmap_ops.h
##########
@@ -0,0 +1,228 @@
+// 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 <memory>
+
+#include "arrow/buffer.h"
+#include "arrow/memory_pool.h"
+#include "arrow/result.h"
+#include "arrow/util/bit_util.h"
+#include "arrow/util/bitmap_reader.h"
+#include "arrow/util/visibility.h"
+
+namespace arrow {
+namespace internal {
+
+// A function that visits each bit in a bitmap and calls a visitor function with a
+// boolean representation of that bit. This is intended to be analogous to
+// GenerateBits.
+template <class Visitor>
+void VisitBits(const uint8_t* bitmap, int64_t start_offset, int64_t length,
+               Visitor&& visit) {
+  BitmapReader reader(bitmap, start_offset, length);
+  for (int64_t index = 0; index < length; ++index) {
+    visit(reader.IsSet());
+    reader.Next();
+  }
+}
+
+// Like VisitBits(), but unrolls its main loop for better performance.
+template <class Visitor>
+void VisitBitsUnrolled(const uint8_t* bitmap, int64_t start_offset, int64_t length,
+                       Visitor&& visit) {
+  if (length == 0) {
+    return;
+  }
+
+  // Start by visiting any bits preceding the first full byte.
+  int64_t num_bits_before_full_bytes =
+      BitUtil::RoundUpToMultipleOf8(start_offset) - start_offset;
+  // Truncate num_bits_before_full_bytes if it is greater than length.
+  if (num_bits_before_full_bytes > length) {
+    num_bits_before_full_bytes = length;
+  }
+  // Use the non loop-unrolled VisitBits since we don't want to add branches
+  VisitBits<Visitor>(bitmap, start_offset, num_bits_before_full_bytes, visit);
+
+  // Shift the start pointer to the first full byte and compute the
+  // number of full bytes to be read.
+  const uint8_t* first_full_byte = bitmap + BitUtil::CeilDiv(start_offset, 8);
+  const int64_t num_full_bytes = (length - num_bits_before_full_bytes) / 8;
+
+  // Iterate over each full byte of the input bitmap and call the visitor in
+  // a loop-unrolled manner.
+  for (int64_t byte_index = 0; byte_index < num_full_bytes; ++byte_index) {
+    // Get the current bit-packed byte value from the bitmap.
+    const uint8_t byte = *(first_full_byte + byte_index);
+
+    // Execute the visitor function on each bit of the current byte.
+    visit(BitUtil::GetBitFromByte(byte, 0));
+    visit(BitUtil::GetBitFromByte(byte, 1));
+    visit(BitUtil::GetBitFromByte(byte, 2));
+    visit(BitUtil::GetBitFromByte(byte, 3));
+    visit(BitUtil::GetBitFromByte(byte, 4));
+    visit(BitUtil::GetBitFromByte(byte, 5));
+    visit(BitUtil::GetBitFromByte(byte, 6));
+    visit(BitUtil::GetBitFromByte(byte, 7));
+  }
+
+  // Write any leftover bits in the last byte.
+  const int64_t num_bits_after_full_bytes = (length - num_bits_before_full_bytes) % 8;
+  VisitBits<Visitor>(first_full_byte + num_full_bytes, 0, num_bits_after_full_bytes,
+                     visit);
+}
+
+// ----------------------------------------------------------------------
+// Bitmap utilities
+
+/// Copy a bit range of an existing bitmap
+///
+/// \param[in] pool memory pool to allocate memory from
+/// \param[in] bitmap source data
+/// \param[in] offset bit offset into the source data
+/// \param[in] length number of bits to copy
+///
+/// \return Status message
+ARROW_EXPORT
+Result<std::shared_ptr<Buffer>> CopyBitmap(MemoryPool* pool, const uint8_t* bitmap,
+                                           int64_t offset, int64_t length);
+
+/// Copy a bit range of an existing bitmap into an existing bitmap
+///
+/// \param[in] bitmap source data
+/// \param[in] offset bit offset into the source data
+/// \param[in] length number of bits to copy
+/// \param[in] dest_offset bit offset into the destination
+/// \param[in] restore_trailing_bits don't clobber bits outside the destination range
+/// \param[out] dest the destination buffer, must have at least space for
+/// (offset + length) bits
+ARROW_EXPORT
+void CopyBitmap(const uint8_t* bitmap, int64_t offset, int64_t length, uint8_t* dest,
+                int64_t dest_offset, bool restore_trailing_bits = true);
+
+/// Invert a bit range of an existing bitmap into an existing bitmap
+///
+/// \param[in] bitmap source data
+/// \param[in] offset bit offset into the source data
+/// \param[in] length number of bits to copy
+/// \param[in] dest_offset bit offset into the destination
+/// \param[out] dest the destination buffer, must have at least space for
+/// (offset + length) bits
+ARROW_EXPORT
+void InvertBitmap(const uint8_t* bitmap, int64_t offset, int64_t length, uint8_t* dest,
+                  int64_t dest_offset);
+
+/// Invert a bit range of an existing bitmap
+///
+/// \param[in] pool memory pool to allocate memory from
+/// \param[in] bitmap source data
+/// \param[in] offset bit offset into the source data
+/// \param[in] length number of bits to copy
+///
+/// \return Status message
+ARROW_EXPORT
+Result<std::shared_ptr<Buffer>> InvertBitmap(MemoryPool* pool, const uint8_t* bitmap,
+                                             int64_t offset, int64_t length);
+
+/// Compute the number of 1's in the given data array
+///
+/// \param[in] data a packed LSB-ordered bitmap as a byte array
+/// \param[in] bit_offset a bitwise offset into the bitmap
+/// \param[in] length the number of bits to inspect in the bitmap relative to
+/// the offset
+///
+/// \return The number of set (1) bits in the range
+ARROW_EXPORT
+int64_t CountSetBits(const uint8_t* data, int64_t bit_offset, int64_t length);
+
+ARROW_EXPORT
+bool BitmapEquals(const uint8_t* left, int64_t left_offset, const uint8_t* right,
+                  int64_t right_offset, int64_t bit_length);
+
+/// \brief Do a "bitmap and" on right and left buffers starting at
+/// their respective bit-offsets for the given bit-length and put
+/// the results in out_buffer starting at the given bit-offset.
+///
+/// out_buffer will be allocated and initialized to zeros using pool before
+/// the operation.
+ARROW_EXPORT
+Result<std::shared_ptr<Buffer>> BitmapAnd(MemoryPool* pool, const uint8_t* left,
+                                          int64_t left_offset, const uint8_t* right,
+                                          int64_t right_offset, int64_t length,
+                                          int64_t out_offset);
+
+/// \brief Do a "bitmap and" on right and left buffers starting at
+/// their respective bit-offsets for the given bit-length and put
+/// the results in out starting at the given bit-offset.
+ARROW_EXPORT
+void BitmapAnd(const uint8_t* left, int64_t left_offset, const uint8_t* right,
+               int64_t right_offset, int64_t length, int64_t out_offset, uint8_t* out);
+
+/// \brief Do a "bitmap or" for the given bit length on right and left buffers
+/// starting at their respective bit-offsets and put the results in out_buffer
+/// starting at the given bit-offset.
+///
+/// out_buffer will be allocated and initialized to zeros using pool before
+/// the operation.
+ARROW_EXPORT
+Result<std::shared_ptr<Buffer>> BitmapOr(MemoryPool* pool, const uint8_t* left,
+                                         int64_t left_offset, const uint8_t* right,
+                                         int64_t right_offset, int64_t length,
+                                         int64_t out_offset);
+
+/// \brief Do a "bitmap or" for the given bit length on right and left buffers
+/// starting at their respective bit-offsets and put the results in out
+/// starting at the given bit-offset.
+ARROW_EXPORT
+void BitmapOr(const uint8_t* left, int64_t left_offset, const uint8_t* right,
+              int64_t right_offset, int64_t length, int64_t out_offset, uint8_t* out);
+
+/// \brief Do a "bitmap xor" for the given bit-length on right and left
+/// buffers starting at their respective bit-offsets and put the results in
+/// out_buffer starting at the given bit offset.
+///
+/// out_buffer will be allocated and initialized to zeros using pool before
+/// the operation.
+ARROW_EXPORT
+Result<std::shared_ptr<Buffer>> BitmapXor(MemoryPool* pool, const uint8_t* left,
+                                          int64_t left_offset, const uint8_t* right,
+                                          int64_t right_offset, int64_t length,
+                                          int64_t out_offset);
+
+/// \brief Do a "bitmap xor" for the given bit-length on right and left
+/// buffers starting at their respective bit-offsets and put the results in
+/// out starting at the given bit offset.
+ARROW_EXPORT
+void BitmapXor(const uint8_t* left, int64_t left_offset, const uint8_t* right,
+               int64_t right_offset, int64_t length, int64_t out_offset, uint8_t* out);
+
+/// \brief Generate Bitmap with all position to `value` except for one found
+/// at `straggler_pos`.
+ARROW_EXPORT

Review comment:
       I think it might pay to move these to a bitmap_builders.h?




----------------------------------------------------------------
This is an automated message from the 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 #7352: ARROW-9032: [C++] Split up arrow/util/bit_util.h into multiple header files

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


   The ARM failure is a flake, so this is merge-ready


----------------------------------------------------------------
This is an automated message from the 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 #7352: ARROW-9032: [C++] Split up arrow/util/bit_util.h into multiple header files

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


   


----------------------------------------------------------------
This is an automated message from the 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 #7352: ARROW-9032: [C++] Split up arrow/util/bit_util.h into multiple header files

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



##########
File path: cpp/src/arrow/util/bitmap_ops.h
##########
@@ -0,0 +1,228 @@
+// 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 <memory>
+
+#include "arrow/buffer.h"
+#include "arrow/memory_pool.h"
+#include "arrow/result.h"
+#include "arrow/util/bit_util.h"
+#include "arrow/util/bitmap_reader.h"
+#include "arrow/util/visibility.h"
+
+namespace arrow {
+namespace internal {
+
+// A function that visits each bit in a bitmap and calls a visitor function with a
+// boolean representation of that bit. This is intended to be analogous to
+// GenerateBits.
+template <class Visitor>
+void VisitBits(const uint8_t* bitmap, int64_t start_offset, int64_t length,

Review comment:
       it might pay to separate VisitBis into its own header.




----------------------------------------------------------------
This is an automated message from the 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 #7352: ARROW-9032: [C++] Split up arrow/util/bit_util.h into multiple header files

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



##########
File path: cpp/src/arrow/util/bitset.h
##########
@@ -0,0 +1,89 @@
+// 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 <algorithm>

Review comment:
       might pay to name this file bitset_stack?  Not your code but is there a bifference between a bitmap and bitset in this context?




----------------------------------------------------------------
This is an automated message from the 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 #7352: ARROW-9032: [C++] Split up arrow/util/bit_util.h into multiple header files

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



##########
File path: cpp/src/arrow/util/bitmap_ops.h
##########
@@ -0,0 +1,228 @@
+// 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 <memory>
+
+#include "arrow/buffer.h"
+#include "arrow/memory_pool.h"
+#include "arrow/result.h"
+#include "arrow/util/bit_util.h"
+#include "arrow/util/bitmap_reader.h"
+#include "arrow/util/visibility.h"
+
+namespace arrow {
+namespace internal {
+
+// A function that visits each bit in a bitmap and calls a visitor function with a
+// boolean representation of that bit. This is intended to be analogous to
+// GenerateBits.
+template <class Visitor>
+void VisitBits(const uint8_t* bitmap, int64_t start_offset, int64_t length,
+               Visitor&& visit) {
+  BitmapReader reader(bitmap, start_offset, length);
+  for (int64_t index = 0; index < length; ++index) {
+    visit(reader.IsSet());
+    reader.Next();
+  }
+}
+
+// Like VisitBits(), but unrolls its main loop for better performance.
+template <class Visitor>
+void VisitBitsUnrolled(const uint8_t* bitmap, int64_t start_offset, int64_t length,
+                       Visitor&& visit) {
+  if (length == 0) {
+    return;
+  }
+
+  // Start by visiting any bits preceding the first full byte.
+  int64_t num_bits_before_full_bytes =
+      BitUtil::RoundUpToMultipleOf8(start_offset) - start_offset;
+  // Truncate num_bits_before_full_bytes if it is greater than length.
+  if (num_bits_before_full_bytes > length) {
+    num_bits_before_full_bytes = length;
+  }
+  // Use the non loop-unrolled VisitBits since we don't want to add branches
+  VisitBits<Visitor>(bitmap, start_offset, num_bits_before_full_bytes, visit);
+
+  // Shift the start pointer to the first full byte and compute the
+  // number of full bytes to be read.
+  const uint8_t* first_full_byte = bitmap + BitUtil::CeilDiv(start_offset, 8);
+  const int64_t num_full_bytes = (length - num_bits_before_full_bytes) / 8;
+
+  // Iterate over each full byte of the input bitmap and call the visitor in
+  // a loop-unrolled manner.
+  for (int64_t byte_index = 0; byte_index < num_full_bytes; ++byte_index) {
+    // Get the current bit-packed byte value from the bitmap.
+    const uint8_t byte = *(first_full_byte + byte_index);
+
+    // Execute the visitor function on each bit of the current byte.
+    visit(BitUtil::GetBitFromByte(byte, 0));
+    visit(BitUtil::GetBitFromByte(byte, 1));
+    visit(BitUtil::GetBitFromByte(byte, 2));
+    visit(BitUtil::GetBitFromByte(byte, 3));
+    visit(BitUtil::GetBitFromByte(byte, 4));
+    visit(BitUtil::GetBitFromByte(byte, 5));
+    visit(BitUtil::GetBitFromByte(byte, 6));
+    visit(BitUtil::GetBitFromByte(byte, 7));
+  }
+
+  // Write any leftover bits in the last byte.
+  const int64_t num_bits_after_full_bytes = (length - num_bits_before_full_bytes) % 8;
+  VisitBits<Visitor>(first_full_byte + num_full_bytes, 0, num_bits_after_full_bytes,
+                     visit);
+}
+
+// ----------------------------------------------------------------------
+// Bitmap utilities
+
+/// Copy a bit range of an existing bitmap
+///
+/// \param[in] pool memory pool to allocate memory from
+/// \param[in] bitmap source data
+/// \param[in] offset bit offset into the source data
+/// \param[in] length number of bits to copy
+///
+/// \return Status message
+ARROW_EXPORT
+Result<std::shared_ptr<Buffer>> CopyBitmap(MemoryPool* pool, const uint8_t* bitmap,
+                                           int64_t offset, int64_t length);
+
+/// Copy a bit range of an existing bitmap into an existing bitmap
+///
+/// \param[in] bitmap source data
+/// \param[in] offset bit offset into the source data
+/// \param[in] length number of bits to copy
+/// \param[in] dest_offset bit offset into the destination
+/// \param[in] restore_trailing_bits don't clobber bits outside the destination range
+/// \param[out] dest the destination buffer, must have at least space for
+/// (offset + length) bits
+ARROW_EXPORT
+void CopyBitmap(const uint8_t* bitmap, int64_t offset, int64_t length, uint8_t* dest,
+                int64_t dest_offset, bool restore_trailing_bits = true);
+
+/// Invert a bit range of an existing bitmap into an existing bitmap
+///
+/// \param[in] bitmap source data
+/// \param[in] offset bit offset into the source data
+/// \param[in] length number of bits to copy
+/// \param[in] dest_offset bit offset into the destination
+/// \param[out] dest the destination buffer, must have at least space for
+/// (offset + length) bits
+ARROW_EXPORT
+void InvertBitmap(const uint8_t* bitmap, int64_t offset, int64_t length, uint8_t* dest,
+                  int64_t dest_offset);
+
+/// Invert a bit range of an existing bitmap
+///
+/// \param[in] pool memory pool to allocate memory from
+/// \param[in] bitmap source data
+/// \param[in] offset bit offset into the source data
+/// \param[in] length number of bits to copy
+///
+/// \return Status message
+ARROW_EXPORT
+Result<std::shared_ptr<Buffer>> InvertBitmap(MemoryPool* pool, const uint8_t* bitmap,
+                                             int64_t offset, int64_t length);
+
+/// Compute the number of 1's in the given data array
+///
+/// \param[in] data a packed LSB-ordered bitmap as a byte array
+/// \param[in] bit_offset a bitwise offset into the bitmap
+/// \param[in] length the number of bits to inspect in the bitmap relative to
+/// the offset
+///
+/// \return The number of set (1) bits in the range
+ARROW_EXPORT
+int64_t CountSetBits(const uint8_t* data, int64_t bit_offset, int64_t length);
+
+ARROW_EXPORT
+bool BitmapEquals(const uint8_t* left, int64_t left_offset, const uint8_t* right,
+                  int64_t right_offset, int64_t bit_length);
+
+/// \brief Do a "bitmap and" on right and left buffers starting at
+/// their respective bit-offsets for the given bit-length and put
+/// the results in out_buffer starting at the given bit-offset.
+///
+/// out_buffer will be allocated and initialized to zeros using pool before
+/// the operation.
+ARROW_EXPORT
+Result<std::shared_ptr<Buffer>> BitmapAnd(MemoryPool* pool, const uint8_t* left,
+                                          int64_t left_offset, const uint8_t* right,
+                                          int64_t right_offset, int64_t length,
+                                          int64_t out_offset);
+
+/// \brief Do a "bitmap and" on right and left buffers starting at
+/// their respective bit-offsets for the given bit-length and put
+/// the results in out starting at the given bit-offset.
+ARROW_EXPORT
+void BitmapAnd(const uint8_t* left, int64_t left_offset, const uint8_t* right,
+               int64_t right_offset, int64_t length, int64_t out_offset, uint8_t* out);
+
+/// \brief Do a "bitmap or" for the given bit length on right and left buffers
+/// starting at their respective bit-offsets and put the results in out_buffer
+/// starting at the given bit-offset.
+///
+/// out_buffer will be allocated and initialized to zeros using pool before
+/// the operation.
+ARROW_EXPORT
+Result<std::shared_ptr<Buffer>> BitmapOr(MemoryPool* pool, const uint8_t* left,
+                                         int64_t left_offset, const uint8_t* right,
+                                         int64_t right_offset, int64_t length,
+                                         int64_t out_offset);
+
+/// \brief Do a "bitmap or" for the given bit length on right and left buffers
+/// starting at their respective bit-offsets and put the results in out
+/// starting at the given bit-offset.
+ARROW_EXPORT
+void BitmapOr(const uint8_t* left, int64_t left_offset, const uint8_t* right,
+              int64_t right_offset, int64_t length, int64_t out_offset, uint8_t* out);
+
+/// \brief Do a "bitmap xor" for the given bit-length on right and left
+/// buffers starting at their respective bit-offsets and put the results in
+/// out_buffer starting at the given bit offset.
+///
+/// out_buffer will be allocated and initialized to zeros using pool before
+/// the operation.
+ARROW_EXPORT
+Result<std::shared_ptr<Buffer>> BitmapXor(MemoryPool* pool, const uint8_t* left,
+                                          int64_t left_offset, const uint8_t* right,
+                                          int64_t right_offset, int64_t length,
+                                          int64_t out_offset);
+
+/// \brief Do a "bitmap xor" for the given bit-length on right and left
+/// buffers starting at their respective bit-offsets and put the results in
+/// out starting at the given bit offset.
+ARROW_EXPORT
+void BitmapXor(const uint8_t* left, int64_t left_offset, const uint8_t* right,
+               int64_t right_offset, int64_t length, int64_t out_offset, uint8_t* out);
+
+/// \brief Generate Bitmap with all position to `value` except for one found
+/// at `straggler_pos`.
+ARROW_EXPORT

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] emkornfield commented on pull request #7352: ARROW-9032: [C++] Split up arrow/util/bit_util.h into multiple header files

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


   Haven't had a chance to review it but I think it is a good thing to do, bit_util.h was definitely getting too crowded.


----------------------------------------------------------------
This is an automated message from the 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 #7352: ARROW-9032: [C++] Split up arrow/util/bit_util.h into multiple header files

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


   Appveyor looks OK https://ci.appveyor.com/project/wesm/arrow/builds/33339226. Merging


----------------------------------------------------------------
This is an automated message from the 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 #7352: ARROW-9032: [C++] Split up arrow/util/bit_util.h into multiple header files

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


   @emkornfield I can definitely rebase #6985 for you


----------------------------------------------------------------
This is an automated message from the 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 #7352: ARROW-9032: [C++] Split up arrow/util/bit_util.h into multiple header files

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


   Rebased


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

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