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

[arrow] branch master updated: ARROW-8111: [C++] User-defined timestamp parser option to CSV, new TimestampParser interface, and strptime-compatible impl

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 46a17cb  ARROW-8111: [C++] User-defined timestamp parser option to CSV, new TimestampParser interface, and strptime-compatible impl
46a17cb is described below

commit 46a17cba19a6ec9799d1af871bf42b9f87c36b9c
Author: Wes McKinney <we...@apache.org>
AuthorDate: Tue May 5 20:12:51 2020 +0200

    ARROW-8111: [C++] User-defined timestamp parser option to CSV, new TimestampParser interface, and strptime-compatible impl
    
    This builds on the work from #6631 while adding unit tests and additional benchmarks.
    
    I also renamed arrow/util/parsing.h to arrow/util/value_parsing.h to make it slightly more discoverable.
    
    Closes #7088 from wesm/ARROW-8111
    
    Lead-authored-by: Wes McKinney <we...@apache.org>
    Co-authored-by: Antoine Pitrou <an...@python.org>
    Co-authored-by: Alexey Prutskov <al...@intel.com>
    Co-authored-by: Artem Alekseev <ar...@intel.com>
    Signed-off-by: Antoine Pitrou <an...@python.org>
---
 LICENSE.txt                                        |  25 ++
 cpp/src/arrow/CMakeLists.txt                       |   3 +-
 cpp/src/arrow/c/bridge.cc                          |   2 +-
 cpp/src/arrow/compute/kernels/cast.cc              |   2 +-
 cpp/src/arrow/csv/converter.cc                     |  88 ++++-
 cpp/src/arrow/csv/converter_benchmark.cc           |  62 ++--
 cpp/src/arrow/csv/converter_test.cc                |  16 +
 cpp/src/arrow/csv/options.h                        |   8 +
 cpp/src/arrow/filesystem/hdfs.cc                   |   2 +-
 cpp/src/arrow/ipc/json_internal.cc                 |   2 +-
 cpp/src/arrow/ipc/json_simple.cc                   |   2 +-
 cpp/src/arrow/json/converter.cc                    |   2 +-
 cpp/src/arrow/python/deserialize.cc                |   2 +-
 cpp/src/arrow/scalar.cc                            |   2 +-
 cpp/src/arrow/util/CMakeLists.txt                  |   6 +-
 cpp/src/arrow/util/decimal.cc                      |   2 +-
 cpp/src/arrow/util/double_conversion.h             |   2 +-
 cpp/src/arrow/util/uri.cc                          |   2 +-
 .../arrow/util/{parsing.cc => value_parsing.cc}    |  43 ++-
 cpp/src/arrow/util/{parsing.h => value_parsing.h}  | 357 ++++++++++++---------
 ...ing_benchmark.cc => value_parsing_benchmark.cc} |  45 ++-
 ...{parsing_util_test.cc => value_parsing_test.cc} |  44 ++-
 cpp/src/arrow/vendored/datetime.h                  |   4 +-
 cpp/src/arrow/vendored/musl/README.md              |  25 ++
 cpp/src/arrow/vendored/musl/strptime.c             | 237 ++++++++++++++
 .../double_conversion.h => vendored/strptime.h}    |  23 +-
 cpp/src/gandiva/date_utils.h                       |  56 ----
 cpp/src/gandiva/precompiled/testing.h              |   8 +-
 cpp/src/gandiva/precompiled/time_test.cc           |   2 +-
 cpp/src/gandiva/to_date_holder.cc                  |   7 +-
 30 files changed, 803 insertions(+), 278 deletions(-)

diff --git a/LICENSE.txt b/LICENSE.txt
index a078f5e..113a894 100644
--- a/LICENSE.txt
+++ b/LICENSE.txt
@@ -1986,3 +1986,28 @@ SHALL THE COPYRIGHT HOLDERS OR ANYONE DISTRIBUTING THE SOFTWARE BE LIABLE
 FOR ANY DAMAGES OR OTHER LIABILITY, WHETHER IN CONTRACT, TORT OR OTHERWISE,
 ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER
 DEALINGS IN THE SOFTWARE.
+
+--------------------------------------------------------------------------------
+
+The file cpp/src/arrow/vendored/musl/strptime.c has the following license
+
+Copyright © 2005-2020 Rich Felker, et al.
+
+Permission is hereby granted, free of charge, to any person obtaining
+a copy of this software and associated documentation files (the
+"Software"), to deal in the Software without restriction, including
+without limitation the rights to use, copy, modify, merge, publish,
+distribute, sublicense, and/or sell copies of the Software, and to
+permit persons to whom the Software is furnished to do so, subject to
+the following conditions:
+
+The above copyright notice and this permission notice shall be
+included in all copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND,
+EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF
+MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT.
+IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY
+CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT,
+TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE
+SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE.
diff --git a/cpp/src/arrow/CMakeLists.txt b/cpp/src/arrow/CMakeLists.txt
index e888716..b06147f 100644
--- a/cpp/src/arrow/CMakeLists.txt
+++ b/cpp/src/arrow/CMakeLists.txt
@@ -179,7 +179,6 @@ set(ARROW_SRCS
     util/logging.cc
     util/key_value_metadata.cc
     util/memory.cc
-    util/parsing.cc
     util/string.cc
     util/string_builder.cc
     util/task_group.cc
@@ -188,6 +187,7 @@ set(ARROW_SRCS
     util/trie.cc
     util/uri.cc
     util/utf8.cc
+    util/value_parsing.cc
     vendored/base64.cpp
     vendored/datetime/tz.cpp
     vendored/double-conversion/bignum.cc
@@ -200,6 +200,7 @@ set(ARROW_SRCS
     vendored/double-conversion/strtod.cc)
 
 set(ARROW_C_SRCS
+    vendored/musl/strptime.c
     vendored/uriparser/UriCommon.c
     vendored/uriparser/UriCompare.c
     vendored/uriparser/UriEscape.c
diff --git a/cpp/src/arrow/c/bridge.cc b/cpp/src/arrow/c/bridge.cc
index 224e9d1..d1dc77c 100644
--- a/cpp/src/arrow/c/bridge.cc
+++ b/cpp/src/arrow/c/bridge.cc
@@ -37,8 +37,8 @@
 #include "arrow/util/key_value_metadata.h"
 #include "arrow/util/logging.h"
 #include "arrow/util/macros.h"
-#include "arrow/util/parsing.h"
 #include "arrow/util/string_view.h"
+#include "arrow/util/value_parsing.h"
 #include "arrow/visitor_inline.h"
 
 namespace arrow {
diff --git a/cpp/src/arrow/compute/kernels/cast.cc b/cpp/src/arrow/compute/kernels/cast.cc
index 4b636fb..f023133 100644
--- a/cpp/src/arrow/compute/kernels/cast.cc
+++ b/cpp/src/arrow/compute/kernels/cast.cc
@@ -37,9 +37,9 @@
 #include "arrow/util/formatting.h"
 #include "arrow/util/logging.h"
 #include "arrow/util/macros.h"
-#include "arrow/util/parsing.h"  // IWYU pragma: keep
 #include "arrow/util/time.h"
 #include "arrow/util/utf8.h"
+#include "arrow/util/value_parsing.h"  // IWYU pragma: keep
 #include "arrow/visitor_inline.h"
 
 #include "arrow/compute/context.h"
diff --git a/cpp/src/arrow/csv/converter.cc b/cpp/src/arrow/csv/converter.cc
index 4bed2a5..0bc325d 100644
--- a/cpp/src/arrow/csv/converter.cc
+++ b/cpp/src/arrow/csv/converter.cc
@@ -30,14 +30,16 @@
 #include "arrow/status.h"
 #include "arrow/type.h"
 #include "arrow/type_traits.h"
+#include "arrow/util/checked_cast.h"
 #include "arrow/util/decimal.h"
-#include "arrow/util/parsing.h"  // IWYU pragma: keep
 #include "arrow/util/trie.h"
 #include "arrow/util/utf8.h"
+#include "arrow/util/value_parsing.h"  // IWYU pragma: keep
 
 namespace arrow {
 namespace csv {
 
+using internal::checked_cast;
 using internal::StringConverter;
 using internal::Trie;
 using internal::TrieBuilder;
@@ -381,32 +383,98 @@ class NumericConverter : public ConcreteConverter {
 /////////////////////////////////////////////////////////////////////////
 // Concrete Converter for timestamps
 
+namespace {
+
+struct InlineISO8601 {
+  TimeUnit::type unit;
+
+  explicit InlineISO8601(TimeUnit::type unit) : unit(unit) {}
+
+  bool operator()(const char* s, size_t length, int64_t* out) const {
+    return internal::ParseTimestampISO8601(s, length, unit, out);
+  }
+};
+
+struct SingleTimestampParser {
+  const TimestampParser& parser;
+  TimeUnit::type unit;
+
+  SingleTimestampParser(const TimestampParser& parser, TimeUnit::type unit)
+      : parser(parser), unit(unit) {}
+
+  bool operator()(const char* s, size_t length, int64_t* out) const {
+    return this->parser(s, length, this->unit, out);
+  }
+};
+
+struct MultipleTimestampParsers {
+  std::vector<const TimestampParser*> parsers;
+  TimeUnit::type unit;
+
+  MultipleTimestampParsers(const std::vector<std::shared_ptr<TimestampParser>>& parsers,
+                           TimeUnit::type unit)
+      : unit(unit) {
+    for (const auto& parser : parsers) {
+      this->parsers.push_back(parser.get());
+    }
+  }
+
+  bool operator()(const char* s, size_t length, int64_t* out) const {
+    for (const auto& parser : this->parsers) {
+      if (parser->operator()(s, length, this->unit, out)) {
+        return true;
+      }
+    }
+    return false;
+  }
+};
+
+}  // namespace
+
 class TimestampConverter : public ConcreteConverter {
  public:
   using ConcreteConverter::ConcreteConverter;
 
-  Result<std::shared_ptr<Array>> Convert(const BlockParser& parser,
-                                         int32_t col_index) override {
+  template <typename ConvertValue>
+  Status ConvertValuesWith(const BlockParser& parser, int32_t col_index,
+                           const ConvertValue& converter, TimestampBuilder* builder) {
     using value_type = TimestampType::c_type;
-
-    TimestampBuilder builder(type_, pool_);
-    StringConverter<TimestampType> converter(type_);
-
     auto visit = [&](const uint8_t* data, uint32_t size, bool quoted) -> Status {
       value_type value = 0;
       if (IsNull(data, size, quoted)) {
-        builder.UnsafeAppendNull();
+        builder->UnsafeAppendNull();
         return Status::OK();
       }
+
       if (ARROW_PREDICT_FALSE(
               !converter(reinterpret_cast<const char*>(data), size, &value))) {
         return GenericConversionError(type_, data, size);
       }
-      builder.UnsafeAppend(value);
+      builder->UnsafeAppend(value);
       return Status::OK();
     };
+    return parser.VisitColumn(col_index, visit);
+  }
+
+  Result<std::shared_ptr<Array>> Convert(const BlockParser& parser,
+                                         int32_t col_index) override {
+    TimestampBuilder builder(type_, pool_);
     RETURN_NOT_OK(builder.Resize(parser.num_rows()));
-    RETURN_NOT_OK(parser.VisitColumn(col_index, visit));
+
+    TimeUnit::type unit = checked_cast<const TimestampType&>(*type_).unit();
+    if (options_.timestamp_parsers.size() == 0) {
+      // Default to ISO-8601
+      InlineISO8601 converter(unit);
+      RETURN_NOT_OK(ConvertValuesWith(parser, col_index, converter, &builder));
+    } else if (options_.timestamp_parsers.size() == 1) {
+      // Single user-supplied converter
+      SingleTimestampParser converter(*options_.timestamp_parsers[0], unit);
+      RETURN_NOT_OK(ConvertValuesWith(parser, col_index, converter, &builder));
+    } else {
+      // Multiple converters, must iterate for each value
+      MultipleTimestampParsers converter(options_.timestamp_parsers, unit);
+      RETURN_NOT_OK(ConvertValuesWith(parser, col_index, converter, &builder));
+    }
 
     std::shared_ptr<Array> res;
     RETURN_NOT_OK(builder.Finish(&res));
diff --git a/cpp/src/arrow/csv/converter_benchmark.cc b/cpp/src/arrow/csv/converter_benchmark.cc
index 0e6983e..b731188 100644
--- a/cpp/src/arrow/csv/converter_benchmark.cc
+++ b/cpp/src/arrow/csv/converter_benchmark.cc
@@ -20,18 +20,21 @@
 #include <sstream>
 #include <string>
 
+#include "arrow/buffer.h"
 #include "arrow/csv/converter.h"
 #include "arrow/csv/options.h"
 #include "arrow/csv/parser.h"
+#include "arrow/csv/reader.h"
 #include "arrow/csv/test_common.h"
+#include "arrow/io/memory.h"
 #include "arrow/testing/gtest_util.h"
+#include "arrow/util/value_parsing.h"
 
 namespace arrow {
 namespace csv {
 
-static std::shared_ptr<BlockParser> BuildInt64Data(int32_t num_rows) {
-  const std::vector<std::string> base_rows = {"123\n", "4\n",   "-317005557\n",
-                                              "\n",    "N/A\n", "0\n"};
+static std::shared_ptr<BlockParser> BuildFromExamples(
+    const std::vector<std::string>& base_rows, int32_t num_rows) {
   std::vector<std::string> rows;
   for (int32_t i = 0; i < num_rows; ++i) {
     rows.push_back(base_rows[i % base_rows.size()]);
@@ -42,36 +45,40 @@ static std::shared_ptr<BlockParser> BuildInt64Data(int32_t num_rows) {
   return result;
 }
 
+static std::shared_ptr<BlockParser> BuildInt64Data(int32_t num_rows) {
+  const std::vector<std::string> base_rows = {"123\n", "4\n",   "-317005557\n",
+                                              "\n",    "N/A\n", "0\n"};
+  return BuildFromExamples(base_rows, num_rows);
+}
+
 static std::shared_ptr<BlockParser> BuildFloatData(int32_t num_rows) {
   const std::vector<std::string> base_rows = {"0\n", "123.456\n", "-3170.55766\n", "\n",
                                               "N/A\n"};
-  std::vector<std::string> rows;
-  for (int32_t i = 0; i < num_rows; ++i) {
-    rows.push_back(base_rows[i % base_rows.size()]);
-  }
-
-  std::shared_ptr<BlockParser> result;
-  MakeCSVParser(rows, &result);
-  return result;
+  return BuildFromExamples(base_rows, num_rows);
 }
 
 static std::shared_ptr<BlockParser> BuildDecimal128Data(int32_t num_rows) {
   const std::vector<std::string> base_rows = {"0\n", "123.456\n", "-3170.55766\n",
                                               "\n",  "N/A\n",     "1233456789.123456789"};
-  std::vector<std::string> rows;
-  for (int32_t i = 0; i < num_rows; ++i) {
-    rows.push_back(base_rows[i % base_rows.size()]);
-  }
-
-  std::shared_ptr<BlockParser> result;
-  MakeCSVParser(rows, &result);
-  return result;
+  return BuildFromExamples(base_rows, num_rows);
 }
 
 static std::shared_ptr<BlockParser> BuildStringData(int32_t num_rows) {
   return BuildDecimal128Data(num_rows);
 }
 
+static std::shared_ptr<BlockParser> BuildISO8601Data(int32_t num_rows) {
+  const std::vector<std::string> base_rows = {
+      "1917-10-17\n", "2018-09-13\n", "1941-06-22 04:00\n", "1945-05-09 09:45:38\n"};
+  return BuildFromExamples(base_rows, num_rows);
+}
+
+static std::shared_ptr<BlockParser> BuildStrptimeData(int32_t num_rows) {
+  const std::vector<std::string> base_rows = {"10/17/1917\n", "9/13/2018\n",
+                                              "9/5/1945\n"};
+  return BuildFromExamples(base_rows, num_rows);
+}
+
 static void BenchmarkConversion(benchmark::State& state,  // NOLINT non-const reference
                                 BlockParser& parser,
                                 const std::shared_ptr<DataType>& type,
@@ -119,10 +126,27 @@ static void StringConversion(benchmark::State& state) {  // NOLINT non-const ref
   BenchmarkConversion(state, *parser, utf8(), options);
 }
 
+static void TimestampConversionDefault(
+    benchmark::State& state) {  // NOLINT non-const reference
+  auto parser = BuildISO8601Data(num_rows);
+  auto options = ConvertOptions::Defaults();
+  BenchmarkConversion(state, *parser, timestamp(TimeUnit::MILLI), options);
+}
+
+static void TimestampConversionStrptime(
+    benchmark::State& state) {  // NOLINT non-const reference
+  auto parser = BuildStrptimeData(num_rows);
+  auto options = ConvertOptions::Defaults();
+  options.timestamp_parsers.push_back(TimestampParser::MakeStrptime("%m/%d/%Y"));
+  BenchmarkConversion(state, *parser, timestamp(TimeUnit::MILLI), options);
+}
+
 BENCHMARK(Int64Conversion);
 BENCHMARK(FloatConversion);
 BENCHMARK(Decimal128Conversion);
 BENCHMARK(StringConversion);
+BENCHMARK(TimestampConversionDefault);
+BENCHMARK(TimestampConversionStrptime);
 
 }  // namespace csv
 }  // namespace arrow
diff --git a/cpp/src/arrow/csv/converter_test.cc b/cpp/src/arrow/csv/converter_test.cc
index 3d54798..edc8e70 100644
--- a/cpp/src/arrow/csv/converter_test.cc
+++ b/cpp/src/arrow/csv/converter_test.cc
@@ -34,6 +34,7 @@
 #include "arrow/util/checked_cast.h"
 #include "arrow/util/decimal.h"
 #include "arrow/util/logging.h"
+#include "arrow/util/value_parsing.h"
 
 namespace arrow {
 namespace csv {
@@ -374,6 +375,21 @@ TEST(TimestampConversion, CustomNulls) {
                                            {{true}, {false}, {false}}, options);
 }
 
+TEST(TimestampConversion, UserDefinedParsers) {
+  auto options = ConvertOptions::Defaults();
+  auto type = timestamp(TimeUnit::MILLI);
+
+  // Test a single parser
+  options.timestamp_parsers = {TimestampParser::MakeStrptime("%m/%d/%Y")};
+  AssertConversion<TimestampType, int64_t>(type, {"01/02/1970,01/03/1970\n"},
+                                           {{86400000}, {172800000}}, options);
+
+  // Test multiple parsers
+  options.timestamp_parsers.push_back(TimestampParser::MakeISO8601());
+  AssertConversion<TimestampType, int64_t>(type, {"01/02/1970,1970-01-03\n"},
+                                           {{86400000}, {172800000}}, options);
+}
+
 Decimal128 Dec128(util::string_view value) {
   Decimal128 dec;
   int32_t scale = 0;
diff --git a/cpp/src/arrow/csv/options.h b/cpp/src/arrow/csv/options.h
index 2bb3a1e..2fcce50 100644
--- a/cpp/src/arrow/csv/options.h
+++ b/cpp/src/arrow/csv/options.h
@@ -28,6 +28,7 @@
 namespace arrow {
 
 class DataType;
+class TimestampParser;
 
 namespace csv {
 
@@ -100,6 +101,13 @@ struct ARROW_EXPORT ConvertOptions {
   /// This option is ignored if `include_columns` is empty.
   bool include_missing_columns = false;
 
+  /// User-defined timestamp parsers, using the virtual parser interface in
+  /// arrow/util/value_parsing.h. More than one parser can be specified, and
+  /// the CSV conversion logic will try parsing values starting from the
+  /// beginning of this vector. If no parsers are specified, we use the default
+  /// built-in ISO-8601 parser
+  std::vector<std::shared_ptr<TimestampParser>> timestamp_parsers;
+
   /// Create conversion options with default values, including conventional
   /// values for `null_values`, `true_values` and `false_values`
   static ConvertOptions Defaults();
diff --git a/cpp/src/arrow/filesystem/hdfs.cc b/cpp/src/arrow/filesystem/hdfs.cc
index c71787b..aa52f2b 100644
--- a/cpp/src/arrow/filesystem/hdfs.cc
+++ b/cpp/src/arrow/filesystem/hdfs.cc
@@ -26,7 +26,7 @@
 #include "arrow/io/hdfs_internal.h"
 #include "arrow/util/checked_cast.h"
 #include "arrow/util/logging.h"
-#include "arrow/util/parsing.h"
+#include "arrow/util/value_parsing.h"
 #include "arrow/util/windows_fixup.h"
 
 namespace arrow {
diff --git a/cpp/src/arrow/ipc/json_internal.cc b/cpp/src/arrow/ipc/json_internal.cc
index d5a6e9d..c771593 100644
--- a/cpp/src/arrow/ipc/json_internal.cc
+++ b/cpp/src/arrow/ipc/json_internal.cc
@@ -41,8 +41,8 @@
 #include "arrow/util/formatting.h"
 #include "arrow/util/key_value_metadata.h"
 #include "arrow/util/logging.h"
-#include "arrow/util/parsing.h"
 #include "arrow/util/string.h"
+#include "arrow/util/value_parsing.h"
 #include "arrow/visitor_inline.h"
 
 namespace arrow {
diff --git a/cpp/src/arrow/ipc/json_simple.cc b/cpp/src/arrow/ipc/json_simple.cc
index 06a6e35..52ec527 100644
--- a/cpp/src/arrow/ipc/json_simple.cc
+++ b/cpp/src/arrow/ipc/json_simple.cc
@@ -26,8 +26,8 @@
 #include "arrow/type_traits.h"
 #include "arrow/util/checked_cast.h"
 #include "arrow/util/decimal.h"
-#include "arrow/util/parsing.h"
 #include "arrow/util/string_view.h"
+#include "arrow/util/value_parsing.h"
 
 #include "arrow/json/rapidjson_defs.h"
 
diff --git a/cpp/src/arrow/json/converter.cc b/cpp/src/arrow/json/converter.cc
index d3f04ac..0872bb7 100644
--- a/cpp/src/arrow/json/converter.cc
+++ b/cpp/src/arrow/json/converter.cc
@@ -25,8 +25,8 @@
 #include "arrow/json/parser.h"
 #include "arrow/type.h"
 #include "arrow/util/logging.h"
-#include "arrow/util/parsing.h"
 #include "arrow/util/string_view.h"
+#include "arrow/util/value_parsing.h"
 
 namespace arrow {
 namespace json {
diff --git a/cpp/src/arrow/python/deserialize.cc b/cpp/src/arrow/python/deserialize.cc
index 60c6534..418eb87 100644
--- a/cpp/src/arrow/python/deserialize.cc
+++ b/cpp/src/arrow/python/deserialize.cc
@@ -38,7 +38,7 @@
 #include "arrow/table.h"
 #include "arrow/util/checked_cast.h"
 #include "arrow/util/logging.h"
-#include "arrow/util/parsing.h"
+#include "arrow/util/value_parsing.h"
 
 #include "arrow/python/common.h"
 #include "arrow/python/datetime.h"
diff --git a/cpp/src/arrow/scalar.cc b/cpp/src/arrow/scalar.cc
index 9ef0527..0cb9af7 100644
--- a/cpp/src/arrow/scalar.cc
+++ b/cpp/src/arrow/scalar.cc
@@ -30,8 +30,8 @@
 #include "arrow/util/formatting.h"
 #include "arrow/util/hashing.h"
 #include "arrow/util/logging.h"
-#include "arrow/util/parsing.h"
 #include "arrow/util/time.h"
+#include "arrow/util/value_parsing.h"
 #include "arrow/visitor_inline.h"
 
 namespace arrow {
diff --git a/cpp/src/arrow/util/CMakeLists.txt b/cpp/src/arrow/util/CMakeLists.txt
index ce08839..bd31cc6 100644
--- a/cpp/src/arrow/util/CMakeLists.txt
+++ b/cpp/src/arrow/util/CMakeLists.txt
@@ -52,7 +52,6 @@ add_arrow_test(utility-test
                ${IO_UTIL_TEST_SOURCES}
                iterator_test.cc
                logging_test.cc
-               parsing_util_test.cc
                range_test.cc
                rle_encoding_test.cc
                stl_util_test.cc
@@ -60,7 +59,8 @@ add_arrow_test(utility-test
                time_test.cc
                trie_test.cc
                uri_test.cc
-               utf8_util_test.cc)
+               utf8_util_test.cc
+               value_parsing_test.cc)
 
 add_arrow_test(threading-utility-test
                SOURCES
@@ -74,8 +74,8 @@ add_arrow_benchmark(decimal_benchmark)
 add_arrow_benchmark(hashing_benchmark)
 add_arrow_benchmark(int_util_benchmark)
 add_arrow_benchmark(machine_benchmark)
-add_arrow_benchmark(number_parsing_benchmark)
 add_arrow_benchmark(range_benchmark)
 add_arrow_benchmark(thread_pool_benchmark)
 add_arrow_benchmark(trie_benchmark)
 add_arrow_benchmark(utf8_util_benchmark)
+add_arrow_benchmark(value_parsing_benchmark)
diff --git a/cpp/src/arrow/util/decimal.cc b/cpp/src/arrow/util/decimal.cc
index af01675..bda0ea3 100644
--- a/cpp/src/arrow/util/decimal.cc
+++ b/cpp/src/arrow/util/decimal.cc
@@ -33,7 +33,7 @@
 #include "arrow/util/int_util.h"
 #include "arrow/util/logging.h"
 #include "arrow/util/macros.h"
-#include "arrow/util/parsing.h"
+#include "arrow/util/value_parsing.h"
 
 namespace arrow {
 
diff --git a/cpp/src/arrow/util/double_conversion.h b/cpp/src/arrow/util/double_conversion.h
index 2c5f325..8edc654 100644
--- a/cpp/src/arrow/util/double_conversion.h
+++ b/cpp/src/arrow/util/double_conversion.h
@@ -17,7 +17,7 @@
 
 #pragma once
 
-#include "arrow/vendored/double-conversion/double-conversion.h"
+#include "arrow/vendored/double-conversion/double-conversion.h"  // IWYU pragma: export
 
 namespace arrow {
 namespace util {
diff --git a/cpp/src/arrow/util/uri.cc b/cpp/src/arrow/util/uri.cc
index 799d4bf..a9bf0c5 100644
--- a/cpp/src/arrow/util/uri.cc
+++ b/cpp/src/arrow/util/uri.cc
@@ -21,8 +21,8 @@
 #include <sstream>
 #include <vector>
 
-#include "arrow/util/parsing.h"
 #include "arrow/util/string_view.h"
+#include "arrow/util/value_parsing.h"
 #include "arrow/vendored/uriparser/Uri.h"
 
 namespace arrow {
diff --git a/cpp/src/arrow/util/parsing.cc b/cpp/src/arrow/util/value_parsing.cc
similarity index 71%
rename from cpp/src/arrow/util/parsing.cc
rename to cpp/src/arrow/util/value_parsing.cc
index 0f8dab5..0fe7e62 100644
--- a/cpp/src/arrow/util/parsing.cc
+++ b/cpp/src/arrow/util/value_parsing.cc
@@ -15,7 +15,11 @@
 // specific language governing permissions and limitations
 // under the License.
 
-#include "arrow/util/parsing.h"
+#include "arrow/util/value_parsing.h"
+
+#include <string>
+#include <utility>
+
 #include "arrow/util/double_conversion.h"
 
 namespace arrow {
@@ -79,5 +83,42 @@ bool StringToFloatConverter::StringToFloat(const char* s, size_t length, double*
   return true;
 }
 
+// ----------------------------------------------------------------------
+// strptime-like parsing
+
+class StrptimeTimestampParser : public TimestampParser {
+ public:
+  explicit StrptimeTimestampParser(std::string format) : format_(std::move(format)) {}
+
+  bool operator()(const char* s, size_t length, TimeUnit::type out_unit,
+                  int64_t* out) const override {
+    return ParseTimestampStrptime(s, length, format_.c_str(),
+                                  /*ignore_time_in_day=*/false,
+                                  /*allow_trailing_chars=*/false, out_unit, out);
+  }
+
+ private:
+  std::string format_;
+};
+
+class ISO8601Parser : public TimestampParser {
+ public:
+  ISO8601Parser() {}
+
+  bool operator()(const char* s, size_t length, TimeUnit::type out_unit,
+                  int64_t* out) const override {
+    return ParseTimestampISO8601(s, length, out_unit, out);
+  }
+};
+
 }  // namespace internal
+
+std::shared_ptr<TimestampParser> TimestampParser::MakeStrptime(std::string format) {
+  return std::make_shared<internal::StrptimeTimestampParser>(std::move(format));
+}
+
+std::shared_ptr<TimestampParser> TimestampParser::MakeISO8601() {
+  return std::make_shared<internal::ISO8601Parser>();
+}
+
 }  // namespace arrow
diff --git a/cpp/src/arrow/util/parsing.h b/cpp/src/arrow/util/value_parsing.h
similarity index 63%
rename from cpp/src/arrow/util/parsing.h
rename to cpp/src/arrow/util/value_parsing.h
index 4bd4d7e..d797891 100644
--- a/cpp/src/arrow/util/parsing.h
+++ b/cpp/src/arrow/util/value_parsing.h
@@ -21,18 +21,38 @@
 
 #include <cassert>
 #include <chrono>
+#include <cstddef>
+#include <cstdint>
 #include <limits>
 #include <memory>
 #include <string>
 #include <type_traits>
 
 #include "arrow/type.h"
-#include "arrow/type_traits.h"
 #include "arrow/util/checked_cast.h"
-#include "arrow/util/config.h"
+#include "arrow/util/macros.h"
+#include "arrow/util/visibility.h"
 #include "arrow/vendored/datetime.h"
+#include "arrow/vendored/strptime.h"
 
 namespace arrow {
+
+/// \brief A virtual string to timestamp parser
+class ARROW_EXPORT TimestampParser {
+ public:
+  virtual ~TimestampParser() = default;
+
+  virtual bool operator()(const char* s, size_t length, TimeUnit::type out_unit,
+                          int64_t* out) const = 0;
+
+  /// \brief Create a TimestampParser that recognizes strptime-like format strings
+  static std::shared_ptr<TimestampParser> MakeStrptime(std::string format);
+
+  /// \brief Create a TimestampParser that recognizes (locale-agnostic) ISO8601
+  /// timestamps
+  static std::shared_ptr<TimestampParser> MakeISO8601();
+};
+
 namespace internal {
 
 /// \brief A class providing conversion from strings to some Arrow data types
@@ -356,177 +376,228 @@ class StringConverter<Int64Type> : public StringToSignedIntConverterMixin<Int64T
   using StringToSignedIntConverterMixin<Int64Type>::StringToSignedIntConverterMixin;
 };
 
-template <>
-class StringConverter<TimestampType> {
- public:
-  using value_type = TimestampType::c_type;
+// Inline-able ISO-8601 parser
 
-  explicit StringConverter(const std::shared_ptr<DataType>& type)
-      : unit_(checked_cast<TimestampType*>(type.get())->unit()) {}
+namespace detail {
 
-  bool operator()(const char* s, size_t length, value_type* out) {
-    // We allow the following formats:
-    // - "YYYY-MM-DD"
-    // - "YYYY-MM-DD[ T]hh"
-    // - "YYYY-MM-DD[ T]hhZ"
-    // - "YYYY-MM-DD[ T]hh:mm"
-    // - "YYYY-MM-DD[ T]hh:mmZ"
-    // - "YYYY-MM-DD[ T]hh:mm:ss"
-    // - "YYYY-MM-DD[ T]hh:mm:ssZ"
-    // UTC is always assumed, and the DataType's timezone is ignored.
-    arrow_vendored::date::year_month_day ymd;
-    if (ARROW_PREDICT_FALSE(length < 10)) {
-      return false;
-    }
-    if (length == 10) {
-      if (ARROW_PREDICT_FALSE(!ParseYYYY_MM_DD(s, &ymd))) {
-        return false;
-      }
-      return ConvertTimePoint(arrow_vendored::date::sys_days(ymd), out);
-    }
-    if (ARROW_PREDICT_FALSE(s[10] != ' ') && ARROW_PREDICT_FALSE(s[10] != 'T')) {
-      return false;
-    }
-    if (s[length - 1] == 'Z') {
-      --length;
-    }
-    if (length == 13) {
-      if (ARROW_PREDICT_FALSE(!ParseYYYY_MM_DD(s, &ymd))) {
-        return false;
-      }
-      std::chrono::duration<value_type> seconds;
-      if (ARROW_PREDICT_FALSE(!ParseHH(s + 11, &seconds))) {
-        return false;
-      }
-      return ConvertTimePoint(arrow_vendored::date::sys_days(ymd) + seconds, out);
-    }
-    if (length == 16) {
-      if (ARROW_PREDICT_FALSE(!ParseYYYY_MM_DD(s, &ymd))) {
-        return false;
-      }
-      std::chrono::duration<value_type> seconds;
-      if (ARROW_PREDICT_FALSE(!ParseHH_MM(s + 11, &seconds))) {
-        return false;
-      }
-      return ConvertTimePoint(arrow_vendored::date::sys_days(ymd) + seconds, out);
-    }
-    if (length == 19) {
-      if (ARROW_PREDICT_FALSE(!ParseYYYY_MM_DD(s, &ymd))) {
-        return false;
-      }
-      std::chrono::duration<value_type> seconds;
-      if (ARROW_PREDICT_FALSE(!ParseHH_MM_SS(s + 11, &seconds))) {
-        return false;
-      }
-      return ConvertTimePoint(arrow_vendored::date::sys_days(ymd) + seconds, out);
-    }
+using ts_type = TimestampType::c_type;
+
+template <class TimePoint>
+static inline ts_type ConvertTimePoint(TimePoint tp, TimeUnit::type unit) {
+  auto duration = tp.time_since_epoch();
+  switch (unit) {
+    case TimeUnit::SECOND:
+      return std::chrono::duration_cast<std::chrono::seconds>(duration).count();
+    case TimeUnit::MILLI:
+      return std::chrono::duration_cast<std::chrono::milliseconds>(duration).count();
+    case TimeUnit::MICRO:
+      return std::chrono::duration_cast<std::chrono::microseconds>(duration).count();
+    case TimeUnit::NANO:
+      return std::chrono::duration_cast<std::chrono::nanoseconds>(duration).count();
+    default:
+      // Compiler errors without default case even though all enum cases are handled
+      assert(0);
+      return 0;
+  }
+}
+
+static inline bool ParseYYYY_MM_DD(const char* s,
+                                   arrow_vendored::date::year_month_day* out) {
+  uint16_t year;
+  uint8_t month, day;
+  if (ARROW_PREDICT_FALSE(s[4] != '-') || ARROW_PREDICT_FALSE(s[7] != '-')) {
+    return false;
+  }
+  if (ARROW_PREDICT_FALSE(!ParseUnsigned(s + 0, 4, &year))) {
+    return false;
+  }
+  if (ARROW_PREDICT_FALSE(!ParseUnsigned(s + 5, 2, &month))) {
     return false;
   }
+  if (ARROW_PREDICT_FALSE(!ParseUnsigned(s + 8, 2, &day))) {
+    return false;
+  }
+  *out = {arrow_vendored::date::year{year}, arrow_vendored::date::month{month},
+          arrow_vendored::date::day{day}};
+  return out->ok();
+}
 
- protected:
-  template <class TimePoint>
-  bool ConvertTimePoint(TimePoint tp, value_type* out) {
-    auto duration = tp.time_since_epoch();
-    switch (unit_) {
-      case TimeUnit::SECOND:
-        *out = std::chrono::duration_cast<std::chrono::seconds>(duration).count();
-        return true;
-      case TimeUnit::MILLI:
-        *out = std::chrono::duration_cast<std::chrono::milliseconds>(duration).count();
-        return true;
-      case TimeUnit::MICRO:
-        *out = std::chrono::duration_cast<std::chrono::microseconds>(duration).count();
-        return true;
-      case TimeUnit::NANO:
-        *out = std::chrono::duration_cast<std::chrono::nanoseconds>(duration).count();
-        return true;
-    }
-    // Unreachable, but suppress compiler warning
-    assert(0);
-    *out = 0;
-    return true;
+static inline bool ParseHH(const char* s, std::chrono::duration<ts_type>* out) {
+  uint8_t hours;
+  if (ARROW_PREDICT_FALSE(!ParseUnsigned(s + 0, 2, &hours))) {
+    return false;
   }
+  if (ARROW_PREDICT_FALSE(hours >= 24)) {
+    return false;
+  }
+  *out = std::chrono::duration<ts_type>(3600U * hours);
+  return true;
+}
 
-  bool ParseYYYY_MM_DD(const char* s, arrow_vendored::date::year_month_day* out) {
-    uint16_t year;
-    uint8_t month, day;
-    if (ARROW_PREDICT_FALSE(s[4] != '-') || ARROW_PREDICT_FALSE(s[7] != '-')) {
-      return false;
-    }
-    if (ARROW_PREDICT_FALSE(!detail::ParseUnsigned(s + 0, 4, &year))) {
-      return false;
-    }
-    if (ARROW_PREDICT_FALSE(!detail::ParseUnsigned(s + 5, 2, &month))) {
-      return false;
-    }
-    if (ARROW_PREDICT_FALSE(!detail::ParseUnsigned(s + 8, 2, &day))) {
-      return false;
-    }
-    *out = {arrow_vendored::date::year{year}, arrow_vendored::date::month{month},
-            arrow_vendored::date::day{day}};
-    return out->ok();
+static inline bool ParseHH_MM(const char* s, std::chrono::duration<ts_type>* out) {
+  uint8_t hours, minutes;
+  if (ARROW_PREDICT_FALSE(s[2] != ':')) {
+    return false;
+  }
+  if (ARROW_PREDICT_FALSE(!ParseUnsigned(s + 0, 2, &hours))) {
+    return false;
   }
+  if (ARROW_PREDICT_FALSE(!ParseUnsigned(s + 3, 2, &minutes))) {
+    return false;
+  }
+  if (ARROW_PREDICT_FALSE(hours >= 24)) {
+    return false;
+  }
+  if (ARROW_PREDICT_FALSE(minutes >= 60)) {
+    return false;
+  }
+  *out = std::chrono::duration<ts_type>(3600U * hours + 60U * minutes);
+  return true;
+}
 
-  bool ParseHH(const char* s, std::chrono::duration<value_type>* out) {
-    uint8_t hours;
-    if (ARROW_PREDICT_FALSE(!detail::ParseUnsigned(s + 0, 2, &hours))) {
-      return false;
-    }
-    if (ARROW_PREDICT_FALSE(hours >= 24)) {
-      return false;
-    }
-    *out = std::chrono::duration<value_type>(3600U * hours);
-    return true;
+static inline bool ParseHH_MM_SS(const char* s, std::chrono::duration<ts_type>* out) {
+  uint8_t hours, minutes, seconds;
+  if (ARROW_PREDICT_FALSE(s[2] != ':') || ARROW_PREDICT_FALSE(s[5] != ':')) {
+    return false;
+  }
+  if (ARROW_PREDICT_FALSE(!ParseUnsigned(s + 0, 2, &hours))) {
+    return false;
+  }
+  if (ARROW_PREDICT_FALSE(!ParseUnsigned(s + 3, 2, &minutes))) {
+    return false;
   }
+  if (ARROW_PREDICT_FALSE(!ParseUnsigned(s + 6, 2, &seconds))) {
+    return false;
+  }
+  if (ARROW_PREDICT_FALSE(hours >= 24)) {
+    return false;
+  }
+  if (ARROW_PREDICT_FALSE(minutes >= 60)) {
+    return false;
+  }
+  if (ARROW_PREDICT_FALSE(seconds >= 60)) {
+    return false;
+  }
+  *out = std::chrono::duration<ts_type>(3600U * hours + 60U * minutes + seconds);
+  return true;
+}
 
-  bool ParseHH_MM(const char* s, std::chrono::duration<value_type>* out) {
-    uint8_t hours, minutes;
-    if (ARROW_PREDICT_FALSE(s[2] != ':')) {
-      return false;
-    }
-    if (ARROW_PREDICT_FALSE(!detail::ParseUnsigned(s + 0, 2, &hours))) {
-      return false;
-    }
-    if (ARROW_PREDICT_FALSE(!detail::ParseUnsigned(s + 3, 2, &minutes))) {
-      return false;
-    }
-    if (ARROW_PREDICT_FALSE(hours >= 24)) {
-      return false;
-    }
-    if (ARROW_PREDICT_FALSE(minutes >= 60)) {
+}  // namespace detail
+
+static inline bool ParseTimestampISO8601(const char* s, size_t length,
+                                         TimeUnit::type unit,
+                                         TimestampType::c_type* out) {
+  using ts_type = TimestampType::c_type;
+
+  // We allow the following formats:
+  // - "YYYY-MM-DD"
+  // - "YYYY-MM-DD[ T]hh"
+  // - "YYYY-MM-DD[ T]hhZ"
+  // - "YYYY-MM-DD[ T]hh:mm"
+  // - "YYYY-MM-DD[ T]hh:mmZ"
+  // - "YYYY-MM-DD[ T]hh:mm:ss"
+  // - "YYYY-MM-DD[ T]hh:mm:ssZ"
+  // UTC is always assumed, and the DataType's timezone is ignored.
+  arrow_vendored::date::year_month_day ymd;
+  if (ARROW_PREDICT_FALSE(length < 10)) {
+    return false;
+  }
+  if (length == 10) {
+    if (ARROW_PREDICT_FALSE(!detail::ParseYYYY_MM_DD(s, &ymd))) {
       return false;
     }
-    *out = std::chrono::duration<value_type>(3600U * hours + 60U * minutes);
+    *out = detail::ConvertTimePoint(arrow_vendored::date::sys_days(ymd), unit);
     return true;
   }
-
-  bool ParseHH_MM_SS(const char* s, std::chrono::duration<value_type>* out) {
-    uint8_t hours, minutes, seconds;
-    if (ARROW_PREDICT_FALSE(s[2] != ':') || ARROW_PREDICT_FALSE(s[5] != ':')) {
-      return false;
-    }
-    if (ARROW_PREDICT_FALSE(!detail::ParseUnsigned(s + 0, 2, &hours))) {
+  if (ARROW_PREDICT_FALSE(s[10] != ' ') && ARROW_PREDICT_FALSE(s[10] != 'T')) {
+    return false;
+  }
+  if (s[length - 1] == 'Z') {
+    --length;
+  }
+  if (length == 13) {
+    if (ARROW_PREDICT_FALSE(!detail::ParseYYYY_MM_DD(s, &ymd))) {
       return false;
     }
-    if (ARROW_PREDICT_FALSE(!detail::ParseUnsigned(s + 3, 2, &minutes))) {
+    std::chrono::duration<ts_type> seconds;
+    if (ARROW_PREDICT_FALSE(!detail::ParseHH(s + 11, &seconds))) {
       return false;
     }
-    if (ARROW_PREDICT_FALSE(!detail::ParseUnsigned(s + 6, 2, &seconds))) {
+    *out = detail::ConvertTimePoint(arrow_vendored::date::sys_days(ymd) + seconds, unit);
+    return true;
+  }
+  if (length == 16) {
+    if (ARROW_PREDICT_FALSE(!detail::ParseYYYY_MM_DD(s, &ymd))) {
       return false;
     }
-    if (ARROW_PREDICT_FALSE(hours >= 24)) {
+    std::chrono::duration<ts_type> seconds;
+    if (ARROW_PREDICT_FALSE(!detail::ParseHH_MM(s + 11, &seconds))) {
       return false;
     }
-    if (ARROW_PREDICT_FALSE(minutes >= 60)) {
+    *out = detail::ConvertTimePoint(arrow_vendored::date::sys_days(ymd) + seconds, unit);
+    return true;
+  }
+  if (length == 19) {
+    if (ARROW_PREDICT_FALSE(!detail::ParseYYYY_MM_DD(s, &ymd))) {
       return false;
     }
-    if (ARROW_PREDICT_FALSE(seconds >= 60)) {
+    std::chrono::duration<ts_type> seconds;
+    if (ARROW_PREDICT_FALSE(!detail::ParseHH_MM_SS(s + 11, &seconds))) {
       return false;
     }
-    *out = std::chrono::duration<value_type>(3600U * hours + 60U * minutes + seconds);
+    *out = detail::ConvertTimePoint(arrow_vendored::date::sys_days(ymd) + seconds, unit);
     return true;
   }
+  return false;
+}
+
+/// \brief Returns time since the UNIX epoch in the requested unit
+static inline bool ParseTimestampStrptime(const char* buf, size_t length,
+                                          const char* format, bool ignore_time_in_day,
+                                          bool allow_trailing_chars, TimeUnit::type unit,
+                                          int64_t* out) {
+  // NOTE: strptime() is more than 10x faster than arrow_vendored::date::parse().
+  // The buffer may not be nul-terminated
+  std::string clean_copy(buf, length);
+  struct tm result;
+  memset(&result, 0, sizeof(struct tm));
+#ifdef _WIN32
+  char* ret = arrow_strptime(clean_copy.c_str(), format, &result);
+#else
+  char* ret = strptime(clean_copy.c_str(), format, &result);
+#endif
+  if (ret == NULLPTR) {
+    return false;
+  }
+  if (!allow_trailing_chars && static_cast<size_t>(ret - clean_copy.c_str()) != length) {
+    return false;
+  }
+  // ignore the time part
+  arrow_vendored::date::sys_seconds secs =
+      arrow_vendored::date::sys_days(arrow_vendored::date::year(result.tm_year + 1900) /
+                                     (result.tm_mon + 1) / result.tm_mday);
+  if (!ignore_time_in_day) {
+    secs += (std::chrono::hours(result.tm_hour) + std::chrono::minutes(result.tm_min) +
+             std::chrono::seconds(result.tm_sec));
+  }
+  *out = detail::ConvertTimePoint(secs, unit);
+  return true;
+}
+
+// A StringConverter that parses ISO8601 at a fixed unit
+template <>
+class StringConverter<TimestampType> {
+ public:
+  using value_type = TimestampType::c_type;
+
+  explicit StringConverter(const std::shared_ptr<DataType>& type)
+      : unit_(checked_cast<TimestampType*>(type.get())->unit()) {}
+
+  bool operator()(const char* s, size_t length, value_type* out) {
+    return ParseTimestampISO8601(s, length, unit_, out);
+  }
 
+ private:
   const TimeUnit::type unit_;
 };
 
diff --git a/cpp/src/arrow/util/number_parsing_benchmark.cc b/cpp/src/arrow/util/value_parsing_benchmark.cc
similarity index 86%
rename from cpp/src/arrow/util/number_parsing_benchmark.cc
rename to cpp/src/arrow/util/value_parsing_benchmark.cc
index dc69bdf..f09c016 100644
--- a/cpp/src/arrow/util/number_parsing_benchmark.cc
+++ b/cpp/src/arrow/util/value_parsing_benchmark.cc
@@ -17,16 +17,23 @@
 
 #include "benchmark/benchmark.h"
 
+#include <algorithm>
+#include <cstdint>
+#include <cstdlib>
+#include <iostream>
 #include <limits>
-#include <numeric>
+#include <memory>
 #include <string>
 #include <type_traits>
 #include <vector>
 
+#include "arrow/status.h"
 #include "arrow/testing/gtest_util.h"
 #include "arrow/testing/random.h"
+#include "arrow/type.h"
 #include "arrow/util/formatting.h"
-#include "arrow/util/parsing.h"
+#include "arrow/util/string_view.h"
+#include "arrow/util/value_parsing.h"
 
 namespace arrow {
 namespace internal {
@@ -137,19 +144,18 @@ static void FloatParsing(benchmark::State& state) {  // NOLINT non-const referen
   state.SetItemsProcessed(state.iterations() * strings.size());
 }
 
-template <TimeUnit::type UNIT>
-static void TimestampParsing(benchmark::State& state) {  // NOLINT non-const reference
+static void BenchTimestampParsing(
+    benchmark::State& state, TimeUnit::type unit,
+    const TimestampParser& parser) {  // NOLINT non-const reference
   using c_type = TimestampType::c_type;
 
   auto strings = MakeTimestampStrings(1000);
-  auto type = timestamp(UNIT);
-  StringConverter<TimestampType> converter(type);
 
-  while (state.KeepRunning()) {
+  for (auto _ : state) {
     c_type total = 0;
     for (const auto& s : strings) {
       c_type value;
-      if (!converter(s.data(), s.length(), &value)) {
+      if (!parser(s.data(), s.length(), unit, &value)) {
         std::cerr << "Conversion failed for '" << s << "'";
         std::abort();
       }
@@ -160,6 +166,20 @@ static void TimestampParsing(benchmark::State& state) {  // NOLINT non-const ref
   state.SetItemsProcessed(state.iterations() * strings.size());
 }
 
+template <TimeUnit::type UNIT>
+static void TimestampParsingISO8601(
+    benchmark::State& state) {  // NOLINT non-const reference
+  auto parser = TimestampParser::MakeISO8601();
+  BenchTimestampParsing(state, UNIT, *parser);
+}
+
+template <TimeUnit::type UNIT>
+static void TimestampParsingStrptime(
+    benchmark::State& state) {  // NOLINT non-const reference
+  auto parser = TimestampParser::MakeStrptime("%Y-%m-%d %H:%M:%S");
+  BenchTimestampParsing(state, UNIT, *parser);
+}
+
 struct DummyAppender {
   Status operator()(util::string_view v) {
     if (pos_ >= static_cast<int32_t>(v.size())) {
@@ -215,10 +235,11 @@ BENCHMARK_TEMPLATE(IntegerParsing, UInt64Type);
 BENCHMARK_TEMPLATE(FloatParsing, FloatType);
 BENCHMARK_TEMPLATE(FloatParsing, DoubleType);
 
-BENCHMARK_TEMPLATE(TimestampParsing, TimeUnit::SECOND);
-BENCHMARK_TEMPLATE(TimestampParsing, TimeUnit::MILLI);
-BENCHMARK_TEMPLATE(TimestampParsing, TimeUnit::MICRO);
-BENCHMARK_TEMPLATE(TimestampParsing, TimeUnit::NANO);
+BENCHMARK_TEMPLATE(TimestampParsingISO8601, TimeUnit::SECOND);
+BENCHMARK_TEMPLATE(TimestampParsingISO8601, TimeUnit::MILLI);
+BENCHMARK_TEMPLATE(TimestampParsingISO8601, TimeUnit::MICRO);
+BENCHMARK_TEMPLATE(TimestampParsingISO8601, TimeUnit::NANO);
+BENCHMARK_TEMPLATE(TimestampParsingStrptime, TimeUnit::MILLI);
 
 BENCHMARK_TEMPLATE(IntegerFormatting, Int8Type);
 BENCHMARK_TEMPLATE(IntegerFormatting, Int16Type);
diff --git a/cpp/src/arrow/util/parsing_util_test.cc b/cpp/src/arrow/util/value_parsing_test.cc
similarity index 90%
rename from cpp/src/arrow/util/parsing_util_test.cc
rename to cpp/src/arrow/util/value_parsing_test.cc
index 0daec5b..830ba67 100644
--- a/cpp/src/arrow/util/parsing_util_test.cc
+++ b/cpp/src/arrow/util/value_parsing_test.cc
@@ -16,16 +16,16 @@
 // under the License.
 
 #include <string>
+#include <vector>
 
 #include <gtest/gtest.h>
 
 #include "arrow/testing/gtest_util.h"
 #include "arrow/type.h"
-#include "arrow/util/parsing.h"
+#include "arrow/util/value_parsing.h"
 
 namespace arrow {
-
-using internal::StringConverter;
+namespace internal {
 
 template <typename ConverterType, typename C_TYPE>
 void AssertConversion(ConverterType& converter, const std::string& s, C_TYPE expected) {
@@ -251,7 +251,7 @@ TEST(StringConversion, ToUInt64) {
   AssertConversionFails(converter, "e");
 }
 
-TEST(StringConversion, ToTimestampDate) {
+TEST(StringConversion, ToTimestampDate_ISO8601) {
   {
     StringConverter<TimestampType> converter(timestamp(TimeUnit::SECOND));
 
@@ -301,7 +301,7 @@ TEST(StringConversion, ToTimestampDate) {
   }
 }
 
-TEST(StringConversion, ToTimestampDateTime) {
+TEST(StringConversion, ToTimestampDateTime_ISO8601) {
   {
     StringConverter<TimestampType> converter(timestamp(TimeUnit::SECOND));
 
@@ -358,4 +358,38 @@ TEST(StringConversion, ToTimestampDateTime) {
   }
 }
 
+TEST(TimestampParser, StrptimeParser) {
+  std::string format = "%m/%d/%Y %H:%M:%S";
+  auto parser = TimestampParser::MakeStrptime(format);
+
+  struct Case {
+    std::string value;
+    std::string iso8601;
+  };
+
+  std::vector<Case> cases = {{"5/31/2000 12:34:56", "2000-05-31 12:34:56"},
+                             {"5/31/2000 00:00:00", "2000-05-31 00:00:00"}};
+
+  std::vector<TimeUnit::type> units = {TimeUnit::SECOND, TimeUnit::MILLI, TimeUnit::MICRO,
+                                       TimeUnit::NANO};
+
+  for (auto unit : units) {
+    for (const auto& case_ : cases) {
+      int64_t converted, expected;
+      ASSERT_TRUE((*parser)(case_.value.c_str(), case_.value.size(), unit, &converted));
+      ASSERT_TRUE(ParseTimestampISO8601(case_.iso8601.c_str(), case_.iso8601.size(), unit,
+                                        &expected));
+      ASSERT_EQ(expected, converted);
+    }
+  }
+
+  // Unparseable strings
+  std::vector<std::string> unparseables = {"foo", "5/1/2000", "5/1/2000 12:34:56:6"};
+  for (auto& value : unparseables) {
+    int64_t dummy;
+    ASSERT_FALSE((*parser)(value.c_str(), value.size(), TimeUnit::SECOND, &dummy));
+  }
+}
+
+}  // namespace internal
 }  // namespace arrow
diff --git a/cpp/src/arrow/vendored/datetime.h b/cpp/src/arrow/vendored/datetime.h
index 424313a..f3b8b9d 100644
--- a/cpp/src/arrow/vendored/datetime.h
+++ b/cpp/src/arrow/vendored/datetime.h
@@ -17,5 +17,5 @@
 
 #pragma once
 
-#include "arrow/vendored/datetime/date.h"
-#include "arrow/vendored/datetime/tz.h"
+#include "arrow/vendored/datetime/date.h"  // IWYU pragma: export
+#include "arrow/vendored/datetime/tz.h"    // IWYU pragma: export
diff --git a/cpp/src/arrow/vendored/musl/README.md b/cpp/src/arrow/vendored/musl/README.md
new file mode 100644
index 0000000..40962a1
--- /dev/null
+++ b/cpp/src/arrow/vendored/musl/README.md
@@ -0,0 +1,25 @@
+<!--
+Copyright © 2005-2020 Rich Felker, et al.
+
+Permission is hereby granted, free of charge, to any person obtaining
+a copy of this software and associated documentation files (the
+"Software"), to deal in the Software without restriction, including
+without limitation the rights to use, copy, modify, merge, publish,
+distribute, sublicense, and/or sell copies of the Software, and to
+permit persons to whom the Software is furnished to do so, subject to
+the following conditions:
+
+The above copyright notice and this permission notice shall be
+included in all copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND,
+EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF
+MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT.
+IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY
+CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT,
+TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE
+SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE.
+-->
+
+Assorted utility functions are adapted from the musl libc project
+(https://musl.libc.org/).
diff --git a/cpp/src/arrow/vendored/musl/strptime.c b/cpp/src/arrow/vendored/musl/strptime.c
new file mode 100644
index 0000000..e8111f5
--- /dev/null
+++ b/cpp/src/arrow/vendored/musl/strptime.c
@@ -0,0 +1,237 @@
+// Vendored from musl git commit 593caa456309714402ca4cb77c3770f4c24da9da
+// + adaptations
+
+#include "arrow/vendored/strptime.h"
+
+#include <ctype.h>
+#include <stddef.h>
+#include <stdlib.h>
+#include <string.h>
+
+#ifdef _WIN32
+#define strncasecmp _strnicmp
+#define strcasecmp _stricmp
+#else
+#include <strings.h>
+#endif
+
+#undef HAVE_LANGINFO
+
+#ifndef _WIN32
+#define HAVE_LANGINFO 1
+#endif
+
+#ifdef HAVE_LANGINFO
+#include <langinfo.h>
+#endif
+
+#define strptime arrow_strptime
+
+char *strptime(const char *__restrict s, const char *__restrict f, struct tm *__restrict tm)
+{
+	int i, w, neg, adj, min, range, *dest, dummy;
+#ifdef HAVE_LANGINFO
+	const char *ex;
+	size_t len;
+#endif
+	int want_century = 0, century = 0, relyear = 0;
+	while (*f) {
+		if (*f != '%') {
+			if (isspace(*f)) for (; *s && isspace(*s); s++);
+			else if (*s != *f) return 0;
+			else s++;
+			f++;
+			continue;
+		}
+		f++;
+		if (*f == '+') f++;
+		if (isdigit(*f)) {
+			char *new_f;
+			w=strtoul(f, &new_f, 10);
+			f = new_f;
+		} else {
+			w=-1;
+		}
+		adj=0;
+		switch (*f++) {
+#ifdef HAVE_LANGINFO
+		case 'a': case 'A':
+			dest = &tm->tm_wday;
+			min = ABDAY_1;
+			range = 7;
+			goto symbolic_range;
+		case 'b': case 'B': case 'h':
+			dest = &tm->tm_mon;
+			min = ABMON_1;
+			range = 12;
+			goto symbolic_range;
+		case 'c':
+			s = strptime(s, nl_langinfo(D_T_FMT), tm);
+			if (!s) return 0;
+			break;
+#endif
+		case 'C':
+			dest = &century;
+			if (w<0) w=2;
+			want_century |= 2;
+			goto numeric_digits;
+		case 'd': case 'e':
+			dest = &tm->tm_mday;
+			min = 1;
+			range = 31;
+			goto numeric_range;
+		case 'D':
+			s = strptime(s, "%m/%d/%y", tm);
+			if (!s) return 0;
+			break;
+		case 'H':
+			dest = &tm->tm_hour;
+			min = 0;
+			range = 24;
+			goto numeric_range;
+		case 'I':
+			dest = &tm->tm_hour;
+			min = 1;
+			range = 12;
+			goto numeric_range;
+		case 'j':
+			dest = &tm->tm_yday;
+			min = 1;
+			range = 366;
+			adj = 1;
+			goto numeric_range;
+		case 'm':
+			dest = &tm->tm_mon;
+			min = 1;
+			range = 12;
+			adj = 1;
+			goto numeric_range;
+		case 'M':
+			dest = &tm->tm_min;
+			min = 0;
+			range = 60;
+			goto numeric_range;
+		case 'n': case 't':
+			for (; *s && isspace(*s); s++);
+			break;
+#ifdef HAVE_LANGINFO
+		case 'p':
+			ex = nl_langinfo(AM_STR);
+			len = strlen(ex);
+			if (!strncasecmp(s, ex, len)) {
+				tm->tm_hour %= 12;
+				s += len;
+				break;
+			}
+			ex = nl_langinfo(PM_STR);
+			len = strlen(ex);
+			if (!strncasecmp(s, ex, len)) {
+				tm->tm_hour %= 12;
+				tm->tm_hour += 12;
+				s += len;
+				break;
+			}
+			return 0;
+		case 'r':
+			s = strptime(s, nl_langinfo(T_FMT_AMPM), tm);
+			if (!s) return 0;
+			break;
+#endif
+		case 'R':
+			s = strptime(s, "%H:%M", tm);
+			if (!s) return 0;
+			break;
+		case 'S':
+			dest = &tm->tm_sec;
+			min = 0;
+			range = 61;
+			goto numeric_range;
+		case 'T':
+			s = strptime(s, "%H:%M:%S", tm);
+			if (!s) return 0;
+			break;
+		case 'U':
+		case 'W':
+			/* Throw away result, for now. (FIXME?) */
+			dest = &dummy;
+			min = 0;
+			range = 54;
+			goto numeric_range;
+		case 'w':
+			dest = &tm->tm_wday;
+			min = 0;
+			range = 7;
+			goto numeric_range;
+#ifdef HAVE_LANGINFO
+		case 'x':
+			s = strptime(s, nl_langinfo(D_FMT), tm);
+			if (!s) return 0;
+			break;
+		case 'X':
+			s = strptime(s, nl_langinfo(T_FMT), tm);
+			if (!s) return 0;
+			break;
+#endif
+		case 'y':
+			dest = &relyear;
+			w = 2;
+			want_century |= 1;
+			goto numeric_digits;
+		case 'Y':
+			dest = &tm->tm_year;
+			if (w<0) w=4;
+			adj = 1900;
+			want_century = 0;
+			goto numeric_digits;
+		case '%':
+			if (*s++ != '%') return 0;
+			break;
+		default:
+			return 0;
+		numeric_range:
+			if (!isdigit(*s)) return 0;
+			*dest = 0;
+			for (i=1; i<=min+range && isdigit(*s); i*=10)
+				*dest = *dest * 10 + *s++ - '0';
+			if (*dest - min >= range) return 0;
+			*dest -= adj;
+			switch((char *)dest - (char *)tm) {
+			case offsetof(struct tm, tm_yday):
+				;
+			}
+			goto update;
+		numeric_digits:
+			neg = 0;
+			if (*s == '+') s++;
+			else if (*s == '-') neg=1, s++;
+			if (!isdigit(*s)) return 0;
+			for (*dest=i=0; i<w && isdigit(*s); i++)
+				*dest = *dest * 10 + *s++ - '0';
+			if (neg) *dest = -*dest;
+			*dest -= adj;
+			goto update;
+#ifdef HAVE_LANGINFO
+		symbolic_range:
+			for (i=2*range-1; i>=0; i--) {
+				ex = nl_langinfo(min+i);
+				len = strlen(ex);
+				if (strncasecmp(s, ex, len)) continue;
+				s += len;
+				*dest = i % range;
+				break;
+			}
+			if (i<0) return 0;
+			goto update;
+#endif
+		update:
+			//FIXME
+			;
+		}
+	}
+	if (want_century) {
+		tm->tm_year = relyear;
+		if (want_century & 2) tm->tm_year += century * 100 - 1900;
+		else if (tm->tm_year <= 68) tm->tm_year += 100;
+	}
+	return (char *)s;
+}
diff --git a/cpp/src/arrow/util/double_conversion.h b/cpp/src/arrow/vendored/strptime.h
similarity index 67%
copy from cpp/src/arrow/util/double_conversion.h
copy to cpp/src/arrow/vendored/strptime.h
index 2c5f325..764a444 100644
--- a/cpp/src/arrow/util/double_conversion.h
+++ b/cpp/src/arrow/vendored/strptime.h
@@ -17,16 +17,19 @@
 
 #pragma once
 
-#include "arrow/vendored/double-conversion/double-conversion.h"
+#include <time.h>
 
-namespace arrow {
-namespace util {
-namespace double_conversion {
+#include "arrow/util/visibility.h"
 
-using ::double_conversion::DoubleToStringConverter;
-using ::double_conversion::StringBuilder;
-using ::double_conversion::StringToDoubleConverter;
+#ifdef __cplusplus
+extern "C" {
+#endif
 
-}  // namespace double_conversion
-}  // namespace util
-}  // namespace arrow
+// A less featureful implementation of strptime() for platforms lacking
+// a standard implementation (e.g. Windows).
+ARROW_EXPORT char* arrow_strptime(const char* __restrict, const char* __restrict,
+                                  struct tm* __restrict);
+
+#ifdef __cplusplus
+}  // extern "C"
+#endif
diff --git a/cpp/src/gandiva/date_utils.h b/cpp/src/gandiva/date_utils.h
index 344f0d5..0d39a5f 100644
--- a/cpp/src/gandiva/date_utils.h
+++ b/cpp/src/gandiva/date_utils.h
@@ -22,14 +22,7 @@
 #include <unordered_map>
 #include <vector>
 
-#if defined(_MSC_VER)
-#include <ctime>
-#include <iomanip>
-#include <sstream>
-#endif
-
 #include "arrow/util/macros.h"
-#include "arrow/vendored/datetime.h"
 
 #include "gandiva/arrow.h"
 #include "gandiva/visibility.h"
@@ -56,53 +49,4 @@ class GANDIVA_EXPORT DateUtils {
   static std::vector<std::string> GetExactMatches(const std::string& pattern);
 };
 
-namespace internal {
-
-/// \brief Returns seconds since the UNIX epoch
-static inline bool ParseTimestamp(const char* buf, const char* format,
-                                  bool ignoreTimeInDay, int64_t* out) {
-#if defined(_MSC_VER)
-  static std::locale lc_all(setlocale(LC_ALL, NULLPTR));
-  std::istringstream stream(buf);
-  stream.imbue(lc_all);
-
-  // TODO: date::parse fails parsing when the hour value is 0.
-  // eg.1886-12-01 00:00:00
-  arrow_vendored::date::sys_seconds seconds;
-  if (ignoreTimeInDay) {
-    arrow_vendored::date::sys_days days;
-    stream >> arrow_vendored::date::parse(format, days);
-    if (stream.fail()) {
-      return false;
-    }
-    seconds = days;
-  } else {
-    stream >> arrow_vendored::date::parse(format, seconds);
-    if (stream.fail()) {
-      return false;
-    }
-  }
-  auto seconds_in_epoch = seconds.time_since_epoch().count();
-  *out = seconds_in_epoch;
-  return true;
-#else
-  struct tm result;
-  char* ret = strptime(buf, format, &result);
-  if (ret == NULLPTR) {
-    return false;
-  }
-  // ignore the time part
-  arrow_vendored::date::sys_seconds secs =
-      arrow_vendored::date::sys_days(arrow_vendored::date::year(result.tm_year + 1900) /
-                                     (result.tm_mon + 1) / result.tm_mday);
-  if (!ignoreTimeInDay) {
-    secs += (std::chrono::hours(result.tm_hour) + std::chrono::minutes(result.tm_min) +
-             std::chrono::seconds(result.tm_sec));
-  }
-  *out = secs.time_since_epoch().count();
-  return true;
-#endif
-}
-
-}  // namespace internal
 }  // namespace gandiva
diff --git a/cpp/src/gandiva/precompiled/testing.h b/cpp/src/gandiva/precompiled/testing.h
index b1e7753..c41bc54 100644
--- a/cpp/src/gandiva/precompiled/testing.h
+++ b/cpp/src/gandiva/precompiled/testing.h
@@ -18,19 +18,23 @@
 #pragma once
 
 #include <ctime>
+#include <string>
 
 #include <gtest/gtest.h>
 
 #include "arrow/util/logging.h"
+#include "arrow/util/value_parsing.h"
 
 #include "gandiva/date_utils.h"
 #include "gandiva/precompiled/types.h"
 
 namespace gandiva {
 
-static inline gdv_timestamp StringToTimestamp(const char* buf) {
+static inline gdv_timestamp StringToTimestamp(const std::string& s) {
   int64_t out = 0;
-  bool success = internal::ParseTimestamp(buf, "%Y-%m-%d %H:%M:%S", false, &out);
+  bool success = ::arrow::internal::ParseTimestampStrptime(
+      s.c_str(), s.length(), "%Y-%m-%d %H:%M:%S", /*ignore_time_in_day=*/false,
+      /*allow_trailing_chars=*/false, ::arrow::TimeUnit::SECOND, &out);
   DCHECK(success);
   ARROW_UNUSED(success);
   return out * 1000;
diff --git a/cpp/src/gandiva/precompiled/time_test.cc b/cpp/src/gandiva/precompiled/time_test.cc
index bb734d9..dade0a6 100644
--- a/cpp/src/gandiva/precompiled/time_test.cc
+++ b/cpp/src/gandiva/precompiled/time_test.cc
@@ -652,7 +652,7 @@ TEST(TestTime, castVarcharTimestamp) {
   out = castVARCHAR_timestamp_int64(context_ptr, ts, 0L, &out_len);
   EXPECT_EQ(std::string(out, out_len), "");
 
-  ts = StringToTimestamp("2-05-01 0:0:4");
+  ts = StringToTimestamp("2-5-1 00:00:04");
   out = castVARCHAR_timestamp_int64(context_ptr, ts, 24L, &out_len);
   EXPECT_EQ(std::string(out, out_len), "0002-05-01 00:00:04.000");
 }
diff --git a/cpp/src/gandiva/to_date_holder.cc b/cpp/src/gandiva/to_date_holder.cc
index ca93881..fe5e470 100644
--- a/cpp/src/gandiva/to_date_holder.cc
+++ b/cpp/src/gandiva/to_date_holder.cc
@@ -18,6 +18,7 @@
 #include <algorithm>
 #include <string>
 
+#include "arrow/util/value_parsing.h"
 #include "arrow/vendored/datetime.h"
 
 #include "gandiva/date_utils.h"
@@ -82,8 +83,10 @@ int64_t ToDateHolder::operator()(ExecutionContext* context, const std::string& d
   // 1. processes date that do not match the format.
   // 2. does not process time in format +08:00 (or) id.
   int64_t seconds_since_epoch = 0;
-  if (!internal::ParseTimestamp(data.c_str(), pattern_.c_str(), true,
-                                &seconds_since_epoch)) {
+  if (!::arrow::internal::ParseTimestampStrptime(
+          data.c_str(), data.length(), pattern_.c_str(),
+          /*ignore_time_in_day=*/true, /*allow_trailing_chars=*/true,
+          ::arrow::TimeUnit::SECOND, &seconds_since_epoch)) {
     return_error(context, data);
     return 0;
   }