This is an automated email from the ASF dual-hosted git repository.

github-bot pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/datafusion.git


The following commit(s) were added to refs/heads/main by this push:
     new 85c696df43 Date / time / interval arithmetic improvements (#19460)
85c696df43 is described below

commit 85c696df432c5fa1495069074a4873b05ecf8ca1
Author: Bruce Ritchie <[email protected]>
AuthorDate: Sun Dec 28 16:09:40 2025 -0500

    Date / time / interval arithmetic improvements (#19460)
    
    ## 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 #19022
    - Closes #19038
    - Closes #19024
    - Closes https://github.com/apache/datafusion/pull/19457
    
    ## Rationale for this change
    
    Improve coverage of date / time / interval arithmetic operations
    
    ## What changes are included in this PR?
    
    type coercion improvements, numerous slt tests.
    
    ## Are these changes tested?
    
    Yes
    
    ## Are there any user-facing changes?
    
    Additional arithmetic support.
    
    Thanks to @foskey51 for the timestamp + duration fix, @pepijnve for the
    initial set of .slt tests.
    
    ---------
    
    Co-authored-by: Pepijn Van Eeckhoudt <[email protected]>
    Co-authored-by: Martin Grigorov <[email protected]>
    Co-authored-by: Andrew Lamb <[email protected]>
---
 datafusion/expr-common/src/type_coercion/binary.rs | 103 ++++++++++++-
 .../src/type_coercion/binary/tests/arithmetic.rs   |   8 +-
 .../src/type_coercion/binary/tests/comparison.rs   |  58 +++----
 datafusion/optimizer/src/analyzer/type_coercion.rs | 166 +++++++++++++++++++--
 .../test_files/datetime/arith_date_date.slt        |  16 ++
 .../test_files/datetime/arith_date_integer.slt     |  89 +++++++++++
 .../test_files/datetime/arith_date_interval.slt    |  37 +++++
 .../test_files/datetime/arith_date_time.slt        | 116 ++++++++++++++
 .../test_files/datetime/arith_interval_double.slt  |  41 +++++
 .../datetime/arith_interval_interval.slt           |  27 ++++
 .../test_files/datetime/arith_negate_interval.slt  |  13 ++
 .../test_files/datetime/arith_time_interval.slt    |  70 +++++++++
 .../test_files/datetime/arith_time_time.slt        |  47 ++++++
 .../datetime/arith_timestamp_duration.slt          | 147 ++++++++++++++++++
 .../datetime/arith_timestamp_interval.slt          |  36 +++++
 .../datetime/arith_timestamp_timestamp.slt         |  13 ++
 .../{ => datetime}/current_date_timezone.slt       |   0
 .../{ => datetime}/current_time_timezone.slt       |   0
 .../test_files/{expr => datetime}/date_part.slt    |   0
 .../test_files/{ => datetime}/dates.slt            |   0
 .../test_files/{ => datetime}/interval.slt         |   0
 .../test_files/{ => datetime}/interval_mysql.slt   |   0
 .../test_files/{ => datetime}/timestamps.slt       |   0
 23 files changed, 933 insertions(+), 54 deletions(-)

diff --git a/datafusion/expr-common/src/type_coercion/binary.rs 
b/datafusion/expr-common/src/type_coercion/binary.rs
index 18603991ae..de16e9e010 100644
--- a/datafusion/expr-common/src/type_coercion/binary.rs
+++ b/datafusion/expr-common/src/type_coercion/binary.rs
@@ -24,6 +24,8 @@ use crate::operator::Operator;
 
 use arrow::array::{Array, new_empty_array};
 use arrow::compute::can_cast_types;
+use arrow::datatypes::IntervalUnit::MonthDayNano;
+use arrow::datatypes::TimeUnit::*;
 use arrow::datatypes::{
     DECIMAL32_MAX_PRECISION, DECIMAL32_MAX_SCALE, DECIMAL64_MAX_PRECISION,
     DECIMAL64_MAX_SCALE, DECIMAL128_MAX_PRECISION, DECIMAL128_MAX_SCALE,
@@ -266,6 +268,18 @@ impl<'a> BinaryTypeCoercer<'a> {
                     rhs: rhs.clone(),
                     ret,
                 })
+            } else if let Some((lhs, rhs)) = temporal_math_coercion(lhs, rhs) {
+                // Temporal arithmetic, e.g. Date32 + int64, Timestamp + 
duration, etc
+                let ret = self.get_result(&lhs, &rhs).map_err(|e| {
+                    plan_datafusion_err!(
+                        "Cannot get result type for temporal operation {} {} 
{}: {e}", self.lhs, self.op, self.rhs
+                    )
+                })?;
+                Ok(Signature {
+                    lhs,
+                    rhs,
+                    ret,
+                })
             } else if let Some(coerced) = 
temporal_coercion_strict_timezone(lhs, rhs) {
                 // Temporal arithmetic by first coercing to a common time 
representation
                 // e.g. Date32 - Timestamp
@@ -945,7 +959,7 @@ fn string_temporal_coercion(
                             None
                         }
                     }
-                    Timestamp(_, tz) => Some(Timestamp(TimeUnit::Nanosecond, 
tz.clone())),
+                    Timestamp(_, tz) => Some(Timestamp(Nanosecond, 
tz.clone())),
                     _ => None,
                 }
             }
@@ -1712,10 +1726,10 @@ pub fn regex_coercion(lhs_type: &DataType, rhs_type: 
&DataType) -> Option<DataTy
 fn is_time_with_valid_unit(datatype: &DataType) -> bool {
     matches!(
         datatype,
-        &DataType::Time32(TimeUnit::Second)
-            | &DataType::Time32(TimeUnit::Millisecond)
-            | &DataType::Time64(TimeUnit::Microsecond)
-            | &DataType::Time64(TimeUnit::Nanosecond)
+        &DataType::Time32(Second)
+            | &DataType::Time32(Millisecond)
+            | &DataType::Time64(Microsecond)
+            | &DataType::Time64(Nanosecond)
     )
 }
 
@@ -1801,6 +1815,73 @@ fn temporal_coercion_strict_timezone(
     }
 }
 
+fn temporal_math_coercion(
+    lhs_type: &DataType,
+    rhs_type: &DataType,
+) -> Option<(DataType, DataType)> {
+    use DataType::*;
+
+    match (lhs_type, rhs_type) {
+        // Coerce Date + int -> Date + Interval
+        (Date32, Int8 | Int16 | Int32 | Int64 | UInt8 | UInt16 | UInt32 | 
UInt64) => {
+            Some((Date32, Interval(MonthDayNano)))
+        }
+        (Int8 | Int16 | Int32 | Int64 | UInt8 | UInt16 | UInt32 | UInt64, 
Date32) => {
+            Some((Interval(MonthDayNano), Date32))
+        }
+        (Date64, Int8 | Int16 | Int32 | Int64 | UInt8 | UInt16 | UInt32 | 
UInt64) => {
+            Some((Date64, Interval(MonthDayNano)))
+        }
+        (Int8 | Int16 | Int32 | Int64 | UInt8 | UInt16 | UInt32 | UInt64, 
Date64) => {
+            Some((Interval(MonthDayNano), Date64))
+        }
+        // Coerce Date + time -> timestamp + Duration
+        (Date32, Time32(_)) => Some((Timestamp(Nanosecond, None), 
Duration(Nanosecond))),
+        (Time32(_), Date32) => Some((Duration(Nanosecond), 
Timestamp(Nanosecond, None))),
+
+        (Date32, Time64(_)) => Some((Timestamp(Nanosecond, None), 
Duration(Nanosecond))),
+        (Time64(_), Date32) => Some((Duration(Nanosecond), 
Timestamp(Nanosecond, None))),
+
+        (Date64, Time32(_)) => Some((Timestamp(Nanosecond, None), 
Duration(Nanosecond))),
+        (Time32(_), Date64) => Some((Duration(Nanosecond), 
Timestamp(Nanosecond, None))),
+
+        (Date64, Time64(_)) => Some((Timestamp(Nanosecond, None), 
Duration(Nanosecond))),
+        (Time64(_), Date64) => Some((Duration(Nanosecond), 
Timestamp(Nanosecond, None))),
+
+        // Coerce Duration to match Timestamp's unit,
+        // e.g. Timestamp(ms) + Duration(s) → Timestamp(ms) + Duration(ms)
+        (Timestamp(ts_unit, tz), Duration(_)) => {
+            Some((Timestamp(*ts_unit, tz.clone()), Duration(*ts_unit)))
+        }
+        (Duration(_), Timestamp(ts_unit, tz)) => {
+            Some((Duration(*ts_unit), Timestamp(*ts_unit, tz.clone())))
+        }
+        // time - time -> Interval
+        (Time32(_) | Time64(_), Time32(_) | Time64(_)) => {
+            Some((Interval(MonthDayNano), Interval(MonthDayNano)))
+        }
+        // time + interval -> Interval
+        (Time32(_) | Time64(_), Interval(_)) => {
+            Some((Interval(MonthDayNano), Interval(MonthDayNano)))
+        }
+        (Interval(_), Time32(_) | Time64(_)) => {
+            Some((Interval(MonthDayNano), Interval(MonthDayNano)))
+        }
+        // Interval * number => Interval
+        (
+            Interval(_),
+            Int8 | Int16 | Int32 | Int64 | UInt8 | UInt16 | UInt32 | UInt64 | 
Float16
+            | Float32 | Float64,
+        ) => Some((Interval(MonthDayNano), Interval(MonthDayNano))),
+        (
+            Int8 | Int16 | Int32 | Int64 | UInt8 | UInt16 | UInt32 | UInt64 | 
Float16
+            | Float32 | Float64,
+            Interval(_),
+        ) => Some((Interval(MonthDayNano), Interval(MonthDayNano))),
+        _ => None,
+    }
+}
+
 fn temporal_coercion(lhs_type: &DataType, rhs_type: &DataType) -> 
Option<DataType> {
     use arrow::datatypes::DataType::*;
     use arrow::datatypes::IntervalUnit::*;
@@ -1810,7 +1891,19 @@ fn temporal_coercion(lhs_type: &DataType, rhs_type: 
&DataType) -> Option<DataTyp
         (Interval(_) | Duration(_), Interval(_) | Duration(_)) => {
             Some(Interval(MonthDayNano))
         }
+        (Date32, Int8 | Int16 | Int32 | Int64 | UInt8 | UInt16 | UInt32 | 
UInt64)
+        | (Int8 | Int16 | Int32 | Int64 | UInt8 | UInt16 | UInt32 | UInt64, 
Date32) => {
+            Some(Date32)
+        }
+        (Date64, Int8 | Int16 | Int32 | Int64 | UInt8 | UInt16 | UInt32 | 
UInt64)
+        | (Int8 | Int16 | Int32 | Int64 | UInt8 | UInt16 | UInt32 | UInt64, 
Date64) => {
+            Some(Date64)
+        }
         (Date64, Date32) | (Date32, Date64) => Some(Date64),
+        (Date32, Time32(_)) | (Time32(_), Date32) => 
Some(Timestamp(Nanosecond, None)),
+        (Date32, Time64(_)) | (Time64(_), Date32) => 
Some(Timestamp(Nanosecond, None)),
+        (Date64, Time32(_)) | (Time32(_), Date64) => 
Some(Timestamp(Nanosecond, None)),
+        (Date64, Time64(_)) | (Time64(_), Date64) => 
Some(Timestamp(Nanosecond, None)),
         (Timestamp(_, None), Date64) | (Date64, Timestamp(_, None)) => {
             Some(Timestamp(Nanosecond, None))
         }
diff --git 
a/datafusion/expr-common/src/type_coercion/binary/tests/arithmetic.rs 
b/datafusion/expr-common/src/type_coercion/binary/tests/arithmetic.rs
index 4a3ccf7bb7..bb9d44953b 100644
--- a/datafusion/expr-common/src/type_coercion/binary/tests/arithmetic.rs
+++ b/datafusion/expr-common/src/type_coercion/binary/tests/arithmetic.rs
@@ -35,13 +35,13 @@ fn test_coercion_error() -> Result<()> {
 #[test]
 fn test_date_timestamp_arithmetic_error() -> Result<()> {
     let (lhs, rhs) = BinaryTypeCoercer::new(
-        &DataType::Timestamp(TimeUnit::Nanosecond, None),
+        &DataType::Timestamp(Nanosecond, None),
         &Operator::Minus,
-        &DataType::Timestamp(TimeUnit::Millisecond, None),
+        &DataType::Timestamp(Millisecond, None),
     )
     .get_input_types()?;
-    assert_eq!(lhs, DataType::Timestamp(TimeUnit::Millisecond, None));
-    assert_eq!(rhs, DataType::Timestamp(TimeUnit::Millisecond, None));
+    assert_eq!(lhs, DataType::Timestamp(Millisecond, None));
+    assert_eq!(rhs, DataType::Timestamp(Millisecond, None));
 
     let err =
         BinaryTypeCoercer::new(&DataType::Date32, &Operator::Plus, 
&DataType::Date64)
diff --git 
a/datafusion/expr-common/src/type_coercion/binary/tests/comparison.rs 
b/datafusion/expr-common/src/type_coercion/binary/tests/comparison.rs
index d19b2c7ba9..5d1b3bea75 100644
--- a/datafusion/expr-common/src/type_coercion/binary/tests/comparison.rs
+++ b/datafusion/expr-common/src/type_coercion/binary/tests/comparison.rs
@@ -122,51 +122,51 @@ fn test_type_coercion() -> Result<()> {
     );
     test_coercion_binary_rule!(
         DataType::Utf8,
-        DataType::Time32(TimeUnit::Second),
+        DataType::Time32(Second),
         Operator::Eq,
-        DataType::Time32(TimeUnit::Second)
+        DataType::Time32(Second)
     );
     test_coercion_binary_rule!(
         DataType::Utf8,
-        DataType::Time32(TimeUnit::Millisecond),
+        DataType::Time32(Millisecond),
         Operator::Eq,
-        DataType::Time32(TimeUnit::Millisecond)
+        DataType::Time32(Millisecond)
     );
     test_coercion_binary_rule!(
         DataType::Utf8,
-        DataType::Time64(TimeUnit::Microsecond),
+        DataType::Time64(Microsecond),
         Operator::Eq,
-        DataType::Time64(TimeUnit::Microsecond)
+        DataType::Time64(Microsecond)
     );
     test_coercion_binary_rule!(
         DataType::Utf8,
-        DataType::Time64(TimeUnit::Nanosecond),
+        DataType::Time64(Nanosecond),
         Operator::Eq,
-        DataType::Time64(TimeUnit::Nanosecond)
+        DataType::Time64(Nanosecond)
     );
     test_coercion_binary_rule!(
         DataType::Utf8,
-        DataType::Timestamp(TimeUnit::Second, None),
+        DataType::Timestamp(Second, None),
         Operator::Lt,
-        DataType::Timestamp(TimeUnit::Nanosecond, None)
+        DataType::Timestamp(Nanosecond, None)
     );
     test_coercion_binary_rule!(
         DataType::Utf8,
-        DataType::Timestamp(TimeUnit::Millisecond, None),
+        DataType::Timestamp(Millisecond, None),
         Operator::Lt,
-        DataType::Timestamp(TimeUnit::Nanosecond, None)
+        DataType::Timestamp(Nanosecond, None)
     );
     test_coercion_binary_rule!(
         DataType::Utf8,
-        DataType::Timestamp(TimeUnit::Microsecond, None),
+        DataType::Timestamp(Microsecond, None),
         Operator::Lt,
-        DataType::Timestamp(TimeUnit::Nanosecond, None)
+        DataType::Timestamp(Nanosecond, None)
     );
     test_coercion_binary_rule!(
         DataType::Utf8,
-        DataType::Timestamp(TimeUnit::Nanosecond, None),
+        DataType::Timestamp(Nanosecond, None),
         Operator::Lt,
-        DataType::Timestamp(TimeUnit::Nanosecond, None)
+        DataType::Timestamp(Nanosecond, None)
     );
     test_coercion_binary_rule!(
         DataType::Utf8,
@@ -552,28 +552,28 @@ fn test_type_coercion_compare() -> Result<()> {
     // Timestamps
     let utc: Option<Arc<str>> = Some("UTC".into());
     test_coercion_binary_rule!(
-        DataType::Timestamp(TimeUnit::Second, utc.clone()),
-        DataType::Timestamp(TimeUnit::Second, utc.clone()),
+        DataType::Timestamp(Second, utc.clone()),
+        DataType::Timestamp(Second, utc.clone()),
         Operator::Eq,
-        DataType::Timestamp(TimeUnit::Second, utc.clone())
+        DataType::Timestamp(Second, utc.clone())
     );
     test_coercion_binary_rule!(
-        DataType::Timestamp(TimeUnit::Second, utc.clone()),
-        DataType::Timestamp(TimeUnit::Second, Some("Europe/Brussels".into())),
+        DataType::Timestamp(Second, utc.clone()),
+        DataType::Timestamp(Second, Some("Europe/Brussels".into())),
         Operator::Eq,
-        DataType::Timestamp(TimeUnit::Second, utc.clone())
+        DataType::Timestamp(Second, utc.clone())
     );
     test_coercion_binary_rule!(
-        DataType::Timestamp(TimeUnit::Second, Some("America/New_York".into())),
-        DataType::Timestamp(TimeUnit::Second, Some("Europe/Brussels".into())),
+        DataType::Timestamp(Second, Some("America/New_York".into())),
+        DataType::Timestamp(Second, Some("Europe/Brussels".into())),
         Operator::Eq,
-        DataType::Timestamp(TimeUnit::Second, Some("America/New_York".into()))
+        DataType::Timestamp(Second, Some("America/New_York".into()))
     );
     test_coercion_binary_rule!(
-        DataType::Timestamp(TimeUnit::Second, Some("Europe/Brussels".into())),
-        DataType::Timestamp(TimeUnit::Second, utc),
+        DataType::Timestamp(Second, Some("Europe/Brussels".into())),
+        DataType::Timestamp(Second, utc),
         Operator::Eq,
-        DataType::Timestamp(TimeUnit::Second, Some("Europe/Brussels".into()))
+        DataType::Timestamp(Second, Some("Europe/Brussels".into()))
     );
 
     // list
@@ -634,7 +634,7 @@ fn test_type_coercion_compare() -> Result<()> {
     );
 
     let inner_timestamp_field = Arc::new(Field::new_list_field(
-        DataType::Timestamp(TimeUnit::Microsecond, None),
+        DataType::Timestamp(Microsecond, None),
         true,
     ));
     let result_type = BinaryTypeCoercer::new(
diff --git a/datafusion/optimizer/src/analyzer/type_coercion.rs 
b/datafusion/optimizer/src/analyzer/type_coercion.rs
index bc317e9c20..0545ffb3b1 100644
--- a/datafusion/optimizer/src/analyzer/type_coercion.rs
+++ b/datafusion/optimizer/src/analyzer/type_coercion.rs
@@ -17,15 +17,16 @@
 
 //! Optimizer rule for type validation and coercion
 
-use std::sync::Arc;
-
+use arrow::compute::can_cast_types;
 use datafusion_expr::binary::BinaryTypeCoercer;
 use itertools::{Itertools as _, izip};
-
-use arrow::datatypes::{DataType, Field, IntervalUnit, Schema};
+use std::sync::Arc;
 
 use crate::analyzer::AnalyzerRule;
 use crate::utils::NamePreserver;
+
+use arrow::datatypes::{DataType, Field, IntervalUnit, Schema, TimeUnit};
+use arrow::temporal_conversions::SECONDS_IN_DAY;
 use datafusion_common::config::ConfigOptions;
 use datafusion_common::tree_node::{Transformed, TreeNode, TreeNodeRewriter};
 use datafusion_common::{
@@ -50,9 +51,9 @@ use datafusion_expr::type_coercion::other::{
 use datafusion_expr::type_coercion::{is_datetime, 
is_utf8_or_utf8view_or_large_utf8};
 use datafusion_expr::utils::merge_schema;
 use datafusion_expr::{
-    AggregateUDF, Expr, ExprSchemable, Join, Limit, LogicalPlan, Operator, 
Projection,
-    ScalarUDF, Union, WindowFrame, WindowFrameBound, WindowFrameUnits, 
is_false,
-    is_not_false, is_not_true, is_not_unknown, is_true, is_unknown, not,
+    AggregateUDF, Cast, Expr, ExprSchemable, Join, Limit, LogicalPlan, 
Operator,
+    Projection, ScalarUDF, Union, WindowFrame, WindowFrameBound, 
WindowFrameUnits,
+    is_false, is_not_false, is_not_true, is_not_unknown, is_true, is_unknown, 
lit, not,
 };
 
 /// Performs type coercion by determining the schema
@@ -290,17 +291,150 @@ impl<'a> TypeCoercionRewriter<'a> {
         right: Expr,
         right_schema: &DFSchema,
     ) -> Result<(Expr, Expr)> {
-        let (left_type, right_type) = BinaryTypeCoercer::new(
-            &left.get_type(left_schema)?,
+        let left_data_type = left.get_type(left_schema)?;
+        let right_data_type = right.get_type(right_schema)?;
+        let (left_type, right_type) =
+            BinaryTypeCoercer::new(&left_data_type, &op, &right_data_type)
+                .get_input_types()?;
+        let left_cast_ok = can_cast_types(&left_data_type, &left_type);
+        let right_cast_ok = can_cast_types(&right_data_type, &right_type);
+
+        // handle special cases for
+        // * Date +/- int => Date
+        // * Date + time => Timestamp
+        let left_expr = if !left_cast_ok {
+            Self::coerce_date_time_math_op(
+                left,
+                &op,
+                &left_data_type,
+                &left_type,
+                &right_type,
+            )?
+        } else {
+            left.cast_to(&left_type, left_schema)?
+        };
+
+        let right_expr = if !right_cast_ok {
+            Self::coerce_date_time_math_op(
+                right,
+                &op,
+                &right_data_type,
+                &right_type,
+                &left_type,
+            )?
+        } else {
+            right.cast_to(&right_type, right_schema)?
+        };
+
+        Ok((left_expr, right_expr))
+    }
+
+    fn coerce_date_time_math_op(
+        expr: Expr,
+        op: &Operator,
+        left_current_type: &DataType,
+        left_target_type: &DataType,
+        right_target_type: &DataType,
+    ) -> Result<Expr, DataFusionError> {
+        use DataType::*;
+
+        fn cast(expr: Expr, target_type: DataType) -> Expr {
+            Expr::Cast(Cast::new(Box::new(expr), target_type))
+        }
+
+        fn time_to_nanos(
+            expr: Expr,
+            expr_type: &DataType,
+        ) -> Result<Expr, DataFusionError> {
+            let expr = match expr_type {
+                Time32(TimeUnit::Second) => {
+                    cast(cast(expr, Int32), Int64)
+                        * lit(ScalarValue::Int64(Some(1_000_000_000)))
+                }
+                Time32(TimeUnit::Millisecond) => {
+                    cast(cast(expr, Int32), Int64)
+                        * lit(ScalarValue::Int64(Some(1_000_000)))
+                }
+                Time64(TimeUnit::Microsecond) => {
+                    cast(expr, Int64) * lit(ScalarValue::Int64(Some(1_000)))
+                }
+                Time64(TimeUnit::Nanosecond) => cast(expr, Int64),
+                t => return internal_err!("Unexpected time data type {t}"),
+            };
+
+            Ok(expr)
+        }
+
+        let e = match (
             &op,
-            &right.get_type(right_schema)?,
-        )
-        .get_input_types()?;
+            &left_current_type,
+            &left_target_type,
+            &right_target_type,
+        ) {
+            // int +/- date => date
+            (
+                Operator::Plus | Operator::Minus,
+                Int8 | Int16 | Int32 | Int64 | UInt8 | UInt16 | UInt32 | 
UInt64,
+                Interval(IntervalUnit::MonthDayNano),
+                Date32 | Date64,
+            ) => {
+                // cast to i64 first
+                let expr = match *left_current_type {
+                    Int64 => expr,
+                    _ => cast(expr, Int64),
+                };
+                // next, multiply by 86400 to get seconds
+                let expr = expr * lit(ScalarValue::from(SECONDS_IN_DAY));
+                // cast to duration
+                let expr = cast(expr, Duration(TimeUnit::Second));
+                // finally cast to interval
+                cast(expr, Interval(IntervalUnit::MonthDayNano))
+            }
+            // These might seem to be a bit convoluted, however for arrow to 
do date + time arithmetic
+            // date must be cast to Timestamp(Nanosecond) and time cast to 
Duration(Nanosecond)
+            // (they must be the same timeunit).
+            //
+            // For Time32/64 we first need to cast to an Int64, convert that 
to nanoseconds based
+            // on the time unit, then cast that to duration.
+            //
+            // Time + date -> timestamp or
+            (
+                Operator::Plus | Operator::Minus,
+                Time32(_) | Time64(_),
+                Duration(TimeUnit::Nanosecond),
+                Timestamp(TimeUnit::Nanosecond, None),
+            ) => {
+                // cast to int64, convert to nanoseconds
+                let expr = time_to_nanos(expr, left_current_type)?;
+                // cast to duration
+                cast(expr, Duration(TimeUnit::Nanosecond))
+            }
+            // Similar to above, for arrow to do time - time we need to 
convert to an interval.
+            // To do that we first need to cast to an Int64, convert that to 
nanoseconds based
+            // on the time unit, then cast that to duration, then finally cast 
to an interval.
+            //
+            // Time - time -> timestamp
+            (
+                Operator::Plus | Operator::Minus,
+                Time32(_) | Time64(_),
+                Interval(IntervalUnit::MonthDayNano),
+                Interval(IntervalUnit::MonthDayNano),
+            ) => {
+                // cast to int64, convert to nanoseconds
+                let expr = time_to_nanos(expr, left_current_type)?;
+                // cast to duration
+                let expr = cast(expr, Duration(TimeUnit::Nanosecond));
+                // finally cast to interval
+                cast(expr, Interval(IntervalUnit::MonthDayNano))
+            }
+            _ => {
+                return plan_err!(
+                    "Cannot automatically convert {left_current_type} to 
{left_target_type}"
+                );
+            }
+        };
 
-        Ok((
-            left.cast_to(&left_type, left_schema)?,
-            right.cast_to(&right_type, right_schema)?,
-        ))
+        Ok(e)
     }
 }
 
diff --git a/datafusion/sqllogictest/test_files/datetime/arith_date_date.slt 
b/datafusion/sqllogictest/test_files/datetime/arith_date_date.slt
new file mode 100644
index 0000000000..f6e4aad78b
--- /dev/null
+++ b/datafusion/sqllogictest/test_files/datetime/arith_date_date.slt
@@ -0,0 +1,16 @@
+# date - date → integer
+# Subtract dates, producing the number of days elapsed
+# date '2001-10-01' - date '2001-09-28' → 3
+
+# note that datafusion returns Duration whereas postgres returns an int
+# Tracking issue: https://github.com/apache/datafusion/issues/19528
+
+query ?
+SELECT '2001-10-01'::date - '2001-09-28'::date
+----
+3 days 0 hours 0 mins 0 secs
+
+query T
+SELECT arrow_typeof('2001-10-01'::date - '2001-09-28'::date)
+----
+Duration(s)
diff --git a/datafusion/sqllogictest/test_files/datetime/arith_date_integer.slt 
b/datafusion/sqllogictest/test_files/datetime/arith_date_integer.slt
new file mode 100644
index 0000000000..512c507d94
--- /dev/null
+++ b/datafusion/sqllogictest/test_files/datetime/arith_date_integer.slt
@@ -0,0 +1,89 @@
+# date + integer → date
+# Add a number of days to a date
+# date '2001-09-28' + 7 → 2001-10-05
+
+query D
+SELECT '2001-09-28'::date + 7
+----
+2001-10-05
+
+query D
+SELECT 7 + '2001-09-28'::date
+----
+2001-10-05
+
+query T
+SELECT arrow_typeof('2001-09-28'::date + 7)
+----
+Date32
+
+query D
+SELECT arrow_cast('2001-09-28', 'Date64') + 7
+----
+2001-10-05T00:00:00
+
+query D
+SELECT 7::smallint + '2001-09-28'::date
+----
+2001-10-05
+
+query D
+SELECT 7::smallint unsigned + '2001-09-28'::date
+----
+2001-10-05
+
+query D
+SELECT 7::int unsigned + '2001-09-28'::date
+----
+2001-10-05
+
+query D
+SELECT 7::bigint + '2001-09-28'::date
+----
+2001-10-05
+
+query D
+SELECT 7::bigint unsigned + '2001-09-28'::date
+----
+2001-10-05
+
+query D
+SELECT 7 + arrow_cast('2001-09-28', 'Date64')
+----
+2001-10-05T00:00:00
+
+query T
+SELECT arrow_typeof(arrow_cast('2001-09-28', 'Date64') + 7)
+----
+Date64
+
+# date - integer → date
+# Subtract a number of days from a date
+# date '2001-10-01' - 7 → 2001-09-24
+
+query D
+SELECT '2001-10-01'::date - 7
+----
+2001-09-24
+
+query D
+SELECT arrow_cast('2001-10-01', 'Date64') - 7
+----
+2001-09-24T00:00:00
+
+query T
+SELECT arrow_typeof('2001-10-01'::date - 7)
+----
+Date32
+
+query error Invalid arithmetic operation
+SELECT 7 - '2001-10-01'::date
+
+query error Invalid date arithmetic operation
+SELECT '2001-10-01'::date * 7
+
+query error Invalid date arithmetic operation
+SELECT '2001-10-01'::date / 7
+
+query error Invalid date arithmetic operation
+SELECT '2001-10-01'::date % 7
diff --git 
a/datafusion/sqllogictest/test_files/datetime/arith_date_interval.slt 
b/datafusion/sqllogictest/test_files/datetime/arith_date_interval.slt
new file mode 100644
index 0000000000..ad2e7ed496
--- /dev/null
+++ b/datafusion/sqllogictest/test_files/datetime/arith_date_interval.slt
@@ -0,0 +1,37 @@
+# postgresql behavior
+#
+# date + interval → timestamp
+# Add an interval to a date
+# date '2001-09-28' + interval '1 hour' → 2001-09-28 01:00:00
+#
+# note that while the above reflects what postgresql does
+# in the case of datafusion/arrow that is not the case. The
+# result will be date32/date64
+#
+# Tracking issue: https://github.com/apache/datafusion/issues/19527
+
+query D
+SELECT '2001-09-28'::date + interval '1 hour'
+----
+2001-09-28
+
+query T
+SELECT arrow_typeof('2001-09-28'::date + interval '1 hour')
+----
+Date32
+
+# postgresql behavior
+#
+# date - interval → timestamp
+# Subtract an interval from a date
+# date '2001-09-28' - interval '1 hour' → 2001-09-27 23:00:00
+
+query D
+SELECT '2001-09-28'::date - interval '25 hour'
+----
+2001-09-27
+
+query T
+SELECT arrow_typeof('2001-09-28'::date - interval '25 hour')
+----
+Date32
diff --git a/datafusion/sqllogictest/test_files/datetime/arith_date_time.slt 
b/datafusion/sqllogictest/test_files/datetime/arith_date_time.slt
new file mode 100644
index 0000000000..bc796a51ff
--- /dev/null
+++ b/datafusion/sqllogictest/test_files/datetime/arith_date_time.slt
@@ -0,0 +1,116 @@
+# date + time → timestamp
+# Add a time-of-day to a date
+# date '2001-09-28' + time '03:00' → 2001-09-28 03:00:00
+
+query P
+SELECT '2001-09-28'::date + '03:00'::time
+----
+2001-09-28T03:00:00
+
+query P
+SELECT '03:00'::time + '2001-09-28'::date
+----
+2001-09-28T03:00:00
+
+query T
+SELECT arrow_typeof('2001-09-28'::date + '03:00'::time)
+----
+Timestamp(ns)
+
+query P
+SELECT '2001-09-28'::date - '03:00'::time
+----
+2001-09-27T21:00:00
+
+query P
+SELECT arrow_cast('2001-09-28', 'Date32') + arrow_cast('03:00', 
'Time32(Second)')
+----
+2001-09-28T03:00:00
+
+query P
+SELECT arrow_cast('2001-09-28', 'Date32') + arrow_cast('03:00', 
'Time32(Millisecond)')
+----
+2001-09-28T03:00:00
+
+query P
+SELECT arrow_cast('2001-09-28', 'Date32') + arrow_cast('03:00', 
'Time64(Microsecond)')
+----
+2001-09-28T03:00:00
+
+query P
+SELECT arrow_cast('2001-09-28', 'Date32') + arrow_cast('03:00', 
'Time64(Nanosecond)')
+----
+2001-09-28T03:00:00
+
+query P
+SELECT arrow_cast('2001-09-28', 'Date64') + arrow_cast('03:00', 
'Time32(Second)')
+----
+2001-09-28T03:00:00
+
+query P
+SELECT arrow_cast('2001-09-28', 'Date64') + arrow_cast('03:00:00.123', 
'Time32(Millisecond)')
+----
+2001-09-28T03:00:00.123
+
+query P
+SELECT arrow_cast('2001-09-28', 'Date64') + arrow_cast('03:00:00.123456', 
'Time64(Microsecond)')
+----
+2001-09-28T03:00:00.123456
+
+query P
+SELECT arrow_cast('2001-09-28', 'Date64') + arrow_cast('03:00:00.001234567', 
'Time64(Nanosecond)')
+----
+2001-09-28T03:00:00.001234567
+
+query P
+SELECT arrow_cast('03:00', 'Time32(Second)') + arrow_cast('2001-09-28', 
'Date32')
+----
+2001-09-28T03:00:00
+
+query P
+SELECT arrow_cast('03:00', 'Time32(Millisecond)') + arrow_cast('2001-09-28', 
'Date32')
+----
+2001-09-28T03:00:00
+
+query P
+SELECT arrow_cast('03:00', 'Time64(Microsecond)') + arrow_cast('2001-09-28', 
'Date32')
+----
+2001-09-28T03:00:00
+
+query P
+SELECT arrow_cast('03:00', 'Time64(Nanosecond)') + arrow_cast('2001-09-28', 
'Date32')
+----
+2001-09-28T03:00:00
+
+query P
+SELECT arrow_cast('03:00', 'Time32(Second)') + arrow_cast('2001-09-28', 
'Date64')
+----
+2001-09-28T03:00:00
+
+query P
+SELECT arrow_cast('03:00:00.123', 'Time32(Millisecond)') + 
arrow_cast('2001-09-28', 'Date64')
+----
+2001-09-28T03:00:00.123
+
+query P
+SELECT arrow_cast('03:00:00.123456', 'Time64(Microsecond)') + 
arrow_cast('2001-09-28', 'Date64')
+----
+2001-09-28T03:00:00.123456
+
+query P
+SELECT arrow_cast('03:00:00.001234567', 'Time64(Nanosecond)') + 
arrow_cast('2001-09-28', 'Date64')
+----
+2001-09-28T03:00:00.001234567
+
+query error Invalid arithmetic operation
+SELECT '03:00'::time - '2001-09-28'::date
+
+query error Invalid timestamp arithmetic operation
+SELECT '2001-09-28'::date * '03:00'::time
+
+query error Invalid timestamp arithmetic operation
+SELECT '2001-09-28'::date / '03:00'::time
+
+query error Invalid timestamp arithmetic operation
+SELECT '2001-09-28'::date % '03:00'::time
+
diff --git 
a/datafusion/sqllogictest/test_files/datetime/arith_interval_double.slt 
b/datafusion/sqllogictest/test_files/datetime/arith_interval_double.slt
new file mode 100644
index 0000000000..d48d2b59c8
--- /dev/null
+++ b/datafusion/sqllogictest/test_files/datetime/arith_interval_double.slt
@@ -0,0 +1,41 @@
+# interval * double precision → interval
+# Multiply an interval by a scalar
+# interval '1 second' * 900 → 00:15:00
+# interval '1 day' * 21 → 21 days
+# interval '1 hour' * 3.5 → 03:30:00
+
+# these currently do not work - https://github.com/apache/arrow-rs/issues/9030
+
+query error Invalid interval arithmetic operation: Interval\(MonthDayNano\) \* 
Interval\(MonthDayNano\)
+SELECT interval '1 second' * 900
+
+
+query error Invalid interval arithmetic operation: Interval\(MonthDayNano\) \* 
Interval\(MonthDayNano\)
+SELECT 900 * interval '1 second'
+
+
+query error Invalid interval arithmetic operation: Interval\(MonthDayNano\) \* 
Interval\(MonthDayNano\)
+SELECT interval '1 day' * 21
+
+
+query error Invalid interval arithmetic operation: Interval\(MonthDayNano\) \* 
Interval\(MonthDayNano\)
+SELECT interval '1 hour' * 3.5
+
+
+query error Invalid interval arithmetic operation: Interval\(MonthDayNano\) \* 
Interval\(MonthDayNano\)
+SELECT 3.5 * interval '1 hour'
+
+
+query error Invalid interval arithmetic operation: Interval\(MonthDayNano\) \* 
Interval\(MonthDayNano\)
+SELECT arrow_typeof(interval '1 second' * 900)
+
+# interval / double precision → interval
+# Divide an interval by a scalar
+# interval '1 hour' / 1.5 → 00:40:00
+
+query error Invalid interval arithmetic operation: Interval\(MonthDayNano\) / 
Interval\(MonthDayNano\)
+SELECT interval '1 hour' / 1.5
+
+
+query error Invalid interval arithmetic operation: Interval\(MonthDayNano\) / 
Interval\(MonthDayNano\)
+SELECT arrow_typeof(interval '1 hour' / 1.5)
diff --git 
a/datafusion/sqllogictest/test_files/datetime/arith_interval_interval.slt 
b/datafusion/sqllogictest/test_files/datetime/arith_interval_interval.slt
new file mode 100644
index 0000000000..d8a701356b
--- /dev/null
+++ b/datafusion/sqllogictest/test_files/datetime/arith_interval_interval.slt
@@ -0,0 +1,27 @@
+# interval + interval → interval
+# Add intervals
+# interval '1 day' + interval '1 hour' → 1 day 01:00:00
+
+query ?
+SELECT interval '1 day' + interval '1 hour'
+----
+1 days 1 hours
+
+query T
+SELECT arrow_typeof(interval '1 day' + interval '1 hour')
+----
+Interval(MonthDayNano)
+
+# interval - interval → interval
+# Subtract intervals
+# interval '1 day' - interval '1 hour' → 1 day -01:00:00
+
+query ?
+SELECT interval '1 day' - interval '1 hour'
+----
+1 days -1 hours
+
+query T
+SELECT arrow_typeof(interval '1 day' - interval '1 hour')
+----
+Interval(MonthDayNano)
diff --git 
a/datafusion/sqllogictest/test_files/datetime/arith_negate_interval.slt 
b/datafusion/sqllogictest/test_files/datetime/arith_negate_interval.slt
new file mode 100644
index 0000000000..52ef046bf2
--- /dev/null
+++ b/datafusion/sqllogictest/test_files/datetime/arith_negate_interval.slt
@@ -0,0 +1,13 @@
+# - interval → interval
+# Negate an interval
+# - interval '23 hours' → -23:00:00
+
+query ?
+SELECT - interval '23 hours'
+----
+-23 hours
+
+query T
+SELECT arrow_typeof(- interval '23 hours')
+----
+Interval(MonthDayNano)
diff --git 
a/datafusion/sqllogictest/test_files/datetime/arith_time_interval.slt 
b/datafusion/sqllogictest/test_files/datetime/arith_time_interval.slt
new file mode 100644
index 0000000000..997eae9b1b
--- /dev/null
+++ b/datafusion/sqllogictest/test_files/datetime/arith_time_interval.slt
@@ -0,0 +1,70 @@
+# postgresql behavior
+#
+# time + interval → time
+# Add an interval to a time
+# time '01:00' + interval '3 hours' → 04:00:00
+#
+# note that while the above reflects what postgresql does
+# in the case of datafusion/arrow that is not the case. The
+# result will be an interval, not a time.
+
+query ?
+SELECT '01:00'::time + interval '3 hours'
+----
+4 hours
+
+query T
+SELECT arrow_typeof('01:00'::time + interval '3 hours')
+----
+Interval(MonthDayNano)
+
+query ?
+SELECT '22:00'::time + interval '3 hours'
+----
+25 hours
+
+query ?
+SELECT interval '3 hours' + '22:00'::time
+----
+25 hours
+
+query ?
+SELECT arrow_cast('22:00', 'Time32(Second)') + interval '3 hours'
+----
+25 hours
+
+query ?
+SELECT arrow_cast('22:00', 'Time32(Millisecond)') + interval '3 hours'
+----
+25 hours
+
+query ?
+SELECT arrow_cast('22:00', 'Time64(Microsecond)') + interval '3 hours'
+----
+25 hours
+
+query ?
+SELECT arrow_cast('22:00', 'Time64(Nanosecond)') + interval '3 hours'
+----
+25 hours
+
+# postgresql behavior
+#
+# time - interval → time
+# Subtract an interval from a time
+# time '05:00' - interval '2 hours' → 03:00:00
+
+query ?
+SELECT '05:00'::time - interval '2 hours'
+----
+3 hours
+
+query T
+SELECT arrow_typeof('05:00'::time - interval '2 hours')
+----
+Interval(MonthDayNano)
+
+query ?
+SELECT '02:00'::time - interval '3 hours'
+----
+-1 hours
diff --git a/datafusion/sqllogictest/test_files/datetime/arith_time_time.slt 
b/datafusion/sqllogictest/test_files/datetime/arith_time_time.slt
new file mode 100644
index 0000000000..4cf081970e
--- /dev/null
+++ b/datafusion/sqllogictest/test_files/datetime/arith_time_time.slt
@@ -0,0 +1,47 @@
+# time - time → interval
+# Subtract times
+# time '05:00' - time '03:00' → 02:00:00
+
+query ?
+SELECT '05:00'::time - '03:00'::time
+----
+2 hours
+
+query T
+SELECT arrow_typeof('05:00'::time - '03:00'::time)
+----
+Interval(MonthDayNano)
+
+query ?
+SELECT '05:00'::time + '03:00'::time
+----
+8 hours
+
+query ?
+SELECT arrow_cast('05:00', 'Time32(Second)') - arrow_cast('03:00', 
'Time32(Millisecond)')
+----
+2 hours
+
+query ?
+SELECT arrow_cast('05:00', 'Time32(Second)') - arrow_cast('03:00', 
'Time64(Microsecond)')
+----
+2 hours
+
+query ?
+SELECT arrow_cast('05:00', 'Time64(Microsecond)') - arrow_cast('03:00', 
'Time32(Millisecond)')
+----
+2 hours
+
+query ?
+SELECT arrow_cast('05:00', 'Time64(Nanosecond)') - arrow_cast('03:00', 
'Time32(Second)')
+----
+2 hours
+
+query error Invalid interval arithmetic operation
+SELECT '05:00'::time * '03:00'::time
+
+query error Invalid interval arithmetic operation
+SELECT '05:00'::time / '03:00'::time
+
+query error Invalid interval arithmetic operation
+SELECT '05:00'::time % '03:00'::time
diff --git 
a/datafusion/sqllogictest/test_files/datetime/arith_timestamp_duration.slt 
b/datafusion/sqllogictest/test_files/datetime/arith_timestamp_duration.slt
new file mode 100644
index 0000000000..10381346f8
--- /dev/null
+++ b/datafusion/sqllogictest/test_files/datetime/arith_timestamp_duration.slt
@@ -0,0 +1,147 @@
+# timestamp + duration → timestamp
+# Add an duration to a timestamp
+# timestamp '2001-09-28 01:00' + arrow_cast(12345000000000, 
'Duration(Nanosecond)') → 2001-09-29 00:00:00
+
+query P
+SELECT '2001-09-28T01:00:00'::timestamp + arrow_cast(12345, 
'Duration(Second)');
+----
+2001-09-28T04:25:45
+
+query P
+SELECT '2001-09-28T01:00:00'::timestamp - arrow_cast(12345, 
'Duration(Second)');
+----
+2001-09-27T21:34:15
+
+query P
+SELECT arrow_cast(12345, 'Duration(Second)') + 
'2001-09-28T01:00:00'::timestamp;
+----
+2001-09-28T04:25:45
+
+query T
+SELECT arrow_typeof('2001-09-28T01:00:00'::timestamp + arrow_cast(12345, 
'Duration(Second)'))
+----
+Timestamp(ns)
+
+query P
+SELECT '2001-09-28T01:00:00'::timestamp + arrow_cast(12345000, 
'Duration(Millisecond)');
+----
+2001-09-28T04:25:45
+
+query P
+SELECT '2001-09-28T01:00:00'::timestamp - arrow_cast(12345000, 
'Duration(Millisecond)');
+----
+2001-09-27T21:34:15
+
+query T
+SELECT arrow_typeof('2001-09-28T01:00:00'::timestamp + arrow_cast(12345000, 
'Duration(Millisecond)'))
+----
+Timestamp(ns)
+
+query P
+SELECT '2001-09-28T01:00:00'::timestamp + arrow_cast(12345000000, 
'Duration(Microsecond)');
+----
+2001-09-28T04:25:45
+
+query P
+SELECT '2001-09-28T01:00:00'::timestamp - arrow_cast(12345000000, 
'Duration(Microsecond)');
+----
+2001-09-27T21:34:15
+
+query T
+SELECT arrow_typeof('2001-09-28T01:00:00'::timestamp + arrow_cast(12345000000, 
'Duration(Microsecond)'))
+----
+Timestamp(ns)
+
+query P
+SELECT '2001-09-28T01:00:00'::timestamp + arrow_cast(12345000000999, 
'Duration(Nanosecond)');
+----
+2001-09-28T04:25:45.000000999
+
+query P
+SELECT '2001-09-28T01:00:00'::timestamp - arrow_cast(12345000000999, 
'Duration(Nanosecond)');
+----
+2001-09-27T21:34:14.999999001
+
+query T
+SELECT arrow_typeof('2001-09-28T01:00:00'::timestamp + 
arrow_cast(12345000000999, 'Duration(Nanosecond)'))
+----
+Timestamp(ns)
+
+# test with other timestamp timeunits beyond the default ns
+
+# second +/- millisecond
+query P
+SELECT arrow_cast('2001-09-28T01:00:00', 'Timestamp(Second)') + 
arrow_cast(12345000, 'Duration(Millisecond)');
+----
+2001-09-28T04:25:45
+
+query P
+SELECT arrow_cast('2001-09-28T01:00:00', 'Timestamp(Second)') - 
arrow_cast(12345000, 'Duration(Millisecond)');
+----
+2001-09-27T21:34:15
+
+query T
+SELECT arrow_typeof(arrow_cast('2001-09-28T01:00:00', 'Timestamp(Second)') + 
arrow_cast(12345000, 'Duration(Millisecond)'))
+----
+Timestamp(s)
+
+# second +/- microsecond
+query P
+SELECT arrow_cast('2001-09-28T01:00:00', 'Timestamp(Second)') + 
arrow_cast(12345000000, 'Duration(Microsecond)');
+----
+2001-09-28T04:25:45
+
+query P
+SELECT arrow_cast('2001-09-28T01:00:00', 'Timestamp(Second)') - 
arrow_cast(12345000000, 'Duration(Microsecond)');
+----
+2001-09-27T21:34:15
+
+query T
+SELECT arrow_typeof(arrow_cast('2001-09-28T01:00:00', 'Timestamp(Second)') + 
arrow_cast(12345000000, 'Duration(Microsecond)'))
+----
+Timestamp(s)
+
+# millisecond +/- nanosecond
+query P
+SELECT arrow_cast('2001-09-28T01:00:00', 'Timestamp(Millisecond)') + 
arrow_cast(12345000000999, 'Duration(Nanosecond)');
+----
+2001-09-28T04:25:45
+
+query P
+SELECT arrow_cast('2001-09-28T01:00:00', 'Timestamp(Millisecond)') - 
arrow_cast(12345000000999, 'Duration(Nanosecond)');
+----
+2001-09-27T21:34:15
+
+query T
+SELECT arrow_typeof(arrow_cast('2001-09-28T01:00:00', 
'Timestamp(Millisecond)') + arrow_cast(12345000000999, 'Duration(Nanosecond)'))
+----
+Timestamp(ms)
+
+# millisecond +/- microsecond
+query P
+SELECT arrow_cast('2001-09-28T01:00:00', 'Timestamp(Millisecond)') + 
arrow_cast(12345000000, 'Duration(Microsecond)');
+----
+2001-09-28T04:25:45
+
+query P
+SELECT arrow_cast('2001-09-28T01:00:00', 'Timestamp(Millisecond)') - 
arrow_cast(12345000000, 'Duration(Microsecond)');
+----
+2001-09-27T21:34:15
+
+query T
+SELECT arrow_typeof(arrow_cast('2001-09-28T01:00:00', 
'Timestamp(Millisecond)') + arrow_cast(12345000000, 'Duration(Microsecond)'))
+----
+Timestamp(ms)
+
+# while timestamp + duration makes sense, duration - timestamp does not
+query error Invalid arithmetic operation: Duration\(ns\) - Timestamp\(ns\)
+SELECT arrow_cast(12345, 'Duration(Second)') - 
'2001-09-28T01:00:00'::timestamp;
+
+query error Invalid timestamp arithmetic operation
+SELECT '2001-09-28T01:00:00'::timestamp * arrow_cast(12345, 
'Duration(Second)');
+
+query error Invalid timestamp arithmetic operation
+SELECT '2001-09-28T01:00:00'::timestamp % arrow_cast(12345, 
'Duration(Second)');
+
+query error Invalid timestamp arithmetic operation
+SELECT '2001-09-28T01:00:00'::timestamp / arrow_cast(12345, 
'Duration(Second)');
\ No newline at end of file
diff --git 
a/datafusion/sqllogictest/test_files/datetime/arith_timestamp_interval.slt 
b/datafusion/sqllogictest/test_files/datetime/arith_timestamp_interval.slt
new file mode 100644
index 0000000000..aaf629f1f0
--- /dev/null
+++ b/datafusion/sqllogictest/test_files/datetime/arith_timestamp_interval.slt
@@ -0,0 +1,36 @@
+# timestamp + interval → timestamp
+# Add an interval to a timestamp
+# timestamp '2001-09-28 01:00' + interval '23 hours' → 2001-09-29 00:00:00
+
+query P
+SELECT '2001-09-28T01:00:00'::timestamp + interval '23 hours'
+----
+2001-09-29T00:00:00
+
+query T
+SELECT arrow_typeof('2001-09-28T01:00:00'::timestamp + interval '23 hours')
+----
+Timestamp(ns)
+
+# timestamp - interval → timestamp
+# Subtract an interval from a timestamp
+# timestamp '2001-09-28 23:00' - interval '23 hours' → 2001-09-28 00:00:00
+
+query P
+SELECT '2001-09-28T23:00:00'::timestamp - interval '23 hours'
+----
+2001-09-28T00:00:00
+
+query T
+SELECT arrow_typeof('2001-09-28T23:00:00'::timestamp - interval '23 hours')
+----
+Timestamp(ns)
+
+query error Cannot coerce arithmetic expression Timestamp\(ns\) \* 
Interval\(MonthDayNano\) to valid types
+SELECT '2001-09-28T23:00:00'::timestamp * interval '23 hours'
+
+query error Cannot coerce arithmetic expression Timestamp\(ns\) / 
Interval\(MonthDayNano\) to valid types
+SELECT '2001-09-28T23:00:00'::timestamp / interval '23 hours'
+
+query error Cannot coerce arithmetic expression Timestamp\(ns\) % 
Interval\(MonthDayNano\) to valid types
+SELECT '2001-09-28T23:00:00'::timestamp % interval '23 hours'
diff --git 
a/datafusion/sqllogictest/test_files/datetime/arith_timestamp_timestamp.slt 
b/datafusion/sqllogictest/test_files/datetime/arith_timestamp_timestamp.slt
new file mode 100644
index 0000000000..975365ae22
--- /dev/null
+++ b/datafusion/sqllogictest/test_files/datetime/arith_timestamp_timestamp.slt
@@ -0,0 +1,13 @@
+# timestamp - timestamp → interval
+# Subtract timestamps (converting 24-hour intervals into days, similarly to 
justify_hours())
+# timestamp '2001-09-29 03:00' - timestamp '2001-07-27 12:00' → 63 days 
15:00:00
+
+query ?
+SELECT '2001-09-29T03:00:00'::timestamp - '2001-07-27T12:00:00'::timestamp
+----
+63 days 15 hours 0 mins 0.000000000 secs
+
+query T
+SELECT arrow_typeof('2001-09-29T03:00:00'::timestamp - 
'2001-07-27T12:00:00'::timestamp)
+----
+Duration(ns)
diff --git a/datafusion/sqllogictest/test_files/current_date_timezone.slt 
b/datafusion/sqllogictest/test_files/datetime/current_date_timezone.slt
similarity index 100%
rename from datafusion/sqllogictest/test_files/current_date_timezone.slt
rename to datafusion/sqllogictest/test_files/datetime/current_date_timezone.slt
diff --git a/datafusion/sqllogictest/test_files/current_time_timezone.slt 
b/datafusion/sqllogictest/test_files/datetime/current_time_timezone.slt
similarity index 100%
rename from datafusion/sqllogictest/test_files/current_time_timezone.slt
rename to datafusion/sqllogictest/test_files/datetime/current_time_timezone.slt
diff --git a/datafusion/sqllogictest/test_files/expr/date_part.slt 
b/datafusion/sqllogictest/test_files/datetime/date_part.slt
similarity index 100%
rename from datafusion/sqllogictest/test_files/expr/date_part.slt
rename to datafusion/sqllogictest/test_files/datetime/date_part.slt
diff --git a/datafusion/sqllogictest/test_files/dates.slt 
b/datafusion/sqllogictest/test_files/datetime/dates.slt
similarity index 100%
rename from datafusion/sqllogictest/test_files/dates.slt
rename to datafusion/sqllogictest/test_files/datetime/dates.slt
diff --git a/datafusion/sqllogictest/test_files/interval.slt 
b/datafusion/sqllogictest/test_files/datetime/interval.slt
similarity index 100%
rename from datafusion/sqllogictest/test_files/interval.slt
rename to datafusion/sqllogictest/test_files/datetime/interval.slt
diff --git a/datafusion/sqllogictest/test_files/interval_mysql.slt 
b/datafusion/sqllogictest/test_files/datetime/interval_mysql.slt
similarity index 100%
rename from datafusion/sqllogictest/test_files/interval_mysql.slt
rename to datafusion/sqllogictest/test_files/datetime/interval_mysql.slt
diff --git a/datafusion/sqllogictest/test_files/timestamps.slt 
b/datafusion/sqllogictest/test_files/datetime/timestamps.slt
similarity index 100%
rename from datafusion/sqllogictest/test_files/timestamps.slt
rename to datafusion/sqllogictest/test_files/datetime/timestamps.slt


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]


Reply via email to