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/05/12 16:54:35 UTC

[arrow-rs] branch master updated: Fix incorrect cast Timestamp with Timezone (#4201)

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 d012bb289 Fix incorrect cast Timestamp with Timezone (#4201)
d012bb289 is described below

commit d012bb289fefddde8c388f7ddb3cc1d31e5a0ca9
Author: Armin Primadi <ap...@gmail.com>
AuthorDate: Fri May 12 23:54:29 2023 +0700

    Fix incorrect cast Timestamp with Timezone (#4201)
    
    * Fix incorrect cast Timestamp with Timezone
    
    * Fix incorrect cast timestamp with timezone
    
    * Support chrono_tz Timezone
    
    * Update arrow-cast/src/cast.rs
    
    Co-authored-by: Raphael Taylor-Davies <17...@users.noreply.github.com>
    
    * Update arrow-cast/src/cast.rs
    
    Co-authored-by: Raphael Taylor-Davies <17...@users.noreply.github.com>
    
    * Move chrono-tz timestamp test to arrow/tests
    
    * Fix clippy and cargo fmt
    
    * Fix clippy
    
    ---------
    
    Co-authored-by: Raphael Taylor-Davies <17...@users.noreply.github.com>
---
 arrow-cast/src/cast.rs    | 151 +++++++++++++++++++++++++++++++++++++++++++++-
 arrow/tests/array_cast.rs |  92 ++++++++++++++++++++++++++++
 2 files changed, 240 insertions(+), 3 deletions(-)

diff --git a/arrow-cast/src/cast.rs b/arrow-cast/src/cast.rs
index 37fede0a6..2b286bfa9 100644
--- a/arrow-cast/src/cast.rs
+++ b/arrow-cast/src/cast.rs
@@ -35,7 +35,7 @@
 //! assert_eq!(7.0, c.value(2));
 //! ```
 
-use chrono::{NaiveTime, TimeZone, Timelike, Utc};
+use chrono::{NaiveTime, Offset, TimeZone, Timelike, Utc};
 use std::cmp::Ordering;
 use std::sync::Arc;
 
@@ -1770,7 +1770,7 @@ pub fn cast_with_options(
             tz.clone(),
         )),
 
-        (Timestamp(from_unit, _), Timestamp(to_unit, to_tz)) => {
+        (Timestamp(from_unit, from_tz), Timestamp(to_unit, to_tz)) => {
             let array = cast_with_options(array, &Int64, cast_options)?;
             let time_array = array.as_primitive::<Int64Type>();
             let from_size = time_unit_multiple(from_unit);
@@ -1792,8 +1792,52 @@ pub fn cast_with_options(
                     }
                 }
             };
+            // Normalize timezone
+            let adjusted = match (from_tz, to_tz) {
+                // Only this case needs to be adjusted because we're casting from
+                // unknown time offset to some time offset, we want the time to be
+                // unchanged.
+                //
+                // i.e. Timestamp('2001-01-01T00:00', None) -> Timestamp('2001-01-01T00:00', '+0700')
+                (None, Some(to_tz)) => {
+                    let to_tz: Tz = to_tz.parse()?;
+                    match to_unit {
+                        TimeUnit::Second => {
+                            adjust_timestamp_to_timezone::<TimestampSecondType>(
+                                converted,
+                                &to_tz,
+                                cast_options,
+                            )?
+                        }
+                        TimeUnit::Millisecond => {
+                            adjust_timestamp_to_timezone::<TimestampMillisecondType>(
+                                converted,
+                                &to_tz,
+                                cast_options,
+                            )?
+                        }
+                        TimeUnit::Microsecond => {
+                            adjust_timestamp_to_timezone::<TimestampMicrosecondType>(
+                                converted,
+                                &to_tz,
+                                cast_options,
+                            )?
+                        }
+                        TimeUnit::Nanosecond => {
+                            adjust_timestamp_to_timezone::<TimestampNanosecondType>(
+                                converted,
+                                &to_tz,
+                                cast_options,
+                            )?
+                        }
+                    }
+                }
+                _ => {
+                    converted
+                }
+            };
             Ok(make_timestamp_array(
-                &converted,
+                &adjusted,
                 to_unit.clone(),
                 to_tz.clone(),
             ))
@@ -3005,6 +3049,30 @@ fn cast_string_to_month_day_nano_interval<Offset: OffsetSizeTrait>(
     Ok(Arc::new(interval_array) as ArrayRef)
 }
 
+fn adjust_timestamp_to_timezone<T: ArrowTimestampType>(
+    array: PrimitiveArray<Int64Type>,
+    to_tz: &Tz,
+    cast_options: &CastOptions,
+) -> Result<PrimitiveArray<Int64Type>, ArrowError> {
+    let adjust = |o| {
+        let local = as_datetime::<T>(o)?;
+        let offset = to_tz.offset_from_local_datetime(&local).single()?;
+        T::make_value(local - offset.fix())
+    };
+    let adjusted = if cast_options.safe {
+        array.unary_opt::<_, Int64Type>(adjust)
+    } else {
+        array.try_unary::<_, Int64Type, _>(|o| {
+            adjust(o).ok_or_else(|| {
+                ArrowError::CastError(
+                    "Cannot cast timezone to different timezone".to_string(),
+                )
+            })
+        })?
+    };
+    Ok(adjusted)
+}
+
 /// Casts Utf8 to Boolean
 fn cast_utf8_to_boolean<OffsetSize>(
     from: &dyn Array,
@@ -5978,6 +6046,83 @@ mod tests {
         assert!(b.is_err());
     }
 
+    // Cast Timestamp(_, None) -> Timestamp(_, Some(timezone))
+    #[test]
+    fn test_cast_timestamp_with_timezone_1() {
+        let string_array: Arc<dyn Array> = Arc::new(StringArray::from(vec![
+            Some("2000-01-01T00:00:00.123456789"),
+            Some("2010-01-01T00:00:00.123456789"),
+            None,
+        ]));
+        let to_type = DataType::Timestamp(TimeUnit::Nanosecond, None);
+        let timestamp_array = cast(&string_array, &to_type).unwrap();
+
+        let to_type = DataType::Timestamp(TimeUnit::Microsecond, Some("+0700".into()));
+        let timestamp_array = cast(&timestamp_array, &to_type).unwrap();
+
+        let string_array = cast(&timestamp_array, &DataType::Utf8).unwrap();
+        let result = string_array.as_string::<i32>();
+        assert_eq!("2000-01-01T00:00:00.123456+07:00", result.value(0));
+        assert_eq!("2010-01-01T00:00:00.123456+07:00", result.value(1));
+        assert!(result.is_null(2));
+    }
+
+    // Cast Timestamp(_, Some(timezone)) -> Timestamp(_, None)
+    #[test]
+    fn test_cast_timestamp_with_timezone_2() {
+        let string_array: Arc<dyn Array> = Arc::new(StringArray::from(vec![
+            Some("2000-01-01T07:00:00.123456789"),
+            Some("2010-01-01T07:00:00.123456789"),
+            None,
+        ]));
+        let to_type = DataType::Timestamp(TimeUnit::Millisecond, Some("+0700".into()));
+        let timestamp_array = cast(&string_array, &to_type).unwrap();
+
+        // Check intermediate representation is correct
+        let string_array = cast(&timestamp_array, &DataType::Utf8).unwrap();
+        let result = string_array.as_string::<i32>();
+        assert_eq!("2000-01-01T07:00:00.123+07:00", result.value(0));
+        assert_eq!("2010-01-01T07:00:00.123+07:00", result.value(1));
+        assert!(result.is_null(2));
+
+        let to_type = DataType::Timestamp(TimeUnit::Nanosecond, None);
+        let timestamp_array = cast(&timestamp_array, &to_type).unwrap();
+
+        let string_array = cast(&timestamp_array, &DataType::Utf8).unwrap();
+        let result = string_array.as_string::<i32>();
+        assert_eq!("2000-01-01T00:00:00.123", result.value(0));
+        assert_eq!("2010-01-01T00:00:00.123", result.value(1));
+        assert!(result.is_null(2));
+    }
+
+    // Cast Timestamp(_, Some(timezone)) -> Timestamp(_, Some(timezone))
+    #[test]
+    fn test_cast_timestamp_with_timezone_3() {
+        let string_array: Arc<dyn Array> = Arc::new(StringArray::from(vec![
+            Some("2000-01-01T07:00:00.123456789"),
+            Some("2010-01-01T07:00:00.123456789"),
+            None,
+        ]));
+        let to_type = DataType::Timestamp(TimeUnit::Microsecond, Some("+0700".into()));
+        let timestamp_array = cast(&string_array, &to_type).unwrap();
+
+        // Check intermediate representation is correct
+        let string_array = cast(&timestamp_array, &DataType::Utf8).unwrap();
+        let result = string_array.as_string::<i32>();
+        assert_eq!("2000-01-01T07:00:00.123456+07:00", result.value(0));
+        assert_eq!("2010-01-01T07:00:00.123456+07:00", result.value(1));
+        assert!(result.is_null(2));
+
+        let to_type = DataType::Timestamp(TimeUnit::Second, Some("-08:00".into()));
+        let timestamp_array = cast(&timestamp_array, &to_type).unwrap();
+
+        let string_array = cast(&timestamp_array, &DataType::Utf8).unwrap();
+        let result = string_array.as_string::<i32>();
+        assert_eq!("1999-12-31T16:00:00-08:00", result.value(0));
+        assert_eq!("2009-12-31T16:00:00-08:00", result.value(1));
+        assert!(result.is_null(2));
+    }
+
     #[test]
     fn test_cast_date64_to_timestamp() {
         let array =
diff --git a/arrow/tests/array_cast.rs b/arrow/tests/array_cast.rs
index bf7e7a326..43dc6dd0e 100644
--- a/arrow/tests/array_cast.rs
+++ b/arrow/tests/array_cast.rs
@@ -18,6 +18,7 @@
 use arrow_array::builder::{
     PrimitiveDictionaryBuilder, StringDictionaryBuilder, UnionBuilder,
 };
+use arrow_array::cast::AsArray;
 use arrow_array::types::{
     ArrowDictionaryKeyType, Decimal128Type, Decimal256Type, Int16Type, Int32Type,
     Int64Type, Int8Type, TimestampMicrosecondType, UInt16Type, UInt32Type, UInt64Type,
@@ -64,6 +65,97 @@ fn test_cast_timestamp_to_string() {
     assert!(c.is_null(2));
 }
 
+// See: https://en.wikipedia.org/wiki/List_of_tz_database_time_zones for list of valid
+// timezones
+
+// Cast Timestamp(_, None) -> Timestamp(_, Some(timezone))
+#[test]
+fn test_cast_timestamp_with_timezone_daylight_1() {
+    let string_array: Arc<dyn Array> = Arc::new(StringArray::from(vec![
+        // This is winter in New York so daylight saving is not in effect
+        // UTC offset is -05:00
+        Some("2000-01-01T00:00:00.123456789"),
+        // This is summer in New York so daylight saving is in effect
+        // UTC offset is -04:00
+        Some("2010-07-01T00:00:00.123456789"),
+        None,
+    ]));
+    let to_type = DataType::Timestamp(TimeUnit::Nanosecond, None);
+    let timestamp_array = cast(&string_array, &to_type).unwrap();
+
+    let to_type =
+        DataType::Timestamp(TimeUnit::Microsecond, Some("America/New_York".into()));
+    let timestamp_array = cast(&timestamp_array, &to_type).unwrap();
+
+    let string_array = cast(&timestamp_array, &DataType::Utf8).unwrap();
+    let result = string_array.as_string::<i32>();
+    assert_eq!("2000-01-01T00:00:00.123456-05:00", result.value(0));
+    assert_eq!("2010-07-01T00:00:00.123456-04:00", result.value(1));
+    assert!(result.is_null(2));
+}
+
+// Cast Timestamp(_, Some(timezone)) -> Timestamp(_, None)
+#[test]
+fn test_cast_timestamp_with_timezone_daylight_2() {
+    let string_array: Arc<dyn Array> = Arc::new(StringArray::from(vec![
+        Some("2000-01-01T07:00:00.123456789"),
+        Some("2010-07-01T07:00:00.123456789"),
+        None,
+    ]));
+    let to_type =
+        DataType::Timestamp(TimeUnit::Millisecond, Some("America/New_York".into()));
+    let timestamp_array = cast(&string_array, &to_type).unwrap();
+
+    // Check intermediate representation is correct
+    let string_array = cast(&timestamp_array, &DataType::Utf8).unwrap();
+    let result = string_array.as_string::<i32>();
+    assert_eq!("2000-01-01T07:00:00.123-05:00", result.value(0));
+    assert_eq!("2010-07-01T07:00:00.123-04:00", result.value(1));
+    assert!(result.is_null(2));
+
+    let to_type = DataType::Timestamp(TimeUnit::Nanosecond, None);
+    let timestamp_array = cast(&timestamp_array, &to_type).unwrap();
+
+    let string_array = cast(&timestamp_array, &DataType::Utf8).unwrap();
+    let result = string_array.as_string::<i32>();
+    assert_eq!("2000-01-01T12:00:00.123", result.value(0));
+    assert_eq!("2010-07-01T11:00:00.123", result.value(1));
+    assert!(result.is_null(2));
+}
+
+// Cast Timestamp(_, Some(timezone)) -> Timestamp(_, Some(timezone))
+#[test]
+fn test_cast_timestamp_with_timezone_daylight_3() {
+    let string_array: Arc<dyn Array> = Arc::new(StringArray::from(vec![
+        // Winter in New York, summer in Sydney
+        // UTC offset is -05:00 (New York) and +11:00 (Sydney)
+        Some("2000-01-01T00:00:00.123456789"),
+        // Summer in New York, winter in Sydney
+        // UTC offset is -04:00 (New York) and +10:00 (Sydney)
+        Some("2010-07-01T00:00:00.123456789"),
+        None,
+    ]));
+    let to_type =
+        DataType::Timestamp(TimeUnit::Microsecond, Some("America/New_York".into()));
+    let timestamp_array = cast(&string_array, &to_type).unwrap();
+
+    // Check intermediate representation is correct
+    let string_array = cast(&timestamp_array, &DataType::Utf8).unwrap();
+    let result = string_array.as_string::<i32>();
+    assert_eq!("2000-01-01T00:00:00.123456-05:00", result.value(0));
+    assert_eq!("2010-07-01T00:00:00.123456-04:00", result.value(1));
+    assert!(result.is_null(2));
+
+    let to_type = DataType::Timestamp(TimeUnit::Second, Some("Australia/Sydney".into()));
+    let timestamp_array = cast(&timestamp_array, &to_type).unwrap();
+
+    let string_array = cast(&timestamp_array, &DataType::Utf8).unwrap();
+    let result = string_array.as_string::<i32>();
+    assert_eq!("2000-01-01T16:00:00+11:00", result.value(0));
+    assert_eq!("2010-07-01T14:00:00+10:00", result.value(1));
+    assert!(result.is_null(2));
+}
+
 #[test]
 #[cfg_attr(miri, ignore)] // running forever
 fn test_can_cast_types() {