You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kudu.apache.org by al...@apache.org on 2022/02/01 21:57:53 UTC

[kudu] branch master updated (ee5e856 -> f4fa4d2)

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

alexey pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/kudu.git.


    from ee5e856  [java] KUDU-3240 Make connection negotiation timeout configurable for Java client
     new 3a5cc44  [benchmarks] updated RLE benchmark
     new bac76fc  [util] optimized version of BitUtil::Ceil()
     new f4fa4d2  [asan] improve asan success rate, kerberos context init and destroy

The 3 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 src/kudu/benchmarks/rle.cc              | 116 +++++++++++++++++++++-----------
 src/kudu/security/init.cc               |  77 +++++++++++++--------
 src/kudu/security/init.h                |   3 +
 src/kudu/security/kinit_context.h       |  13 ++++
 src/kudu/server/server_base.cc          |   2 +
 src/kudu/util/bit-stream-utils.h        |   4 +-
 src/kudu/util/bit-stream-utils.inline.h |   4 +-
 src/kudu/util/bit-util.h                |  10 +++
 src/kudu/util/rle-encoding.h            |   8 +--
 src/kudu/util/rle-test.cc               |   8 +--
 10 files changed, 167 insertions(+), 78 deletions(-)

[kudu] 01/03: [benchmarks] updated RLE benchmark

Posted by al...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 3a5cc4409fac2e264da5553d7ec8defce9457397
Author: Alexey Serbin <al...@apache.org>
AuthorDate: Mon Jan 31 21:47:38 2022 -0800

    [benchmarks] updated RLE benchmark
    
    This patch updates the RLE benchmark to remove some non-RLE operations
    (such as IO, memory allocation, etc.) out from the StopWatch scope.
    
    The results of the updated RLE benchmark is used to compare current and
    optimized BitUtil::Ceil() implementation: the latter comes in a
    follow-up patch.
    
    Change-Id: I8b9df4d20ffb333910f94bbb52a8dcfd0ed6efe9
    Reviewed-on: http://gerrit.cloudera.org:8080/18186
    Tested-by: Alexey Serbin <as...@cloudera.com>
    Reviewed-by: Attila Bukor <ab...@apache.org>
---
 src/kudu/benchmarks/rle.cc | 83 ++++++++++++++++++++++++----------------------
 1 file changed, 44 insertions(+), 39 deletions(-)

diff --git a/src/kudu/benchmarks/rle.cc b/src/kudu/benchmarks/rle.cc
index 2d3d041..a1b1c0c 100644
--- a/src/kudu/benchmarks/rle.cc
+++ b/src/kudu/benchmarks/rle.cc
@@ -42,47 +42,36 @@ DEFINE_int32(bitstream_num_bytes, 1 * 1024 * 1024,
 namespace kudu {
 
 // Measure writing and reading single-bit streams
-void BooleanBitStream() {
-  faststring buffer(FLAGS_bitstream_num_bytes);
-  BitWriter writer(&buffer);
-
+int BooleanBitStream(faststring* buffer) {
   // Write alternating strings of repeating 0's and 1's
-  for (int i = 0; i < FLAGS_bitstream_num_bytes; ++i) {
-    writer.PutValue(i % 2, 1);
-    writer.PutValue(i % 2, 1);
-    writer.PutValue(i % 2, 1);
-    writer.PutValue(i % 2, 1);
-    writer.PutValue(i % 2, 1);
-    writer.PutValue(i % 2, 1);
-    writer.PutValue(i % 2, 1);
-    writer.PutValue(i % 2, 1);
+  const auto num_bytes = buffer->capacity();
+  BitWriter writer(buffer);
+  for (auto i = 0; i < num_bytes; ++i) {
+    auto val = i % 2;
+    for (auto j = 0; j < 8; ++j) {
+      writer.PutValue(val, 1);
+    }
   }
   writer.Flush();
+  const auto bytes_written = writer.bytes_written();
 
-  LOG(INFO) << "Wrote " << writer.bytes_written() << " bytes";
-
-  BitReader reader(buffer.data(), writer.bytes_written());
-  for (int i = 0; i < FLAGS_bitstream_num_bytes; ++i) {
+  BitReader reader(buffer->data(), bytes_written);
+  for (auto i = 0; i < num_bytes; ++i) {
     bool val;
-    reader.GetValue(1, &val);
-    reader.GetValue(1, &val);
-    reader.GetValue(1, &val);
-    reader.GetValue(1, &val);
-    reader.GetValue(1, &val);
-    reader.GetValue(1, &val);
-    reader.GetValue(1, &val);
-    reader.GetValue(1, &val);
+    for (auto j = 0; j < 8; ++j) {
+      reader.GetValue(1, &val);
+    }
   }
+
+  return bytes_written;
 }
 
 // Measure bulk puts and decoding runs of RLE bools
-void BooleanRLE() {
-  const int num_iters = 3 * 1024;
+int BooleanRLE(faststring* buffer) {
+  constexpr int kNumIters = 3 * 1024;
 
-  faststring buffer(45 * 1024);
-  RleEncoder<bool> encoder(&buffer, 1);
-
-  for (int i = 0; i < num_iters; i++) {
+  RleEncoder<bool> encoder(buffer, 1);
+  for (auto i = 0; i < kNumIters; ++i) {
     encoder.Put(false, 100 * 1024);
     encoder.Put(true, 3);
     encoder.Put(false, 3);
@@ -92,28 +81,44 @@ void BooleanRLE() {
     encoder.Put(false, 4);
   }
 
-  LOG(INFO) << "Wrote " << encoder.len() << " bytes";
+  const auto bytes_written = encoder.len();
 
-  RleDecoder<bool> decoder(buffer.data(), encoder.len(), 1);
+  RleDecoder<bool> decoder(buffer->data(), encoder.len(), 1);
   bool val = false;
-  for (int i = 0; i < num_iters * 7; i++) {
+  for (auto i = 0; i < kNumIters * 7; ++i) {
     ignore_result(decoder.GetNextRun(&val, MathLimits<size_t>::kMax));
   }
+
+  return bytes_written;
 }
 
 } // namespace kudu
 
-int main(int argc, char **argv) {
+int main(int argc, char** argv) {
   FLAGS_logtostderr = 1;
   google::ParseCommandLineFlags(&argc, &argv, true);
   kudu::InitGoogleLoggingSafe(argv[0]);
 
-  LOG_TIMING(INFO, "BooleanBitStream") {
-    kudu::BooleanBitStream();
+  {
+    // kudu::BooleanBitStream() assumes fastring::capacity() returns
+    // the actual capacity of the buffer.
+    CHECK_GT(FLAGS_bitstream_num_bytes, kudu::faststring::kInitialCapacity);
+
+    int bytes_written = 0;
+    kudu::faststring buffer(FLAGS_bitstream_num_bytes);
+    LOG_TIMING(INFO, "BooleanBitStream") {
+      bytes_written = kudu::BooleanBitStream(&buffer);
+    }
+    LOG(INFO) << "Wrote " << bytes_written << " bytes";
   }
 
-  LOG_TIMING(INFO, "BooleanRLE") {
-    kudu::BooleanRLE();
+  {
+    int bytes_written = 0;
+    kudu::faststring buffer(45 * 1024);
+    LOG_TIMING(INFO, "BooleanRLE") {
+      bytes_written = kudu::BooleanRLE(&buffer);
+    }
+    LOG(INFO) << "Wrote " << bytes_written << " bytes";
   }
 
   return 0;

[kudu] 03/03: [asan] improve asan success rate, kerberos context init and destroy

Posted by al...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit f4fa4d22f2dc122c2c130fd08142317869729e28
Author: shenxingwuying <sh...@gmail.com>
AuthorDate: Mon Jan 10 19:00:36 2022 +0800

    [asan] improve asan success rate, kerberos context init and destroy
    
    kerberos's init context is global, KinitContext* g_kinit_ctx,
    it used new operator, but no delete operator.
    It release the memory by os when program stopping.
    Some asan tests may failed, when MiniCluster restart/stop.
    KinitContext should be deleted safely.
    
    Change-Id: I76a639e35fdf951787f14e0603e73e9e19da6691
    Reviewed-on: http://gerrit.cloudera.org:8080/18135
    Tested-by: Kudu Jenkins
    Reviewed-by: Alexey Serbin <as...@cloudera.com>
---
 src/kudu/security/init.cc         | 77 +++++++++++++++++++++++++--------------
 src/kudu/security/init.h          |  3 ++
 src/kudu/security/kinit_context.h | 13 +++++++
 src/kudu/server/server_base.cc    |  2 +
 4 files changed, 68 insertions(+), 27 deletions(-)

diff --git a/src/kudu/security/init.cc b/src/kudu/security/init.cc
index 1a512dc..8d59d51 100644
--- a/src/kudu/security/init.cc
+++ b/src/kudu/security/init.cc
@@ -37,9 +37,11 @@
 #include <glog/logging.h>
 
 #include "kudu/gutil/macros.h"
+#include "kudu/gutil/ref_counted.h"
 #include "kudu/gutil/strings/substitute.h"
 #include "kudu/gutil/strings/util.h"
 #include "kudu/security/kinit_context.h"
+#include "kudu/util/countdown_latch.h"
 #include "kudu/util/flag_tags.h"
 #include "kudu/util/monotime.h"
 #include "kudu/util/net/net_util.h"
@@ -94,7 +96,7 @@ namespace kudu {
 namespace security {
 
 // Global instance of the context used by the kinit/reacquire thread.
-KinitContext* g_kinit_ctx;
+KinitContext* g_kinit_ctx = nullptr;
 
 namespace {
 
@@ -160,34 +162,13 @@ Status Krb5UnparseName(krb5_principal princ, string* name) {
   return Status::OK();
 }
 
-// Periodically calls DoRenewal().
-void RenewThread() {
-  uint32_t failure_retries = 0;
-  while (true) {
-    // This thread is run immediately after the first Kinit, so sleep first.
-    int64_t renew_interval_s = g_kinit_ctx->GetNextRenewInterval(failure_retries);
-    if (failure_retries > 0) {
-      // Log in the abnormal case where something failed.
-      LOG(INFO) << Substitute("Renew thread sleeping after $0 failures for $1s",
-          failure_retries, renew_interval_s);
-    }
-    SleepFor(MonoDelta::FromSeconds(renew_interval_s));
-
-    Status s = g_kinit_ctx->DoRenewal();
-    WARN_NOT_OK(s, "Kerberos reacquire error: ");
-    if (!s.ok()) {
-      ++failure_retries;
-    } else {
-      failure_retries = 0;
-    }
-  }
-}
 } // anonymous namespace
 
-KinitContext::KinitContext() {}
+KinitContext::KinitContext() : stop_latch_(1) {}
 
 KinitContext::~KinitContext() {
   // Free memory associated with these objects.
+  Kdestroy();
   if (principal_ != nullptr) krb5_free_principal(g_krb5_ctx, principal_);
   if (keytab_ != nullptr) krb5_kt_close(g_krb5_ctx, keytab_);
   if (ccache_ != nullptr) krb5_cc_close(g_krb5_ctx, ccache_);
@@ -329,7 +310,19 @@ Status KinitContext::Kinit(const string& keytab_path, const string& principal) {
 
   KRB5_RETURN_NOT_OK_PREPEND(krb5_get_init_creds_opt_alloc(g_krb5_ctx, &opts_),
                              "unable to allocate get_init_creds_opt struct");
-  return KinitInternal();
+  RETURN_NOT_OK(KinitInternal());
+
+  // Start the thread to renew and reacquire Kerberos tickets.
+  RETURN_NOT_OK(Thread::Create("kerberos", "reacquire thread",
+                               [this]() { this->RenewThread(); }, &reacquire_thread_));
+  return Status::OK();
+}
+
+void KinitContext::Kdestroy() {
+  stop_latch_.CountDown();
+  if (reacquire_thread_.get() != nullptr) {
+    reacquire_thread_->Join();
+  }
 }
 
 Status KinitContext::KinitInternal() {
@@ -372,6 +365,30 @@ Status KinitContext::KinitInternal() {
   return Status::OK();
 }
 
+// Periodically calls DoRenewal().
+void KinitContext::RenewThread() {
+  uint32_t failure_retries = 0;
+  int64_t renew_interval_s = GetNextRenewInterval(failure_retries);
+  while (!stop_latch_.WaitFor(MonoDelta::FromSeconds(renew_interval_s))) {
+    Status s = DoRenewal();
+    WARN_NOT_OK(s, "Kerberos reacquire error: ");
+    if (!s.ok()) {
+      ++failure_retries;
+    } else {
+      failure_retries = 0;
+    }
+
+    if (failure_retries > 0) {
+      // Log in the abnormal case where something failed.
+      LOG(INFO) << Substitute("Renew thread sleeping after $0 failures for $1s",
+          failure_retries, renew_interval_s);
+    }
+
+    // This thread is run immediately after the first Kinit, so sleep first.
+    renew_interval_s = GetNextRenewInterval(failure_retries);
+  }
+}
+
 RWMutex* KerberosReinitLock() {
   return g_kerberos_reinit_lock;
 }
@@ -495,8 +512,14 @@ Status InitKerberosForServer(const std::string& raw_principal, const std::string
   RETURN_NOT_OK_PREPEND(g_kinit_ctx->Kinit(
       keytab_file, configured_principal), "unable to kinit");
 
-  // Start the thread to renew and reacquire Kerberos tickets.
-  return Thread::Create("kerberos", "reacquire thread", &RenewThread, nullptr);
+  return Status::OK();
+}
+
+void DestroyKerberosForServer() {
+  if (g_kinit_ctx == nullptr) return;
+
+  delete g_kinit_ctx;
+  g_kinit_ctx = nullptr;
 }
 
 string GetKrb5ConfigFile() {
diff --git a/src/kudu/security/init.h b/src/kudu/security/init.h
index 8b76e94..aa8a8b8 100644
--- a/src/kudu/security/init.h
+++ b/src/kudu/security/init.h
@@ -55,6 +55,9 @@ Status InitKerberosForServer(const std::string& raw_principal,
                              const std::string& krb5ccname = kKrb5CCName,
                              bool disable_krb5_replay_cache = true);
 
+// Destroy Kerberos for a server.
+void DestroyKerberosForServer();
+
 // Returns the process lock 'kerberos_reinit_lock'
 // This lock is taken in write mode while the ticket is being reacquired, and
 // taken in read mode before using the SASL library which might require a ticket.
diff --git a/src/kudu/security/kinit_context.h b/src/kudu/security/kinit_context.h
index a6c2b41..9a9e644 100644
--- a/src/kudu/security/kinit_context.h
+++ b/src/kudu/security/kinit_context.h
@@ -21,7 +21,9 @@
 
 #include <krb5/krb5.h>
 
+#include "kudu/gutil/strings/substitute.h"
 #include "kudu/util/status.h"
+#include "kudu/util/thread.h"
 
 namespace kudu {
 namespace security {
@@ -63,6 +65,12 @@ class KinitContext {
  private:
   Status KinitInternal();
 
+  // Safe stop the renewal thread before destroying KinitContext
+  void Kdestroy();
+
+  // Periodically calls DoRenewal().
+  void RenewThread();
+
   // Helper for DoRenewal() that tries to do a renewal. On success, returns OK and sets
   // *found_in_cache = true. If there is an error doing the renewal itself, returns an
   // error. If the TGT to be renewed was not found in the cache, return OK and set
@@ -79,6 +87,11 @@ class KinitContext {
 
   // This is the time that the current TGT in use expires.
   int32_t ticket_end_timestamp_;
+
+  // To stop reacquire_thread_ when process stopping.
+  CountDownLatch stop_latch_;
+  // A thread to renew and reacquire Kerberos credentials.
+  scoped_refptr<Thread> reacquire_thread_;
 };
 
 } // namespace security
diff --git a/src/kudu/server/server_base.cc b/src/kudu/server/server_base.cc
index ec44729..6123f77 100644
--- a/src/kudu/server/server_base.cc
+++ b/src/kudu/server/server_base.cc
@@ -869,6 +869,8 @@ void ServerBase::ShutdownImpl() {
     tcmalloc_memory_gc_thread_->Join();
   }
 #endif
+
+  security::DestroyKerberosForServer();
 }
 
 #ifdef TCMALLOC_ENABLED

[kudu] 02/03: [util] optimized version of BitUtil::Ceil()

Posted by al...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit bac76fcf9d496995591e968dffa822fe0fb21d5d
Author: Alexey Serbin <al...@apache.org>
AuthorDate: Mon Jan 31 15:42:09 2022 -0800

    [util] optimized version of BitUtil::Ceil()
    
    This patch introduces an optimized version of BitUtil::Ceil() for the
    case when the divisor is a power of two.  As it turns out, all usages
    of the BitUtil::Ceil() in Kudu are for divisor of 8, so I updated all
    the call sites correspondingly.
    
    This patch also contains the updated RLE benchmark.  The comparison
    results are the following for a RELEASE configuration built with
    c++ (GCC) 8.3.1 20190311 (Red Hat 8.3.1-3)):
    
    Non-optimized implementation:
      Time spent BooleanBitStream: real 0.487s  user 0.486s     sys 0.001s
      Wrote 1048576 bytes
    
      Time spent BooleanRLE: real 2.302s     user 2.304s      sys 0.001s
      Wrote 46080 bytes
    
    Optimized implementation:
      Time spent BooleanBitStream: real 0.017s  user 0.016s     sys 0.000s
      Wrote 1048576 bytes
    
      Time spent BooleanRLE: real 2.055s     user 2.056s      sys 0.001s
      Wrote 46080 bytes
    
    As for benmarking direct calls of these functions:
      Time spent BitUtil::Ceil(..., 8): real 0.720s  user 0.721s     sys 0.000s
      Time spent BitUtil::Ceil<3>(...): real 0.402s  user 0.402s     sys 0.000s
    
    Change-Id: Ia383856aa9a189681f6ee2a0d317476fe3c847bd
    Reviewed-on: http://gerrit.cloudera.org:8080/18187
    Tested-by: Kudu Jenkins
    Reviewed-by: Attila Bukor <ab...@apache.org>
---
 src/kudu/benchmarks/rle.cc              | 33 +++++++++++++++++++++++++++++++++
 src/kudu/util/bit-stream-utils.h        |  4 ++--
 src/kudu/util/bit-stream-utils.inline.h |  4 ++--
 src/kudu/util/bit-util.h                | 10 ++++++++++
 src/kudu/util/rle-encoding.h            |  8 ++++----
 src/kudu/util/rle-test.cc               |  8 ++++----
 6 files changed, 55 insertions(+), 12 deletions(-)

diff --git a/src/kudu/benchmarks/rle.cc b/src/kudu/benchmarks/rle.cc
index a1b1c0c..89520df 100644
--- a/src/kudu/benchmarks/rle.cc
+++ b/src/kudu/benchmarks/rle.cc
@@ -31,6 +31,7 @@
 #include "kudu/gutil/mathlimits.h"
 #include "kudu/util/bit-stream-utils.h"
 #include "kudu/util/bit-stream-utils.inline.h"
+#include "kudu/util/bit-util.h"
 #include "kudu/util/faststring.h"
 #include "kudu/util/logging.h"
 #include "kudu/util/rle-encoding.h"
@@ -92,6 +93,22 @@ int BooleanRLE(faststring* buffer) {
   return bytes_written;
 }
 
+int BitUtilCeil(int num_iter) {
+  volatile int res = 0;
+  for (int i = 0; i < num_iter; ++i) {
+    res = BitUtil::Ceil(i, 8);
+  }
+  return res;
+}
+
+int BitUtilCeilLog2Div(int num_iter) {
+  volatile int res;
+  for (int i = 0; i < num_iter; ++i) {
+    res = BitUtil::Ceil<3>(i);
+  }
+  return res;
+}
+
 } // namespace kudu
 
 int main(int argc, char** argv) {
@@ -121,5 +138,21 @@ int main(int argc, char** argv) {
     LOG(INFO) << "Wrote " << bytes_written << " bytes";
   }
 
+  {
+    int res = 0;
+    LOG_TIMING(INFO, "BitUtil::Ceil(..., 8)") {
+      res = kudu::BitUtilCeil(1000000000);
+    }
+    LOG(INFO) << "Result: " << res;
+  }
+
+  {
+    int res = 0;
+    LOG_TIMING(INFO, "BitUtil::Ceil<3>(...)") {
+      res = kudu::BitUtilCeilLog2Div(1000000000);
+    }
+    LOG(INFO) << "Result: " << res;
+  }
+
   return 0;
 }
diff --git a/src/kudu/util/bit-stream-utils.h b/src/kudu/util/bit-stream-utils.h
index a772c7c..61a74bc 100644
--- a/src/kudu/util/bit-stream-utils.h
+++ b/src/kudu/util/bit-stream-utils.h
@@ -44,7 +44,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_ + BitUtil::Ceil(bit_offset_, 8); }
+  int bytes_written() const { return byte_offset_ + BitUtil::Ceil<3>(bit_offset_); }
 
   // Writes a value to buffered_values_, flushing to buffer_ if necessary.  This is bit
   // packed. num_bits must be <= 32. If 'v' is larger than 'num_bits' bits, the higher
@@ -117,7 +117,7 @@ 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_ + BitUtil::Ceil(bit_offset_, 8)); }
+  int bytes_left() { return max_bytes_ - (byte_offset_ + BitUtil::Ceil<3>(bit_offset_)); }
 
   // Current position in the stream, by bit.
   int position() const { return byte_offset_ * 8 + bit_offset_; }
diff --git a/src/kudu/util/bit-stream-utils.inline.h b/src/kudu/util/bit-stream-utils.inline.h
index f9186f5..41fa632 100644
--- a/src/kudu/util/bit-stream-utils.inline.h
+++ b/src/kudu/util/bit-stream-utils.inline.h
@@ -49,7 +49,7 @@ inline void BitWriter::PutValue(uint64_t v, int num_bits) {
 }
 
 inline void BitWriter::Flush(bool align) {
-  int num_bytes = BitUtil::Ceil(bit_offset_, 8);
+  int num_bytes = BitUtil::Ceil<3>(bit_offset_);
   buffer_->reserve(KUDU_ALIGN_UP(byte_offset_ + num_bytes, 8));
   buffer_->resize(byte_offset_ + num_bytes);
   DCHECK_LE(byte_offset_ + num_bytes, buffer_->capacity());
@@ -172,7 +172,7 @@ inline void BitReader::SeekToBit(uint stream_position) {
 template<typename T>
 inline bool BitReader::GetAligned(int num_bytes, T* v) {
   DCHECK_LE(num_bytes, sizeof(T));
-  int bytes_read = BitUtil::Ceil(bit_offset_, 8);
+  int bytes_read = BitUtil::Ceil<3>(bit_offset_);
   if (PREDICT_FALSE(byte_offset_ + bytes_read + num_bytes > max_bytes_)) return false;
 
   // Advance byte_offset to next unread byte and read num_bytes
diff --git a/src/kudu/util/bit-util.h b/src/kudu/util/bit-util.h
index 25bf41c..9c0ae23 100644
--- a/src/kudu/util/bit-util.h
+++ b/src/kudu/util/bit-util.h
@@ -32,6 +32,16 @@ class BitUtil {
     return value / divisor + (value % divisor != 0);
   }
 
+  // Similar to the above, but a bit optimized for the case when the divisor
+  // is a power of two: LOG2_DIV is log2(divisor), e.g. 3 for the divisor of 8.
+  template <int LOG2_DIV>
+  static inline int Ceil(int value) {
+    constexpr int kDivisor = 1 << LOG2_DIV;
+    constexpr int kComplement = kDivisor - 1;
+    constexpr int kComplementMask = -kDivisor;
+    return ((value + kComplement) & kComplementMask) >> LOG2_DIV;
+  }
+
   // Returns the 'num_bits' least-significant bits of 'v'.
   static inline uint64_t TrailingBits(uint64_t v, int num_bits) {
     if (PREDICT_FALSE(num_bits == 0)) return 0;
diff --git a/src/kudu/util/rle-encoding.h b/src/kudu/util/rle-encoding.h
index 4a00148..58ca87d 100644
--- a/src/kudu/util/rle-encoding.h
+++ b/src/kudu/util/rle-encoding.h
@@ -247,7 +247,7 @@ inline bool RleDecoder<T>::ReadHeader() {
       repeat_count_ = indicator_value >> 1;
       DCHECK_GT(repeat_count_, 0);
       bool result = bit_reader_.GetAligned<T>(
-          BitUtil::Ceil(bit_width_, 8), reinterpret_cast<T*>(&current_value_));
+          BitUtil::Ceil<3>(bit_width_), reinterpret_cast<T*>(&current_value_));
       DCHECK(result);
     }
   }
@@ -440,7 +440,7 @@ inline void RleEncoder<T>::FlushLiteralRun(bool update_indicator_byte) {
     // We only reserve one byte, to allow for streaming writes of literal values.
     // The logic makes sure we flush literal runs often enough to not overrun
     // the 1 byte.
-    int num_groups = BitUtil::Ceil(literal_count_, 8);
+    int num_groups = BitUtil::Ceil<3>(literal_count_);
     int32_t indicator_value = (num_groups << 1) | 1;
     DCHECK_EQ(indicator_value & 0xFFFFFF00, 0);
     bit_writer_.buffer()->data()[literal_indicator_byte_idx_] = indicator_value;
@@ -455,7 +455,7 @@ inline void RleEncoder<T>::FlushRepeatedRun() {
   // The lsb of 0 indicates this is a repeated run
   int32_t indicator_value = repeat_count_ << 1 | 0;
   bit_writer_.PutVlqInt(indicator_value);
-  bit_writer_.PutAligned(current_value_, BitUtil::Ceil(bit_width_, 8));
+  bit_writer_.PutAligned(current_value_, BitUtil::Ceil<3>(bit_width_));
   num_buffered_values_ = 0;
   repeat_count_ = 0;
 }
@@ -480,7 +480,7 @@ inline void RleEncoder<T>::FlushBufferedValues(bool done) {
   }
 
   literal_count_ += num_buffered_values_;
-  int num_groups = BitUtil::Ceil(literal_count_, 8);
+  int num_groups = BitUtil::Ceil<3>(literal_count_);
   if (num_groups + 1 >= (1 << 6)) {
     // We need to start a new literal run because the indicator byte we've reserved
     // cannot store more values.
diff --git a/src/kudu/util/rle-test.cc b/src/kudu/util/rle-test.cc
index e443b6d..c8467c3 100644
--- a/src/kudu/util/rle-test.cc
+++ b/src/kudu/util/rle-test.cc
@@ -123,7 +123,7 @@ TEST(BitArray, TestBool) {
 
 // Writes 'num_vals' values with width 'bit_width' and reads them back.
 void TestBitArrayValues(int bit_width, int num_vals) {
-  const int kTestLen = BitUtil::Ceil(bit_width * num_vals, 8);
+  const int kTestLen = BitUtil::Ceil<3>(bit_width * num_vals);
   const uint64_t mod = bit_width == 64? 1 : 1LL << bit_width;
 
   faststring buffer(kTestLen);
@@ -248,14 +248,14 @@ TEST(Rle, SpecificSequences) {
   }
 
   for (int width = 9; width <= kMaxWidth; ++width) {
-    ValidateRle(values, width, nullptr, 2 * (1 + BitUtil::Ceil(width, 8)));
+    ValidateRle(values, width, nullptr, 2 * (1 + BitUtil::Ceil<3>(width)));
   }
 
   // Test 100 0's and 1's alternating
   for (int i = 0; i < 100; ++i) {
     values[i] = i % 2;
   }
-  int num_groups = BitUtil::Ceil(100, 8);
+  int num_groups = BitUtil::Ceil<3>(100);
   expected_buffer[0] = (num_groups << 1) | 1;
   for (int i = 0; i < 100/8; ++i) {
     expected_buffer[i + 1] = BOOST_BINARY(1 0 1 0 1 0 1 0); // 0xaa
@@ -266,7 +266,7 @@ 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 <= kMaxWidth; ++width) {
-    ValidateRle(values, width, nullptr, 1 + BitUtil::Ceil(width * 100, 8));
+    ValidateRle(values, width, nullptr, 1 + BitUtil::Ceil<3>(width * 100));
   }
 }