From 196b71823c417615dd90350f4e77d8acc5b3ed60 Mon Sep 17 00:00:00 2001 From: Bruce Ritchie Date: Tue, 13 Feb 2024 09:40:16 -0500 Subject: [PATCH] Add `to_char` function implementation using chrono formats (#9181) * initial to_char impl #9147 * fleshed out to_char impl, added tests and docs. * adding sqllogictests * adding time support, improved tests in datatime_expressions.rs, fixed scalar doc example * Doc fix. * Prettier * Update datafusion/physical-expr/src/datetime_expressions.rs Co-authored-by: Andrew Lamb * Updating the chrono dependency since Chrono 0.4.34 changed Display for the Duration type (which is now an alias for TimeDelta) and this broke the timestamp tests. * Updates and fixes based on PR feedback. * Add escape to rustdoc. * Fixed example * Handled null formats better - now any null formats will result in a default format being used. --------- Co-authored-by: Andrew Lamb --- Cargo.toml | 2 +- datafusion-cli/Cargo.lock | 4 +- datafusion-examples/README.md | 1 + datafusion-examples/examples/to_char.rs | 197 +++++++ datafusion/execution/Cargo.toml | 2 +- datafusion/expr/src/built_in_function.rs | 40 ++ datafusion/expr/src/expr_fn.rs | 6 + datafusion/physical-expr/Cargo.toml | 4 + datafusion/physical-expr/benches/to_char.rs | 133 +++++ .../physical-expr/src/datetime_expressions.rs | 525 +++++++++++++++++- datafusion/physical-expr/src/functions.rs | 1 + datafusion/physical-plan/Cargo.toml | 2 +- datafusion/proto/proto/datafusion.proto | 1 + datafusion/proto/src/generated/pbjson.rs | 3 + datafusion/proto/src/generated/prost.rs | 3 + .../proto/src/logical_plan/from_proto.rs | 11 + datafusion/proto/src/logical_plan/to_proto.rs | 1 + .../sqllogictest/test_files/timestamps.slt | 113 ++++ .../source/user-guide/sql/scalar_functions.md | 37 ++ 19 files changed, 1078 insertions(+), 8 deletions(-) create mode 100644 datafusion-examples/examples/to_char.rs create mode 100644 datafusion/physical-expr/benches/to_char.rs diff --git a/Cargo.toml b/Cargo.toml index 6046f820c7bf..2e2a0103973d 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -43,7 +43,7 @@ arrow-string = { version = "50.0.0", default-features = false } async-trait = "0.1.73" bigdecimal = "0.4.1" bytes = "1.4" -chrono = { version = "0.4.31", default-features = false } +chrono = { version = "0.4.34", default-features = false } ctor = "0.2.0" dashmap = "5.4.0" datafusion = { path = "datafusion/core", version = "35.0.0" } diff --git a/datafusion-cli/Cargo.lock b/datafusion-cli/Cargo.lock index 5ff41ed61d11..25bb30e5bc56 100644 --- a/datafusion-cli/Cargo.lock +++ b/datafusion-cli/Cargo.lock @@ -867,9 +867,9 @@ checksum = "baf1de4339761588bc0619e3cbc0120ee582ebb74b53b4efbf79117bd2da40fd" [[package]] name = "chrono" -version = "0.4.33" +version = "0.4.34" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9f13690e35a5e4ace198e7beea2895d29f3a9cc55015fcebe6336bd2010af9eb" +checksum = "5bc015644b92d5890fab7489e49d21f879d5c990186827d42ec511919404f38b" dependencies = [ "android-tzdata", "iana-time-zone", diff --git a/datafusion-examples/README.md b/datafusion-examples/README.md index 9646cee45e7a..5cb7cec503a6 100644 --- a/datafusion-examples/README.md +++ b/datafusion-examples/README.md @@ -63,6 +63,7 @@ cargo run --example csv_sql - [`query-http-csv.rs`](examples/query-http-csv.rs): Configure `object_store` and run a query against files vi HTTP - [`regexp.rs`](examples/regexp.rs): Examples of using regular expression functions - [`rewrite_expr.rs`](examples/rewrite_expr.rs): Define and invoke a custom Query Optimizer pass +- [`to_char.rs`](examples/to_char.rs): Examples of using the to_char function - [`to_timestamp.rs`](examples/to_timestamp.rs): Examples of using to_timestamp functions - [`simple_udf.rs`](examples/simple_udf.rs): Define and invoke a User Defined Scalar Function (UDF) - [`advanced_udf.rs`](examples/advanced_udf.rs): Define and invoke a more complicated User Defined Scalar Function (UDF) diff --git a/datafusion-examples/examples/to_char.rs b/datafusion-examples/examples/to_char.rs new file mode 100644 index 000000000000..e99f69fbcd55 --- /dev/null +++ b/datafusion-examples/examples/to_char.rs @@ -0,0 +1,197 @@ +// 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 arrow::array::Date32Array; +use datafusion::arrow::array::StringArray; +use datafusion::arrow::datatypes::{DataType, Field, Schema}; +use datafusion::arrow::record_batch::RecordBatch; +use datafusion::assert_batches_eq; +use datafusion::error::Result; +use datafusion::prelude::*; +use std::sync::Arc; + +/// This example demonstrates how to use the to_char function via sql +/// +/// This function accepts date, time, timestamp and duration values +/// in the first argument and string values for the second +#[tokio::main] +async fn main() -> Result<()> { + let schema = Arc::new(Schema::new(vec![ + Field::new("values", DataType::Date32, false), + Field::new("patterns", DataType::Utf8, false), + ])); + + let batch = RecordBatch::try_new( + schema, + vec![ + Arc::new(Date32Array::from(vec![18506, 18507, 18508, 18509])), + Arc::new(StringArray::from(vec![ + "%Y-%m-%d", "%Y:%m:%d", "%Y%m%d", "%d-%m-%Y", + ])), + ], + )?; + + // 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 _ = ctx.table("t").await?; + + // use to_char function to convert col 'values' to timestamp type using + // patterns stored in col 'patterns' + let result = ctx + .sql("SELECT to_char(values, patterns) from t") + .await? + .collect() + .await?; + + assert_batches_eq!( + &[ + "+------------------------------+", + "| to_char(t.values,t.patterns) |", + "+------------------------------+", + "| 2020-09-01 |", + "| 2020:09:02 |", + "| 20200903 |", + "| 04-09-2020 |", + "+------------------------------+", + ], + &result + ); + + // the date_format alias for the to_char function can be used as well + let result = ctx + .sql("SELECT date_format(values, patterns) from t") + .await? + .collect() + .await?; + + assert_batches_eq!( + &[ + "+------------------------------+", + "| to_char(t.values,t.patterns) |", + "+------------------------------+", + "| 2020-09-01 |", + "| 2020:09:02 |", + "| 20200903 |", + "| 04-09-2020 |", + "+------------------------------+", + ], + &result + ); + + // use to_char function to convert col 'values' with a fixed format + let result = ctx + .sql("SELECT to_char(values, '%m-%d-%Y') FROM t") + .await? + .collect() + .await?; + + assert_batches_eq!( + &[ + "+------------------------------------+", + "| to_char(t.values,Utf8(\"%m-%d-%Y\")) |", + "+------------------------------------+", + "| 09-01-2020 |", + "| 09-02-2020 |", + "| 09-03-2020 |", + "| 09-04-2020 |", + "+------------------------------------+", + ], + &result + ); + + // if you want to just use the default format cast to a string + let result = ctx + .sql("SELECT arrow_cast(values, 'Utf8') from t") + .await? + .collect() + .await?; + + assert_batches_eq!( + &[ + "+------------+", + "| t.values |", + "+------------+", + "| 2020-09-01 |", + "| 2020-09-02 |", + "| 2020-09-03 |", + "| 2020-09-04 |", + "+------------+", + ], + &result + ); + + // use can use literals as well (note the use of timestamp here) + let result = ctx + .sql("SELECT to_char(arrow_cast(TIMESTAMP '2023-08-03 14:38:50Z', 'Timestamp(Second, None)'), '%d-%m-%Y %H:%M:%S')") + .await? + .collect() + .await?; + + assert_batches_eq!( + &[ + "+-----------------------------------------------------------------+", + "| to_char(Utf8(\"2023-08-03 14:38:50Z\"),Utf8(\"%d-%m-%Y %H:%M:%S\")) |", + "+-----------------------------------------------------------------+", + "| 03-08-2023 14:38:50 |", + "+-----------------------------------------------------------------+", + ], + &result + ); + + // durations are supported though the output format is limited to two formats + // 'pretty' and 'ISO8601' + let result = ctx + .sql("SELECT to_char(arrow_cast(123456, 'Duration(Second)'), 'pretty')") + .await? + .collect() + .await?; + + assert_batches_eq!( + &[ + "+---------------------------------------+", + "| to_char(Int64(123456),Utf8(\"pretty\")) |", + "+---------------------------------------+", + "| 1 days 10 hours 17 mins 36 secs |", + "+---------------------------------------+", + ], + &result + ); + + // durations are supported though the output format is limited to two formats + // 'pretty' and 'ISO8601' + let result = ctx + .sql("SELECT to_char(arrow_cast(123456, 'Duration(Second)'), 'iso8601')") + .await? + .collect() + .await?; + + assert_batches_eq!( + &[ + "+----------------------------------------+", + "| to_char(Int64(123456),Utf8(\"iso8601\")) |", + "+----------------------------------------+", + "| PT123456S |", + "+----------------------------------------+", + ], + &result + ); + + Ok(()) +} diff --git a/datafusion/execution/Cargo.toml b/datafusion/execution/Cargo.toml index e9bb87e9f8ac..5c3360e681d1 100644 --- a/datafusion/execution/Cargo.toml +++ b/datafusion/execution/Cargo.toml @@ -34,7 +34,7 @@ path = "src/lib.rs" [dependencies] arrow = { workspace = true } -chrono = { version = "0.4", default-features = false } +chrono = { workspace = true } dashmap = { workspace = true } datafusion-common = { workspace = true } datafusion-expr = { workspace = true } diff --git a/datafusion/expr/src/built_in_function.rs b/datafusion/expr/src/built_in_function.rs index 274a6fa9c2dc..e4a9471f07a8 100644 --- a/datafusion/expr/src/built_in_function.rs +++ b/datafusion/expr/src/built_in_function.rs @@ -315,6 +315,8 @@ pub enum BuiltinScalarFunction { SubstrIndex, /// find_in_set FindInSet, + /// to_char + ToChar, } /// Maps the sql function name to `BuiltinScalarFunction` @@ -479,6 +481,7 @@ impl BuiltinScalarFunction { BuiltinScalarFunction::Strpos => Volatility::Immutable, BuiltinScalarFunction::Substr => Volatility::Immutable, BuiltinScalarFunction::ToHex => Volatility::Immutable, + BuiltinScalarFunction::ToChar => Volatility::Immutable, BuiltinScalarFunction::ToTimestamp => Volatility::Immutable, BuiltinScalarFunction::ToTimestampMillis => Volatility::Immutable, BuiltinScalarFunction::ToTimestampMicros => Volatility::Immutable, @@ -799,6 +802,7 @@ impl BuiltinScalarFunction { BuiltinScalarFunction::FindInSet => { utf8_to_int_type(&input_expr_types[0], "find_in_set") } + BuiltinScalarFunction::ToChar => Ok(Utf8), BuiltinScalarFunction::ToTimestamp | BuiltinScalarFunction::ToTimestampNanos => Ok(Timestamp(Nanosecond, None)), BuiltinScalarFunction::ToTimestampMillis => Ok(Timestamp(Millisecond, None)), @@ -1059,6 +1063,41 @@ impl BuiltinScalarFunction { vec![Exact(vec![Utf8, Int64]), Exact(vec![LargeUtf8, Int64])], self.volatility(), ), + BuiltinScalarFunction::ToChar => Signature::one_of( + vec![ + Exact(vec![Date32, Utf8]), + Exact(vec![Date64, Utf8]), + Exact(vec![Time32(Millisecond), Utf8]), + Exact(vec![Time32(Second), Utf8]), + Exact(vec![Time64(Microsecond), Utf8]), + Exact(vec![Time64(Nanosecond), Utf8]), + Exact(vec![Timestamp(Second, None), Utf8]), + Exact(vec![ + Timestamp(Second, Some(TIMEZONE_WILDCARD.into())), + Utf8, + ]), + Exact(vec![Timestamp(Millisecond, None), Utf8]), + Exact(vec![ + Timestamp(Millisecond, Some(TIMEZONE_WILDCARD.into())), + Utf8, + ]), + Exact(vec![Timestamp(Microsecond, None), Utf8]), + Exact(vec![ + Timestamp(Microsecond, Some(TIMEZONE_WILDCARD.into())), + Utf8, + ]), + Exact(vec![Timestamp(Nanosecond, None), Utf8]), + Exact(vec![ + Timestamp(Nanosecond, Some(TIMEZONE_WILDCARD.into())), + Utf8, + ]), + Exact(vec![Duration(Second), Utf8]), + Exact(vec![Duration(Millisecond), Utf8]), + Exact(vec![Duration(Microsecond), Utf8]), + Exact(vec![Duration(Nanosecond), Utf8]), + ], + self.volatility(), + ), BuiltinScalarFunction::ToTimestamp | BuiltinScalarFunction::ToTimestampSeconds | BuiltinScalarFunction::ToTimestampMillis @@ -1517,6 +1556,7 @@ impl BuiltinScalarFunction { BuiltinScalarFunction::DateBin => &["date_bin"], BuiltinScalarFunction::DateTrunc => &["date_trunc", "datetrunc"], BuiltinScalarFunction::DatePart => &["date_part", "datepart"], + BuiltinScalarFunction::ToChar => &["to_char", "date_format"], BuiltinScalarFunction::ToTimestamp => &["to_timestamp"], BuiltinScalarFunction::ToTimestampMillis => &["to_timestamp_millis"], BuiltinScalarFunction::ToTimestampMicros => &["to_timestamp_micros"], diff --git a/datafusion/expr/src/expr_fn.rs b/datafusion/expr/src/expr_fn.rs index 9c20763c89dd..ad0eae898534 100644 --- a/datafusion/expr/src/expr_fn.rs +++ b/datafusion/expr/src/expr_fn.rs @@ -890,6 +890,12 @@ nary_scalar_expr!( scalar_expr!(DatePart, date_part, part date, "extracts a subfield from the date"); scalar_expr!(DateTrunc, date_trunc, part date, "truncates the date to a specified level of precision"); scalar_expr!(DateBin, date_bin, stride source origin, "coerces an arbitrary timestamp to the start of the nearest specified interval"); +scalar_expr!( + ToChar, + to_char, + datetime format, + "converts a date, time, timestamp or duration to a string based on the provided format" +); nary_scalar_expr!( ToTimestamp, to_timestamp, diff --git a/datafusion/physical-expr/Cargo.toml b/datafusion/physical-expr/Cargo.toml index e040e71233d5..9aa08fe780ec 100644 --- a/datafusion/physical-expr/Cargo.toml +++ b/datafusion/physical-expr/Cargo.toml @@ -94,3 +94,7 @@ name = "to_timestamp" [[bench]] harness = false name = "regexp" + +[[bench]] +harness = false +name = "to_char" diff --git a/datafusion/physical-expr/benches/to_char.rs b/datafusion/physical-expr/benches/to_char.rs new file mode 100644 index 000000000000..3d08a02bc231 --- /dev/null +++ b/datafusion/physical-expr/benches/to_char.rs @@ -0,0 +1,133 @@ +// 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. + +extern crate criterion; + +use std::sync::Arc; + +use arrow_array::{ArrayRef, Date32Array, StringArray}; +use chrono::prelude::*; +use chrono::Duration; +use criterion::{black_box, criterion_group, criterion_main, Criterion}; +use rand::rngs::ThreadRng; +use rand::seq::SliceRandom; +use rand::Rng; + +use datafusion_common::ScalarValue; +use datafusion_common::ScalarValue::TimestampNanosecond; +use datafusion_expr::ColumnarValue; +use datafusion_physical_expr::datetime_expressions::to_char; + +fn random_date_in_range( + rng: &mut ThreadRng, + start_date: NaiveDate, + end_date: NaiveDate, +) -> NaiveDate { + let days_in_range = (end_date - start_date).num_days(); + let random_days: i64 = rng.gen_range(0..days_in_range); + start_date + Duration::days(random_days) +} + +fn data(rng: &mut ThreadRng) -> Date32Array { + let mut data: Vec = vec![]; + let unix_days_from_ce = NaiveDate::from_ymd_opt(1970, 1, 1) + .unwrap() + .num_days_from_ce(); + let start_date = "1970-01-01" + .parse::() + .expect("Date should parse"); + let end_date = "2050-12-31" + .parse::() + .expect("Date should parse"); + for _ in 0..1000 { + data.push( + random_date_in_range(rng, start_date, end_date).num_days_from_ce() + - unix_days_from_ce, + ); + } + + Date32Array::from(data) +} + +fn patterns(rng: &mut ThreadRng) -> StringArray { + let samples = vec![ + "%Y:%m:%d".to_string(), + "%d-%m-%Y".to_string(), + "%d%m%Y".to_string(), + "%Y%m%d".to_string(), + "%Y...%m...%d".to_string(), + ]; + let mut data: Vec = vec![]; + for _ in 0..1000 { + data.push(samples.choose(rng).unwrap().to_string()); + } + + StringArray::from(data) +} + +fn criterion_benchmark(c: &mut Criterion) { + c.bench_function("to_char_array_array_1000", |b| { + let mut rng = rand::thread_rng(); + let data = ColumnarValue::Array(Arc::new(data(&mut rng)) as ArrayRef); + let patterns = ColumnarValue::Array(Arc::new(patterns(&mut rng)) as ArrayRef); + + b.iter(|| { + black_box( + to_char(&[data.clone(), patterns.clone()]) + .expect("to_char should work on valid values"), + ) + }) + }); + + c.bench_function("to_char_array_scalar_1000", |b| { + let mut rng = rand::thread_rng(); + let data = ColumnarValue::Array(Arc::new(data(&mut rng)) as ArrayRef); + let patterns = + ColumnarValue::Scalar(ScalarValue::Utf8(Some("%Y-%m-%d".to_string()))); + + b.iter(|| { + black_box( + to_char(&[data.clone(), patterns.clone()]) + .expect("to_char should work on valid values"), + ) + }) + }); + + c.bench_function("to_char_scalar_scalar_1000", |b| { + let timestamp = "2026-07-08T09:10:11" + .parse::() + .unwrap() + .with_nanosecond(56789) + .unwrap() + .timestamp_nanos_opt() + .unwrap(); + let data = ColumnarValue::Scalar(TimestampNanosecond(Some(timestamp), None)); + let pattern = ColumnarValue::Scalar(ScalarValue::Utf8(Some( + "%d-%m-%Y %H:%M:%S".to_string(), + ))); + + b.iter(|| { + black_box( + to_char(&[data.clone(), pattern.clone()]) + .expect("to_char should work on valid values"), + ) + }) + }); +} + +criterion_group!(benches, criterion_benchmark); +criterion_main!(benches); diff --git a/datafusion/physical-expr/src/datetime_expressions.rs b/datafusion/physical-expr/src/datetime_expressions.rs index 500edef5cf07..e125c6447154 100644 --- a/datafusion/physical-expr/src/datetime_expressions.rs +++ b/datafusion/physical-expr/src/datetime_expressions.rs @@ -22,6 +22,7 @@ use std::str::FromStr; use std::sync::Arc; use arrow::compute::cast; +use arrow::util::display::{ArrayFormatter, DurationFormat, FormatOptions}; use arrow::{ array::{Array, ArrayRef, Float64Array, OffsetSizeTrait, PrimitiveArray}, compute::kernels::cast_utils::string_to_timestamp_nanos, @@ -41,7 +42,7 @@ use arrow_array::cast::AsArray; use arrow_array::temporal_conversions::NANOSECONDS; use arrow_array::timezone::Tz; use arrow_array::types::{ArrowTimestampType, Date32Type, Int32Type}; -use arrow_array::GenericStringArray; +use arrow_array::{GenericStringArray, StringArray}; use chrono::prelude::*; use chrono::LocalResult::Single; use chrono::{Duration, LocalResult, Months, NaiveDate}; @@ -502,6 +503,209 @@ pub fn make_current_time( move |_arg| Ok(ColumnarValue::Scalar(ScalarValue::Time64Nanosecond(nano))) } +/// Returns a string representation of a date, time, timestamp or duration based +/// on a Chrono pattern. +/// +/// The syntax for the patterns can be found at +/// +/// +/// # Examples +/// +/// ``` +/// # use chrono::prelude::*; +/// # use datafusion::prelude::*; +/// # use datafusion::error::Result; +/// # use datafusion_common::ScalarValue::TimestampNanosecond; +/// # use std::sync::Arc; +/// # use arrow_array::{Date32Array, RecordBatch, StringArray}; +/// # use arrow_schema::{DataType, Field, Schema}; +/// # #[tokio::main] +/// # async fn main() -> Result<()> { +/// let schema = Arc::new(Schema::new(vec![ +/// Field::new("values", DataType::Date32, false), +/// Field::new("patterns", DataType::Utf8, false), +/// ])); +/// +/// let batch = RecordBatch::try_new( +/// schema, +/// vec![ +/// Arc::new(Date32Array::from(vec![ +/// 18506, +/// 18507, +/// 18508, +/// 18509, +/// ])), +/// Arc::new(StringArray::from(vec![ +/// "%Y-%m-%d", +/// "%Y:%m:%d", +/// "%Y%m%d", +/// "%d-%m-%Y", +/// ])), +/// ], +/// )?; +/// +/// let ctx = SessionContext::new(); +/// ctx.register_batch("t", batch)?; +/// let df = ctx.table("t").await?; +/// +/// // use the to_char function to convert col 'values', +/// // to strings using patterns in col 'patterns' +/// let df = df.with_column( +/// "date_str", +/// to_char(col("values"), col("patterns")) +/// )?; +/// // Note that providing a scalar value for the pattern +/// // is more performant +/// let df = df.with_column( +/// "date_str2", +/// to_char(col("values"), lit("%d-%m-%Y")) +/// )?; +/// // literals can be used as well with dataframe calls +/// let timestamp = "2026-07-08T09:10:11" +/// .parse::() +/// .unwrap() +/// .with_nanosecond(56789) +/// .unwrap() +/// .timestamp_nanos_opt() +/// .unwrap(); +/// let df = df.with_column( +/// "timestamp_str", +/// to_char(lit(TimestampNanosecond(Some(timestamp), None)), lit("%d-%m-%Y %H:%M:%S")) +/// )?; +/// +/// df.show().await?; +/// +/// # Ok(()) +/// # } +/// ``` +pub fn to_char(args: &[ColumnarValue]) -> Result { + if args.len() != 2 { + return exec_err!("to_char function requires 2 arguments, got {}", args.len()); + } + + match &args[1] { + // null format, use default formats + ColumnarValue::Scalar(ScalarValue::Utf8(None)) + | ColumnarValue::Scalar(ScalarValue::Null) => { + _to_char_scalar(args[0].clone(), None) + } + // constant format + ColumnarValue::Scalar(ScalarValue::Utf8(Some(format))) => { + // invoke to_char_scalar with the known string, without converting to array + _to_char_scalar(args[0].clone(), Some(format)) + } + ColumnarValue::Array(_) => _to_char_array(args), + _ => { + exec_err!( + "Format for `to_char` must be non-null Utf8, received {:?}", + args[1].data_type() + ) + } + } +} + +fn _build_format_options<'a>( + data_type: &DataType, + format: Option<&'a str>, +) -> Result, Result> { + let Some(format) = format else { + return Ok(FormatOptions::new()); + }; + let format_options = match data_type { + DataType::Date32 => FormatOptions::new().with_date_format(Some(format)), + DataType::Date64 => FormatOptions::new().with_datetime_format(Some(format)), + DataType::Time32(_) => FormatOptions::new().with_time_format(Some(format)), + DataType::Time64(_) => FormatOptions::new().with_time_format(Some(format)), + DataType::Timestamp(_, _) => FormatOptions::new() + .with_timestamp_format(Some(format)) + .with_timestamp_tz_format(Some(format)), + DataType::Duration(_) => FormatOptions::new().with_duration_format( + if "ISO8601".eq_ignore_ascii_case(format) { + DurationFormat::ISO8601 + } else { + DurationFormat::Pretty + }, + ), + other => { + return Err(exec_err!( + "to_char only supports date, time, timestamp and duration data types, received {other:?}" + )); + } + }; + Ok(format_options) +} + +/// Special version when arg\[1] is a scalar +fn _to_char_scalar( + expression: ColumnarValue, + format: Option<&str>, +) -> Result { + // it's possible that the expression is a scalar however because + // of the implementation in arrow-rs we need to convert it to an array + let data_type = &expression.data_type(); + let is_scalar_expression = matches!(&expression, ColumnarValue::Scalar(_)); + let array = expression.into_array(1)?; + let format_options = match _build_format_options(data_type, format) { + Ok(value) => value, + Err(value) => return value, + }; + + let formatter = ArrayFormatter::try_new(array.as_ref(), &format_options)?; + let formatted: Result, arrow_schema::ArrowError> = (0..array.len()) + .map(|i| formatter.value(i).try_to_string()) + .collect(); + + if let Ok(formatted) = formatted { + if is_scalar_expression { + Ok(ColumnarValue::Scalar(ScalarValue::Utf8(Some( + formatted.first().unwrap().to_string(), + )))) + } else { + Ok(ColumnarValue::Array( + Arc::new(StringArray::from(formatted)) as ArrayRef + )) + } + } else { + exec_err!("{}", formatted.unwrap_err()) + } +} + +fn _to_char_array(args: &[ColumnarValue]) -> Result { + let arrays = ColumnarValue::values_to_arrays(args)?; + let mut results: Vec = vec![]; + let format_array = arrays[1].as_string::(); + let data_type = arrays[0].data_type(); + + for idx in 0..arrays[0].len() { + let format = if format_array.is_null(idx) { + None + } else { + Some(format_array.value(idx)) + }; + let format_options = match _build_format_options(data_type, format) { + Ok(value) => value, + Err(value) => return value, + }; + // this isn't ideal but this can't use ValueFormatter as it isn't independent + // from ArrayFormatter + let formatter = ArrayFormatter::try_new(arrays[0].as_ref(), &format_options)?; + let result = formatter.value(idx).try_to_string(); + match result { + Ok(value) => results.push(value), + Err(e) => return exec_err!("{}", e), + } + } + + match args[0] { + ColumnarValue::Array(_) => Ok(ColumnarValue::Array(Arc::new(StringArray::from( + results, + )) as ArrayRef)), + ColumnarValue::Scalar(_) => Ok(ColumnarValue::Scalar(ScalarValue::Utf8(Some( + results.first().unwrap().to_string(), + )))), + } +} + /// make_date(year, month, day) SQL function implementation pub fn make_date(args: &[ColumnarValue]) -> Result { if args.len() != 3 { @@ -1574,8 +1778,10 @@ mod tests { }; use arrow_array::types::Int64Type; use arrow_array::{ - Date32Array, Int32Array, TimestampMicrosecondArray, TimestampMillisecondArray, - TimestampNanosecondArray, TimestampSecondArray, UInt32Array, + Date32Array, Date64Array, Int32Array, Time32MillisecondArray, Time32SecondArray, + Time64MicrosecondArray, Time64NanosecondArray, TimestampMicrosecondArray, + TimestampMillisecondArray, TimestampNanosecondArray, TimestampSecondArray, + UInt32Array, }; use datafusion_common::assert_contains; @@ -2820,4 +3026,317 @@ mod tests { "Arrow error: Cast error: Can't cast value 4294967295 to type Int32" ); } + + #[test] + fn test_to_char() { + let date = "2020-01-02T03:04:05" + .parse::() + .unwrap() + .with_nanosecond(12345) + .unwrap(); + let date2 = "2026-07-08T09:10:11" + .parse::() + .unwrap() + .with_nanosecond(56789) + .unwrap(); + + let scalar_data = vec![ + ( + ScalarValue::Date32(Some(18506)), + ScalarValue::Utf8(Some("%Y::%m::%d".to_string())), + "2020::09::01".to_string(), + ), + ( + ScalarValue::Date64(Some(date.timestamp_millis())), + ScalarValue::Utf8(Some("%Y::%m::%d".to_string())), + "2020::01::02".to_string(), + ), + ( + ScalarValue::Time32Second(Some(31851)), + ScalarValue::Utf8(Some("%H-%M-%S".to_string())), + "08-50-51".to_string(), + ), + ( + ScalarValue::Time32Millisecond(Some(18506000)), + ScalarValue::Utf8(Some("%H-%M-%S".to_string())), + "05-08-26".to_string(), + ), + ( + ScalarValue::Time64Microsecond(Some(12344567000)), + ScalarValue::Utf8(Some("%H-%M-%S %f".to_string())), + "03-25-44 567000000".to_string(), + ), + ( + ScalarValue::Time64Nanosecond(Some(12344567890000)), + ScalarValue::Utf8(Some("%H-%M-%S %f".to_string())), + "03-25-44 567890000".to_string(), + ), + ( + ScalarValue::TimestampSecond(Some(date.timestamp()), None), + ScalarValue::Utf8(Some("%Y::%m::%d %S::%M::%H".to_string())), + "2020::01::02 05::04::03".to_string(), + ), + ( + ScalarValue::TimestampMillisecond(Some(date.timestamp_millis()), None), + ScalarValue::Utf8(Some("%Y::%m::%d %S::%M::%H".to_string())), + "2020::01::02 05::04::03".to_string(), + ), + ( + ScalarValue::TimestampMicrosecond(Some(date.timestamp_micros()), None), + ScalarValue::Utf8(Some("%Y::%m::%d %S::%M::%H %f".to_string())), + "2020::01::02 05::04::03 000012000".to_string(), + ), + ( + ScalarValue::TimestampNanosecond( + Some(date.timestamp_nanos_opt().unwrap()), + None, + ), + ScalarValue::Utf8(Some("%Y::%m::%d %S::%M::%H %f".to_string())), + "2020::01::02 05::04::03 000012345".to_string(), + ), + ]; + + for (value, format, expected) in scalar_data { + let result = + to_char(&[ColumnarValue::Scalar(value), ColumnarValue::Scalar(format)]) + .expect("that to_char parsed values without error"); + + if let ColumnarValue::Scalar(ScalarValue::Utf8(date)) = result { + assert_eq!(expected, date.unwrap()); + } else { + panic!("Expected a scalar value") + } + } + + let scalar_array_data = vec![ + ( + ScalarValue::Date32(Some(18506)), + StringArray::from(vec!["%Y::%m::%d".to_string()]), + "2020::09::01".to_string(), + ), + ( + ScalarValue::Date64(Some(date.timestamp_millis())), + StringArray::from(vec!["%Y::%m::%d".to_string()]), + "2020::01::02".to_string(), + ), + ( + ScalarValue::Time32Second(Some(31851)), + StringArray::from(vec!["%H-%M-%S".to_string()]), + "08-50-51".to_string(), + ), + ( + ScalarValue::Time32Millisecond(Some(18506000)), + StringArray::from(vec!["%H-%M-%S".to_string()]), + "05-08-26".to_string(), + ), + ( + ScalarValue::Time64Microsecond(Some(12344567000)), + StringArray::from(vec!["%H-%M-%S %f".to_string()]), + "03-25-44 567000000".to_string(), + ), + ( + ScalarValue::Time64Nanosecond(Some(12344567890000)), + StringArray::from(vec!["%H-%M-%S %f".to_string()]), + "03-25-44 567890000".to_string(), + ), + ( + ScalarValue::TimestampSecond(Some(date.timestamp()), None), + StringArray::from(vec!["%Y::%m::%d %S::%M::%H".to_string()]), + "2020::01::02 05::04::03".to_string(), + ), + ( + ScalarValue::TimestampMillisecond(Some(date.timestamp_millis()), None), + StringArray::from(vec!["%Y::%m::%d %S::%M::%H".to_string()]), + "2020::01::02 05::04::03".to_string(), + ), + ( + ScalarValue::TimestampMicrosecond(Some(date.timestamp_micros()), None), + StringArray::from(vec!["%Y::%m::%d %S::%M::%H %f".to_string()]), + "2020::01::02 05::04::03 000012000".to_string(), + ), + ( + ScalarValue::TimestampNanosecond( + Some(date.timestamp_nanos_opt().unwrap()), + None, + ), + StringArray::from(vec!["%Y::%m::%d %S::%M::%H %f".to_string()]), + "2020::01::02 05::04::03 000012345".to_string(), + ), + ]; + + for (value, format, expected) in scalar_array_data { + let result = to_char(&[ + ColumnarValue::Scalar(value), + ColumnarValue::Array(Arc::new(format) as ArrayRef), + ]) + .expect("that to_char parsed values without error"); + + if let ColumnarValue::Scalar(ScalarValue::Utf8(date)) = result { + assert_eq!(expected, date.unwrap()); + } else { + panic!("Expected a scalar value") + } + } + + let array_scalar_data = vec![ + ( + Arc::new(Date32Array::from(vec![18506, 18507])) as ArrayRef, + ScalarValue::Utf8(Some("%Y::%m::%d".to_string())), + StringArray::from(vec!["2020::09::01", "2020::09::02"]), + ), + ( + Arc::new(Date64Array::from(vec![ + date.timestamp_millis(), + date2.timestamp_millis(), + ])) as ArrayRef, + ScalarValue::Utf8(Some("%Y::%m::%d".to_string())), + StringArray::from(vec!["2020::01::02", "2026::07::08"]), + ), + ]; + + let array_array_data = vec![ + ( + Arc::new(Date32Array::from(vec![18506, 18507])) as ArrayRef, + StringArray::from(vec!["%Y::%m::%d", "%d::%m::%Y"]), + StringArray::from(vec!["2020::09::01", "02::09::2020"]), + ), + ( + Arc::new(Date64Array::from(vec![ + date.timestamp_millis(), + date2.timestamp_millis(), + ])) as ArrayRef, + StringArray::from(vec!["%Y::%m::%d", "%d::%m::%Y"]), + StringArray::from(vec!["2020::01::02", "08::07::2026"]), + ), + ( + Arc::new(Time32MillisecondArray::from(vec![1850600, 1860700])) + as ArrayRef, + StringArray::from(vec!["%H:%M:%S", "%H::%M::%S"]), + StringArray::from(vec!["00:30:50", "00::31::00"]), + ), + ( + Arc::new(Time32SecondArray::from(vec![18506, 18507])) as ArrayRef, + StringArray::from(vec!["%H:%M:%S", "%H::%M::%S"]), + StringArray::from(vec!["05:08:26", "05::08::27"]), + ), + ( + Arc::new(Time64MicrosecondArray::from(vec![12344567000, 22244567000])) + as ArrayRef, + StringArray::from(vec!["%H:%M:%S", "%H::%M::%S"]), + StringArray::from(vec!["03:25:44", "06::10::44"]), + ), + ( + Arc::new(Time64NanosecondArray::from(vec![ + 1234456789000, + 2224456789000, + ])) as ArrayRef, + StringArray::from(vec!["%H:%M:%S", "%H::%M::%S"]), + StringArray::from(vec!["00:20:34", "00::37::04"]), + ), + ( + Arc::new(TimestampSecondArray::from(vec![ + date.timestamp(), + date2.timestamp(), + ])) as ArrayRef, + StringArray::from(vec!["%Y::%m::%d %S::%M::%H", "%d::%m::%Y %S-%M-%H"]), + StringArray::from(vec![ + "2020::01::02 05::04::03", + "08::07::2026 11-10-09", + ]), + ), + ( + Arc::new(TimestampMillisecondArray::from(vec![ + date.timestamp_millis(), + date2.timestamp_millis(), + ])) as ArrayRef, + StringArray::from(vec![ + "%Y::%m::%d %S::%M::%H %f", + "%d::%m::%Y %S-%M-%H %f", + ]), + StringArray::from(vec![ + "2020::01::02 05::04::03 000000000", + "08::07::2026 11-10-09 000000000", + ]), + ), + ( + Arc::new(TimestampMicrosecondArray::from(vec![ + date.timestamp_micros(), + date2.timestamp_micros(), + ])) as ArrayRef, + StringArray::from(vec![ + "%Y::%m::%d %S::%M::%H %f", + "%d::%m::%Y %S-%M-%H %f", + ]), + StringArray::from(vec![ + "2020::01::02 05::04::03 000012000", + "08::07::2026 11-10-09 000056000", + ]), + ), + ( + Arc::new(TimestampNanosecondArray::from(vec![ + date.timestamp_nanos_opt().unwrap(), + date2.timestamp_nanos_opt().unwrap(), + ])) as ArrayRef, + StringArray::from(vec![ + "%Y::%m::%d %S::%M::%H %f", + "%d::%m::%Y %S-%M-%H %f", + ]), + StringArray::from(vec![ + "2020::01::02 05::04::03 000012345", + "08::07::2026 11-10-09 000056789", + ]), + ), + ]; + + for (value, format, expected) in array_scalar_data { + let result = to_char(&[ + ColumnarValue::Array(value as ArrayRef), + ColumnarValue::Scalar(format), + ]) + .expect("that to_char parsed values without error"); + + if let ColumnarValue::Array(result) = result { + assert_eq!(result.len(), 2); + assert_eq!(&expected as &dyn Array, result.as_ref()); + } else { + panic!("Expected an array value") + } + } + + for (value, format, expected) in array_array_data { + let result = to_char(&[ + ColumnarValue::Array(value), + ColumnarValue::Array(Arc::new(format) as ArrayRef), + ]) + .expect("that to_char parsed values without error"); + + if let ColumnarValue::Array(result) = result { + assert_eq!(result.len(), 2); + assert_eq!(&expected as &dyn Array, result.as_ref()); + } else { + panic!("Expected an array value") + } + } + + // + // Fallible test cases + // + + // invalid number of arguments + let result = to_char(&[ColumnarValue::Scalar(ScalarValue::Int32(Some(1)))]); + assert_eq!( + result.err().unwrap().strip_backtrace(), + "Execution error: to_char function requires 2 arguments, got 1" + ); + + // invalid type + let result = to_char(&[ + ColumnarValue::Scalar(ScalarValue::Int32(Some(1))), + ColumnarValue::Scalar(ScalarValue::TimestampNanosecond(Some(1), None)), + ]); + assert_eq!( + result.err().unwrap().strip_backtrace(), + "Execution error: Format for `to_char` must be non-null Utf8, received Timestamp(Nanosecond, None)" + ); + } } diff --git a/datafusion/physical-expr/src/functions.rs b/datafusion/physical-expr/src/functions.rs index 81f433611af8..ca073516986c 100644 --- a/datafusion/physical-expr/src/functions.rs +++ b/datafusion/physical-expr/src/functions.rs @@ -516,6 +516,7 @@ pub fn create_physical_fun( )) } BuiltinScalarFunction::MakeDate => Arc::new(datetime_expressions::make_date), + BuiltinScalarFunction::ToChar => Arc::new(datetime_expressions::to_char), BuiltinScalarFunction::ToTimestamp => { Arc::new(datetime_expressions::to_timestamp_invoke) } diff --git a/datafusion/physical-plan/Cargo.toml b/datafusion/physical-plan/Cargo.toml index 74650b5b068f..59c7a9671a91 100644 --- a/datafusion/physical-plan/Cargo.toml +++ b/datafusion/physical-plan/Cargo.toml @@ -41,7 +41,7 @@ arrow-array = { workspace = true } arrow-buffer = { workspace = true } arrow-schema = { workspace = true } async-trait = { workspace = true } -chrono = { version = "0.4.23", default-features = false } +chrono = { workspace = true } datafusion-common = { workspace = true } datafusion-execution = { workspace = true } datafusion-expr = { workspace = true } diff --git a/datafusion/proto/proto/datafusion.proto b/datafusion/proto/proto/datafusion.proto index d50336bd0f4c..3c8bd4ef305b 100644 --- a/datafusion/proto/proto/datafusion.proto +++ b/datafusion/proto/proto/datafusion.proto @@ -678,6 +678,7 @@ enum ScalarFunction { MakeDate = 133; ArrayReverse = 134; RegexpLike = 135; + ToChar = 136; } message ScalarFunctionNode { diff --git a/datafusion/proto/src/generated/pbjson.rs b/datafusion/proto/src/generated/pbjson.rs index 592c2609b678..89e170d3ec26 100644 --- a/datafusion/proto/src/generated/pbjson.rs +++ b/datafusion/proto/src/generated/pbjson.rs @@ -22436,6 +22436,7 @@ impl serde::Serialize for ScalarFunction { Self::MakeDate => "MakeDate", Self::ArrayReverse => "ArrayReverse", Self::RegexpLike => "RegexpLike", + Self::ToChar => "ToChar", }; serializer.serialize_str(variant) } @@ -22580,6 +22581,7 @@ impl<'de> serde::Deserialize<'de> for ScalarFunction { "MakeDate", "ArrayReverse", "RegexpLike", + "ToChar", ]; struct GeneratedVisitor; @@ -22753,6 +22755,7 @@ impl<'de> serde::Deserialize<'de> for ScalarFunction { "MakeDate" => Ok(ScalarFunction::MakeDate), "ArrayReverse" => Ok(ScalarFunction::ArrayReverse), "RegexpLike" => Ok(ScalarFunction::RegexpLike), + "ToChar" => Ok(ScalarFunction::ToChar), _ => 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 a65df74bbcf3..679df2b8d937 100644 --- a/datafusion/proto/src/generated/prost.rs +++ b/datafusion/proto/src/generated/prost.rs @@ -2765,6 +2765,7 @@ pub enum ScalarFunction { MakeDate = 133, ArrayReverse = 134, RegexpLike = 135, + ToChar = 136, } impl ScalarFunction { /// String value of the enum field names used in the ProtoBuf definition. @@ -2906,6 +2907,7 @@ impl ScalarFunction { ScalarFunction::MakeDate => "MakeDate", ScalarFunction::ArrayReverse => "ArrayReverse", ScalarFunction::RegexpLike => "RegexpLike", + ScalarFunction::ToChar => "ToChar", } } /// Creates an enum from field names used in the ProtoBuf definition. @@ -3044,6 +3046,7 @@ impl ScalarFunction { "MakeDate" => Some(Self::MakeDate), "ArrayReverse" => Some(Self::ArrayReverse), "RegexpLike" => Some(Self::RegexpLike), + "ToChar" => Some(Self::ToChar), _ => None, } } diff --git a/datafusion/proto/src/logical_plan/from_proto.rs b/datafusion/proto/src/logical_plan/from_proto.rs index 07590e0d93ae..3a2494c01568 100644 --- a/datafusion/proto/src/logical_plan/from_proto.rs +++ b/datafusion/proto/src/logical_plan/from_proto.rs @@ -550,6 +550,7 @@ impl From<&protobuf::ScalarFunction> for BuiltinScalarFunction { ScalarFunction::Strpos => Self::Strpos, ScalarFunction::Substr => Self::Substr, ScalarFunction::ToHex => Self::ToHex, + ScalarFunction::ToChar => Self::ToChar, ScalarFunction::ToTimestamp => Self::ToTimestamp, ScalarFunction::ToTimestampMillis => Self::ToTimestampMillis, ScalarFunction::ToTimestampMicros => Self::ToTimestampMicros, @@ -1704,6 +1705,16 @@ pub fn parse_expr( args, ))) } + ScalarFunction::ToChar => { + let args: Vec<_> = args + .iter() + .map(|expr| parse_expr(expr, registry)) + .collect::>()?; + Ok(Expr::ScalarFunction(expr::ScalarFunction::new( + BuiltinScalarFunction::ToChar, + 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 7a8fbde07b6f..9510846f7063 100644 --- a/datafusion/proto/src/logical_plan/to_proto.rs +++ b/datafusion/proto/src/logical_plan/to_proto.rs @@ -1456,6 +1456,7 @@ impl TryFrom<&BuiltinScalarFunction> for protobuf::ScalarFunction { BuiltinScalarFunction::Trim => Self::Trim, BuiltinScalarFunction::Ltrim => Self::Ltrim, BuiltinScalarFunction::Rtrim => Self::Rtrim, + BuiltinScalarFunction::ToChar => Self::ToChar, BuiltinScalarFunction::ToTimestamp => Self::ToTimestamp, BuiltinScalarFunction::ArrayAppend => Self::ArrayAppend, BuiltinScalarFunction::ArraySort => Self::ArraySort, diff --git a/datafusion/sqllogictest/test_files/timestamps.slt b/datafusion/sqllogictest/test_files/timestamps.slt index 980545e922c4..d7085631777c 100644 --- a/datafusion/sqllogictest/test_files/timestamps.slt +++ b/datafusion/sqllogictest/test_files/timestamps.slt @@ -2565,3 +2565,116 @@ 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, ''); + + +########## +## to_char tests +########## + +statement ok +create table formats ( + dates date, + times time, + timestamps timestamp, + date_format varchar, + time_format varchar, + timestamp_format varchar) +as values + ('2000-01-01'::date, '23:45:01'::time, '2024-01-01 06:00:00'::timestamp, '%d:%m:%Y', '%H-%M-%S', '%d:%m:%Y %H-%M-%S'), + ('2003-04-05'::date, '04:56:32'::time, '2025-01-01 23:59:58'::timestamp, '%d:%m:%Y', '%H::%M::%S', '%d:%m:%Y %H-%M-%S'); + + +query T +select to_char(dates, date_format) from formats; +---- +01:01:2000 +05:04:2003 + +query T +select date_format(dates, date_format) from formats; +---- +01:01:2000 +05:04:2003 + +query T +select to_char(times, time_format) from formats; +---- +23-45-01 +04::56::32 + +query T +select to_char(timestamps, date_format) from formats; +---- +01:01:2024 +01:01:2025 + +query T +select to_char(timestamps, timestamp_format) from formats; +---- +01:01:2024 06-00-00 +01:01:2025 23-59-58 + +query T +select to_char('2000-02-03'::date, '%Y:%d:%m'); +---- +2000:03:02 + +query T +select to_char(arrow_cast(12345::int, 'Time32(Second)'), '%H-%M-%S') +---- +03-25-45 + +query T +select to_char(arrow_cast(12344567::int, 'Time32(Millisecond)'), '%H-%M-%S %f') +---- +03-25-44 567000000 + +query T +select to_char(arrow_cast(12344567000, 'Time64(Microsecond)'), '%H-%M-%S %f') +---- +03-25-44 567000000 + +query T +select to_char(arrow_cast(12344567890000, 'Time64(Nanosecond)'), '%H-%M-%S %f') +---- +03-25-44 567890000 + +query T +select to_char(arrow_cast(TIMESTAMP '2023-08-03 14:38:50Z', 'Timestamp(Second, None)'), '%d-%m-%Y %H-%M-%S') +---- +03-08-2023 14-38-50 + +query T +select to_char(arrow_cast(123456, 'Duration(Second)'), 'pretty'); +---- +1 days 10 hours 17 mins 36 secs + +query T +select to_char(arrow_cast(123456, 'Duration(Second)'), 'iso8601'); +---- +PT123456S + +query T +select to_char(arrow_cast(123456, 'Duration(Second)'), null); +---- +PT123456S + +query error DataFusion error: Execution error: Cast error: Format error +SELECT to_char(timestamps, '%X%K') from formats; + +query error DataFusion error: Execution error: Cast error: Format error +SELECT to_char('2000-02-03'::date, '%X%K'); + +query T +SELECT to_char(timestamps, null) from formats; +---- +2024-01-01T06:00:00Z +2025-01-01T23:59:58Z + +query T +SELECT to_char(null, '%d-%m-%Y'); +---- +(empty) + +statement ok +drop table formats; diff --git a/docs/source/user-guide/sql/scalar_functions.md b/docs/source/user-guide/sql/scalar_functions.md index c9c32a67ce53..a6e872cab4c1 100644 --- a/docs/source/user-guide/sql/scalar_functions.md +++ b/docs/source/user-guide/sql/scalar_functions.md @@ -1425,6 +1425,7 @@ position(substr in origstr) - [extract](#extract) - [today](#today) - [make_date](#make_date) +- [to_char](#to_char) - [to_timestamp](#to_timestamp) - [to_timestamp_millis](#to_timestamp_millis) - [to_timestamp_micros](#to_timestamp_micros) @@ -1649,6 +1650,42 @@ make_date(year, month, day) Additional examples can be found [here](https://github.com/apache/arrow-datafusion/blob/main/datafusion-examples/examples/make_date.rs) +### `to_char` + +Returns a string representation of a date, time, timestamp or duration based +on a [Chrono format]. Unlike the PostgreSQL equivalent of this function +numerical formatting is not supported. + +``` +to_char(expression, format) +``` + +#### Arguments + +- **expression**: Expression to operate on. + Can be a constant, column, or function that results in a + date, time, timestamp or duration. +- **format**: A [Chrono format] string to use to convert the expression. + +#### Example + +``` +❯ ❯ select to_char('2023-03-01'::date, '%d-%m-%Y'); ++----------------------------------------------+ +| to_char(Utf8("2023-03-01"),Utf8("%d-%m-%Y")) | ++----------------------------------------------+ +| 01-03-2023 | ++----------------------------------------------+ +``` + +Additional examples can be found [here] + +[here]: https://github.com/apache/arrow-datafusion/blob/main/datafusion-examples/examples/to_char.rs + +#### Aliases + +- date_format + ### `to_timestamp` Converts a value to a timestamp (`YYYY-MM-DDT00:00:00Z`).