From a87e19e0b6ff1783754e97f71c7d6b394d649659 Mon Sep 17 00:00:00 2001 From: Cancai Cai <77189278+caicancai@users.noreply.github.com> Date: Mon, 15 Jan 2024 20:43:05 +0800 Subject: [PATCH] Port tests in timestamp.rs to sqllogictest (#8859) * Port tests in timestamp.rs to sqllogictest * Fix clippy error --------- Co-authored-by: Andrew Lamb --- datafusion/core/tests/sql/mod.rs | 92 --- datafusion/core/tests/sql/timestamp.rs | 582 ------------------ .../sqllogictest/test_files/timestamps.slt | 148 +++++ 3 files changed, 148 insertions(+), 674 deletions(-) delete mode 100644 datafusion/core/tests/sql/timestamp.rs diff --git a/datafusion/core/tests/sql/mod.rs b/datafusion/core/tests/sql/mod.rs index 3f52d2aae894..d24f87ba3899 100644 --- a/datafusion/core/tests/sql/mod.rs +++ b/datafusion/core/tests/sql/mod.rs @@ -22,7 +22,6 @@ use arrow::{ util::display::array_value_to_string, }; use chrono::prelude::*; -use chrono::Duration; use datafusion::datasource::TableProvider; use datafusion::error::{DataFusionError, Result}; @@ -84,7 +83,6 @@ pub mod references; pub mod repartition; pub mod select; mod sql_api; -pub mod timestamp; fn create_join_context( column_left: &str, @@ -583,96 +581,6 @@ async fn register_alltypes_parquet(ctx: &SessionContext) { .unwrap(); } -fn make_timestamp_table() -> Result> -where - A: ArrowTimestampType, -{ - make_timestamp_tz_table::(None) -} - -fn make_timestamp_tz_table(tz: Option>) -> Result> -where - A: ArrowTimestampType, -{ - let schema = Arc::new(Schema::new(vec![ - Field::new("ts", DataType::Timestamp(A::UNIT, tz.clone()), false), - Field::new("value", DataType::Int32, true), - ])); - - let divisor = match A::UNIT { - TimeUnit::Nanosecond => 1, - TimeUnit::Microsecond => 1000, - TimeUnit::Millisecond => 1_000_000, - TimeUnit::Second => 1_000_000_000, - }; - - let timestamps = vec![ - 1599572549190855000i64 / divisor, // 2020-09-08T13:42:29.190855+00:00 - 1599568949190855000 / divisor, // 2020-09-08T12:42:29.190855+00:00 - 1599565349190855000 / divisor, // 2020-09-08T11:42:29.190855+00:00 - ]; // 2020-09-08T11:42:29.190855+00:00 - - let array = PrimitiveArray::::from_iter_values(timestamps).with_timezone_opt(tz); - - let data = RecordBatch::try_new( - schema.clone(), - vec![ - Arc::new(array), - Arc::new(Int32Array::from(vec![Some(1), Some(2), Some(3)])), - ], - )?; - let table = MemTable::try_new(schema, vec![vec![data]])?; - Ok(Arc::new(table)) -} - -fn make_timestamp_tz_sub_table( - tz1: Option>, - tz2: Option>, -) -> Result> -where - A: ArrowTimestampType, -{ - let schema = Arc::new(Schema::new(vec![ - Field::new("ts1", DataType::Timestamp(A::UNIT, tz1.clone()), false), - Field::new("ts2", DataType::Timestamp(A::UNIT, tz2.clone()), false), - Field::new("val", DataType::Int32, true), - ])); - - let divisor = match A::UNIT { - TimeUnit::Nanosecond => 1, - TimeUnit::Microsecond => 1000, - TimeUnit::Millisecond => 1_000_000, - TimeUnit::Second => 1_000_000_000, - }; - - let timestamps1 = vec![ - 1_678_892_420_000_000_000i64 / divisor, //2023-03-15T15:00:20.000_000_000 - 1_678_892_410_000_000_000i64 / divisor, //2023-03-15T15:00:10.000_000_000 - 1_678_892_430_000_000_000i64 / divisor, //2023-03-15T15:00:30.000_000_000 - ]; - let timestamps2 = vec![ - 1_678_892_400_000_000_000i64 / divisor, //2023-03-15T15:00:00.000_000_000 - 1_678_892_400_000_000_000i64 / divisor, //2023-03-15T15:00:00.000_000_000 - 1_678_892_400_000_000_000i64 / divisor, //2023-03-15T15:00:00.000_000_000 - ]; - - let array1 = - PrimitiveArray::::from_iter_values(timestamps1).with_timezone_opt(tz1); - let array2 = - PrimitiveArray::::from_iter_values(timestamps2).with_timezone_opt(tz2); - - let data = RecordBatch::try_new( - schema.clone(), - vec![ - Arc::new(array1), - Arc::new(array2), - Arc::new(Int32Array::from(vec![Some(1), Some(2), Some(3)])), - ], - )?; - let table = MemTable::try_new(schema, vec![vec![data]])?; - Ok(Arc::new(table)) -} - /// Return a new table provider that has a single Int32 column with /// values between `seq_start` and `seq_end` pub fn table_with_sequence( diff --git a/datafusion/core/tests/sql/timestamp.rs b/datafusion/core/tests/sql/timestamp.rs deleted file mode 100644 index e74857cb313b..000000000000 --- a/datafusion/core/tests/sql/timestamp.rs +++ /dev/null @@ -1,582 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -use super::*; -use std::ops::Add; - -#[tokio::test] -async fn group_by_timestamp_millis() -> Result<()> { - let ctx = SessionContext::new(); - - let schema = Arc::new(Schema::new(vec![ - Field::new( - "timestamp", - DataType::Timestamp(TimeUnit::Millisecond, None), - false, - ), - Field::new("count", DataType::Int32, false), - ])); - let base_dt = Utc.with_ymd_and_hms(2018, 7, 1, 6, 0, 0).unwrap(); // 2018-Jul-01 06:00 - let hour1 = Duration::hours(1); - let timestamps = vec![ - base_dt.timestamp_millis(), - (base_dt + hour1).timestamp_millis(), - base_dt.timestamp_millis(), - base_dt.timestamp_millis(), - (base_dt + hour1).timestamp_millis(), - (base_dt + hour1).timestamp_millis(), - ]; - let data = RecordBatch::try_new( - schema.clone(), - vec![ - Arc::new(TimestampMillisecondArray::from(timestamps)), - Arc::new(Int32Array::from(vec![10, 20, 30, 40, 50, 60])), - ], - )?; - ctx.register_batch("t1", data).unwrap(); - - let sql = - "SELECT timestamp, SUM(count) FROM t1 GROUP BY timestamp ORDER BY timestamp ASC"; - let actual = execute_to_batches(&ctx, sql).await; - let expected = [ - "+---------------------+---------------+", - "| timestamp | SUM(t1.count) |", - "+---------------------+---------------+", - "| 2018-07-01T06:00:00 | 80 |", - "| 2018-07-01T07:00:00 | 130 |", - "+---------------------+---------------+", - ]; - assert_batches_eq!(expected, &actual); - Ok(()) -} - -#[tokio::test] -async fn timestamp_add_interval_second() -> Result<()> { - let ctx = SessionContext::new(); - - let sql = "SELECT NOW(), NOW() + INTERVAL '1' SECOND;"; - let results = execute_to_batches(&ctx, sql).await; - let actual = result_vec(&results); - - let res1 = actual[0][0].as_str(); - let res2 = actual[0][1].as_str(); - - let t1_naive = DateTime::parse_from_rfc3339(res1).unwrap(); - let t2_naive = DateTime::parse_from_rfc3339(res2).unwrap(); - - assert_eq!(t1_naive.add(Duration::seconds(1)), t2_naive); - Ok(()) -} - -#[tokio::test] -async fn timestamp_sub_interval_days() -> Result<()> { - let ctx = SessionContext::new(); - - let sql = "SELECT NOW(), NOW() - INTERVAL '8' DAY;"; - let results = execute_to_batches(&ctx, sql).await; - let actual = result_vec(&results); - - let res1 = actual[0][0].as_str(); - let res2 = actual[0][1].as_str(); - - let t1_naive = DateTime::parse_from_rfc3339(res1).unwrap(); - let t2_naive = chrono::DateTime::parse_from_rfc3339(res2).unwrap(); - - assert_eq!(t1_naive.sub(Duration::days(8)), t2_naive); - Ok(()) -} - -#[tokio::test] -async fn timestamp_add_interval_months() -> Result<()> { - let ctx = SessionContext::new(); - let table_a = - make_timestamp_tz_table::(Some("+00:00".into()))?; - ctx.register_table("table_a", table_a)?; - - let sql = "SELECT ts, ts + INTERVAL '17' MONTH FROM table_a;"; - let results = execute_to_batches(&ctx, sql).await; - let actual_vec = result_vec(&results); - - for actual in actual_vec { - let res1 = actual[0].as_str(); - let res2 = actual[1].as_str(); - - let format = "%Y-%m-%dT%H:%M:%S%.6fZ"; - let t1_naive = NaiveDateTime::parse_from_str(res1, format).unwrap(); - let t2_naive = NaiveDateTime::parse_from_str(res2, format).unwrap(); - - let year = t1_naive.year() + (t1_naive.month0() as i32 + 17) / 12; - let month = (t1_naive.month0() + 17) % 12 + 1; - - assert_eq!( - t1_naive.with_year(year).unwrap().with_month(month).unwrap(), - t2_naive - ); - } - Ok(()) -} - -#[tokio::test] -async fn timestamp_sub_interval_years() -> Result<()> { - let ctx = SessionContext::new(); - - let sql = "SELECT NOW(), NOW() - INTERVAL '16' YEAR;"; - let results = execute_to_batches(&ctx, sql).await; - let actual = result_vec(&results); - - let res1 = actual[0][0].as_str(); - let res2 = actual[0][1].as_str(); - - let t1_naive = DateTime::parse_from_rfc3339(res1).unwrap(); - let t2_naive = DateTime::parse_from_rfc3339(res2).unwrap(); - - assert_eq!(t1_naive.with_year(t1_naive.year() - 16).unwrap(), t2_naive); - Ok(()) -} - -#[tokio::test] -async fn timestamp_array_add_interval() -> Result<()> { - let ctx = SessionContext::new(); - let table_a = make_timestamp_table::()?; - let table_b = make_timestamp_table::()?; - ctx.register_table("table_a", table_a)?; - ctx.register_table("table_b", table_b)?; - - let sql = "SELECT ts, ts - INTERVAL '8' MILLISECONDS FROM table_a"; - let actual = execute_to_batches(&ctx, sql).await; - let expected = [ - "+----------------------------+----------------------------------------------+", - "| ts | table_a.ts - IntervalMonthDayNano(\"8000000\") |", - "+----------------------------+----------------------------------------------+", - "| 2020-09-08T13:42:29.190855 | 2020-09-08T13:42:29.182855 |", - "| 2020-09-08T12:42:29.190855 | 2020-09-08T12:42:29.182855 |", - "| 2020-09-08T11:42:29.190855 | 2020-09-08T11:42:29.182855 |", - "+----------------------------+----------------------------------------------+", - ]; - assert_batches_eq!(expected, &actual); - - let sql = "SELECT ts, ts + INTERVAL '1' SECOND FROM table_b"; - let actual = execute_to_batches(&ctx, sql).await; - let expected = ["+----------------------------+-------------------------------------------------+", - "| ts | table_b.ts + IntervalMonthDayNano(\"1000000000\") |", - "+----------------------------+-------------------------------------------------+", - "| 2020-09-08T13:42:29.190855 | 2020-09-08T13:42:30.190855 |", - "| 2020-09-08T12:42:29.190855 | 2020-09-08T12:42:30.190855 |", - "| 2020-09-08T11:42:29.190855 | 2020-09-08T11:42:30.190855 |", - "+----------------------------+-------------------------------------------------+"]; - assert_batches_eq!(expected, &actual); - - let sql = "SELECT ts, ts + INTERVAL '2' MONTH FROM table_b"; - let actual = execute_to_batches(&ctx, sql).await; - let expected = ["+----------------------------+---------------------------------------------------------------------+", - "| ts | table_b.ts + IntervalMonthDayNano(\"158456325028528675187087900672\") |", - "+----------------------------+---------------------------------------------------------------------+", - "| 2020-09-08T13:42:29.190855 | 2020-11-08T13:42:29.190855 |", - "| 2020-09-08T12:42:29.190855 | 2020-11-08T12:42:29.190855 |", - "| 2020-09-08T11:42:29.190855 | 2020-11-08T11:42:29.190855 |", - "+----------------------------+---------------------------------------------------------------------+"]; - assert_batches_eq!(expected, &actual); - - let sql = "SELECT ts, ts - INTERVAL '16' YEAR FROM table_b"; - let actual = execute_to_batches(&ctx, sql).await; - let expected = ["+----------------------------+-----------------------------------------------------------------------+", - "| ts | table_b.ts - IntervalMonthDayNano(\"15211807202738752817960438464512\") |", - "+----------------------------+-----------------------------------------------------------------------+", - "| 2020-09-08T13:42:29.190855 | 2004-09-08T13:42:29.190855 |", - "| 2020-09-08T12:42:29.190855 | 2004-09-08T12:42:29.190855 |", - "| 2020-09-08T11:42:29.190855 | 2004-09-08T11:42:29.190855 |", - "+----------------------------+-----------------------------------------------------------------------+"]; - assert_batches_eq!(expected, &actual); - Ok(()) -} - -#[tokio::test] -async fn cast_timestamp_before_1970() -> Result<()> { - // this is a repro for issue #3082 - let ctx = SessionContext::new(); - - let sql = "select cast('1969-01-01T00:00:00Z' as timestamp);"; - let actual = execute_to_batches(&ctx, sql).await; - let expected = [ - "+------------------------------+", - "| Utf8(\"1969-01-01T00:00:00Z\") |", - "+------------------------------+", - "| 1969-01-01T00:00:00 |", - "+------------------------------+", - ]; - - assert_batches_eq!(expected, &actual); - - let sql = "select cast('1969-01-01T00:00:00.1Z' as timestamp);"; - let actual = execute_to_batches(&ctx, sql).await; - let expected = [ - "+--------------------------------+", - "| Utf8(\"1969-01-01T00:00:00.1Z\") |", - "+--------------------------------+", - "| 1969-01-01T00:00:00.100 |", - "+--------------------------------+", - ]; - - assert_batches_eq!(expected, &actual); - - Ok(()) -} - -#[tokio::test] -async fn test_arrow_typeof() -> Result<()> { - let ctx = SessionContext::new(); - - let sql = "select arrow_typeof(date_trunc('minute', to_timestamp_seconds(61)));"; - let actual = execute_to_batches(&ctx, sql).await; - - let expected = [ - "+--------------------------------------------------------------------------+", - "| arrow_typeof(date_trunc(Utf8(\"minute\"),to_timestamp_seconds(Int64(61)))) |", - "+--------------------------------------------------------------------------+", - "| Timestamp(Second, None) |", - "+--------------------------------------------------------------------------+", - ]; - assert_batches_eq!(expected, &actual); - - let sql = "select arrow_typeof(date_trunc('second', to_timestamp_millis(61)));"; - let actual = execute_to_batches(&ctx, sql).await; - let expected = [ - "+-------------------------------------------------------------------------+", - "| arrow_typeof(date_trunc(Utf8(\"second\"),to_timestamp_millis(Int64(61)))) |", - "+-------------------------------------------------------------------------+", - "| Timestamp(Millisecond, None) |", - "+-------------------------------------------------------------------------+", - ]; - assert_batches_eq!(expected, &actual); - - let sql = "select arrow_typeof(date_trunc('millisecond', to_timestamp_micros(61)));"; - let actual = execute_to_batches(&ctx, sql).await; - let expected = ["+------------------------------------------------------------------------------+", - "| arrow_typeof(date_trunc(Utf8(\"millisecond\"),to_timestamp_micros(Int64(61)))) |", - "+------------------------------------------------------------------------------+", - "| Timestamp(Microsecond, None) |", - "+------------------------------------------------------------------------------+"]; - assert_batches_eq!(expected, &actual); - - let sql = "select arrow_typeof(date_trunc('microsecond', to_timestamp(61)));"; - let actual = execute_to_batches(&ctx, sql).await; - let expected = [ - "+-----------------------------------------------------------------------+", - "| arrow_typeof(date_trunc(Utf8(\"microsecond\"),to_timestamp(Int64(61)))) |", - "+-----------------------------------------------------------------------+", - "| Timestamp(Nanosecond, None) |", - "+-----------------------------------------------------------------------+", - ]; - assert_batches_eq!(expected, &actual); - - Ok(()) -} - -#[tokio::test] -async fn cast_timestamp_to_timestamptz() -> Result<()> { - let ctx = SessionContext::new(); - let table_a = make_timestamp_table::()?; - - ctx.register_table("table_a", table_a)?; - - let sql = "SELECT ts::timestamptz, arrow_typeof(ts::timestamptz) FROM table_a;"; - let actual = execute_to_batches(&ctx, sql).await; - - let expected = [ - "+-----------------------------+---------------------------------------+", - "| table_a.ts | arrow_typeof(table_a.ts) |", - "+-----------------------------+---------------------------------------+", - "| 2020-09-08T13:42:29.190855Z | Timestamp(Nanosecond, Some(\"+00:00\")) |", - "| 2020-09-08T12:42:29.190855Z | Timestamp(Nanosecond, Some(\"+00:00\")) |", - "| 2020-09-08T11:42:29.190855Z | Timestamp(Nanosecond, Some(\"+00:00\")) |", - "+-----------------------------+---------------------------------------+", - ]; - assert_batches_eq!(expected, &actual); - - Ok(()) -} - -#[tokio::test] -async fn test_cast_to_time() -> Result<()> { - let ctx = SessionContext::new(); - let sql = "SELECT 0::TIME"; - let actual = execute_to_batches(&ctx, sql).await; - - let expected = [ - "+----------+", - "| Int64(0) |", - "+----------+", - "| 00:00:00 |", - "+----------+", - ]; - assert_batches_eq!(expected, &actual); - - Ok(()) -} - -#[tokio::test] -async fn test_cast_to_time_with_time_zone_should_not_work() -> Result<()> { - // this should not work until we implement tz for DataType::Time64 - let ctx = SessionContext::new(); - let sql = "SELECT 0::TIME WITH TIME ZONE"; - let results = plan_and_collect(&ctx, sql).await.unwrap_err(); - - assert_eq!( - results.strip_backtrace(), - "This feature is not implemented: Unsupported SQL type Time(None, WithTimeZone)" - ); - - Ok(()) -} - -#[tokio::test] -async fn test_cast_to_time_without_time_zone() -> Result<()> { - let ctx = SessionContext::new(); - let sql = "SELECT 0::TIME WITHOUT TIME ZONE"; - let actual = execute_to_batches(&ctx, sql).await; - - let expected = [ - "+----------+", - "| Int64(0) |", - "+----------+", - "| 00:00:00 |", - "+----------+", - ]; - assert_batches_eq!(expected, &actual); - - Ok(()) -} - -#[tokio::test] -async fn test_cast_to_timetz_should_not_work() -> Result<()> { - // this should not work until we implement tz for DataType::Time64 - let ctx = SessionContext::new(); - let sql = "SELECT 0::TIMETZ"; - let results = plan_and_collect(&ctx, sql).await.unwrap_err(); - - assert_eq!( - results.strip_backtrace(), - "This feature is not implemented: Unsupported SQL type Time(None, Tz)" - ); - Ok(()) -} - -#[tokio::test] -async fn test_current_date() -> Result<()> { - let ctx = SessionContext::new(); - - let sql = "select current_date() dt"; - let results = execute_to_batches(&ctx, sql).await; - assert_eq!( - results[0] - .schema() - .field_with_name("dt") - .unwrap() - .data_type() - .to_owned(), - DataType::Date32 - ); - - let sql = "select case when current_date() = cast(now() as date) then 'OK' else 'FAIL' end result"; - let results = execute_to_batches(&ctx, sql).await; - - let expected = [ - "+--------+", - "| result |", - "+--------+", - "| OK |", - "+--------+", - ]; - - assert_batches_eq!(expected, &results); - - Ok(()) -} - -#[tokio::test] -async fn test_current_time() -> Result<()> { - let ctx = SessionContext::new(); - - let sql = "select current_time() dt"; - let results = execute_to_batches(&ctx, sql).await; - assert_eq!( - results[0] - .schema() - .field_with_name("dt") - .unwrap() - .data_type() - .to_owned(), - DataType::Time64(TimeUnit::Nanosecond) - ); - - let sql = "select case when current_time() = (now()::bigint % 86400000000000)::time then 'OK' else 'FAIL' end result"; - let results = execute_to_batches(&ctx, sql).await; - - let expected = [ - "+--------+", - "| result |", - "+--------+", - "| OK |", - "+--------+", - ]; - - assert_batches_eq!(expected, &results); - Ok(()) -} - -#[tokio::test] -async fn test_ts_dt_binary_ops() -> Result<()> { - let ctx = SessionContext::new(); - // test cast in where clause - let sql = - "select count(1) result from (select now() as n) a where n = '2000-01-01'::date"; - let results = execute_to_batches(&ctx, sql).await; - - let expected = [ - "+--------+", - "| result |", - "+--------+", - "| 0 |", - "+--------+", - ]; - - assert_batches_eq!(expected, &results); - - // test cast in where ge clause - let sql = - "select count(1) result from (select now() as n) a where n >= '2000-01-01'::date"; - let results = execute_to_batches(&ctx, sql).await; - - let expected = [ - "+--------+", - "| result |", - "+--------+", - "| 1 |", - "+--------+", - ]; - - assert_batches_eq!(expected, &results); - - // test cast in equal select - let sql = "select now() = '2000-01-01'::date as result"; - let results = execute_to_batches(&ctx, sql).await; - - let expected = [ - "+--------+", - "| result |", - "+--------+", - "| false |", - "+--------+", - ]; - - assert_batches_eq!(expected, &results); - - // test cast in gt select - let sql = "select now() >= '2000-01-01'::date as result"; - let results = execute_to_batches(&ctx, sql).await; - - let expected = [ - "+--------+", - "| result |", - "+--------+", - "| true |", - "+--------+", - ]; - - assert_batches_eq!(expected, &results); - - //test cast path timestamp date using literals - let sql = "select '2000-01-01'::timestamp >= '2000-01-01'::date"; - let df = ctx.sql(sql).await.unwrap(); - - let plan = df.explain(true, false)?.collect().await?; - let batch = &plan[0]; - let mut res: Option = None; - for row in 0..batch.num_rows() { - if &array_value_to_string(batch.column(0), row)? == "initial_logical_plan" { - res = Some(array_value_to_string(batch.column(1), row)?); - break; - } - } - assert_eq!(res, Some("Projection: CAST(Utf8(\"2000-01-01\") AS Timestamp(Nanosecond, None)) >= CAST(Utf8(\"2000-01-01\") AS Date32)\n EmptyRelation".to_string())); - - //test cast path timestamp date using function - let sql = "select now() >= '2000-01-01'::date"; - let df = ctx.sql(sql).await.unwrap(); - - let plan = df.explain(true, false)?.collect().await?; - let batch = &plan[0]; - let mut res: Option = None; - for row in 0..batch.num_rows() { - if &array_value_to_string(batch.column(0), row)? == "initial_logical_plan" { - res = Some(array_value_to_string(batch.column(1), row)?); - break; - } - } - assert_eq!( - res, - Some( - "Projection: now() >= CAST(Utf8(\"2000-01-01\") AS Date32)\n EmptyRelation" - .to_string() - ) - ); - - let sql = "select now() = current_date()"; - let df = ctx.sql(sql).await.unwrap(); - - let plan = df.explain(true, false)?.collect().await?; - let batch = &plan[0]; - let mut res: Option = None; - for row in 0..batch.num_rows() { - if &array_value_to_string(batch.column(0), row)? == "initial_logical_plan" { - res = Some(array_value_to_string(batch.column(1), row)?); - break; - } - } - assert_eq!( - res, - Some("Projection: now() = current_date()\n EmptyRelation".to_string()) - ); - - Ok(()) -} - -// Cannot remove to sqllogictest, timezone support is not ready there. -#[tokio::test] -async fn timestamp_sub_with_tz() -> Result<()> { - let ctx = SessionContext::new(); - let table_a = make_timestamp_tz_sub_table::( - Some("America/Los_Angeles".into()), - Some("Europe/Istanbul".into()), - )?; - ctx.register_table("table_a", table_a)?; - - let sql = "SELECT val, ts1 - ts2 AS ts_diff FROM table_a ORDER BY ts2 - ts1"; - let actual = execute_to_batches(&ctx, sql).await; - let expected = [ - "+-----+-----------------------------------+", - "| val | ts_diff |", - "+-----+-----------------------------------+", - "| 3 | 0 days 0 hours 0 mins 30.000 secs |", - "| 1 | 0 days 0 hours 0 mins 20.000 secs |", - "| 2 | 0 days 0 hours 0 mins 10.000 secs |", - "+-----+-----------------------------------+", - ]; - assert_batches_eq!(expected, &actual); - - Ok(()) -} diff --git a/datafusion/sqllogictest/test_files/timestamps.slt b/datafusion/sqllogictest/test_files/timestamps.slt index 7829ce53ac9a..2ab3dbdac61b 100644 --- a/datafusion/sqllogictest/test_files/timestamps.slt +++ b/datafusion/sqllogictest/test_files/timestamps.slt @@ -70,6 +70,19 @@ select cast(now() as date) = today(); ---- true +########## +## Current time Tests +########## + +query B +select cast(now() as time) = current_time(); +---- +true + +query T +select case when current_time() = (now()::bigint % 86400000000000)::time then 'OK' else 'FAIL' end result +---- +OK ########## ## Timestamp Handling Tests @@ -195,6 +208,9 @@ SELECT to_timestamp_seconds(ts / 1000) FROM t1 LIMIT 3 2009-03-01T00:01:00 2009-04-01T00:00:00 +statement ok +drop table t1 + # query_cast_timestamp_nanos_to_others @@ -1811,6 +1827,26 @@ SELECT arrow_typeof(date_trunc('day', time)) FROM foo LIMIT 1 ---- Timestamp(Nanosecond, Some("+05:00")) +query T +select arrow_typeof(date_trunc('minute', to_timestamp_seconds(61))) +---- +Timestamp(Second, None) + +query T +select arrow_typeof(date_trunc('second', to_timestamp_millis(61))) +---- +Timestamp(Millisecond, None) + +query T +select arrow_typeof(date_trunc('millisecond', to_timestamp_micros(61))) +---- +Timestamp(Microsecond, None) + +query T +select arrow_typeof(date_trunc('microsecond', to_timestamp(61))) +---- +Timestamp(Nanosecond, None) + # check date_bin query P SELECT date_bin(INTERVAL '1 day', time, '1970-01-01T00:00:00+05:00') FROM foo @@ -1945,5 +1981,117 @@ SELECT table_a.ts, table_b.ts, table_a.ts = table_b.ts FROM table_a, table_b ord 2020-09-08T11:42:29 2020-09-08T11:42:29.190 false +statement ok +drop table table_a + +statement ok +drop table table_b + +########## +## Test query Group BY Timestamp Millisecond +########## + +statement ok +create table t1 (val int, ts timestamp) as values (80, '2018-07-01T06:00:00'::timestamp), (130, '2018-07-01T07:00:00'::timestamp) + +query PI +SELECT t1.ts, SUM(val) FROM t1 GROUP BY t1.ts ORDER BY t1.ts ASC +---- +2018-07-01T06:00:00 80 +2018-07-01T07:00:00 130 + +########## +## Test query Timestamp Add Interval Months +########## + +query PP +SELECT t1.ts, t1.ts + INTERVAL '17' MONTH FROM t1; +---- +2018-07-01T06:00:00 2019-12-01T06:00:00 +2018-07-01T07:00:00 2019-12-01T07:00:00 + +########## +## Test query Timestamp Add Interval Years +########## + +query PP +SELECT t1.ts, t1.ts + INTERVAL '1' YEAR FROM t1; +---- +2018-07-01T06:00:00 2019-07-01T06:00:00 +2018-07-01T07:00:00 2019-07-01T07:00:00 + +########## +## Test query Timestamp Add Interval MILLISECONDS +########## + +query PP +SELECT t1.ts, t1.ts - INTERVAL '8' MILLISECONDS FROM t1; +---- +2018-07-01T06:00:00 2018-07-01T05:59:59.992 +2018-07-01T07:00:00 2018-07-01T06:59:59.992 + +########## +## Test query Timestamp Add Interval SECOND +########## + +query PP +SELECT t1.ts, t1.ts + INTERVAL '1' SECOND FROM t1; +---- +2018-07-01T06:00:00 2018-07-01T06:00:01 +2018-07-01T07:00:00 2018-07-01T07:00:01 + +########## +## Test query CAST +########## + +query PT +SELECT t1.ts::timestamptz, arrow_typeof(t1.ts::timestamptz) FROM t1; +---- +2018-07-01T06:00:00Z Timestamp(Nanosecond, Some("+00")) +2018-07-01T07:00:00Z Timestamp(Nanosecond, Some("+00")) + +query D +SELECT 0::TIME +---- +00:00:00 + +query D +SELECT 0::TIME WITHOUT TIME ZONE +---- +00:00:00 + +query I +select count(1) result from (select now() as n) a where n = '2000-01-01'::date +---- +0 + +query I +select count(1) result from (select now() as n) a where n >= '2000-01-01'::date +---- +1 + +query B +select now() = '2000-01-01'::date as result +---- +false + +query B +select now() >= '2000-01-01'::date as result +---- +true + +statement ok +drop table t1 + +statement ok +create table table_a (val int, ts1 timestamp, ts2 timestamp) as values + (1, '2018-07-01T06:00:00'::timestamp, '2018-07-01T07:00:00'::timestamp), + (2, '2018-07-01T07:00:00'::timestamp, '2018-07-01T08:00:00'::timestamp) + +query I? +SELECT val, ts1 - ts2 AS ts_diff FROM table_a ORDER BY ts2 - ts1 +---- +1 0 days -1 hours 0 mins 0.000000000 secs +2 0 days -1 hours 0 mins 0.000000000 secs