You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@arrow.apache.org by we...@apache.org on 2018/08/08 16:34:20 UTC

[arrow] branch master updated: ARROW-2950: [C++] Clean up util/bit-util.h

This is an automated email from the ASF dual-hosted git repository.

wesm pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/arrow.git


The following commit(s) were added to refs/heads/master by this push:
     new 479c011  ARROW-2950: [C++] Clean up util/bit-util.h
479c011 is described below

commit 479c011a6ac7a8f1e6d77ecf651a4b2be9e5eec0
Author: Antoine Pitrou <an...@python.org>
AuthorDate: Wed Aug 8 12:34:16 2018 -0400

    ARROW-2950: [C++] Clean up util/bit-util.h
    
    Remove unused, little-used, ill-named or duplicate APIs.
    Reconcile some implementations (e.g. CountLeadingZeros / Log2).
    
    Requires https://github.com/apache/parquet-cpp/pull/483
    
    Author: Antoine Pitrou <an...@python.org>
    
    Closes #2355 from pitrou/ARROW-2950-cleanup-bit-util and squashes the following commits:
    
    25533313 <Antoine Pitrou> Add deprecated GetEmptyBitmap() alias (for parquet-cpp)
    816d0203 <Antoine Pitrou> Rename GetEmptyBitmap to AllocateEmptyBitmap and move it to buffer.h
    945ea24b <Antoine Pitrou> Use anonymous namespace
    e48e217c <Antoine Pitrou> ARROW-2950:  Clean up util/bit-util.h
---
 cpp/src/arrow/array-test.cc             |   2 +-
 cpp/src/arrow/array.h                   |   2 +-
 cpp/src/arrow/buffer.cc                 |  15 ++
 cpp/src/arrow/buffer.h                  |  28 +++
 cpp/src/arrow/ipc/json-internal.cc      |   2 +-
 cpp/src/arrow/test-util.h               |   2 +-
 cpp/src/arrow/util/bit-stream-utils.h   |   9 +-
 cpp/src/arrow/util/bit-util-test.cc     | 141 +++++++--------
 cpp/src/arrow/util/bit-util.cc          |  30 ++--
 cpp/src/arrow/util/bit-util.h           | 297 +++++++++++---------------------
 cpp/src/arrow/util/rle-encoding-test.cc |  11 +-
 cpp/src/arrow/util/rle-encoding.h       |  16 +-
 12 files changed, 251 insertions(+), 304 deletions(-)

diff --git a/cpp/src/arrow/array-test.cc b/cpp/src/arrow/array-test.cc
index 8b78762..702b819 100644
--- a/cpp/src/arrow/array-test.cc
+++ b/cpp/src/arrow/array-test.cc
@@ -2893,7 +2893,7 @@ TEST_F(TestListArray, TestFromArrays) {
   test::AssertArraysEqual(expected3, *list3);
 
   // Check that the last offset bit is zero
-  ASSERT_TRUE(BitUtil::BitNotSet(list3->null_bitmap()->data(), length + 1));
+  ASSERT_FALSE(BitUtil::GetBit(list3->null_bitmap()->data(), length + 1));
 
   ListArray expected4(list_type, length, offsets2->data()->buffers[1], values,
                       offsets4->data()->buffers[0], 1);
diff --git a/cpp/src/arrow/array.h b/cpp/src/arrow/array.h
index 7d26793..a2d1d5e 100644
--- a/cpp/src/arrow/array.h
+++ b/cpp/src/arrow/array.h
@@ -204,7 +204,7 @@ class ARROW_EXPORT Array {
   /// \brief Return true if value at index is null. Does not boundscheck
   bool IsNull(int64_t i) const {
     return null_bitmap_data_ != NULLPTR &&
-           BitUtil::BitNotSet(null_bitmap_data_, i + data_->offset);
+           !BitUtil::GetBit(null_bitmap_data_, i + data_->offset);
   }
 
   /// \brief Return true if value at index is valid (not null). Does not
diff --git a/cpp/src/arrow/buffer.cc b/cpp/src/arrow/buffer.cc
index 1303534..91b0002 100644
--- a/cpp/src/arrow/buffer.cc
+++ b/cpp/src/arrow/buffer.cc
@@ -173,4 +173,19 @@ Status AllocateResizableBuffer(const int64_t size,
   return AllocateResizableBuffer(default_memory_pool(), size, out);
 }
 
+Status AllocateEmptyBitmap(MemoryPool* pool, int64_t length,
+                           std::shared_ptr<Buffer>* out) {
+  RETURN_NOT_OK(AllocateBuffer(pool, BitUtil::BytesForBits(length), out));
+  memset((*out)->mutable_data(), 0, static_cast<size_t>((*out)->size()));
+  return Status::OK();
+}
+
+Status AllocateEmptyBitmap(int64_t length, std::shared_ptr<Buffer>* out) {
+  return AllocateEmptyBitmap(default_memory_pool(), length, out);
+}
+
+Status GetEmptyBitmap(MemoryPool* pool, int64_t length, std::shared_ptr<Buffer>* out) {
+  return AllocateEmptyBitmap(pool, length, out);
+}
+
 }  // namespace arrow
diff --git a/cpp/src/arrow/buffer.h b/cpp/src/arrow/buffer.h
index ea4637c..486f046 100644
--- a/cpp/src/arrow/buffer.h
+++ b/cpp/src/arrow/buffer.h
@@ -260,6 +260,34 @@ Status AllocateResizableBuffer(MemoryPool* pool, const int64_t size,
 ARROW_EXPORT
 Status AllocateResizableBuffer(const int64_t size, std::shared_ptr<ResizableBuffer>* out);
 
+/// \brief Allocate a zero-initialized bitmap buffer from a memory pool
+///
+/// \param[in] pool memory pool to allocate memory from
+/// \param[in] length size in bits of bitmap to allocate
+/// \param[out] out the resulting buffer
+///
+/// \return Status message
+ARROW_EXPORT
+Status AllocateEmptyBitmap(MemoryPool* pool, int64_t length,
+                           std::shared_ptr<Buffer>* out);
+
+/// \brief Allocate a zero-initialized bitmap buffer from the default memory pool
+///
+/// \param[in] length size in bits of bitmap to allocate
+/// \param[out] out the resulting buffer
+///
+/// \return Status message
+ARROW_EXPORT
+Status AllocateEmptyBitmap(int64_t length, std::shared_ptr<Buffer>* out);
+
+/// \deprecated Use AllocateEmptyBitmap instead.
+ARROW_DEPRECATED("Use AllocateEmptyBitmap instead")
+ARROW_EXPORT
+Status GetEmptyBitmap(MemoryPool* pool, int64_t length, std::shared_ptr<Buffer>* out);
+
+// ----------------------------------------------------------------------
+// Buffer builder classes
+
 /// \class BufferBuilder
 /// \brief A class for incrementally building a contiguous chunk of in-memory data
 class ARROW_EXPORT BufferBuilder {
diff --git a/cpp/src/arrow/ipc/json-internal.cc b/cpp/src/arrow/ipc/json-internal.cc
index df1dcbc..d58bc96 100644
--- a/cpp/src/arrow/ipc/json-internal.cc
+++ b/cpp/src/arrow/ipc/json-internal.cc
@@ -961,7 +961,7 @@ class ArrayReader {
     int length = static_cast<int>(is_valid.size());
 
     std::shared_ptr<Buffer> out_buffer;
-    RETURN_NOT_OK(GetEmptyBitmap(pool_, length, &out_buffer));
+    RETURN_NOT_OK(AllocateEmptyBitmap(pool_, length, &out_buffer));
     uint8_t* bitmap = out_buffer->mutable_data();
 
     *null_count = 0;
diff --git a/cpp/src/arrow/test-util.h b/cpp/src/arrow/test-util.h
index ccd4ad7..fff0ee1 100644
--- a/cpp/src/arrow/test-util.h
+++ b/cpp/src/arrow/test-util.h
@@ -141,7 +141,7 @@ static inline Status GetBitmapFromVector(const std::vector<T>& is_valid,
   size_t length = is_valid.size();
 
   std::shared_ptr<Buffer> buffer;
-  RETURN_NOT_OK(GetEmptyBitmap(default_memory_pool(), length, &buffer));
+  RETURN_NOT_OK(AllocateEmptyBitmap(length, &buffer));
 
   uint8_t* bitmap = buffer->mutable_data();
   for (size_t i = 0; i < static_cast<size_t>(length); ++i) {
diff --git a/cpp/src/arrow/util/bit-stream-utils.h b/cpp/src/arrow/util/bit-stream-utils.h
index 8c1f067..c6b345a 100644
--- a/cpp/src/arrow/util/bit-stream-utils.h
+++ b/cpp/src/arrow/util/bit-stream-utils.h
@@ -51,7 +51,7 @@ class BitWriter {
   /// The number of current bytes written, including the current byte (i.e. may include a
   /// fraction of a byte). Includes buffered values.
   int bytes_written() const {
-    return byte_offset_ + static_cast<int>(BitUtil::Ceil(bit_offset_, 8));
+    return byte_offset_ + static_cast<int>(BitUtil::BytesForBits(bit_offset_));
   }
   uint8_t* buffer() const { return buffer_; }
   int buffer_len() const { return max_bytes_; }
@@ -148,7 +148,8 @@ class BitReader {
   /// Returns the number of bytes left in the stream, not including the current
   /// byte (i.e., there may be an additional fraction of a byte).
   int bytes_left() {
-    return max_bytes_ - (byte_offset_ + static_cast<int>(BitUtil::Ceil(bit_offset_, 8)));
+    return max_bytes_ -
+           (byte_offset_ + static_cast<int>(BitUtil::BytesForBits(bit_offset_)));
   }
 
   /// Maximum byte length of a vlq encoded int
@@ -190,7 +191,7 @@ inline bool BitWriter::PutValue(uint64_t v, int num_bits) {
 }
 
 inline void BitWriter::Flush(bool align) {
-  int num_bytes = static_cast<int>(BitUtil::Ceil(bit_offset_, 8));
+  int num_bytes = static_cast<int>(BitUtil::BytesForBits(bit_offset_));
   DCHECK_LE(byte_offset_ + num_bytes, max_bytes_);
   memcpy(buffer_ + byte_offset_, &buffered_values_, num_bytes);
 
@@ -355,7 +356,7 @@ inline int BitReader::GetBatch(int num_bits, T* v, int batch_size) {
 template <typename T>
 inline bool BitReader::GetAligned(int num_bytes, T* v) {
   DCHECK_LE(num_bytes, static_cast<int>(sizeof(T)));
-  int bytes_read = static_cast<int>(BitUtil::Ceil(bit_offset_, 8));
+  int bytes_read = static_cast<int>(BitUtil::BytesForBits(bit_offset_));
   if (ARROW_PREDICT_FALSE(byte_offset_ + bytes_read + num_bytes > max_bytes_))
     return false;
 
diff --git a/cpp/src/arrow/util/bit-util-test.cc b/cpp/src/arrow/util/bit-util-test.cc
index e13902b..87202fc 100644
--- a/cpp/src/arrow/util/bit-util-test.cc
+++ b/cpp/src/arrow/util/bit-util-test.cc
@@ -38,12 +38,6 @@
 
 namespace arrow {
 
-static void EnsureCpuInfoInitialized() {
-  if (!CpuInfo::initialized()) {
-    CpuInfo::Init();
-  }
-}
-
 template <class BitmapWriter>
 void WriteVectorToWriter(BitmapWriter& writer, const std::vector<int> values) {
   for (const auto& value : values) {
@@ -61,7 +55,7 @@ void BitmapFromVector(const std::vector<int>& values, int64_t bit_offset,
                       std::shared_ptr<Buffer>* out_buffer, int64_t* out_length) {
   const int64_t length = values.size();
   *out_length = length;
-  ASSERT_OK(GetEmptyBitmap(default_memory_pool(), length + bit_offset, out_buffer));
+  ASSERT_OK(AllocateEmptyBitmap(length + bit_offset, out_buffer));
   auto writer = internal::BitmapWriter((*out_buffer)->mutable_data(), bit_offset, length);
   WriteVectorToWriter(writer, values);
 }
@@ -456,18 +450,18 @@ TEST(BitUtilTests, TestCopyBitmap) {
   }
 }
 
-TEST(BitUtil, Ceil) {
-  EXPECT_EQ(BitUtil::Ceil(0, 1), 0);
-  EXPECT_EQ(BitUtil::Ceil(1, 1), 1);
-  EXPECT_EQ(BitUtil::Ceil(1, 2), 1);
-  EXPECT_EQ(BitUtil::Ceil(1, 8), 1);
-  EXPECT_EQ(BitUtil::Ceil(7, 8), 1);
-  EXPECT_EQ(BitUtil::Ceil(8, 8), 1);
-  EXPECT_EQ(BitUtil::Ceil(9, 8), 2);
-  EXPECT_EQ(BitUtil::Ceil(9, 9), 1);
-  EXPECT_EQ(BitUtil::Ceil(10000000000, 10), 1000000000);
-  EXPECT_EQ(BitUtil::Ceil(10, 10000000000), 1);
-  EXPECT_EQ(BitUtil::Ceil(100000000000, 10000000000), 10);
+TEST(BitUtil, CeilDiv) {
+  EXPECT_EQ(BitUtil::CeilDiv(0, 1), 0);
+  EXPECT_EQ(BitUtil::CeilDiv(1, 1), 1);
+  EXPECT_EQ(BitUtil::CeilDiv(1, 2), 1);
+  EXPECT_EQ(BitUtil::CeilDiv(1, 8), 1);
+  EXPECT_EQ(BitUtil::CeilDiv(7, 8), 1);
+  EXPECT_EQ(BitUtil::CeilDiv(8, 8), 1);
+  EXPECT_EQ(BitUtil::CeilDiv(9, 8), 2);
+  EXPECT_EQ(BitUtil::CeilDiv(9, 9), 1);
+  EXPECT_EQ(BitUtil::CeilDiv(10000000000, 10), 1000000000);
+  EXPECT_EQ(BitUtil::CeilDiv(10, 10000000000), 1);
+  EXPECT_EQ(BitUtil::CeilDiv(100000000000, 10000000000), 10);
 }
 
 TEST(BitUtil, RoundUp) {
@@ -482,31 +476,6 @@ TEST(BitUtil, RoundUp) {
   EXPECT_EQ(BitUtil::RoundUp(100000000000, 10000000000), 100000000000);
 }
 
-TEST(BitUtil, RoundDown) {
-  EXPECT_EQ(BitUtil::RoundDown(0, 1), 0);
-  EXPECT_EQ(BitUtil::RoundDown(1, 1), 1);
-  EXPECT_EQ(BitUtil::RoundDown(1, 2), 0);
-  EXPECT_EQ(BitUtil::RoundDown(6, 2), 6);
-  EXPECT_EQ(BitUtil::RoundDown(7, 3), 6);
-  EXPECT_EQ(BitUtil::RoundDown(9, 9), 9);
-  EXPECT_EQ(BitUtil::RoundDown(10000000001, 10), 10000000000);
-  EXPECT_EQ(BitUtil::RoundDown(10, 10000000000), 0);
-  EXPECT_EQ(BitUtil::RoundDown(100000000000, 10000000000), 100000000000);
-}
-
-TEST(BitUtil, Popcount) {
-  EnsureCpuInfoInitialized();
-
-  EXPECT_EQ(BitUtil::Popcount(BOOST_BINARY(0 1 0 1 0 1 0 1)), 4);
-  EXPECT_EQ(BitUtil::PopcountNoHw(BOOST_BINARY(0 1 0 1 0 1 0 1)), 4);
-  EXPECT_EQ(BitUtil::Popcount(BOOST_BINARY(1 1 1 1 0 1 0 1)), 6);
-  EXPECT_EQ(BitUtil::PopcountNoHw(BOOST_BINARY(1 1 1 1 0 1 0 1)), 6);
-  EXPECT_EQ(BitUtil::Popcount(BOOST_BINARY(1 1 1 1 1 1 1 1)), 8);
-  EXPECT_EQ(BitUtil::PopcountNoHw(BOOST_BINARY(1 1 1 1 1 1 1 1)), 8);
-  EXPECT_EQ(BitUtil::Popcount(0), 0);
-  EXPECT_EQ(BitUtil::PopcountNoHw(0), 0);
-}
-
 TEST(BitUtil, TrailingBits) {
   EXPECT_EQ(BitUtil::TrailingBits(BOOST_BINARY(1 1 1 1 1 1 1 1), 0), 0);
   EXPECT_EQ(BitUtil::TrailingBits(BOOST_BINARY(1 1 1 1 1 1 1 1), 1), 1);
@@ -549,47 +518,71 @@ TEST(BitUtil, Log2) {
   EXPECT_EQ(BitUtil::Log2(3), 2);
   EXPECT_EQ(BitUtil::Log2(4), 2);
   EXPECT_EQ(BitUtil::Log2(5), 3);
+  EXPECT_EQ(BitUtil::Log2(8), 3);
+  EXPECT_EQ(BitUtil::Log2(9), 4);
   EXPECT_EQ(BitUtil::Log2(INT_MAX), 31);
   EXPECT_EQ(BitUtil::Log2(UINT_MAX), 32);
   EXPECT_EQ(BitUtil::Log2(ULLONG_MAX), 64);
 }
 
+TEST(BitUtil, NumRequiredBits) {
+  EXPECT_EQ(BitUtil::NumRequiredBits(0), 0);
+  EXPECT_EQ(BitUtil::NumRequiredBits(1), 1);
+  EXPECT_EQ(BitUtil::NumRequiredBits(2), 2);
+  EXPECT_EQ(BitUtil::NumRequiredBits(3), 2);
+  EXPECT_EQ(BitUtil::NumRequiredBits(4), 3);
+  EXPECT_EQ(BitUtil::NumRequiredBits(5), 3);
+  EXPECT_EQ(BitUtil::NumRequiredBits(7), 3);
+  EXPECT_EQ(BitUtil::NumRequiredBits(8), 4);
+  EXPECT_EQ(BitUtil::NumRequiredBits(9), 4);
+  EXPECT_EQ(BitUtil::NumRequiredBits(UINT_MAX - 1), 32);
+  EXPECT_EQ(BitUtil::NumRequiredBits(UINT_MAX), 32);
+  EXPECT_EQ(BitUtil::NumRequiredBits(static_cast<uint64_t>(UINT_MAX) + 1), 33);
+  EXPECT_EQ(BitUtil::NumRequiredBits(ULLONG_MAX / 2), 63);
+  EXPECT_EQ(BitUtil::NumRequiredBits(ULLONG_MAX / 2 + 1), 64);
+  EXPECT_EQ(BitUtil::NumRequiredBits(ULLONG_MAX - 1), 64);
+  EXPECT_EQ(BitUtil::NumRequiredBits(ULLONG_MAX), 64);
+}
+
+#define U32(x) static_cast<uint32_t>(x)
+#define U64(x) static_cast<uint64_t>(x)
+
+TEST(BitUtil, CountLeadingZeros) {
+  EXPECT_EQ(BitUtil::CountLeadingZeros(U32(0)), 32);
+  EXPECT_EQ(BitUtil::CountLeadingZeros(U32(1)), 31);
+  EXPECT_EQ(BitUtil::CountLeadingZeros(U32(2)), 30);
+  EXPECT_EQ(BitUtil::CountLeadingZeros(U32(3)), 30);
+  EXPECT_EQ(BitUtil::CountLeadingZeros(U32(4)), 29);
+  EXPECT_EQ(BitUtil::CountLeadingZeros(U32(7)), 29);
+  EXPECT_EQ(BitUtil::CountLeadingZeros(U32(8)), 28);
+  EXPECT_EQ(BitUtil::CountLeadingZeros(U32(UINT_MAX / 2)), 1);
+  EXPECT_EQ(BitUtil::CountLeadingZeros(U32(UINT_MAX / 2 + 1)), 0);
+  EXPECT_EQ(BitUtil::CountLeadingZeros(U32(UINT_MAX)), 0);
+
+  EXPECT_EQ(BitUtil::CountLeadingZeros(U64(0)), 64);
+  EXPECT_EQ(BitUtil::CountLeadingZeros(U64(1)), 63);
+  EXPECT_EQ(BitUtil::CountLeadingZeros(U64(2)), 62);
+  EXPECT_EQ(BitUtil::CountLeadingZeros(U64(3)), 62);
+  EXPECT_EQ(BitUtil::CountLeadingZeros(U64(4)), 61);
+  EXPECT_EQ(BitUtil::CountLeadingZeros(U64(7)), 61);
+  EXPECT_EQ(BitUtil::CountLeadingZeros(U64(8)), 60);
+  EXPECT_EQ(BitUtil::CountLeadingZeros(U64(UINT_MAX)), 32);
+  EXPECT_EQ(BitUtil::CountLeadingZeros(U64(UINT_MAX) + 1), 31);
+  EXPECT_EQ(BitUtil::CountLeadingZeros(U64(ULLONG_MAX / 2)), 1);
+  EXPECT_EQ(BitUtil::CountLeadingZeros(U64(ULLONG_MAX / 2 + 1)), 0);
+  EXPECT_EQ(BitUtil::CountLeadingZeros(U64(ULLONG_MAX)), 0);
+}
+
+#undef U32
+#undef U64
+
 TEST(BitUtil, RoundUpToPowerOf2) {
   EXPECT_EQ(BitUtil::RoundUpToPowerOf2(7, 8), 8);
   EXPECT_EQ(BitUtil::RoundUpToPowerOf2(8, 8), 8);
   EXPECT_EQ(BitUtil::RoundUpToPowerOf2(9, 8), 16);
 }
 
-TEST(BitUtil, RoundDownToPowerOf2) {
-  EXPECT_EQ(BitUtil::RoundDownToPowerOf2(7, 8), 0);
-  EXPECT_EQ(BitUtil::RoundDownToPowerOf2(8, 8), 8);
-  EXPECT_EQ(BitUtil::RoundDownToPowerOf2(9, 8), 8);
-}
-
-TEST(BitUtil, RoundUpDown) {
-  EXPECT_EQ(BitUtil::RoundUpNumBytes(7), 1);
-  EXPECT_EQ(BitUtil::RoundUpNumBytes(8), 1);
-  EXPECT_EQ(BitUtil::RoundUpNumBytes(9), 2);
-  EXPECT_EQ(BitUtil::RoundDownNumBytes(7), 0);
-  EXPECT_EQ(BitUtil::RoundDownNumBytes(8), 1);
-  EXPECT_EQ(BitUtil::RoundDownNumBytes(9), 1);
-
-  EXPECT_EQ(BitUtil::RoundUpNumi32(31), 1);
-  EXPECT_EQ(BitUtil::RoundUpNumi32(32), 1);
-  EXPECT_EQ(BitUtil::RoundUpNumi32(33), 2);
-  EXPECT_EQ(BitUtil::RoundDownNumi32(31), 0);
-  EXPECT_EQ(BitUtil::RoundDownNumi32(32), 1);
-  EXPECT_EQ(BitUtil::RoundDownNumi32(33), 1);
-
-  EXPECT_EQ(BitUtil::RoundUpNumi64(63), 1);
-  EXPECT_EQ(BitUtil::RoundUpNumi64(64), 1);
-  EXPECT_EQ(BitUtil::RoundUpNumi64(65), 2);
-  EXPECT_EQ(BitUtil::RoundDownNumi64(63), 0);
-  EXPECT_EQ(BitUtil::RoundDownNumi64(64), 1);
-  EXPECT_EQ(BitUtil::RoundDownNumi64(65), 1);
-}
-
-void TestZigZag(int32_t v) {
+static void TestZigZag(int32_t v) {
   uint8_t buffer[BitReader::MAX_VLQ_BYTE_LEN];
   BitWriter writer(buffer, sizeof(buffer));
   BitReader reader(buffer, sizeof(buffer));
diff --git a/cpp/src/arrow/util/bit-util.cc b/cpp/src/arrow/util/bit-util.cc
index e8f47bb..0d2929e 100644
--- a/cpp/src/arrow/util/bit-util.cc
+++ b/cpp/src/arrow/util/bit-util.cc
@@ -35,7 +35,11 @@
 
 namespace arrow {
 
-void BitUtil::FillBitsFromBytes(const std::vector<uint8_t>& bytes, uint8_t* bits) {
+namespace BitUtil {
+
+namespace {
+
+void FillBitsFromBytes(const std::vector<uint8_t>& bytes, uint8_t* bits) {
   for (size_t i = 0; i < bytes.size(); ++i) {
     if (bytes[i] > 0) {
       SetBit(bits, i);
@@ -43,9 +47,11 @@ void BitUtil::FillBitsFromBytes(const std::vector<uint8_t>& bytes, uint8_t* bits
   }
 }
 
-Status BitUtil::BytesToBits(const std::vector<uint8_t>& bytes, MemoryPool* pool,
-                            std::shared_ptr<Buffer>* out) {
-  int64_t bit_length = BitUtil::BytesForBits(bytes.size());
+}  // namespace
+
+Status BytesToBits(const std::vector<uint8_t>& bytes, MemoryPool* pool,
+                   std::shared_ptr<Buffer>* out) {
+  int64_t bit_length = BytesForBits(bytes.size());
 
   std::shared_ptr<Buffer> buffer;
   RETURN_NOT_OK(AllocateBuffer(pool, bit_length, &buffer));
@@ -57,6 +63,8 @@ Status BitUtil::BytesToBits(const std::vector<uint8_t>& bytes, MemoryPool* pool,
   return Status::OK();
 }
 
+}  // namespace BitUtil
+
 int64_t CountSetBits(const uint8_t* data, int64_t bit_offset, int64_t length) {
   constexpr int64_t pop_len = sizeof(uint64_t) * 8;
 
@@ -98,16 +106,10 @@ int64_t CountSetBits(const uint8_t* data, int64_t bit_offset, int64_t length) {
   return count;
 }
 
-Status GetEmptyBitmap(MemoryPool* pool, int64_t length, std::shared_ptr<Buffer>* result) {
-  RETURN_NOT_OK(AllocateBuffer(pool, BitUtil::BytesForBits(length), result));
-  memset((*result)->mutable_data(), 0, static_cast<size_t>((*result)->size()));
-  return Status::OK();
-}
-
 Status CopyBitmap(MemoryPool* pool, const uint8_t* data, int64_t offset, int64_t length,
                   std::shared_ptr<Buffer>* out) {
   std::shared_ptr<Buffer> buffer;
-  RETURN_NOT_OK(GetEmptyBitmap(pool, length, &buffer));
+  RETURN_NOT_OK(AllocateEmptyBitmap(pool, length, &buffer));
   uint8_t* dest = buffer->mutable_data();
 
   int64_t byte_offset = offset / 8;
@@ -137,7 +139,7 @@ Status CopyBitmap(MemoryPool* pool, const uint8_t* data, int64_t offset, int64_t
 
   for (int64_t i = length; i < length + bits_to_zero; ++i) {
     // Both branches may copy extra bits - unsetting to match specification.
-    BitUtil::SetBitTo(dest, i, false);
+    BitUtil::ClearBit(dest, i);
   }
 
   *out = buffer;
@@ -214,12 +216,12 @@ Status BitmapAnd(MemoryPool* pool, const uint8_t* left, int64_t left_offset,
   if ((out_offset % 8 == left_offset % 8) && (out_offset % 8 == right_offset % 8)) {
     // Fast case: can use bytewise AND
     const int64_t phys_bits = length + out_offset;
-    RETURN_NOT_OK(GetEmptyBitmap(pool, phys_bits, out_buffer));
+    RETURN_NOT_OK(AllocateEmptyBitmap(pool, phys_bits, out_buffer));
     AlignedBitmapAnd(left, left_offset, right, right_offset,
                      (*out_buffer)->mutable_data(), out_offset, length);
   } else {
     // Unaligned
-    RETURN_NOT_OK(GetEmptyBitmap(pool, length + out_offset, out_buffer));
+    RETURN_NOT_OK(AllocateEmptyBitmap(pool, length + out_offset, out_buffer));
     UnalignedBitmapAnd(left, left_offset, right, right_offset,
                        (*out_buffer)->mutable_data(), out_offset, length);
   }
diff --git a/cpp/src/arrow/util/bit-util.h b/cpp/src/arrow/util/bit-util.h
index 6c06d9c..d36ea5d 100644
--- a/cpp/src/arrow/util/bit-util.h
+++ b/cpp/src/arrow/util/bit-util.h
@@ -69,6 +69,10 @@
 
 namespace arrow {
 
+class Buffer;
+class MemoryPool;
+class Status;
+
 namespace detail {
 
 template <typename Integer>
@@ -78,79 +82,24 @@ typename std::make_unsigned<Integer>::type as_unsigned(Integer x) {
 
 }  // namespace detail
 
-class Buffer;
-class MemoryPool;
-class MutableBuffer;
-class Status;
-
 namespace BitUtil {
 
 //
-// Utilities for reading and writing individual bits by their index
-// in a memory area.
+// Bit-related computations on integer values
 //
 
-// Bitmask selecting the k-th bit in a byte
-static constexpr uint8_t kBitmask[] = {1, 2, 4, 8, 16, 32, 64, 128};
-
-// the bitwise complement version of kBitmask
-static constexpr uint8_t kFlippedBitmask[] = {254, 253, 251, 247, 239, 223, 191, 127};
-
-// Bitmask selecting the (k - 1) preceding bits in a byte
-static constexpr uint8_t kPrecedingBitmask[] = {0, 1, 3, 7, 15, 31, 63, 127};
-
-// the bitwise complement version of kPrecedingBitmask
-static constexpr uint8_t kTrailingBitmask[] = {255, 254, 252, 248, 240, 224, 192, 128};
-
-static inline int64_t CeilByte(int64_t size) { return (size + 7) & ~7; }
-
-static inline int64_t BytesForBits(int64_t size) { return CeilByte(size) / 8; }
-
-static inline int64_t Ceil2Bytes(int64_t size) { return (size + 15) & ~15; }
-
-static inline bool GetBit(const uint8_t* bits, int64_t i) {
-  return (bits[i / 8] & kBitmask[i % 8]) != 0;
-}
-
-static inline bool BitNotSet(const uint8_t* bits, int64_t i) {
-  return (bits[i / 8] & kBitmask[i % 8]) == 0;
-}
-
-static inline void ClearBit(uint8_t* bits, int64_t i) {
-  bits[i / 8] &= kFlippedBitmask[i % 8];
-}
-
-static inline void SetBit(uint8_t* bits, int64_t i) { bits[i / 8] |= kBitmask[i % 8]; }
-
-/// Set bit if is_set is true, but cannot clear bit
-static inline void SetArrayBit(uint8_t* bits, int i, bool is_set) {
-  if (is_set) {
-    SetBit(bits, i);
-  }
-}
-
-static inline void SetBitTo(uint8_t* bits, int64_t i, bool bit_is_set) {
-  // https://graphics.stanford.edu/~seander/bithacks.html
-  // "Conditionally set or clear bits without branching"
-  // NOTE: this seems to confuse Valgrind as it reads from potentially
-  // uninitialized memory
-  bits[i / 8] ^= static_cast<uint8_t>(-static_cast<uint8_t>(bit_is_set) ^ bits[i / 8]) &
-                 kBitmask[i % 8];
+// Returns the ceil of value/divisor
+static inline int64_t CeilDiv(int64_t value, int64_t divisor) {
+  return value / divisor + (value % divisor != 0);
 }
 
-// Returns the minimum number of bits needed to represent the value of 'x'
-static inline int NumRequiredBits(uint64_t x) {
-  for (int i = 63; i >= 0; --i) {
-    if (x & (UINT64_C(1) << i)) return i + 1;
-  }
-  return 0;
-}
+static inline int64_t BytesForBits(int64_t bits) { return (bits + 7) >> 3; }
 
-/// Returns the smallest power of two that contains v. Taken from
-/// http://graphics.stanford.edu/~seander/bithacks.html#RoundUpPowerOf2
-/// TODO: Pick a better name, as it is not clear what happens when the input is
-/// already a power of two.
+// Returns the smallest power of two that contains v.  If v is already a
+// power of two, it is returned as is.
 static inline int64_t NextPower2(int64_t n) {
+  // Taken from
+  // http://graphics.stanford.edu/~seander/bithacks.html#RoundUpPowerOf2
   n--;
   n |= n >> 1;
   n |= n >> 2;
@@ -166,105 +115,29 @@ static inline bool IsMultipleOf64(int64_t n) { return (n & 63) == 0; }
 
 static inline bool IsMultipleOf8(int64_t n) { return (n & 7) == 0; }
 
-/// Returns the ceil of value/divisor
-static inline int64_t Ceil(int64_t value, int64_t divisor) {
-  return value / divisor + (value % divisor != 0);
-}
-
-/// Returns 'value' rounded up to the nearest multiple of 'factor'
-inline int64_t RoundUp(int64_t value, int64_t factor) {
+// Returns 'value' rounded up to the nearest multiple of 'factor'
+static inline int64_t RoundUp(int64_t value, int64_t factor) {
   return (value + (factor - 1)) / factor * factor;
 }
 
-/// Returns 'value' rounded down to the nearest multiple of 'factor'
-static inline int64_t RoundDown(int64_t value, int64_t factor) {
-  return (value / factor) * factor;
-}
-
-/// Returns 'value' rounded up to the nearest multiple of 'factor' when factor is
-/// a power of two
-static inline int RoundUpToPowerOf2(int value, int factor) {
+// Returns 'value' rounded up to the nearest multiple of 'factor' when factor
+// is a power of two.
+// The result is undefined on overflow, i.e. if `value > 2**64 - factor`,
+// since we cannot return the correct result which would be 2**64.
+static inline int64_t RoundUpToPowerOf2(int64_t value, int64_t factor) {
   // DCHECK((factor > 0) && ((factor & (factor - 1)) == 0));
   return (value + (factor - 1)) & ~(factor - 1);
 }
 
-static inline int RoundDownToPowerOf2(int value, int factor) {
-  // DCHECK((factor > 0) && ((factor & (factor - 1)) == 0));
-  return value & ~(factor - 1);
-}
-
-/// Specialized round up and down functions for frequently used factors,
-/// like 8 (bits->bytes), 32 (bits->i32), and 64 (bits->i64).
-/// Returns the rounded up number of bytes that fit the number of bits.
-static inline uint32_t RoundUpNumBytes(uint32_t bits) { return (bits + 7) >> 3; }
-
-/// Returns the rounded down number of bytes that fit the number of bits.
-static inline uint32_t RoundDownNumBytes(uint32_t bits) { return bits >> 3; }
-
-/// Returns the rounded up to 32 multiple. Used for conversions of bits to i32.
-static inline uint32_t RoundUpNumi32(uint32_t bits) { return (bits + 31) >> 5; }
-
-/// Returns the rounded up 32 multiple.
-static inline uint32_t RoundDownNumi32(uint32_t bits) { return bits >> 5; }
-
-/// Returns the rounded up to 64 multiple. Used for conversions of bits to i64.
-static inline uint32_t RoundUpNumi64(uint32_t bits) { return (bits + 63) >> 6; }
-
-/// Returns the rounded down to 64 multiple.
-static inline uint32_t RoundDownNumi64(uint32_t bits) { return bits >> 6; }
-
-template <int64_t ROUND_TO>
-static inline int64_t RoundToPowerOfTwo(int64_t num) {
-  // TODO(wesm): is this definitely needed?
-  // DCHECK_GE(num, 0);
-  constexpr int64_t force_carry_addend = ROUND_TO - 1;
-  constexpr int64_t truncate_bitmask = ~(ROUND_TO - 1);
-  constexpr int64_t max_roundable_num = std::numeric_limits<int64_t>::max() - ROUND_TO;
-  if (num <= max_roundable_num) {
-    return (num + force_carry_addend) & truncate_bitmask;
-  }
-  // handle overflow case.  This should result in a malloc error upstream
-  return num;
-}
-
-static inline int64_t RoundUpToMultipleOf64(int64_t num) {
-  return RoundToPowerOfTwo<64>(num);
-}
-
 static inline int64_t RoundUpToMultipleOf8(int64_t num) {
-  return RoundToPowerOfTwo<8>(num);
-}
-
-/// Non hw accelerated pop count.
-/// TODO: we don't use this in any perf sensitive code paths currently.  There
-/// might be a much faster way to implement this.
-static inline int PopcountNoHw(uint64_t x) {
-  int count = 0;
-  for (; x != 0; ++count) x &= x - 1;
-  return count;
+  return RoundUpToPowerOf2(num, 8);
 }
 
-/// Returns the number of set bits in x
-static inline int Popcount(uint64_t x) {
-#ifdef ARROW_USE_SSE
-  if (ARROW_PREDICT_TRUE(CpuInfo::IsSupported(CpuInfo::POPCNT))) {
-    return POPCNT_popcnt_u64(x);
-  } else {
-    return PopcountNoHw(x);
-  }
-#else
-  return PopcountNoHw(x);
-#endif
-}
-
-// Compute correct population count for various-width signed integers
-template <typename T>
-static inline int PopcountSigned(T v) {
-  // Converting to same-width unsigned then extending preserves the bit pattern.
-  return BitUtil::Popcount(detail::as_unsigned(v));
+static inline int64_t RoundUpToMultipleOf64(int64_t num) {
+  return RoundUpToPowerOf2(num, 64);
 }
 
-/// Returns the 'num_bits' least-significant bits of 'v'.
+// Returns the 'num_bits' least-significant bits of 'v'.
 static inline uint64_t TrailingBits(uint64_t v, int num_bits) {
   if (ARROW_PREDICT_FALSE(num_bits == 0)) return 0;
   if (ARROW_PREDICT_FALSE(num_bits >= 64)) return v;
@@ -272,42 +145,63 @@ static inline uint64_t TrailingBits(uint64_t v, int num_bits) {
   return (v << n) >> n;
 }
 
-/// Returns ceil(log2(x)).
-/// TODO: this could be faster if we use __builtin_clz.  Fix this if this ever shows up
-/// in a hot path.
-static inline int Log2(uint64_t x) {
-  // DCHECK_GT(x, 0);
-  if (x == 1) return 0;
-  // Compute result = ceil(log2(x))
-  //                = floor(log2(x - 1)) + 1, for x > 1
-  // by finding the position of the most significant bit (1-indexed) of x - 1
-  // (floor(log2(n)) = MSB(n) (0-indexed))
-  --x;
-  int result = 1;
-  while (x >>= 1) ++result;
-  return result;
+/// \brief Count the number of leading zeros in an unsigned integer.
+static inline int CountLeadingZeros(uint32_t value) {
+#if defined(__clang__) || defined(__GNUC__)
+  if (value == 0) return 32;
+  return static_cast<int>(__builtin_clz(value));
+#elif defined(_MSC_VER)
+  unsigned long index;                                               // NOLINT
+  if (_BitScanReverse(&index, static_cast<unsigned long>(value))) {  // NOLINT
+    return 31 - static_cast<int>(index);
+  } else {
+    return 32;
+  }
+#else
+  int bitpos = 0;
+  while (value != 0) {
+    value >>= 1;
+    ++bitpos;
+  }
+  return 32 - bitpos;
+#endif
 }
 
-/// \brief Count the number of leading zeros in a 32 bit integer.
-static inline int64_t CountLeadingZeros(uint32_t value) {
-// DCHECK_NE(value, 0);
+static inline int CountLeadingZeros(uint64_t value) {
 #if defined(__clang__) || defined(__GNUC__)
-  return static_cast<int64_t>(__builtin_clz(value));
+  if (value == 0) return 64;
+  return static_cast<int>(__builtin_clzl(value));
 #elif defined(_MSC_VER)
-  unsigned long index;                                         // NOLINT
-  _BitScanReverse(&index, static_cast<unsigned long>(value));  // NOLINT
-  return 31LL - static_cast<int64_t>(index);
+  unsigned long index;                     // NOLINT
+  if (_BitScanReverse64(&index, value)) {  // NOLINT
+    return 63 - static_cast<int>(index);
+  } else {
+    return 64;
+  }
 #else
-  int64_t bitpos = 0;
+  int bitpos = 0;
   while (value != 0) {
     value >>= 1;
     ++bitpos;
   }
-  return 32LL - bitpos;
+  return 64 - bitpos;
 #endif
 }
 
-/// Swaps the byte order (i.e. endianess)
+// Returns the minimum number of bits needed to represent an unsigned value
+static inline int NumRequiredBits(uint64_t x) { return 64 - CountLeadingZeros(x); }
+
+// Returns ceil(log2(x)).
+static inline int Log2(uint64_t x) {
+  // DCHECK_GT(x, 0);
+  return NumRequiredBits(x - 1);
+}
+
+//
+// Byte-swap 16-bit, 32-bit and 64-bit values
+//
+
+// Swap the byte order (i.e. endianess)
 static inline int64_t ByteSwap(int64_t value) { return ARROW_BYTE_SWAP64(value); }
 static inline uint64_t ByteSwap(uint64_t value) {
   return static_cast<uint64_t>(ARROW_BYTE_SWAP64(value));
@@ -324,7 +218,7 @@ static inline uint16_t ByteSwap(uint16_t value) {
   return static_cast<uint16_t>(ByteSwap(static_cast<int16_t>(value)));
 }
 
-/// Write the swapped bytes into dst. Src and st cannot overlap.
+// Write the swapped bytes into dst. Src and dst cannot overlap.
 static inline void ByteSwap(void* dst, const void* src, int len) {
   switch (len) {
     case 1:
@@ -350,8 +244,7 @@ static inline void ByteSwap(void* dst, const void* src, int len) {
   }
 }
 
-/// Converts to big endian format (if not already in big endian) from the
-/// machine's native endian format.
+// Convert to little/big endian format from the machine's native endian format.
 #if ARROW_LITTLE_ENDIAN
 template <typename T, typename = EnableIfIsOneOf<T, int64_t, uint64_t, int32_t, uint32_t,
                                                  int16_t, uint16_t>>
@@ -370,9 +263,15 @@ template <typename T, typename = EnableIfIsOneOf<T, int64_t, uint64_t, int32_t,
 static inline T ToBigEndian(T value) {
   return value;
 }
+
+template <typename T, typename = EnableIfIsOneOf<T, int64_t, uint64_t, int32_t, uint32_t,
+                                                 int16_t, uint16_t>>
+static inline T ToLittleEndian(T value) {
+  return ByteSwap(value);
+}
 #endif
 
-/// Converts from big endian format to the machine's native endian format.
+// Convert from big/little endian format to the machine's native endian format.
 #if ARROW_LITTLE_ENDIAN
 template <typename T, typename = EnableIfIsOneOf<T, int64_t, uint64_t, int32_t, uint32_t,
                                                  int16_t, uint16_t>>
@@ -399,16 +298,32 @@ static inline T FromLittleEndian(T value) {
 }
 #endif
 
-// Logical right shift for signed integer types
-// This is needed because the C >> operator does arithmetic right shift
-// Negative shift amounts lead to undefined behavior
-template <typename T>
-static T ShiftRightLogical(T v, int shift) {
-  // Conversion to unsigned ensures most significant bits always filled with 0's
-  return detail::as_unsigned(v) >> shift;
+//
+// Utilities for reading and writing individual bits by their index
+// in a memory area.
+//
+
+// Bitmask selecting the k-th bit in a byte
+static constexpr uint8_t kBitmask[] = {1, 2, 4, 8, 16, 32, 64, 128};
+
+// the bitwise complement version of kBitmask
+static constexpr uint8_t kFlippedBitmask[] = {254, 253, 251, 247, 239, 223, 191, 127};
+
+// Bitmask selecting the (k - 1) preceding bits in a byte
+static constexpr uint8_t kPrecedingBitmask[] = {0, 1, 3, 7, 15, 31, 63, 127};
+
+// the bitwise complement version of kPrecedingBitmask
+static constexpr uint8_t kTrailingBitmask[] = {255, 254, 252, 248, 240, 224, 192, 128};
+
+static inline bool GetBit(const uint8_t* bits, int64_t i) {
+  return (bits[i / 8] & kBitmask[i % 8]) != 0;
+}
+
+static inline void ClearBit(uint8_t* bits, int64_t i) {
+  bits[i / 8] &= kFlippedBitmask[i % 8];
 }
 
-void FillBitsFromBytes(const std::vector<uint8_t>& bytes, uint8_t* bits);
+static inline void SetBit(uint8_t* bits, int64_t i) { bits[i / 8] |= kBitmask[i % 8]; }
 
 /// \brief Convert vector of bytes to bitmap buffer
 ARROW_EXPORT
@@ -430,12 +345,7 @@ class BitmapReader {
     }
   }
 
-#if defined(_MSC_VER)
-  // MSVC is finicky about this cast
   bool IsSet() const { return (current_byte_ & (1 << bit_offset_)) != 0; }
-#else
-  bool IsSet() const { return current_byte_ & (1 << bit_offset_); }
-#endif
 
   bool IsNotSet() const { return (current_byte_ & (1 << bit_offset_)) == 0; }
 
@@ -647,9 +557,6 @@ void GenerateBitsUnrolled(uint8_t* bitmap, int64_t start_offset, int64_t length,
 // ----------------------------------------------------------------------
 // Bitmap utilities
 
-ARROW_EXPORT
-Status GetEmptyBitmap(MemoryPool* pool, int64_t length, std::shared_ptr<Buffer>* result);
-
 /// Copy a bit range of an existing bitmap
 ///
 /// \param[in] pool memory pool to allocate memory from
diff --git a/cpp/src/arrow/util/rle-encoding-test.cc b/cpp/src/arrow/util/rle-encoding-test.cc
index 09bcb1a..a6583bd 100644
--- a/cpp/src/arrow/util/rle-encoding-test.cc
+++ b/cpp/src/arrow/util/rle-encoding-test.cc
@@ -101,7 +101,7 @@ TEST(BitArray, TestBool) {
 
 // Writes 'num_vals' values with width 'bit_width' and reads them back.
 void TestBitArrayValues(int bit_width, int num_vals) {
-  int len = static_cast<int>(BitUtil::Ceil(bit_width * num_vals, 8));
+  int len = static_cast<int>(BitUtil::BytesForBits(bit_width * num_vals));
   EXPECT_GT(len, 0);
   const uint64_t mod = bit_width == 64 ? 1 : 1LL << bit_width;
 
@@ -281,14 +281,15 @@ TEST(Rle, SpecificSequences) {
   }
 
   for (int width = 9; width <= MAX_WIDTH; ++width) {
-    ValidateRle(values, width, NULL, 2 * (1 + static_cast<int>(BitUtil::Ceil(width, 8))));
+    ValidateRle(values, width, NULL,
+                2 * (1 + static_cast<int>(BitUtil::CeilDiv(width, 8))));
   }
 
   // Test 100 0's and 1's alternating
   for (int i = 0; i < 100; ++i) {
     values[i] = i % 2;
   }
-  int num_groups = static_cast<int>(BitUtil::Ceil(100, 8));
+  int num_groups = static_cast<int>(BitUtil::CeilDiv(100, 8));
   expected_buffer[0] = static_cast<uint8_t>((num_groups << 1) | 1);
   for (int i = 1; i <= 100 / 8; ++i) {
     expected_buffer[i] = BOOST_BINARY(1 0 1 0 1 0 1 0);
@@ -299,9 +300,9 @@ TEST(Rle, SpecificSequences) {
   // num_groups and expected_buffer only valid for bit width = 1
   ValidateRle(values, 1, expected_buffer, 1 + num_groups);
   for (int width = 2; width <= MAX_WIDTH; ++width) {
-    int num_values = static_cast<int>(BitUtil::Ceil(100, 8)) * 8;
+    int num_values = static_cast<int>(BitUtil::CeilDiv(100, 8)) * 8;
     ValidateRle(values, width, NULL,
-                1 + static_cast<int>(BitUtil::Ceil(width * num_values, 8)));
+                1 + static_cast<int>(BitUtil::CeilDiv(width * num_values, 8)));
   }
 }
 
diff --git a/cpp/src/arrow/util/rle-encoding.h b/cpp/src/arrow/util/rle-encoding.h
index f343b74..8d9ac1d 100644
--- a/cpp/src/arrow/util/rle-encoding.h
+++ b/cpp/src/arrow/util/rle-encoding.h
@@ -169,10 +169,11 @@ class RleEncoder {
   static int MinBufferSize(int bit_width) {
     /// 1 indicator byte and MAX_VALUES_PER_LITERAL_RUN 'bit_width' values.
     int max_literal_run_size =
-        1 + static_cast<int>(BitUtil::Ceil(MAX_VALUES_PER_LITERAL_RUN * bit_width, 8));
+        1 +
+        static_cast<int>(BitUtil::BytesForBits(MAX_VALUES_PER_LITERAL_RUN * bit_width));
     /// Up to MAX_VLQ_BYTE_LEN indicator and a single 'bit_width' value.
     int max_repeated_run_size =
-        BitReader::MAX_VLQ_BYTE_LEN + static_cast<int>(BitUtil::Ceil(bit_width, 8));
+        BitReader::MAX_VLQ_BYTE_LEN + static_cast<int>(BitUtil::BytesForBits(bit_width));
     return std::max(max_literal_run_size, max_repeated_run_size);
   }
 
@@ -181,17 +182,16 @@ class RleEncoder {
     // For a bit_width > 1, the worst case is the repetition of "literal run of length 8
     // and then a repeated run of length 8".
     // 8 values per smallest run, 8 bits per byte
-    // int bytes_per_run = BitUtil::Ceil(bit_width * 8, 8);
     int bytes_per_run = bit_width;
-    int num_runs = static_cast<int>(BitUtil::Ceil(num_values, 8));
+    int num_runs = static_cast<int>(BitUtil::CeilDiv(num_values, 8));
     int literal_max_size = num_runs + num_runs * bytes_per_run;
 
     // In the very worst case scenario, the data is a concatenation of repeated
     // runs of 8 values. Repeated run has a 1 byte varint followed by the
     // bit-packed repeated value
-    int min_repeated_run_size = 1 + static_cast<int>(BitUtil::Ceil(bit_width, 8));
+    int min_repeated_run_size = 1 + static_cast<int>(BitUtil::BytesForBits(bit_width));
     int repeated_max_size =
-        static_cast<int>(BitUtil::Ceil(num_values, 8)) * min_repeated_run_size;
+        static_cast<int>(BitUtil::CeilDiv(num_values, 8)) * min_repeated_run_size;
 
     return std::max(literal_max_size, repeated_max_size);
   }
@@ -436,7 +436,7 @@ bool RleDecoder::NextCounts() {
   } else {
     repeat_count_ = indicator_value >> 1;
     bool result =
-        bit_reader_.GetAligned<T>(static_cast<int>(BitUtil::Ceil(bit_width_, 8)),
+        bit_reader_.GetAligned<T>(static_cast<int>(BitUtil::CeilDiv(bit_width_, 8)),
                                   reinterpret_cast<T*>(&current_value_));
     DCHECK(result);
   }
@@ -513,7 +513,7 @@ inline void RleEncoder::FlushRepeatedRun() {
   int32_t indicator_value = repeat_count_ << 1 | 0;
   result &= bit_writer_.PutVlqInt(indicator_value);
   result &= bit_writer_.PutAligned(current_value_,
-                                   static_cast<int>(BitUtil::Ceil(bit_width_, 8)));
+                                   static_cast<int>(BitUtil::CeilDiv(bit_width_, 8)));
   DCHECK(result);
   num_buffered_values_ = 0;
   repeat_count_ = 0;