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 2021/12/02 11:09:01 UTC

[arrow] branch master updated: ARROW-13536: [C++] Use decimal-point aware conversion from fast-float

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 b8431fb  ARROW-13536: [C++] Use decimal-point aware conversion from fast-float
b8431fb is described below

commit b8431fba68e2540b3e57def0bd0ad718652c4b98
Author: Antoine Pitrou <an...@python.org>
AuthorDate: Thu Dec 2 12:07:15 2021 +0100

    ARROW-13536: [C++] Use decimal-point aware conversion from fast-float
    
    The custom wrapper is still used for decimals.
    
    Closes #11817 from pitrou/ARROW-13536-fast-float
    
    Authored-by: Antoine Pitrou <an...@python.org>
    Signed-off-by: Antoine Pitrou <an...@python.org>
---
 cpp/src/arrow/csv/converter.cc                     |  36 +-
 cpp/src/arrow/util/value_parsing.cc                |  14 +-
 cpp/src/arrow/util/value_parsing.h                 |  41 +-
 cpp/src/arrow/util/value_parsing_test.cc           |  83 ++-
 cpp/src/arrow/vendored/fast_float/README.md        |   2 +-
 cpp/src/arrow/vendored/fast_float/ascii_number.h   | 214 +++-----
 cpp/src/arrow/vendored/fast_float/bigint.h         | 592 +++++++++++++++++++++
 .../arrow/vendored/fast_float/decimal_to_binary.h  |  44 +-
 .../arrow/vendored/fast_float/digit_comparison.h   | 409 ++++++++++++++
 cpp/src/arrow/vendored/fast_float/fast_float.h     |  18 +
 cpp/src/arrow/vendored/fast_float/fast_table.h     |  16 +-
 cpp/src/arrow/vendored/fast_float/float_common.h   | 240 +++++----
 cpp/src/arrow/vendored/fast_float/parse_number.h   |  52 +-
 .../fast_float/simple_decimal_conversion.h         |  44 +-
 14 files changed, 1460 insertions(+), 345 deletions(-)

diff --git a/cpp/src/arrow/csv/converter.cc b/cpp/src/arrow/csv/converter.cc
index 4d00cbe..c395c27 100644
--- a/cpp/src/arrow/csv/converter.cc
+++ b/cpp/src/arrow/csv/converter.cc
@@ -195,18 +195,34 @@ struct BinaryValueDecoder : public ValueDecoder {
 // Value decoder for integers, floats and temporals
 //
 
+template <typename T, typename Enable = void>
+struct StringConverterFromOptions {
+  static internal::StringConverter<T> Make(const ConvertOptions&) {
+    return internal::StringConverter<T>{};
+  }
+};
+
+template <typename T>
+struct StringConverterFromOptions<T, enable_if_floating_point<T>> {
+  static internal::StringConverter<T> Make(const ConvertOptions& options) {
+    return internal::StringConverter<T>{options.decimal_point};
+  }
+};
+
 template <typename T>
 struct NumericValueDecoder : public ValueDecoder {
   using value_type = typename T::c_type;
 
-  explicit NumericValueDecoder(const std::shared_ptr<DataType>& type,
-                               const ConvertOptions& options)
-      : ValueDecoder(type, options), concrete_type_(checked_cast<const T&>(*type)) {}
+  NumericValueDecoder(const std::shared_ptr<DataType>& type,
+                      const ConvertOptions& options)
+      : ValueDecoder(type, options),
+        concrete_type_(checked_cast<const T&>(*type)),
+        string_converter_(StringConverterFromOptions<T>::Make(options)) {}
 
   Status Decode(const uint8_t* data, uint32_t size, bool quoted, value_type* out) {
     // XXX should quoted values be allowed at all?
     TrimWhiteSpace(&data, &size);
-    if (ARROW_PREDICT_FALSE(!internal::ParseValue<T>(
+    if (ARROW_PREDICT_FALSE(!string_converter_.Convert(
             concrete_type_, reinterpret_cast<const char*>(data), size, out))) {
       return GenericConversionError(type_, data, size);
     }
@@ -215,6 +231,7 @@ struct NumericValueDecoder : public ValueDecoder {
 
  protected:
   const T& concrete_type_;
+  internal::StringConverter<T> string_converter_;
 };
 
 //
@@ -292,8 +309,7 @@ struct DecimalValueDecoder : public ValueDecoder {
 };
 
 //
-// Value decoder wrapper for floating-point and decimals
-// with a non-default decimal point
+// Value decoder wrapper for decimals with a non-default decimal point
 //
 
 template <typename WrappedDecoder>
@@ -686,8 +702,8 @@ Result<std::shared_ptr<Converter>> Converter::Make(const std::shared_ptr<DataTyp
     NUMERIC_CONVERTER_CASE(Type::UINT16, UInt16Type)
     NUMERIC_CONVERTER_CASE(Type::UINT32, UInt32Type)
     NUMERIC_CONVERTER_CASE(Type::UINT64, UInt64Type)
-    REAL_CONVERTER_CASE(Type::FLOAT, FloatType, NumericValueDecoder<FloatType>)
-    REAL_CONVERTER_CASE(Type::DOUBLE, DoubleType, NumericValueDecoder<DoubleType>)
+    NUMERIC_CONVERTER_CASE(Type::FLOAT, FloatType)
+    NUMERIC_CONVERTER_CASE(Type::DOUBLE, DoubleType)
     REAL_CONVERTER_CASE(Type::DECIMAL, Decimal128Type, DecimalValueDecoder)
     NUMERIC_CONVERTER_CASE(Type::DATE32, Date32Type)
     NUMERIC_CONVERTER_CASE(Type::DATE64, Date64Type)
@@ -774,8 +790,8 @@ Result<std::shared_ptr<DictionaryConverter>> DictionaryConverter::Make(
     CONVERTER_CASE(Type::INT64, Int64Type, NumericValueDecoder<Int64Type>)
     CONVERTER_CASE(Type::UINT32, UInt32Type, NumericValueDecoder<UInt32Type>)
     CONVERTER_CASE(Type::UINT64, UInt64Type, NumericValueDecoder<UInt64Type>)
-    REAL_CONVERTER_CASE(Type::FLOAT, FloatType, NumericValueDecoder<FloatType>)
-    REAL_CONVERTER_CASE(Type::DOUBLE, DoubleType, NumericValueDecoder<DoubleType>)
+    CONVERTER_CASE(Type::FLOAT, FloatType, NumericValueDecoder<FloatType>)
+    CONVERTER_CASE(Type::DOUBLE, DoubleType, NumericValueDecoder<DoubleType>)
     REAL_CONVERTER_CASE(Type::DECIMAL, Decimal128Type, DecimalValueDecoder)
     CONVERTER_CASE(Type::FIXED_SIZE_BINARY, FixedSizeBinaryType,
                    FixedSizeBinaryValueDecoder)
diff --git a/cpp/src/arrow/util/value_parsing.cc b/cpp/src/arrow/util/value_parsing.cc
index ccd7674..9249561 100644
--- a/cpp/src/arrow/util/value_parsing.cc
+++ b/cpp/src/arrow/util/value_parsing.cc
@@ -25,13 +25,19 @@
 namespace arrow {
 namespace internal {
 
-bool StringToFloat(const char* s, size_t length, float* out) {
-  const auto res = ::arrow_vendored::fast_float::from_chars(s, s + length, *out);
+bool StringToFloat(const char* s, size_t length, char decimal_point, float* out) {
+  ::arrow_vendored::fast_float::parse_options options{
+      ::arrow_vendored::fast_float::chars_format::general, decimal_point};
+  const auto res =
+      ::arrow_vendored::fast_float::from_chars_advanced(s, s + length, *out, options);
   return res.ec == std::errc() && res.ptr == s + length;
 }
 
-bool StringToFloat(const char* s, size_t length, double* out) {
-  const auto res = ::arrow_vendored::fast_float::from_chars(s, s + length, *out);
+bool StringToFloat(const char* s, size_t length, char decimal_point, double* out) {
+  ::arrow_vendored::fast_float::parse_options options{
+      ::arrow_vendored::fast_float::chars_format::general, decimal_point};
+  const auto res =
+      ::arrow_vendored::fast_float::from_chars_advanced(s, s + length, *out, options);
   return res.ec == std::errc() && res.ptr == s + length;
 }
 
diff --git a/cpp/src/arrow/util/value_parsing.h b/cpp/src/arrow/util/value_parsing.h
index fd28571..fbbbcf1 100644
--- a/cpp/src/arrow/util/value_parsing.h
+++ b/cpp/src/arrow/util/value_parsing.h
@@ -93,7 +93,7 @@ template <>
 struct StringConverter<BooleanType> {
   using value_type = bool;
 
-  static bool Convert(const BooleanType&, const char* s, size_t length, value_type* out) {
+  bool Convert(const BooleanType&, const char* s, size_t length, value_type* out) {
     if (length == 1) {
       // "0" or "1"?
       if (s[0] == '0') {
@@ -129,27 +129,37 @@ struct StringConverter<BooleanType> {
 // - https://github.com/achan001/dtoa-fast
 
 ARROW_EXPORT
-bool StringToFloat(const char* s, size_t length, float* out);
+bool StringToFloat(const char* s, size_t length, char decimal_point, float* out);
 
 ARROW_EXPORT
-bool StringToFloat(const char* s, size_t length, double* out);
+bool StringToFloat(const char* s, size_t length, char decimal_point, double* out);
 
 template <>
 struct StringConverter<FloatType> {
   using value_type = float;
 
-  static bool Convert(const FloatType&, const char* s, size_t length, value_type* out) {
-    return ARROW_PREDICT_TRUE(StringToFloat(s, length, out));
+  explicit StringConverter(char decimal_point = '.') : decimal_point(decimal_point) {}
+
+  bool Convert(const FloatType&, const char* s, size_t length, value_type* out) {
+    return ARROW_PREDICT_TRUE(StringToFloat(s, length, decimal_point, out));
   }
+
+ private:
+  const char decimal_point;
 };
 
 template <>
 struct StringConverter<DoubleType> {
   using value_type = double;
 
-  static bool Convert(const DoubleType&, const char* s, size_t length, value_type* out) {
-    return ARROW_PREDICT_TRUE(StringToFloat(s, length, out));
+  explicit StringConverter(char decimal_point = '.') : decimal_point(decimal_point) {}
+
+  bool Convert(const DoubleType&, const char* s, size_t length, value_type* out) {
+    return ARROW_PREDICT_TRUE(StringToFloat(s, length, decimal_point, out));
   }
+
+ private:
+  const char decimal_point;
 };
 
 // NOTE: HalfFloatType would require a half<->float conversion library
@@ -302,7 +312,7 @@ template <class ARROW_TYPE>
 struct StringToUnsignedIntConverterMixin {
   using value_type = typename ARROW_TYPE::c_type;
 
-  static bool Convert(const ARROW_TYPE&, const char* s, size_t length, value_type* out) {
+  bool Convert(const ARROW_TYPE&, const char* s, size_t length, value_type* out) {
     if (ARROW_PREDICT_FALSE(length == 0)) {
       return false;
     }
@@ -350,7 +360,7 @@ struct StringToSignedIntConverterMixin {
   using value_type = typename ARROW_TYPE::c_type;
   using unsigned_type = typename std::make_unsigned<value_type>::type;
 
-  static bool Convert(const ARROW_TYPE&, const char* s, size_t length, value_type* out) {
+  bool Convert(const ARROW_TYPE&, const char* s, size_t length, value_type* out) {
     static constexpr auto max_positive =
         static_cast<unsigned_type>(std::numeric_limits<value_type>::max());
     // Assuming two's complement
@@ -806,8 +816,7 @@ template <>
 struct StringConverter<TimestampType> {
   using value_type = int64_t;
 
-  static bool Convert(const TimestampType& type, const char* s, size_t length,
-                      value_type* out) {
+  bool Convert(const TimestampType& type, const char* s, size_t length, value_type* out) {
     return ParseTimestampISO8601(s, length, type.unit(), out);
   }
 };
@@ -827,8 +836,7 @@ struct StringConverter<DATE_TYPE, enable_if_date<DATE_TYPE>> {
                                 arrow_vendored::date::days,
                                 std::chrono::milliseconds>::type;
 
-  static bool Convert(const DATE_TYPE& type, const char* s, size_t length,
-                      value_type* out) {
+  bool Convert(const DATE_TYPE& type, const char* s, size_t length, value_type* out) {
     if (ARROW_PREDICT_FALSE(length != 10)) {
       return false;
     }
@@ -860,8 +868,7 @@ struct StringConverter<TIME_TYPE, enable_if_time<TIME_TYPE>> {
   // We allow the following formats for unit == NANO:
   // - "hh:mm:ss.s{7,9}"
 
-  static bool Convert(const TIME_TYPE& type, const char* s, size_t length,
-                      value_type* out) {
+  bool Convert(const TIME_TYPE& type, const char* s, size_t length, value_type* out) {
     const auto unit = type.unit();
     std::chrono::seconds since_midnight;
 
@@ -906,14 +913,14 @@ struct StringConverter<TIME_TYPE, enable_if_time<TIME_TYPE>> {
 template <typename T>
 bool ParseValue(const T& type, const char* s, size_t length,
                 typename StringConverter<T>::value_type* out) {
-  return StringConverter<T>::Convert(type, s, length, out);
+  return StringConverter<T>{}.Convert(type, s, length, out);
 }
 
 template <typename T>
 enable_if_parameter_free<T, bool> ParseValue(
     const char* s, size_t length, typename StringConverter<T>::value_type* out) {
   static T type;
-  return StringConverter<T>::Convert(type, s, length, out);
+  return StringConverter<T>{}.Convert(type, s, length, out);
 }
 
 }  // namespace internal
diff --git a/cpp/src/arrow/util/value_parsing_test.cc b/cpp/src/arrow/util/value_parsing_test.cc
index 708d5ec..2a986f3 100644
--- a/cpp/src/arrow/util/value_parsing_test.cc
+++ b/cpp/src/arrow/util/value_parsing_test.cc
@@ -15,7 +15,9 @@
 // specific language governing permissions and limitations
 // under the License.
 
+#include <cmath>
 #include <string>
+#include <type_traits>
 #include <vector>
 
 #include <gtest/gtest.h>
@@ -28,11 +30,44 @@ namespace arrow {
 namespace internal {
 
 template <typename T>
+void AssertValueEquals(T a, T b) {
+  ASSERT_EQ(a, b);
+}
+
+template <>
+void AssertValueEquals<float>(float a, float b) {
+  ASSERT_EQ(a, b);
+  ASSERT_EQ(std::signbit(a), std::signbit(b));
+}
+
+template <>
+void AssertValueEquals<double>(double a, double b) {
+  ASSERT_EQ(a, b);
+  ASSERT_EQ(std::signbit(a), std::signbit(b));
+}
+
+template <typename T>
+void AssertConversion(StringConverter<T>* converter, const T& type, const std::string& s,
+                      typename T::c_type expected) {
+  ARROW_SCOPED_TRACE("When converting: '", s, "', expecting: ", expected);
+  typename T::c_type out{};
+  ASSERT_TRUE(converter->Convert(type, s.data(), s.length(), &out));
+  AssertValueEquals(out, expected);
+}
+
+template <typename T>
+void AssertConversion(StringConverter<T>* converter, const std::string& s,
+                      typename T::c_type expected) {
+  auto type = checked_pointer_cast<T>(TypeTraits<T>::type_singleton());
+  AssertConversion(converter, *type, s, expected);
+}
+
+template <typename T>
 void AssertConversion(const T& type, const std::string& s, typename T::c_type expected) {
+  ARROW_SCOPED_TRACE("When converting: '", s, "', expecting: ", expected);
   typename T::c_type out{};
-  ASSERT_TRUE(ParseValue(type, s.data(), s.length(), &out))
-      << "Conversion failed for '" << s << "' (expected to return " << expected << ")";
-  ASSERT_EQ(out, expected) << "Conversion failed for '" << s << "'";
+  ASSERT_TRUE(ParseValue(type, s.data(), s.length(), &out));
+  AssertValueEquals(out, expected);
 }
 
 template <typename T>
@@ -42,6 +77,20 @@ void AssertConversion(const std::string& s, typename T::c_type expected) {
 }
 
 template <typename T>
+void AssertConversionFails(StringConverter<T>* converter, const T& type,
+                           const std::string& s) {
+  typename T::c_type out{};
+  ASSERT_FALSE(converter->Convert(type, s.data(), s.length(), &out))
+      << "Conversion should have failed for '" << s << "' (returned " << out << ")";
+}
+
+template <typename T>
+void AssertConversionFails(StringConverter<T>* converter, const std::string& s) {
+  auto type = checked_pointer_cast<T>(TypeTraits<T>::type_singleton());
+  AssertConversionFails(converter, *type, s);
+}
+
+template <typename T>
 void AssertConversionFails(const T& type, const std::string& s) {
   typename T::c_type out{};
   ASSERT_FALSE(ParseValue(type, s.data(), s.length(), &out))
@@ -68,23 +117,33 @@ TEST(StringConversion, ToBoolean) {
 TEST(StringConversion, ToFloat) {
   AssertConversion<FloatType>("1.5", 1.5f);
   AssertConversion<FloatType>("0", 0.0f);
-  // XXX ASSERT_EQ doesn't distinguish signed zeros
   AssertConversion<FloatType>("-0.0", -0.0f);
   AssertConversion<FloatType>("-1e20", -1e20f);
 
   AssertConversionFails<FloatType>("");
   AssertConversionFails<FloatType>("e");
+  AssertConversionFails<FloatType>("1,5");
+
+  StringConverter<FloatType> converter(/*decimal_point=*/',');
+  AssertConversion(&converter, "1,5", 1.5f);
+  AssertConversion(&converter, "0", 0.0f);
+  AssertConversionFails(&converter, "1.5");
 }
 
 TEST(StringConversion, ToDouble) {
   AssertConversion<DoubleType>("1.5", 1.5);
   AssertConversion<DoubleType>("0", 0);
-  // XXX ASSERT_EQ doesn't distinguish signed zeros
   AssertConversion<DoubleType>("-0.0", -0.0);
   AssertConversion<DoubleType>("-1e100", -1e100);
 
   AssertConversionFails<DoubleType>("");
   AssertConversionFails<DoubleType>("e");
+  AssertConversionFails<DoubleType>("1,5");
+
+  StringConverter<DoubleType> converter(/*decimal_point=*/',');
+  AssertConversion(&converter, "1,5", 1.5);
+  AssertConversion(&converter, "0", 0.0);
+  AssertConversionFails(&converter, "1.5");
 }
 
 #if !defined(_WIN32) || defined(NDEBUG)
@@ -94,13 +153,25 @@ TEST(StringConversion, ToFloatLocale) {
   LocaleGuard locale_guard("fr_FR.UTF-8");
 
   AssertConversion<FloatType>("1.5", 1.5f);
+  AssertConversionFails<FloatType>("1,5");
+
+  StringConverter<FloatType> converter(/*decimal_point=*/'#');
+  AssertConversion(&converter, "1#5", 1.5f);
+  AssertConversionFails(&converter, "1.5");
+  AssertConversionFails(&converter, "1,5");
 }
 
 TEST(StringConversion, ToDoubleLocale) {
   // French locale uses the comma as decimal point
   LocaleGuard locale_guard("fr_FR.UTF-8");
 
-  AssertConversion<DoubleType>("1.5", 1.5f);
+  AssertConversion<DoubleType>("1.5", 1.5);
+  AssertConversionFails<DoubleType>("1,5");
+
+  StringConverter<DoubleType> converter(/*decimal_point=*/'#');
+  AssertConversion(&converter, "1#5", 1.5);
+  AssertConversionFails(&converter, "1.5");
+  AssertConversionFails(&converter, "1,5");
 }
 
 #endif  // _WIN32
diff --git a/cpp/src/arrow/vendored/fast_float/README.md b/cpp/src/arrow/vendored/fast_float/README.md
index 7d2e705..c1c7316 100644
--- a/cpp/src/arrow/vendored/fast_float/README.md
+++ b/cpp/src/arrow/vendored/fast_float/README.md
@@ -1,5 +1,5 @@
 The files in this directory are vendored from fast_float
-git changeset `d4bc0f28a276ac05f8663826eadef324de3a3399`.
+git tag `v3.4.0`.
 
 See https://github.com/fastfloat/fast_float
 
diff --git a/cpp/src/arrow/vendored/fast_float/ascii_number.h b/cpp/src/arrow/vendored/fast_float/ascii_number.h
index 8a5afde..92a7665 100644
--- a/cpp/src/arrow/vendored/fast_float/ascii_number.h
+++ b/cpp/src/arrow/vendored/fast_float/ascii_number.h
@@ -1,10 +1,10 @@
 #ifndef FASTFLOAT_ASCII_NUMBER_H
 #define FASTFLOAT_ASCII_NUMBER_H
 
-#include <cstdio>
 #include <cctype>
 #include <cstdint>
 #include <cstring>
+#include <iterator>
 
 #include "float_common.h"
 
@@ -15,6 +15,34 @@ namespace fast_float {
 // able to optimize it well.
 fastfloat_really_inline bool is_integer(char c)  noexcept  { return c >= '0' && c <= '9'; }
 
+fastfloat_really_inline uint64_t byteswap(uint64_t val) {
+  return (val & 0xFF00000000000000) >> 56
+    | (val & 0x00FF000000000000) >> 40
+    | (val & 0x0000FF0000000000) >> 24
+    | (val & 0x000000FF00000000) >> 8
+    | (val & 0x00000000FF000000) << 8
+    | (val & 0x0000000000FF0000) << 24
+    | (val & 0x000000000000FF00) << 40
+    | (val & 0x00000000000000FF) << 56;
+}
+
+fastfloat_really_inline uint64_t read_u64(const char *chars) {
+  uint64_t val;
+  ::memcpy(&val, chars, sizeof(uint64_t));
+#if FASTFLOAT_IS_BIG_ENDIAN == 1
+  // Need to read as-if the number was in little-endian order.
+  val = byteswap(val);
+#endif
+  return val;
+}
+
+fastfloat_really_inline void write_u64(uint8_t *chars, uint64_t val) {
+#if FASTFLOAT_IS_BIG_ENDIAN == 1
+  // Need to read as-if the number was in little-endian order.
+  val = byteswap(val);
+#endif
+  ::memcpy(chars, &val, sizeof(uint64_t));
+}
 
 // credit  @aqrit
 fastfloat_really_inline uint32_t  parse_eight_digits_unrolled(uint64_t val) {
@@ -28,47 +56,50 @@ fastfloat_really_inline uint32_t  parse_eight_digits_unrolled(uint64_t val) {
 }
 
 fastfloat_really_inline uint32_t parse_eight_digits_unrolled(const char *chars)  noexcept  {
-  uint64_t val;
-  ::memcpy(&val, chars, sizeof(uint64_t));
-  return parse_eight_digits_unrolled(val);
+  return parse_eight_digits_unrolled(read_u64(chars));
 }
 
 // credit @aqrit
 fastfloat_really_inline bool is_made_of_eight_digits_fast(uint64_t val)  noexcept  {
   return !((((val + 0x4646464646464646) | (val - 0x3030303030303030)) &
-     0x8080808080808080)); 
+     0x8080808080808080));
 }
 
 fastfloat_really_inline bool is_made_of_eight_digits_fast(const char *chars)  noexcept  {
-  uint64_t val;
-  ::memcpy(&val, chars, 8);
-  return is_made_of_eight_digits_fast(val);
+  return is_made_of_eight_digits_fast(read_u64(chars));
 }
 
+typedef span<const char> byte_span;
+
 struct parsed_number_string {
-  int64_t exponent;
-  uint64_t mantissa;
-  const char *lastmatch;
-  bool negative;
-  bool valid;
-  bool too_many_digits;
+  int64_t exponent{0};
+  uint64_t mantissa{0};
+  const char *lastmatch{nullptr};
+  bool negative{false};
+  bool valid{false};
+  bool too_many_digits{false};
+  // contains the range of the significant digits
+  byte_span integer{};  // non-nullable
+  byte_span fraction{}; // nullable
 };
 
-
 // Assuming that you use no more than 19 digits, this will
 // parse an ASCII string.
 fastfloat_really_inline
-parsed_number_string parse_number_string(const char *p, const char *pend, chars_format fmt) noexcept {
+parsed_number_string parse_number_string(const char *p, const char *pend, parse_options options) noexcept {
+  const chars_format fmt = options.format;
+  const char decimal_point = options.decimal_point;
+
   parsed_number_string answer;
   answer.valid = false;
   answer.too_many_digits = false;
   answer.negative = (*p == '-');
-  if ((*p == '-') || (*p == '+')) {
+  if (*p == '-') { // C++17 20.19.3.(7.1) explicitly forbids '+' sign here
     ++p;
     if (p == pend) {
       return answer;
     }
-    if (!is_integer(*p) && (*p != '.')) { // a  sign must be followed by an integer or the dot
+    if (!is_integer(*p) && (*p != decimal_point)) { // a sign must be followed by an integer or the dot
       return answer;
     }
   }
@@ -76,6 +107,10 @@ parsed_number_string parse_number_string(const char *p, const char *pend, chars_
 
   uint64_t i = 0; // an unsigned int avoids signed overflows (which are bad)
 
+  while ((std::distance(p, pend) >= 8) && is_made_of_eight_digits_fast(p)) {
+    i = i * 100000000 + parse_eight_digits_unrolled(p); // in rare cases, this will overflow, but that's ok
+    p += 8;
+  }
   while ((p != pend) && is_integer(*p)) {
     // a multiplication by 10 is cheaper than an arbitrary integer
     // multiplication
@@ -85,26 +120,24 @@ parsed_number_string parse_number_string(const char *p, const char *pend, chars_
   }
   const char *const end_of_integer_part = p;
   int64_t digit_count = int64_t(end_of_integer_part - start_digits);
+  answer.integer = byte_span(start_digits, size_t(digit_count));
   int64_t exponent = 0;
-  if ((p != pend) && (*p == '.')) {
+  if ((p != pend) && (*p == decimal_point)) {
     ++p;
-#if FASTFLOAT_IS_BIG_ENDIAN == 0
-    // Fast approach only tested under little endian systems
-    if ((p + 8 <= pend) && is_made_of_eight_digits_fast(p)) {
+    const char* before = p;
+    // can occur at most twice without overflowing, but let it occur more, since
+    // for integers with many digits, digit parsing is the primary bottleneck.
+    while ((std::distance(p, pend) >= 8) && is_made_of_eight_digits_fast(p)) {
       i = i * 100000000 + parse_eight_digits_unrolled(p); // in rare cases, this will overflow, but that's ok
       p += 8;
-      if ((p + 8 <= pend) && is_made_of_eight_digits_fast(p)) {
-        i = i * 100000000 + parse_eight_digits_unrolled(p); // in rare cases, this will overflow, but that's ok
-        p += 8;
-      }
     }
-#endif
     while ((p != pend) && is_integer(*p)) {
       uint8_t digit = uint8_t(*p - '0');
       ++p;
       i = i * 10 + digit; // in rare cases, this will overflow, but that's ok
     }
-    exponent = end_of_integer_part + 1 - p;
+    exponent = before - p;
+    answer.fraction = byte_span(before, size_t(p - before));
     digit_count -= exponent;
   }
   // we must have encountered at least one integer!
@@ -119,7 +152,7 @@ parsed_number_string parse_number_string(const char *p, const char *pend, chars_
     if ((p != pend) && ('-' == *p)) {
       neg_exp = true;
       ++p;
-    } else if ((p != pend) && ('+' == *p)) {
+    } else if ((p != pend) && ('+' == *p)) { // '+' on exponent is allowed by C++17 20.19.3.(7.1)
       ++p;
     }
     if ((p == pend) || !is_integer(*p)) {
@@ -132,7 +165,7 @@ parsed_number_string parse_number_string(const char *p, const char *pend, chars_
     } else {
       while ((p != pend) && is_integer(*p)) {
         uint8_t digit = uint8_t(*p - '0');
-        if (exp_number < 0x10000) {
+        if (exp_number < 0x10000000) {
           exp_number = 10 * exp_number + digit;
         }
         ++p;
@@ -158,30 +191,33 @@ parsed_number_string parse_number_string(const char *p, const char *pend, chars_
     // We need to be mindful of the case where we only have zeroes...
     // E.g., 0.000000000...000.
     const char *start = start_digits;
-    while ((start != pend) && (*start == '0' || *start == '.')) {
+    while ((start != pend) && (*start == '0' || *start == decimal_point)) {
       if(*start == '0') { digit_count --; }
       start++;
     }
     if (digit_count > 19) {
       answer.too_many_digits = true;
       // Let us start again, this time, avoiding overflows.
+      // We don't need to check if is_integer, since we use the
+      // pre-tokenized spans from above.
       i = 0;
-      p = start_digits;
+      p = answer.integer.ptr;
+      const char* int_end = p + answer.integer.len();
       const uint64_t minimal_nineteen_digit_integer{1000000000000000000};
-      while((i < minimal_nineteen_digit_integer) && (p != pend) && is_integer(*p)) {
+      while((i < minimal_nineteen_digit_integer) && (p != int_end)) {
         i = i * 10 + uint64_t(*p - '0');
         ++p;
       }
       if (i >= minimal_nineteen_digit_integer) { // We have a big integers
         exponent = end_of_integer_part - p + exp_number;
       } else { // We have a value with a fractional component.
-          p++; // skip the '.'
-          const char *first_after_period = p;
-          while((i < minimal_nineteen_digit_integer) && (p != pend) && is_integer(*p)) {
+          p = answer.fraction.ptr;
+          const char* frac_end = p + answer.fraction.len();
+          while((i < minimal_nineteen_digit_integer) && (p != frac_end)) {
             i = i * 10 + uint64_t(*p - '0');
             ++p;
           }
-          exponent = first_after_period - p + exp_number;
+          exponent = answer.fraction.ptr - p + exp_number;
       }
       // We have now corrected both exponent and i, to a truncated value
     }
@@ -191,110 +227,6 @@ parsed_number_string parse_number_string(const char *p, const char *pend, chars_
   return answer;
 }
 
-
-// This should always succeed since it follows a call to parse_number_string
-// This function could be optimized. In particular, we could stop after 19 digits
-// and try to bail out. Furthermore, we should be able to recover the computed
-// exponent from the pass in parse_number_string.
-fastfloat_really_inline decimal parse_decimal(const char *p, const char *pend) noexcept {
-  decimal answer;
-  answer.num_digits = 0;
-  answer.decimal_point = 0;
-  answer.truncated = false;
-  // any whitespace has been skipped.
-  answer.negative = (*p == '-');
-  if ((*p == '-') || (*p == '+')) {
-    ++p;
-  }
-  // skip leading zeroes
-  while ((p != pend) && (*p == '0')) {
-    ++p;
-  }
-  while ((p != pend) && is_integer(*p)) {
-    if (answer.num_digits < max_digits) {
-      answer.digits[answer.num_digits] = uint8_t(*p - '0');
-    }
-    answer.num_digits++;
-    ++p;
-  }
-  if ((p != pend) && (*p == '.')) {
-    ++p;
-    const char *first_after_period = p;
-    // if we have not yet encountered a zero, we have to skip it as well
-    if(answer.num_digits == 0) {
-      // skip zeros
-      while ((p != pend) && (*p == '0')) {
-       ++p;
-      }
-    }
-#if FASTFLOAT_IS_BIG_ENDIAN == 0
-    // We expect that this loop will often take the bulk of the running time
-    // because when a value has lots of digits, these digits often
-    while ((p + 8 <= pend) && (answer.num_digits + 8 < max_digits)) {
-      uint64_t val;
-      ::memcpy(&val, p, sizeof(uint64_t));
-      if(! is_made_of_eight_digits_fast(val)) { break; }
-      // We have eight digits, process them in one go!
-      val -= 0x3030303030303030;
-      ::memcpy(answer.digits + answer.num_digits, &val, sizeof(uint64_t));
-      answer.num_digits += 8;
-      p += 8;
-    }
-#endif
-    while ((p != pend) && is_integer(*p)) {
-      if (answer.num_digits < max_digits) {
-        answer.digits[answer.num_digits] = uint8_t(*p - '0');
-      }
-      answer.num_digits++;
-      ++p;
-    }
-    answer.decimal_point = int32_t(first_after_period - p);
-  }
-  // We want num_digits to be the number of significant digits, excluding
-  // leading *and* trailing zeros! Otherwise the truncated flag later is
-  // going to be misleading.
-  if(answer.num_digits > 0) {
-    // We potentially need the answer.num_digits > 0 guard because we
-    // prune leading zeros. So with answer.num_digits > 0, we know that
-    // we have at least one non-zero digit.
-    const char *preverse = p - 1;
-    int32_t trailing_zeros = 0;
-    while ((*preverse == '0') || (*preverse == '.')) {
-      if(*preverse == '0') { trailing_zeros++; };
-      --preverse;
-    }
-    answer.decimal_point += int32_t(answer.num_digits);
-    answer.num_digits -= uint32_t(trailing_zeros);
-  }
-  if(answer.num_digits > max_digits) {
-    answer.truncated = true;
-    answer.num_digits = max_digits;
-  }
-  if ((p != pend) && (('e' == *p) || ('E' == *p))) {
-    ++p;
-    bool neg_exp = false;
-    if ((p != pend) && ('-' == *p)) {
-      neg_exp = true;
-      ++p;
-    } else if ((p != pend) && ('+' == *p)) {
-      ++p;
-    }
-    int32_t exp_number = 0; // exponential part
-    while ((p != pend) && is_integer(*p)) {
-      uint8_t digit = uint8_t(*p - '0');
-      if (exp_number < 0x10000) {
-        exp_number = 10 * exp_number + digit;
-      }    
-      ++p;
-    }
-    answer.decimal_point += (neg_exp ? -exp_number : exp_number);
-  }
-  // In very rare cases, we may have fewer than 19 digits, we want to be able to reliably
-  // assume that all digits up to max_digit_without_overflow have been initialized.
-  for(uint32_t i = answer.num_digits; i < max_digit_without_overflow; i++) { answer.digits[i] = 0; }
-
-  return answer;
-}
 } // namespace fast_float
 } // namespace arrow_vendored
 
diff --git a/cpp/src/arrow/vendored/fast_float/bigint.h b/cpp/src/arrow/vendored/fast_float/bigint.h
new file mode 100644
index 0000000..38ac5c8
--- /dev/null
+++ b/cpp/src/arrow/vendored/fast_float/bigint.h
@@ -0,0 +1,592 @@
+#ifndef FASTFLOAT_BIGINT_H
+#define FASTFLOAT_BIGINT_H
+
+#include <algorithm>
+#include <cstdint>
+#include <climits>
+#include <cstring>
+
+#include "float_common.h"
+
+namespace arrow_vendored {
+namespace fast_float {
+
+// the limb width: we want efficient multiplication of double the bits in
+// limb, or for 64-bit limbs, at least 64-bit multiplication where we can
+// extract the high and low parts efficiently. this is every 64-bit
+// architecture except for sparc, which emulates 128-bit multiplication.
+// we might have platforms where `CHAR_BIT` is not 8, so let's avoid
+// doing `8 * sizeof(limb)`.
+#if defined(FASTFLOAT_64BIT) && !defined(__sparc)
+#define FASTFLOAT_64BIT_LIMB
+typedef uint64_t limb;
+constexpr size_t limb_bits = 64;
+#else
+#define FASTFLOAT_32BIT_LIMB
+typedef uint32_t limb;
+constexpr size_t limb_bits = 32;
+#endif
+
+typedef span<limb> limb_span;
+
+// number of bits in a bigint. this needs to be at least the number
+// of bits required to store the largest bigint, which is
+// `log2(10**(digits + max_exp))`, or `log2(10**(767 + 342))`, or
+// ~3600 bits, so we round to 4000.
+constexpr size_t bigint_bits = 4000;
+constexpr size_t bigint_limbs = bigint_bits / limb_bits;
+
+// vector-like type that is allocated on the stack. the entire
+// buffer is pre-allocated, and only the length changes.
+template <uint16_t size>
+struct stackvec {
+  limb data[size];
+  // we never need more than 150 limbs
+  uint16_t length{0};
+
+  stackvec() = default;
+  stackvec(const stackvec &) = delete;
+  stackvec &operator=(const stackvec &) = delete;
+  stackvec(stackvec &&) = delete;
+  stackvec &operator=(stackvec &&other) = delete;
+
+  // create stack vector from existing limb span.
+  stackvec(limb_span s) {
+    FASTFLOAT_ASSERT(try_extend(s));
+  }
+
+  limb& operator[](size_t index) noexcept {
+    FASTFLOAT_DEBUG_ASSERT(index < length);
+    return data[index];
+  }
+  const limb& operator[](size_t index) const noexcept {
+    FASTFLOAT_DEBUG_ASSERT(index < length);
+    return data[index];
+  }
+  // index from the end of the container
+  const limb& rindex(size_t index) const noexcept {
+    FASTFLOAT_DEBUG_ASSERT(index < length);
+    size_t rindex = length - index - 1;
+    return data[rindex];
+  }
+
+  // set the length, without bounds checking.
+  void set_len(size_t len) noexcept {
+    length = uint16_t(len);
+  }
+  constexpr size_t len() const noexcept {
+    return length;
+  }
+  constexpr bool is_empty() const noexcept {
+    return length == 0;
+  }
+  constexpr size_t capacity() const noexcept {
+    return size;
+  }
+  // append item to vector, without bounds checking
+  void push_unchecked(limb value) noexcept {
+    data[length] = value;
+    length++;
+  }
+  // append item to vector, returning if item was added
+  bool try_push(limb value) noexcept {
+    if (len() < capacity()) {
+      push_unchecked(value);
+      return true;
+    } else {
+      return false;
+    }
+  }
+  // add items to the vector, from a span, without bounds checking
+  void extend_unchecked(limb_span s) noexcept {
+    limb* ptr = data + length;
+    ::memcpy((void*)ptr, (const void*)s.ptr, sizeof(limb) * s.len());
+    set_len(len() + s.len());
+  }
+  // try to add items to the vector, returning if items were added
+  bool try_extend(limb_span s) noexcept {
+    if (len() + s.len() <= capacity()) {
+      extend_unchecked(s);
+      return true;
+    } else {
+      return false;
+    }
+  }
+  // resize the vector, without bounds checking
+  // if the new size is longer than the vector, assign value to each
+  // appended item.
+  void resize_unchecked(size_t new_len, limb value) noexcept {
+    if (new_len > len()) {
+      size_t count = new_len - len();
+      limb* first = data + len();
+      limb* last = first + count;
+      ::std::fill(first, last, value);
+      set_len(new_len);
+    } else {
+      set_len(new_len);
+    }
+  }
+  // try to resize the vector, returning if the vector was resized.
+  bool try_resize(size_t new_len, limb value) noexcept {
+    if (new_len > capacity()) {
+      return false;
+    } else {
+      resize_unchecked(new_len, value);
+      return true;
+    }
+  }
+  // check if any limbs are non-zero after the given index.
+  // this needs to be done in reverse order, since the index
+  // is relative to the most significant limbs.
+  bool nonzero(size_t index) const noexcept {
+    while (index < len()) {
+      if (rindex(index) != 0) {
+        return true;
+      }
+      index++;
+    }
+    return false;
+  }
+  // normalize the big integer, so most-significant zero limbs are removed.
+  void normalize() noexcept {
+    while (len() > 0 && rindex(0) == 0) {
+      length--;
+    }
+  }
+};
+
+fastfloat_really_inline
+uint64_t empty_hi64(bool& truncated) noexcept {
+  truncated = false;
+  return 0;
+}
+
+fastfloat_really_inline
+uint64_t uint64_hi64(uint64_t r0, bool& truncated) noexcept {
+  truncated = false;
+  int shl = leading_zeroes(r0);
+  return r0 << shl;
+}
+
+fastfloat_really_inline
+uint64_t uint64_hi64(uint64_t r0, uint64_t r1, bool& truncated) noexcept {
+  int shl = leading_zeroes(r0);
+  if (shl == 0) {
+    truncated = r1 != 0;
+    return r0;
+  } else {
+    int shr = 64 - shl;
+    truncated = (r1 << shl) != 0;
+    return (r0 << shl) | (r1 >> shr);
+  }
+}
+
+fastfloat_really_inline
+uint64_t uint32_hi64(uint32_t r0, bool& truncated) noexcept {
+  return uint64_hi64(r0, truncated);
+}
+
+fastfloat_really_inline
+uint64_t uint32_hi64(uint32_t r0, uint32_t r1, bool& truncated) noexcept {
+  uint64_t x0 = r0;
+  uint64_t x1 = r1;
+  return uint64_hi64((x0 << 32) | x1, truncated);
+}
+
+fastfloat_really_inline
+uint64_t uint32_hi64(uint32_t r0, uint32_t r1, uint32_t r2, bool& truncated) noexcept {
+  uint64_t x0 = r0;
+  uint64_t x1 = r1;
+  uint64_t x2 = r2;
+  return uint64_hi64(x0, (x1 << 32) | x2, truncated);
+}
+
+// add two small integers, checking for overflow.
+// we want an efficient operation. for msvc, where
+// we don't have built-in intrinsics, this is still
+// pretty fast.
+fastfloat_really_inline
+limb scalar_add(limb x, limb y, bool& overflow) noexcept {
+  limb z;
+
+// gcc and clang
+#if defined(__has_builtin)
+  #if __has_builtin(__builtin_add_overflow)
+    overflow = __builtin_add_overflow(x, y, &z);
+    return z;
+  #endif
+#endif
+
+  // generic, this still optimizes correctly on MSVC.
+  z = x + y;
+  overflow = z < x;
+  return z;
+}
+
+// multiply two small integers, getting both the high and low bits.
+fastfloat_really_inline
+limb scalar_mul(limb x, limb y, limb& carry) noexcept {
+#ifdef FASTFLOAT_64BIT_LIMB
+  #if defined(__SIZEOF_INT128__)
+  // GCC and clang both define it as an extension.
+  __uint128_t z = __uint128_t(x) * __uint128_t(y) + __uint128_t(carry);
+  carry = limb(z >> limb_bits);
+  return limb(z);
+  #else
+  // fallback, no native 128-bit integer multiplication with carry.
+  // on msvc, this optimizes identically, somehow.
+  value128 z = full_multiplication(x, y);
+  bool overflow;
+  z.low = scalar_add(z.low, carry, overflow);
+  z.high += uint64_t(overflow);  // cannot overflow
+  carry = z.high;
+  return z.low;
+  #endif
+#else
+  uint64_t z = uint64_t(x) * uint64_t(y) + uint64_t(carry);
+  carry = limb(z >> limb_bits);
+  return limb(z);
+#endif
+}
+
+// add scalar value to bigint starting from offset.
+// used in grade school multiplication
+template <uint16_t size>
+inline bool small_add_from(stackvec<size>& vec, limb y, size_t start) noexcept {
+  size_t index = start;
+  limb carry = y;
+  bool overflow;
+  while (carry != 0 && index < vec.len()) {
+    vec[index] = scalar_add(vec[index], carry, overflow);
+    carry = limb(overflow);
+    index += 1;
+  }
+  if (carry != 0) {
+    FASTFLOAT_TRY(vec.try_push(carry));
+  }
+  return true;
+}
+
+// add scalar value to bigint.
+template <uint16_t size>
+fastfloat_really_inline bool small_add(stackvec<size>& vec, limb y) noexcept {
+  return small_add_from(vec, y, 0);
+}
+
+// multiply bigint by scalar value.
+template <uint16_t size>
+inline bool small_mul(stackvec<size>& vec, limb y) noexcept {
+  limb carry = 0;
+  for (size_t index = 0; index < vec.len(); index++) {
+    vec[index] = scalar_mul(vec[index], y, carry);
+  }
+  if (carry != 0) {
+    FASTFLOAT_TRY(vec.try_push(carry));
+  }
+  return true;
+}
+
+// add bigint to bigint starting from index.
+// used in grade school multiplication
+template <uint16_t size>
+bool large_add_from(stackvec<size>& x, limb_span y, size_t start) noexcept {
+  // the effective x buffer is from `xstart..x.len()`, so exit early
+  // if we can't get that current range.
+  if (x.len() < start || y.len() > x.len() - start) {
+      FASTFLOAT_TRY(x.try_resize(y.len() + start, 0));
+  }
+
+  bool carry = false;
+  for (size_t index = 0; index < y.len(); index++) {
+    limb xi = x[index + start];
+    limb yi = y[index];
+    bool c1 = false;
+    bool c2 = false;
+    xi = scalar_add(xi, yi, c1);
+    if (carry) {
+      xi = scalar_add(xi, 1, c2);
+    }
+    x[index + start] = xi;
+    carry = c1 | c2;
+  }
+
+  // handle overflow
+  if (carry) {
+    FASTFLOAT_TRY(small_add_from(x, 1, y.len() + start));
+  }
+  return true;
+}
+
+// add bigint to bigint.
+template <uint16_t size>
+fastfloat_really_inline bool large_add_from(stackvec<size>& x, limb_span y) noexcept {
+  return large_add_from(x, y, 0);
+}
+
+// grade-school multiplication algorithm
+template <uint16_t size>
+bool long_mul(stackvec<size>& x, limb_span y) noexcept {
+  limb_span xs = limb_span(x.data, x.len());
+  stackvec<size> z(xs);
+  limb_span zs = limb_span(z.data, z.len());
+
+  if (y.len() != 0) {
+    limb y0 = y[0];
+    FASTFLOAT_TRY(small_mul(x, y0));
+    for (size_t index = 1; index < y.len(); index++) {
+      limb yi = y[index];
+      stackvec<size> zi;
+      if (yi != 0) {
+        // re-use the same buffer throughout
+        zi.set_len(0);
+        FASTFLOAT_TRY(zi.try_extend(zs));
+        FASTFLOAT_TRY(small_mul(zi, yi));
+        limb_span zis = limb_span(zi.data, zi.len());
+        FASTFLOAT_TRY(large_add_from(x, zis, index));
+      }
+    }
+  }
+
+  x.normalize();
+  return true;
+}
+
+// grade-school multiplication algorithm
+template <uint16_t size>
+bool large_mul(stackvec<size>& x, limb_span y) noexcept {
+  if (y.len() == 1) {
+    FASTFLOAT_TRY(small_mul(x, y[0]));
+  } else {
+    FASTFLOAT_TRY(long_mul(x, y));
+  }
+  return true;
+}
+
+// big integer type. implements a small subset of big integer
+// arithmetic, using simple algorithms since asymptotically
+// faster algorithms are slower for a small number of limbs.
+// all operations assume the big-integer is normalized.
+struct bigint {
+  // storage of the limbs, in little-endian order.
+  stackvec<bigint_limbs> vec;
+
+  bigint(): vec() {}
+  bigint(const bigint &) = delete;
+  bigint &operator=(const bigint &) = delete;
+  bigint(bigint &&) = delete;
+  bigint &operator=(bigint &&other) = delete;
+
+  bigint(uint64_t value): vec() {
+#ifdef FASTFLOAT_64BIT_LIMB
+    vec.push_unchecked(value);
+#else
+    vec.push_unchecked(uint32_t(value));
+    vec.push_unchecked(uint32_t(value >> 32));
+#endif
+    vec.normalize();
+  }
+
+  // get the high 64 bits from the vector, and if bits were truncated.
+  // this is to get the significant digits for the float.
+  uint64_t hi64(bool& truncated) const noexcept {
+#ifdef FASTFLOAT_64BIT_LIMB
+    if (vec.len() == 0) {
+      return empty_hi64(truncated);
+    } else if (vec.len() == 1) {
+      return uint64_hi64(vec.rindex(0), truncated);
+    } else {
+      uint64_t result = uint64_hi64(vec.rindex(0), vec.rindex(1), truncated);
+      truncated |= vec.nonzero(2);
+      return result;
+    }
+#else
+    if (vec.len() == 0) {
+      return empty_hi64(truncated);
+    } else if (vec.len() == 1) {
+      return uint32_hi64(vec.rindex(0), truncated);
+    } else if (vec.len() == 2) {
+      return uint32_hi64(vec.rindex(0), vec.rindex(1), truncated);
+    } else {
+      uint64_t result = uint32_hi64(vec.rindex(0), vec.rindex(1), vec.rindex(2), truncated);
+      truncated |= vec.nonzero(3);
+      return result;
+    }
+#endif
+  }
+
+  // compare two big integers, returning the large value.
+  // assumes both are normalized. if the return value is
+  // negative, other is larger, if the return value is
+  // positive, this is larger, otherwise they are equal.
+  // the limbs are stored in little-endian order, so we
+  // must compare the limbs in ever order.
+  int compare(const bigint& other) const noexcept {
+    if (vec.len() > other.vec.len()) {
+      return 1;
+    } else if (vec.len() < other.vec.len()) {
+      return -1;
+    } else {
+      for (size_t index = vec.len(); index > 0; index--) {
+        limb xi = vec[index - 1];
+        limb yi = other.vec[index - 1];
+        if (xi > yi) {
+          return 1;
+        } else if (xi < yi) {
+          return -1;
+        }
+      }
+      return 0;
+    }
+  }
+
+  // shift left each limb n bits, carrying over to the new limb
+  // returns true if we were able to shift all the digits.
+  bool shl_bits(size_t n) noexcept {
+    // Internally, for each item, we shift left by n, and add the previous
+    // right shifted limb-bits.
+    // For example, we transform (for u8) shifted left 2, to:
+    //      b10100100 b01000010
+    //      b10 b10010001 b00001000
+    FASTFLOAT_DEBUG_ASSERT(n != 0);
+    FASTFLOAT_DEBUG_ASSERT(n < sizeof(limb) * 8);
+
+    size_t shl = n;
+    size_t shr = limb_bits - shl;
+    limb prev = 0;
+    for (size_t index = 0; index < vec.len(); index++) {
+      limb xi = vec[index];
+      vec[index] = (xi << shl) | (prev >> shr);
+      prev = xi;
+    }
+
+    limb carry = prev >> shr;
+    if (carry != 0) {
+      return vec.try_push(carry);
+    }
+    return true;
+  }
+
+  // move the limbs left by `n` limbs.
+  bool shl_limbs(size_t n) noexcept {
+    FASTFLOAT_DEBUG_ASSERT(n != 0);
+    if (n + vec.len() > vec.capacity()) {
+      return false;
+    } else if (!vec.is_empty()) {
+      // move limbs
+      limb* dst = vec.data + n;
+      const limb* src = vec.data;
+      ::memmove(dst, src, sizeof(limb) * vec.len());
+      // fill in empty limbs
+      limb* first = vec.data;
+      limb* last = first + n;
+      ::std::fill(first, last, 0);
+      vec.set_len(n + vec.len());
+      return true;
+    } else {
+      return true;
+    }
+  }
+
+  // move the limbs left by `n` bits.
+  bool shl(size_t n) noexcept {
+    size_t rem = n % limb_bits;
+    size_t div = n / limb_bits;
+    if (rem != 0) {
+      FASTFLOAT_TRY(shl_bits(rem));
+    }
+    if (div != 0) {
+      FASTFLOAT_TRY(shl_limbs(div));
+    }
+    return true;
+  }
+
+  // get the number of leading zeros in the bigint.
+  int ctlz() const noexcept {
+    if (vec.is_empty()) {
+      return 0;
+    } else {
+#ifdef FASTFLOAT_64BIT_LIMB
+      return leading_zeroes(vec.rindex(0));
+#else
+      // no use defining a specialized leading_zeroes for a 32-bit type.
+      uint64_t r0 = vec.rindex(0);
+      return leading_zeroes(r0 << 32);
+#endif
+    }
+  }
+
+  // get the number of bits in the bigint.
+  int bit_length() const noexcept {
+    int lz = ctlz();
+    return int(limb_bits * vec.len()) - lz;
+  }
+
+  bool mul(limb y) noexcept {
+    return small_mul(vec, y);
+  }
+
+  bool add(limb y) noexcept {
+    return small_add(vec, y);
+  }
+
+  // multiply as if by 2 raised to a power.
+  bool pow2(uint32_t exp) noexcept {
+    return shl(exp);
+  }
+
+  // multiply as if by 5 raised to a power.
+  bool pow5(uint32_t exp) noexcept {
+    // multiply by a power of 5
+    static constexpr uint32_t large_step = 135;
+    static constexpr uint64_t small_power_of_5[] = {
+      1UL, 5UL, 25UL, 125UL, 625UL, 3125UL, 15625UL, 78125UL, 390625UL,
+      1953125UL, 9765625UL, 48828125UL, 244140625UL, 1220703125UL,
+      6103515625UL, 30517578125UL, 152587890625UL, 762939453125UL,
+      3814697265625UL, 19073486328125UL, 95367431640625UL, 476837158203125UL,
+      2384185791015625UL, 11920928955078125UL, 59604644775390625UL,
+      298023223876953125UL, 1490116119384765625UL, 7450580596923828125UL,
+    };
+#ifdef FASTFLOAT_64BIT_LIMB
+    constexpr static limb large_power_of_5[] = {
+      1414648277510068013UL, 9180637584431281687UL, 4539964771860779200UL,
+      10482974169319127550UL, 198276706040285095UL};
+#else
+    constexpr static limb large_power_of_5[] = {
+      4279965485U, 329373468U, 4020270615U, 2137533757U, 4287402176U,
+      1057042919U, 1071430142U, 2440757623U, 381945767U, 46164893U};
+#endif
+    size_t large_length = sizeof(large_power_of_5) / sizeof(limb);
+    limb_span large = limb_span(large_power_of_5, large_length);
+    while (exp >= large_step) {
+      FASTFLOAT_TRY(large_mul(vec, large));
+      exp -= large_step;
+    }
+#ifdef FASTFLOAT_64BIT_LIMB
+    uint32_t small_step = 27;
+    limb max_native = 7450580596923828125UL;
+#else
+    uint32_t small_step = 13;
+    limb max_native = 1220703125U;
+#endif
+    while (exp >= small_step) {
+      FASTFLOAT_TRY(small_mul(vec, max_native));
+      exp -= small_step;
+    }
+    if (exp != 0) {
+      FASTFLOAT_TRY(small_mul(vec, limb(small_power_of_5[exp])));
+    }
+
+    return true;
+  }
+
+  // multiply as if by 10 raised to a power.
+  bool pow10(uint32_t exp) noexcept {
+    FASTFLOAT_TRY(pow5(exp));
+    return pow2(exp);
+  }
+};
+
+} // namespace fast_float
+} // namespace arrow_vendored
+
+#endif
diff --git a/cpp/src/arrow/vendored/fast_float/decimal_to_binary.h b/cpp/src/arrow/vendored/fast_float/decimal_to_binary.h
index 2419de7..8ae481d 100644
--- a/cpp/src/arrow/vendored/fast_float/decimal_to_binary.h
+++ b/cpp/src/arrow/vendored/fast_float/decimal_to_binary.h
@@ -7,7 +7,6 @@
 #include <cinttypes>
 #include <cmath>
 #include <cstdint>
-#include <cstdio>
 #include <cstdlib>
 #include <cstring>
 
@@ -21,18 +20,18 @@ namespace fast_float {
 template <int bit_precision>
 fastfloat_really_inline
 value128 compute_product_approximation(int64_t q, uint64_t w) {
-  const int index = 2 * int(q - smallest_power_of_five);
+  const int index = 2 * int(q - powers::smallest_power_of_five);
   // For small values of q, e.g., q in [0,27], the answer is always exact because
   // The line value128 firstproduct = full_multiplication(w, power_of_five_128[index]);
   // gives the exact answer.
-  value128 firstproduct = full_multiplication(w, power_of_five_128[index]);
+  value128 firstproduct = full_multiplication(w, powers::power_of_five_128[index]);
   static_assert((bit_precision >= 0) && (bit_precision <= 64), " precision should  be in (0,64]");
   constexpr uint64_t precision_mask = (bit_precision < 64) ?
                (uint64_t(0xFFFFFFFFFFFFFFFF) >> bit_precision)
                : uint64_t(0xFFFFFFFFFFFFFFFF);
   if((firstproduct.high & precision_mask) == precision_mask) { // could further guard with  (lower + w < lower)
     // regarding the second product, we only need secondproduct.high, but our expectation is that the compiler will optimize this extra work away if needed.
-    value128 secondproduct = full_multiplication(w, power_of_five_128[index + 1]);
+    value128 secondproduct = full_multiplication(w, powers::power_of_five_128[index + 1]);
     firstproduct.low += secondproduct.high;
     if(secondproduct.high > firstproduct.low) {
       firstproduct.high++;
@@ -41,7 +40,7 @@ value128 compute_product_approximation(int64_t q, uint64_t w) {
   return firstproduct;
 }
 
-namespace {
+namespace detail {
 /**
  * For q in (0,350), we have that
  *  f = (((152170 + 65536) * q ) >> 16);
@@ -57,11 +56,34 @@ namespace {
  * where
  *   p = log(5**-q)/log(2) = -q * log(5)/log(2)
  */
-  fastfloat_really_inline int power(int q)  noexcept  {
+  constexpr fastfloat_really_inline int32_t power(int32_t q)  noexcept  {
     return (((152170 + 65536) * q) >> 16) + 63;
   }
-} // namespace
+} // namespace detail
 
+// create an adjusted mantissa, biased by the invalid power2
+// for significant digits already multiplied by 10 ** q.
+template <typename binary>
+fastfloat_really_inline
+adjusted_mantissa compute_error_scaled(int64_t q, uint64_t w, int lz) noexcept  {
+  int hilz = int(w >> 63) ^ 1;
+  adjusted_mantissa answer;
+  answer.mantissa = w << hilz;
+  int bias = binary::mantissa_explicit_bits() - binary::minimum_exponent();
+  answer.power2 = int32_t(detail::power(int32_t(q)) + bias - hilz - lz - 62 + invalid_am_bias);
+  return answer;
+}
+
+// w * 10 ** q, without rounding the representation up.
+// the power2 in the exponent will be adjusted by invalid_am_bias.
+template <typename binary>
+fastfloat_really_inline
+adjusted_mantissa compute_error(int64_t q, uint64_t w)  noexcept  {
+  int lz = leading_zeroes(w);
+  w <<= lz;
+  value128 product = compute_product_approximation<binary::mantissa_explicit_bits() + 3>(q, w);
+  return compute_error_scaled<binary>(q, product.high, lz);
+}
 
 // w * 10 ** q
 // The returned value should be a valid ieee64 number that simply need to be packed.
@@ -84,7 +106,7 @@ adjusted_mantissa compute_float(int64_t q, uint64_t w)  noexcept  {
     answer.mantissa = 0;
     return answer;
   }
-  // At this point in time q is in [smallest_power_of_five, largest_power_of_five].
+  // At this point in time q is in [powers::smallest_power_of_five, powers::largest_power_of_five].
 
   // We want the most significant bit of i to be 1. Shift if needed.
   int lz = leading_zeroes(w);
@@ -103,8 +125,7 @@ adjusted_mantissa compute_float(int64_t q, uint64_t w)  noexcept  {
     const bool inside_safe_exponent = (q >= -27) && (q <= 55); // always good because 5**q <2**128 when q>=0, 
     // and otherwise, for q<0, we have 5**-q<2**64 and the 128-bit reciprocal allows for exact computation.
     if(!inside_safe_exponent) {
-      answer.power2 = -1; // This (a negative value) indicates an error condition.
-      return answer;
+      return compute_error_scaled<binary>(q, product.high, lz);
     }
   }
   // The "compute_product_approximation" function can be slightly slower than a branchless approach:
@@ -115,7 +136,7 @@ adjusted_mantissa compute_float(int64_t q, uint64_t w)  noexcept  {
 
   answer.mantissa = product.high >> (upperbit + 64 - binary::mantissa_explicit_bits() - 3);
 
-  answer.power2 = int(power(int(q)) + upperbit - lz - binary::minimum_exponent());
+  answer.power2 = int32_t(detail::power(int32_t(q)) + upperbit - lz - binary::minimum_exponent());
   if (answer.power2 <= 0) { // we have a subnormal?
     // Here have that answer.power2 <= 0 so -answer.power2 >= 0
     if(-answer.power2 + 1 >= 64) { // if we have more than 64 bits below the minimum exponent, you have a zero for sure.
@@ -169,7 +190,6 @@ adjusted_mantissa compute_float(int64_t q, uint64_t w)  noexcept  {
   return answer;
 }
 
-
 } // namespace fast_float
 } // namespace arrow_vendored
 
diff --git a/cpp/src/arrow/vendored/fast_float/digit_comparison.h b/cpp/src/arrow/vendored/fast_float/digit_comparison.h
new file mode 100644
index 0000000..ba8471a
--- /dev/null
+++ b/cpp/src/arrow/vendored/fast_float/digit_comparison.h
@@ -0,0 +1,409 @@
+#ifndef FASTFLOAT_DIGIT_COMPARISON_H
+#define FASTFLOAT_DIGIT_COMPARISON_H
+
+#include <algorithm>
+#include <cstdint>
+#include <cstring>
+#include <iterator>
+
+#include "float_common.h"
+#include "bigint.h"
+#include "ascii_number.h"
+
+namespace arrow_vendored {
+namespace fast_float {
+
+// 1e0 to 1e19
+constexpr static uint64_t powers_of_ten_uint64[] = {
+    1UL, 10UL, 100UL, 1000UL, 10000UL, 100000UL, 1000000UL, 10000000UL, 100000000UL,
+    1000000000UL, 10000000000UL, 100000000000UL, 1000000000000UL, 10000000000000UL,
+    100000000000000UL, 1000000000000000UL, 10000000000000000UL, 100000000000000000UL,
+    1000000000000000000UL, 10000000000000000000UL};
+
+// calculate the exponent, in scientific notation, of the number.
+// this algorithm is not even close to optimized, but it has no practical
+// effect on performance: in order to have a faster algorithm, we'd need
+// to slow down performance for faster algorithms, and this is still fast.
+fastfloat_really_inline int32_t scientific_exponent(parsed_number_string& num) noexcept {
+  uint64_t mantissa = num.mantissa;
+  int32_t exponent = int32_t(num.exponent);
+  while (mantissa >= 10000) {
+    mantissa /= 10000;
+    exponent += 4;
+  }
+  while (mantissa >= 100) {
+    mantissa /= 100;
+    exponent += 2;
+  }
+  while (mantissa >= 10) {
+    mantissa /= 10;
+    exponent += 1;
+  }
+  return exponent;
+}
+
+// this converts a native floating-point number to an extended-precision float.
+template <typename T>
+fastfloat_really_inline adjusted_mantissa to_extended(T value) noexcept {
+  using equiv_uint = typename binary_format<T>::equiv_uint;
+  constexpr equiv_uint exponent_mask = binary_format<T>::exponent_mask();
+  constexpr equiv_uint mantissa_mask = binary_format<T>::mantissa_mask();
+  constexpr equiv_uint hidden_bit_mask = binary_format<T>::hidden_bit_mask();
+
+  adjusted_mantissa am;
+  int32_t bias = binary_format<T>::mantissa_explicit_bits() - binary_format<T>::minimum_exponent();
+  equiv_uint bits;
+  ::memcpy(&bits, &value, sizeof(T));
+  if ((bits & exponent_mask) == 0) {
+    // denormal
+    am.power2 = 1 - bias;
+    am.mantissa = bits & mantissa_mask;
+  } else {
+    // normal
+    am.power2 = int32_t((bits & exponent_mask) >> binary_format<T>::mantissa_explicit_bits());
+    am.power2 -= bias;
+    am.mantissa = (bits & mantissa_mask) | hidden_bit_mask;
+  }
+
+  return am;
+}
+
+// get the extended precision value of the halfway point between b and b+u.
+// we are given a native float that represents b, so we need to adjust it
+// halfway between b and b+u.
+template <typename T>
+fastfloat_really_inline adjusted_mantissa to_extended_halfway(T value) noexcept {
+  adjusted_mantissa am = to_extended(value);
+  am.mantissa <<= 1;
+  am.mantissa += 1;
+  am.power2 -= 1;
+  return am;
+}
+
+// round an extended-precision float to the nearest machine float.
+template <typename T, typename callback>
+fastfloat_really_inline void round(adjusted_mantissa& am, callback cb) noexcept {
+  int32_t mantissa_shift = 64 - binary_format<T>::mantissa_explicit_bits() - 1;
+  if (-am.power2 >= mantissa_shift) {
+    // have a denormal float
+    int32_t shift = -am.power2 + 1;
+    cb(am, std::min(shift, 64));
+    // check for round-up: if rounding-nearest carried us to the hidden bit.
+    am.power2 = (am.mantissa < (uint64_t(1) << binary_format<T>::mantissa_explicit_bits())) ? 0 : 1;
+    return;
+  }
+
+  // have a normal float, use the default shift.
+  cb(am, mantissa_shift);
+
+  // check for carry
+  if (am.mantissa >= (uint64_t(2) << binary_format<T>::mantissa_explicit_bits())) {
+    am.mantissa = (uint64_t(1) << binary_format<T>::mantissa_explicit_bits());
+    am.power2++;
+  }
+
+  // check for infinite: we could have carried to an infinite power
+  am.mantissa &= ~(uint64_t(1) << binary_format<T>::mantissa_explicit_bits());
+  if (am.power2 >= binary_format<T>::infinite_power()) {
+    am.power2 = binary_format<T>::infinite_power();
+    am.mantissa = 0;
+  }
+}
+
+template <typename callback>
+fastfloat_really_inline
+void round_nearest_tie_even(adjusted_mantissa& am, int32_t shift, callback cb) noexcept {
+  uint64_t mask;
+  uint64_t halfway;
+  if (shift == 64) {
+    mask = UINT64_MAX;
+  } else {
+    mask = (uint64_t(1) << shift) - 1;
+  }
+  if (shift == 0) {
+    halfway = 0;
+  } else {
+    halfway = uint64_t(1) << (shift - 1);
+  }
+  uint64_t truncated_bits = am.mantissa & mask;
+  uint64_t is_above = truncated_bits > halfway;
+  uint64_t is_halfway = truncated_bits == halfway;
+
+  // shift digits into position
+  if (shift == 64) {
+    am.mantissa = 0;
+  } else {
+    am.mantissa >>= shift;
+  }
+  am.power2 += shift;
+
+  bool is_odd = (am.mantissa & 1) == 1;
+  am.mantissa += uint64_t(cb(is_odd, is_halfway, is_above));
+}
+
+fastfloat_really_inline void round_down(adjusted_mantissa& am, int32_t shift) noexcept {
+  if (shift == 64) {
+    am.mantissa = 0;
+  } else {
+    am.mantissa >>= shift;
+  }
+  am.power2 += shift;
+}
+
+fastfloat_really_inline void skip_zeros(const char*& first, const char* last) noexcept {
+  uint64_t val;
+  while (std::distance(first, last) >= 8) {
+    ::memcpy(&val, first, sizeof(uint64_t));
+    if (val != 0x3030303030303030) {
+      break;
+    }
+    first += 8;
+  }
+  while (first != last) {
+    if (*first != '0') {
+      break;
+    }
+    first++;
+  }
+}
+
+// determine if any non-zero digits were truncated.
+// all characters must be valid digits.
+fastfloat_really_inline bool is_truncated(const char* first, const char* last) noexcept {
+  // do 8-bit optimizations, can just compare to 8 literal 0s.
+  uint64_t val;
+  while (std::distance(first, last) >= 8) {
+    ::memcpy(&val, first, sizeof(uint64_t));
+    if (val != 0x3030303030303030) {
+      return true;
+    }
+    first += 8;
+  }
+  while (first != last) {
+    if (*first != '0') {
+      return true;
+    }
+    first++;
+  }
+  return false;
+}
+
+fastfloat_really_inline bool is_truncated(byte_span s) noexcept {
+  return is_truncated(s.ptr, s.ptr + s.len());
+}
+
+fastfloat_really_inline
+void parse_eight_digits(const char*& p, limb& value, size_t& counter, size_t& count) noexcept {
+  value = value * 100000000 + parse_eight_digits_unrolled(p);
+  p += 8;
+  counter += 8;
+  count += 8;
+}
+
+fastfloat_really_inline
+void parse_one_digit(const char*& p, limb& value, size_t& counter, size_t& count) noexcept {
+  value = value * 10 + limb(*p - '0');
+  p++;
+  counter++;
+  count++;
+}
+
+fastfloat_really_inline
+void add_native(bigint& big, limb power, limb value) noexcept {
+  big.mul(power);
+  big.add(value);
+}
+
+fastfloat_really_inline void round_up_bigint(bigint& big, size_t& count) noexcept {
+  // need to round-up the digits, but need to avoid rounding
+  // ....9999 to ...10000, which could cause a false halfway point.
+  add_native(big, 10, 1);
+  count++;
+}
+
+// parse the significant digits into a big integer
+inline void parse_mantissa(bigint& result, parsed_number_string& num, size_t max_digits, size_t& digits) noexcept {
+  // try to minimize the number of big integer and scalar multiplication.
+  // therefore, try to parse 8 digits at a time, and multiply by the largest
+  // scalar value (9 or 19 digits) for each step.
+  size_t counter = 0;
+  digits = 0;
+  limb value = 0;
+#ifdef FASTFLOAT_64BIT_LIMB
+  size_t step = 19;
+#else
+  size_t step = 9;
+#endif
+
+  // process all integer digits.
+  const char* p = num.integer.ptr;
+  const char* pend = p + num.integer.len();
+  skip_zeros(p, pend);
+  // process all digits, in increments of step per loop
+  while (p != pend) {
+    while ((std::distance(p, pend) >= 8) && (step - counter >= 8) && (max_digits - digits >= 8)) {
+      parse_eight_digits(p, value, counter, digits);
+    }
+    while (counter < step && p != pend && digits < max_digits) {
+      parse_one_digit(p, value, counter, digits);
+    }
+    if (digits == max_digits) {
+      // add the temporary value, then check if we've truncated any digits
+      add_native(result, limb(powers_of_ten_uint64[counter]), value);
+      bool truncated = is_truncated(p, pend);
+      if (num.fraction.ptr != nullptr) {
+        truncated |= is_truncated(num.fraction);
+      }
+      if (truncated) {
+        round_up_bigint(result, digits);
+      }
+      return;
+    } else {
+      add_native(result, limb(powers_of_ten_uint64[counter]), value);
+      counter = 0;
+      value = 0;
+    }
+  }
+
+  // add our fraction digits, if they're available.
+  if (num.fraction.ptr != nullptr) {
+    p = num.fraction.ptr;
+    pend = p + num.fraction.len();
+    if (digits == 0) {
+      skip_zeros(p, pend);
+    }
+    // process all digits, in increments of step per loop
+    while (p != pend) {
+      while ((std::distance(p, pend) >= 8) && (step - counter >= 8) && (max_digits - digits >= 8)) {
+        parse_eight_digits(p, value, counter, digits);
+      }
+      while (counter < step && p != pend && digits < max_digits) {
+        parse_one_digit(p, value, counter, digits);
+      }
+      if (digits == max_digits) {
+        // add the temporary value, then check if we've truncated any digits
+        add_native(result, limb(powers_of_ten_uint64[counter]), value);
+        bool truncated = is_truncated(p, pend);
+        if (truncated) {
+          round_up_bigint(result, digits);
+        }
+        return;
+      } else {
+        add_native(result, limb(powers_of_ten_uint64[counter]), value);
+        counter = 0;
+        value = 0;
+      }
+    }
+  }
+
+  if (counter != 0) {
+    add_native(result, limb(powers_of_ten_uint64[counter]), value);
+  }
+}
+
+template <typename T>
+inline adjusted_mantissa positive_digit_comp(bigint& bigmant, int32_t exponent) noexcept {
+  FASTFLOAT_ASSERT(bigmant.pow10(uint32_t(exponent)));
+  adjusted_mantissa answer;
+  bool truncated;
+  answer.mantissa = bigmant.hi64(truncated);
+  int bias = binary_format<T>::mantissa_explicit_bits() - binary_format<T>::minimum_exponent();
+  answer.power2 = bigmant.bit_length() - 64 + bias;
+
+  round<T>(answer, [truncated](adjusted_mantissa& a, int32_t shift) {
+    round_nearest_tie_even(a, shift, [truncated](bool is_odd, bool is_halfway, bool is_above) -> bool {
+      return is_above || (is_halfway && truncated) || (is_odd && is_halfway);
+    });
+  });
+
+  return answer;
+}
+
+// the scaling here is quite simple: we have, for the real digits `m * 10^e`,
+// and for the theoretical digits `n * 2^f`. Since `e` is always negative,
+// to scale them identically, we do `n * 2^f * 5^-f`, so we now have `m * 2^e`.
+// we then need to scale by `2^(f- e)`, and then the two significant digits
+// are of the same magnitude.
+template <typename T>
+inline adjusted_mantissa negative_digit_comp(bigint& bigmant, adjusted_mantissa am, int32_t exponent) noexcept {
+  bigint& real_digits = bigmant;
+  int32_t real_exp = exponent;
+
+  // get the value of `b`, rounded down, and get a bigint representation of b+h
+  adjusted_mantissa am_b = am;
+  // gcc7 buf: use a lambda to remove the noexcept qualifier bug with -Wnoexcept-type.
+  round<T>(am_b, [](adjusted_mantissa&a, int32_t shift) { round_down(a, shift); });
+  T b;
+  to_float(false, am_b, b);
+  adjusted_mantissa theor = to_extended_halfway(b);
+  bigint theor_digits(theor.mantissa);
+  int32_t theor_exp = theor.power2;
+
+  // scale real digits and theor digits to be same power.
+  int32_t pow2_exp = theor_exp - real_exp;
+  uint32_t pow5_exp = uint32_t(-real_exp);
+  if (pow5_exp != 0) {
+    FASTFLOAT_ASSERT(theor_digits.pow5(pow5_exp));
+  }
+  if (pow2_exp > 0) {
+    FASTFLOAT_ASSERT(theor_digits.pow2(uint32_t(pow2_exp)));
+  } else if (pow2_exp < 0) {
+    FASTFLOAT_ASSERT(real_digits.pow2(uint32_t(-pow2_exp)));
+  }
+
+  // compare digits, and use it to director rounding
+  int ord = real_digits.compare(theor_digits);
+  adjusted_mantissa answer = am;
+  round<T>(answer, [ord](adjusted_mantissa& a, int32_t shift) {
+    round_nearest_tie_even(a, shift, [ord](bool is_odd, bool _, bool __) -> bool {
+      (void)_;  // not needed, since we've done our comparison
+      (void)__; // not needed, since we've done our comparison
+      if (ord > 0) {
+        return true;
+      } else if (ord < 0) {
+        return false;
+      } else {
+        return is_odd;
+      }
+    });
+  });
+
+  return answer;
+}
+
+// parse the significant digits as a big integer to unambiguously round the
+// the significant digits. here, we are trying to determine how to round
+// an extended float representation close to `b+h`, halfway between `b`
+// (the float rounded-down) and `b+u`, the next positive float. this
+// algorithm is always correct, and uses one of two approaches. when
+// the exponent is positive relative to the significant digits (such as
+// 1234), we create a big-integer representation, get the high 64-bits,
+// determine if any lower bits are truncated, and use that to direct
+// rounding. in case of a negative exponent relative to the significant
+// digits (such as 1.2345), we create a theoretical representation of
+// `b` as a big-integer type, scaled to the same binary exponent as
+// the actual digits. we then compare the big integer representations
+// of both, and use that to direct rounding.
+template <typename T>
+inline adjusted_mantissa digit_comp(parsed_number_string& num, adjusted_mantissa am) noexcept {
+  // remove the invalid exponent bias
+  am.power2 -= invalid_am_bias;
+
+  int32_t sci_exp = scientific_exponent(num);
+  size_t max_digits = binary_format<T>::max_digits();
+  size_t digits = 0;
+  bigint bigmant;
+  parse_mantissa(bigmant, num, max_digits, digits);
+  // can't underflow, since digits is at most max_digits.
+  int32_t exponent = sci_exp + 1 - int32_t(digits);
+  if (exponent >= 0) {
+    return positive_digit_comp<T>(bigmant, exponent);
+  } else {
+    return negative_digit_comp<T>(bigmant, am, exponent);
+  }
+}
+
+} // namespace fast_float
+} // namespace arrow_vendored
+
+#endif
diff --git a/cpp/src/arrow/vendored/fast_float/fast_float.h b/cpp/src/arrow/vendored/fast_float/fast_float.h
index 3e39cac..3b82134 100644
--- a/cpp/src/arrow/vendored/fast_float/fast_float.h
+++ b/cpp/src/arrow/vendored/fast_float/fast_float.h
@@ -18,6 +18,17 @@ struct from_chars_result {
   std::errc ec;
 };
 
+struct parse_options {
+  constexpr explicit parse_options(chars_format fmt = chars_format::general,
+                         char dot = '.')
+    : format(fmt), decimal_point(dot) {}
+
+  /** Which number formats are accepted */
+  chars_format format;
+  /** The character used as decimal point */
+  char decimal_point;
+};
+
 /**
  * This function parses the character sequence [first,last) for a number. It parses floating-point numbers expecting
  * a locale-indepent format equivalent to what is used by std::strtod in the default ("C") locale.
@@ -41,6 +52,13 @@ template<typename T>
 from_chars_result from_chars(const char *first, const char *last,
                              T &value, chars_format fmt = chars_format::general)  noexcept;
 
+/**
+ * Like from_chars, but accepts an `options` argument to govern number parsing.
+ */
+template<typename T>
+from_chars_result from_chars_advanced(const char *first, const char *last,
+                                      T &value, parse_options options)  noexcept;
+
 }
 } // namespace arrow_vendored
 
diff --git a/cpp/src/arrow/vendored/fast_float/fast_table.h b/cpp/src/arrow/vendored/fast_float/fast_table.h
index c1ca175..4861cab 100644
--- a/cpp/src/arrow/vendored/fast_float/fast_table.h
+++ b/cpp/src/arrow/vendored/fast_float/fast_table.h
@@ -1,5 +1,6 @@
 #ifndef FASTFLOAT_FAST_TABLE_H
 #define FASTFLOAT_FAST_TABLE_H
+
 #include <cstdint>
 
 namespace arrow_vendored {
@@ -29,10 +30,18 @@ namespace fast_float {
  * infinite in binary64 so we never need to worry about powers
  * of 5 greater than 308.
  */
-constexpr int smallest_power_of_five = -342;
-constexpr int largest_power_of_five = 308;
+template <class unused = void>
+struct powers_template {
+
+constexpr static int smallest_power_of_five = binary_format<double>::smallest_power_of_ten();
+constexpr static int largest_power_of_five = binary_format<double>::largest_power_of_ten();
+constexpr static int number_of_entries = 2 * (largest_power_of_five - smallest_power_of_five + 1);
 // Powers of five from 5^-342 all the way to 5^308 rounded toward one.
-const uint64_t power_of_five_128[]= {
+static const uint64_t power_of_five_128[number_of_entries];
+};
+
+template <class unused>
+const uint64_t powers_template<unused>::power_of_five_128[number_of_entries] = {
         0xeef453d6923bd65a,0x113faa2906a13b3f,
         0x9558b4661b6565f8,0x4ac7ca59a424c507,
         0xbaaee17fa23ebf76,0x5d79bcf00d2df649,
@@ -684,6 +693,7 @@ const uint64_t power_of_five_128[]= {
         0xb6472e511c81471d,0xe0133fe4adf8e952,
         0xe3d8f9e563a198e5,0x58180fddd97723a6,
         0x8e679c2f5e44ff8f,0x570f09eaa7ea7648,};
+using powers = powers_template<>;
 
 }
 } // namespace arrow_vendored
diff --git a/cpp/src/arrow/vendored/fast_float/float_common.h b/cpp/src/arrow/vendored/fast_float/float_common.h
index f7b7662..76241b9 100644
--- a/cpp/src/arrow/vendored/fast_float/float_common.h
+++ b/cpp/src/arrow/vendored/fast_float/float_common.h
@@ -4,6 +4,8 @@
 #include <cfloat>
 #include <cstdint>
 #include <cassert>
+#include <cstring>
+#include <type_traits>
 
 #if (defined(__x86_64) || defined(__x86_64__) || defined(_M_X64)   \
        || defined(__amd64) || defined(__aarch64__) || defined(_M_ARM64) \
@@ -13,11 +15,22 @@
        || defined(__EMSCRIPTEN__))
 #define FASTFLOAT_64BIT
 #elif (defined(__i386) || defined(__i386__) || defined(_M_IX86)   \
-     || defined(__arm__)                                        \
+     || defined(__arm__) || defined(_M_ARM)                   \
      || defined(__MINGW32__))
 #define FASTFLOAT_32BIT
 #else
-#error Unknown platform (not 32-bit, not 64-bit?)
+  // Need to check incrementally, since SIZE_MAX is a size_t, avoid overflow.
+  // We can never tell the register width, but the SIZE_MAX is a good approximation.
+  // UINTPTR_MAX and INTPTR_MAX are optional, so avoid them for max portability.
+  #if SIZE_MAX == 0xffff
+    #error Unknown platform (16-bit, unsupported)
+  #elif SIZE_MAX == 0xffffffff
+    #define FASTFLOAT_32BIT
+  #elif SIZE_MAX == 0xffffffffffffffff
+    #define FASTFLOAT_64BIT
+  #else
+    #error Unknown platform (not 32-bit, not 64-bit?)
+  #endif
 #endif
 
 #if ((defined(_WIN32) || defined(_WIN64)) && !defined(__clang__))
@@ -62,6 +75,18 @@
 #define fastfloat_really_inline inline __attribute__((always_inline))
 #endif
 
+#ifndef FASTFLOAT_ASSERT
+#define FASTFLOAT_ASSERT(x)  { if (!(x)) abort(); }
+#endif
+
+#ifndef FASTFLOAT_DEBUG_ASSERT
+#include <cassert>
+#define FASTFLOAT_DEBUG_ASSERT(x) assert(x)
+#endif
+
+// rust style `try!()` macro, or `?` operator
+#define FASTFLOAT_TRY(x) { if (!(x)) return false; }
+
 namespace arrow_vendored {
 namespace fast_float {
 
@@ -79,27 +104,23 @@ inline bool fastfloat_strncasecmp(const char *input1, const char *input2,
 #error "FLT_EVAL_METHOD should be defined, please include cfloat."
 #endif
 
-inline bool is_space(uint8_t c) {
-  static const bool table[] = {
-      0, 0, 0, 0, 0, 0, 0, 0, 0, 1, 1, 1, 1, 1, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0,
-      0, 0, 0, 0, 0, 0, 0, 0, 1, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0,
-      0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0,
-      0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0,
-      0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0,
-      0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0,
-      0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0,
-      0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0,
-      0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0,
-      0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0,
-      0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0};
-  return table[c];
-}
-
-namespace {
-constexpr uint32_t max_digits = 768;
-constexpr uint32_t max_digit_without_overflow = 19;
-constexpr int32_t decimal_point_range = 2047;
-} // namespace
+// a pointer and a length to a contiguous block of memory
+template <typename T>
+struct span {
+  const T* ptr;
+  size_t length;
+  span(const T* _ptr, size_t _length) : ptr(_ptr), length(_length) {}
+  span() : ptr(nullptr), length(0) {}
+
+  constexpr size_t len() const noexcept {
+    return length;
+  }
+
+  const T& operator[](size_t index) const noexcept {
+    FASTFLOAT_DEBUG_ASSERT(index < length);
+    return ptr[index];
+  }
+};
 
 struct value128 {
   uint64_t low;
@@ -135,23 +156,21 @@ fastfloat_really_inline int leading_zeroes(uint64_t input_num) {
 
 #ifdef FASTFLOAT_32BIT
 
-#if (!defined(_WIN32)) || defined(__MINGW32__)
 // slow emulation routine for 32-bit
-fastfloat_really_inline uint64_t __emulu(uint32_t x, uint32_t y) {
+fastfloat_really_inline uint64_t emulu(uint32_t x, uint32_t y) {
     return x * (uint64_t)y;
 }
-#endif
 
 // slow emulation routine for 32-bit
 #if !defined(__MINGW64__)
 fastfloat_really_inline uint64_t _umul128(uint64_t ab, uint64_t cd,
                                           uint64_t *hi) {
-  uint64_t ad = __emulu((uint32_t)(ab >> 32), (uint32_t)cd);
-  uint64_t bd = __emulu((uint32_t)ab, (uint32_t)cd);
-  uint64_t adbc = ad + __emulu((uint32_t)ab, (uint32_t)(cd >> 32));
+  uint64_t ad = emulu((uint32_t)(ab >> 32), (uint32_t)cd);
+  uint64_t bd = emulu((uint32_t)ab, (uint32_t)cd);
+  uint64_t adbc = ad + emulu((uint32_t)ab, (uint32_t)(cd >> 32));
   uint64_t adbc_carry = !!(adbc < ad);
   uint64_t lo = bd + (adbc << 32);
-  *hi = __emulu((uint32_t)(ab >> 32), (uint32_t)(cd >> 32)) + (adbc >> 32) +
+  *hi = emulu((uint32_t)(ab >> 32), (uint32_t)(cd >> 32)) + (adbc >> 32) +
         (adbc_carry << 32) + !!(lo < bd);
   return lo;
 }
@@ -180,10 +199,9 @@ fastfloat_really_inline value128 full_multiplication(uint64_t a,
   return answer;
 }
 
-
 struct adjusted_mantissa {
   uint64_t mantissa{0};
-  int power2{0}; // a negative value indicates an invalid result
+  int32_t power2{0}; // a negative value indicates an invalid result
   adjusted_mantissa() = default;
   bool operator==(const adjusted_mantissa &o) const {
     return mantissa == o.mantissa && power2 == o.power2;
@@ -193,21 +211,8 @@ struct adjusted_mantissa {
   }
 };
 
-struct decimal {
-  uint32_t num_digits{0};
-  int32_t decimal_point{0};
-  bool negative{false};
-  bool truncated{false};
-  uint8_t digits[max_digits];
-  decimal() = default;
-  // Copies are not allowed since this is a fat object.
-  decimal(const decimal &) = delete;
-  // Copies are not allowed since this is a fat object.
-  decimal &operator=(const decimal &) = delete;
-  // Moves are allowed:
-  decimal(decimal &&) = default;
-  decimal &operator=(decimal &&other) = default;
-};
+// Bias so we can get the real exponent with an invalid adjusted_mantissa.
+constexpr static int32_t invalid_am_bias = -0x8000;
 
 constexpr static double powers_of_ten_double[] = {
     1e0,  1e1,  1e2,  1e3,  1e4,  1e5,  1e6,  1e7,  1e8,  1e9,  1e10, 1e11,
@@ -216,68 +221,74 @@ constexpr static float powers_of_ten_float[] = {1e0, 1e1, 1e2, 1e3, 1e4, 1e5,
                                                 1e6, 1e7, 1e8, 1e9, 1e10};
 
 template <typename T> struct binary_format {
-  static constexpr int mantissa_explicit_bits();
-  static constexpr int minimum_exponent();
-  static constexpr int infinite_power();
-  static constexpr int sign_index();
-  static constexpr int min_exponent_fast_path();
-  static constexpr int max_exponent_fast_path();
-  static constexpr int max_exponent_round_to_even();
-  static constexpr int min_exponent_round_to_even();
-  static constexpr uint64_t max_mantissa_fast_path();
-  static constexpr int largest_power_of_ten();
-  static constexpr int smallest_power_of_ten();
-  static constexpr T exact_power_of_ten(int64_t power);
+  using equiv_uint = typename std::conditional<sizeof(T) == 4, uint32_t, uint64_t>::type;
+
+  static inline constexpr int mantissa_explicit_bits();
+  static inline constexpr int minimum_exponent();
+  static inline constexpr int infinite_power();
+  static inline constexpr int sign_index();
+  static inline constexpr int min_exponent_fast_path();
+  static inline constexpr int max_exponent_fast_path();
+  static inline constexpr int max_exponent_round_to_even();
+  static inline constexpr int min_exponent_round_to_even();
+  static inline constexpr uint64_t max_mantissa_fast_path();
+  static inline constexpr int largest_power_of_ten();
+  static inline constexpr int smallest_power_of_ten();
+  static inline constexpr T exact_power_of_ten(int64_t power);
+  static inline constexpr size_t max_digits();
+  static inline constexpr equiv_uint exponent_mask();
+  static inline constexpr equiv_uint mantissa_mask();
+  static inline constexpr equiv_uint hidden_bit_mask();
 };
 
-template <> constexpr int binary_format<double>::mantissa_explicit_bits() {
+template <> inline constexpr int binary_format<double>::mantissa_explicit_bits() {
   return 52;
 }
-template <> constexpr int binary_format<float>::mantissa_explicit_bits() {
+template <> inline constexpr int binary_format<float>::mantissa_explicit_bits() {
   return 23;
 }
 
-template <> constexpr int binary_format<double>::max_exponent_round_to_even() {
+template <> inline constexpr int binary_format<double>::max_exponent_round_to_even() {
   return 23;
 }
 
-template <> constexpr int binary_format<float>::max_exponent_round_to_even() {
+template <> inline constexpr int binary_format<float>::max_exponent_round_to_even() {
   return 10;
 }
 
-template <> constexpr int binary_format<double>::min_exponent_round_to_even() {
+template <> inline constexpr int binary_format<double>::min_exponent_round_to_even() {
   return -4;
 }
 
-template <> constexpr int binary_format<float>::min_exponent_round_to_even() {
+template <> inline constexpr int binary_format<float>::min_exponent_round_to_even() {
   return -17;
 }
 
-template <> constexpr int binary_format<double>::minimum_exponent() {
+template <> inline constexpr int binary_format<double>::minimum_exponent() {
   return -1023;
 }
-template <> constexpr int binary_format<float>::minimum_exponent() {
+template <> inline constexpr int binary_format<float>::minimum_exponent() {
   return -127;
 }
 
-template <> constexpr int binary_format<double>::infinite_power() {
+template <> inline constexpr int binary_format<double>::infinite_power() {
   return 0x7FF;
 }
-template <> constexpr int binary_format<float>::infinite_power() {
+template <> inline constexpr int binary_format<float>::infinite_power() {
   return 0xFF;
 }
 
-template <> constexpr int binary_format<double>::sign_index() { return 63; }
-template <> constexpr int binary_format<float>::sign_index() { return 31; }
+template <> inline constexpr int binary_format<double>::sign_index() { return 63; }
+template <> inline constexpr int binary_format<float>::sign_index() { return 31; }
 
-template <> constexpr int binary_format<double>::min_exponent_fast_path() {
+template <> inline constexpr int binary_format<double>::min_exponent_fast_path() {
 #if (FLT_EVAL_METHOD != 1) && (FLT_EVAL_METHOD != 0)
   return 0;
 #else
   return -22;
 #endif
 }
-template <> constexpr int binary_format<float>::min_exponent_fast_path() {
+template <> inline constexpr int binary_format<float>::min_exponent_fast_path() {
 #if (FLT_EVAL_METHOD != 1) && (FLT_EVAL_METHOD != 0)
   return 0;
 #else
@@ -285,61 +296,102 @@ template <> constexpr int binary_format<float>::min_exponent_fast_path() {
 #endif
 }
 
-template <> constexpr int binary_format<double>::max_exponent_fast_path() {
+template <> inline constexpr int binary_format<double>::max_exponent_fast_path() {
   return 22;
 }
-template <> constexpr int binary_format<float>::max_exponent_fast_path() {
+template <> inline constexpr int binary_format<float>::max_exponent_fast_path() {
   return 10;
 }
 
-template <> constexpr uint64_t binary_format<double>::max_mantissa_fast_path() {
+template <> inline constexpr uint64_t binary_format<double>::max_mantissa_fast_path() {
   return uint64_t(2) << mantissa_explicit_bits();
 }
-template <> constexpr uint64_t binary_format<float>::max_mantissa_fast_path() {
+template <> inline constexpr uint64_t binary_format<float>::max_mantissa_fast_path() {
   return uint64_t(2) << mantissa_explicit_bits();
 }
 
 template <>
-constexpr double binary_format<double>::exact_power_of_ten(int64_t power) {
+inline constexpr double binary_format<double>::exact_power_of_ten(int64_t power) {
   return powers_of_ten_double[power];
 }
 template <>
-constexpr float binary_format<float>::exact_power_of_ten(int64_t power) {
+inline constexpr float binary_format<float>::exact_power_of_ten(int64_t power) {
 
   return powers_of_ten_float[power];
 }
 
 
 template <>
-constexpr int binary_format<double>::largest_power_of_ten() {
+inline constexpr int binary_format<double>::largest_power_of_ten() {
   return 308;
 }
 template <>
-constexpr int binary_format<float>::largest_power_of_ten() {
+inline constexpr int binary_format<float>::largest_power_of_ten() {
   return 38;
 }
 
 template <>
-constexpr int binary_format<double>::smallest_power_of_ten() {
+inline constexpr int binary_format<double>::smallest_power_of_ten() {
   return -342;
 }
 template <>
-constexpr int binary_format<float>::smallest_power_of_ten() {
+inline constexpr int binary_format<float>::smallest_power_of_ten() {
   return -65;
 }
 
-} // namespace fast_float
-} // namespace arrow_vendored
+template <> inline constexpr size_t binary_format<double>::max_digits() {
+  return 769;
+}
+template <> inline constexpr size_t binary_format<float>::max_digits() {
+  return 114;
+}
 
-// for convenience:
-template<class OStream>
-inline OStream& operator<<(OStream &out, const arrow_vendored::fast_float::decimal &d) {
-  out << "0.";
-  for (size_t i = 0; i < d.num_digits; i++) {
-    out << int32_t(d.digits[i]);
-  }
-  out << " * 10 ** " << d.decimal_point;
-  return out;
+template <> inline constexpr binary_format<float>::equiv_uint
+    binary_format<float>::exponent_mask() {
+  return 0x7F800000;
+}
+template <> inline constexpr binary_format<double>::equiv_uint
+    binary_format<double>::exponent_mask() {
+  return 0x7FF0000000000000;
+}
+
+template <> inline constexpr binary_format<float>::equiv_uint
+    binary_format<float>::mantissa_mask() {
+  return 0x007FFFFF;
+}
+template <> inline constexpr binary_format<double>::equiv_uint
+    binary_format<double>::mantissa_mask() {
+  return 0x000FFFFFFFFFFFFF;
+}
+
+template <> inline constexpr binary_format<float>::equiv_uint
+    binary_format<float>::hidden_bit_mask() {
+  return 0x00800000;
 }
+template <> inline constexpr binary_format<double>::equiv_uint
+    binary_format<double>::hidden_bit_mask() {
+  return 0x0010000000000000;
+}
+
+template<typename T>
+fastfloat_really_inline void to_float(bool negative, adjusted_mantissa am, T &value) {
+  uint64_t word = am.mantissa;
+  word |= uint64_t(am.power2) << binary_format<T>::mantissa_explicit_bits();
+  word = negative
+  ? word | (uint64_t(1) << binary_format<T>::sign_index()) : word;
+#if FASTFLOAT_IS_BIG_ENDIAN == 1
+   if (std::is_same<T, float>::value) {
+     ::memcpy(&value, (char *)&word + 4, sizeof(T)); // extract value at offset 4-7 if float on big-endian
+   } else {
+     ::memcpy(&value, &word, sizeof(T));
+   }
+#else
+   // For little-endian systems:
+   ::memcpy(&value, &word, sizeof(T));
+#endif
+}
+
+} // namespace fast_float
+} // namespace arrow_vendored
 
 #endif
diff --git a/cpp/src/arrow/vendored/fast_float/parse_number.h b/cpp/src/arrow/vendored/fast_float/parse_number.h
index d530f76..83dce69 100644
--- a/cpp/src/arrow/vendored/fast_float/parse_number.h
+++ b/cpp/src/arrow/vendored/fast_float/parse_number.h
@@ -1,10 +1,10 @@
 #ifndef FASTFLOAT_PARSE_NUMBER_H
 #define FASTFLOAT_PARSE_NUMBER_H
+
 #include "ascii_number.h"
 #include "decimal_to_binary.h"
-#include "simple_decimal_conversion.h"
+#include "digit_comparison.h"
 
-#include <cassert>
 #include <cmath>
 #include <cstring>
 #include <limits>
@@ -14,7 +14,7 @@ namespace arrow_vendored {
 namespace fast_float {
 
 
-namespace {
+namespace detail {
 /**
  * Special case +inf, -inf, nan, infinity, -infinity.
  * The case comparisons could be made much faster given that we know that the
@@ -26,11 +26,9 @@ from_chars_result parse_infnan(const char *first, const char *last, T &value)  n
   answer.ptr = first;
   answer.ec = std::errc(); // be optimistic
   bool minusSign = false;
-  if (*first == '-') { // assume first < last, so dereference without checks
+  if (*first == '-') { // assume first < last, so dereference without checks; C++17 20.19.3.(7.1) explicitly forbids '+' here
       minusSign = true;
       ++first;
-  } else if( *first == '+' ) { // C++17 20.19.3.7 explicitly forbids '+' here, but anyway
-      ++first;
   }
   if (last - first >= 3) {
     if (fastfloat_strncasecmp(first, "nan", 3)) {
@@ -63,46 +61,30 @@ from_chars_result parse_infnan(const char *first, const char *last, T &value)  n
   return answer;
 }
 
-template<typename T>
-fastfloat_really_inline void to_float(bool negative, adjusted_mantissa am, T &value) {
-  uint64_t word = am.mantissa;
-  word |= uint64_t(am.power2) << binary_format<T>::mantissa_explicit_bits();
-  word = negative
-  ? word | (uint64_t(1) << binary_format<T>::sign_index()) : word;
-#if FASTFLOAT_IS_BIG_ENDIAN == 1
-   if (std::is_same<T, float>::value) {
-     ::memcpy(&value, (char *)&word + 4, sizeof(T)); // extract value at offset 4-7 if float on big-endian
-   } else {
-     ::memcpy(&value, &word, sizeof(T));
-   }
-#else
-   // For little-endian systems:
-   ::memcpy(&value, &word, sizeof(T));
-#endif
-}
-
-} // namespace
-
-
+} // namespace detail
 
 template<typename T>
 from_chars_result from_chars(const char *first, const char *last,
                              T &value, chars_format fmt /*= chars_format::general*/)  noexcept  {
+  return from_chars_advanced(first, last, value, parse_options{fmt});
+}
+
+template<typename T>
+from_chars_result from_chars_advanced(const char *first, const char *last,
+                                      T &value, parse_options options)  noexcept  {
+
   static_assert (std::is_same<T, double>::value || std::is_same<T, float>::value, "only float and double are supported");
 
 
   from_chars_result answer;
-  while ((first != last) && fast_float::is_space(uint8_t(*first))) {
-    first++;
-  }
   if (first == last) {
     answer.ec = std::errc::invalid_argument;
     answer.ptr = first;
     return answer;
   }
-  parsed_number_string pns = parse_number_string(first, last, fmt);
+  parsed_number_string pns = parse_number_string(first, last, options);
   if (!pns.valid) {
-    return parse_infnan(first, last, value);
+    return detail::parse_infnan(first, last, value);
   }
   answer.ec = std::errc(); // be optimistic
   answer.ptr = pns.lastmatch;
@@ -115,14 +97,14 @@ from_chars_result from_chars(const char *first, const char *last,
     return answer;
   }
   adjusted_mantissa am = compute_float<binary_format<T>>(pns.exponent, pns.mantissa);
-  if(pns.too_many_digits) {
+  if(pns.too_many_digits && am.power2 >= 0) {
     if(am != compute_float<binary_format<T>>(pns.exponent, pns.mantissa + 1)) {
-      am.power2 = -1; // value is invalid.
+      am = compute_error<binary_format<T>>(pns.exponent, pns.mantissa);
     }
   }
   // If we called compute_float<binary_format<T>>(pns.exponent, pns.mantissa) and we have an invalid power (am.power2 < 0),
   // then we need to go the long way around again. This is very uncommon.
-  if(am.power2 < 0) { am = parse_long_mantissa<binary_format<T>>(first,last); }
+  if(am.power2 < 0) { am = digit_comp<T>(pns, am); }
   to_float(pns.negative, am, value);
   return answer;
 }
diff --git a/cpp/src/arrow/vendored/fast_float/simple_decimal_conversion.h b/cpp/src/arrow/vendored/fast_float/simple_decimal_conversion.h
index 486724c..fb6ad2b 100644
--- a/cpp/src/arrow/vendored/fast_float/simple_decimal_conversion.h
+++ b/cpp/src/arrow/vendored/fast_float/simple_decimal_conversion.h
@@ -20,7 +20,7 @@
 namespace arrow_vendored {
 namespace fast_float {
 
-namespace {
+namespace detail {
 
 // remove all final zeroes
 inline void trim(decimal &h) {
@@ -31,9 +31,9 @@ inline void trim(decimal &h) {
 
 
 
-uint32_t number_of_digits_decimal_left_shift(const decimal &h, uint32_t shift) {
+inline uint32_t number_of_digits_decimal_left_shift(const decimal &h, uint32_t shift) {
   shift &= 63;
-  const static uint16_t number_of_digits_decimal_left_shift_table[65] = {
+  constexpr uint16_t number_of_digits_decimal_left_shift_table[65] = {
     0x0000, 0x0800, 0x0801, 0x0803, 0x1006, 0x1009, 0x100D, 0x1812, 0x1817,
     0x181D, 0x2024, 0x202B, 0x2033, 0x203C, 0x2846, 0x2850, 0x285B, 0x3067,
     0x3073, 0x3080, 0x388E, 0x389C, 0x38AB, 0x38BB, 0x40CC, 0x40DD, 0x40EF,
@@ -48,7 +48,7 @@ uint32_t number_of_digits_decimal_left_shift(const decimal &h, uint32_t shift) {
   uint32_t num_new_digits = x_a >> 11;
   uint32_t pow5_a = 0x7FF & x_a;
   uint32_t pow5_b = 0x7FF & x_b;
-  const static uint8_t
+  constexpr uint8_t
     number_of_digits_decimal_left_shift_table_powers_of_5[0x051C] = {
         5, 2, 5, 1, 2, 5, 6, 2, 5, 3, 1, 2, 5, 1, 5, 6, 2, 5, 7, 8, 1, 2, 5, 3,
         9, 0, 6, 2, 5, 1, 9, 5, 3, 1, 2, 5, 9, 7, 6, 5, 6, 2, 5, 4, 8, 8, 2, 8,
@@ -124,7 +124,7 @@ uint32_t number_of_digits_decimal_left_shift(const decimal &h, uint32_t shift) {
   return num_new_digits;
 }
 
-uint64_t round(decimal &h) {
+inline uint64_t round(decimal &h) {
   if ((h.num_digits == 0) || (h.decimal_point < 0)) {
     return 0;
   } else if (h.decimal_point > 18) {
@@ -151,7 +151,7 @@ uint64_t round(decimal &h) {
 }
 
 // computes h * 2^-shift
-void decimal_left_shift(decimal &h, uint32_t shift) {
+inline void decimal_left_shift(decimal &h, uint32_t shift) {
   if (h.num_digits == 0) {
     return;
   }
@@ -193,7 +193,7 @@ void decimal_left_shift(decimal &h, uint32_t shift) {
 }
 
 // computes h * 2^shift
-void decimal_right_shift(decimal &h, uint32_t shift) {
+inline void decimal_right_shift(decimal &h, uint32_t shift) {
   uint32_t read_index = 0;
   uint32_t write_index = 0;
 
@@ -239,7 +239,7 @@ void decimal_right_shift(decimal &h, uint32_t shift) {
   trim(h);
 }
 
-} // end of anonymous namespace
+} // namespace detail
 
 template <typename binary>
 adjusted_mantissa compute_float(decimal &d) {
@@ -272,17 +272,17 @@ adjusted_mantissa compute_float(decimal &d) {
     answer.mantissa = 0;
     return answer;
   }
-  static const uint32_t max_shift = 60;
-  static const uint32_t num_powers = 19;
-  static const uint8_t powers[19] = {
+  constexpr uint32_t max_shift = 60;
+  constexpr uint32_t num_powers = 19;
+  constexpr uint8_t decimal_powers[19] = {
       0,  3,  6,  9,  13, 16, 19, 23, 26, 29, //
       33, 36, 39, 43, 46, 49, 53, 56, 59,     //
   };
   int32_t exp2 = 0;
   while (d.decimal_point > 0) {
     uint32_t n = uint32_t(d.decimal_point);
-    uint32_t shift = (n < num_powers) ? powers[n] : max_shift;
-    decimal_right_shift(d, shift);
+    uint32_t shift = (n < num_powers) ? decimal_powers[n] : max_shift;
+    detail::decimal_right_shift(d, shift);
     if (d.decimal_point < -decimal_point_range) {
       // should be zero
       answer.power2 = 0;
@@ -301,9 +301,9 @@ adjusted_mantissa compute_float(decimal &d) {
       shift = (d.digits[0] < 2) ? 2 : 1;
     } else {
       uint32_t n = uint32_t(-d.decimal_point);
-      shift = (n < num_powers) ? powers[n] : max_shift;
+      shift = (n < num_powers) ? decimal_powers[n] : max_shift;
     }
-    decimal_left_shift(d, shift);
+    detail::decimal_left_shift(d, shift);
     if (d.decimal_point > decimal_point_range) {
       // we want to get infinity:
       answer.power2 = binary::infinite_power();
@@ -320,7 +320,7 @@ adjusted_mantissa compute_float(decimal &d) {
     if (n > max_shift) {
       n = max_shift;
     }
-    decimal_right_shift(d, n);
+    detail::decimal_right_shift(d, n);
     exp2 += int32_t(n);
   }
   if ((exp2 - minimum_exponent) >= binary::infinite_power()) {
@@ -330,15 +330,15 @@ adjusted_mantissa compute_float(decimal &d) {
   }
 
   const int mantissa_size_in_bits = binary::mantissa_explicit_bits() + 1;
-  decimal_left_shift(d, mantissa_size_in_bits);
+  detail::decimal_left_shift(d, mantissa_size_in_bits);
 
-  uint64_t mantissa = round(d);
+  uint64_t mantissa = detail::round(d);
   // It is possible that we have an overflow, in which case we need
   // to shift back.
   if(mantissa >= (uint64_t(1) << mantissa_size_in_bits)) {
-    decimal_right_shift(d, 1);
+    detail::decimal_right_shift(d, 1);
     exp2 += 1;
-    mantissa = round(d);
+    mantissa = detail::round(d);
     if ((exp2 - minimum_exponent) >= binary::infinite_power()) {
       answer.power2 = binary::infinite_power();
       answer.mantissa = 0;
@@ -352,8 +352,8 @@ adjusted_mantissa compute_float(decimal &d) {
 }
 
 template <typename binary>
-adjusted_mantissa parse_long_mantissa(const char *first, const char* last) {
-    decimal d = parse_decimal(first, last);
+adjusted_mantissa parse_long_mantissa(const char *first, const char* last, parse_options options) {
+    decimal d = parse_decimal(first, last, options);
     return compute_float<binary>(d);
 }