You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@arrow.apache.org by "alamb (via GitHub)" <gi...@apache.org> on 2023/03/27 17:44:18 UTC

[GitHub] [arrow-rs] alamb opened a new issue, #3964: Subtracting timestamps should result in an interval

alamb opened a new issue, #3964:
URL: https://github.com/apache/arrow-rs/issues/3964

   **Describe the bug**
   Subtracting two `Timestamp` columns results in another Timestamp which is not correct
   
   **To Reproduce**
   
   ```rust
   fn subtract_timestamps() {
       let arr1 = TimestampNanosecondArray::from(vec![
           1_000,
           1_000_000,
           1_000_000_000,
           1_000_000_000_000,
           1_000_000_000_000_000,
           1_000_000_000_000_000_000,
       ]);
   
       let arr2 = TimestampNanosecondArray::from(vec![
           2_000,
           2_000_000,
           2_000_000_000,
           2_000_000_000_000,
           2_000_000_000_000_000,
           2_000_000_000_000_000_000,
       ]);
   
       println!(
           "input:\n{}\n{}",
           pretty_format_columns("arr1", &[Arc::new(arr1.clone()) as _]).unwrap(),
           pretty_format_columns("arr2", &[Arc::new(arr2.clone()) as _]).unwrap(),
       );
   
       let interval = subtract_dyn(&arr2, &arr1).unwrap();
   
       println!(
           "output{}:\n{}",
           interval.data_type(),
           pretty_format_columns("interval", &[Arc::new(interval.clone()) as _]).unwrap(),
       );
   }
   ```
   
   Which produces
   
   ```
   input:
   +----------------------------+
   | arr1                       |
   +----------------------------+
   | 1970-01-01T00:00:00.000001 |
   | 1970-01-01T00:00:00.001    |
   | 1970-01-01T00:00:01        |
   | 1970-01-01T00:16:40        |
   | 1970-01-12T13:46:40        |
   | 2001-09-09T01:46:40        |
   +----------------------------+
   +----------------------------+
   | arr2                       |
   +----------------------------+
   | 1970-01-01T00:00:00.000002 |
   | 1970-01-01T00:00:00.002    |
   | 1970-01-01T00:00:02        |
   | 1970-01-01T00:33:20        |
   | 1970-01-24T03:33:20        |
   | 2033-05-18T03:33:20        |
   +----------------------------+
   output Timestamp(Nanosecond, None):
   +----------------------------+
   | interval                   |
   +----------------------------+
   | 1970-01-01T00:00:00.000001 |
   | 1970-01-01T00:00:00.001    |
   | 1970-01-01T00:00:01        |
   | 1970-01-01T00:16:40        |
   | 1970-01-12T13:46:40        |
   | 2001-09-09T01:46:40        |
   +----------------------------+
   ```
   
   **Expected behavior**
   I expect the output to be an interval of type `Interval(MonthDayNano)` (not Timestamp)
   
   
   **Additional context**
   Here is what postgres does:
   
   ```sql
   postgres=# create table test as select now() - now();
   SELECT 1
   postgres=# \d test
                    Table "public.test"
     Column  |   Type   | Collation | Nullable | Default
   ----------+----------+-----------+----------+---------
    ?column? | interval |           |          |
   
   postgres=#
   ```
   


-- 
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.apache.org

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


[GitHub] [arrow-rs] leetcode-1533 commented on issue #3964: Subtracting `Timestamp` from `Timestamp` should produce a `Duration` (not `Timestamp`)

Posted by "leetcode-1533 (via GitHub)" <gi...@apache.org>.
leetcode-1533 commented on issue #3964:
URL: https://github.com/apache/arrow-rs/issues/3964#issuecomment-1485826934

   it is my first rust related PR, just want to highlight my plan for implement this: 
   
   Within function subtract_dyn(), we currently use typed_dict_math_op() and math_op().
   
   I would like to impement antoher function for timestamp types, called timestamp_math_op(), which deals with timestamp type specific operations. And always returns  Duration when calculating between two timestamps.


-- 
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] tustvold commented on issue #3964: Subtracting `Timestamp` from `Timestamp` should produce a `Duration` (not `Timestamp`)

Posted by "tustvold (via GitHub)" <gi...@apache.org>.
tustvold commented on issue #3964:
URL: https://github.com/apache/arrow-rs/issues/3964#issuecomment-1554480222

   `label_issue.py` automatically added labels {'arrow'} from #4244


-- 
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] tustvold commented on issue #3964: Subtracting `Timestamp` from `Timestamp` should be `Interval` (not `Timestamp`)

Posted by "tustvold (via GitHub)" <gi...@apache.org>.
tustvold commented on issue #3964:
URL: https://github.com/apache/arrow-rs/issues/3964#issuecomment-1485574728

   > I expect the output to be an interval of type Interval(MonthDayNano) (not Timestamp)
   
   Should it be an interval, or should it be a duration?


-- 
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] leetcode-1533 commented on issue #3964: Subtracting `Timestamp` from `Timestamp` should produce a `Duration` (not `Timestamp`)

Posted by "leetcode-1533 (via GitHub)" <gi...@apache.org>.
leetcode-1533 commented on issue #3964:
URL: https://github.com/apache/arrow-rs/issues/3964#issuecomment-1485709454

   hi, I would like to take this task


-- 
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] tustvold commented on issue #3964: Subtracting `Timestamp` from `Timestamp` should produce a `Duration` (not `Timestamp`)

Posted by "tustvold (via GitHub)" <gi...@apache.org>.
tustvold commented on issue #3964:
URL: https://github.com/apache/arrow-rs/issues/3964#issuecomment-1492995569

   > After merging that PR in datafusion, timestamp - timestamp operations give results in Interval types,
   
   Can we revisit this decision in DataFusion, interval types are probably not the correct type to be returning for such an operation


-- 
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] ozankabak commented on issue #3964: Subtracting `Timestamp` from `Timestamp` should produce a `Duration` (not `Timestamp`)

Posted by "ozankabak (via GitHub)" <gi...@apache.org>.
ozankabak commented on issue #3964:
URL: https://github.com/apache/arrow-rs/issues/3964#issuecomment-1494723280

   SGTM


-- 
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 issue #3964: Subtracting `Timestamp` from `Timestamp` should be `Interval` (not `Timestamp`)

Posted by "alamb (via GitHub)" <gi...@apache.org>.
alamb commented on issue #3964:
URL: https://github.com/apache/arrow-rs/issues/3964#issuecomment-1485577060

   > Should it be an interval, or should it be a duration?
   
   I guess I was thinking MonthDayNano with `month=0` and `day=0` would be the same as a Duration 🤔 Not sure if that makes snese


-- 
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 issue #3964: Subtracting `Timestamp` from `Timestamp` should produce a `Duration` (not `Timestamp`)

Posted by "alamb (via GitHub)" <gi...@apache.org>.
alamb commented on issue #3964:
URL: https://github.com/apache/arrow-rs/issues/3964#issuecomment-1485644415

   I think this is a good first issue because the semantics are well defined and there are some existing examples


-- 
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 issue #3964: Subtracting `Timestamp` from `Timestamp` should produce a `Duration` (not `Timestamp`)

Posted by "alamb (via GitHub)" <gi...@apache.org>.
alamb commented on issue #3964:
URL: https://github.com/apache/arrow-rs/issues/3964#issuecomment-1493015404

   I filed https://github.com/apache/arrow-rs/issues/3998 to track casting durations to/from intervals


-- 
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 issue #3964: Subtracting `Timestamp` from `Timestamp` should be `Interval` (not `Timestamp`)

Posted by "alamb (via GitHub)" <gi...@apache.org>.
alamb commented on issue #3964:
URL: https://github.com/apache/arrow-rs/issues/3964#issuecomment-1485629572

   I would also be fine with `Duration ` -- changing ticket


-- 
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 issue #3964: Subtracting `Timestamp` from `Timestamp` should produce a `Duration` (not `Timestamp`)

Posted by "alamb (via GitHub)" <gi...@apache.org>.
alamb commented on issue #3964:
URL: https://github.com/apache/arrow-rs/issues/3964#issuecomment-1485847939

   > I would like to impement antoher function for timestamp types, called timestamp_math_op(), which deals with timestamp type specific operations. And always returns Duration when calculating between two timestamps.
   
   Sounds good to me 


-- 
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] berkaysynnada commented on issue #3964: Subtracting `Timestamp` from `Timestamp` should produce a `Duration` (not `Timestamp`)

Posted by "berkaysynnada (via GitHub)" <gi...@apache.org>.
berkaysynnada commented on issue #3964:
URL: https://github.com/apache/arrow-rs/issues/3964#issuecomment-1492982046

   > it is my first rust related PR, just want to highlight my plan for implement this:
   > 
   > Within function subtract_dyn(), we currently use typed_dict_math_op() and math_op().
   > 
   > I would like to impement antoher function for timestamp types, called timestamp_math_op(), which deals with timestamp type specific operations. And always returns Duration when calculating between two timestamps.
   
   Thanks for working on this issue :)
   After merging that [PR](https://github.com/apache/arrow-datafusion/pull/5764) in datafusion, timestamp - timestamp operations give results in Interval types, and it is planned to get arithmetics moved to arrow-rs. Can you check that PR to not have an inconsistent behavior?


-- 
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] ozankabak commented on issue #3964: Subtracting `Timestamp` from `Timestamp` should produce a `Duration` (not `Timestamp`)

Posted by "ozankabak (via GitHub)" <gi...@apache.org>.
ozankabak commented on issue #3964:
URL: https://github.com/apache/arrow-rs/issues/3964#issuecomment-1493079290

   > The latter is significantly easier to reason about, although it would appear postgres lacks such a construct.
   
   Yes, the issue is most DB's (and SQL itself) simply use the timestamp - timestamp = interval pattern. However, from our perspective using durations is fine as long as there is casting/coercing mechanism to take care of the transformation cheaply.
   
   


-- 
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] tustvold closed issue #3964: Subtracting `Timestamp` from `Timestamp` should produce a `Duration` (not `Timestamp`)

Posted by "tustvold (via GitHub)" <gi...@apache.org>.
tustvold closed issue #3964: Subtracting `Timestamp` from `Timestamp`  should produce a `Duration` (not `Timestamp`) 
URL: https://github.com/apache/arrow-rs/issues/3964


-- 
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 issue #3964: Subtracting `Timestamp` from `Timestamp` should produce a `Duration` (not `Timestamp`)

Posted by "alamb (via GitHub)" <gi...@apache.org>.
alamb commented on issue #3964:
URL: https://github.com/apache/arrow-rs/issues/3964#issuecomment-1494660009

   > Yes, the issue is most DB's (and SQL itself) simply use the timestamp - timestamp = interval pattern. However, from our perspective using durations is fine as long as there is casting/coercing mechanism to take care of the transformation cheaply.
   
   I think the core problem is clear distinction between "duration" and "interval" that is made in Arrow and Rust is not found in SQL (e.g. there is no SQL duration type). The coercion / casting logic in DataFusion I think is the right place to reconcile the various Arrow types (Intervals with different time units, durations with different time units)


-- 
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 issue #3964: Subtracting `Timestamp` from `Timestamp` should produce a `Duration` (not `Timestamp`)

Posted by "alamb (via GitHub)" <gi...@apache.org>.
alamb commented on issue #3964:
URL: https://github.com/apache/arrow-rs/issues/3964#issuecomment-1493012999

   I think it would be fine to support converting to duration in arrow-rs (and we can convert to Interval in datafusion as needed)
   
   The reason we are pushing ahead with interval (rather than interval and duration) in DataFusion is to get something working incrementally without having to sort out all the subtleties with Intervals, Durations, arithmetic and conversions. 
   
   Then I think over time we can and will add more sophistication (like making the distinction between Duration and Interval and coercing automatically between them)  to DataFusion
   
   Thus, I suggest we get the kernels correct in arrow-rs (and provide the appropriate casting operations) and then we can upgrade DataFusion to use them. 
   
   So in this case, let's have timestamp - timestamp produce duration in arrow.rs sounds good.
   
   I will also file a ticket about casting to/from Duration and Interval


-- 
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