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/06/02 13:32:26 UTC

[arrow-rs] branch master updated: feat: cast between `Intervals` (#4182)

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 863d59981 feat: cast between `Intervals` (#4182)
863d59981 is described below

commit 863d59981bb015570e0ed3a9618f3807750942ab
Author: Igor Izvekov <iz...@gmail.com>
AuthorDate: Fri Jun 2 16:32:20 2023 +0300

    feat: cast between `Intervals` (#4182)
    
    * feat: cast between Intervals
    
    * feat: cast: IntervalUnit::YearMonth -> IntervalUnit::MonthDayNano and IntervalUnit::DayTime -> IntervalUnit::MonthDayNano
    
    * refactoring
---
 arrow-cast/src/cast.rs | 117 ++++++++++++++++++++++++++++++++++++++++++++++---
 1 file changed, 112 insertions(+), 5 deletions(-)

diff --git a/arrow-cast/src/cast.rs b/arrow-cast/src/cast.rs
index 9652047c7..839326d08 100644
--- a/arrow-cast/src/cast.rs
+++ b/arrow-cast/src/cast.rs
@@ -254,6 +254,8 @@ pub fn can_cast_types(from_type: &DataType, to_type: &DataType) -> bool {
         },
         (Duration(_), Interval(MonthDayNano)) => true,
         (Interval(MonthDayNano), Duration(_)) => true,
+        (Interval(IntervalUnit::YearMonth), Interval(IntervalUnit::MonthDayNano)) => true,
+        (Interval(IntervalUnit::DayTime), Interval(IntervalUnit::MonthDayNano)) => true,
         (_, _) => false,
     }
 }
@@ -409,6 +411,33 @@ where
     }
 }
 
+/// Cast the array from interval year month to month day nano
+fn cast_interval_year_month_to_interval_month_day_nano(
+    array: &dyn Array,
+    _cast_options: &CastOptions,
+) -> Result<ArrayRef, ArrowError> {
+    let array = array.as_primitive::<IntervalYearMonthType>();
+
+    Ok(Arc::new(array.unary::<_, IntervalMonthDayNanoType>(|v| {
+        let months = IntervalYearMonthType::to_months(v);
+        IntervalMonthDayNanoType::make_value(months, 0, 0)
+    })))
+}
+
+/// Cast the array from interval day time to month day nano
+fn cast_interval_day_time_to_interval_month_day_nano(
+    array: &dyn Array,
+    _cast_options: &CastOptions,
+) -> Result<ArrayRef, ArrowError> {
+    let array = array.as_primitive::<IntervalDayTimeType>();
+    let mul = 1_000_000;
+
+    Ok(Arc::new(array.unary::<_, IntervalMonthDayNanoType>(|v| {
+        let (days, ms) = IntervalDayTimeType::to_parts(v);
+        IntervalMonthDayNanoType::make_value(0, days, ms as i64 * mul)
+    })))
+}
+
 /// Cast the array from interval to duration
 fn cast_interval_to_duration<D: ArrowTemporalType<Native = i64>>(
     array: &dyn Array,
@@ -2137,18 +2166,24 @@ pub fn cast_with_options(
         (Duration(TimeUnit::Nanosecond), Interval(IntervalUnit::MonthDayNano)) => {
             cast_duration_to_interval::<DurationNanosecondType>(array, cast_options)
         }
-        (DataType::Interval(IntervalUnit::MonthDayNano), DataType::Duration(TimeUnit::Second)) => {
+        (Interval(IntervalUnit::MonthDayNano), DataType::Duration(TimeUnit::Second)) => {
             cast_interval_to_duration::<DurationSecondType>(array, cast_options)
         }
-        (DataType::Interval(IntervalUnit::MonthDayNano), DataType::Duration(TimeUnit::Millisecond)) => {
+        (Interval(IntervalUnit::MonthDayNano), DataType::Duration(TimeUnit::Millisecond)) => {
             cast_interval_to_duration::<DurationMillisecondType>(array, cast_options)
         }
-        (DataType::Interval(IntervalUnit::MonthDayNano), DataType::Duration(TimeUnit::Microsecond)) => {
+        (Interval(IntervalUnit::MonthDayNano), DataType::Duration(TimeUnit::Microsecond)) => {
             cast_interval_to_duration::<DurationMicrosecondType>(array, cast_options)
         }
-        (DataType::Interval(IntervalUnit::MonthDayNano), DataType::Duration(TimeUnit::Nanosecond)) => {
+        (Interval(IntervalUnit::MonthDayNano), DataType::Duration(TimeUnit::Nanosecond)) => {
             cast_interval_to_duration::<DurationNanosecondType>(array, cast_options)
         }
+        (Interval(IntervalUnit::YearMonth), Interval(IntervalUnit::MonthDayNano)) => {
+            cast_interval_year_month_to_interval_month_day_nano(array, cast_options)
+        }
+        (Interval(IntervalUnit::DayTime), Interval(IntervalUnit::MonthDayNano)) => {
+            cast_interval_day_time_to_interval_month_day_nano(array, cast_options)
+        }
         (Interval(IntervalUnit::YearMonth), Int64) => {
             cast_numeric_arrays::<IntervalYearMonthType, Int64Type>(array, cast_options)
         }
@@ -9030,7 +9065,7 @@ mod tests {
         assert_eq!(casted_array.value(0), 9223372036854775807);
     }
 
-    // helper function to test casting from interval to duration
+    /// helper function to test casting from interval to duration
     fn cast_from_interval_to_duration<T: ArrowTemporalType>(
         array: Vec<i128>,
         cast_options: &CastOptions,
@@ -9173,6 +9208,78 @@ mod tests {
         assert!(casted_array.is_err());
     }
 
+    /// helper function to test casting from interval year month to interval month day nano
+    fn cast_from_interval_year_month_to_interval_month_day_nano(
+        array: Vec<i32>,
+        cast_options: &CastOptions,
+    ) -> Result<PrimitiveArray<IntervalMonthDayNanoType>, ArrowError> {
+        let array = PrimitiveArray::<IntervalYearMonthType>::from(array);
+        let array = Arc::new(array) as ArrayRef;
+        let casted_array = cast_with_options(
+            &array,
+            &DataType::Interval(IntervalUnit::MonthDayNano),
+            cast_options,
+        )?;
+        casted_array
+            .as_any()
+            .downcast_ref::<IntervalMonthDayNanoArray>()
+            .ok_or_else(|| {
+                ArrowError::ComputeError(
+                    "Failed to downcast to IntervalMonthDayNanoArray".to_string(),
+                )
+            })
+            .cloned()
+    }
+
+    #[test]
+    fn test_cast_from_interval_year_month_to_interval_month_day_nano() {
+        // from interval year month to interval month day nano
+        let array = vec![1234567];
+        let casted_array = cast_from_interval_year_month_to_interval_month_day_nano(
+            array,
+            &CastOptions::default(),
+        )
+        .unwrap();
+        assert_eq!(
+            casted_array.data_type(),
+            &DataType::Interval(IntervalUnit::MonthDayNano)
+        );
+        assert_eq!(casted_array.value(0), 97812474910747780469848774134464512);
+    }
+
+    /// helper function to test casting from interval day time to interval month day nano
+    fn cast_from_interval_day_time_to_interval_month_day_nano(
+        array: Vec<i64>,
+        cast_options: &CastOptions,
+    ) -> Result<PrimitiveArray<IntervalMonthDayNanoType>, ArrowError> {
+        let array = PrimitiveArray::<IntervalDayTimeType>::from(array);
+        let array = Arc::new(array) as ArrayRef;
+        let casted_array = cast_with_options(
+            &array,
+            &DataType::Interval(IntervalUnit::MonthDayNano),
+            cast_options,
+        )?;
+        Ok(casted_array
+            .as_primitive::<IntervalMonthDayNanoType>()
+            .clone())
+    }
+
+    #[test]
+    fn test_cast_from_interval_day_time_to_interval_month_day_nano() {
+        // from interval day time to interval month day nano
+        let array = vec![123];
+        let casted_array = cast_from_interval_day_time_to_interval_month_day_nano(
+            array,
+            &CastOptions::default(),
+        )
+        .unwrap();
+        assert_eq!(
+            casted_array.data_type(),
+            &DataType::Interval(IntervalUnit::MonthDayNano)
+        );
+        assert_eq!(casted_array.value(0), 123000000);
+    }
+
     #[test]
     fn test_cast_below_unixtimestamp() {
         let valid = StringArray::from(vec![