You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@impala.apache.org by ta...@apache.org on 2016/04/12 23:19:15 UTC

[41/50] incubator-impala git commit: Use more efficient gutil implementation of Log2Ceiling

Use more efficient gutil implementation of Log2Ceiling

This was used in FreePool::Allocate, which is called a lot in some
workloads, so may have some perf impact. I was looking at using this
function for an allocator for the new buffer pool and it makes the
allocator code simpler if I can call this function freely without
perf concerns.

Modified gutil's implementation slightly so that gcc optimises it to
the following branch-free code:

.LHOTB8:
        .align 2
        .p2align 4,,15
        .globl  _ZN4Bits20Log2CeilingNonZero64Ey
        .type   _ZN4Bits20Log2CeilingNonZero64Ey, @function
_ZN4Bits20Log2CeilingNonZero64Ey:
.LFB1797:
        .loc 1 90 0
        .cfi_startproc
.LVL40:
.LBB33:
        .loc 1 92 0
        leaq    -1(%rdi), %rdx  #, D.40305
.LBB34:
.LBB35:
        .loc 2 155 0
        bsrq    %rdi, %rax      # n, tmp89
.LVL41:
.LBE35:
.LBE34:
        .loc 1 92 0
        andq    %rdx, %rdi      # D.40305, D.40305
.LVL42:
        .loc 1 95 0
        cmpq    $1, %rdi        #, D.40305
        sbbl    $-1, %eax       #, D.40304
.LVL43:
.LBE33:
        .loc 1 96 0
        ret
        .cfi_endproc

Change-Id: Ifbffb49b5c86394e5a9ccd711132543856321be0
Reviewed-on: http://gerrit.cloudera.org:8080/2735
Reviewed-by: Tim Armstrong <ta...@cloudera.com>
Tested-by: Internal Jenkins


Project: http://git-wip-us.apache.org/repos/asf/incubator-impala/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-impala/commit/92fafa1c
Tree: http://git-wip-us.apache.org/repos/asf/incubator-impala/tree/92fafa1c
Diff: http://git-wip-us.apache.org/repos/asf/incubator-impala/diff/92fafa1c

Branch: refs/heads/master
Commit: 92fafa1c0687141481e56ea6f09a67987d571d6e
Parents: 344fbc0
Author: Tim Armstrong <ta...@cloudera.com>
Authored: Fri Apr 8 10:23:03 2016 -0700
Committer: Tim Armstrong <ta...@cloudera.com>
Committed: Tue Apr 12 14:03:44 2016 -0700

----------------------------------------------------------------------
 be/src/exec/hdfs-parquet-scanner.cc |  3 ++-
 be/src/gutil/bits.cc                | 16 ---------------
 be/src/gutil/bits.h                 | 34 ++++++++++++++++++++++++++++++++
 be/src/runtime/disk-io-mgr.cc       | 12 ++++++-----
 be/src/runtime/free-pool.h          |  4 ++--
 be/src/runtime/runtime-filter.cc    |  6 +++---
 be/src/util/bit-util-test.cc        | 22 ++++++++++++---------
 be/src/util/bit-util.h              | 16 ---------------
 be/src/util/bloom-filter.h          |  4 ++--
 be/src/util/dict-encoding.h         |  3 ++-
 10 files changed, 65 insertions(+), 55 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/92fafa1c/be/src/exec/hdfs-parquet-scanner.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/hdfs-parquet-scanner.cc b/be/src/exec/hdfs-parquet-scanner.cc
index 9e18c66..ead3628 100644
--- a/be/src/exec/hdfs-parquet-scanner.cc
+++ b/be/src/exec/hdfs-parquet-scanner.cc
@@ -27,6 +27,7 @@
 #include "exec/scanner-context.inline.h"
 #include "exec/read-write-util.h"
 #include "exprs/expr.h"
+#include "gutil/bits.h"
 #include "runtime/collection-value-builder.h"
 #include "runtime/descriptors.h"
 #include "runtime/runtime-state.h"
@@ -1231,7 +1232,7 @@ Status HdfsParquetScanner::LevelDecoder::Init(const string& filename,
       if (num_bytes < 0) {
         return Status(TErrorCode::PARQUET_CORRUPT_VALUE, filename, num_bytes);
       }
-      int bit_width = BitUtil::Log2(max_level + 1);
+      int bit_width = Bits::Log2Ceiling64(max_level + 1);
       Reset(*data, num_bytes, bit_width);
       break;
     }

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/92fafa1c/be/src/gutil/bits.cc
----------------------------------------------------------------------
diff --git a/be/src/gutil/bits.cc b/be/src/gutil/bits.cc
index bc4e451..dec6f51 100644
--- a/be/src/gutil/bits.cc
+++ b/be/src/gutil/bits.cc
@@ -71,22 +71,6 @@ int Bits::Log2Floor_Portable(uint32 n) {
   return log;
 }
 
-int Bits::Log2Ceiling(uint32 n) {
-  int floor = Log2Floor(n);
-  if (n == (n &~ (n - 1)))              // zero or a power of two
-    return floor;
-  else
-    return floor + 1;
-}
-
-int Bits::Log2Ceiling64(uint64 n) {
-  int floor = Log2Floor64(n);
-  if (n == (n &~ (n - 1)))              // zero or a power of two
-    return floor;
-  else
-    return floor + 1;
-}
-
 int Bits::FindLSBSetNonZero_Portable(uint32 n) {
   int rc = 31;
   for (int i = 4, shift = 1 << 4; i >= 0; --i) {

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/92fafa1c/be/src/gutil/bits.h
----------------------------------------------------------------------
diff --git a/be/src/gutil/bits.h b/be/src/gutil/bits.h
index bacd655..579d954 100644
--- a/be/src/gutil/bits.h
+++ b/be/src/gutil/bits.h
@@ -79,6 +79,10 @@ class Bits {
   static int Log2Ceiling(uint32 n);
   static int Log2Ceiling64(uint64 n);
 
+  // Potentially faster version of Log2Ceiling() that returns an
+  // undefined value if n == 0
+  static int Log2CeilingNonZero64(uint64 n);
+
   // Return the first set least / most significant bit, 0-indexed.  Returns an
   // undefined value if n == 0.  FindLSBSetNonZero() is similar to ffs() except
   // that it's 0-indexed, while FindMSBSetNonZero() is the same as
@@ -139,6 +143,36 @@ inline int Bits::Log2FloorNonZero(uint32 n) {
   return 31 ^ __builtin_clz(n);
 }
 
+inline int Bits::Log2Ceiling(uint32 n) {
+  int floor = Log2Floor(n);
+  // Check if zero or a power of two. This pattern is recognised by gcc and optimised
+  // into branch-free code.
+  if (0 == (n & (n - 1)))
+    return floor;
+  else
+    return floor + 1;
+}
+
+inline int Bits::Log2Ceiling64(uint64 n) {
+  int floor = Log2Floor64(n);
+  // Check if zero or a power of two. This pattern is recognised by gcc and optimised
+  // into branch-free code.
+  if (0 == (n & (n - 1)))
+    return floor;
+  else
+    return floor + 1;
+}
+
+inline int Bits::Log2CeilingNonZero64(uint64 n) {
+  int floor = Log2FloorNonZero64(n);
+  // Check if zero or a power of two. This pattern is recognised by gcc and optimised
+  // into branch-free code.
+  if (0 == (n & (n - 1)))
+    return floor;
+  else
+    return floor + 1;
+}
+
 inline int Bits::FindLSBSetNonZero(uint32 n) {
   return __builtin_ctz(n);
 }

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/92fafa1c/be/src/runtime/disk-io-mgr.cc
----------------------------------------------------------------------
diff --git a/be/src/runtime/disk-io-mgr.cc b/be/src/runtime/disk-io-mgr.cc
index 80675d7..4788739 100644
--- a/be/src/runtime/disk-io-mgr.cc
+++ b/be/src/runtime/disk-io-mgr.cc
@@ -14,11 +14,13 @@
 
 #include "runtime/disk-io-mgr.h"
 #include "runtime/disk-io-mgr-internal.h"
-#include "util/hdfs-util.h"
 
-#include <gutil/strings/substitute.h>
 #include <boost/algorithm/string.hpp>
 
+#include "gutil/bits.h"
+#include "gutil/strings/substitute.h"
+#include "util/hdfs-util.h"
+
 DECLARE_bool(disable_mem_pools);
 
 #include "common/names.h"
@@ -270,7 +272,7 @@ DiskIoMgr::DiskIoMgr() :
         FileSystemUtil::MaxNumFileHandles()),
         &HdfsCachedFileHandle::Release) {
   int64_t max_buffer_size_scaled = BitUtil::Ceil(max_buffer_size_, min_buffer_size_);
-  free_buffers_.resize(BitUtil::Log2(max_buffer_size_scaled) + 1);
+  free_buffers_.resize(Bits::Log2Ceiling64(max_buffer_size_scaled) + 1);
   int num_local_disks = FLAGS_num_disks == 0 ? DiskInfo::num_disks() : FLAGS_num_disks;
   disk_queues_.resize(num_local_disks + REMOTE_NUM_DISKS);
   CheckSseSupport();
@@ -288,7 +290,7 @@ DiskIoMgr::DiskIoMgr(int num_local_disks, int threads_per_disk, int min_buffer_s
     file_handle_cache_(min(FLAGS_max_cached_file_handles,
             FileSystemUtil::MaxNumFileHandles()), &HdfsCachedFileHandle::Release) {
   int64_t max_buffer_size_scaled = BitUtil::Ceil(max_buffer_size_, min_buffer_size_);
-  free_buffers_.resize(BitUtil::Log2(max_buffer_size_scaled) + 1);
+  free_buffers_.resize(Bits::Log2Ceiling64(max_buffer_size_scaled) + 1);
   if (num_local_disks == 0) num_local_disks = DiskInfo::num_disks();
   disk_queues_.resize(num_local_disks + REMOTE_NUM_DISKS);
   CheckSseSupport();
@@ -1129,7 +1131,7 @@ Status DiskIoMgr::WriteRangeHelper(FILE* file_handle, WriteRange* write_range) {
 
 int DiskIoMgr::free_buffers_idx(int64_t buffer_size) {
   int64_t buffer_size_scaled = BitUtil::Ceil(buffer_size, min_buffer_size_);
-  int idx = BitUtil::Log2(buffer_size_scaled);
+  int idx = Bits::Log2Ceiling64(buffer_size_scaled);
   DCHECK_GE(idx, 0);
   DCHECK_LT(idx, free_buffers_.size());
   return idx;

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/92fafa1c/be/src/runtime/free-pool.h
----------------------------------------------------------------------
diff --git a/be/src/runtime/free-pool.h b/be/src/runtime/free-pool.h
index 2985ecb..dfabaf0 100644
--- a/be/src/runtime/free-pool.h
+++ b/be/src/runtime/free-pool.h
@@ -20,6 +20,7 @@
 #include <string.h>
 #include <string>
 #include "common/logging.h"
+#include "gutil/bits.h"
 #include "runtime/mem-pool.h"
 #include "util/bit-util.h"
 
@@ -66,8 +67,7 @@ class FreePool {
     /// This is the typical malloc behavior. NULL is reserved for failures.
     if (size == 0) return reinterpret_cast<uint8_t*>(0x1);
 
-    /// Do ceil(log_2(size))
-    int free_list_idx = BitUtil::Log2(size);
+    int free_list_idx = Bits::Log2Ceiling64(size);
     DCHECK_LT(free_list_idx, NUM_LISTS);
 
     FreeListNode* allocation = lists_[free_list_idx].next;

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/92fafa1c/be/src/runtime/runtime-filter.cc
----------------------------------------------------------------------
diff --git a/be/src/runtime/runtime-filter.cc b/be/src/runtime/runtime-filter.cc
index 675e0f3..a68eb69 100644
--- a/be/src/runtime/runtime-filter.cc
+++ b/be/src/runtime/runtime-filter.cc
@@ -14,9 +14,9 @@
 
 #include "runtime/runtime-filter.inline.h"
 
-#include <gutil/strings/substitute.h>
-
 #include "common/names.h"
+#include "gutil/bits.h"
+#include "gutil/strings/substitute.h"
 #include "runtime/client-cache.h"
 #include "runtime/exec-env.h"
 #include "service/impala-server.h"
@@ -43,7 +43,7 @@ RuntimeFilterBank::RuntimeFilterBank(const TQueryCtx& query_ctx, RuntimeState* s
   int32_t bloom_filter_size = query_ctx_.request.query_options.runtime_bloom_filter_size;
   bloom_filter_size = std::max(bloom_filter_size, MIN_BLOOM_FILTER_SIZE);
   bloom_filter_size = std::min(bloom_filter_size, MAX_BLOOM_FILTER_SIZE);
-  log_filter_size_ = BitUtil::Log2(bloom_filter_size);
+  log_filter_size_ = Bits::Log2Ceiling64(bloom_filter_size);
 }
 
 RuntimeFilter* RuntimeFilterBank::RegisterFilter(const TRuntimeFilterDesc& filter_desc,

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/92fafa1c/be/src/util/bit-util-test.cc
----------------------------------------------------------------------
diff --git a/be/src/util/bit-util-test.cc b/be/src/util/bit-util-test.cc
index 9886429..f376737 100644
--- a/be/src/util/bit-util-test.cc
+++ b/be/src/util/bit-util-test.cc
@@ -19,6 +19,8 @@
 
 #include <boost/utility.hpp>
 #include <gtest/gtest.h>
+
+#include "gutil/bits.h"
 #include "util/bit-util.h"
 #include "util/cpu-info.h"
 
@@ -99,7 +101,7 @@ TEST(BitUtil, ByteSwap) {
   EXPECT_EQ(BitUtil::ByteSwap(static_cast<uint64_t>(0)), 0);
   EXPECT_EQ(BitUtil::ByteSwap(
       static_cast<uint64_t>(0x1122334455667788)), 0x8877665544332211);
-  
+
   EXPECT_EQ(BitUtil::ByteSwap(static_cast<int64_t>(0)), 0);
   EXPECT_EQ(BitUtil::ByteSwap(
       static_cast<int64_t>(0x1122334455667788)), 0x8877665544332211);
@@ -112,14 +114,16 @@ TEST(BitUtil, ByteSwap) {
 }
 
 TEST(BitUtil, Log2) {
-  EXPECT_EQ(BitUtil::Log2(1), 0);
-  EXPECT_EQ(BitUtil::Log2(2), 1);
-  EXPECT_EQ(BitUtil::Log2(3), 2);
-  EXPECT_EQ(BitUtil::Log2(4), 2);
-  EXPECT_EQ(BitUtil::Log2(5), 3);
-  EXPECT_EQ(BitUtil::Log2(INT_MAX), 31);
-  EXPECT_EQ(BitUtil::Log2(UINT_MAX), 32);
-  EXPECT_EQ(BitUtil::Log2(ULLONG_MAX), 64);
+  // We use gutil's implementation in place of an older custom implementation in BitUtil.
+  // We leave this test here to ensure no test coverage is lost.
+  EXPECT_EQ(Bits::Log2CeilingNonZero64(1), 0);
+  EXPECT_EQ(Bits::Log2CeilingNonZero64(2), 1);
+  EXPECT_EQ(Bits::Log2CeilingNonZero64(3), 2);
+  EXPECT_EQ(Bits::Log2CeilingNonZero64(4), 2);
+  EXPECT_EQ(Bits::Log2CeilingNonZero64(5), 3);
+  EXPECT_EQ(Bits::Log2CeilingNonZero64(INT_MAX), 31);
+  EXPECT_EQ(Bits::Log2CeilingNonZero64(UINT_MAX), 32);
+  EXPECT_EQ(Bits::Log2CeilingNonZero64(ULLONG_MAX), 64);
 }
 
 TEST(BitUtil, RoundUpToPowerOf2) {

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/92fafa1c/be/src/util/bit-util.h
----------------------------------------------------------------------
diff --git a/be/src/util/bit-util.h b/be/src/util/bit-util.h
index 0031558..e255f0c 100644
--- a/be/src/util/bit-util.h
+++ b/be/src/util/bit-util.h
@@ -144,22 +144,6 @@ class BitUtil {
     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;
-  }
-
   /// Swaps the byte order (i.e. endianess)
   static inline int64_t ByteSwap(int64_t value) {
     return __builtin_bswap64(value);

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/92fafa1c/be/src/util/bloom-filter.h
----------------------------------------------------------------------
diff --git a/be/src/util/bloom-filter.h b/be/src/util/bloom-filter.h
index 32f6ee6..6fdd140 100644
--- a/be/src/util/bloom-filter.h
+++ b/be/src/util/bloom-filter.h
@@ -109,8 +109,8 @@ class BloomFilter {
   typedef uint64_t BucketWord;
 
   // log2(number of bits in a BucketWord)
-  // TODO: Use BitUtil::Log2(numeric_limits<BucketWord>::digits) once we enable C++14 for
-  // codegen.
+  // TODO: Use Bits::Log2Ceiling64(numeric_limits<BucketWord>::digits) once we enable
+  // C++14 for codegen.
   static const int LOG_BUCKET_WORD_BITS = 6;
   static const BucketWord BUCKET_WORD_MASK = 63; // 2^LOG_BUCKET_WORD_BITS - 1
 

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/92fafa1c/be/src/util/dict-encoding.h
----------------------------------------------------------------------
diff --git a/be/src/util/dict-encoding.h b/be/src/util/dict-encoding.h
index e629f33..75e89ac 100644
--- a/be/src/util/dict-encoding.h
+++ b/be/src/util/dict-encoding.h
@@ -20,6 +20,7 @@
 #include <boost/foreach.hpp>
 #include <boost/unordered_map.hpp>
 
+#include "gutil/bits.h"
 #include "exec/parquet-common.h"
 #include "runtime/mem-pool.h"
 #include "runtime/string-value.h"
@@ -70,7 +71,7 @@ class DictEncoderBase {
   int bit_width() const {
     if (UNLIKELY(num_entries() == 0)) return 0;
     if (UNLIKELY(num_entries() == 1)) return 1;
-    return BitUtil::Log2(num_entries());
+    return Bits::Log2Ceiling64(num_entries());
   }
 
   /// Writes out any buffered indices to buffer preceded by the bit width of this data.