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:55 UTC

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

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));
   }
 }