You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kudu.apache.org by mp...@apache.org on 2017/12/12 23:56:55 UTC

[1/3] kudu git commit: Add initial internal INT128/__int128 support

Repository: kudu
Updated Branches:
  refs/heads/master 2a6de8ea4 -> 887443e0d


Add initial internal INT128/__int128 support

This patch adds internal support for INT128/__int128 in Kudu.
This preliminary functionality is required to support the DECIMAL
type in KUDU-721.

Additionally this patch adds user defined numberic literals
_i128 and _u128 to allow for literals and constants to be used
until official compiler support is added.

Change-Id: I36458a54dfdd28be03f80d83688c0d658944e8e1
Reviewed-on: http://gerrit.cloudera.org:8080/8533
Tested-by: Kudu Jenkins
Reviewed-by: Dan Burkert <da...@apache.org>


Project: http://git-wip-us.apache.org/repos/asf/kudu/repo
Commit: http://git-wip-us.apache.org/repos/asf/kudu/commit/22d5fbdb
Tree: http://git-wip-us.apache.org/repos/asf/kudu/tree/22d5fbdb
Diff: http://git-wip-us.apache.org/repos/asf/kudu/diff/22d5fbdb

Branch: refs/heads/master
Commit: 22d5fbdb2bfc393ab78f624e624dae2ee54c2017
Parents: 2a6de8e
Author: Grant Henke <gr...@gmail.com>
Authored: Mon Nov 13 12:25:27 2017 -0600
Committer: Grant Henke <gr...@gmail.com>
Committed: Tue Dec 12 19:29:45 2017 +0000

----------------------------------------------------------------------
 src/kudu/cfile/bshuf_block.h           |   1 +
 src/kudu/cfile/cfile-test-base.h       |  16 ++++
 src/kudu/cfile/encoding-test.cc        |   6 ++
 src/kudu/cfile/type_encodings.cc       |   4 +
 src/kudu/common/common.proto           |   1 +
 src/kudu/common/types.cc               |   1 +
 src/kudu/common/types.h                |  30 +++++++
 src/kudu/gutil/CMakeLists.txt          |   1 +
 src/kudu/gutil/endian.h                |  14 ++++
 src/kudu/gutil/integral_types.h        |   1 +
 src/kudu/gutil/mathlimits.cc           |   1 +
 src/kudu/gutil/mathlimits.h            |   2 +
 src/kudu/gutil/strings/numbers-test.cc |  45 +++++++++++
 src/kudu/gutil/strings/numbers.cc      |  29 +++++++
 src/kudu/gutil/strings/numbers.h       |  14 ++++
 src/kudu/gutil/strings/substitute.h    |   6 ++
 src/kudu/gutil/type_traits.h           |   2 +
 src/kudu/util/CMakeLists.txt           |   2 +
 src/kudu/util/int128-test.cc           |  91 +++++++++++++++++++++
 src/kudu/util/int128.cc                |  36 +++++++++
 src/kudu/util/int128.h                 | 120 ++++++++++++++++++++++++++++
 21 files changed, 423 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/22d5fbdb/src/kudu/cfile/bshuf_block.h
----------------------------------------------------------------------
diff --git a/src/kudu/cfile/bshuf_block.h b/src/kudu/cfile/bshuf_block.h
index ea6df42..11442eb 100644
--- a/src/kudu/cfile/bshuf_block.h
+++ b/src/kudu/cfile/bshuf_block.h
@@ -265,6 +265,7 @@ class BShufBlockDecoder final : public BlockDecoder {
       case 2:
       case 4:
       case 8:
+      case 16:
         break;
       default:
         return Status::Corruption(strings::Substitute("invalid size_of_elem: $0", size_of_elem_));

http://git-wip-us.apache.org/repos/asf/kudu/blob/22d5fbdb/src/kudu/cfile/cfile-test-base.h
----------------------------------------------------------------------
diff --git a/src/kudu/cfile/cfile-test-base.h b/src/kudu/cfile/cfile-test-base.h
index f7d97c5..f282b1e 100644
--- a/src/kudu/cfile/cfile-test-base.h
+++ b/src/kudu/cfile/cfile-test-base.h
@@ -31,6 +31,7 @@
 #include "kudu/common/columnblock.h"
 #include "kudu/fs/fs_manager.h"
 #include "kudu/gutil/stringprintf.h"
+#include "kudu/util/int128.h"
 #include "kudu/util/test_macros.h"
 #include "kudu/util/test_util.h"
 #include "kudu/util/stopwatch.h"
@@ -228,6 +229,16 @@ class Int64DataGenerator : public DataGenerator<INT64, HAS_NULLS> {
   }
 };
 
+template<bool HAS_NULLS>
+class Int128DataGenerator : public DataGenerator<INT128, HAS_NULLS> {
+public:
+  Int128DataGenerator() {}
+  int128_t BuildTestValue(size_t /*block_index*/, size_t value) OVERRIDE {
+    int128_t r = (value * 0x123456789abcdefULL) & 0x7fffffffffffffffULL;
+    return value % 2 == 0 ? r : -r;
+  }
+};
+
 // Floating-point data generator.
 // This works for both floats and doubles.
 template<DataType DATA_TYPE, bool HAS_NULLS>
@@ -510,6 +521,11 @@ void TimeReadFile(FsManager* fs_manager, const BlockId& block_id, size_t *count_
       TimeReadFileForDataType<INT64, uint64_t>(iter, count);
       break;
     }
+    case INT128:
+    {
+      TimeReadFileForDataType<INT128, int128_t>(iter, count);
+      break;
+    }
     case FLOAT:
     {
       TimeReadFileForDataType<FLOAT, float>(iter, count);

http://git-wip-us.apache.org/repos/asf/kudu/blob/22d5fbdb/src/kudu/cfile/encoding-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/cfile/encoding-test.cc b/src/kudu/cfile/encoding-test.cc
index 3a05c0c..d8691ea 100644
--- a/src/kudu/cfile/encoding-test.cc
+++ b/src/kudu/cfile/encoding-test.cc
@@ -877,6 +877,8 @@ TYPED_TEST(IntEncodingTest, TestSeekAllTypes) {
   this->template DoIntSeekTest<INT32>(10000, 1000, true);
   this->template DoIntSeekTest<UINT64>(10000, 1000, true);
   this->template DoIntSeekTest<INT64>(10000, 1000, true);
+  // TODO: Uncomment when adding 128 bit support to RLE
+  // this->template DoIntSeekTest<INT128>();
 }
 
 TYPED_TEST(IntEncodingTest, IntSeekTestTinyBlockAllTypes) {
@@ -888,6 +890,8 @@ TYPED_TEST(IntEncodingTest, IntSeekTestTinyBlockAllTypes) {
   this->template DoIntSeekTestTinyBlock<INT32>();
   this->template DoIntSeekTestTinyBlock<UINT64>();
   this->template DoIntSeekTestTinyBlock<INT64>();
+  // TODO: Uncomment when adding 128 bit support to RLE
+  // this->template DoIntSeekTestTinyBlock<INT128>();
 }
 
 TYPED_TEST(IntEncodingTest, TestRoundTrip) {
@@ -899,6 +903,8 @@ TYPED_TEST(IntEncodingTest, TestRoundTrip) {
   this->template DoIntRoundTripTest<INT32>();
   this->template DoIntRoundTripTest<UINT64>();
   this->template DoIntRoundTripTest<INT64>();
+  // TODO: Uncomment when adding 128 bit support to RLE
+  // this->template DoIntRoundTripTest<INT128>();
 }
 
 #ifdef NDEBUG

http://git-wip-us.apache.org/repos/asf/kudu/blob/22d5fbdb/src/kudu/cfile/type_encodings.cc
----------------------------------------------------------------------
diff --git a/src/kudu/cfile/type_encodings.cc b/src/kudu/cfile/type_encodings.cc
index 3485192..d184ea6 100644
--- a/src/kudu/cfile/type_encodings.cc
+++ b/src/kudu/cfile/type_encodings.cc
@@ -274,6 +274,10 @@ class TypeEncodingResolver {
     AddMapping<BINARY, PREFIX_ENCODING>();
     AddMapping<BOOL, RLE>();
     AddMapping<BOOL, PLAIN_ENCODING>();
+    AddMapping<INT128, BIT_SHUFFLE>();
+    AddMapping<INT128, PLAIN_ENCODING>();
+    // TODO: Add 128 bit support to RLE
+    // AddMapping<INT128, RLE>();
   }
 
   template<DataType type, EncodingType encoding> void AddMapping() {

http://git-wip-us.apache.org/repos/asf/kudu/blob/22d5fbdb/src/kudu/common/common.proto
----------------------------------------------------------------------
diff --git a/src/kudu/common/common.proto b/src/kudu/common/common.proto
index c156ecf..faa4d58 100644
--- a/src/kudu/common/common.proto
+++ b/src/kudu/common/common.proto
@@ -50,6 +50,7 @@ enum DataType {
   DOUBLE = 11;
   BINARY = 12;
   UNIXTIME_MICROS = 13;
+  INT128 = 14;
 }
 
 enum EncodingType {

http://git-wip-us.apache.org/repos/asf/kudu/blob/22d5fbdb/src/kudu/common/types.cc
----------------------------------------------------------------------
diff --git a/src/kudu/common/types.cc b/src/kudu/common/types.cc
index 7717752..0a4052b 100644
--- a/src/kudu/common/types.cc
+++ b/src/kudu/common/types.cc
@@ -84,6 +84,7 @@ class TypeInfoResolver {
     AddMapping<FLOAT>();
     AddMapping<DOUBLE>();
     AddMapping<BINARY>();
+    AddMapping<INT128>();
   }
 
   template<DataType type> void AddMapping() {

http://git-wip-us.apache.org/repos/asf/kudu/blob/22d5fbdb/src/kudu/common/types.h
----------------------------------------------------------------------
diff --git a/src/kudu/common/types.h b/src/kudu/common/types.h
index b7cab77..7e9ac97 100644
--- a/src/kudu/common/types.h
+++ b/src/kudu/common/types.h
@@ -35,6 +35,7 @@
 #include "kudu/gutil/mathlimits.h"
 #include "kudu/gutil/strings/escaping.h"
 #include "kudu/gutil/strings/numbers.h"
+#include "kudu/util/int128.h"
 #include "kudu/util/make_shared.h"
 #include "kudu/util/slice.h"
 // IWYU pragma: no_include "kudu/util/status.h"
@@ -324,6 +325,30 @@ struct DataTypeTraits<INT64> {
 };
 
 template<>
+struct DataTypeTraits<INT128> {
+  static const DataType physical_type = INT128;
+  typedef int128_t cpp_type;
+  static const char *name() {
+    return "int128";
+  }
+  static void AppendDebugStringForValue(const void *val, std::string *str) {
+    str->append(SimpleItoa(*reinterpret_cast<const int128_t *>(val)));
+  }
+  static int Compare(const void *lhs, const void *rhs) {
+    return GenericCompare<INT128>(lhs, rhs);
+  }
+  static bool AreConsecutive(const void* a, const void* b) {
+    return AreIntegersConsecutive<INT128>(a, b);
+  }
+  static const cpp_type* min_value() {
+    return &MathLimits<cpp_type>::kMin;
+  }
+  static const cpp_type* max_value() {
+    return &MathLimits<cpp_type>::kMin;
+  }
+};
+
+template<>
 struct DataTypeTraits<FLOAT> {
   static const DataType physical_type = FLOAT;
   typedef float cpp_type;
@@ -578,6 +603,9 @@ class Variant {
       case UINT64:
         numeric_.u64 = *static_cast<const uint64_t *>(value);
         break;
+      case INT128:
+        numeric_.i128 = *static_cast<const int128_t *>(value);
+        break;
       case FLOAT:
         numeric_.float_val = *static_cast<const float *>(value);
         break;
@@ -641,6 +669,7 @@ class Variant {
       case INT32:        return &(numeric_.i32);
       case UINT32:       return &(numeric_.u32);
       case INT64:        return &(numeric_.i64);
+      case INT128:       return &(numeric_.i128);
       case UNIXTIME_MICROS:    return &(numeric_.i64);
       case UINT64:       return &(numeric_.u64);
       case FLOAT:        return (&numeric_.float_val);
@@ -682,6 +711,7 @@ class Variant {
     uint32_t u32;
     int64_t  i64;
     uint64_t u64;
+    int128_t i128;
     float    float_val;
     double   double_val;
   };

http://git-wip-us.apache.org/repos/asf/kudu/blob/22d5fbdb/src/kudu/gutil/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/src/kudu/gutil/CMakeLists.txt b/src/kudu/gutil/CMakeLists.txt
index 1c61267..3d487eb 100644
--- a/src/kudu/gutil/CMakeLists.txt
+++ b/src/kudu/gutil/CMakeLists.txt
@@ -69,3 +69,4 @@ ADD_EXPORTABLE_LIBRARY(gutil
   COMPILE_FLAGS "-funsigned-char -Wno-char-subscripts")
 
 add_kudu_test(strings/string_util-test)
+add_kudu_test(strings/numbers-test)

http://git-wip-us.apache.org/repos/asf/kudu/blob/22d5fbdb/src/kudu/gutil/endian.h
----------------------------------------------------------------------
diff --git a/src/kudu/gutil/endian.h b/src/kudu/gutil/endian.h
index db06410..bffe915 100644
--- a/src/kudu/gutil/endian.h
+++ b/src/kudu/gutil/endian.h
@@ -54,6 +54,11 @@ inline uint64 gbswap_64(uint64 host_int) {
 #endif  // bswap_64
 }
 
+inline unsigned __int128 gbswap_128(unsigned __int128 host_int) {
+  return static_cast<unsigned __int128>(bswap_64(static_cast<uint64>(host_int >> 64))) |
+         (static_cast<unsigned __int128>(bswap_64(static_cast<uint64>(host_int))) << 64);
+}
+
 #ifdef IS_LITTLE_ENDIAN
 
 // Definitions for ntohl etc. that don't require us to include
@@ -109,6 +114,9 @@ class LittleEndian {
   static uint64 FromHost64(uint64 x) { return x; }
   static uint64 ToHost64(uint64 x) { return x; }
 
+  static unsigned __int128 FromHost128(unsigned __int128 x) { return x; }
+  static unsigned __int128 ToHost128(unsigned __int128 x) { return x; }
+
   static bool IsLittleEndian() { return true; }
 
 #elif defined IS_BIG_ENDIAN
@@ -240,6 +248,9 @@ class BigEndian {
   static uint64 FromHost64(uint64 x) { return gbswap_64(x); }
   static uint64 ToHost64(uint64 x) { return gbswap_64(x); }
 
+  static unsigned __int128 FromHost128(unsigned __int128 x) { return gbswap_128(x); }
+  static unsigned __int128 ToHost128(unsigned __int128 x) { return gbswap_128(x); }
+
   static bool IsLittleEndian() { return true; }
 
 #elif defined IS_BIG_ENDIAN
@@ -253,6 +264,9 @@ class BigEndian {
   static uint64 FromHost64(uint64 x) { return x; }
   static uint64 ToHost64(uint64 x) { return x; }
 
+  static uint128 FromHost128(uint128 x) { return x; }
+  static uint128 ToHost128(uint128 x) { return x; }
+
   static bool IsLittleEndian() { return false; }
 
 #endif /* ENDIAN */

http://git-wip-us.apache.org/repos/asf/kudu/blob/22d5fbdb/src/kudu/gutil/integral_types.h
----------------------------------------------------------------------
diff --git a/src/kudu/gutil/integral_types.h b/src/kudu/gutil/integral_types.h
index cbcf917..e84a42e 100644
--- a/src/kudu/gutil/integral_types.h
+++ b/src/kudu/gutil/integral_types.h
@@ -29,6 +29,7 @@ typedef __int64             int64;
 #else
 typedef int64_t             int64;
 #endif /* _MSC_VER */
+typedef __int128            int128;
 
 // NOTE: unsigned types are DANGEROUS in loops and other arithmetical
 // places.  Use the signed types unless your variable represents a bit

http://git-wip-us.apache.org/repos/asf/kudu/blob/22d5fbdb/src/kudu/gutil/mathlimits.cc
----------------------------------------------------------------------
diff --git a/src/kudu/gutil/mathlimits.cc b/src/kudu/gutil/mathlimits.cc
index dcc261d..b0a64c3 100644
--- a/src/kudu/gutil/mathlimits.cc
+++ b/src/kudu/gutil/mathlimits.cc
@@ -107,6 +107,7 @@ DEF_SIGNED_INT_LIMITS(int8)
 DEF_SIGNED_INT_LIMITS(int16)
 DEF_SIGNED_INT_LIMITS(int32)
 DEF_SIGNED_INT_LIMITS(int64)
+DEF_SIGNED_INT_LIMITS(int128)
 DEF_UNSIGNED_INT_LIMITS(uint8)
 DEF_UNSIGNED_INT_LIMITS(uint16)
 DEF_UNSIGNED_INT_LIMITS(uint32)

http://git-wip-us.apache.org/repos/asf/kudu/blob/22d5fbdb/src/kudu/gutil/mathlimits.h
----------------------------------------------------------------------
diff --git a/src/kudu/gutil/mathlimits.h b/src/kudu/gutil/mathlimits.h
index 6b09aa0..77847b8 100644
--- a/src/kudu/gutil/mathlimits.h
+++ b/src/kudu/gutil/mathlimits.h
@@ -182,11 +182,13 @@ DECL_SIGNED_INT_LIMITS(signed short int, unsigned short int)
 DECL_SIGNED_INT_LIMITS(signed int, unsigned int)
 DECL_SIGNED_INT_LIMITS(signed long int, unsigned long int)
 DECL_SIGNED_INT_LIMITS(signed long long int, unsigned long long int)
+DECL_SIGNED_INT_LIMITS(signed __int128, unsigned __int128)
 DECL_UNSIGNED_INT_LIMITS(unsigned char)
 DECL_UNSIGNED_INT_LIMITS(unsigned short int)
 DECL_UNSIGNED_INT_LIMITS(unsigned int)
 DECL_UNSIGNED_INT_LIMITS(unsigned long int)
 DECL_UNSIGNED_INT_LIMITS(unsigned long long int)
+DECL_UNSIGNED_INT_LIMITS(unsigned __int128)
 
 #undef DECL_SIGNED_INT_LIMITS
 #undef DECL_UNSIGNED_INT_LIMITS

http://git-wip-us.apache.org/repos/asf/kudu/blob/22d5fbdb/src/kudu/gutil/strings/numbers-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/gutil/strings/numbers-test.cc b/src/kudu/gutil/strings/numbers-test.cc
new file mode 100644
index 0000000..266a9f2
--- /dev/null
+++ b/src/kudu/gutil/strings/numbers-test.cc
@@ -0,0 +1,45 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <string>
+
+#include <gtest/gtest.h>
+
+#include "kudu/gutil/strings/numbers.h"
+#include "kudu/util/int128.h"
+
+namespace kudu {
+
+TEST(TestNumbers, FastInt128ToBufferLeft) {
+  char buf[64];
+  std::string maxStr = std::string(buf, FastInt128ToBufferLeft(INT128_MAX, buf));
+  ASSERT_EQ("170141183460469231731687303715884105727", maxStr);
+
+  char buf2[64];
+  std::string minStr = std::string(buf2, FastInt128ToBufferLeft(INT128_MIN, buf2));
+  ASSERT_EQ("-170141183460469231731687303715884105728", minStr);
+
+  char buf3[64];
+  std::string shortStr = std::string(buf3, FastInt128ToBufferLeft(INT128_MIN / 10, buf3));
+  ASSERT_EQ("-17014118346046923173168730371588410572", shortStr);
+
+  char buf4[64];
+  std::string shorterStr = std::string(buf4, FastInt128ToBufferLeft(INT128_MIN / 100000, buf4));
+  ASSERT_EQ("-1701411834604692317316873037158841", shorterStr);
+}
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/kudu/blob/22d5fbdb/src/kudu/gutil/strings/numbers.cc
----------------------------------------------------------------------
diff --git a/src/kudu/gutil/strings/numbers.cc b/src/kudu/gutil/strings/numbers.cc
index 33b13ee..528d58b 100644
--- a/src/kudu/gutil/strings/numbers.cc
+++ b/src/kudu/gutil/strings/numbers.cc
@@ -1073,6 +1073,35 @@ char* FastInt64ToBufferLeft(int64 i, char* buffer) {
   return FastUInt64ToBufferLeft(u, buffer);
 }
 
+char* FastUInt128ToBufferLeft(unsigned __int128 i, char* buffer) {
+  static const unsigned __int128 TWENTY_DIGITS =
+      static_cast<unsigned __int128>(10000000000) * static_cast<unsigned __int128>(10000000000);
+
+  uint64 u = static_cast<uint64>(i);
+  if (u == i) return FastUInt64ToBufferLeft(u, buffer);
+
+  unsigned __int128 top_19_digits = i / TWENTY_DIGITS;
+  buffer = FastUInt64ToBufferLeft(top_19_digits, buffer);
+  unsigned __int128 rem128 = i - (top_19_digits * TWENTY_DIGITS);
+
+  unsigned __int128 middle_19_digits = rem128 / 10;
+  buffer = FastUInt64ToBufferLeft(middle_19_digits, buffer);
+  u = rem128 - (middle_19_digits * 10);
+
+  buffer = FastUInt32ToBufferLeft(u, buffer);
+
+  return buffer;
+}
+
+char* FastInt128ToBufferLeft(__int128 i, char* buffer) {
+  unsigned __int128 u = i;
+  if (i < 0) {
+    *buffer++ = '-';
+    u = static_cast<unsigned __int128>(0) - u;
+  }
+  return FastUInt128ToBufferLeft(u, buffer);
+}
+
 int HexDigitsPrefix(const char* buf, int num_digits) {
   for (int i = 0; i < num_digits; i++)
     if (!ascii_isxdigit(buf[i]))

http://git-wip-us.apache.org/repos/asf/kudu/blob/22d5fbdb/src/kudu/gutil/strings/numbers.h
----------------------------------------------------------------------
diff --git a/src/kudu/gutil/strings/numbers.h b/src/kudu/gutil/strings/numbers.h
index 0152def..f858891 100644
--- a/src/kudu/gutil/strings/numbers.h
+++ b/src/kudu/gutil/strings/numbers.h
@@ -149,6 +149,8 @@ inline char* FastUIntToBuffer(unsigned int i, char* buffer) {
 // FastUInt32ToBufferLeft()
 // FastInt64ToBufferLeft()
 // FastUInt64ToBufferLeft()
+// FastInt128ToBufferLeft()
+// FastUInt128ToBufferLeft()
 //
 // Like the Fast*ToBuffer() functions above, these are intended for speed.
 // Unlike the Fast*ToBuffer() functions, however, these functions write
@@ -164,6 +166,8 @@ char* FastInt32ToBufferLeft(int32 i, char* buffer);    // at least 12 bytes
 char* FastUInt32ToBufferLeft(uint32 i, char* buffer);    // at least 12 bytes
 char* FastInt64ToBufferLeft(int64 i, char* buffer);    // at least 22 bytes
 char* FastUInt64ToBufferLeft(uint64 i, char* buffer);    // at least 22 bytes
+char* FastInt128ToBufferLeft(__int128 i, char* buffer);
+char* FastUInt128ToBufferLeft(unsigned __int128 i, char* buffer);
 
 // Just define these in terms of the above.
 inline char* FastUInt32ToBuffer(uint32 i, char* buffer) {
@@ -385,6 +389,16 @@ inline std::string SimpleItoa(uint64 i) {
   return std::string(buf, FastUInt64ToBufferLeft(i, buf));
 }
 
+inline std::string SimpleItoa(__int128 i) {
+  char buf[64];  // Longest is -170141183460469231731687303715884105728
+  return std::string(buf, FastInt128ToBufferLeft(i, buf));
+}
+
+inline std::string SimpleItoa(unsigned __int128 i) {
+  char buf[64];  // Longest is 340282366920938463463374607431768211455
+  return std::string(buf, FastUInt128ToBufferLeft(i, buf));
+}
+
 // SimpleAtoi converts a string to an integer.
 // Uses safe_strto?() for actual parsing, so strict checking is
 // applied, which is to say, the string must be a base-10 integer, optionally

http://git-wip-us.apache.org/repos/asf/kudu/blob/22d5fbdb/src/kudu/gutil/strings/substitute.h
----------------------------------------------------------------------
diff --git a/src/kudu/gutil/strings/substitute.h b/src/kudu/gutil/strings/substitute.h
index 9def694..cb2c818 100644
--- a/src/kudu/gutil/strings/substitute.h
+++ b/src/kudu/gutil/strings/substitute.h
@@ -113,6 +113,12 @@ class SubstituteArg {
   inline SubstituteArg(unsigned long long value)  // NOLINT(runtime/explicit)
     : text_(scratch_),
       size_(FastUInt64ToBufferLeft(value, scratch_) - scratch_) {}
+  inline SubstituteArg(__int128 value)  // NOLINT(runtime/explicit)
+      : text_(scratch_),
+        size_(FastInt64ToBufferLeft(value, scratch_) - scratch_) {}
+  inline SubstituteArg(unsigned __int128 value)  // NOLINT(runtime/explicit)
+      : text_(scratch_),
+        size_(FastUInt64ToBufferLeft(value, scratch_) - scratch_) {}
   inline SubstituteArg(float value)  // NOLINT(runtime/explicit)
     : text_(FloatToBuffer(value, scratch_)), size_(strlen(text_)) {}
   inline SubstituteArg(double value)  // NOLINT(runtime/explicit)

http://git-wip-us.apache.org/repos/asf/kudu/blob/22d5fbdb/src/kudu/gutil/type_traits.h
----------------------------------------------------------------------
diff --git a/src/kudu/gutil/type_traits.h b/src/kudu/gutil/type_traits.h
index 2f72028..20cd90a 100644
--- a/src/kudu/gutil/type_traits.h
+++ b/src/kudu/gutil/type_traits.h
@@ -128,6 +128,8 @@ template<> struct is_integral<unsigned long> : true_type { };
 template<> struct is_integral<long long> : true_type { };
 template<> struct is_integral<unsigned long long> : true_type { };
 #endif
+template<> struct is_integral<__int128> : true_type { };
+template<> struct is_integral<unsigned __int128> : true_type { };
 template <class T> struct is_integral<const T> : is_integral<T> { };
 template <class T> struct is_integral<volatile T> : is_integral<T> { };
 template <class T> struct is_integral<const volatile T> : is_integral<T> { };

http://git-wip-us.apache.org/repos/asf/kudu/blob/22d5fbdb/src/kudu/util/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/src/kudu/util/CMakeLists.txt b/src/kudu/util/CMakeLists.txt
index a6e0497..b107b73 100644
--- a/src/kudu/util/CMakeLists.txt
+++ b/src/kudu/util/CMakeLists.txt
@@ -145,6 +145,7 @@ set(UTIL_SRCS
   hdr_histogram.cc
   hexdump.cc
   init.cc
+  int128.cc
   jsonreader.cc
   jsonwriter.cc
   kernel_stack_watchdog.cc
@@ -351,6 +352,7 @@ ADD_KUDU_TEST(flags-test)
 ADD_KUDU_TEST(group_varint-test)
 ADD_KUDU_TEST(hash_util-test)
 ADD_KUDU_TEST(hdr_histogram-test)
+ADD_KUDU_TEST(int128-test)
 ADD_KUDU_TEST(inline_slice-test)
 ADD_KUDU_TEST(interval_tree-test)
 ADD_KUDU_TEST(jsonreader-test)

http://git-wip-us.apache.org/repos/asf/kudu/blob/22d5fbdb/src/kudu/util/int128-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/util/int128-test.cc b/src/kudu/util/int128-test.cc
new file mode 100644
index 0000000..175c4e4
--- /dev/null
+++ b/src/kudu/util/int128-test.cc
@@ -0,0 +1,91 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <cstddef>
+#include <cstdint>
+#include <iosfwd>
+#include <string>
+
+#include <gtest/gtest.h>
+
+#include "kudu/gutil/macros.h"
+#include "kudu/util/int128.h"
+
+using std::string;
+
+namespace kudu {
+
+TEST(TestInt128, TestOstreamSigned) {
+  int128_t INTEGERS[] = {0, -1, 1, -1234567890,
+                         -12345678901011121314151617181920212223_i128,
+                         INT64_MIN, UINT64_MAX,
+                         INT128_MIN,
+                         INT128_MAX};
+  std::string STRINGS[] = {"0", "-1", "1", "-1234567890",
+                           "-12345678901011121314151617181920212223",
+                           "-9223372036854775808", "18446744073709551615",
+                           "-170141183460469231731687303715884105728",
+                           "170141183460469231731687303715884105727"};
+  for (size_t i = 0; i < arraysize(INTEGERS); i++) {
+    std::ostringstream ss;
+    ss << INTEGERS[i];
+    ASSERT_EQ(STRINGS[i], ss.str());
+  }
+}
+
+TEST(TestInt128, TestOstreamUnsigned) {
+  uint128_t INTEGERS[] = {0, 1, 1234567890, 123456789101112131415161718192021222324_u128,
+                          UINT128_MIN, UINT128_MAX};
+  string STRINGS[] = {"0", "1", "1234567890", "123456789101112131415161718192021222324",
+                      "0", "340282366920938463463374607431768211455"};
+  for (size_t i = 0; i < arraysize(INTEGERS); i++) {
+    std::ostringstream ss;
+    ss << INTEGERS[i];
+    ASSERT_EQ(STRINGS[i], ss.str());
+  }
+}
+
+TEST(TestInt128, TestCasting) {
+  uint128_t mathToMax = (static_cast<uint128_t>(INT128_MAX) * 2) + 1;
+  ASSERT_EQ(UINT128_MAX, mathToMax);
+
+  uint128_t castToMax = static_cast<uint128_t>(-1);
+  ASSERT_EQ(UINT128_MAX, castToMax);
+}
+
+TEST(TestInt128, TestSuffix) {
+  int128_t imax = 170141183460469231731687303715884105727_i128;
+  ASSERT_EQ(INT128_MAX, imax);
+
+  // Note: We can't represent the absolute minimum because numeric literals
+  // never represent negative numbers and the - operator is applied after
+  // the conversion to an int128_t.
+  int128_t imin = -170141183460469231731687303715884105727_i128;
+  ASSERT_EQ(INT128_MIN + 1, imin);
+
+  uint128_t umax = 340282366920938463463374607431768211455_u128;
+  ASSERT_EQ(UINT128_MAX, umax);
+
+  uint128_t umin = 0_u128;
+  ASSERT_EQ(UINT128_MIN, umin);
+
+  // The values below will fail to compile
+  // 170141183460469231731687303715884105728_i128; // Too large int128_t
+  // 3340282366920938463463374607431768211456_u128; // Too large uint128_t
+}
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/kudu/blob/22d5fbdb/src/kudu/util/int128.cc
----------------------------------------------------------------------
diff --git a/src/kudu/util/int128.cc b/src/kudu/util/int128.cc
new file mode 100644
index 0000000..cbd0621
--- /dev/null
+++ b/src/kudu/util/int128.cc
@@ -0,0 +1,36 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "kudu/util/int128.h"
+
+#include <string>
+
+#include "kudu/gutil/strings/numbers.h"
+
+namespace std {
+
+std::ostream& operator<<(std::ostream& os, const __int128& val) {
+  os << SimpleItoa(val);
+  return os;
+}
+
+std::ostream& operator<<(std::ostream& os, const unsigned __int128& val) {
+  os << SimpleItoa(val);
+  return os;
+}
+
+} // namespace std

http://git-wip-us.apache.org/repos/asf/kudu/blob/22d5fbdb/src/kudu/util/int128.h
----------------------------------------------------------------------
diff --git a/src/kudu/util/int128.h b/src/kudu/util/int128.h
new file mode 100644
index 0000000..e78e450
--- /dev/null
+++ b/src/kudu/util/int128.h
@@ -0,0 +1,120 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+// This file is the central location for defining the int128 type
+// used by Kudu. Though this file is small it ensures flexibility
+// as choices and standards around int128 change.
+#pragma once
+
+#include <iostream>
+
+#include "kudu/gutil/mathlimits.h"
+
+namespace kudu {
+
+typedef unsigned __int128 uint128_t;
+typedef signed __int128 int128_t;
+
+// Note: We don't use numeric_limits because it can give incorrect
+// values for __int128 and unsigned __int128.
+static const uint128_t UINT128_MIN = MathLimits<uint128_t>::kMin;
+static const uint128_t UINT128_MAX = MathLimits<uint128_t>::kMax;
+static const int128_t INT128_MIN = MathLimits<int128_t>::kMin;
+static const int128_t INT128_MAX = MathLimits<int128_t>::kMax;
+
+namespace int128Suffix {
+
+// Convert the characters 0 through 9 to their int value.
+constexpr uint128_t CharValue(char c) {
+  return c - '0';
+}
+
+// Terminate the recursion.
+template <uint128_t VALUE>
+constexpr uint128_t ValidateU128Helper() {
+  return true;
+}
+
+// Recurse the literal from left to right and validate the input.
+// Return true if the input is valid.
+template <uint128_t VALUE, char C, char... CS>
+constexpr bool ValidateU128Helper() {
+  return (VALUE <= UINT128_MAX / 10) &&
+         VALUE * 10 <= UINT128_MAX - CharValue(C) &&
+         ValidateU128Helper<VALUE * 10 + CharValue(C), CS...>();
+}
+
+template <char... CS>
+constexpr bool ValidateU128() {
+  return ValidateU128Helper<0, CS...>();
+}
+
+// Terminate the recursion.
+template <uint128_t VALUE>
+constexpr uint128_t MakeU128Helper() {
+  return VALUE;
+}
+
+// Recurse the literal from left to right calculating the resulting value.
+// C is the current left most character, CS is the rest.
+template <uint128_t VALUE, char C, char... CS>
+constexpr uint128_t MakeU128Helper() {
+  return MakeU128Helper<VALUE * 10 + CharValue(C), CS...>();
+}
+
+template <char... CS>
+constexpr uint128_t MakeU128() {
+  return MakeU128Helper<0, CS...>();
+}
+
+template <char... CS>
+constexpr bool ValidateI128() {
+  return ValidateU128Helper<0, CS...>() && MakeU128<CS...>() <= INT128_MAX;
+}
+
+}  // namespace int128Suffix
+
+// User-defined literal "_u128" to support basic integer constants
+// for uint128_t until officially supported by compilers.
+template <char... CS>
+constexpr uint128_t operator"" _u128() {
+  static_assert(int128Suffix::ValidateU128<CS...>(),
+                "integer literal is too large to be represented in "
+                    "uint128_t integer type");
+  return int128Suffix::MakeU128<CS...>();
+}
+
+// User-defined literal "_u128" to support basic integer constants
+// for int128_t until officially supported by compilers.
+template <char... CS>
+constexpr int128_t operator"" _i128() {
+  static_assert(int128Suffix::ValidateI128<CS...>(),
+                "integer literal is too large to be represented in "
+                    "int128_t integer type");
+  return static_cast<int128_t>(int128Suffix::MakeU128<CS...>());
+}
+
+} // namespace kudu
+
+namespace std {
+
+  // Support the << operator on int128_t and uint128_t types.
+  // We use __int128 here because these are not in the Kudu namespace.
+  std::ostream& operator<<(std::ostream& os, const __int128& val);
+  std::ostream& operator<<(std::ostream& os, const unsigned __int128& val);
+
+} // namespace std


[3/3] kudu git commit: [quorum_util] fix build

Posted by mp...@apache.org.
[quorum_util] fix build

This is a follow-up for 83b19ef320dba9a85a86845dc515587ce28c9ac3.
It seems the previous patch needed rebase prior to pushing.

Change-Id: Ided12a63f0cae539fa2dd690a44c29423ad7ff73
Reviewed-on: http://gerrit.cloudera.org:8080/8821
Reviewed-by: Mike Percy <mp...@apache.org>
Tested-by: Alexey Serbin <as...@cloudera.com>


Project: http://git-wip-us.apache.org/repos/asf/kudu/repo
Commit: http://git-wip-us.apache.org/repos/asf/kudu/commit/887443e0
Tree: http://git-wip-us.apache.org/repos/asf/kudu/tree/887443e0
Diff: http://git-wip-us.apache.org/repos/asf/kudu/diff/887443e0

Branch: refs/heads/master
Commit: 887443e0d44d462f7a57507917a4a77b3495c4c7
Parents: 83b19ef
Author: Alexey Serbin <as...@cloudera.com>
Authored: Tue Dec 12 15:02:27 2017 -0800
Committer: Alexey Serbin <as...@cloudera.com>
Committed: Tue Dec 12 23:11:23 2017 +0000

----------------------------------------------------------------------
 src/kudu/consensus/quorum_util-test.cc | 20 ++++++++++----------
 1 file changed, 10 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/887443e0/src/kudu/consensus/quorum_util-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/consensus/quorum_util-test.cc b/src/kudu/consensus/quorum_util-test.cc
index 2014d71..3d23bcb 100644
--- a/src/kudu/consensus/quorum_util-test.cc
+++ b/src/kudu/consensus/quorum_util-test.cc
@@ -847,13 +847,13 @@ TEST(QuorumUtilTest, ReplaceAttributeBasic) {
     AddPeer(&config, "C", V, '+', {{"REPLACE", true}});
     AddPeer(&config, "D", V, replica_health, {{"REPLACE", true}});
     string to_evict;
-    ASSERT_TRUE(CanEvictReplica(config, "A", 3, &to_evict));
+    ASSERT_TRUE(ShouldEvictReplica(config, "A", 3, &to_evict));
     if (replica_health == '+') {
       EXPECT_NE("A", to_evict);
     } else {
       EXPECT_EQ("D", to_evict);
     }
-    EXPECT_TRUE(IsUnderReplicated(config, 3));
+    EXPECT_TRUE(ShouldAddReplica(config, 3));
   }
   for (auto health_status : { '?', '-' }) {
     RaftConfigPB config;
@@ -865,9 +865,9 @@ TEST(QuorumUtilTest, ReplaceAttributeBasic) {
       SCOPED_TRACE(Substitute("health status '$0', leader $1",
                               health_status, leader_replica));
       string to_evict;
-      ASSERT_TRUE(CanEvictReplica(config, leader_replica, 3, &to_evict));
+      ASSERT_TRUE(ShouldEvictReplica(config, leader_replica, 3, &to_evict));
       EXPECT_EQ("A", to_evict);
-      EXPECT_TRUE(IsUnderReplicated(config, 3));
+      EXPECT_TRUE(ShouldAddReplica(config, 3));
     }
   }
   {
@@ -877,9 +877,9 @@ TEST(QuorumUtilTest, ReplaceAttributeBasic) {
     AddPeer(&config, "C", V, '+', {{"REPLACE", true}});
     AddPeer(&config, "D", V, '-');
     string to_evict;
-    ASSERT_TRUE(CanEvictReplica(config, "A", 3, &to_evict));
+    ASSERT_TRUE(ShouldEvictReplica(config, "A", 3, &to_evict));
     EXPECT_EQ("D", to_evict);
-    EXPECT_TRUE(IsUnderReplicated(config, 3));
+    EXPECT_TRUE(ShouldAddReplica(config, 3));
   }
   {
     RaftConfigPB config;
@@ -887,8 +887,8 @@ TEST(QuorumUtilTest, ReplaceAttributeBasic) {
     AddPeer(&config, "B", V, '+', {{"REPLACE", true}});
     AddPeer(&config, "C", V, '+', {{"REPLACE", true}});
     AddPeer(&config, "D", V, '?');
-    EXPECT_FALSE(CanEvictReplica(config, "B", 3));
-    EXPECT_TRUE(IsUnderReplicated(config, 3));
+    EXPECT_FALSE(ShouldEvictReplica(config, "B", 3));
+    EXPECT_TRUE(ShouldAddReplica(config, 3));
   }
   for (auto health_status : { '?', '-' }) {
     SCOPED_TRACE(Substitute("health status '$0'", health_status));
@@ -899,9 +899,9 @@ TEST(QuorumUtilTest, ReplaceAttributeBasic) {
     AddPeer(&config, "D", V, health_status, {{"REPLACE", true}});
     AddPeer(&config, "E", V, '+');
     string to_evict;
-    ASSERT_TRUE(CanEvictReplica(config, "A", 3, &to_evict));
+    ASSERT_TRUE(ShouldEvictReplica(config, "A", 3, &to_evict));
     EXPECT_EQ("D", to_evict);
-    EXPECT_TRUE(IsUnderReplicated(config, 3));
+    EXPECT_TRUE(ShouldAddReplica(config, 3));
   }
 }
 


[2/3] kudu git commit: [quorum_util] rename 3-4-3-related utility functions

Posted by mp...@apache.org.
[quorum_util] rename 3-4-3-related utility functions

Renaming a couple of 3-4-3-related utility functions to make the naming
more symmetric:

  * IsUnderReplicated() --> ShouldAddReplica()
  * CanEvictReplica() --> ShouldEvictReplica()

This changelist does not contain any functional modifications.

Change-Id: I2f6d27588f6ccfb5c889471107b2df969e239bea
Reviewed-on: http://gerrit.cloudera.org:8080/8806
Tested-by: Kudu Jenkins
Reviewed-by: Mike Percy <mp...@apache.org>


Project: http://git-wip-us.apache.org/repos/asf/kudu/repo
Commit: http://git-wip-us.apache.org/repos/asf/kudu/commit/83b19ef3
Tree: http://git-wip-us.apache.org/repos/asf/kudu/tree/83b19ef3
Diff: http://git-wip-us.apache.org/repos/asf/kudu/diff/83b19ef3

Branch: refs/heads/master
Commit: 83b19ef320dba9a85a86845dc515587ce28c9ac3
Parents: 22d5fbd
Author: Alexey Serbin <as...@cloudera.com>
Authored: Fri Dec 8 18:19:42 2017 -0800
Committer: Mike Percy <mp...@apache.org>
Committed: Tue Dec 12 21:14:25 2017 +0000

----------------------------------------------------------------------
 src/kudu/consensus/quorum_util-test.cc | 355 ++++++++++++++--------------
 src/kudu/consensus/quorum_util.cc      |   4 +-
 src/kudu/consensus/quorum_util.h       |  24 +-
 src/kudu/master/catalog_manager.cc     |  11 +-
 4 files changed, 198 insertions(+), 196 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/83b19ef3/src/kudu/consensus/quorum_util-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/consensus/quorum_util-test.cc b/src/kudu/consensus/quorum_util-test.cc
index 3f525c0..2014d71 100644
--- a/src/kudu/consensus/quorum_util-test.cc
+++ b/src/kudu/consensus/quorum_util-test.cc
@@ -273,91 +273,92 @@ TEST(QuorumUtilTest, TestIsRaftConfigVoter) {
   ASSERT_FALSE(ReplicaTypesEqual(*peer_b, *peer_c));
 }
 
-// Verify logic of the kudu::consensus::IsUnderReplicated.
-TEST(QuorumUtilTest, IsUnderReplicated) {
+// Verify basic functionality of the kudu::consensus::ShouldAddReplica() utility
+// function.
+TEST(QuorumUtilTest, ShouldAddReplica) {
   {
     RaftConfigPB config;
     AddPeer(&config, "A", V);
     AddPeer(&config, "B", V);
     AddPeer(&config, "C", V);
-    EXPECT_FALSE(IsUnderReplicated(config, 2));
-    EXPECT_FALSE(IsUnderReplicated(config, 3));
-    EXPECT_TRUE(IsUnderReplicated(config, 4));
+    EXPECT_FALSE(ShouldAddReplica(config, 2));
+    EXPECT_FALSE(ShouldAddReplica(config, 3));
+    EXPECT_TRUE(ShouldAddReplica(config, 4));
   }
   {
     RaftConfigPB config;
     AddPeer(&config, "A", V, '?');
     AddPeer(&config, "B", V, '?');
     AddPeer(&config, "C", V, '?');
-    EXPECT_FALSE(IsUnderReplicated(config, 2));
-    EXPECT_FALSE(IsUnderReplicated(config, 3));
-    EXPECT_TRUE(IsUnderReplicated(config, 4));
+    EXPECT_FALSE(ShouldAddReplica(config, 2));
+    EXPECT_FALSE(ShouldAddReplica(config, 3));
+    EXPECT_TRUE(ShouldAddReplica(config, 4));
   }
   {
     RaftConfigPB config;
     AddPeer(&config, "A", V, '?');
     AddPeer(&config, "B", V, '?');
     AddPeer(&config, "C", V, '-');
-    EXPECT_FALSE(IsUnderReplicated(config, 2));
-    EXPECT_TRUE(IsUnderReplicated(config, 3));
+    EXPECT_FALSE(ShouldAddReplica(config, 2));
+    EXPECT_TRUE(ShouldAddReplica(config, 3));
   }
   {
     RaftConfigPB config;
     AddPeer(&config, "A", V, '+');
     AddPeer(&config, "B", V, '+');
     AddPeer(&config, "C", N, '+');
-    EXPECT_FALSE(IsUnderReplicated(config, 2));
-    EXPECT_TRUE(IsUnderReplicated(config, 3));
+    EXPECT_FALSE(ShouldAddReplica(config, 2));
+    EXPECT_TRUE(ShouldAddReplica(config, 3));
   }
   {
     RaftConfigPB config;
     AddPeer(&config, "A", V, '?');
     AddPeer(&config, "B", V, '?');
     AddPeer(&config, "C", N, '+');
-    EXPECT_FALSE(IsUnderReplicated(config, 2));
-    EXPECT_TRUE(IsUnderReplicated(config, 3));
+    EXPECT_FALSE(ShouldAddReplica(config, 2));
+    EXPECT_TRUE(ShouldAddReplica(config, 3));
   }
   {
     RaftConfigPB config;
     AddPeer(&config, "A", V, '?');
     AddPeer(&config, "B", V, '-');
     AddPeer(&config, "C", N, '+');
-    EXPECT_FALSE(IsUnderReplicated(config, 1));
-    EXPECT_TRUE(IsUnderReplicated(config, 2));
-    EXPECT_TRUE(IsUnderReplicated(config, 3));
+    EXPECT_FALSE(ShouldAddReplica(config, 1));
+    EXPECT_TRUE(ShouldAddReplica(config, 2));
+    EXPECT_TRUE(ShouldAddReplica(config, 3));
   }
   {
     RaftConfigPB config;
     AddPeer(&config, "A", V, '?');
     AddPeer(&config, "B", V, '-');
     AddPeer(&config, "C", N, '+', {{"PROMOTE", true}});
-    EXPECT_FALSE(IsUnderReplicated(config, 1));
-    EXPECT_FALSE(IsUnderReplicated(config, 2));
-    EXPECT_TRUE(IsUnderReplicated(config, 3));
+    EXPECT_FALSE(ShouldAddReplica(config, 1));
+    EXPECT_FALSE(ShouldAddReplica(config, 2));
+    EXPECT_TRUE(ShouldAddReplica(config, 3));
   }
   {
     RaftConfigPB config;
     AddPeer(&config, "A", V, '?');
     AddPeer(&config, "B", V, '-');
     AddPeer(&config, "C", N, '-', {{"PROMOTE", true}});
-    EXPECT_FALSE(IsUnderReplicated(config, 1));
-    EXPECT_TRUE(IsUnderReplicated(config, 2));
+    EXPECT_FALSE(ShouldAddReplica(config, 1));
+    EXPECT_TRUE(ShouldAddReplica(config, 2));
   }
   {
     RaftConfigPB config;
     AddPeer(&config, "A", V, '+');
     AddPeer(&config, "B", V, '+');
     AddPeer(&config, "C", V, '-');
-    EXPECT_FALSE(IsUnderReplicated(config, 2));
-    EXPECT_TRUE(IsUnderReplicated(config, 3));
+    EXPECT_FALSE(ShouldAddReplica(config, 2));
+    EXPECT_TRUE(ShouldAddReplica(config, 3));
   }
   {
     RaftConfigPB config;
     AddPeer(&config, "A", V, '+');
     AddPeer(&config, "B", V, '+');
     AddPeer(&config, "C", V, '+', {{"REPLACE", true}});
-    EXPECT_TRUE(IsUnderReplicated(config, 3));
-    EXPECT_FALSE(IsUnderReplicated(config, 2));
+    EXPECT_TRUE(ShouldAddReplica(config, 3));
+    EXPECT_FALSE(ShouldAddReplica(config, 2));
   }
   {
     RaftConfigPB config;
@@ -365,9 +366,9 @@ TEST(QuorumUtilTest, IsUnderReplicated) {
     AddPeer(&config, "B", V, '+');
     AddPeer(&config, "C", V, '+', {{"REPLACE", true}});
     AddPeer(&config, "D", N, '+');
-    EXPECT_TRUE(IsUnderReplicated(config, 4));
-    EXPECT_TRUE(IsUnderReplicated(config, 3));
-    EXPECT_FALSE(IsUnderReplicated(config, 2));
+    EXPECT_TRUE(ShouldAddReplica(config, 4));
+    EXPECT_TRUE(ShouldAddReplica(config, 3));
+    EXPECT_FALSE(ShouldAddReplica(config, 2));
   }
   {
     RaftConfigPB config;
@@ -375,9 +376,9 @@ TEST(QuorumUtilTest, IsUnderReplicated) {
     AddPeer(&config, "B", V, '+');
     AddPeer(&config, "C", V, '+', {{"REPLACE", true}});
     AddPeer(&config, "D", N, '+', {{"PROMOTE", true}});
-    EXPECT_TRUE(IsUnderReplicated(config, 4));
-    EXPECT_FALSE(IsUnderReplicated(config, 3));
-    EXPECT_FALSE(IsUnderReplicated(config, 2));
+    EXPECT_TRUE(ShouldAddReplica(config, 4));
+    EXPECT_FALSE(ShouldAddReplica(config, 3));
+    EXPECT_FALSE(ShouldAddReplica(config, 2));
   }
   {
     RaftConfigPB config;
@@ -385,8 +386,8 @@ TEST(QuorumUtilTest, IsUnderReplicated) {
     AddPeer(&config, "B", V, '+');
     AddPeer(&config, "C", V, '-');
     AddPeer(&config, "D", N, '-');
-    EXPECT_FALSE(IsUnderReplicated(config, 2));
-    EXPECT_TRUE(IsUnderReplicated(config, 3));
+    EXPECT_FALSE(ShouldAddReplica(config, 2));
+    EXPECT_TRUE(ShouldAddReplica(config, 3));
   }
   {
     RaftConfigPB config;
@@ -394,11 +395,11 @@ TEST(QuorumUtilTest, IsUnderReplicated) {
     AddPeer(&config, "B", V, '+');
     AddPeer(&config, "C", V, '-');
     AddPeer(&config, "D", N, '+');
-    EXPECT_FALSE(IsUnderReplicated(config, 2));
+    EXPECT_FALSE(ShouldAddReplica(config, 2));
     // The non-voter replica does not have the PROMOTE attribute,
     // so a new one is needed.
-    EXPECT_TRUE(IsUnderReplicated(config, 3));
-    EXPECT_TRUE(IsUnderReplicated(config, 4));
+    EXPECT_TRUE(ShouldAddReplica(config, 3));
+    EXPECT_TRUE(ShouldAddReplica(config, 4));
   }
   {
     RaftConfigPB config;
@@ -406,9 +407,9 @@ TEST(QuorumUtilTest, IsUnderReplicated) {
     AddPeer(&config, "B", V, '+');
     AddPeer(&config, "C", V, '-');
     AddPeer(&config, "D", N, '+', {{"PROMOTE", true}});
-    EXPECT_FALSE(IsUnderReplicated(config, 2));
-    EXPECT_FALSE(IsUnderReplicated(config, 3));
-    EXPECT_TRUE(IsUnderReplicated(config, 4));
+    EXPECT_FALSE(ShouldAddReplica(config, 2));
+    EXPECT_FALSE(ShouldAddReplica(config, 3));
+    EXPECT_TRUE(ShouldAddReplica(config, 4));
   }
   {
     RaftConfigPB config;
@@ -416,8 +417,8 @@ TEST(QuorumUtilTest, IsUnderReplicated) {
     AddPeer(&config, "B", V, '+');
     AddPeer(&config, "C", V, '-');
     AddPeer(&config, "D", N, '-', {{"PROMOTE", true}});
-    EXPECT_FALSE(IsUnderReplicated(config, 2));
-    EXPECT_TRUE(IsUnderReplicated(config, 3));
+    EXPECT_FALSE(ShouldAddReplica(config, 2));
+    EXPECT_TRUE(ShouldAddReplica(config, 3));
   }
   {
     RaftConfigPB config;
@@ -426,7 +427,7 @@ TEST(QuorumUtilTest, IsUnderReplicated) {
     AddPeer(&config, "C", V, '+');
     AddPeer(&config, "D", N, '-', {{"PROMOTE", true}});
     AddPeer(&config, "E", N, '+', {{"PROMOTE", true}});
-    EXPECT_FALSE(IsUnderReplicated(config, 3));
+    EXPECT_FALSE(ShouldAddReplica(config, 3));
   }
   {
     RaftConfigPB config;
@@ -435,7 +436,7 @@ TEST(QuorumUtilTest, IsUnderReplicated) {
     AddPeer(&config, "C", V, '+');
     AddPeer(&config, "D", N, '-', {{"PROMOTE", true}});
     AddPeer(&config, "E", N, '+', {{"PROMOTE", false}});
-    EXPECT_TRUE(IsUnderReplicated(config, 3));
+    EXPECT_TRUE(ShouldAddReplica(config, 3));
   }
   {
     RaftConfigPB config;
@@ -445,28 +446,28 @@ TEST(QuorumUtilTest, IsUnderReplicated) {
     // The catalog manager will be able to carry on the required update of the
     // configuration after achieving the majority.
     // TODO(aserbin): add an integration test for that.
-    EXPECT_TRUE(IsUnderReplicated(config, 3));
+    EXPECT_TRUE(ShouldAddReplica(config, 3));
   }
 }
 
-// Verify logic of the kudu::consensus::CanEvictReplica(), anticipating
+// Verify logic of the kudu::consensus::ShouldEvictReplica(), anticipating
 // removal of a voter replica.
-TEST(QuorumUtilTest, CanEvictReplicaVoters) {
+TEST(QuorumUtilTest, ShouldEvictReplicaVoters) {
   {
     RaftConfigPB config;
     AddPeer(&config, "A", V);
     AddPeer(&config, "B", V);
     AddPeer(&config, "C", V);
-    EXPECT_FALSE(CanEvictReplica(config, "", 3));
-    EXPECT_FALSE(CanEvictReplica(config, "", 2));
+    EXPECT_FALSE(ShouldEvictReplica(config, "", 3));
+    EXPECT_FALSE(ShouldEvictReplica(config, "", 2));
   }
   {
     RaftConfigPB config;
     AddPeer(&config, "A", V, '?');
     AddPeer(&config, "B", V, '?');
     AddPeer(&config, "C", V, '-');
-    EXPECT_FALSE(CanEvictReplica(config, "", 3));
-    EXPECT_FALSE(CanEvictReplica(config, "", 2));
+    EXPECT_FALSE(ShouldEvictReplica(config, "", 3));
+    EXPECT_FALSE(ShouldEvictReplica(config, "", 2));
   }
   {
     RaftConfigPB config;
@@ -474,11 +475,11 @@ TEST(QuorumUtilTest, CanEvictReplicaVoters) {
     AddPeer(&config, "B", V, '+');
     AddPeer(&config, "C", V, '-');
     string to_evict;
-    ASSERT_TRUE(CanEvictReplica(config, "A", 1, &to_evict));
+    ASSERT_TRUE(ShouldEvictReplica(config, "A", 1, &to_evict));
     EXPECT_EQ("C", to_evict);
-    ASSERT_TRUE(CanEvictReplica(config, "A", 2));
+    ASSERT_TRUE(ShouldEvictReplica(config, "A", 2));
     EXPECT_EQ("C", to_evict);
-    EXPECT_FALSE(CanEvictReplica(config, "A", 3));
+    EXPECT_FALSE(ShouldEvictReplica(config, "A", 3));
   }
   {
     RaftConfigPB config;
@@ -487,17 +488,17 @@ TEST(QuorumUtilTest, CanEvictReplicaVoters) {
     AddPeer(&config, "C", V, '+');
     // Not safe to evict because we don't have enough healthy nodes to commit
     // the eviction.
-    EXPECT_FALSE(CanEvictReplica(config, "C", 1));
-    EXPECT_FALSE(CanEvictReplica(config, "C", 2));
-    EXPECT_FALSE(CanEvictReplica(config, "C", 3));
+    EXPECT_FALSE(ShouldEvictReplica(config, "C", 1));
+    EXPECT_FALSE(ShouldEvictReplica(config, "C", 2));
+    EXPECT_FALSE(ShouldEvictReplica(config, "C", 3));
   }
   {
     RaftConfigPB config;
     AddPeer(&config, "A", V, '+', {{"REPLACE", true}});
     AddPeer(&config, "B", V);
     AddPeer(&config, "C", V);
-    EXPECT_FALSE(CanEvictReplica(config, "A", 3));
-    EXPECT_FALSE(CanEvictReplica(config, "A", 2));
+    EXPECT_FALSE(ShouldEvictReplica(config, "A", 3));
+    EXPECT_FALSE(ShouldEvictReplica(config, "A", 2));
   }
   {
     RaftConfigPB config;
@@ -505,9 +506,9 @@ TEST(QuorumUtilTest, CanEvictReplicaVoters) {
     AddPeer(&config, "B", V, '+', {{"REPLACE", false}});
     AddPeer(&config, "C", V, '-');
     AddPeer(&config, "D", V, '+');
-    EXPECT_FALSE(CanEvictReplica(config, "A", 4));
+    EXPECT_FALSE(ShouldEvictReplica(config, "A", 4));
     string uuid_to_evict;
-    ASSERT_TRUE(CanEvictReplica(config, "A", 3, &uuid_to_evict));
+    ASSERT_TRUE(ShouldEvictReplica(config, "A", 3, &uuid_to_evict));
     EXPECT_EQ("C", uuid_to_evict);
   }
   {
@@ -516,9 +517,9 @@ TEST(QuorumUtilTest, CanEvictReplicaVoters) {
     AddPeer(&config, "B", V, '?');
     AddPeer(&config, "C", V, '+');
     AddPeer(&config, "D", V, '+');
-    EXPECT_FALSE(CanEvictReplica(config, "A", 4));
+    EXPECT_FALSE(ShouldEvictReplica(config, "A", 4));
     string uuid_to_evict;
-    ASSERT_TRUE(CanEvictReplica(config, "A", 3, &uuid_to_evict));
+    ASSERT_TRUE(ShouldEvictReplica(config, "A", 3, &uuid_to_evict));
     EXPECT_EQ("B", uuid_to_evict);
   }
   for (char health_status : kHealthStatuses) {
@@ -532,7 +533,7 @@ TEST(QuorumUtilTest, CanEvictReplicaVoters) {
     // with only A and B, regardless of D's health and regardless of the
     // desired replication factor.
     string uuid_to_evict;
-    ASSERT_TRUE(CanEvictReplica(config, "A", 2, &uuid_to_evict));
+    ASSERT_TRUE(ShouldEvictReplica(config, "A", 2, &uuid_to_evict));
     // The priority of voter replica replacement (decreasing):
     //   * failed & slated for replacement
     //   * failed
@@ -542,14 +543,14 @@ TEST(QuorumUtilTest, CanEvictReplicaVoters) {
     } else {
       EXPECT_EQ("C", uuid_to_evict);
     }
-    ASSERT_TRUE(CanEvictReplica(config, "A", 3, &uuid_to_evict));
+    ASSERT_TRUE(ShouldEvictReplica(config, "A", 3, &uuid_to_evict));
     if (health_status == '-') {
       EXPECT_EQ("D", uuid_to_evict);
     } else {
       EXPECT_EQ("C", uuid_to_evict);
     }
     // Since we are not over-replicated, we will not evict in this case.
-    EXPECT_FALSE(CanEvictReplica(config, "A", 4));
+    EXPECT_FALSE(ShouldEvictReplica(config, "A", 4));
   }
   {
     RaftConfigPB config;
@@ -564,10 +565,10 @@ TEST(QuorumUtilTest, CanEvictReplicaVoters) {
     // and then it's better to keep 'D' around to provide the required
     // replication factor. It's necessary to wait for more deterministic status
     // of replica 'C' before making proper eviction decision.
-    EXPECT_FALSE(CanEvictReplica(config, "A", 3));
+    EXPECT_FALSE(ShouldEvictReplica(config, "A", 3));
 
     string uuid_to_evict;
-    ASSERT_TRUE(CanEvictReplica(config, "A", 2, &uuid_to_evict));
+    ASSERT_TRUE(ShouldEvictReplica(config, "A", 2, &uuid_to_evict));
     EXPECT_EQ("D", uuid_to_evict);
   }
   {
@@ -576,33 +577,33 @@ TEST(QuorumUtilTest, CanEvictReplicaVoters) {
     AddPeer(&config, "B", V, '?');
     AddPeer(&config, "C", V, '+');
     AddPeer(&config, "D", V, '+');
-    EXPECT_FALSE(CanEvictReplica(config, "A", 4));
+    EXPECT_FALSE(ShouldEvictReplica(config, "A", 4));
     string uuid_to_evict;
-    ASSERT_TRUE(CanEvictReplica(config, "A", 3, &uuid_to_evict));
+    ASSERT_TRUE(ShouldEvictReplica(config, "A", 3, &uuid_to_evict));
     EXPECT_EQ("B", uuid_to_evict);
   }
 }
 
-// Verify logic of the kudu::consensus::CanEvictReplica(), anticipating
+// Verify logic of the kudu::consensus::ShouldEvictReplica(), anticipating
 // removal of a non-voter replica.
-TEST(QuorumUtilTest, CanEvictReplicaNonVoters) {
+TEST(QuorumUtilTest, ShouldEvictReplicaNonVoters) {
   {
     RaftConfigPB config;
     AddPeer(&config, "A", V);
-    EXPECT_FALSE(CanEvictReplica(config, "", 1));
+    EXPECT_FALSE(ShouldEvictReplica(config, "", 1));
   }
   {
     RaftConfigPB config;
     AddPeer(&config, "A", V, '+');
-    EXPECT_FALSE(CanEvictReplica(config, "A", 1));
+    EXPECT_FALSE(ShouldEvictReplica(config, "A", 1));
   }
   {
     RaftConfigPB config;
     AddPeer(&config, "A", V, '+');
     AddPeer(&config, "B", N);
-    EXPECT_FALSE(CanEvictReplica(config, "A", 2));
+    EXPECT_FALSE(ShouldEvictReplica(config, "A", 2));
     string uuid_to_evict;
-    ASSERT_TRUE(CanEvictReplica(config, "A", 1, &uuid_to_evict));
+    ASSERT_TRUE(ShouldEvictReplica(config, "A", 1, &uuid_to_evict));
     EXPECT_EQ("B", uuid_to_evict);
   }
   {
@@ -611,9 +612,9 @@ TEST(QuorumUtilTest, CanEvictReplicaNonVoters) {
     AddPeer(&config, "B", V, '+');
     AddPeer(&config, "C", N, '+');
     string uuid_to_evict;
-    ASSERT_TRUE(CanEvictReplica(config, "A", 2, &uuid_to_evict));
+    ASSERT_TRUE(ShouldEvictReplica(config, "A", 2, &uuid_to_evict));
     EXPECT_EQ("C", uuid_to_evict);
-    ASSERT_TRUE(CanEvictReplica(config, "A", 1, &uuid_to_evict));
+    ASSERT_TRUE(ShouldEvictReplica(config, "A", 1, &uuid_to_evict));
     EXPECT_EQ("C", uuid_to_evict);
   }
   {
@@ -623,9 +624,9 @@ TEST(QuorumUtilTest, CanEvictReplicaNonVoters) {
     string uuid_to_evict;
     // It's always safe to evict an unhealthy non-voter if we have enough
     // healthy voters to commit the config change.
-    ASSERT_TRUE(CanEvictReplica(config, "A", 2, &uuid_to_evict));
+    ASSERT_TRUE(ShouldEvictReplica(config, "A", 2, &uuid_to_evict));
     EXPECT_EQ("B", uuid_to_evict);
-    ASSERT_TRUE(CanEvictReplica(config, "A", 1, &uuid_to_evict));
+    ASSERT_TRUE(ShouldEvictReplica(config, "A", 1, &uuid_to_evict));
     EXPECT_EQ("B", uuid_to_evict);
   }
   {
@@ -634,9 +635,9 @@ TEST(QuorumUtilTest, CanEvictReplicaNonVoters) {
     AddPeer(&config, "B", N, '-');
     AddPeer(&config, "C", N);
     string uuid_to_evict;
-    ASSERT_TRUE(CanEvictReplica(config, "A", 2, &uuid_to_evict));
+    ASSERT_TRUE(ShouldEvictReplica(config, "A", 2, &uuid_to_evict));
     EXPECT_EQ("B", uuid_to_evict);
-    ASSERT_TRUE(CanEvictReplica(config, "A", 1, &uuid_to_evict));
+    ASSERT_TRUE(ShouldEvictReplica(config, "A", 1, &uuid_to_evict));
     EXPECT_EQ("B", uuid_to_evict);
   }
   {
@@ -644,9 +645,9 @@ TEST(QuorumUtilTest, CanEvictReplicaNonVoters) {
     AddPeer(&config, "A", V, '+');
     AddPeer(&config, "B", N, '?');
     AddPeer(&config, "C", N, '+');
-    EXPECT_FALSE(CanEvictReplica(config, "A", 2));
+    EXPECT_FALSE(ShouldEvictReplica(config, "A", 2));
     string uuid_to_evict;
-    ASSERT_TRUE(CanEvictReplica(config, "A", 1, &uuid_to_evict));
+    ASSERT_TRUE(ShouldEvictReplica(config, "A", 1, &uuid_to_evict));
     EXPECT_EQ("B", uuid_to_evict);
   }
   {
@@ -655,9 +656,9 @@ TEST(QuorumUtilTest, CanEvictReplicaNonVoters) {
     AddPeer(&config, "B", V, '+');
     AddPeer(&config, "C", N);
     string uuid_to_evict;
-    ASSERT_TRUE(CanEvictReplica(config, "A", 2, &uuid_to_evict));
+    ASSERT_TRUE(ShouldEvictReplica(config, "A", 2, &uuid_to_evict));
     EXPECT_EQ("C", uuid_to_evict);
-    ASSERT_TRUE(CanEvictReplica(config, "A", 1, &uuid_to_evict));
+    ASSERT_TRUE(ShouldEvictReplica(config, "A", 1, &uuid_to_evict));
     EXPECT_EQ("C", uuid_to_evict);
   }
   {
@@ -665,19 +666,19 @@ TEST(QuorumUtilTest, CanEvictReplicaNonVoters) {
     AddPeer(&config, "A", V, '+');
     AddPeer(&config, "B", V);
     AddPeer(&config, "C", N);
-    EXPECT_FALSE(CanEvictReplica(config, "A", 2));
+    EXPECT_FALSE(ShouldEvictReplica(config, "A", 2));
     // Would evict a non-voter first, but it's not known whether the majority
     // of the voter replicas are on-line to commence the operation: that's
     // because the state of B is unknown. So, in this case the voter replica B
     // will be removed first.
     string uuid_to_evict;
-    ASSERT_TRUE(CanEvictReplica(config, "A", 1, &uuid_to_evict));
+    ASSERT_TRUE(ShouldEvictReplica(config, "A", 1, &uuid_to_evict));
     EXPECT_EQ("B", uuid_to_evict);
 
     RemovePeer(&config, "B");
     // Now, having just a single online replica, it's possible to evict the
     // failed non-voter replica C.
-    ASSERT_TRUE(CanEvictReplica(config, "A", 1, &uuid_to_evict));
+    ASSERT_TRUE(ShouldEvictReplica(config, "A", 1, &uuid_to_evict));
     EXPECT_EQ("C", uuid_to_evict);
   }
   {
@@ -685,26 +686,26 @@ TEST(QuorumUtilTest, CanEvictReplicaNonVoters) {
     AddPeer(&config, "A", V, '-');
     AddPeer(&config, "B", V);
     AddPeer(&config, "C", N);
-    EXPECT_FALSE(CanEvictReplica(config, "", 2));
-    EXPECT_FALSE(CanEvictReplica(config, "", 1));
+    EXPECT_FALSE(ShouldEvictReplica(config, "", 2));
+    EXPECT_FALSE(ShouldEvictReplica(config, "", 1));
   }
   {
     RaftConfigPB config;
     AddPeer(&config, "A", V, '+');
     AddPeer(&config, "B", V, '-');
     AddPeer(&config, "C", N, '-', {{"PROMOTE", true}});
-    EXPECT_FALSE(CanEvictReplica(config, "A", 2));
+    EXPECT_FALSE(ShouldEvictReplica(config, "A", 2));
     // Would evict a non-voter first, but replica B is reported as failed and
     // the configuration does not have enough healthy voter replicas to have a
     // majority of votes. So, the voter replica B will be removed first.
     string uuid_to_evict;
-    ASSERT_TRUE(CanEvictReplica(config, "A", 1, &uuid_to_evict));
+    ASSERT_TRUE(ShouldEvictReplica(config, "A", 1, &uuid_to_evict));
     EXPECT_EQ("B", uuid_to_evict);
 
     RemovePeer(&config, "B");
     // Now, having just a single online replica, it's possible to evict the
     // failed non-voter replica C.
-    ASSERT_TRUE(CanEvictReplica(config, "A", 1, &uuid_to_evict));
+    ASSERT_TRUE(ShouldEvictReplica(config, "A", 1, &uuid_to_evict));
     EXPECT_EQ("C", uuid_to_evict);
   }
   {
@@ -713,9 +714,9 @@ TEST(QuorumUtilTest, CanEvictReplicaNonVoters) {
     AddPeer(&config, "B", V, '+');
     AddPeer(&config, "C", V, '+');
     AddPeer(&config, "D", N, '+', {{"PROMOTE", true}});
-    EXPECT_FALSE(CanEvictReplica(config, "B", 3));
+    EXPECT_FALSE(ShouldEvictReplica(config, "B", 3));
     string uuid_to_evict;
-    ASSERT_TRUE(CanEvictReplica(config, "B", 2, &uuid_to_evict));
+    ASSERT_TRUE(ShouldEvictReplica(config, "B", 2, &uuid_to_evict));
     EXPECT_EQ("D", uuid_to_evict);
   }
   {
@@ -727,9 +728,9 @@ TEST(QuorumUtilTest, CanEvictReplicaNonVoters) {
     AddPeer(&config, "C", V, '+');
     AddPeer(&config, "D", N, '-', {{"PROMOTE", true}});
     string uuid_to_evict;
-    ASSERT_TRUE(CanEvictReplica(config, "B", 4, &uuid_to_evict));
+    ASSERT_TRUE(ShouldEvictReplica(config, "B", 4, &uuid_to_evict));
     EXPECT_EQ("D", uuid_to_evict);
-    ASSERT_TRUE(CanEvictReplica(config, "C", 3, &uuid_to_evict));
+    ASSERT_TRUE(ShouldEvictReplica(config, "C", 3, &uuid_to_evict));
     EXPECT_EQ("D", uuid_to_evict);
   }
   {
@@ -738,8 +739,8 @@ TEST(QuorumUtilTest, CanEvictReplicaNonVoters) {
     AddPeer(&config, "B", V, '+');
     AddPeer(&config, "C", V, '+');
     AddPeer(&config, "D", N, '?', {{"PROMOTE", true}});
-    EXPECT_FALSE(CanEvictReplica(config, "B", 3));
-    EXPECT_FALSE(CanEvictReplica(config, "B", 4));
+    EXPECT_FALSE(ShouldEvictReplica(config, "B", 3));
+    EXPECT_FALSE(ShouldEvictReplica(config, "B", 4));
   }
 }
 
@@ -754,7 +755,7 @@ TEST(QuorumUtilTest, TestDontEvictLeader) {
   AddPeer(&config, nodes[3], V, '+');
   for (const auto& leader_node : nodes) {
     string to_evict;
-    ASSERT_TRUE(CanEvictReplica(config, leader_node, 3, &to_evict));
+    ASSERT_TRUE(ShouldEvictReplica(config, leader_node, 3, &to_evict));
     ASSERT_NE(leader_node, to_evict);
   }
 }
@@ -769,9 +770,9 @@ TEST(QuorumUtilTest, TooManyVoters) {
     AddPeer(&config, "C", V, '?');
     AddPeer(&config, "D", V, '-');
     string to_evict;
-    ASSERT_TRUE(CanEvictReplica(config, "A", 3, &to_evict));
+    ASSERT_TRUE(ShouldEvictReplica(config, "A", 3, &to_evict));
     EXPECT_EQ("D", to_evict);
-    EXPECT_FALSE(IsUnderReplicated(config, 3));
+    EXPECT_FALSE(ShouldAddReplica(config, 3));
   }
   {
     RaftConfigPB config;
@@ -780,9 +781,9 @@ TEST(QuorumUtilTest, TooManyVoters) {
     AddPeer(&config, "C", V, '-');
     AddPeer(&config, "D", V, '-');
     string to_evict;
-    ASSERT_TRUE(CanEvictReplica(config, "A", 3, &to_evict));
+    ASSERT_TRUE(ShouldEvictReplica(config, "A", 3, &to_evict));
     EXPECT_TRUE(to_evict == "C" || to_evict == "D") << to_evict;
-    EXPECT_TRUE(IsUnderReplicated(config, 3));
+    EXPECT_TRUE(ShouldAddReplica(config, 3));
   }
 }
 
@@ -793,8 +794,8 @@ TEST(QuorumUtilTest, ReplaceAttributeBasic) {
     AddPeer(&config, "A", V, '+', {{"REPLACE", true}});
     AddPeer(&config, "B", V, '+');
     AddPeer(&config, "C", V, '+');
-    EXPECT_FALSE(CanEvictReplica(config, "A", 3));
-    EXPECT_TRUE(IsUnderReplicated(config, 3));
+    EXPECT_FALSE(ShouldEvictReplica(config, "A", 3));
+    EXPECT_TRUE(ShouldAddReplica(config, 3));
   }
   {
     RaftConfigPB config;
@@ -802,8 +803,8 @@ TEST(QuorumUtilTest, ReplaceAttributeBasic) {
     AddPeer(&config, "B", V, '+');
     AddPeer(&config, "C", V, '+');
     AddPeer(&config, "D", V, '+');
-    EXPECT_FALSE(CanEvictReplica(config, "A", 3));
-    EXPECT_FALSE(IsUnderReplicated(config, 3));
+    EXPECT_FALSE(ShouldEvictReplica(config, "A", 3));
+    EXPECT_FALSE(ShouldAddReplica(config, 3));
   }
   for (auto health_status : { '-', '?' }) {
     RaftConfigPB config;
@@ -815,9 +816,9 @@ TEST(QuorumUtilTest, ReplaceAttributeBasic) {
       SCOPED_TRACE(Substitute("health status '$0', leader $1",
                               health_status, leader_replica));
       string to_evict;
-      ASSERT_TRUE(CanEvictReplica(config, leader_replica, 3, &to_evict));
+      ASSERT_TRUE(ShouldEvictReplica(config, leader_replica, 3, &to_evict));
       EXPECT_EQ("A", to_evict);
-      EXPECT_FALSE(IsUnderReplicated(config, 3));
+      EXPECT_FALSE(ShouldAddReplica(config, 3));
     }
   }
   {
@@ -828,13 +829,13 @@ TEST(QuorumUtilTest, ReplaceAttributeBasic) {
     AddPeer(&config, "D", V, '+');
     AddPeer(&config, "E", V, '+');
     string to_evict;
-    ASSERT_TRUE(CanEvictReplica(config, "A", 3, &to_evict));
+    ASSERT_TRUE(ShouldEvictReplica(config, "A", 3, &to_evict));
     EXPECT_NE("A", to_evict);
-    EXPECT_FALSE(IsUnderReplicated(config, 3));
+    EXPECT_FALSE(ShouldAddReplica(config, 3));
 
     for (const auto& leader_replica : { "B", "C", "D", "E" }) {
       string to_evict;
-      ASSERT_TRUE(CanEvictReplica(config, leader_replica, 3, &to_evict));
+      ASSERT_TRUE(ShouldEvictReplica(config, leader_replica, 3, &to_evict));
       EXPECT_EQ("A", to_evict);
     }
   }
@@ -916,8 +917,8 @@ TEST(QuorumUtilTest, LeaderReplicaWithReplaceAttribute) {
     AddPeer(&config, "B", V, '+');
     AddPeer(&config, "C", V, '+');
     AddPeer(&config, "D", N, health_status, {{"PROMOTE", true}});
-    EXPECT_FALSE(CanEvictReplica(config, "A", 3));
-    EXPECT_FALSE(IsUnderReplicated(config, 3));
+    EXPECT_FALSE(ShouldEvictReplica(config, "A", 3));
+    EXPECT_FALSE(ShouldAddReplica(config, 3));
   }
   for (auto health_status : { '+', '?' }) {
     SCOPED_TRACE(Substitute("voter replica with status '$0'", health_status));
@@ -926,8 +927,8 @@ TEST(QuorumUtilTest, LeaderReplicaWithReplaceAttribute) {
     AddPeer(&config, "B", V, '+');
     AddPeer(&config, "C", V, '+');
     AddPeer(&config, "D", V, health_status);
-    EXPECT_FALSE(CanEvictReplica(config, "A", 3));
-    EXPECT_FALSE(IsUnderReplicated(config, 3));
+    EXPECT_FALSE(ShouldEvictReplica(config, "A", 3));
+    EXPECT_FALSE(ShouldAddReplica(config, 3));
   }
   for (auto promote : { false, true }) {
     SCOPED_TRACE(Substitute("failed non-voter replica with PROMOTE attribute $0",
@@ -938,9 +939,9 @@ TEST(QuorumUtilTest, LeaderReplicaWithReplaceAttribute) {
     AddPeer(&config, "C", V, '+');
     AddPeer(&config, "D", N, '-', {{"PROMOTE", promote}});
     string to_evict;
-    ASSERT_TRUE(CanEvictReplica(config, "A", 3, &to_evict));
+    ASSERT_TRUE(ShouldEvictReplica(config, "A", 3, &to_evict));
     EXPECT_EQ("D", to_evict);
-    EXPECT_TRUE(IsUnderReplicated(config, 3));
+    EXPECT_TRUE(ShouldAddReplica(config, 3));
   }
   {
     RaftConfigPB config;
@@ -949,9 +950,9 @@ TEST(QuorumUtilTest, LeaderReplicaWithReplaceAttribute) {
     AddPeer(&config, "C", V, '+');
     AddPeer(&config, "D", V, '-');
     string to_evict;
-    ASSERT_TRUE(CanEvictReplica(config, "A", 3, &to_evict));
+    ASSERT_TRUE(ShouldEvictReplica(config, "A", 3, &to_evict));
     EXPECT_EQ("D", to_evict);
-    EXPECT_TRUE(IsUnderReplicated(config, 3));
+    EXPECT_TRUE(ShouldAddReplica(config, 3));
   }
   {
     // Current algorithm is conservative in the cases like below, but we might
@@ -963,8 +964,8 @@ TEST(QuorumUtilTest, LeaderReplicaWithReplaceAttribute) {
     AddPeer(&config, "C", V, '+');
     AddPeer(&config, "D", N, '+', {{"PROMOTE", true}});
     AddPeer(&config, "E", N, '+');
-    EXPECT_FALSE(CanEvictReplica(config, "A", 3));
-    EXPECT_FALSE(IsUnderReplicated(config, 3));
+    EXPECT_FALSE(ShouldEvictReplica(config, "A", 3));
+    EXPECT_FALSE(ShouldAddReplica(config, 3));
   }
   {
     // The non-voter replica does not have the 'promote' attribute, so
@@ -977,9 +978,9 @@ TEST(QuorumUtilTest, LeaderReplicaWithReplaceAttribute) {
     AddPeer(&config, "D", V, '+');
     AddPeer(&config, "E", N, '+');
     string to_evict;
-    ASSERT_TRUE(CanEvictReplica(config, "A", 3, &to_evict));
+    ASSERT_TRUE(ShouldEvictReplica(config, "A", 3, &to_evict));
     EXPECT_EQ("E", to_evict);
-    EXPECT_FALSE(IsUnderReplicated(config, 3));
+    EXPECT_FALSE(ShouldAddReplica(config, 3));
   }
   {
     // In the case below the non-voter replica 'D' is not needed. The
@@ -993,9 +994,9 @@ TEST(QuorumUtilTest, LeaderReplicaWithReplaceAttribute) {
     AddPeer(&config, "D", N, '+', {{"PROMOTE", true}});
     AddPeer(&config, "E", V, '+');
     string to_evict;
-    ASSERT_TRUE(CanEvictReplica(config, "A", 3, &to_evict));
+    ASSERT_TRUE(ShouldEvictReplica(config, "A", 3, &to_evict));
     EXPECT_EQ("D", to_evict);
-    EXPECT_FALSE(IsUnderReplicated(config, 3));
+    EXPECT_FALSE(ShouldAddReplica(config, 3));
   }
 }
 
@@ -1011,9 +1012,9 @@ TEST(QuorumUtilTest, MultipleReplicasWithReplaceAttribute) {
     AddPeer(&config, "C", V, '+', {{"REPLACE", true}});
     AddPeer(&config, "D", replica_type, '-');
     string to_evict;
-    ASSERT_TRUE(CanEvictReplica(config, "A", 3, &to_evict));
+    ASSERT_TRUE(ShouldEvictReplica(config, "A", 3, &to_evict));
     EXPECT_EQ("D", to_evict);
-    EXPECT_TRUE(IsUnderReplicated(config, 3));
+    EXPECT_TRUE(ShouldAddReplica(config, 3));
   }
   for (auto replica_health : { '+', '?' }) {
     SCOPED_TRACE(Substitute("NON_VOTER replica with health status '$0'", replica_health));
@@ -1022,8 +1023,8 @@ TEST(QuorumUtilTest, MultipleReplicasWithReplaceAttribute) {
     AddPeer(&config, "B", V, '+', {{"REPLACE", true}});
     AddPeer(&config, "C", V, '+', {{"REPLACE", true}});
     AddPeer(&config, "D", N, replica_health);
-    EXPECT_FALSE(CanEvictReplica(config, "A", 3));
-    EXPECT_TRUE(IsUnderReplicated(config, 3));
+    EXPECT_FALSE(ShouldEvictReplica(config, "A", 3));
+    EXPECT_TRUE(ShouldAddReplica(config, 3));
   }
   for (const auto& leader_replica : { "A", "B", "C" }) {
     RaftConfigPB config;
@@ -1031,8 +1032,8 @@ TEST(QuorumUtilTest, MultipleReplicasWithReplaceAttribute) {
     AddPeer(&config, "B", V, '+', {{"REPLACE", true}});
     AddPeer(&config, "C", V, '+', {{"REPLACE", true}});
     AddPeer(&config, "D", V, '?');
-    EXPECT_FALSE(CanEvictReplica(config, leader_replica, 3));
-    EXPECT_TRUE(IsUnderReplicated(config, 3));
+    EXPECT_FALSE(ShouldEvictReplica(config, leader_replica, 3));
+    EXPECT_TRUE(ShouldAddReplica(config, 3));
   }
   for (const auto& leader_replica : { "A", "C" }) {
     RaftConfigPB config;
@@ -1041,16 +1042,16 @@ TEST(QuorumUtilTest, MultipleReplicasWithReplaceAttribute) {
     AddPeer(&config, "C", V, '+');
     AddPeer(&config, "D", N, '+', {{"PROMOTE", true}});
     AddPeer(&config, "E", N, '+', {{"PROMOTE", true}});
-    EXPECT_FALSE(CanEvictReplica(config, leader_replica, 3));
-    EXPECT_FALSE(IsUnderReplicated(config, 3));
+    EXPECT_FALSE(ShouldEvictReplica(config, leader_replica, 3));
+    EXPECT_FALSE(ShouldAddReplica(config, 3));
   }
   {
     RaftConfigPB config;
     AddPeer(&config, "A", V, '+', {{"REPLACE", true}});
     AddPeer(&config, "B", V, '+', {{"REPLACE", true}});
     AddPeer(&config, "C", V, '+', {{"REPLACE", true}});
-    EXPECT_FALSE(CanEvictReplica(config, "A", 3));
-    EXPECT_TRUE(IsUnderReplicated(config, 3));
+    EXPECT_FALSE(ShouldEvictReplica(config, "A", 3));
+    EXPECT_TRUE(ShouldAddReplica(config, 3));
   }
   {
     RaftConfigPB config;
@@ -1059,9 +1060,9 @@ TEST(QuorumUtilTest, MultipleReplicasWithReplaceAttribute) {
     AddPeer(&config, "C", V, '+');
     AddPeer(&config, "D", V, '+');
     string to_evict;
-    ASSERT_TRUE(CanEvictReplica(config, "A", 3, &to_evict));
+    ASSERT_TRUE(ShouldEvictReplica(config, "A", 3, &to_evict));
     EXPECT_EQ("B", to_evict);
-    EXPECT_TRUE(IsUnderReplicated(config, 3));
+    EXPECT_TRUE(ShouldAddReplica(config, 3));
   }
   {
     RaftConfigPB config;
@@ -1070,9 +1071,9 @@ TEST(QuorumUtilTest, MultipleReplicasWithReplaceAttribute) {
     AddPeer(&config, "C", V, '+', {{"REPLACE", true}});
     AddPeer(&config, "D", V, '+');
     string to_evict;
-    ASSERT_TRUE(CanEvictReplica(config, "A", 3, &to_evict));
+    ASSERT_TRUE(ShouldEvictReplica(config, "A", 3, &to_evict));
     EXPECT_TRUE(to_evict == "B" || to_evict == "C");
-    EXPECT_TRUE(IsUnderReplicated(config, 3));
+    EXPECT_TRUE(ShouldAddReplica(config, 3));
   }
   {
     RaftConfigPB config;
@@ -1080,10 +1081,10 @@ TEST(QuorumUtilTest, MultipleReplicasWithReplaceAttribute) {
     AddPeer(&config, "B", V, '+');
     AddPeer(&config, "C", V, '+');
     AddPeer(&config, "D", N, '+');
-    EXPECT_FALSE(CanEvictReplica(config, "A", 3));
+    EXPECT_FALSE(ShouldEvictReplica(config, "A", 3));
     // The non-voter replica does not have the PROMOTE attribute, so it the
     // configuration should be considered under-replicated.
-    EXPECT_TRUE(IsUnderReplicated(config, 3));
+    EXPECT_TRUE(ShouldAddReplica(config, 3));
   }
   for (auto replica_status : { '+', '?' }) {
     RaftConfigPB config;
@@ -1091,8 +1092,8 @@ TEST(QuorumUtilTest, MultipleReplicasWithReplaceAttribute) {
     AddPeer(&config, "B", V, '+');
     AddPeer(&config, "C", V, '+');
     AddPeer(&config, "D", N, replica_status, {{"PROMOTE", true}});
-    EXPECT_FALSE(CanEvictReplica(config, "A", 3));
-    EXPECT_FALSE(IsUnderReplicated(config, 3));
+    EXPECT_FALSE(ShouldEvictReplica(config, "A", 3));
+    EXPECT_FALSE(ShouldAddReplica(config, 3));
   }
   {
     RaftConfigPB config;
@@ -1101,9 +1102,9 @@ TEST(QuorumUtilTest, MultipleReplicasWithReplaceAttribute) {
     AddPeer(&config, "C", V, '+');
     AddPeer(&config, "D", N, '-');
     string to_evict;
-    ASSERT_TRUE(CanEvictReplica(config, "A", 3, &to_evict));
+    ASSERT_TRUE(ShouldEvictReplica(config, "A", 3, &to_evict));
     EXPECT_EQ("D", to_evict);
-    EXPECT_TRUE(IsUnderReplicated(config, 3));
+    EXPECT_TRUE(ShouldAddReplica(config, 3));
   }
   {
     RaftConfigPB config;
@@ -1113,9 +1114,9 @@ TEST(QuorumUtilTest, MultipleReplicasWithReplaceAttribute) {
     AddPeer(&config, "D", V, '+');
     AddPeer(&config, "E", V, '+');
     string to_evict;
-    ASSERT_TRUE(CanEvictReplica(config, "A", 3, &to_evict));
+    ASSERT_TRUE(ShouldEvictReplica(config, "A", 3, &to_evict));
     EXPECT_TRUE(to_evict == "B" || to_evict == "C");
-    EXPECT_TRUE(IsUnderReplicated(config, 3));
+    EXPECT_TRUE(ShouldAddReplica(config, 3));
   }
   {
     RaftConfigPB config;
@@ -1128,16 +1129,16 @@ TEST(QuorumUtilTest, MultipleReplicasWithReplaceAttribute) {
 
     for (const auto& leader_replica : { "A", "B", "C" }) {
       string to_evict;
-      ASSERT_TRUE(CanEvictReplica(config, leader_replica, 3, &to_evict));
+      ASSERT_TRUE(ShouldEvictReplica(config, leader_replica, 3, &to_evict));
       EXPECT_TRUE(to_evict == "A" || to_evict == "B" || to_evict == "C");
       EXPECT_NE(leader_replica, to_evict);
     }
     for (const auto& leader_replica : { "D", "E", "F" }) {
       string to_evict;
-      ASSERT_TRUE(CanEvictReplica(config, leader_replica, 3, &to_evict));
+      ASSERT_TRUE(ShouldEvictReplica(config, leader_replica, 3, &to_evict));
       EXPECT_TRUE(to_evict == "A" || to_evict == "B" || to_evict == "C");
     }
-    EXPECT_FALSE(IsUnderReplicated(config, 3));
+    EXPECT_FALSE(ShouldAddReplica(config, 3));
   }
 }
 
@@ -1152,33 +1153,33 @@ TEST(QuorumUtilTest, NewlyPromotedReplicaCrashes) {
   AddPeer(&config, "B", V, '+');
   AddPeer(&config, "C", V, '+');
 
-  EXPECT_FALSE(CanEvictReplica(config, "A", kReplicationFactor));
-  EXPECT_FALSE(IsUnderReplicated(config, kReplicationFactor));
+  EXPECT_FALSE(ShouldEvictReplica(config, "A", kReplicationFactor));
+  EXPECT_FALSE(ShouldAddReplica(config, kReplicationFactor));
 
   // Replica B fails.
   SetPeerHealth(&config, "B", '-');
-  EXPECT_FALSE(CanEvictReplica(config, "A", kReplicationFactor));
-  EXPECT_TRUE(IsUnderReplicated(config, kReplicationFactor));
+  EXPECT_FALSE(ShouldEvictReplica(config, "A", kReplicationFactor));
+  EXPECT_TRUE(ShouldAddReplica(config, kReplicationFactor));
 
   // Adding a non-voter to replace B.
   AddPeer(&config, "D", N, '?', {{"PROMOTE", true}});
-  EXPECT_FALSE(CanEvictReplica(config, "A", kReplicationFactor));
-  EXPECT_FALSE(IsUnderReplicated(config, kReplicationFactor));
+  EXPECT_FALSE(ShouldEvictReplica(config, "A", kReplicationFactor));
+  EXPECT_FALSE(ShouldAddReplica(config, kReplicationFactor));
 
   // The new non-voter replica becomes healthy.
   SetPeerHealth(&config, "D", '+');
-  EXPECT_FALSE(CanEvictReplica(config, "A", kReplicationFactor));
-  EXPECT_FALSE(IsUnderReplicated(config, kReplicationFactor));
+  EXPECT_FALSE(ShouldEvictReplica(config, "A", kReplicationFactor));
+  EXPECT_FALSE(ShouldAddReplica(config, kReplicationFactor));
 
   // The newly added non-voter replica is promoted.
   PromotePeer(&config, "D");
   {
     // B would be evicted, if it's reported as is.
     string to_evict;
-    ASSERT_TRUE(CanEvictReplica(config, "A", kReplicationFactor, &to_evict));
+    ASSERT_TRUE(ShouldEvictReplica(config, "A", kReplicationFactor, &to_evict));
     EXPECT_EQ("B", to_evict);
   }
-  EXPECT_FALSE(IsUnderReplicated(config, kReplicationFactor));
+  EXPECT_FALSE(ShouldAddReplica(config, kReplicationFactor));
 
   // However, the newly promoted replica crashes prior to B getting evicted.
   // The system should add a new replica for replacement.
@@ -1187,36 +1188,36 @@ TEST(QuorumUtilTest, NewlyPromotedReplicaCrashes) {
   // the eviction config change.
   SetPeerHealth(&config, "D", '?');
   string to_evict;
-  ASSERT_TRUE(CanEvictReplica(config, "A", kReplicationFactor, &to_evict));
+  ASSERT_TRUE(ShouldEvictReplica(config, "A", kReplicationFactor, &to_evict));
   EXPECT_EQ("B", to_evict);
-  EXPECT_FALSE(IsUnderReplicated(config, kReplicationFactor));
+  EXPECT_FALSE(ShouldAddReplica(config, kReplicationFactor));
 
   SetPeerHealth(&config, "D", '-');
-  ASSERT_TRUE(CanEvictReplica(config, "A", kReplicationFactor, &to_evict));
+  ASSERT_TRUE(ShouldEvictReplica(config, "A", kReplicationFactor, &to_evict));
   EXPECT_TRUE(to_evict == "B" || to_evict == "D") << to_evict;
-  EXPECT_TRUE(IsUnderReplicated(config, kReplicationFactor));
+  EXPECT_TRUE(ShouldAddReplica(config, kReplicationFactor));
 
   RemovePeer(&config, to_evict);
-  EXPECT_FALSE(CanEvictReplica(config, "A", kReplicationFactor));
-  EXPECT_TRUE(IsUnderReplicated(config, kReplicationFactor));
+  EXPECT_FALSE(ShouldEvictReplica(config, "A", kReplicationFactor));
+  EXPECT_TRUE(ShouldAddReplica(config, kReplicationFactor));
 
   AddPeer(&config, "E", N, '?', {{"PROMOTE", true}});
-  EXPECT_FALSE(CanEvictReplica(config, "A", kReplicationFactor));
-  EXPECT_FALSE(IsUnderReplicated(config, kReplicationFactor));
+  EXPECT_FALSE(ShouldEvictReplica(config, "A", kReplicationFactor));
+  EXPECT_FALSE(ShouldAddReplica(config, kReplicationFactor));
 
   SetPeerHealth(&config, "E", '+');
-  EXPECT_FALSE(CanEvictReplica(config, "A", kReplicationFactor));
-  EXPECT_FALSE(IsUnderReplicated(config, kReplicationFactor));
+  EXPECT_FALSE(ShouldEvictReplica(config, "A", kReplicationFactor));
+  EXPECT_FALSE(ShouldAddReplica(config, kReplicationFactor));
 
   PromotePeer(&config, "E");
-  ASSERT_TRUE(CanEvictReplica(config, "A", kReplicationFactor, &to_evict));
+  ASSERT_TRUE(ShouldEvictReplica(config, "A", kReplicationFactor, &to_evict));
   EXPECT_TRUE(to_evict == "B" || to_evict == "D") << to_evict;
-  EXPECT_FALSE(IsUnderReplicated(config, kReplicationFactor));
+  EXPECT_FALSE(ShouldAddReplica(config, kReplicationFactor));
 
   RemovePeer(&config, to_evict);
   // The processs converges: 3 voter replicas, all are healthy.
-  EXPECT_FALSE(CanEvictReplica(config, "A", kReplicationFactor));
-  EXPECT_FALSE(IsUnderReplicated(config, kReplicationFactor));
+  EXPECT_FALSE(ShouldEvictReplica(config, "A", kReplicationFactor));
+  EXPECT_FALSE(ShouldAddReplica(config, kReplicationFactor));
 }
 
 } // namespace consensus

http://git-wip-us.apache.org/repos/asf/kudu/blob/83b19ef3/src/kudu/consensus/quorum_util.cc
----------------------------------------------------------------------
diff --git a/src/kudu/consensus/quorum_util.cc b/src/kudu/consensus/quorum_util.cc
index 6e6b3c4..8bf400d 100644
--- a/src/kudu/consensus/quorum_util.cc
+++ b/src/kudu/consensus/quorum_util.cc
@@ -401,7 +401,7 @@ string DiffConsensusStates(const ConsensusStatePB& old_state,
 //
 // TODO(aserbin): add a test scenario for the leader replica's logic to cover
 //                the latter case.
-bool IsUnderReplicated(const RaftConfigPB& config, int replication_factor) {
+bool ShouldAddReplica(const RaftConfigPB& config, int replication_factor) {
   int num_voters_total = 0;
   int num_voters_need_replacement = 0;
   int num_non_voters_to_promote = 0;
@@ -438,7 +438,7 @@ bool IsUnderReplicated(const RaftConfigPB& config, int replication_factor) {
 }
 
 // Whether there is an excess replica to evict.
-bool CanEvictReplica(const RaftConfigPB& config,
+bool ShouldEvictReplica(const RaftConfigPB& config,
                      const string& leader_uuid,
                      int replication_factor,
                      string* uuid_to_evict) {

http://git-wip-us.apache.org/repos/asf/kudu/blob/83b19ef3/src/kudu/consensus/quorum_util.h
----------------------------------------------------------------------
diff --git a/src/kudu/consensus/quorum_util.h b/src/kudu/consensus/quorum_util.h
index 9afe13f..d875def 100644
--- a/src/kudu/consensus/quorum_util.h
+++ b/src/kudu/consensus/quorum_util.h
@@ -88,19 +88,21 @@ std::string DiffConsensusStates(const ConsensusStatePB& old_state,
 std::string DiffRaftConfigs(const RaftConfigPB& old_config,
                             const RaftConfigPB& new_config);
 
-// Return true iff the current cluster is under-replicated given the Raft
-// configuration and the included health status of the members.
-bool IsUnderReplicated(const RaftConfigPB& config, int replication_factor);
+// Return 'true' iff the specified tablet configuration is under-replicated
+// given the 'replication_factor' and should add a replica. The decision is
+// based on the health information provided by the Raft configuration
+// in the 'config' parameter.
+bool ShouldAddReplica(const RaftConfigPB& config, int replication_factor);
 
 // Check if the given Raft configuration contains at least one extra replica
-// which can be removed in accordance with the specified replication
-// factor and current Raft leader. If so, then return 'true' and set the UUID
-// of the best suited replica into the 'uuid_to_evict' out parameter. Otherwise,
-// return 'false'.
-bool CanEvictReplica(const RaftConfigPB& config,
-                     const std::string& leader_uuid,
-                     int replication_factor,
-                     std::string* uuid_to_evict = nullptr);
+// which should (and can) be removed in accordance with the specified
+// replication factor and current Raft leader. If so, then return 'true' and set
+// the UUID of the best candidate for eviction into the 'uuid_to_evict'
+// out parameter. Otherwise, return 'false'.
+bool ShouldEvictReplica(const RaftConfigPB& config,
+                        const std::string& leader_uuid,
+                        int replication_factor,
+                        std::string* uuid_to_evict = nullptr);
 
 }  // namespace consensus
 }  // namespace kudu

http://git-wip-us.apache.org/repos/asf/kudu/blob/83b19ef3/src/kudu/master/catalog_manager.cc
----------------------------------------------------------------------
diff --git a/src/kudu/master/catalog_manager.cc b/src/kudu/master/catalog_manager.cc
index 2b7bcc4..2bf7c00 100644
--- a/src/kudu/master/catalog_manager.cc
+++ b/src/kudu/master/catalog_manager.cc
@@ -3508,22 +3508,21 @@ Status CatalogManager::ProcessTabletReport(
       // consider whether to add or evict replicas based on the health report
       // included in the leader's tablet report. Since only the leader tracks
       // health, we ignore reports from non-leaders in this case. Also, making
-      // the changes recommended by CanEvictReplica()/IsUnderReplicated()
-      // assumes that the leader replica has already committed the configuration
-      // it's working with.
+      // the changes recommended by Should{Add,Evict}Replica() assumes that the
+      // leader replica has already committed the configuration it's working with.
       } else if (!cstate.has_pending_config() &&
                  !cstate.leader_uuid().empty() &&
                  cstate.leader_uuid() == ts_desc->permanent_uuid()) {
         const RaftConfigPB& config = cstate.committed_config();
         string to_evict;
         if (PREDICT_TRUE(FLAGS_catalog_manager_evict_excess_replicas) &&
-                         CanEvictReplica(config, cstate.leader_uuid(), replication_factor,
-                                         &to_evict)) {
+                         ShouldEvictReplica(config, cstate.leader_uuid(),
+                                            replication_factor, &to_evict)) {
           DCHECK(!to_evict.empty());
           rpcs.emplace_back(new AsyncEvictReplicaTask(
               master_, tablet, cstate, std::move(to_evict)));
         } else if (FLAGS_master_add_server_when_underreplicated &&
-                   IsUnderReplicated(config, replication_factor)) {
+                   ShouldAddReplica(config, replication_factor)) {
           rpcs.emplace_back(new AsyncAddReplicaTask(
               master_, tablet, cstate, RaftPeerPB::NON_VOTER, &rng_));
         }