diff --git a/datafusion-examples/README.md b/datafusion-examples/README.md index 298ee9364efe..98a1fce3594f 100644 --- a/datafusion-examples/README.md +++ b/datafusion-examples/README.md @@ -64,6 +64,7 @@ cargo run --example csv_sql - [`simple_udaf.rs`](examples/simple_udaf.rs): Define and invoke a User Defined Aggregate Function (UDAF) - [`advanced_udaf.rs`](examples/advanced_udaf.rs): Define and invoke a more complicated User Defined Aggregate Function (UDAF) - [`simple_udfw.rs`](examples/simple_udwf.rs): Define and invoke a User Defined Window Function (UDWF) +- [`make_date.rs`](examples/make_date.rs): Examples of using the make_date function - [`to_timestamp.rs`](examples/to_timestamp.rs): Examples of using the to_timestamp functions - [`advanced_udwf.rs`](examples/advanced_udwf.rs): Define and invoke a more complicated User Defined Window Function (UDWF) diff --git a/datafusion-examples/examples/make_date.rs b/datafusion-examples/examples/make_date.rs new file mode 100644 index 000000000000..98bbb21bbff8 --- /dev/null +++ b/datafusion-examples/examples/make_date.rs @@ -0,0 +1,120 @@ +// 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 std::sync::Arc; + +use datafusion::arrow::array::Int32Array; +use datafusion::arrow::datatypes::{DataType, Field, Schema}; +use datafusion::arrow::record_batch::RecordBatch; +use datafusion::error::Result; +use datafusion::prelude::*; +use datafusion_common::assert_contains; + +/// This example demonstrates how to use the make_date +/// function in the DataFrame API as well as via sql. +#[tokio::main] +async fn main() -> Result<()> { + // define a schema. + let schema = Arc::new(Schema::new(vec![ + Field::new("y", DataType::Int32, false), + Field::new("m", DataType::Int32, false), + Field::new("d", DataType::Int32, false), + ])); + + // define data. + let batch = RecordBatch::try_new( + schema, + vec![ + Arc::new(Int32Array::from(vec![2020, 2021, 2022, 2023, 2024])), + Arc::new(Int32Array::from(vec![1, 2, 3, 4, 5])), + Arc::new(Int32Array::from(vec![15, 16, 17, 18, 19])), + ], + )?; + + // declare a new context. In spark API, this corresponds to a new spark SQLsession + let ctx = SessionContext::new(); + + // declare a table in memory. In spark API, this corresponds to createDataFrame(...). + ctx.register_batch("t", batch)?; + let df = ctx.table("t").await?; + + // use make_date function to convert col 'y', 'm' & 'd' to a date + let df = df.with_column("a", make_date(col("y"), col("m"), col("d")))?; + // use make_date function to convert col 'y' & 'm' with a static day to a date + let df = df.with_column("b", make_date(col("y"), col("m"), lit(22)))?; + + let df = df.select_columns(&["a", "b"])?; + + // print the results + df.show().await?; + + // use sql to convert col 'y', 'm' & 'd' to a date + let df = ctx.sql("select make_date(y, m, d) from t").await?; + + // print the results + df.show().await?; + + // use sql to convert col 'y' & 'm' with a static string day to a date + let df = ctx.sql("select make_date(y, m, '22') from t").await?; + + // print the results + df.show().await?; + + // math expressions work + let df = ctx.sql("select make_date(y + 1, m, d) from t").await?; + + // print the results + df.show().await?; + + // you can cast to supported types (int, bigint, varchar) if required + let df = ctx + .sql("select make_date(2024::bigint, 01::bigint, 27::varchar(3))") + .await?; + + // print the results + df.show().await?; + + // arrow casts also work + let df = ctx + .sql("select make_date(arrow_cast(2024, 'Int64'), arrow_cast(1, 'Int64'), arrow_cast(27, 'Int64'))") + .await?; + + // print the results + df.show().await?; + + // invalid column values will result in an error + let result = ctx + .sql("select make_date(2024, null, 23)") + .await? + .collect() + .await; + + let expected = "Execution error: Unable to parse date from null/empty value"; + assert_contains!(result.unwrap_err().to_string(), expected); + + // invalid date values will also result in an error + let result = ctx + .sql("select make_date(2024, 01, 32)") + .await? + .collect() + .await; + + let expected = "Execution error: Unable to parse date from 2024, 1, 32"; + assert_contains!(result.unwrap_err().to_string(), expected); + + Ok(()) +} diff --git a/datafusion/expr/src/built_in_function.rs b/datafusion/expr/src/built_in_function.rs index e86d6172cecd..d8c22e69335d 100644 --- a/datafusion/expr/src/built_in_function.rs +++ b/datafusion/expr/src/built_in_function.rs @@ -295,6 +295,8 @@ pub enum BuiltinScalarFunction { CurrentDate, /// current_time CurrentTime, + /// make_date + MakeDate, /// translate Translate, /// trim @@ -484,6 +486,7 @@ impl BuiltinScalarFunction { BuiltinScalarFunction::ToTimestampMicros => Volatility::Immutable, BuiltinScalarFunction::ToTimestampNanos => Volatility::Immutable, BuiltinScalarFunction::ToTimestampSeconds => Volatility::Immutable, + BuiltinScalarFunction::MakeDate => Volatility::Immutable, BuiltinScalarFunction::Translate => Volatility::Immutable, BuiltinScalarFunction::Trim => Volatility::Immutable, BuiltinScalarFunction::Upper => Volatility::Immutable, @@ -834,6 +837,7 @@ impl BuiltinScalarFunction { } BuiltinScalarFunction::CurrentDate => Ok(Date32), BuiltinScalarFunction::CurrentTime => Ok(Time64(Nanosecond)), + BuiltinScalarFunction::MakeDate => Ok(Date32), BuiltinScalarFunction::Translate => { utf8_to_str_type(&input_expr_types[0], "translate") } @@ -1379,6 +1383,11 @@ impl BuiltinScalarFunction { | BuiltinScalarFunction::CurrentTime => { Signature::uniform(0, vec![], self.volatility()) } + BuiltinScalarFunction::MakeDate => Signature::uniform( + 3, + vec![Int32, Int64, UInt32, UInt64, Utf8], + self.volatility(), + ), BuiltinScalarFunction::Isnan | BuiltinScalarFunction::Iszero => { Signature::one_of( vec![Exact(vec![Float32]), Exact(vec![Float64])], @@ -1523,6 +1532,7 @@ impl BuiltinScalarFunction { BuiltinScalarFunction::Now => &["now"], BuiltinScalarFunction::CurrentDate => &["current_date", "today"], BuiltinScalarFunction::CurrentTime => &["current_time"], + BuiltinScalarFunction::MakeDate => &["make_date"], BuiltinScalarFunction::DateBin => &["date_bin"], BuiltinScalarFunction::DateTrunc => &["date_trunc", "datetrunc"], BuiltinScalarFunction::DatePart => &["date_part", "datepart"], diff --git a/datafusion/expr/src/expr_fn.rs b/datafusion/expr/src/expr_fn.rs index a0223fd953fc..dddf176dbe9f 100644 --- a/datafusion/expr/src/expr_fn.rs +++ b/datafusion/expr/src/expr_fn.rs @@ -921,6 +921,7 @@ scalar_expr!( scalar_expr!(CurrentDate, current_date, ,"returns current UTC date as a [`DataType::Date32`] value"); scalar_expr!(Now, now, ,"returns current timestamp in nanoseconds, using the same value for all instances of now() in same statement"); scalar_expr!(CurrentTime, current_time, , "returns current UTC time as a [`DataType::Time64`] value"); +scalar_expr!(MakeDate, make_date, year month day, "make a date from year, month and day component parts"); scalar_expr!(Nanvl, nanvl, x y, "returns x if x is not NaN otherwise returns y"); scalar_expr!( Isnan, diff --git a/datafusion/physical-expr/src/datetime_expressions.rs b/datafusion/physical-expr/src/datetime_expressions.rs index d21d89c19d2e..430220faf815 100644 --- a/datafusion/physical-expr/src/datetime_expressions.rs +++ b/datafusion/physical-expr/src/datetime_expressions.rs @@ -17,7 +17,9 @@ //! DateTime expressions -use crate::expressions::cast_column; +use std::str::FromStr; +use std::sync::Arc; + use arrow::compute::cast; use arrow::{ array::{Array, ArrayRef, Float64Array, OffsetSizeTrait, PrimitiveArray}, @@ -33,13 +35,17 @@ use arrow::{ datatypes::TimeUnit, temporal_conversions::{as_datetime_with_timezone, timestamp_ns_to_datetime}, }; +use arrow_array::builder::PrimitiveBuilder; +use arrow_array::cast::AsArray; use arrow_array::temporal_conversions::NANOSECONDS; use arrow_array::timezone::Tz; -use arrow_array::types::ArrowTimestampType; +use arrow_array::types::{ArrowTimestampType, Date32Type, Int32Type}; use arrow_array::GenericStringArray; use chrono::prelude::*; use chrono::LocalResult::Single; use chrono::{Duration, Months, NaiveDate}; +use itertools::Either; + use datafusion_common::cast::{ as_date32_array, as_date64_array, as_generic_string_array, as_primitive_array, as_timestamp_microsecond_array, as_timestamp_millisecond_array, @@ -50,9 +56,8 @@ use datafusion_common::{ ScalarValue, }; use datafusion_expr::ColumnarValue; -use itertools::Either; -use std::str::FromStr; -use std::sync::Arc; + +use crate::expressions::cast_column; /// Error message if nanosecond conversion request beyond supported interval const ERR_NANOSECONDS_NOT_SUPPORTED: &str = "The dates that can be represented as nanoseconds have to be between 1677-09-21T00:12:44.0 and 2262-04-11T23:47:16.854775804"; @@ -497,6 +502,99 @@ pub fn make_current_time( move |_arg| Ok(ColumnarValue::Scalar(ScalarValue::Time64Nanosecond(nano))) } +/// make_date(year, month, day) SQL function implementation +pub fn make_date(args: &[ColumnarValue]) -> Result { + if args.len() != 3 { + return exec_err!( + "make_date function requires 3 arguments, got {}", + args.len() + ); + } + + // first, identify if any of the arguments is an Array. If yes, store its `len`, + // as any scalar will need to be converted to an array of len `len`. + let len = args + .iter() + .fold(Option::::None, |acc, arg| match arg { + ColumnarValue::Scalar(_) => acc, + ColumnarValue::Array(a) => Some(a.len()), + }); + + let is_scalar = len.is_none(); + let array_size = if is_scalar { 1 } else { len.unwrap() }; + + let years = cast_column(&args[0], &DataType::Int32, None)?; + let months = cast_column(&args[1], &DataType::Int32, None)?; + let days = cast_column(&args[2], &DataType::Int32, None)?; + + let value_fn = |col: &ColumnarValue, pos: usize| -> Result { + match col { + ColumnarValue::Array(a) => Ok(a.as_primitive::().value(pos)), + ColumnarValue::Scalar(s) => match s { + ScalarValue::Int32(i) => match i { + Some(i) => Ok(*i), + None => { + exec_err!("Unable to parse date from null/empty value") + } + }, + _ => unreachable!(), + }, + } + }; + + // since the epoch for the date32 datatype is the unix epoch + // we need to subtract the unix epoch from the current date + // note this can result in a negative value + let unix_days_from_ce = NaiveDate::from_ymd_opt(1970, 1, 1) + .unwrap() + .num_days_from_ce(); + + let mut builder: PrimitiveBuilder = PrimitiveArray::builder(array_size); + + for i in 0..array_size { + let y = value_fn(&years, i)?; + let m = u32::try_from(value_fn(&months, i)?); + let d = u32::try_from(value_fn(&days, i)?); + + if m.is_err() { + return exec_err!( + "Month value '{:?}' is out of range", + value_fn(&months, i).unwrap() + ); + } + if d.is_err() { + return exec_err!( + "Day value '{:?}' is out of range", + value_fn(&days, i).unwrap() + ); + } + + let date = NaiveDate::from_ymd_opt(y, m.unwrap(), d.unwrap()); + + match date { + Some(d) => builder.append_value(d.num_days_from_ce() - unix_days_from_ce), + None => { + return exec_err!( + "Unable to parse date from {y}, {}, {}", + m.unwrap(), + d.unwrap() + ) + } + }; + } + + let arr = builder.finish(); + + if is_scalar { + // If all inputs are scalar, keeps output as scalar + Ok(ColumnarValue::Scalar(ScalarValue::Date32(Some( + arr.value(0), + )))) + } else { + Ok(ColumnarValue::Array(Arc::new(arr))) + } +} + fn quarter_month(date: &T) -> u32 where T: chrono::Datelike, @@ -1426,9 +1524,10 @@ mod tests { }; use arrow_array::types::Int64Type; use arrow_array::{ - TimestampMicrosecondArray, TimestampMillisecondArray, TimestampNanosecondArray, - TimestampSecondArray, + Date32Array, Int32Array, TimestampMicrosecondArray, TimestampMillisecondArray, + TimestampNanosecondArray, TimestampSecondArray, UInt32Array, }; + use datafusion_common::assert_contains; use datafusion_expr::ScalarFunctionImplementation; @@ -2368,4 +2467,112 @@ mod tests { assert_contains!(actual, expected); } } + + #[test] + fn test_make_date() { + let res = make_date(&[ + ColumnarValue::Scalar(ScalarValue::Int32(Some(2024))), + ColumnarValue::Scalar(ScalarValue::Int64(Some(1))), + ColumnarValue::Scalar(ScalarValue::UInt32(Some(14))), + ]) + .expect("that make_date parsed values without error"); + + if let ColumnarValue::Scalar(ScalarValue::Date32(date)) = res { + assert_eq!(19736, date.unwrap()); + } else { + panic!("Expected a scalar value") + } + + let res = make_date(&[ + ColumnarValue::Scalar(ScalarValue::Int64(Some(2024))), + ColumnarValue::Scalar(ScalarValue::UInt64(Some(1))), + ColumnarValue::Scalar(ScalarValue::UInt32(Some(14))), + ]) + .expect("that make_date parsed values without error"); + + if let ColumnarValue::Scalar(ScalarValue::Date32(date)) = res { + assert_eq!(19736, date.unwrap()); + } else { + panic!("Expected a scalar value") + } + + let res = make_date(&[ + ColumnarValue::Scalar(ScalarValue::Utf8(Some("2024".to_string()))), + ColumnarValue::Scalar(ScalarValue::LargeUtf8(Some("1".to_string()))), + ColumnarValue::Scalar(ScalarValue::Utf8(Some("14".to_string()))), + ]) + .expect("that make_date parsed values without error"); + + if let ColumnarValue::Scalar(ScalarValue::Date32(date)) = res { + assert_eq!(19736, date.unwrap()); + } else { + panic!("Expected a scalar value") + } + + let years = Arc::new((2021..2025).map(Some).collect::()); + let months = Arc::new((1..5).map(Some).collect::()); + let days = Arc::new((11..15).map(Some).collect::()); + let res = make_date(&[ + ColumnarValue::Array(years), + ColumnarValue::Array(months), + ColumnarValue::Array(days), + ]) + .expect("that make_date parsed values without error"); + + if let ColumnarValue::Array(array) = res { + assert_eq!(array.len(), 4); + let mut builder = Date32Array::builder(4); + builder.append_value(18_638); + builder.append_value(19_035); + builder.append_value(19_429); + builder.append_value(19_827); + assert_eq!(&builder.finish() as &dyn Array, array.as_ref()); + } else { + panic!("Expected a columnar array") + } + + // + // Fallible test cases + // + + // invalid number of arguments + let res = make_date(&[ColumnarValue::Scalar(ScalarValue::Int32(Some(1)))]); + assert_eq!( + res.err().unwrap().strip_backtrace(), + "Execution error: make_date function requires 3 arguments, got 1" + ); + + // invalid type + let res = make_date(&[ + ColumnarValue::Scalar(ScalarValue::IntervalYearMonth(Some(1))), + ColumnarValue::Scalar(ScalarValue::TimestampNanosecond(Some(1), None)), + ColumnarValue::Scalar(ScalarValue::TimestampNanosecond(Some(1), None)), + ]); + assert_eq!( + res.err().unwrap().strip_backtrace(), + "Arrow error: Cast error: Casting from Interval(YearMonth) to Int32 not supported" + ); + + // overflow of month + let res = make_date(&[ + ColumnarValue::Scalar(ScalarValue::Int32(Some(2023))), + ColumnarValue::Scalar(ScalarValue::UInt64(Some(u64::MAX))), + ColumnarValue::Scalar(ScalarValue::Int32(Some(22))), + ]); + assert_eq!( + res.err().unwrap().strip_backtrace(), + "Arrow error: Cast error: Can't cast value 18446744073709551615 to type Int32" + ); + + // overflow of day + let res = make_date(&[ + ColumnarValue::Scalar(ScalarValue::Int32(Some(2023))), + ColumnarValue::Scalar(ScalarValue::Int32(Some(22))), + ColumnarValue::Scalar(ScalarValue::UInt32(Some(u32::MAX))), + ]); + assert_eq!( + res.err().unwrap().strip_backtrace(), + "Arrow error: Cast error: Can't cast value 4294967295 to type Int32" + ); + } } diff --git a/datafusion/physical-expr/src/functions.rs b/datafusion/physical-expr/src/functions.rs index 2bfdf499123b..0bde08759971 100644 --- a/datafusion/physical-expr/src/functions.rs +++ b/datafusion/physical-expr/src/functions.rs @@ -574,6 +574,7 @@ pub fn create_physical_fun( execution_props.query_execution_start_time, )) } + BuiltinScalarFunction::MakeDate => Arc::new(datetime_expressions::make_date), BuiltinScalarFunction::ToTimestamp => { Arc::new(datetime_expressions::to_timestamp_invoke) } diff --git a/datafusion/proto/proto/datafusion.proto b/datafusion/proto/proto/datafusion.proto index b5fd202f513e..0ac7120d242b 100644 --- a/datafusion/proto/proto/datafusion.proto +++ b/datafusion/proto/proto/datafusion.proto @@ -671,6 +671,7 @@ enum ScalarFunction { ArrayResize = 130; EndsWith = 131; InStr = 132; + MakeDate = 133; } message ScalarFunctionNode { diff --git a/datafusion/proto/src/generated/pbjson.rs b/datafusion/proto/src/generated/pbjson.rs index 599a8e5bb520..c0cc0b943ada 100644 --- a/datafusion/proto/src/generated/pbjson.rs +++ b/datafusion/proto/src/generated/pbjson.rs @@ -22423,6 +22423,7 @@ impl serde::Serialize for ScalarFunction { Self::ArrayResize => "ArrayResize", Self::EndsWith => "EndsWith", Self::InStr => "InStr", + Self::MakeDate => "MakeDate", }; serializer.serialize_str(variant) } @@ -22567,6 +22568,7 @@ impl<'de> serde::Deserialize<'de> for ScalarFunction { "ArrayResize", "EndsWith", "InStr", + "MakeDate", ]; struct GeneratedVisitor; @@ -22740,6 +22742,7 @@ impl<'de> serde::Deserialize<'de> for ScalarFunction { "ArrayResize" => Ok(ScalarFunction::ArrayResize), "EndsWith" => Ok(ScalarFunction::EndsWith), "InStr" => Ok(ScalarFunction::InStr), + "MakeDate" => Ok(ScalarFunction::MakeDate), _ => Err(serde::de::Error::unknown_variant(value, FIELDS)), } } diff --git a/datafusion/proto/src/generated/prost.rs b/datafusion/proto/src/generated/prost.rs index 83d704ac142d..e3b83748b89b 100644 --- a/datafusion/proto/src/generated/prost.rs +++ b/datafusion/proto/src/generated/prost.rs @@ -2766,6 +2766,7 @@ pub enum ScalarFunction { ArrayResize = 130, EndsWith = 131, InStr = 132, + MakeDate = 133, } impl ScalarFunction { /// String value of the enum field names used in the ProtoBuf definition. @@ -2907,6 +2908,7 @@ impl ScalarFunction { ScalarFunction::ArrayResize => "ArrayResize", ScalarFunction::EndsWith => "EndsWith", ScalarFunction::InStr => "InStr", + ScalarFunction::MakeDate => "MakeDate", } } /// Creates an enum from field names used in the ProtoBuf definition. @@ -3045,6 +3047,7 @@ impl ScalarFunction { "ArrayResize" => Some(Self::ArrayResize), "EndsWith" => Some(Self::EndsWith), "InStr" => Some(Self::InStr), + "MakeDate" => Some(Self::MakeDate), _ => None, } } diff --git a/datafusion/proto/src/logical_plan/from_proto.rs b/datafusion/proto/src/logical_plan/from_proto.rs index eb72d1f9c3e8..bbaa280d63c5 100644 --- a/datafusion/proto/src/logical_plan/from_proto.rs +++ b/datafusion/proto/src/logical_plan/from_proto.rs @@ -555,6 +555,7 @@ impl From<&protobuf::ScalarFunction> for BuiltinScalarFunction { ScalarFunction::Now => Self::Now, ScalarFunction::CurrentDate => Self::CurrentDate, ScalarFunction::CurrentTime => Self::CurrentTime, + ScalarFunction::MakeDate => Self::MakeDate, ScalarFunction::Uuid => Self::Uuid, ScalarFunction::Translate => Self::Translate, ScalarFunction::RegexpMatch => Self::RegexpMatch, @@ -1704,6 +1705,16 @@ pub fn parse_expr( parse_expr(&args[1], registry)?, )), ScalarFunction::ToHex => Ok(to_hex(parse_expr(&args[0], registry)?)), + ScalarFunction::MakeDate => { + let args: Vec<_> = args + .iter() + .map(|expr| parse_expr(expr, registry)) + .collect::>()?; + Ok(Expr::ScalarFunction(expr::ScalarFunction::new( + BuiltinScalarFunction::MakeDate, + args, + ))) + } ScalarFunction::ToTimestamp => { let args: Vec<_> = args .iter() diff --git a/datafusion/proto/src/logical_plan/to_proto.rs b/datafusion/proto/src/logical_plan/to_proto.rs index e1fc3f0c8525..e7b9474a2d23 100644 --- a/datafusion/proto/src/logical_plan/to_proto.rs +++ b/datafusion/proto/src/logical_plan/to_proto.rs @@ -1553,6 +1553,7 @@ impl TryFrom<&BuiltinScalarFunction> for protobuf::ScalarFunction { BuiltinScalarFunction::Now => Self::Now, BuiltinScalarFunction::CurrentDate => Self::CurrentDate, BuiltinScalarFunction::CurrentTime => Self::CurrentTime, + BuiltinScalarFunction::MakeDate => Self::MakeDate, BuiltinScalarFunction::Translate => Self::Translate, BuiltinScalarFunction::RegexpMatch => Self::RegexpMatch, BuiltinScalarFunction::Coalesce => Self::Coalesce, diff --git a/datafusion/sqllogictest/test_files/timestamps.slt b/datafusion/sqllogictest/test_files/timestamps.slt index 5c7687aa27b2..8f565453a12f 100644 --- a/datafusion/sqllogictest/test_files/timestamps.slt +++ b/datafusion/sqllogictest/test_files/timestamps.slt @@ -2233,4 +2233,168 @@ 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 +########## +## make date tests +########## + +query D +select make_date(2024, 1, 27); +---- +2024-01-27 + +query D +select make_date(42, 1, 27); +---- +0042-01-27 + +query D +select make_date(99, 1, 27); +---- +0099-01-27 + +query D +select make_date(2024, 2, 29); +---- +2024-02-29 + +query D +select make_date(10001, 1, 27); +---- ++10001-01-27 + +query D +select make_date('2024', '01', '27'); +---- +2024-01-27 + +query D +select make_date(12 + 2012, '01', '27'); +---- +2024-01-27 + +query D +select make_date(2024::bigint, 01::bigint, 27::bigint); +---- +2024-01-27 + +query D +select make_date(arrow_cast(2024, 'Int64'), arrow_cast(1, 'Int64'), arrow_cast(27, 'Int64')); +---- +2024-01-27 + +query D +select make_date(arrow_cast(2024, 'Int32'), arrow_cast(1, 'Int32'), arrow_cast(27, 'Int32')); +---- +2024-01-27 + +statement ok +create table table_nums (year int, month int, day int) as values + (2024, 1, 23), + (2023, 11, 30); + +query D +select make_date(t.year, t.month, t.day) from table_nums t; +---- +2024-01-23 +2023-11-30 + +query D +select make_date(2021, t.month, t.day) from table_nums t; +---- +2021-01-23 +2021-11-30 + +query D +select make_date(t.year, 3, t.day) from table_nums t; +---- +2024-03-23 +2023-03-30 + +query D +select make_date(t.year, t.month, 4) from table_nums t; +---- +2024-01-04 +2023-11-04 + +query D +select make_date('2021', t.month, t.day) from table_nums t; +---- +2021-01-23 +2021-11-30 + +query D +select make_date(t.year, '3', t.day) from table_nums t; +---- +2024-03-23 +2023-03-30 + +query D +select make_date(t.year, t.month, '4') from table_nums t; +---- +2024-01-04 +2023-11-04 + +statement ok +insert into table_nums values (2024, null, 23); + +query error DataFusion error: Execution error: Unable to parse date from 2024, 0, 23 +select make_date(t.year, t.month, t.day) from table_nums t; + +statement ok +drop table table_nums; + +statement ok +create table table_strings (year varchar(4), month varchar(2), day varchar(2)) as values + ('2024', '1', '23'), + ('2023', '11', '30'); + +query D +select make_date(t.year, t.month, t.day) from table_strings t; +---- +2024-01-23 +2023-11-30 + +statement ok +insert into table_strings values (2024, null, 23); + +query error DataFusion error: Execution error: Unable to parse date from 2024, 0, 23 +select make_date(t.year, t.month, t.day) from table_strings t; + +statement ok +drop table table_strings; + +query error DataFusion error: Execution error: Unable to parse date from 2024, 13, 23 +select make_date(2024, 13, 23); + +query error DataFusion error: Execution error: Unable to parse date from 2024, 1, 32 +select make_date(2024, 01, 32); + +query error DataFusion error: Execution error: Unable to parse date from 2024, 0, 23 +select make_date(2024, 0, 23); + +query error DataFusion error: Execution error: Month value '\-1' is out of range +select make_date(2024, -1, 23); + +query error DataFusion error: Execution error: Unable to parse date from 2024, 12, 0 +select make_date(2024, 12, 0); + +query error DataFusion error: Execution error: Day value '\-1' is out of range +select make_date(2024, 13, -1); + +query error DataFusion error: Execution error: Unable to parse date from null/empty value +select make_date(null, 1, 23); + +query error DataFusion error: Arrow error: Cast error: Cannot cast string '' to value of Int32 type +select make_date('', 1, 23); + +query error DataFusion error: Execution error: Unable to parse date from null/empty value +select make_date(2024, null, 23); + +query error DataFusion error: Arrow error: Cast error: Cannot cast string '' to value of Int32 type +select make_date(2024, '', 27); + +query error DataFusion error: Execution error: Unable to parse date from null/empty value +select make_date(2024, 1, null); +query error DataFusion error: Arrow error: Cast error: Cannot cast string '' to value of Int32 type +select make_date(2024, 1, ''); diff --git a/docs/source/user-guide/sql/scalar_functions.md b/docs/source/user-guide/sql/scalar_functions.md index 04044560f793..7bec80b55e26 100644 --- a/docs/source/user-guide/sql/scalar_functions.md +++ b/docs/source/user-guide/sql/scalar_functions.md @@ -1312,6 +1312,7 @@ regexp_replace(str, regexp, replacement, flags) - [datepart](#datepart) - [extract](#extract) - [today](#today) +- [make_date](#make_date) - [to_timestamp](#to_timestamp) - [to_timestamp_millis](#to_timestamp_millis) - [to_timestamp_micros](#to_timestamp_micros) @@ -1500,6 +1501,44 @@ extract(field FROM source) - **source**: Source time expression to operate on. Can be a constant, column, or function. +### `make_date` + +Make a date from year/month/day component parts. + +``` +make_date(year, month, day) +``` + +#### Arguments + +- **year**: Year to use when making the date. + Can be a constant, column or function, and any combination of arithmetic operators. +- **month**: Month to use when making the date. + Can be a constant, column or function, and any combination of arithmetic operators. +- **day**: Day to use when making the date. + Can be a constant, column or function, and any combination of arithmetic operators. + +#### Example + +``` +❯ select make_date(2023, 1, 31); ++-------------------------------------------+ +| make_date(Int64(2023),Int64(1),Int64(31)) | ++-------------------------------------------+ +| 2023-01-31 | ++-------------------------------------------+ +❯ select make_date('2023', '01', '31'); ++-----------------------------------------------+ +| make_date(Utf8("2023"),Utf8("01"),Utf8("31")) | ++-----------------------------------------------+ +| 2023-01-31 | ++-----------------------------------------------+ +``` + +Additional examples can be found [here] + +[here]: https://github.com/apache/arrow-datafusion/blob/main/datafusion-examples/examples/make_date.rs + ### `to_timestamp` Converts a value to a timestamp (`YYYY-MM-DDT00:00:00Z`).