You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kudu.apache.org by aw...@apache.org on 2020/04/09 04:03:04 UTC

[kudu] 03/03: encoding-test: Clean up bitshuffle tests a little

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

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

commit 972edb9332dde7cc6fba542343c7a7434ef6cbff
Author: Bankim Bhavsar <ba...@cloudera.com>
AuthorDate: Tue Mar 31 12:40:47 2020 -0700

    encoding-test: Clean up bitshuffle tests a little
    
    Also adds functions in random number generator to generate 128-bit integers.
    
    Change-Id: I8b7bee1f952c46c09df5102ac1916141c6935284
    Reviewed-on: http://gerrit.cloudera.org:8080/15043
    Reviewed-by: Andrew Wong <aw...@cloudera.com>
    Tested-by: Kudu Jenkins
---
 src/kudu/cfile/encoding-test.cc   | 58 ++++++++++++++++++++++-----------------
 src/kudu/util/random.h            | 53 +++++++++++++++++++++++++++++++----
 src/kudu/util/random_util-test.cc | 50 +++++++++++++++++++++++++++++++++
 src/kudu/util/random_util.h       | 31 +++++++++++++++++++--
 4 files changed, 158 insertions(+), 34 deletions(-)

diff --git a/src/kudu/cfile/encoding-test.cc b/src/kudu/cfile/encoding-test.cc
index cb25c1b..007f5d4 100644
--- a/src/kudu/cfile/encoding-test.cc
+++ b/src/kudu/cfile/encoding-test.cc
@@ -23,6 +23,7 @@
 #include <cstdio>
 #include <cstdlib>
 #include <functional>
+#include <initializer_list>
 #include <limits>
 #include <memory>
 #include <ostream>
@@ -436,7 +437,7 @@ class TestEncoding : public KuduTest {
   }
 
   template <DataType Type, class BlockBuilder, class BlockDecoder>
-  void TestEncodeDecodeTemplateBlockEncoder(typename TypeTraits<Type>::cpp_type* src,
+  void TestEncodeDecodeTemplateBlockEncoder(const typename TypeTraits<Type>::cpp_type* src,
                                             size_t size) {
     typedef typename TypeTraits<Type>::cpp_type CppType;
     const uint32_t kOrdinalPosBase = 12345;
@@ -719,39 +720,46 @@ TEST_F(TestEncoding, TestPlainBlockEncoder) {
 
 // Test for bitshuffle block, for INT32, INT64, INT128, FLOAT, DOUBLE
 TEST_F(TestEncoding, TestBShufInt32BlockEncoder) {
-  const uint32_t kSize = 10000;
-
-  unique_ptr<int32_t[]> ints(new int32_t[kSize]);
-  for (int i = 0; i < kSize; i++) {
-    ints.get()[i] = random();
+  using limits = std::numeric_limits<int32_t>;
+  Random rng(SeedRandom());
+  auto sequences = {
+      CreateRandomIntegersInRange<int32_t>(10000, 1000, 2000, &rng),
+      CreateRandomIntegersInRange<int32_t>(10000, 0, limits::max(), &rng),
+      CreateRandomIntegersInRange<int32_t>(10000, limits::min(), limits::max(), &rng)
+  };
+  for (const auto& ints : sequences) {
+    TestEncodeDecodeTemplateBlockEncoder<INT32, BShufBlockBuilder<INT32>,
+                                         BShufBlockDecoder<INT32> >(ints.data(), ints.size());
   }
-
-  TestEncodeDecodeTemplateBlockEncoder<INT32, BShufBlockBuilder<INT32>,
-      BShufBlockDecoder<INT32>>(ints.get(), kSize);
 }
 
 TEST_F(TestEncoding, TestBShufInt64BlockEncoder) {
-  const uint32_t kSize = 10000;
-
-  unique_ptr<int64_t[]> ints(new int64_t[kSize]);
-  for (int i = 0; i < kSize; i++) {
-    ints.get()[i] = random();
+  using limits = std::numeric_limits<int64_t>;
+  Random rng(SeedRandom());
+  auto sequences = {
+      CreateRandomIntegersInRange<int64_t>(10000, 1000, 2000, &rng),
+      CreateRandomIntegersInRange<int64_t>(10000, 0, limits::max(), &rng),
+      CreateRandomIntegersInRange<int64_t>(10000, limits::min(), limits::max(), &rng)
+  };
+  for (const auto& ints : sequences) {
+    TestEncodeDecodeTemplateBlockEncoder<INT64, BShufBlockBuilder<INT64>,
+                                         BShufBlockDecoder<INT64> >(ints.data(), ints.size());
   }
-
-  TestEncodeDecodeTemplateBlockEncoder<INT64, BShufBlockBuilder<INT64>,
-      BShufBlockDecoder<INT64>>(ints.get(), kSize);
 }
 
 TEST_F(TestEncoding, TestBShufInt128BlockEncoder) {
-  const uint32_t kSize = 10000;
-
-  unique_ptr<int128_t[]> ints(new int128_t[kSize]);
-  for (int i = 0; i < kSize; i++) {
-    ints.get()[i] = random();
+  Random rng(SeedRandom());
+  // As per the note in int128.h, numeric_limits<> on int128_t can give incorrect results.
+  // Hence using predefined min, max constants.
+  auto sequences = {
+      CreateRandomIntegersInRange<int128_t>(10000, 1000, 2000, &rng),
+      CreateRandomIntegersInRange<int128_t>(10000, 0, INT128_MAX, &rng),
+      CreateRandomIntegersInRange<int128_t>(10000, INT128_MIN, INT128_MAX, &rng)
+  };
+  for (const auto& ints : sequences) {
+    TestEncodeDecodeTemplateBlockEncoder<INT128, BShufBlockBuilder<INT128>,
+                                         BShufBlockDecoder<INT128> >(ints.data(), ints.size());
   }
-
-  TestEncodeDecodeTemplateBlockEncoder<INT128, BShufBlockBuilder<INT128>,
-      BShufBlockDecoder<INT128>>(ints.get(), kSize);
 }
 
 TEST_F(TestEncoding, TestBShufFloatBlockEncoder) {
diff --git a/src/kudu/util/random.h b/src/kudu/util/random.h
index 51e578b..9c2722d 100644
--- a/src/kudu/util/random.h
+++ b/src/kudu/util/random.h
@@ -12,6 +12,7 @@
 
 #include "kudu/gutil/casts.h"
 #include "kudu/gutil/map-util.h"
+#include "kudu/util/int128.h"
 #include "kudu/util/locks.h"
 
 namespace kudu {
@@ -82,6 +83,16 @@ class Random {
     return large;
   }
 
+  // Next pseudo-random 128-bit unsigned integer.
+  uint128_t Next128() {
+    uint128_t large = Next64();
+    large <<= 64U;
+    large |= Next64();
+    // Next64() provides entire range of numbers up to 64th bit.
+    // So unlike Next64(), no need to fill MSB bit(s).
+    return large;
+  }
+
   // Returns a uniformly distributed value in the range [0..n-1]
   // REQUIRES: n > 0
   uint32_t Uniform(uint32_t n) { return Next() % n; }
@@ -93,6 +104,9 @@ class Random {
   // REQUIRES: n > 0
   uint64_t Uniform64(uint64_t n) { return Next64() % n; }
 
+  // Returns a uniformly distributed 128-bit value in the range [0..n-1]
+  uint128_t Uniform128(uint128_t n) { return Next128() % n; }
+
   // Randomly returns true ~"1/n" of the time, and false otherwise.
   // REQUIRES: n > 0
   bool OneIn(int n) { return (Next() % n) == 0; }
@@ -140,6 +154,11 @@ class ThreadSafeRandom {
     return random_.Next64();
   }
 
+  uint128_t Next128() {
+    std::lock_guard<simple_spinlock> l(lock_);
+    return random_.Next128();
+  }
+
   uint32_t Uniform(uint32_t n) {
     std::lock_guard<simple_spinlock> l(lock_);
     return random_.Uniform(n);
@@ -155,6 +174,11 @@ class ThreadSafeRandom {
     return random_.Uniform64(n);
   }
 
+  uint128_t Uniform128(uint128_t n) {
+    std::lock_guard<simple_spinlock> l(lock_);
+    return random_.Uniform128(n);
+  }
+
   bool OneIn(int n) {
     std::lock_guard<simple_spinlock> l(lock_);
     return random_.OneIn(n);
@@ -207,18 +231,35 @@ inline double Random::Normal(double mean, double std_dev) {
 }
 
 // Generate next random integer with data-type specified by IntType template
-// parameter which must be a 32-bit or 64-bit integer. This constexpr function
+// parameter which must be a 32-bit, 64-bit, or 128-bit unsigned integer. This constexpr function
 // is useful when generating random numbers in a loop with template parameter
-// and avoids the run-time cost of determining Next32() v/s Next64() in RNG.
+// and avoids the run-time cost of determining Next32() v/s Next64() v/s Next128() in RNG.
 //
 // Note: This constexpr function can't be a member function of
 // Random/ThreadSafeRandom class because it invokes non-const member function.
 template<typename IntType, class RNG>
 constexpr IntType GetNextRandom(RNG* rand) {
-  static_assert(
-      std::is_integral<IntType>::value && (sizeof(IntType) == 4 || sizeof(IntType) == 8),
-      "Only 32-bit and 64-bit integers supported");
-  return sizeof(IntType) == 4 ? rand->Next32() : rand->Next64();
+  // type_traits not defined for 128-bit int data types, hence not checking for integral value.
+  static_assert(sizeof(IntType) == 4 || sizeof(IntType) == 8 || sizeof(IntType) == 16,
+                "Only 32-bit, 64-bit, or 128-bit integers supported");
+
+  // if/switch statement disallowed in constexpr function in C++11.
+  return sizeof(IntType) == 4 ? rand->Next32() :
+                                sizeof(IntType) == 8 ? rand->Next64() : rand->Next128();
+}
+
+// Generate next random integer in the [0, n-1] range with data-type specified by IntType template
+// parameter which must be a 32-bit, 64-bit, or 128-bit unsigned integer.
+// Note: Same comments apply to this constexpr function as GetNextRandom() function above.
+template<typename IntType, class RNG>
+constexpr IntType GetNextUniformRandom(IntType n, RNG* rand) {
+  // type_traits not defined for 128-bit int data types, hence not checking for integral value.
+  static_assert(sizeof(n) == 4 || sizeof(n) == 8 || sizeof(n) == 16,
+                "Only 32-bit, 64-bit, or 128-bit integers generated");
+
+  // if/switch statement disallowed in constexpr function in C++11.
+  return sizeof(n) == 4 ? rand->Uniform32(n) :
+                          sizeof(n) == 8 ? rand->Uniform64(n) : rand->Uniform128(n);
 }
 
 }  // namespace kudu
diff --git a/src/kudu/util/random_util-test.cc b/src/kudu/util/random_util-test.cc
index cf17225..9170aca 100644
--- a/src/kudu/util/random_util-test.cc
+++ b/src/kudu/util/random_util-test.cc
@@ -19,13 +19,16 @@
 
 #include <cstdint>
 #include <cstring>
+#include <limits>
 #include <ostream>
 #include <unordered_set>
+#include <vector>
 
 #include <glog/logging.h>
 #include <gtest/gtest.h>
 
 #include "kudu/gutil/map-util.h"
+#include "kudu/util/int128.h"
 #include "kudu/util/random.h"
 #include "kudu/util/test_util.h"
 
@@ -77,6 +80,26 @@ TEST_F(RandomUtilTest, TestRandomString) {
   CheckEmpty(start, 0, 0, kLenMax);
 }
 
+// Static helper function for testing generation of random numbers in range.
+// CreateRandomUniqueIntegers() doesn't support 128-bit integers yet. Hence a separate
+// template function instead of member of TemplateRandomUtilTest class.
+// TODO(bankim): Once CreateRandomIntegersInRange() supports 128-bit integers make it a
+//               member function of TemplateRandomUtilTest class.
+template <typename IntType>
+static void RunCreateRandomIntegersInRangerHelper(int num_trials, IntType min_val, IntType max_val,
+                                                  Random* rng) {
+  static constexpr int kMaxNumVals = 1000;
+  for (int i = 0; i < num_trials; ++i) {
+    int num_vals = rng->Uniform(kMaxNumVals);
+    auto vals = CreateRandomIntegersInRange<IntType>(num_vals, min_val, max_val, rng);
+    ASSERT_EQ(num_vals, vals.size());
+    for (const auto& v : vals) {
+      ASSERT_GE(v, min_val);
+      ASSERT_LT(v, max_val);
+    }
+  }
+}
+
 template<typename IntType>
 class TemplateRandomUtilTest : public RandomUtilTest {
  public:
@@ -95,6 +118,15 @@ class TemplateRandomUtilTest : public RandomUtilTest {
       }
     }
   }
+
+  void RunCreateRandomIntegersInRange() {
+    static constexpr IntType min_val = std::numeric_limits<IntType>::min();
+    static constexpr IntType max_val = std::numeric_limits<IntType>::max();
+    // Exercise entire range.
+    RunCreateRandomIntegersInRangerHelper(kNumTrials, min_val, max_val, &rng_);
+    // Exercise partial range.
+    RunCreateRandomIntegersInRangerHelper(kNumTrials, min_val / 2, max_val / 2, &rng_);
+  }
 };
 
 // Testing with char, short data-types will result in compile-time error, as expected.
@@ -106,4 +138,22 @@ TYPED_TEST(TemplateRandomUtilTest, RunCreateRandomUniqueIntegers) {
   this->RunCreateRandomUniqueIntegers();
 }
 
+TYPED_TEST(TemplateRandomUtilTest, RunCreateRandomIntegersInRange) {
+  this->RunCreateRandomIntegersInRange();
+}
+
+// TODO(bankim): CreateRandomUniqueIntegers() doesn't support 128-bit integers yet.
+//               Once it does add int128_t and uint128_t types to IntTypes and use
+//               templatized TemplateRandomUtilTest instead.
+TEST_F(RandomUtilTest, RunCreateRandom128BitIntegersInRange) {
+  // Exercise entire range.
+  RunCreateRandomIntegersInRangerHelper<int128_t>(kNumTrials, INT128_MIN, INT128_MAX, &rng_);
+  RunCreateRandomIntegersInRangerHelper<uint128_t>(kNumTrials, 0, UINT128_MAX, &rng_);
+
+  // Exercise partial range.
+  RunCreateRandomIntegersInRangerHelper<int128_t>(kNumTrials, INT128_MIN / 2, INT128_MAX / 2,
+                                                  &rng_);
+  RunCreateRandomIntegersInRangerHelper<uint128_t>(kNumTrials, 0, UINT128_MAX / 2, &rng_);
+}
+
 } // namespace kudu
diff --git a/src/kudu/util/random_util.h b/src/kudu/util/random_util.h
index b89ea3c..4448023 100644
--- a/src/kudu/util/random_util.h
+++ b/src/kudu/util/random_util.h
@@ -27,6 +27,7 @@
 #include <glog/logging.h>
 
 #include "kudu/gutil/map-util.h"
+#include "kudu/util/int128_util.h"
 #include "kudu/util/random.h"
 
 namespace kudu {
@@ -103,13 +104,13 @@ void ReservoirSample(const Collection& c, int k, const Set& avoid,
 
 // Generates random and unique 32-bit or 64-bit integers that are not present in
 // the "avoid" set.
+// TODO(bankim): Add support for 128-bit integers by providing hash value for 128-bit data types.
 template<typename IntType, class Set, class RNG>
 std::unordered_set<IntType> CreateRandomUniqueIntegers(const int num_values,
                                                        const Set& avoid,
                                                        RNG* rand) {
-  static_assert(
-      std::is_integral<IntType>::value && (sizeof(IntType) == 4 || sizeof(IntType) == 8),
-      "Only 32-bit and 64-bit integers generated");
+  static_assert(std::is_integral<IntType>::value && (sizeof(IntType) == 4 || sizeof(IntType) == 8),
+                "Only 32-bit or 64-bit integers generated");
 
   std::unordered_set<IntType> result;
   while (result.size() < num_values) {
@@ -124,5 +125,29 @@ std::unordered_set<IntType> CreateRandomUniqueIntegers(const int num_values,
   return result;
 }
 
+// Generates random 32-bit, 64-bit, or 128-bit integers in the [min_val, max_val) range.
+template<typename IntType, class RNG>
+// When entire range is specified for signed IntType, result of max_val - min_val is -1
+// which when converted to unsigned type correctly represents the max value for the
+// same unsigned IntType. Variants of Uniform() in RNG work with unsigned types.
+// Hence the ASAN suppression.
+ATTRIBUTE_NO_SANITIZE_INTEGER
+std::vector<IntType> CreateRandomIntegersInRange(const int num_values,
+                                                 IntType min_val,
+                                                 IntType max_val,
+                                                 RNG* rand) {
+  // type_traits not defined for 128-bit int data types, hence not checking for integral value.
+  static_assert(sizeof(IntType) == 4 || sizeof(IntType) == 8 || sizeof(IntType) == 16,
+                "Only 32-bit, 64-bit, or 128-bit integers generated");
+  CHECK_LT(min_val, max_val);
+
+  std::vector<IntType> result(num_values);
+  for (auto& v : result) {
+    v = min_val + GetNextUniformRandom(max_val - min_val, rand);
+  }
+  return result;
+}
+
+
 } // namespace kudu