You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@arrow.apache.org by tu...@apache.org on 2022/11/07 20:44:07 UTC

[arrow-rs] branch master updated: Move reader_parser to arrow-cast (#3022) (#3043)

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

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


The following commit(s) were added to refs/heads/master by this push:
     new b7bc79bf2 Move reader_parser to arrow-cast (#3022) (#3043)
b7bc79bf2 is described below

commit b7bc79bf2cbf593fafa0dc552cc2bb16b084d132
Author: Raphael Taylor-Davies <17...@users.noreply.github.com>
AuthorDate: Tue Nov 8 09:44:02 2022 +1300

    Move reader_parser to arrow-cast (#3022) (#3043)
    
    * Move reader_parser to arrow-cast (#3022)
    
    * Format
---
 arrow-cast/src/parse.rs         | 122 ++++++++++++++++++++++++++++++++++
 arrow/src/csv/reader.rs         |   2 +-
 arrow/src/json/reader.rs        |   2 +-
 arrow/src/util/mod.rs           |   1 -
 arrow/src/util/reader_parser.rs | 142 ----------------------------------------
 5 files changed, 124 insertions(+), 145 deletions(-)

diff --git a/arrow-cast/src/parse.rs b/arrow-cast/src/parse.rs
index 8a9d34b4c..126beb902 100644
--- a/arrow-cast/src/parse.rs
+++ b/arrow-cast/src/parse.rs
@@ -15,6 +15,8 @@
 // specific language governing permissions and limitations
 // under the License.
 
+use arrow_array::types::*;
+use arrow_array::ArrowPrimitiveType;
 use arrow_schema::ArrowError;
 use chrono::prelude::*;
 
@@ -130,6 +132,126 @@ pub fn string_to_timestamp_nanos(s: &str) -> Result<i64, ArrowError> {
     )))
 }
 
+/// Specialized parsing implementations
+/// used by csv and json reader
+pub trait Parser: ArrowPrimitiveType {
+    fn parse(string: &str) -> Option<Self::Native>;
+
+    fn parse_formatted(string: &str, _format: &str) -> Option<Self::Native> {
+        Self::parse(string)
+    }
+}
+
+impl Parser for Float32Type {
+    fn parse(string: &str) -> Option<f32> {
+        lexical_core::parse(string.as_bytes()).ok()
+    }
+}
+
+impl Parser for Float64Type {
+    fn parse(string: &str) -> Option<f64> {
+        lexical_core::parse(string.as_bytes()).ok()
+    }
+}
+
+macro_rules! parser_primitive {
+    ($t:ty) => {
+        impl Parser for $t {
+            fn parse(string: &str) -> Option<Self::Native> {
+                string.parse::<Self::Native>().ok()
+            }
+        }
+    };
+}
+parser_primitive!(UInt64Type);
+parser_primitive!(UInt32Type);
+parser_primitive!(UInt16Type);
+parser_primitive!(UInt8Type);
+parser_primitive!(Int64Type);
+parser_primitive!(Int32Type);
+parser_primitive!(Int16Type);
+parser_primitive!(Int8Type);
+
+impl Parser for TimestampNanosecondType {
+    fn parse(string: &str) -> Option<i64> {
+        string_to_timestamp_nanos(string).ok()
+    }
+}
+
+impl Parser for TimestampMicrosecondType {
+    fn parse(string: &str) -> Option<i64> {
+        let nanos = string_to_timestamp_nanos(string).ok();
+        nanos.map(|x| x / 1000)
+    }
+}
+
+impl Parser for TimestampMillisecondType {
+    fn parse(string: &str) -> Option<i64> {
+        let nanos = string_to_timestamp_nanos(string).ok();
+        nanos.map(|x| x / 1_000_000)
+    }
+}
+
+impl Parser for TimestampSecondType {
+    fn parse(string: &str) -> Option<i64> {
+        let nanos = string_to_timestamp_nanos(string).ok();
+        nanos.map(|x| x / 1_000_000_000)
+    }
+}
+
+parser_primitive!(Time64NanosecondType);
+parser_primitive!(Time64MicrosecondType);
+parser_primitive!(Time32MillisecondType);
+parser_primitive!(Time32SecondType);
+
+/// Number of days between 0001-01-01 and 1970-01-01
+const EPOCH_DAYS_FROM_CE: i32 = 719_163;
+
+impl Parser for Date32Type {
+    fn parse(string: &str) -> Option<i32> {
+        let date = string.parse::<chrono::NaiveDate>().ok()?;
+        Some(date.num_days_from_ce() - EPOCH_DAYS_FROM_CE)
+    }
+
+    fn parse_formatted(string: &str, format: &str) -> Option<i32> {
+        let date = chrono::NaiveDate::parse_from_str(string, format).ok()?;
+        Some(date.num_days_from_ce() - EPOCH_DAYS_FROM_CE)
+    }
+}
+
+impl Parser for Date64Type {
+    fn parse(string: &str) -> Option<i64> {
+        let date_time = string.parse::<NaiveDateTime>().ok()?;
+        Some(date_time.timestamp_millis())
+    }
+
+    fn parse_formatted(string: &str, format: &str) -> Option<i64> {
+        use chrono::format::Fixed;
+        use chrono::format::StrftimeItems;
+        let fmt = StrftimeItems::new(format);
+        let has_zone = fmt.into_iter().any(|item| match item {
+            chrono::format::Item::Fixed(fixed_item) => matches!(
+                fixed_item,
+                Fixed::RFC2822
+                    | Fixed::RFC3339
+                    | Fixed::TimezoneName
+                    | Fixed::TimezoneOffsetColon
+                    | Fixed::TimezoneOffsetColonZ
+                    | Fixed::TimezoneOffset
+                    | Fixed::TimezoneOffsetZ
+            ),
+            _ => false,
+        });
+        if has_zone {
+            let date_time = chrono::DateTime::parse_from_str(string, format).ok()?;
+            Some(date_time.timestamp_millis())
+        } else {
+            let date_time = NaiveDateTime::parse_from_str(string, format).ok()?;
+            Some(date_time.timestamp_millis())
+        }
+    }
+}
+
 #[cfg(test)]
 mod tests {
     use super::*;
diff --git a/arrow/src/csv/reader.rs b/arrow/src/csv/reader.rs
index ff6df5514..404f37e93 100644
--- a/arrow/src/csv/reader.rs
+++ b/arrow/src/csv/reader.rs
@@ -56,7 +56,7 @@ use crate::array::{
 use crate::datatypes::*;
 use crate::error::{ArrowError, Result};
 use crate::record_batch::{RecordBatch, RecordBatchOptions};
-use crate::util::reader_parser::Parser;
+use arrow_cast::parse::Parser;
 
 use crate::csv::map_csv_error;
 use csv_crate::{ByteRecord, StringRecord};
diff --git a/arrow/src/json/reader.rs b/arrow/src/json/reader.rs
index a7382128e..78c51559a 100644
--- a/arrow/src/json/reader.rs
+++ b/arrow/src/json/reader.rs
@@ -60,8 +60,8 @@ use crate::datatypes::*;
 use crate::error::{ArrowError, Result};
 use crate::record_batch::{RecordBatch, RecordBatchOptions};
 use crate::util::bit_util;
-use crate::util::reader_parser::Parser;
 use crate::{array::*, buffer::Buffer};
+use arrow_cast::parse::Parser;
 
 #[derive(Debug, Clone)]
 enum InferredType {
diff --git a/arrow/src/util/mod.rs b/arrow/src/util/mod.rs
index 9a0ca852a..4369ebe7d 100644
--- a/arrow/src/util/mod.rs
+++ b/arrow/src/util/mod.rs
@@ -31,4 +31,3 @@ pub mod string_writer;
 pub mod test_util;
 
 pub use arrow_cast::display;
-pub(crate) mod reader_parser;
diff --git a/arrow/src/util/reader_parser.rs b/arrow/src/util/reader_parser.rs
deleted file mode 100644
index efee62905..000000000
--- a/arrow/src/util/reader_parser.rs
+++ /dev/null
@@ -1,142 +0,0 @@
-// Licensed to the Apache Software Foundation (ASF) under one
-// or more contributor license agreements.  See the NOTICE file
-// distributed with this work for additional information
-// regarding copyright ownership.  The ASF licenses this file
-// to you under the Apache License, Version 2.0 (the
-// "License"); you may not use this file except in compliance
-// with the License.  You may obtain a copy of the License at
-//
-//   http://www.apache.org/licenses/LICENSE-2.0
-//
-// Unless required by applicable law or agreed to in writing,
-// software distributed under the License is distributed on an
-// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
-// KIND, either express or implied.  See the License for the
-// specific language governing permissions and limitations
-// under the License.
-
-use arrow_array::types::*;
-use arrow_array::*;
-use arrow_cast::parse::string_to_timestamp_nanos;
-
-/// Specialized parsing implementations
-/// used by csv and json reader
-pub(crate) trait Parser: ArrowPrimitiveType {
-    fn parse(string: &str) -> Option<Self::Native>;
-
-    fn parse_formatted(string: &str, _format: &str) -> Option<Self::Native> {
-        Self::parse(string)
-    }
-}
-
-impl Parser for Float32Type {
-    fn parse(string: &str) -> Option<f32> {
-        lexical_core::parse(string.as_bytes()).ok()
-    }
-}
-
-impl Parser for Float64Type {
-    fn parse(string: &str) -> Option<f64> {
-        lexical_core::parse(string.as_bytes()).ok()
-    }
-}
-
-macro_rules! parser_primitive {
-    ($t:ty) => {
-        impl Parser for $t {
-            fn parse(string: &str) -> Option<Self::Native> {
-                string.parse::<Self::Native>().ok()
-            }
-        }
-    };
-}
-parser_primitive!(UInt64Type);
-parser_primitive!(UInt32Type);
-parser_primitive!(UInt16Type);
-parser_primitive!(UInt8Type);
-parser_primitive!(Int64Type);
-parser_primitive!(Int32Type);
-parser_primitive!(Int16Type);
-parser_primitive!(Int8Type);
-
-impl Parser for TimestampNanosecondType {
-    fn parse(string: &str) -> Option<i64> {
-        string_to_timestamp_nanos(string).ok()
-    }
-}
-
-impl Parser for TimestampMicrosecondType {
-    fn parse(string: &str) -> Option<i64> {
-        let nanos = string_to_timestamp_nanos(string).ok();
-        nanos.map(|x| x / 1000)
-    }
-}
-
-impl Parser for TimestampMillisecondType {
-    fn parse(string: &str) -> Option<i64> {
-        let nanos = string_to_timestamp_nanos(string).ok();
-        nanos.map(|x| x / 1_000_000)
-    }
-}
-
-impl Parser for TimestampSecondType {
-    fn parse(string: &str) -> Option<i64> {
-        let nanos = string_to_timestamp_nanos(string).ok();
-        nanos.map(|x| x / 1_000_000_000)
-    }
-}
-
-parser_primitive!(Time64NanosecondType);
-parser_primitive!(Time64MicrosecondType);
-parser_primitive!(Time32MillisecondType);
-parser_primitive!(Time32SecondType);
-
-/// Number of days between 0001-01-01 and 1970-01-01
-const EPOCH_DAYS_FROM_CE: i32 = 719_163;
-
-impl Parser for Date32Type {
-    fn parse(string: &str) -> Option<i32> {
-        use chrono::Datelike;
-        let date = string.parse::<chrono::NaiveDate>().ok()?;
-        Some(date.num_days_from_ce() - EPOCH_DAYS_FROM_CE)
-    }
-
-    fn parse_formatted(string: &str, format: &str) -> Option<i32> {
-        use chrono::Datelike;
-        let date = chrono::NaiveDate::parse_from_str(string, format).ok()?;
-        Some(date.num_days_from_ce() - EPOCH_DAYS_FROM_CE)
-    }
-}
-
-impl Parser for Date64Type {
-    fn parse(string: &str) -> Option<i64> {
-        let date_time = string.parse::<chrono::NaiveDateTime>().ok()?;
-        Some(date_time.timestamp_millis())
-    }
-
-    fn parse_formatted(string: &str, format: &str) -> Option<i64> {
-        use chrono::format::Fixed;
-        use chrono::format::StrftimeItems;
-        let fmt = StrftimeItems::new(format);
-        let has_zone = fmt.into_iter().any(|item| match item {
-            chrono::format::Item::Fixed(fixed_item) => matches!(
-                fixed_item,
-                Fixed::RFC2822
-                    | Fixed::RFC3339
-                    | Fixed::TimezoneName
-                    | Fixed::TimezoneOffsetColon
-                    | Fixed::TimezoneOffsetColonZ
-                    | Fixed::TimezoneOffset
-                    | Fixed::TimezoneOffsetZ
-            ),
-            _ => false,
-        });
-        if has_zone {
-            let date_time = chrono::DateTime::parse_from_str(string, format).ok()?;
-            Some(date_time.timestamp_millis())
-        } else {
-            let date_time = chrono::NaiveDateTime::parse_from_str(string, format).ok()?;
-            Some(date_time.timestamp_millis())
-        }
-    }
-}