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 2023/01/04 12:15:24 UTC

[arrow-rs] branch master updated: parquet record API: timestamp as signed integer (#3437)

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 65ff80e4d parquet record API: timestamp as signed integer (#3437)
65ff80e4d is described below

commit 65ff80e4d80e39d087d61df0ac0a4b2a79833a0c
Author: Your friendly neighborhood geek <sh...@gmail.com>
AuthorDate: Wed Jan 4 17:45:19 2023 +0530

    parquet record API: timestamp as signed integer (#3437)
    
    - Use signed integers to store 'Date', 'TimestampMillis' and 'TimestampMicros' in 'enum Field'
    - remove timezone from string representation of Date
---
 parquet/src/record/api.rs | 55 ++++++++++++++++++++++++-----------------------
 1 file changed, 28 insertions(+), 27 deletions(-)

diff --git a/parquet/src/record/api.rs b/parquet/src/record/api.rs
index b64ff51ee..2d15e126f 100644
--- a/parquet/src/record/api.rs
+++ b/parquet/src/record/api.rs
@@ -123,8 +123,8 @@ pub trait RowAccessor {
     fn get_ulong(&self, i: usize) -> Result<u64>;
     fn get_float(&self, i: usize) -> Result<f32>;
     fn get_double(&self, i: usize) -> Result<f64>;
-    fn get_timestamp_millis(&self, i: usize) -> Result<u64>;
-    fn get_timestamp_micros(&self, i: usize) -> Result<u64>;
+    fn get_timestamp_millis(&self, i: usize) -> Result<i64>;
+    fn get_timestamp_micros(&self, i: usize) -> Result<i64>;
     fn get_decimal(&self, i: usize) -> Result<&Decimal>;
     fn get_string(&self, i: usize) -> Result<&String>;
     fn get_bytes(&self, i: usize) -> Result<&ByteArray>;
@@ -219,9 +219,9 @@ impl RowAccessor for Row {
 
     row_primitive_accessor!(get_double, Double, f64);
 
-    row_primitive_accessor!(get_timestamp_millis, TimestampMillis, u64);
+    row_primitive_accessor!(get_timestamp_millis, TimestampMillis, i64);
 
-    row_primitive_accessor!(get_timestamp_micros, TimestampMicros, u64);
+    row_primitive_accessor!(get_timestamp_micros, TimestampMicros, i64);
 
     row_complex_accessor!(get_decimal, Decimal, Decimal);
 
@@ -295,8 +295,8 @@ pub trait ListAccessor {
     fn get_ulong(&self, i: usize) -> Result<u64>;
     fn get_float(&self, i: usize) -> Result<f32>;
     fn get_double(&self, i: usize) -> Result<f64>;
-    fn get_timestamp_millis(&self, i: usize) -> Result<u64>;
-    fn get_timestamp_micros(&self, i: usize) -> Result<u64>;
+    fn get_timestamp_millis(&self, i: usize) -> Result<i64>;
+    fn get_timestamp_micros(&self, i: usize) -> Result<i64>;
     fn get_decimal(&self, i: usize) -> Result<&Decimal>;
     fn get_string(&self, i: usize) -> Result<&String>;
     fn get_bytes(&self, i: usize) -> Result<&ByteArray>;
@@ -362,9 +362,9 @@ impl ListAccessor for List {
 
     list_primitive_accessor!(get_double, Double, f64);
 
-    list_primitive_accessor!(get_timestamp_millis, TimestampMillis, u64);
+    list_primitive_accessor!(get_timestamp_millis, TimestampMillis, i64);
 
-    list_primitive_accessor!(get_timestamp_micros, TimestampMicros, u64);
+    list_primitive_accessor!(get_timestamp_micros, TimestampMicros, i64);
 
     list_complex_accessor!(get_decimal, Decimal, Decimal);
 
@@ -453,9 +453,9 @@ impl<'a> ListAccessor for MapList<'a> {
 
     map_list_primitive_accessor!(get_double, Double, f64);
 
-    map_list_primitive_accessor!(get_timestamp_millis, TimestampMillis, u64);
+    map_list_primitive_accessor!(get_timestamp_millis, TimestampMillis, i64);
 
-    map_list_primitive_accessor!(get_timestamp_micros, TimestampMicros, u64);
+    map_list_primitive_accessor!(get_timestamp_micros, TimestampMicros, i64);
 
     list_complex_accessor!(get_decimal, Decimal, Decimal);
 
@@ -522,11 +522,11 @@ pub enum Field {
     Bytes(ByteArray),
     /// Date without a time of day, stores the number of days from the
     /// Unix epoch, 1 January 1970.
-    Date(u32),
+    Date(i32),
     /// Milliseconds from the Unix epoch, 1 January 1970.
-    TimestampMillis(u64),
+    TimestampMillis(i64),
     /// Microseconds from the Unix epoch, 1 Janiary 1970.
-    TimestampMicros(u64),
+    TimestampMicros(i64),
 
     // ----------------------------------------------------------------------
     // Complex types
@@ -590,7 +590,7 @@ impl Field {
             ConvertedType::UINT_8 => Field::UByte(value as u8),
             ConvertedType::UINT_16 => Field::UShort(value as u16),
             ConvertedType::UINT_32 => Field::UInt(value as u32),
-            ConvertedType::DATE => Field::Date(value as u32),
+            ConvertedType::DATE => Field::Date(value),
             ConvertedType::DECIMAL => Field::Decimal(Decimal::from_i32(
                 value,
                 descr.type_precision(),
@@ -606,8 +606,8 @@ impl Field {
         match descr.converted_type() {
             ConvertedType::INT_64 | ConvertedType::NONE => Field::Long(value),
             ConvertedType::UINT_64 => Field::ULong(value as u64),
-            ConvertedType::TIMESTAMP_MILLIS => Field::TimestampMillis(value as u64),
-            ConvertedType::TIMESTAMP_MICROS => Field::TimestampMicros(value as u64),
+            ConvertedType::TIMESTAMP_MILLIS => Field::TimestampMillis(value),
+            ConvertedType::TIMESTAMP_MICROS => Field::TimestampMicros(value),
             ConvertedType::DECIMAL => Field::Decimal(Decimal::from_i64(
                 value,
                 descr.type_precision(),
@@ -621,7 +621,7 @@ impl Field {
     /// `Timestamp` value.
     #[inline]
     pub fn convert_int96(_descr: &ColumnDescPtr, value: Int96) -> Self {
-        Field::TimestampMillis(value.to_i64() as u64)
+        Field::TimestampMillis(value.to_i64())
     }
 
     /// Converts Parquet FLOAT type with logical type into `f32` value.
@@ -793,12 +793,12 @@ impl fmt::Display for Field {
 /// Input `value` is a number of days since the epoch in UTC.
 /// Date is displayed in local timezone.
 #[inline]
-fn convert_date_to_string(value: u32) -> String {
+fn convert_date_to_string(value: i32) -> String {
     static NUM_SECONDS_IN_DAY: i64 = 60 * 60 * 24;
     let dt = Utc
         .timestamp_opt(value as i64 * NUM_SECONDS_IN_DAY, 0)
         .unwrap();
-    format!("{}", dt.format("%Y-%m-%d %:z"))
+    format!("{}", dt.format("%Y-%m-%d"))
 }
 
 /// Helper method to convert Parquet timestamp into a string.
@@ -814,16 +814,16 @@ fn convert_timestamp_secs_to_string(value: i64) -> String {
 /// Input `value` is a number of milliseconds since the epoch in UTC.
 /// Datetime is displayed in local timezone.
 #[inline]
-fn convert_timestamp_millis_to_string(value: u64) -> String {
-    convert_timestamp_secs_to_string(value as i64 / 1000)
+fn convert_timestamp_millis_to_string(value: i64) -> String {
+    convert_timestamp_secs_to_string(value / 1000)
 }
 
 /// Helper method to convert Parquet timestamp into a string.
 /// Input `value` is a number of microseconds since the epoch in UTC.
 /// Datetime is displayed in local timezone.
 #[inline]
-fn convert_timestamp_micros_to_string(value: u64) -> String {
-    convert_timestamp_secs_to_string(value as i64 / 1000000)
+fn convert_timestamp_micros_to_string(value: i64) -> String {
+    convert_timestamp_secs_to_string(value / 1000000)
 }
 
 /// Helper method to convert Parquet decimal into a string.
@@ -1083,11 +1083,12 @@ mod tests {
                 .and_hms_opt(0, 0, 0)
                 .unwrap();
             let dt = Utc.from_utc_datetime(&datetime);
-            let res = convert_date_to_string((dt.timestamp() / 60 / 60 / 24) as u32);
-            let exp = format!("{}", dt.format("%Y-%m-%d %:z"));
+            let res = convert_date_to_string((dt.timestamp() / 60 / 60 / 24) as i32);
+            let exp = format!("{}", dt.format("%Y-%m-%d"));
             assert_eq!(res, exp);
         }
 
+        check_date_conversion(1969, 12, 31);
         check_date_conversion(2010, 1, 2);
         check_date_conversion(2014, 5, 1);
         check_date_conversion(2016, 2, 29);
@@ -1103,7 +1104,7 @@ mod tests {
                 .and_hms_opt(h, mi, s)
                 .unwrap();
             let dt = Utc.from_utc_datetime(&datetime);
-            let res = convert_timestamp_millis_to_string(dt.timestamp_millis() as u64);
+            let res = convert_timestamp_millis_to_string(dt.timestamp_millis());
             let exp = format!("{}", dt.format("%Y-%m-%d %H:%M:%S %:z"));
             assert_eq!(res, exp);
         }
@@ -1124,7 +1125,7 @@ mod tests {
                 .and_hms_opt(h, mi, s)
                 .unwrap();
             let dt = Utc.from_utc_datetime(&datetime);
-            let res = convert_timestamp_micros_to_string(dt.timestamp_micros() as u64);
+            let res = convert_timestamp_micros_to_string(dt.timestamp_micros());
             let exp = format!("{}", dt.format("%Y-%m-%d %H:%M:%S %:z"));
             assert_eq!(res, exp);
         }