You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@arrow.apache.org by "Weijun-H (via GitHub)" <gi...@apache.org> on 2023/04/09 09:43:52 UTC

[GitHub] [arrow-rs] Weijun-H opened a new pull request, #4038: feat: Support Timestamp +/- Interval types

Weijun-H opened a new pull request, #4038:
URL: https://github.com/apache/arrow-rs/pull/4038

   # Which issue does this PR close?
   
   <!--
   We generally require a GitHub issue to be filed for all bug fixes and enhancements and this helps us generate change logs for our releases. You can link an issue to this PR using the GitHub syntax. For example `Closes #123` indicates that this PR will close issue #123.
   -->
   
   Closes #3963
   
   # Rationale for this change
    
   <!--
   Why are you proposing this change? If this is already explained clearly in the issue then this section is not needed.
   Explaining clearly why changes are proposed helps reviewers understand your changes and offer better suggestions for fixes.
   -->
   
   # What changes are included in this PR?
   
   <!--
   There is no need to duplicate the description in the issue here but it is sometimes worth providing a summary of the individual changes in this PR.
   -->
   
   # Are there any user-facing changes?
   
   
   <!--
   If there are user-facing changes then we may require documentation to be updated before approving the PR.
   -->
   
   <!---
   If there are any breaking changes to public APIs, please add the `breaking change` label.
   -->
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-rs] alamb commented on a diff in pull request #4038: feat: Support Timestamp +/- Interval types

Posted by "alamb (via GitHub)" <gi...@apache.org>.
alamb commented on code in PR #4038:
URL: https://github.com/apache/arrow-rs/pull/4038#discussion_r1167578173


##########
arrow-arith/src/arithmetic.rs:
##########
@@ -739,6 +739,105 @@ pub fn add_dyn(left: &dyn Array, right: &dyn Array) -> Result<ArrayRef, ArrowErr
                 ))),
             }
         }
+        DataType::Timestamp(TimeUnit::Second, _) => {

Review Comment:
   Follow on ticket: https://github.com/apache/arrow-rs/issues/4094



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-rs] metesynnada commented on pull request #4038: feat: Support Timestamp +/- Interval types

Posted by "metesynnada (via GitHub)" <gi...@apache.org>.
metesynnada commented on PR #4038:
URL: https://github.com/apache/arrow-rs/pull/4038#issuecomment-1503611881

   LGTM!


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-rs] alamb commented on pull request #4038: feat: Support Timestamp +/- Interval types

Posted by "alamb (via GitHub)" <gi...@apache.org>.
alamb commented on PR #4038:
URL: https://github.com/apache/arrow-rs/pull/4038#issuecomment-1509886165

   Here is a PR to bump the chrono version:  https://github.com/apache/arrow-rs/pull/4093


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-rs] alamb commented on pull request #4038: feat: Support Timestamp +/- Interval types

Posted by "alamb (via GitHub)" <gi...@apache.org>.
alamb commented on PR #4038:
URL: https://github.com/apache/arrow-rs/pull/4038#issuecomment-1509186806

   Also, when testing locally I got this error I think because this PR uses APIs that were introduced in chrono `0.4.24`
   
   ```
      Compiling arrow-array v37.0.0 (/Users/alamb/Software/arrow-rs2/arrow-array)
   error[E0599]: no function or associated item named `from_timestamp_micros` found for struct `NaiveDateTime` in the current scope
      --> arrow-array/src/types.rs:520:36
       |
   520 |         let prior = NaiveDateTime::from_timestamp_micros(timestamp).ok_or_else(|| {
       |                                    ^^^^^^^^^^^^^^^^^^^^^
       |                                    |
       |                                    function or associated item not found in `NaiveDateTime`
       |                                    help: there is an associated function with a similar name: `from_timestamp`
   
   
   ```
   
   I needed to make this change (which I think we should do in the PR):
   
   
   ```diff
   diff --git a/arrow-array/Cargo.toml b/arrow-array/Cargo.toml
   index 1b417bb0e..634a0aa64 100644
   --- a/arrow-array/Cargo.toml
   +++ b/arrow-array/Cargo.toml
   @@ -44,7 +44,7 @@ ahash = { version = "0.8", default-features = false, features = ["runtime-rng"]
    arrow-buffer = { workspace = true }
    arrow-schema = { workspace = true }
    arrow-data = { workspace = true }
   -chrono = { version = "0.4.23", default-features = false, features = ["clock"] }
   +chrono = { version = "0.4.24", default-features = false, features = ["clock"] }
    chrono-tz = { version = "0.8", optional = true }
    num = { version = "0.4", default-features = false, features = ["std"] }
    half = { version = "2.1", default-features = false, features = ["num-traits"] }
   ```
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-rs] alamb commented on a diff in pull request #4038: feat: Support Timestamp +/- Interval types

Posted by "alamb (via GitHub)" <gi...@apache.org>.
alamb commented on code in PR #4038:
URL: https://github.com/apache/arrow-rs/pull/4038#discussion_r1167214146


##########
arrow-arith/src/arithmetic.rs:
##########
@@ -3665,4 +3863,588 @@ mod tests {
             "1234567890.0000000000000000000000000000"
         );
     }
+
+    #[test]
+    fn test_timestamp_second_add_interval() {
+        // timestamp second + interval year month
+        let a = TimestampSecondArray::from(vec![1, 2, 3, 4, 5]);
+        let b = IntervalYearMonthArray::from(vec![
+            Some(IntervalYearMonthType::make_value(0, 1)),
+            Some(IntervalYearMonthType::make_value(0, 1)),
+            Some(IntervalYearMonthType::make_value(0, 1)),

Review Comment:
   It would be good if at least one of these intervals had a year field set to makes sure it survive the roundtrip
   
   Like
   ```
               Some(IntervalYearMonthType::make_value(1, 2)),
   ```



##########
arrow-arith/src/arithmetic.rs:
##########
@@ -739,6 +739,105 @@ pub fn add_dyn(left: &dyn Array, right: &dyn Array) -> Result<ArrayRef, ArrowErr
                 ))),
             }
         }
+        DataType::Timestamp(TimeUnit::Second, _) => {

Review Comment:
   One thing I noticed while reviewing this code is that it will only support `Timestamp + Interval` not `Interval + Timestamp` - given this seems to be the case for `Date + Interval` as well, I don't think we have to fix it as part of this PR, but I will file a follow on ticket to try and improve the situation.



##########
arrow-arith/src/arithmetic.rs:
##########
@@ -3665,4 +3863,588 @@ mod tests {
             "1234567890.0000000000000000000000000000"
         );
     }
+
+    #[test]
+    fn test_timestamp_second_add_interval() {
+        // timestamp second + interval year month
+        let a = TimestampSecondArray::from(vec![1, 2, 3, 4, 5]);
+        let b = IntervalYearMonthArray::from(vec![
+            Some(IntervalYearMonthType::make_value(0, 1)),
+            Some(IntervalYearMonthType::make_value(0, 1)),
+            Some(IntervalYearMonthType::make_value(0, 1)),
+            Some(IntervalYearMonthType::make_value(0, 1)),
+            Some(IntervalYearMonthType::make_value(0, 1)),
+        ]);
+
+        let result = add_dyn(&a, &b).unwrap();
+        let result = result
+            .as_any()
+            .downcast_ref::<TimestampSecondArray>()
+            .unwrap();

Review Comment:
   FYI you can write this more concisely like 
   ```suggestion
           let result = result.as_primitive::<TimestampSecondType>();
   ```



##########
arrow-arith/src/arithmetic.rs:
##########
@@ -739,6 +739,105 @@ pub fn add_dyn(left: &dyn Array, right: &dyn Array) -> Result<ArrayRef, ArrowErr
                 ))),
             }
         }
+        DataType::Timestamp(TimeUnit::Second, _) => {
+            let l = left.as_primitive::<TimestampSecondType>();
+            match right.data_type() {
+                DataType::Interval(IntervalUnit::YearMonth) => {
+                    let r = right.as_primitive::<IntervalYearMonthType>();
+                    let res = math_checked_op(l, r, TimestampSecondType::add_year_months)?;
+                    Ok(Arc::new(res))

Review Comment:
   @Weijun-H  do you plan to make the change suggested by @metesynnada ? it seems a good one .



##########
arrow-arith/src/arithmetic.rs:
##########
@@ -3665,4 +3863,588 @@ mod tests {
             "1234567890.0000000000000000000000000000"
         );
     }
+
+    #[test]
+    fn test_timestamp_second_add_interval() {
+        // timestamp second + interval year month
+        let a = TimestampSecondArray::from(vec![1, 2, 3, 4, 5]);
+        let b = IntervalYearMonthArray::from(vec![
+            Some(IntervalYearMonthType::make_value(0, 1)),
+            Some(IntervalYearMonthType::make_value(0, 1)),
+            Some(IntervalYearMonthType::make_value(0, 1)),

Review Comment:
   Likewise in all the tests below, it would be nice if more than one field was non zero so that we can be sure the field values are passed through to the corresponding operation



##########
arrow-array/src/types.rs:
##########
@@ -350,6 +350,650 @@ impl ArrowTimestampType for TimestampNanosecondType {
     }
 }
 
+impl TimestampSecondType {
+    /// Adds the given IntervalYearMonthType to an arrow TimestampSecondType
+    ///
+    /// # Arguments
+    ///
+    /// * `timestamp` - The date on which to perform the operation
+    /// * `delta` - The interval to add
+    pub fn add_year_months(
+        timestamp: <TimestampSecondType as ArrowPrimitiveType>::Native,
+        delta: <IntervalYearMonthType as ArrowPrimitiveType>::Native,

Review Comment:
   I tried to come up with a better formulation of this signature to avoid the boiler plate repetition (by finding some way to make the timestamp types have a templated `to_naive` function but I couldn't come up with anything better.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-rs] alamb commented on a diff in pull request #4038: feat: Support Timestamp +/- Interval types

Posted by "alamb (via GitHub)" <gi...@apache.org>.
alamb commented on code in PR #4038:
URL: https://github.com/apache/arrow-rs/pull/4038#discussion_r1167576968


##########
arrow-arith/src/arithmetic.rs:
##########
@@ -3665,4 +3863,540 @@ mod tests {
             "1234567890.0000000000000000000000000000"
         );
     }
+
+    #[test]
+    fn test_timestamp_second_add_interval() {
+        // timestamp second + interval year month
+        let a = TimestampSecondArray::from(vec![1, 2, 3, 4, 5]);
+        let b = IntervalYearMonthArray::from(vec![
+            Some(IntervalYearMonthType::make_value(1, 1)),

Review Comment:
   The reason I was suggesting tests that had different non zero fields (e.g. `years` and `month`s in this example) 
   
   e.g.
   ```rust 
               Some(IntervalYearMonthType::make_value(1, 2)),
   ```
   
   Would be to catch errors such as when the year and month fields were mixed up in the code. For example, with
   
   ```rust
               Some(IntervalYearMonthType::make_value(1, 1)),
   ```
   
   If the code made a mistake with which field , these tests would still pass.
   
   
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-rs] metesynnada commented on a diff in pull request #4038: feat: Support Timestamp +/- Interval types

Posted by "metesynnada (via GitHub)" <gi...@apache.org>.
metesynnada commented on code in PR #4038:
URL: https://github.com/apache/arrow-rs/pull/4038#discussion_r1161263389


##########
arrow-arith/src/arithmetic.rs:
##########
@@ -739,6 +739,105 @@ pub fn add_dyn(left: &dyn Array, right: &dyn Array) -> Result<ArrayRef, ArrowErr
                 ))),
             }
         }
+        DataType::Timestamp(TimeUnit::Second, _) => {
+            let l = left.as_primitive::<TimestampSecondType>();
+            match right.data_type() {
+                DataType::Interval(IntervalUnit::YearMonth) => {
+                    let r = right.as_primitive::<IntervalYearMonthType>();
+                    let res = math_checked_op(l, r, TimestampSecondType::add_year_months)?;
+                    Ok(Arc::new(res))

Review Comment:
   You can use `Arc::new(res.with_timezone_opt(TZ))` to support timezone output (in all match arms) where `TZ` in `DataType::Timestamp(TimeUnit::Second, TZ)`



##########
arrow-array/src/types.rs:
##########
@@ -350,6 +350,650 @@ impl ArrowTimestampType for TimestampNanosecondType {
     }
 }
 
+impl TimestampSecondType {
+    /// Adds the given IntervalYearMonthType to an arrow TimestampSecondType
+    ///
+    /// # Arguments
+    ///
+    /// * `timestamp` - The date on which to perform the operation
+    /// * `delta` - The interval to add
+    pub fn add_year_months(
+        timestamp: <TimestampSecondType as ArrowPrimitiveType>::Native,
+        delta: <IntervalYearMonthType as ArrowPrimitiveType>::Native,
+    ) -> Result<<TimestampSecondType as ArrowPrimitiveType>::Native, ArrowError> {
+        let prior = NaiveDateTime::from_timestamp_opt(timestamp, 0).ok_or_else(|| {
+            ArrowError::ComputeError("Timestamp out of range".to_string())
+        })?;
+
+        let months = IntervalYearMonthType::to_months(delta);
+        let posterior = shift_months(prior, months);
+        TimestampSecondType::make_value(posterior)
+            .ok_or_else(|| ArrowError::ComputeError("Timestamp out of range".to_string()))
+    }
+
+    /// Adds the given IntervalDayTimeType to an arrow TimestampSecondType
+    ///
+    /// # Arguments
+    ///
+    /// * `timestamp` - The date on which to perform the operation
+    /// * `delta` - The interval to add
+    pub fn add_day_time(
+        timestamp: <TimestampSecondType as ArrowPrimitiveType>::Native,
+        delta: <IntervalDayTimeType as ArrowPrimitiveType>::Native,
+    ) -> Result<<TimestampSecondType as ArrowPrimitiveType>::Native, ArrowError> {
+        let (days, ms) = IntervalDayTimeType::to_parts(delta);
+        let res = NaiveDateTime::from_timestamp_opt(timestamp, 0).ok_or_else(|| {
+            ArrowError::ComputeError("Timestamp out of range".to_string())
+        })?;
+        let res = res
+            .checked_add_signed(Duration::days(days as i64))
+            .ok_or_else(|| {
+                ArrowError::ComputeError("Timestamp out of range".to_string())
+            })?;
+        let res = res
+            .checked_add_signed(Duration::milliseconds(ms as i64))
+            .ok_or_else(|| {
+                ArrowError::ComputeError("Timestamp out of range".to_string())
+            })?;
+        TimestampSecondType::make_value(res)
+            .ok_or_else(|| ArrowError::ComputeError("Timestamp out of range".to_string()))
+    }
+
+    /// Adds the given IntervalMonthDayNanoType to an arrow TimestampSecondType
+    ///
+    /// # Arguments
+    ///
+    /// * `timestamp` - The date on which to perform the operation
+    /// * `delta` - The interval to add
+    pub fn add_month_day_nano(
+        timestamp: <TimestampSecondType as ArrowPrimitiveType>::Native,
+        delta: <IntervalMonthDayNanoType as ArrowPrimitiveType>::Native,
+    ) -> Result<<TimestampSecondType as ArrowPrimitiveType>::Native, ArrowError> {
+        let (months, days, nanos) = IntervalMonthDayNanoType::to_parts(delta);
+        let res = NaiveDateTime::from_timestamp_opt(timestamp, 0).ok_or_else(|| {
+            ArrowError::ComputeError("Timestamp out of range".to_string())
+        })?;
+        let res = shift_months(res, months);
+        let res = res
+            .checked_add_signed(Duration::days(days as i64))
+            .ok_or_else(|| {
+                ArrowError::ComputeError("Timestamp out of range".to_string())
+            })?;
+        let res = res
+            .checked_add_signed(Duration::nanoseconds(nanos))
+            .ok_or_else(|| {
+                ArrowError::ComputeError("Timestamp out of range".to_string())
+            })?;
+        TimestampSecondType::make_value(res)
+            .ok_or_else(|| ArrowError::ComputeError("Timestamp out of range".to_string()))
+    }
+
+    /// Subtracts the given IntervalYearMonthType to an arrow TimestampSecondType
+    ///
+    /// # Arguments
+    ///
+    /// * `timestamp` - The date on which to perform the operation
+    /// * `delta` - The interval to add
+    pub fn subtract_year_months(
+        timestamp: <TimestampSecondType as ArrowPrimitiveType>::Native,
+        delta: <IntervalYearMonthType as ArrowPrimitiveType>::Native,
+    ) -> Result<<TimestampSecondType as ArrowPrimitiveType>::Native, ArrowError> {
+        let prior = NaiveDateTime::from_timestamp_opt(timestamp, 0).ok_or_else(|| {
+            ArrowError::ComputeError("Timestamp out of range".to_string())
+        })?;
+        let months = IntervalYearMonthType::to_months(-delta);
+        let posterior = shift_months(prior, months);
+        TimestampSecondType::make_value(posterior)
+            .ok_or_else(|| ArrowError::ComputeError("Timestamp out of range".to_string()))
+    }
+
+    /// Subtracts the given IntervalDayTimeType to an arrow TimestampSecondType
+    ///
+    /// # Arguments
+    ///
+    /// * `timestamp` - The date on which to perform the operation
+    /// * `delta` - The interval to add
+    pub fn subtract_day_time(
+        timestamp: <TimestampSecondType as ArrowPrimitiveType>::Native,
+        delta: <IntervalDayTimeType as ArrowPrimitiveType>::Native,
+    ) -> Result<<TimestampSecondType as ArrowPrimitiveType>::Native, ArrowError> {
+        let (days, ms) = IntervalDayTimeType::to_parts(-delta);
+        let res = NaiveDateTime::from_timestamp_opt(timestamp, 0).ok_or_else(|| {
+            ArrowError::ComputeError("Timestamp out of range".to_string())
+        })?;
+        let res = res
+            .checked_add_signed(Duration::days(days as i64))
+            .ok_or_else(|| {
+                ArrowError::ComputeError("Timestamp out of range".to_string())
+            })?;
+        let res = res
+            .checked_add_signed(Duration::microseconds(ms as i64))
+            .ok_or_else(|| {
+                ArrowError::ComputeError("Timestamp out of range".to_string())
+            })?;
+        TimestampSecondType::make_value(res)
+            .ok_or_else(|| ArrowError::ComputeError("Timestamp out of range".to_string()))
+    }
+
+    /// Subtracts the given IntervalMonthDayNanoType to an arrow TimestampSecondType
+    ///
+    /// # Arguments
+    ///
+    /// * `timestamp` - The date on which to perform the operation
+    /// * `delta` - The interval to add
+    pub fn subtract_month_day_nano(
+        timestamp: <TimestampSecondType as ArrowPrimitiveType>::Native,
+        delta: <IntervalMonthDayNanoType as ArrowPrimitiveType>::Native,
+    ) -> Result<<TimestampSecondType as ArrowPrimitiveType>::Native, ArrowError> {
+        let (months, days, nanos) = IntervalMonthDayNanoType::to_parts(delta);
+        let res = NaiveDateTime::from_timestamp_opt(timestamp, 0).ok_or_else(|| {
+            ArrowError::ComputeError("Timestamp out of range".to_string())
+        })?;
+        let res = shift_months(res, -months);
+        let res = res
+            .checked_add_signed(Duration::days(-days as i64))
+            .ok_or_else(|| {
+                ArrowError::ComputeError("Timestamp out of range".to_string())
+            })?;
+        let res = res
+            .checked_add_signed(Duration::nanoseconds(-nanos))
+            .ok_or_else(|| {
+                ArrowError::ComputeError("Timestamp out of range".to_string())
+            })?;
+        TimestampSecondType::make_value(res)
+            .ok_or_else(|| ArrowError::ComputeError("Timestamp out of range".to_string()))
+    }
+}
+
+impl TimestampMicrosecondType {
+    /// Adds the given IntervalYearMonthType to an arrow TimestampMicrosecondType
+    ///
+    /// # Arguments
+    ///
+    /// * `timestamp` - The date on which to perform the operation
+    /// * `delta` - The interval to add
+    pub fn add_year_months(
+        timestamp: <TimestampMicrosecondType as ArrowPrimitiveType>::Native,
+        delta: <IntervalYearMonthType as ArrowPrimitiveType>::Native,
+    ) -> Result<<TimestampMicrosecondType as ArrowPrimitiveType>::Native, ArrowError>
+    {
+        let prior = NaiveDateTime::from_timestamp_micros(timestamp).ok_or_else(|| {
+            ArrowError::ComputeError("Timestamp out of range".to_string())
+        })?;
+        let months = IntervalYearMonthType::to_months(delta);
+        let posterior = shift_months(prior, months);
+        TimestampMicrosecondType::make_value(posterior)
+            .ok_or_else(|| ArrowError::ComputeError("Timestamp out of range".to_string()))
+    }
+
+    /// Adds the given IntervalDayTimeType to an arrow TimestampMicrosecondType
+    ///
+    /// # Arguments
+    ///
+    /// * `timestamp` - The date on which to perform the operation
+    /// * `delta` - The interval to add
+    pub fn add_day_time(
+        timestamp: <TimestampMicrosecondType as ArrowPrimitiveType>::Native,
+        delta: <IntervalDayTimeType as ArrowPrimitiveType>::Native,
+    ) -> Result<<TimestampMicrosecondType as ArrowPrimitiveType>::Native, ArrowError>
+    {
+        let (days, ms) = IntervalDayTimeType::to_parts(delta);
+        let res = NaiveDateTime::from_timestamp_micros(timestamp).ok_or_else(|| {
+            ArrowError::ComputeError("Timestamp out of range".to_string())
+        })?;
+        let res = res
+            .checked_add_signed(Duration::days(days as i64))
+            .ok_or_else(|| {
+                ArrowError::ComputeError("Timestamp out of range".to_string())
+            })?;
+        let res = res
+            .checked_add_signed(Duration::milliseconds(ms as i64))
+            .ok_or_else(|| {
+                ArrowError::ComputeError("Timestamp out of range".to_string())
+            })?;
+        TimestampMicrosecondType::make_value(res)
+            .ok_or_else(|| ArrowError::ComputeError("Timestamp out of range".to_string()))
+    }
+
+    /// Adds the given IntervalMonthDayNanoType to an arrow TimestampMicrosecondType
+    ///
+    /// # Arguments
+    ///
+    /// * `timestamp` - The date on which to perform the operation
+    /// * `delta` - The interval to add
+    pub fn add_month_day_nano(
+        timestamp: <TimestampMicrosecondType as ArrowPrimitiveType>::Native,
+        delta: <IntervalMonthDayNanoType as ArrowPrimitiveType>::Native,
+    ) -> Result<<TimestampMicrosecondType as ArrowPrimitiveType>::Native, ArrowError>
+    {
+        let (months, days, nanos) = IntervalMonthDayNanoType::to_parts(delta);
+        let res = NaiveDateTime::from_timestamp_micros(timestamp).ok_or_else(|| {
+            ArrowError::ComputeError("Timestamp out of range".to_string())
+        })?;
+        let res = shift_months(res, months);
+        let res = res
+            .checked_add_signed(Duration::days(days as i64))
+            .ok_or_else(|| {
+                ArrowError::ComputeError("Timestamp out of range".to_string())
+            })?;
+        let res = res
+            .checked_add_signed(Duration::nanoseconds(nanos))
+            .ok_or_else(|| {
+                ArrowError::ComputeError("Timestamp out of range".to_string())
+            })?;
+        TimestampMicrosecondType::make_value(res)
+            .ok_or_else(|| ArrowError::ComputeError("Timestamp out of range".to_string()))
+    }
+
+    /// Subtracts the given IntervalYearMonthType to an arrow TimestampMicrosecondType
+    ///
+    /// # Arguments
+    ///
+    /// * `timestamp` - The date on which to perform the operation
+    /// * `delta` - The interval to add
+    pub fn subtract_year_months(
+        timestamp: <TimestampMicrosecondType as ArrowPrimitiveType>::Native,
+        delta: <IntervalYearMonthType as ArrowPrimitiveType>::Native,
+    ) -> Result<<TimestampMicrosecondType as ArrowPrimitiveType>::Native, ArrowError>
+    {
+        let prior = NaiveDateTime::from_timestamp_micros(timestamp).ok_or_else(|| {
+            ArrowError::ComputeError("Timestamp out of range".to_string())
+        })?;
+        let months = IntervalYearMonthType::to_months(-delta);
+        let posterior = shift_months(prior, months);
+        TimestampMicrosecondType::make_value(posterior)
+            .ok_or_else(|| ArrowError::ComputeError("Timestamp out of range".to_string()))
+    }
+
+    /// Subtracts the given IntervalDayTimeType to an arrow TimestampMicrosecondType
+    ///
+    /// # Arguments
+    ///
+    /// * `timestamp` - The date on which to perform the operation
+    /// * `delta` - The interval to add
+    pub fn subtract_day_time(
+        timestamp: <TimestampMicrosecondType as ArrowPrimitiveType>::Native,
+        delta: <IntervalDayTimeType as ArrowPrimitiveType>::Native,
+    ) -> Result<<TimestampMicrosecondType as ArrowPrimitiveType>::Native, ArrowError>
+    {
+        let (days, ms) = IntervalDayTimeType::to_parts(-delta);
+        let res = NaiveDateTime::from_timestamp_micros(timestamp).ok_or_else(|| {
+            ArrowError::ComputeError("Timestamp out of range".to_string())
+        })?;
+        let res = res
+            .checked_add_signed(Duration::days(days as i64))
+            .ok_or_else(|| {
+                ArrowError::ComputeError("Timestamp out of range".to_string())
+            })?;
+        let res = res
+            .checked_add_signed(Duration::milliseconds(ms as i64))
+            .ok_or_else(|| {
+                ArrowError::ComputeError("Timestamp out of range".to_string())
+            })?;
+        TimestampMicrosecondType::make_value(res)
+            .ok_or_else(|| ArrowError::ComputeError("Timestamp out of range".to_string()))
+    }
+
+    /// Subtracts the given IntervalMonthDayNanoType to an arrow TimestampMicrosecondType
+    ///
+    /// # Arguments
+    ///
+    /// * `timestamp` - The date on which to perform the operation
+    /// * `delta` - The interval to add
+    pub fn subtract_month_day_nano(
+        timestamp: <TimestampMicrosecondType as ArrowPrimitiveType>::Native,
+        delta: <IntervalMonthDayNanoType as ArrowPrimitiveType>::Native,
+    ) -> Result<<TimestampMicrosecondType as ArrowPrimitiveType>::Native, ArrowError>
+    {
+        let (months, days, nanos) = IntervalMonthDayNanoType::to_parts(delta);
+        let res = NaiveDateTime::from_timestamp_micros(timestamp).ok_or_else(|| {
+            ArrowError::ComputeError("Timestamp out of range".to_string())
+        })?;
+        let res = shift_months(res, -months);
+        let res = res
+            .checked_add_signed(Duration::days(-days as i64))
+            .ok_or_else(|| {
+                ArrowError::ComputeError("Timestamp out of range".to_string())
+            })?;
+        let res = res
+            .checked_add_signed(Duration::nanoseconds(-nanos))
+            .ok_or_else(|| {
+                ArrowError::ComputeError("Timestamp out of range".to_string())
+            })?;
+        TimestampMicrosecondType::make_value(res)
+            .ok_or_else(|| ArrowError::ComputeError("Timestamp out of range".to_string()))
+    }
+}
+
+impl TimestampMillisecondType {
+    /// Adds the given IntervalYearMonthType to an arrow TimestampMillisecondType
+    ///
+    /// # Arguments
+    ///
+    /// * `timestamp` - The date on which to perform the operation
+    /// * `delta` - The interval to add
+    pub fn add_year_months(
+        timestamp: <TimestampMillisecondType as ArrowPrimitiveType>::Native,
+        delta: <IntervalYearMonthType as ArrowPrimitiveType>::Native,
+    ) -> Result<<TimestampMillisecondType as ArrowPrimitiveType>::Native, ArrowError>
+    {
+        let prior = NaiveDateTime::from_timestamp_millis(timestamp).ok_or_else(|| {
+            ArrowError::ComputeError("Timestamp out of range".to_string())
+        })?;
+        let months = IntervalYearMonthType::to_months(delta);
+        let posterior = shift_months(prior, months);
+        TimestampMillisecondType::make_value(posterior)
+            .ok_or_else(|| ArrowError::ComputeError("Timestamp out of range".to_string()))
+    }
+
+    /// Adds the given IntervalDayTimeType to an arrow TimestampMillisecondType
+    ///
+    /// # Arguments
+    ///
+    /// * `timestamp` - The date on which to perform the operation
+    /// * `delta` - The interval to add
+    pub fn add_day_time(
+        timestamp: <TimestampMillisecondType as ArrowPrimitiveType>::Native,
+        delta: <IntervalDayTimeType as ArrowPrimitiveType>::Native,
+    ) -> Result<<TimestampMillisecondType as ArrowPrimitiveType>::Native, ArrowError>
+    {
+        let (days, ms) = IntervalDayTimeType::to_parts(delta);
+        let res = NaiveDateTime::from_timestamp_millis(timestamp).ok_or_else(|| {
+            ArrowError::ComputeError("Timestamp out of range".to_string())
+        })?;
+        let res = res
+            .checked_add_signed(Duration::days(days as i64))
+            .ok_or_else(|| {
+                ArrowError::ComputeError("Timestamp out of range".to_string())
+            })?;
+        let res = res
+            .checked_add_signed(Duration::milliseconds(ms as i64))
+            .ok_or_else(|| {
+                ArrowError::ComputeError("Timestamp out of range".to_string())
+            })?;
+        TimestampMillisecondType::make_value(res)
+            .ok_or_else(|| ArrowError::ComputeError("Timestamp out of range".to_string()))
+    }
+
+    /// Adds the given IntervalMonthDayNanoType to an arrow TimestaTimestampMillisecondTypempSecondType

Review Comment:
   ```suggestion
       /// Adds the given IntervalMonthDayNanoType to an arrow TimestampMillisecondType
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-rs] alamb merged pull request #4038: feat: Support Timestamp +/- Interval types

Posted by "alamb (via GitHub)" <gi...@apache.org>.
alamb merged PR #4038:
URL: https://github.com/apache/arrow-rs/pull/4038


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow-rs] alamb commented on a diff in pull request #4038: feat: Support Timestamp +/- Interval types

Posted by "alamb (via GitHub)" <gi...@apache.org>.
alamb commented on code in PR #4038:
URL: https://github.com/apache/arrow-rs/pull/4038#discussion_r1167577107


##########
arrow-arith/src/arithmetic.rs:
##########
@@ -3665,4 +3863,540 @@ mod tests {
             "1234567890.0000000000000000000000000000"
         );
     }
+
+    #[test]
+    fn test_timestamp_second_add_interval() {
+        // timestamp second + interval year month
+        let a = TimestampSecondArray::from(vec![1, 2, 3, 4, 5]);
+        let b = IntervalYearMonthArray::from(vec![
+            Some(IntervalYearMonthType::make_value(1, 1)),

Review Comment:
   While I still think the tests can be improved, I also think they are good enough to merge



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org