You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@arrow.apache.org by ap...@apache.org on 2020/08/17 18:07:23 UTC

[arrow] branch master updated: ARROW-9710: [C++] Improve performance of Decimal128::ToString by 10x, and make the implementation reusable for Decimal256.

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

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


The following commit(s) were added to refs/heads/master by this push:
     new 2f98d1e  ARROW-9710: [C++] Improve performance of Decimal128::ToString by 10x, and make the implementation reusable for Decimal256.
2f98d1e is described below

commit 2f98d1e65ef8821aecd9f716cfd6176f21315969
Author: Mingyu Zhong <my...@google.com>
AuthorDate: Mon Aug 17 20:06:54 2020 +0200

    ARROW-9710: [C++] Improve performance of Decimal128::ToString by 10x, and make the implementation reusable for Decimal256.
    
    Added a benchmark to decimal_benchmark.
    
    ```
    Running release/arrow-decimal-benchmark
    Run on (12 X 4500 MHz CPU s)
    CPU Caches:
      L1 Data 32 KiB (x6)
      L1 Instruction 32 KiB (x6)
      L2 Unified 1024 KiB (x6)
      L3 Unified 8448 KiB (x1)
    Load Average: 0.23, 0.22, 0.23
    ----------------------------------------------------------------------------------
    Benchmark                        Time             CPU   Iterations UserCounters...
    ----------------------------------------------------------------------------------
    ToString (before)             2205 ns         2205 ns       321603 items_per_second=2.72094M/s
    ToString (after)               216 ns          216 ns      3235082 items_per_second=27.7865M/s
    ```
    
    Closes #7945 from MingyuZhong/decimal_to_string
    
    Lead-authored-by: Mingyu Zhong <my...@google.com>
    Co-authored-by: Antoine Pitrou <an...@python.org>
    Signed-off-by: Antoine Pitrou <an...@python.org>
---
 cpp/src/arrow/compute/kernels/scalar_string.cc |   4 +-
 cpp/src/arrow/util/decimal.cc                  | 217 +++++++++++++++----------
 cpp/src/arrow/util/decimal_benchmark.cc        |  46 +++++-
 cpp/src/arrow/util/decimal_test.cc             | 141 ++++++++++++----
 4 files changed, 280 insertions(+), 128 deletions(-)

diff --git a/cpp/src/arrow/compute/kernels/scalar_string.cc b/cpp/src/arrow/compute/kernels/scalar_string.cc
index 0332be9..7e61617 100644
--- a/cpp/src/arrow/compute/kernels/scalar_string.cc
+++ b/cpp/src/arrow/compute/kernels/scalar_string.cc
@@ -861,10 +861,10 @@ void AddBinaryLength(FunctionRegistry* registry) {
       applicator::ScalarUnaryNotNull<Int32Type, StringType, BinaryLength>::Exec;
   ArrayKernelExec exec_offset_64 =
       applicator::ScalarUnaryNotNull<Int64Type, LargeStringType, BinaryLength>::Exec;
-  for (const auto input_type : {binary(), utf8()}) {
+  for (const auto& input_type : {binary(), utf8()}) {
     DCHECK_OK(func->AddKernel({input_type}, int32(), exec_offset_32));
   }
-  for (const auto input_type : {large_binary(), large_utf8()}) {
+  for (const auto& input_type : {large_binary(), large_utf8()}) {
     DCHECK_OK(func->AddKernel({input_type}, int64(), exec_offset_64));
   }
   DCHECK_OK(registry->AddFunction(std::move(func)));
diff --git a/cpp/src/arrow/util/decimal.cc b/cpp/src/arrow/util/decimal.cc
index c9c63f8..9df9222 100644
--- a/cpp/src/arrow/util/decimal.cc
+++ b/cpp/src/arrow/util/decimal.cc
@@ -31,6 +31,7 @@
 #include "arrow/status.h"
 #include "arrow/util/bit_util.h"
 #include "arrow/util/decimal.h"
+#include "arrow/util/formatting.h"
 #include "arrow/util/int_util_internal.h"
 #include "arrow/util/logging.h"
 #include "arrow/util/macros.h"
@@ -45,10 +46,6 @@ Decimal128::Decimal128(const std::string& str) : Decimal128() {
   *this = Decimal128::FromString(str).ValueOrDie();
 }
 
-static const Decimal128 kTenTo36(static_cast<int64_t>(0xC097CE7BC90715),
-                                 0xB34B9F1000000000);
-static const Decimal128 kTenTo18(0xDE0B6B3A7640000);
-
 static constexpr auto kInt64DecimalDigits =
     static_cast<size_t>(std::numeric_limits<int64_t>::digits10);
 
@@ -195,42 +192,86 @@ double Decimal128::ToDouble(int32_t scale) const {
   return DecimalDoubleConversion::ToReal(*this, scale);
 }
 
-std::string Decimal128::ToIntegerString() const {
-  Decimal128 remainder;
-  std::stringstream buf;
-  bool need_fill = false;
-
-  // get anything above 10 ** 36 and print it
-  Decimal128 top;
-  std::tie(top, remainder) = Divide(kTenTo36).ValueOrDie();
-
-  if (top != 0) {
-    buf << static_cast<int64_t>(top);
-    remainder.Abs();
-    need_fill = true;
-  }
-
-  // now get anything above 10 ** 18 and print it
-  Decimal128 tail;
-  std::tie(top, tail) = remainder.Divide(kTenTo18).ValueOrDie();
-
-  if (need_fill || top != 0) {
-    if (need_fill) {
-      buf << std::setw(18) << std::setfill('0');
-    } else {
-      need_fill = true;
-      tail.Abs();
-    }
-
-    buf << static_cast<int64_t>(top);
-  }
+template <size_t n>
+static void AppendLittleEndianArrayToString(const std::array<uint64_t, n>& array,
+                                            std::string* result) {
+  const auto most_significant_non_zero =
+      find_if(array.rbegin(), array.rend(), [](uint64_t v) { return v != 0; });
+  if (most_significant_non_zero == array.rend()) {
+    result->push_back('0');
+    return;
+  }
+
+  size_t most_significant_elem_idx = &*most_significant_non_zero - array.data();
+  std::array<uint64_t, n> copy = array;
+  constexpr uint32_t k1e9 = 1000000000U;
+  constexpr size_t kNumBits = n * 64;
+  // Segments will contain the array split into groups that map to decimal digits,
+  // in little endian order. Each segment will hold at most 9 decimal digits.
+  // For example, if the input represents 9876543210123456789, then segments will be
+  // [123456789, 876543210, 9].
+  // The max number of segments needed = ceil(kNumBits * log(2) / log(1e9))
+  // = ceil(kNumBits / 29.897352854) <= ceil(kNumBits / 29).
+  std::array<uint32_t, (kNumBits + 28) / 29> segments;
+  size_t num_segments = 0;
+  uint64_t* most_significant_elem = &copy[most_significant_elem_idx];
+  do {
+    // Compute remainder = copy % 1e9 and copy = copy / 1e9.
+    uint32_t remainder = 0;
+    uint64_t* elem = most_significant_elem;
+    do {
+      // Compute dividend = (remainder << 32) | *elem  (a virtual 96-bit integer);
+      // *elem = dividend / 1e9;
+      // remainder = dividend % 1e9.
+      uint32_t hi = static_cast<uint32_t>(*elem >> 32);
+      uint32_t lo = static_cast<uint32_t>(*elem & BitUtil::LeastSignficantBitMask(32));
+      uint64_t dividend_hi = (static_cast<uint64_t>(remainder) << 32) | hi;
+      uint64_t quotient_hi = dividend_hi / k1e9;
+      remainder = static_cast<uint32_t>(dividend_hi % k1e9);
+      uint64_t dividend_lo = (static_cast<uint64_t>(remainder) << 32) | lo;
+      uint64_t quotient_lo = dividend_lo / k1e9;
+      remainder = static_cast<uint32_t>(dividend_lo % k1e9);
+      *elem = (quotient_hi << 32) | quotient_lo;
+    } while (elem-- != copy.data());
+
+    segments[num_segments++] = remainder;
+  } while (*most_significant_elem != 0 || most_significant_elem-- != copy.data());
+
+  size_t old_size = result->size();
+  size_t new_size = old_size + num_segments * 9;
+  result->resize(new_size, '0');
+  char* output = &result->at(old_size);
+  const uint32_t* segment = &segments[num_segments - 1];
+  internal::StringFormatter<UInt32Type> format;
+  // First segment is formatted as-is.
+  format(*segment, [&output](util::string_view formatted) {
+    memcpy(output, formatted.data(), formatted.size());
+    output += formatted.size();
+  });
+  while (segment != segments.data()) {
+    --segment;
+    // Right-pad formatted segment such that e.g. 123 is formatted as "000000123".
+    output += 9;
+    format(*segment, [output](util::string_view formatted) {
+      memcpy(output - formatted.size(), formatted.data(), formatted.size());
+    });
+  }
+  result->resize(output - result->data());
+}
 
-  // finally print the tail, which is less than 10**18
-  if (need_fill) {
-    buf << std::setw(18) << std::setfill('0');
+std::string Decimal128::ToIntegerString() const {
+  std::string result;
+  if (high_bits() < 0) {
+    result.push_back('-');
+    Decimal128 abs = *this;
+    abs.Negate();
+    AppendLittleEndianArrayToString<2>(
+        {abs.low_bits(), static_cast<uint64_t>(abs.high_bits())}, &result);
+  } else {
+    AppendLittleEndianArrayToString<2>({low_bits(), static_cast<uint64_t>(high_bits())},
+                                       &result);
   }
-  buf << static_cast<int64_t>(tail);
-  return buf.str();
+  return result;
 }
 
 Decimal128::operator int64_t() const {
@@ -241,64 +282,70 @@ Decimal128::operator int64_t() const {
   return static_cast<int64_t>(low_bits());
 }
 
-static std::string ToStringNegativeScale(const std::string& str,
-                                         int32_t adjusted_exponent, bool is_negative) {
-  std::stringstream buf;
-
-  size_t offset = 0;
-  buf << str[offset++];
-
-  if (is_negative) {
-    buf << str[offset++];
-  }
-
-  buf << '.' << str.substr(offset, std::string::npos) << 'E' << std::showpos
-      << adjusted_exponent;
-  return buf.str();
-}
-
-std::string Decimal128::ToString(int32_t scale) const {
-  const std::string str(ToIntegerString());
-
+static void AdjustIntegerStringWithScale(int32_t scale, std::string* str) {
   if (scale == 0) {
-    return str;
+    return;
   }
-
-  const bool is_negative = *this < 0;
-
-  const auto len = static_cast<int32_t>(str.size());
+  DCHECK(str != nullptr);
+  DCHECK(!str->empty());
+  const bool is_negative = str->front() == '-';
   const auto is_negative_offset = static_cast<int32_t>(is_negative);
-  const int32_t adjusted_exponent = -scale + (len - 1 - is_negative_offset);
+  const auto len = static_cast<int32_t>(str->size());
+  const int32_t num_digits = len - is_negative_offset;
+  const int32_t adjusted_exponent = num_digits - 1 - scale;
 
   /// Note that the -6 is taken from the Java BigDecimal documentation.
   if (scale < 0 || adjusted_exponent < -6) {
-    return ToStringNegativeScale(str, adjusted_exponent, is_negative);
-  }
-
-  if (is_negative) {
-    if (len - 1 > scale) {
-      const auto n = static_cast<size_t>(len - scale);
-      return str.substr(0, n) + "." + str.substr(n, static_cast<size_t>(scale));
+    // Example 1:
+    // Precondition: *str = "123", is_negative_offset = 0, num_digits = 3, scale = -2,
+    //               adjusted_exponent = 4
+    // After inserting decimal point: *str = "1.23"
+    // After appending exponent: *str = "1.23E+4"
+    // Example 2:
+    // Precondition: *str = "-123", is_negative_offset = 1, num_digits = 3, scale = 9,
+    //               adjusted_exponent = -7
+    // After inserting decimal point: *str = "-1.23"
+    // After appending exponent: *str = "-1.23E-7"
+    str->insert(str->begin() + 1 + is_negative_offset, '.');
+    str->push_back('E');
+    if (adjusted_exponent >= 0) {
+      str->push_back('+');
     }
-
-    if (len - 1 == scale) {
-      return "-0." + str.substr(1, std::string::npos);
-    }
-
-    std::string result("-0." + std::string(static_cast<size_t>(scale - len + 1), '0'));
-    return result + str.substr(1, std::string::npos);
+    internal::StringFormatter<Int32Type> format;
+    format(adjusted_exponent, [str](util::string_view formatted) {
+      str->append(formatted.data(), formatted.size());
+    });
+    return;
   }
 
-  if (len > scale) {
+  if (num_digits > scale) {
     const auto n = static_cast<size_t>(len - scale);
-    return str.substr(0, n) + "." + str.substr(n, static_cast<size_t>(scale));
-  }
-
-  if (len == scale) {
-    return "0." + str;
-  }
+    // Example 1:
+    // Precondition: *str = "123", len = num_digits = 3, scale = 1, n = 2
+    // After inserting decimal point: *str = "12.3"
+    // Example 2:
+    // Precondition: *str = "-123", len = 4, num_digits = 3, scale = 1, n = 3
+    // After inserting decimal point: *str = "-12.3"
+    str->insert(str->begin() + n, '.');
+    return;
+  }
+
+  // Example 1:
+  // Precondition: *str = "123", is_negative_offset = 0, num_digits = 3, scale = 4
+  // After insert: *str = "000123"
+  // After setting decimal point: *str = "0.0123"
+  // Example 2:
+  // Precondition: *str = "-123", is_negative_offset = 1, num_digits = 3, scale = 4
+  // After insert: *str = "-000123"
+  // After setting decimal point: *str = "-0.0123"
+  str->insert(is_negative_offset, scale - num_digits + 2, '0');
+  str->at(is_negative_offset + 1) = '.';
+}
 
-  return "0." + std::string(static_cast<size_t>(scale - len), '0') + str;
+std::string Decimal128::ToString(int32_t scale) const {
+  std::string str(ToIntegerString());
+  AdjustIntegerStringWithScale(scale, &str);
+  return str;
 }
 
 // Iterates over data and for each group of kInt64DecimalDigits multiple out by
diff --git a/cpp/src/arrow/util/decimal_benchmark.cc b/cpp/src/arrow/util/decimal_benchmark.cc
index 620593c..c1acefc 100644
--- a/cpp/src/arrow/util/decimal_benchmark.cc
+++ b/cpp/src/arrow/util/decimal_benchmark.cc
@@ -21,19 +21,40 @@
 #include <vector>
 
 #include "arrow/util/decimal.h"
+#include "arrow/util/logging.h"
 #include "arrow/util/macros.h"
 
 namespace arrow {
 namespace Decimal {
 
-static void FromString(benchmark::State& state) {  // NOLINT non-const reference
-  std::vector<std::string> values = {"0",
-                                     "1.23",
-                                     "12.345e6",
-                                     "-12.345e-6",
-                                     "123456789.123456789",
-                                     "1231234567890.451234567890"};
+static const std::vector<std::string>& GetValuesAsString() {
+  static const std::vector<std::string> kValues = {"0",
+                                                   "1.23",
+                                                   "12.345e6",
+                                                   "-12.345e-6",
+                                                   "123456789.123456789",
+                                                   "1231234567890.451234567890"};
+  return kValues;
+}
+
+struct DecimalValueAndScale {
+  Decimal128 decimal;
+  int32_t scale;
+};
+
+static std::vector<DecimalValueAndScale> GetDecimalValuesAndScales() {
+  const std::vector<std::string>& value_strs = GetValuesAsString();
+  std::vector<DecimalValueAndScale> result(value_strs.size());
+  for (size_t i = 0; i < value_strs.size(); ++i) {
+    int32_t precision;
+    ARROW_CHECK_OK(Decimal128::FromString(value_strs[i], &result[i].decimal,
+                                          &result[i].scale, &precision));
+  }
+  return result;
+}
 
+static void FromString(benchmark::State& state) {  // NOLINT non-const reference
+  const std::vector<std::string>& values = GetValuesAsString();
   for (auto _ : state) {
     for (const auto& value : values) {
       Decimal128 dec;
@@ -44,6 +65,16 @@ static void FromString(benchmark::State& state) {  // NOLINT non-const reference
   state.SetItemsProcessed(state.iterations() * values.size());
 }
 
+static void ToString(benchmark::State& state) {  // NOLINT non-const reference
+  static const std::vector<DecimalValueAndScale> values = GetDecimalValuesAndScales();
+  for (auto _ : state) {
+    for (const DecimalValueAndScale& item : values) {
+      benchmark::DoNotOptimize(item.decimal.ToString(item.scale));
+    }
+  }
+  state.SetItemsProcessed(state.iterations() * values.size());
+}
+
 constexpr int32_t kValueSize = 10;
 
 static void BinaryCompareOp(benchmark::State& state) {  // NOLINT non-const reference
@@ -158,6 +189,7 @@ static void BinaryBitOp(benchmark::State& state) {  // NOLINT non-const referenc
 }
 
 BENCHMARK(FromString);
+BENCHMARK(ToString);
 BENCHMARK(BinaryMathOp);
 BENCHMARK(BinaryMathOpAggregate);
 BENCHMARK(BinaryCompareOp);
diff --git a/cpp/src/arrow/util/decimal_test.cc b/cpp/src/arrow/util/decimal_test.cc
index 856f10e..d037f1c 100644
--- a/cpp/src/arrow/util/decimal_test.cc
+++ b/cpp/src/arrow/util/decimal_test.cc
@@ -45,13 +45,6 @@ class DecimalTestFixture : public ::testing::Test {
   std::string string_value_;
 };
 
-TEST_F(DecimalTestFixture, TestToString) {
-  Decimal128 decimal(this->integer_value_);
-  int32_t scale = 5;
-  std::string result = decimal.ToString(scale);
-  ASSERT_EQ(result, this->string_value_);
-}
-
 TEST_F(DecimalTestFixture, TestFromString) {
   Decimal128 expected(this->integer_value_);
   Decimal128 result;
@@ -106,6 +99,35 @@ TEST(DecimalTest, TestFromDecimalString128) {
   ASSERT_NE(result.high_bits(), 0);
 }
 
+TEST(DecimalTest, TestStringRoundTrip) {
+  static constexpr uint64_t kTestBits[] = {
+      0,
+      1,
+      999,
+      1000,
+      std::numeric_limits<int32_t>::max(),
+      (1ull << 31),
+      std::numeric_limits<uint32_t>::max(),
+      (1ull << 32),
+      std::numeric_limits<int64_t>::max(),
+      (1ull << 63),
+      std::numeric_limits<uint64_t>::max(),
+  };
+  static constexpr int32_t kScales[] = {-10, -1, 0, 1, 10};
+  for (uint64_t high_bits : kTestBits) {
+    for (uint64_t low_bits : kTestBits) {
+      // When high_bits = 1ull << 63 or std::numeric_limits<uint64_t>::max(), decimal is
+      // negative.
+      Decimal128 decimal(high_bits, low_bits);
+      for (int32_t scale : kScales) {
+        std::string str = decimal.ToString(scale);
+        ASSERT_OK_AND_ASSIGN(Decimal128 result, Decimal128::FromString(str));
+        EXPECT_EQ(decimal, result);
+      }
+    }
+  }
+}
+
 TEST(DecimalTest, TestDecimal32SignedRoundTrip) {
   Decimal128 expected("-3402692");
 
@@ -291,28 +313,88 @@ TEST(Decimal128Test, PrintMinValue) {
   ASSERT_EQ(string_value, printed_value);
 }
 
-class Decimal128PrintingTest
-    : public ::testing::TestWithParam<std::tuple<int32_t, int32_t, std::string>> {};
-
-TEST_P(Decimal128PrintingTest, Print) {
-  int32_t test_value;
+struct ToStringTestData {
+  int64_t test_value;
   int32_t scale;
   std::string expected_string;
-  std::tie(test_value, scale, expected_string) = GetParam();
-  const Decimal128 value(test_value);
-  const std::string printed_value = value.ToString(scale);
-  ASSERT_EQ(expected_string, printed_value);
+};
+
+static const ToStringTestData kToStringTestData[] = {
+    {0, -1, "0.E+1"},
+    {0, 0, "0"},
+    {0, 1, "0.0"},
+    {0, 6, "0.000000"},
+    {2, 7, "2.E-7"},
+    {2, -1, "2.E+1"},
+    {2, 0, "2"},
+    {2, 1, "0.2"},
+    {2, 6, "0.000002"},
+    {-2, 7, "-2.E-7"},
+    {-2, 7, "-2.E-7"},
+    {-2, -1, "-2.E+1"},
+    {-2, 0, "-2"},
+    {-2, 1, "-0.2"},
+    {-2, 6, "-0.000002"},
+    {-2, 7, "-2.E-7"},
+    {123, -3, "1.23E+5"},
+    {123, -1, "1.23E+3"},
+    {123, 1, "12.3"},
+    {123, 0, "123"},
+    {123, 5, "0.00123"},
+    {123, 8, "0.00000123"},
+    {123, 9, "1.23E-7"},
+    {123, 10, "1.23E-8"},
+    {-123, -3, "-1.23E+5"},
+    {-123, -1, "-1.23E+3"},
+    {-123, 1, "-12.3"},
+    {-123, 0, "-123"},
+    {-123, 5, "-0.00123"},
+    {-123, 8, "-0.00000123"},
+    {-123, 9, "-1.23E-7"},
+    {-123, 10, "-1.23E-8"},
+    {1000000000, -3, "1.000000000E+12"},
+    {1000000000, -1, "1.000000000E+10"},
+    {1000000000, 0, "1000000000"},
+    {1000000000, 1, "100000000.0"},
+    {1000000000, 5, "10000.00000"},
+    {1000000000, 15, "0.000001000000000"},
+    {1000000000, 16, "1.000000000E-7"},
+    {1000000000, 17, "1.000000000E-8"},
+    {-1000000000, -3, "-1.000000000E+12"},
+    {-1000000000, -1, "-1.000000000E+10"},
+    {-1000000000, 0, "-1000000000"},
+    {-1000000000, 1, "-100000000.0"},
+    {-1000000000, 5, "-10000.00000"},
+    {-1000000000, 15, "-0.000001000000000"},
+    {-1000000000, 16, "-1.000000000E-7"},
+    {-1000000000, 17, "-1.000000000E-8"},
+    {1234567890123456789LL, -3, "1.234567890123456789E+21"},
+    {1234567890123456789LL, -1, "1.234567890123456789E+19"},
+    {1234567890123456789LL, 0, "1234567890123456789"},
+    {1234567890123456789LL, 1, "123456789012345678.9"},
+    {1234567890123456789LL, 5, "12345678901234.56789"},
+    {1234567890123456789LL, 24, "0.000001234567890123456789"},
+    {1234567890123456789LL, 25, "1.234567890123456789E-7"},
+    {-1234567890123456789LL, -3, "-1.234567890123456789E+21"},
+    {-1234567890123456789LL, -1, "-1.234567890123456789E+19"},
+    {-1234567890123456789LL, 0, "-1234567890123456789"},
+    {-1234567890123456789LL, 1, "-123456789012345678.9"},
+    {-1234567890123456789LL, 5, "-12345678901234.56789"},
+    {-1234567890123456789LL, 24, "-0.000001234567890123456789"},
+    {-1234567890123456789LL, 25, "-1.234567890123456789E-7"},
+};
+
+class Decimal128ToStringTest : public ::testing::TestWithParam<ToStringTestData> {};
+
+TEST_P(Decimal128ToStringTest, ToString) {
+  const ToStringTestData& data = GetParam();
+  const Decimal128 value(data.test_value);
+  const std::string printed_value = value.ToString(data.scale);
+  ASSERT_EQ(data.expected_string, printed_value);
 }
 
-INSTANTIATE_TEST_SUITE_P(Decimal128PrintingTest, Decimal128PrintingTest,
-                         ::testing::Values(std::make_tuple(123, 1, "12.3"),
-                                           std::make_tuple(123, 5, "0.00123"),
-                                           std::make_tuple(123, 10, "1.23E-8"),
-                                           std::make_tuple(123, -1, "1.23E+3"),
-                                           std::make_tuple(-123, -1, "-1.23E+3"),
-                                           std::make_tuple(123, -3, "1.23E+5"),
-                                           std::make_tuple(-123, -3, "-1.23E+5"),
-                                           std::make_tuple(12345, -3, "1.2345E+7")));
+INSTANTIATE_TEST_SUITE_P(Decimal128ToStringTest, Decimal128ToStringTest,
+                         ::testing::ValuesIn(kToStringTestData));
 
 class Decimal128ParsingTest
     : public ::testing::TestWithParam<std::tuple<std::string, uint64_t, int32_t>> {};
@@ -706,15 +788,6 @@ TEST_F(TestDecimalToRealDouble, Precision) {
 
 #endif  // __MINGW32__
 
-TEST(Decimal128Test, TestSmallNumberFormat) {
-  Decimal128 value("0.2");
-  std::string expected("0.2");
-
-  const int32_t scale = 1;
-  std::string result = value.ToString(scale);
-  ASSERT_EQ(expected, result);
-}
-
 TEST(Decimal128Test, TestNoDecimalPointExponential) {
   Decimal128 value;
   int32_t precision;