From 2b15ad16f3516f29d6540c974170591a6c085478 Mon Sep 17 00:00:00 2001 From: zjregee Date: Tue, 31 Dec 2024 22:22:10 +0800 Subject: [PATCH 01/17] consolidate dataframe_subquery.rs into dataframe.rs (#13950) --- datafusion-examples/README.md | 2 +- datafusion-examples/examples/dataframe.rs | 91 ++++++++++++++ .../examples/dataframe_subquery.rs | 118 ------------------ 3 files changed, 92 insertions(+), 119 deletions(-) delete mode 100644 datafusion-examples/examples/dataframe_subquery.rs diff --git a/datafusion-examples/README.md b/datafusion-examples/README.md index 3ec008a6026d..23cf8830e36d 100644 --- a/datafusion-examples/README.md +++ b/datafusion-examples/README.md @@ -57,7 +57,7 @@ cargo run --example dataframe - [`custom_datasource.rs`](examples/custom_datasource.rs): Run queries against a custom datasource (TableProvider) - [`custom_file_format.rs`](examples/custom_file_format.rs): Write data to a custom file format - [`dataframe-to-s3.rs`](examples/external_dependency/dataframe-to-s3.rs): Run a query using a DataFrame against a parquet file from s3 and writing back to s3 -- [`dataframe.rs`](examples/dataframe.rs): Run a query using a DataFrame API against parquet files, csv files, and in-memory data. Also demonstrates the various methods to write out a DataFrame to a table, parquet file, csv file, and json file. +- [`dataframe.rs`](examples/dataframe.rs): Run a query using a DataFrame API against parquet files, csv files, and in-memory data, including multiple subqueries. Also demonstrates the various methods to write out a DataFrame to a table, parquet file, csv file, and json file. - [`deserialize_to_struct.rs`](examples/deserialize_to_struct.rs): Convert query results into rust structs using serde - [`expr_api.rs`](examples/expr_api.rs): Create, execute, simplify, analyze and coerce `Expr`s - [`file_stream_provider.rs`](examples/file_stream_provider.rs): Run a query on `FileStreamProvider` which implements `StreamProvider` for reading and writing to arbitrary stream sources / sinks. diff --git a/datafusion-examples/examples/dataframe.rs b/datafusion-examples/examples/dataframe.rs index 90d7d778ea5c..91d62135b913 100644 --- a/datafusion-examples/examples/dataframe.rs +++ b/datafusion-examples/examples/dataframe.rs @@ -19,10 +19,13 @@ use arrow::array::{ArrayRef, Int32Array, RecordBatch, StringArray}; use datafusion::arrow::datatypes::{DataType, Field, Schema}; use datafusion::dataframe::DataFrameWriteOptions; use datafusion::error::Result; +use datafusion::functions_aggregate::average::avg; +use datafusion::functions_aggregate::min_max::max; use datafusion::prelude::*; use datafusion_common::config::CsvOptions; use datafusion_common::parsers::CompressionTypeVariant; use datafusion_common::DataFusionError; +use datafusion_common::ScalarValue; use std::fs::File; use std::io::Write; use std::sync::Arc; @@ -44,7 +47,14 @@ use tempfile::tempdir; /// /// * [write_out]: write out a DataFrame to a table, parquet file, csv file, or json file /// +/// # Executing subqueries +/// +/// * [where_scalar_subquery]: execute a scalar subquery +/// * [where_in_subquery]: execute a subquery with an IN clause +/// * [where_exist_subquery]: execute a subquery with an EXISTS clause +/// /// # Querying data +/// /// * [query_to_date]: execute queries against parquet files #[tokio::main] async fn main() -> Result<()> { @@ -55,6 +65,11 @@ async fn main() -> Result<()> { read_memory(&ctx).await?; write_out(&ctx).await?; query_to_date().await?; + register_aggregate_test_data("t1", &ctx).await?; + register_aggregate_test_data("t2", &ctx).await?; + where_scalar_subquery(&ctx).await?; + where_in_subquery(&ctx).await?; + where_exist_subquery(&ctx).await?; Ok(()) } @@ -250,3 +265,79 @@ async fn query_to_date() -> Result<()> { Ok(()) } + +/// Use the DataFrame API to execute the following subquery: +/// select c1,c2 from t1 where (select avg(t2.c2) from t2 where t1.c1 = t2.c1)>0 limit 3; +async fn where_scalar_subquery(ctx: &SessionContext) -> Result<()> { + ctx.table("t1") + .await? + .filter( + scalar_subquery(Arc::new( + ctx.table("t2") + .await? + .filter(out_ref_col(DataType::Utf8, "t1.c1").eq(col("t2.c1")))? + .aggregate(vec![], vec![avg(col("t2.c2"))])? + .select(vec![avg(col("t2.c2"))])? + .into_unoptimized_plan(), + )) + .gt(lit(0u8)), + )? + .select(vec![col("t1.c1"), col("t1.c2")])? + .limit(0, Some(3))? + .show() + .await?; + Ok(()) +} + +/// Use the DataFrame API to execute the following subquery: +/// select t1.c1, t1.c2 from t1 where t1.c2 in (select max(t2.c2) from t2 where t2.c1 > 0 ) limit 3; +async fn where_in_subquery(ctx: &SessionContext) -> Result<()> { + ctx.table("t1") + .await? + .filter(in_subquery( + col("t1.c2"), + Arc::new( + ctx.table("t2") + .await? + .filter(col("t2.c1").gt(lit(ScalarValue::UInt8(Some(0)))))? + .aggregate(vec![], vec![max(col("t2.c2"))])? + .select(vec![max(col("t2.c2"))])? + .into_unoptimized_plan(), + ), + ))? + .select(vec![col("t1.c1"), col("t1.c2")])? + .limit(0, Some(3))? + .show() + .await?; + Ok(()) +} + +/// Use the DataFrame API to execute the following subquery: +/// select t1.c1, t1.c2 from t1 where exists (select t2.c2 from t2 where t1.c1 = t2.c1) limit 3; +async fn where_exist_subquery(ctx: &SessionContext) -> Result<()> { + ctx.table("t1") + .await? + .filter(exists(Arc::new( + ctx.table("t2") + .await? + .filter(out_ref_col(DataType::Utf8, "t1.c1").eq(col("t2.c1")))? + .select(vec![col("t2.c2")])? + .into_unoptimized_plan(), + )))? + .select(vec![col("t1.c1"), col("t1.c2")])? + .limit(0, Some(3))? + .show() + .await?; + Ok(()) +} + +async fn register_aggregate_test_data(name: &str, ctx: &SessionContext) -> Result<()> { + let testdata = datafusion::test_util::arrow_test_data(); + ctx.register_csv( + name, + &format!("{testdata}/csv/aggregate_test_100.csv"), + CsvReadOptions::default(), + ) + .await?; + Ok(()) +} diff --git a/datafusion-examples/examples/dataframe_subquery.rs b/datafusion-examples/examples/dataframe_subquery.rs deleted file mode 100644 index 3e3d0c1b5a84..000000000000 --- a/datafusion-examples/examples/dataframe_subquery.rs +++ /dev/null @@ -1,118 +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 arrow_schema::DataType; -use std::sync::Arc; - -use datafusion::error::Result; -use datafusion::functions_aggregate::average::avg; -use datafusion::functions_aggregate::min_max::max; -use datafusion::prelude::*; -use datafusion::test_util::arrow_test_data; -use datafusion_common::ScalarValue; - -/// This example demonstrates how to use the DataFrame API to create a subquery. -#[tokio::main] -async fn main() -> Result<()> { - let ctx = SessionContext::new(); - register_aggregate_test_data("t1", &ctx).await?; - register_aggregate_test_data("t2", &ctx).await?; - - where_scalar_subquery(&ctx).await?; - - where_in_subquery(&ctx).await?; - - where_exist_subquery(&ctx).await?; - - Ok(()) -} - -//select c1,c2 from t1 where (select avg(t2.c2) from t2 where t1.c1 = t2.c1)>0 limit 3; -async fn where_scalar_subquery(ctx: &SessionContext) -> Result<()> { - ctx.table("t1") - .await? - .filter( - scalar_subquery(Arc::new( - ctx.table("t2") - .await? - .filter(out_ref_col(DataType::Utf8, "t1.c1").eq(col("t2.c1")))? - .aggregate(vec![], vec![avg(col("t2.c2"))])? - .select(vec![avg(col("t2.c2"))])? - .into_unoptimized_plan(), - )) - .gt(lit(0u8)), - )? - .select(vec![col("t1.c1"), col("t1.c2")])? - .limit(0, Some(3))? - .show() - .await?; - Ok(()) -} - -//SELECT t1.c1, t1.c2 FROM t1 WHERE t1.c2 in (select max(t2.c2) from t2 where t2.c1 > 0 ) limit 3; -async fn where_in_subquery(ctx: &SessionContext) -> Result<()> { - ctx.table("t1") - .await? - .filter(in_subquery( - col("t1.c2"), - Arc::new( - ctx.table("t2") - .await? - .filter(col("t2.c1").gt(lit(ScalarValue::UInt8(Some(0)))))? - .aggregate(vec![], vec![max(col("t2.c2"))])? - .select(vec![max(col("t2.c2"))])? - .into_unoptimized_plan(), - ), - ))? - .select(vec![col("t1.c1"), col("t1.c2")])? - .limit(0, Some(3))? - .show() - .await?; - Ok(()) -} - -//SELECT t1.c1, t1.c2 FROM t1 WHERE EXISTS (select t2.c2 from t2 where t1.c1 = t2.c1) limit 3; -async fn where_exist_subquery(ctx: &SessionContext) -> Result<()> { - ctx.table("t1") - .await? - .filter(exists(Arc::new( - ctx.table("t2") - .await? - .filter(out_ref_col(DataType::Utf8, "t1.c1").eq(col("t2.c1")))? - .select(vec![col("t2.c2")])? - .into_unoptimized_plan(), - )))? - .select(vec![col("t1.c1"), col("t1.c2")])? - .limit(0, Some(3))? - .show() - .await?; - Ok(()) -} - -pub async fn register_aggregate_test_data( - name: &str, - ctx: &SessionContext, -) -> Result<()> { - let testdata = arrow_test_data(); - ctx.register_csv( - name, - &format!("{testdata}/csv/aggregate_test_100.csv"), - CsvReadOptions::default(), - ) - .await?; - Ok(()) -} From 9d393586cec133745cc5c2c4b63c70b0aff2cacc Mon Sep 17 00:00:00 2001 From: delamarch3 <68732277+delamarch3@users.noreply.github.com> Date: Tue, 31 Dec 2024 17:33:58 +0000 Subject: [PATCH 02/17] migrate btrim to user_doc macro (#13952) --- datafusion/functions/src/string/btrim.rs | 53 +++++++++++------------- datafusion/macros/src/user_doc.rs | 8 ++-- 2 files changed, 28 insertions(+), 33 deletions(-) diff --git a/datafusion/functions/src/string/btrim.rs b/datafusion/functions/src/string/btrim.rs index 298d64f04ae9..05a2f646e969 100644 --- a/datafusion/functions/src/string/btrim.rs +++ b/datafusion/functions/src/string/btrim.rs @@ -21,12 +21,11 @@ use arrow::array::{ArrayRef, OffsetSizeTrait}; use arrow::datatypes::DataType; use datafusion_common::{exec_err, Result}; use datafusion_expr::function::Hint; -use datafusion_expr::scalar_doc_sections::DOC_SECTION_STRING; use datafusion_expr::{ ColumnarValue, Documentation, ScalarUDFImpl, Signature, TypeSignature, Volatility, }; +use datafusion_macros::user_doc; use std::any::Any; -use std::sync::OnceLock; /// Returns the longest string with leading and trailing characters removed. If the characters are not specified, whitespace is removed. /// btrim('xyxtrimyyx', 'xyz') = 'trim' @@ -35,6 +34,28 @@ fn btrim(args: &[ArrayRef]) -> Result { general_trim::(args, TrimType::Both, use_string_view) } +#[user_doc( + doc_section(label = "String Functions"), + description = "Trims the specified trim string from the start and end of a string. If no trim string is provided, all whitespace is removed from the start and end of the input string.", + syntax_example = "btrim(str[, trim_str])", + sql_example = r#"```sql +> select btrim('__datafusion____', '_'); ++-------------------------------------------+ +| btrim(Utf8("__datafusion____"),Utf8("_")) | ++-------------------------------------------+ +| datafusion | ++-------------------------------------------+ +```"#, + standard_argument(name = "str", prefix = "String"), + argument( + name = "trim_str", + description = r"String expression to operate on. Can be a constant, column, or function, and any combination of operators. _Default is whitespace characters._" + ), + alternative_syntax = "trim(BOTH trim_str FROM str)", + alternative_syntax = "trim(trim_str FROM str)", + related_udf(name = "ltrim"), + related_udf(name = "rtrim") +)] #[derive(Debug)] pub struct BTrimFunc { signature: Signature, @@ -106,36 +127,10 @@ impl ScalarUDFImpl for BTrimFunc { } fn documentation(&self) -> Option<&Documentation> { - Some(get_btrim_doc()) + self.doc() } } -static DOCUMENTATION: OnceLock = OnceLock::new(); - -fn get_btrim_doc() -> &'static Documentation { - DOCUMENTATION.get_or_init(|| { - Documentation::builder( - DOC_SECTION_STRING, - "Trims the specified trim string from the start and end of a string. If no trim string is provided, all whitespace is removed from the start and end of the input string.", - "btrim(str[, trim_str])") - .with_sql_example(r#"```sql -> select btrim('__datafusion____', '_'); -+-------------------------------------------+ -| btrim(Utf8("__datafusion____"),Utf8("_")) | -+-------------------------------------------+ -| datafusion | -+-------------------------------------------+ -```"#) - .with_standard_argument("str", Some("String")) - .with_argument("trim_str", "String expression to operate on. Can be a constant, column, or function, and any combination of operators. _Default is whitespace characters._") - .with_alternative_syntax("trim(BOTH trim_str FROM str)") - .with_alternative_syntax("trim(trim_str FROM str)") - .with_related_udf("ltrim") - .with_related_udf("rtrim") - .build() - }) -} - #[cfg(test)] mod tests { use arrow::array::{Array, StringArray, StringViewArray}; diff --git a/datafusion/macros/src/user_doc.rs b/datafusion/macros/src/user_doc.rs index 441b3db2a133..e4b6a0f3b14a 100644 --- a/datafusion/macros/src/user_doc.rs +++ b/datafusion/macros/src/user_doc.rs @@ -103,7 +103,7 @@ pub fn user_doc(args: TokenStream, input: TokenStream) -> TokenStream { let mut description: Option = None; let mut syntax_example: Option = None; - let mut alt_syntax_example: Option = None; + let mut alt_syntax_example: Vec> = vec![]; let mut sql_example: Option = None; let mut standard_args: Vec<(Option, Option)> = vec![]; let mut udf_args: Vec<(Option, Option)> = vec![]; @@ -131,7 +131,7 @@ pub fn user_doc(args: TokenStream, input: TokenStream) -> TokenStream { syntax_example = Some(meta.value()?.parse()?); Ok(()) } else if meta.path.is_ident("alternative_syntax") { - alt_syntax_example = Some(meta.value()?.parse()?); + alt_syntax_example.push(Some(meta.value()?.parse()?)); Ok(()) } else if meta.path.is_ident("sql_example") { sql_example = Some(meta.value()?.parse()?); @@ -242,7 +242,7 @@ pub fn user_doc(args: TokenStream, input: TokenStream) -> TokenStream { }) .collect::>(); - let alt_syntax_example = alt_syntax_example.map(|syn| { + let alt_syntax_example = alt_syntax_example.iter().map(|syn| { quote! { .with_alternative_syntax(#syn) } @@ -258,7 +258,7 @@ pub fn user_doc(args: TokenStream, input: TokenStream) -> TokenStream { datafusion_doc::Documentation::builder(datafusion_doc::DocSection { include: #doc_section_include, label: #doc_section_lbl, description: #doc_section_description }, #description.to_string(), #syntax_example.to_string()) #sql_example - #alt_syntax_example + #(#alt_syntax_example)* #(#standard_args)* #(#udf_args)* #(#related_udfs)* From 9eca7d165c3ddcd3449e833df9391b8216e0f5bc Mon Sep 17 00:00:00 2001 From: Ian Lai <108986288+Chen-Yuan-Lai@users.noreply.github.com> Date: Wed, 1 Jan 2025 01:35:43 +0800 Subject: [PATCH 03/17] doc-gen: migrate scalar functions (datetime) documentation 2/2 (#13921) * doc-gen: migrate scalar functions (datetime) documentation 2/2 * fix: fix typo and update function docs * doc: update function docs * doc-gen: remove slash --------- Co-authored-by: Cheng-Yuan-Lai --- .../functions/src/datetime/make_date.rs | 77 ++-- datafusion/functions/src/datetime/now.rs | 33 +- datafusion/functions/src/datetime/to_char.rs | 66 ++-- datafusion/functions/src/datetime/to_date.rs | 32 +- .../functions/src/datetime/to_local_time.rs | 123 +++---- .../functions/src/datetime/to_timestamp.rs | 347 ++++++++---------- .../functions/src/datetime/to_unixtime.rs | 68 ++-- .../source/user-guide/sql/scalar_functions.md | 16 +- 8 files changed, 351 insertions(+), 411 deletions(-) diff --git a/datafusion/functions/src/datetime/make_date.rs b/datafusion/functions/src/datetime/make_date.rs index a9340f82f23d..3345ea2412b8 100644 --- a/datafusion/functions/src/datetime/make_date.rs +++ b/datafusion/functions/src/datetime/make_date.rs @@ -16,7 +16,7 @@ // under the License. use std::any::Any; -use std::sync::{Arc, OnceLock}; +use std::sync::Arc; use arrow::array::builder::PrimitiveBuilder; use arrow::array::cast::AsArray; @@ -27,11 +27,45 @@ use arrow::datatypes::DataType::{Date32, Int32, Int64, UInt32, UInt64, Utf8, Utf use chrono::prelude::*; use datafusion_common::{exec_err, Result, ScalarValue}; -use datafusion_expr::scalar_doc_sections::DOC_SECTION_DATETIME; use datafusion_expr::{ ColumnarValue, Documentation, ScalarUDFImpl, Signature, Volatility, }; +use datafusion_macros::user_doc; +#[user_doc( + doc_section(label = "Time and Date Functions"), + description = "Make a date from year/month/day component parts.", + syntax_example = "make_date(year, month, day)", + sql_example = r#"```sql +> 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](https://github.com/apache/datafusion/blob/main/datafusion-examples/examples/make_date.rs) +"#, + argument( + name = "year", + description = "Year to use when making the date. Can be a constant, column or function, and any combination of arithmetic operators." + ), + argument( + name = "month", + description = "Month to use when making the date. Can be a constant, column or function, and any combination of arithmetic operators." + ), + argument( + name = "day", + description = "Day to use when making the date. Can be a constant, column or function, and any combination of arithmetic operators." + ) +)] #[derive(Debug)] pub struct MakeDateFunc { signature: Signature, @@ -156,47 +190,10 @@ impl ScalarUDFImpl for MakeDateFunc { Ok(value) } fn documentation(&self) -> Option<&Documentation> { - Some(get_make_date_doc()) + self.doc() } } -static DOCUMENTATION: OnceLock = OnceLock::new(); - -fn get_make_date_doc() -> &'static Documentation { - DOCUMENTATION.get_or_init(|| { - Documentation::builder( - DOC_SECTION_DATETIME, - "Make a date from year/month/day component parts.", - "make_date(year, month, day)") - .with_argument( - "year", - " Year to use when making the date. Can be a constant, column or function, and any combination of arithmetic operators.", ) - .with_argument( - "month", - "Month to use when making the date. Can be a constant, column or function, and any combination of arithmetic operators.", - ) - .with_argument("day", "Day to use when making the date. Can be a constant, column or function, and any combination of arithmetic operators.") - .with_sql_example(r#"```sql -> 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](https://github.com/apache/datafusion/blob/main/datafusion-examples/examples/make_date.rs) -"#) - .build() - }) -} - /// Converts the year/month/day fields to an `i32` representing the days from /// the unix epoch and invokes `date_consumer_fn` with the value fn make_date_inner( diff --git a/datafusion/functions/src/datetime/now.rs b/datafusion/functions/src/datetime/now.rs index 58381473a9ab..67cd49b7fd84 100644 --- a/datafusion/functions/src/datetime/now.rs +++ b/datafusion/functions/src/datetime/now.rs @@ -19,15 +19,23 @@ use arrow::datatypes::DataType; use arrow::datatypes::DataType::Timestamp; use arrow::datatypes::TimeUnit::Nanosecond; use std::any::Any; -use std::sync::OnceLock; use datafusion_common::{internal_err, ExprSchema, Result, ScalarValue}; -use datafusion_expr::scalar_doc_sections::DOC_SECTION_DATETIME; use datafusion_expr::simplify::{ExprSimplifyResult, SimplifyInfo}; use datafusion_expr::{ ColumnarValue, Documentation, Expr, ScalarUDFImpl, Signature, Volatility, }; +use datafusion_macros::user_doc; +#[user_doc( + doc_section(label = "Time and Date Functions"), + description = r#" +Returns the current UTC timestamp. + +The `now()` return value is determined at query time and will return the same timestamp, no matter when in the query plan the function executes. +"#, + syntax_example = "now()" +)] #[derive(Debug)] pub struct NowFunc { signature: Signature, @@ -93,9 +101,6 @@ impl ScalarUDFImpl for NowFunc { ScalarValue::TimestampNanosecond(now_ts, Some("+00:00".into())), ))) } - fn documentation(&self) -> Option<&Documentation> { - Some(get_to_unixtime_doc()) - } fn aliases(&self) -> &[String] { &self.aliases @@ -104,20 +109,8 @@ impl ScalarUDFImpl for NowFunc { fn is_nullable(&self, _args: &[Expr], _schema: &dyn ExprSchema) -> bool { false } -} -static DOCUMENTATION: OnceLock = OnceLock::new(); - -fn get_to_unixtime_doc() -> &'static Documentation { - DOCUMENTATION.get_or_init(|| { - Documentation::builder( - DOC_SECTION_DATETIME, - r#" -Returns the current UTC timestamp. - -The `now()` return value is determined at query time and will return the same timestamp, no matter when in the query plan the function executes. -"#, - "now()") - .build() - }) + fn documentation(&self) -> Option<&Documentation> { + self.doc() + } } diff --git a/datafusion/functions/src/datetime/to_char.rs b/datafusion/functions/src/datetime/to_char.rs index 4e3fcd1dc13a..a6d4c2b425ac 100644 --- a/datafusion/functions/src/datetime/to_char.rs +++ b/datafusion/functions/src/datetime/to_char.rs @@ -16,7 +16,7 @@ // under the License. use std::any::Any; -use std::sync::{Arc, OnceLock}; +use std::sync::Arc; use arrow::array::cast::AsArray; use arrow::array::{new_null_array, Array, ArrayRef, StringArray}; @@ -29,12 +29,40 @@ use arrow::error::ArrowError; use arrow::util::display::{ArrayFormatter, DurationFormat, FormatOptions}; use datafusion_common::{exec_err, Result, ScalarValue}; -use datafusion_expr::scalar_doc_sections::DOC_SECTION_DATETIME; use datafusion_expr::TypeSignature::Exact; use datafusion_expr::{ ColumnarValue, Documentation, ScalarUDFImpl, Signature, Volatility, TIMEZONE_WILDCARD, }; +use datafusion_macros::user_doc; +#[user_doc( + doc_section(label = "Time and Date Functions"), + description = "Returns a string representation of a date, time, timestamp or duration based on a [Chrono format](https://docs.rs/chrono/latest/chrono/format/strftime/index.html). Unlike the PostgreSQL equivalent of this function numerical formatting is not supported.", + syntax_example = "to_char(expression, format)", + sql_example = r#"```sql +> 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](https://github.com/apache/datafusion/blob/main/datafusion-examples/examples/to_char.rs) +"#, + argument( + name = "expression", + description = "Expression to operate on. Can be a constant, column, or function that results in a date, time, timestamp or duration." + ), + argument( + name = "format", + description = "A [Chrono format](https://docs.rs/chrono/latest/chrono/format/strftime/index.html) string to use to convert the expression." + ), + argument( + name = "day", + description = "Day to use when making the date. Can be a constant, column or function, and any combination of arithmetic operators." + ) +)] #[derive(Debug)] pub struct ToCharFunc { signature: Signature, @@ -143,42 +171,10 @@ impl ScalarUDFImpl for ToCharFunc { &self.aliases } fn documentation(&self) -> Option<&Documentation> { - Some(get_to_char_doc()) + self.doc() } } -static DOCUMENTATION: OnceLock = OnceLock::new(); - -fn get_to_char_doc() -> &'static Documentation { - DOCUMENTATION.get_or_init(|| { - Documentation::builder( - DOC_SECTION_DATETIME, - "Returns a string representation of a date, time, timestamp or duration based on a [Chrono format](https://docs.rs/chrono/latest/chrono/format/strftime/index.html). Unlike the PostgreSQL equivalent of this function numerical formatting is not supported.", - "to_char(expression, format)") - .with_argument( - "expression", - " Expression to operate on. Can be a constant, column, or function that results in a date, time, timestamp or duration." - ) - .with_argument( - "format", - "A [Chrono format](https://docs.rs/chrono/latest/chrono/format/strftime/index.html) string to use to convert the expression.", - ) - .with_argument("day", "Day to use when making the date. Can be a constant, column or function, and any combination of arithmetic operators.") - .with_sql_example(r#"```sql -> 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](https://github.com/apache/datafusion/blob/main/datafusion-examples/examples/to_char.rs) -"#) - .build() - }) -} - fn _build_format_options<'a>( data_type: &DataType, format: Option<&'a str>, diff --git a/datafusion/functions/src/datetime/to_date.rs b/datafusion/functions/src/datetime/to_date.rs index 091d0ba37644..6d873ab52427 100644 --- a/datafusion/functions/src/datetime/to_date.rs +++ b/datafusion/functions/src/datetime/to_date.rs @@ -38,21 +38,23 @@ Returns the corresponding date. Note: `to_date` returns Date32, which represents its values as the number of days since unix epoch(`1970-01-01`) stored as signed 32 bit value. The largest supported date value is `9999-12-31`.", syntax_example = "to_date('2017-05-31', '%Y-%m-%d')", - sql_example = "```sql\n\ -> select to_date('2023-01-31');\n\ -+-----------------------------+\n\ -| to_date(Utf8(\"2023-01-31\")) |\n\ -+-----------------------------+\n\ -| 2023-01-31 |\n\ -+-----------------------------+\n\ -> select to_date('2023/01/31', '%Y-%m-%d', '%Y/%m/%d');\n\ -+---------------------------------------------------------------+\n\ -| to_date(Utf8(\"2023/01/31\"),Utf8(\"%Y-%m-%d\"),Utf8(\"%Y/%m/%d\")) |\n\ -+---------------------------------------------------------------+\n\ -| 2023-01-31 |\n\ -+---------------------------------------------------------------+\n\ -```\n\n\ -Additional examples can be found [here](https://github.com/apache/datafusion/blob/main/datafusion-examples/examples/to_date.rs)", + sql_example = r#"```sql +> select to_date('2023-01-31'); ++-------------------------------+ +| to_date(Utf8("2023-01-31")) | ++-------------------------------+ +| 2023-01-31 | ++-------------------------------+ +> select to_date('2023/01/31', '%Y-%m-%d', '%Y/%m/%d'); ++---------------------------------------------------------------------+ +| to_date(Utf8("2023/01/31"),Utf8("%Y-%m-%d"),Utf8("%Y/%m/%d")) | ++---------------------------------------------------------------------+ +| 2023-01-31 | ++---------------------------------------------------------------------+ +``` + +Additional examples can be found [here](https://github.com/apache/datafusion/blob/main/datafusion-examples/examples/to_date.rs) +"#, standard_argument(name = "expression", prefix = "String"), argument( name = "format_n", diff --git a/datafusion/functions/src/datetime/to_local_time.rs b/datafusion/functions/src/datetime/to_local_time.rs index 9f95b780ea4f..b350819a55ec 100644 --- a/datafusion/functions/src/datetime/to_local_time.rs +++ b/datafusion/functions/src/datetime/to_local_time.rs @@ -17,7 +17,7 @@ use std::any::Any; use std::ops::Add; -use std::sync::{Arc, OnceLock}; +use std::sync::Arc; use arrow::array::timezone::Tz; use arrow::array::{Array, ArrayRef, PrimitiveBuilder}; @@ -31,14 +31,69 @@ use arrow::datatypes::{ use chrono::{DateTime, MappedLocalTime, Offset, TimeDelta, TimeZone, Utc}; use datafusion_common::cast::as_primitive_array; use datafusion_common::{exec_err, plan_err, DataFusionError, Result, ScalarValue}; -use datafusion_expr::scalar_doc_sections::DOC_SECTION_DATETIME; use datafusion_expr::{ ColumnarValue, Documentation, ScalarUDFImpl, Signature, Volatility, }; +use datafusion_macros::user_doc; /// A UDF function that converts a timezone-aware timestamp to local time (with no offset or /// timezone information). In other words, this function strips off the timezone from the timestamp, /// while keep the display value of the timestamp the same. +#[user_doc( + doc_section(label = "Time and Date Functions"), + description = "Converts a timestamp with a timezone to a timestamp without a timezone (with no offset or timezone information). This function handles daylight saving time changes.", + syntax_example = "to_local_time(expression)", + sql_example = r#"```sql +> SELECT to_local_time('2024-04-01T00:00:20Z'::timestamp); ++---------------------------------------------+ +| to_local_time(Utf8("2024-04-01T00:00:20Z")) | ++---------------------------------------------+ +| 2024-04-01T00:00:20 | ++---------------------------------------------+ + +> SELECT to_local_time('2024-04-01T00:00:20Z'::timestamp AT TIME ZONE 'Europe/Brussels'); ++---------------------------------------------+ +| to_local_time(Utf8("2024-04-01T00:00:20Z")) | ++---------------------------------------------+ +| 2024-04-01T00:00:20 | ++---------------------------------------------+ + +> SELECT + time, + arrow_typeof(time) as type, + to_local_time(time) as to_local_time, + arrow_typeof(to_local_time(time)) as to_local_time_type +FROM ( + SELECT '2024-04-01T00:00:20Z'::timestamp AT TIME ZONE 'Europe/Brussels' AS time +); ++---------------------------+------------------------------------------------+---------------------+-----------------------------+ +| time | type | to_local_time | to_local_time_type | ++---------------------------+------------------------------------------------+---------------------+-----------------------------+ +| 2024-04-01T00:00:20+02:00 | Timestamp(Nanosecond, Some("Europe/Brussels")) | 2024-04-01T00:00:20 | Timestamp(Nanosecond, None) | ++---------------------------+------------------------------------------------+---------------------+-----------------------------+ + +# combine `to_local_time()` with `date_bin()` to bin on boundaries in the timezone rather +# than UTC boundaries + +> SELECT date_bin(interval '1 day', to_local_time('2024-04-01T00:00:20Z'::timestamp AT TIME ZONE 'Europe/Brussels')) AS date_bin; ++---------------------+ +| date_bin | ++---------------------+ +| 2024-04-01T00:00:00 | ++---------------------+ + +> SELECT date_bin(interval '1 day', to_local_time('2024-04-01T00:00:20Z'::timestamp AT TIME ZONE 'Europe/Brussels')) AT TIME ZONE 'Europe/Brussels' AS date_bin_with_timezone; ++---------------------------+ +| date_bin_with_timezone | ++---------------------------+ +| 2024-04-01T00:00:00+02:00 | ++---------------------------+ +```"#, + argument( + name = "expression", + description = "Time expression to operate on. Can be a constant, column, or function." + ) +)] #[derive(Debug)] pub struct ToLocalTimeFunc { signature: Signature, @@ -359,72 +414,10 @@ impl ScalarUDFImpl for ToLocalTimeFunc { } } fn documentation(&self) -> Option<&Documentation> { - Some(get_to_local_time_doc()) + self.doc() } } -static DOCUMENTATION: OnceLock = OnceLock::new(); - -fn get_to_local_time_doc() -> &'static Documentation { - DOCUMENTATION.get_or_init(|| { - Documentation::builder( - DOC_SECTION_DATETIME, - "Converts a timestamp with a timezone to a timestamp without a timezone (with no offset or timezone information). This function handles daylight saving time changes.", - "to_local_time(expression)") - .with_argument( - "expression", - "Time expression to operate on. Can be a constant, column, or function." - ) - .with_sql_example(r#"```sql -> SELECT to_local_time('2024-04-01T00:00:20Z'::timestamp); -+---------------------------------------------+ -| to_local_time(Utf8("2024-04-01T00:00:20Z")) | -+---------------------------------------------+ -| 2024-04-01T00:00:20 | -+---------------------------------------------+ - -> SELECT to_local_time('2024-04-01T00:00:20Z'::timestamp AT TIME ZONE 'Europe/Brussels'); -+---------------------------------------------+ -| to_local_time(Utf8("2024-04-01T00:00:20Z")) | -+---------------------------------------------+ -| 2024-04-01T00:00:20 | -+---------------------------------------------+ - -> SELECT - time, - arrow_typeof(time) as type, - to_local_time(time) as to_local_time, - arrow_typeof(to_local_time(time)) as to_local_time_type -FROM ( - SELECT '2024-04-01T00:00:20Z'::timestamp AT TIME ZONE 'Europe/Brussels' AS time -); -+---------------------------+------------------------------------------------+---------------------+-----------------------------+ -| time | type | to_local_time | to_local_time_type | -+---------------------------+------------------------------------------------+---------------------+-----------------------------+ -| 2024-04-01T00:00:20+02:00 | Timestamp(Nanosecond, Some("Europe/Brussels")) | 2024-04-01T00:00:20 | Timestamp(Nanosecond, None) | -+---------------------------+------------------------------------------------+---------------------+-----------------------------+ - -# combine `to_local_time()` with `date_bin()` to bin on boundaries in the timezone rather -# than UTC boundaries - -> SELECT date_bin(interval '1 day', to_local_time('2024-04-01T00:00:20Z'::timestamp AT TIME ZONE 'Europe/Brussels')) AS date_bin; -+---------------------+ -| date_bin | -+---------------------+ -| 2024-04-01T00:00:00 | -+---------------------+ - -> SELECT date_bin(interval '1 day', to_local_time('2024-04-01T00:00:20Z'::timestamp AT TIME ZONE 'Europe/Brussels')) AT TIME ZONE 'Europe/Brussels' AS date_bin_with_timezone; -+---------------------------+ -| date_bin_with_timezone | -+---------------------------+ -| 2024-04-01T00:00:00+02:00 | -+---------------------------+ -```"#) - .build() - }) -} - #[cfg(test)] mod tests { use std::sync::Arc; diff --git a/datafusion/functions/src/datetime/to_timestamp.rs b/datafusion/functions/src/datetime/to_timestamp.rs index 430d4a970810..9d88fc00b990 100644 --- a/datafusion/functions/src/datetime/to_timestamp.rs +++ b/datafusion/functions/src/datetime/to_timestamp.rs @@ -16,7 +16,7 @@ // under the License. use std::any::Any; -use std::sync::{Arc, OnceLock}; +use std::sync::Arc; use arrow::datatypes::DataType::*; use arrow::datatypes::TimeUnit::{Microsecond, Millisecond, Nanosecond, Second}; @@ -27,31 +27,180 @@ use arrow::datatypes::{ use crate::datetime::common::*; use datafusion_common::{exec_err, Result, ScalarType}; -use datafusion_expr::scalar_doc_sections::DOC_SECTION_DATETIME; use datafusion_expr::{ ColumnarValue, Documentation, ScalarUDFImpl, Signature, Volatility, }; +use datafusion_macros::user_doc; +#[user_doc( + doc_section(label = "Time and Date Functions"), + description = r#" +Converts a value to a timestamp (`YYYY-MM-DDT00:00:00Z`). Supports strings, integer, unsigned integer, and double types as input. Strings are parsed as RFC3339 (e.g. '2023-07-20T05:44:00') if no [Chrono formats] are provided. Integers, unsigned integers, and doubles are interpreted as seconds since the unix epoch (`1970-01-01T00:00:00Z`). Returns the corresponding timestamp. + +Note: `to_timestamp` returns `Timestamp(Nanosecond)`. The supported range for integer input is between `-9223372037` and `9223372036`. Supported range for string input is between `1677-09-21T00:12:44.0` and `2262-04-11T23:47:16.0`. Please use `to_timestamp_seconds` for the input outside of supported bounds. +"#, + syntax_example = "to_timestamp(expression[, ..., format_n])", + sql_example = r#"```sql +> select to_timestamp('2023-01-31T09:26:56.123456789-05:00'); ++-----------------------------------------------------------+ +| to_timestamp(Utf8("2023-01-31T09:26:56.123456789-05:00")) | ++-----------------------------------------------------------+ +| 2023-01-31T14:26:56.123456789 | ++-----------------------------------------------------------+ +> select to_timestamp('03:59:00.123456789 05-17-2023', '%c', '%+', '%H:%M:%S%.f %m-%d-%Y'); ++--------------------------------------------------------------------------------------------------------+ +| to_timestamp(Utf8("03:59:00.123456789 05-17-2023"),Utf8("%c"),Utf8("%+"),Utf8("%H:%M:%S%.f %m-%d-%Y")) | ++--------------------------------------------------------------------------------------------------------+ +| 2023-05-17T03:59:00.123456789 | ++--------------------------------------------------------------------------------------------------------+ +``` +Additional examples can be found [here](https://github.com/apache/datafusion/blob/main/datafusion-examples/examples/to_timestamp.rs) +"#, + argument( + name = "expression", + description = "Expression to operate on. Can be a constant, column, or function, and any combination of arithmetic operators." + ), + argument( + name = "format_n", + description = "Optional [Chrono format](https://docs.rs/chrono/latest/chrono/format/strftime/index.html) strings to use to parse the expression. Formats will be tried in the order they appear with the first successful one being returned. If none of the formats successfully parse the expression an error will be returned." + ) +)] #[derive(Debug)] pub struct ToTimestampFunc { signature: Signature, } +#[user_doc( + doc_section(label = "Time and Date Functions"), + description = "Converts a value to a timestamp (`YYYY-MM-DDT00:00:00.000Z`). Supports strings, integer, and unsigned integer types as input. Strings are parsed as RFC3339 (e.g. '2023-07-20T05:44:00') if no [Chrono format](https://docs.rs/chrono/latest/chrono/format/strftime/index.html)s are provided. Integers and unsigned integers are interpreted as seconds since the unix epoch (`1970-01-01T00:00:00Z`). Returns the corresponding timestamp.", + syntax_example = "to_timestamp_seconds(expression[, ..., format_n])", + sql_example = r#"```sql +> select to_timestamp_seconds('2023-01-31T09:26:56.123456789-05:00'); ++-------------------------------------------------------------------+ +| to_timestamp_seconds(Utf8("2023-01-31T09:26:56.123456789-05:00")) | ++-------------------------------------------------------------------+ +| 2023-01-31T14:26:56 | ++-------------------------------------------------------------------+ +> select to_timestamp_seconds('03:59:00.123456789 05-17-2023', '%c', '%+', '%H:%M:%S%.f %m-%d-%Y'); ++----------------------------------------------------------------------------------------------------------------+ +| to_timestamp_seconds(Utf8("03:59:00.123456789 05-17-2023"),Utf8("%c"),Utf8("%+"),Utf8("%H:%M:%S%.f %m-%d-%Y")) | ++----------------------------------------------------------------------------------------------------------------+ +| 2023-05-17T03:59:00 | ++----------------------------------------------------------------------------------------------------------------+ +``` +Additional examples can be found [here](https://github.com/apache/datafusion/blob/main/datafusion-examples/examples/to_timestamp.rs) +"#, + argument( + name = "expression", + description = "Expression to operate on. Can be a constant, column, or function, and any combination of arithmetic operators." + ), + argument( + name = "format_n", + description = "Optional [Chrono format](https://docs.rs/chrono/latest/chrono/format/strftime/index.html) strings to use to parse the expression. Formats will be tried in the order they appear with the first successful one being returned. If none of the formats successfully parse the expression an error will be returned." + ) +)] #[derive(Debug)] pub struct ToTimestampSecondsFunc { signature: Signature, } +#[user_doc( + doc_section(label = "Time and Date Functions"), + description = "Converts a value to a timestamp (`YYYY-MM-DDT00:00:00.000Z`). Supports strings, integer, and unsigned integer types as input. Strings are parsed as RFC3339 (e.g. '2023-07-20T05:44:00') if no [Chrono formats](https://docs.rs/chrono/latest/chrono/format/strftime/index.html) are provided. Integers and unsigned integers are interpreted as milliseconds since the unix epoch (`1970-01-01T00:00:00Z`). Returns the corresponding timestamp.", + syntax_example = "to_timestamp_millis(expression[, ..., format_n])", + sql_example = r#"```sql +> select to_timestamp_millis('2023-01-31T09:26:56.123456789-05:00'); ++------------------------------------------------------------------+ +| to_timestamp_millis(Utf8("2023-01-31T09:26:56.123456789-05:00")) | ++------------------------------------------------------------------+ +| 2023-01-31T14:26:56.123 | ++------------------------------------------------------------------+ +> select to_timestamp_millis('03:59:00.123456789 05-17-2023', '%c', '%+', '%H:%M:%S%.f %m-%d-%Y'); ++---------------------------------------------------------------------------------------------------------------+ +| to_timestamp_millis(Utf8("03:59:00.123456789 05-17-2023"),Utf8("%c"),Utf8("%+"),Utf8("%H:%M:%S%.f %m-%d-%Y")) | ++---------------------------------------------------------------------------------------------------------------+ +| 2023-05-17T03:59:00.123 | ++---------------------------------------------------------------------------------------------------------------+ +``` +Additional examples can be found [here](https://github.com/apache/datafusion/blob/main/datafusion-examples/examples/to_timestamp.rs) +"#, + argument( + name = "expression", + description = "Expression to operate on. Can be a constant, column, or function, and any combination of arithmetic operators." + ), + argument( + name = "format_n", + description = "Optional [Chrono format](https://docs.rs/chrono/latest/chrono/format/strftime/index.html) strings to use to parse the expression. Formats will be tried in the order they appear with the first successful one being returned. If none of the formats successfully parse the expression an error will be returned." + ) +)] #[derive(Debug)] pub struct ToTimestampMillisFunc { signature: Signature, } +#[user_doc( + doc_section(label = "Time and Date Functions"), + description = "Converts a value to a timestamp (`YYYY-MM-DDT00:00:00.000000Z`). Supports strings, integer, and unsigned integer types as input. Strings are parsed as RFC3339 (e.g. '2023-07-20T05:44:00') if no [Chrono format](https://docs.rs/chrono/latest/chrono/format/strftime/index.html)s are provided. Integers and unsigned integers are interpreted as microseconds since the unix epoch (`1970-01-01T00:00:00Z`) Returns the corresponding timestamp.", + syntax_example = "to_timestamp_micros(expression[, ..., format_n])", + sql_example = r#"```sql +> select to_timestamp_micros('2023-01-31T09:26:56.123456789-05:00'); ++------------------------------------------------------------------+ +| to_timestamp_micros(Utf8("2023-01-31T09:26:56.123456789-05:00")) | ++------------------------------------------------------------------+ +| 2023-01-31T14:26:56.123456 | ++------------------------------------------------------------------+ +> select to_timestamp_micros('03:59:00.123456789 05-17-2023', '%c', '%+', '%H:%M:%S%.f %m-%d-%Y'); ++---------------------------------------------------------------------------------------------------------------+ +| to_timestamp_micros(Utf8("03:59:00.123456789 05-17-2023"),Utf8("%c"),Utf8("%+"),Utf8("%H:%M:%S%.f %m-%d-%Y")) | ++---------------------------------------------------------------------------------------------------------------+ +| 2023-05-17T03:59:00.123456 | ++---------------------------------------------------------------------------------------------------------------+ +``` +Additional examples can be found [here](https://github.com/apache/datafusion/blob/main/datafusion-examples/examples/to_timestamp.rs) +"#, + argument( + name = "expression", + description = "Expression to operate on. Can be a constant, column, or function, and any combination of arithmetic operators." + ), + argument( + name = "format_n", + description = "Optional [Chrono format](https://docs.rs/chrono/latest/chrono/format/strftime/index.html) strings to use to parse the expression. Formats will be tried in the order they appear with the first successful one being returned. If none of the formats successfully parse the expression an error will be returned." + ) +)] #[derive(Debug)] pub struct ToTimestampMicrosFunc { signature: Signature, } +#[user_doc( + doc_section(label = "Time and Date Functions"), + description = "Converts a value to a timestamp (`YYYY-MM-DDT00:00:00.000000000Z`). Supports strings, integer, and unsigned integer types as input. Strings are parsed as RFC3339 (e.g. '2023-07-20T05:44:00') if no [Chrono format](https://docs.rs/chrono/latest/chrono/format/strftime/index.html)s are provided. Integers and unsigned integers are interpreted as nanoseconds since the unix epoch (`1970-01-01T00:00:00Z`). Returns the corresponding timestamp.", + syntax_example = "to_timestamp_nanos(expression[, ..., format_n])", + sql_example = r#"```sql +> select to_timestamp_nanos('2023-01-31T09:26:56.123456789-05:00'); ++-----------------------------------------------------------------+ +| to_timestamp_nanos(Utf8("2023-01-31T09:26:56.123456789-05:00")) | ++-----------------------------------------------------------------+ +| 2023-01-31T14:26:56.123456789 | ++-----------------------------------------------------------------+ +> select to_timestamp_nanos('03:59:00.123456789 05-17-2023', '%c', '%+', '%H:%M:%S%.f %m-%d-%Y'); ++--------------------------------------------------------------------------------------------------------------+ +| to_timestamp_nanos(Utf8("03:59:00.123456789 05-17-2023"),Utf8("%c"),Utf8("%+"),Utf8("%H:%M:%S%.f %m-%d-%Y")) | ++--------------------------------------------------------------------------------------------------------------+ +| 2023-05-17T03:59:00.123456789 | ++---------------------------------------------------------------------------------------------------------------+ +``` +Additional examples can be found [here](https://github.com/apache/datafusion/blob/main/datafusion-examples/examples/to_timestamp.rs) +"#, + argument( + name = "expression", + description = "Expression to operate on. Can be a constant, column, or function, and any combination of arithmetic operators." + ), + argument( + name = "format_n", + description = "Optional [Chrono format](https://docs.rs/chrono/latest/chrono/format/strftime/index.html) strings to use to parse the expression. Formats will be tried in the order they appear with the first successful one being returned. If none of the formats successfully parse the expression an error will be returned." + ) +)] #[derive(Debug)] pub struct ToTimestampNanosFunc { signature: Signature, @@ -189,50 +338,10 @@ impl ScalarUDFImpl for ToTimestampFunc { } } fn documentation(&self) -> Option<&Documentation> { - Some(get_to_timestamp_doc()) + self.doc() } } -static DOCUMENTATION: OnceLock = OnceLock::new(); - -fn get_to_timestamp_doc() -> &'static Documentation { - DOCUMENTATION.get_or_init(|| { - Documentation::builder( - DOC_SECTION_DATETIME, - r#" -Converts a value to a timestamp (`YYYY-MM-DDT00:00:00Z`). Supports strings, integer, unsigned integer, and double types as input. Strings are parsed as RFC3339 (e.g. '2023-07-20T05:44:00') if no [Chrono formats] are provided. Integers, unsigned integers, and doubles are interpreted as seconds since the unix epoch (`1970-01-01T00:00:00Z`). Returns the corresponding timestamp. - -Note: `to_timestamp` returns `Timestamp(Nanosecond)`. The supported range for integer input is between `-9223372037` and `9223372036`. Supported range for string input is between `1677-09-21T00:12:44.0` and `2262-04-11T23:47:16.0`. Please use `to_timestamp_seconds` for the input outside of supported bounds. -"#, - "to_timestamp(expression[, ..., format_n])") - .with_argument( - "expression", - "Expression to operate on. Can be a constant, column, or function, and any combination of arithmetic operators." - ) - .with_argument( - "format_n", - "Optional [Chrono format](https://docs.rs/chrono/latest/chrono/format/strftime/index.html) strings to use to parse the expression. Formats will be tried in the order they appear with the first successful one being returned. If none of the formats successfully parse the expression an error will be returned.", - ) - .with_sql_example(r#"```sql -> select to_timestamp('2023-01-31T09:26:56.123456789-05:00'); -+-----------------------------------------------------------+ -| to_timestamp(Utf8("2023-01-31T09:26:56.123456789-05:00")) | -+-----------------------------------------------------------+ -| 2023-01-31T14:26:56.123456789 | -+-----------------------------------------------------------+ -> select to_timestamp('03:59:00.123456789 05-17-2023', '%c', '%+', '%H:%M:%S%.f %m-%d-%Y'); -+--------------------------------------------------------------------------------------------------------+ -| to_timestamp(Utf8("03:59:00.123456789 05-17-2023"),Utf8("%c"),Utf8("%+"),Utf8("%H:%M:%S%.f %m-%d-%Y")) | -+--------------------------------------------------------------------------------------------------------+ -| 2023-05-17T03:59:00.123456789 | -+--------------------------------------------------------------------------------------------------------+ -``` -Additional examples can be found [here](https://github.com/apache/datafusion/blob/main/datafusion-examples/examples/to_timestamp.rs) -"#) - .build() - }) -} - impl ScalarUDFImpl for ToTimestampSecondsFunc { fn as_any(&self) -> &dyn Any { self @@ -284,46 +393,10 @@ impl ScalarUDFImpl for ToTimestampSecondsFunc { } } fn documentation(&self) -> Option<&Documentation> { - Some(get_to_timestamp_seconds_doc()) + self.doc() } } -static TO_TIMESTAMP_SECONDS_DOC: OnceLock = OnceLock::new(); - -fn get_to_timestamp_seconds_doc() -> &'static Documentation { - TO_TIMESTAMP_SECONDS_DOC.get_or_init(|| { - Documentation::builder( - DOC_SECTION_DATETIME, - "Converts a value to a timestamp (`YYYY-MM-DDT00:00:00.000Z`). Supports strings, integer, and unsigned integer types as input. Strings are parsed as RFC3339 (e.g. '2023-07-20T05:44:00') if no [Chrono format](https://docs.rs/chrono/latest/chrono/format/strftime/index.html)s are provided. Integers and unsigned integers are interpreted as seconds since the unix epoch (`1970-01-01T00:00:00Z`). Returns the corresponding timestamp.", - "to_timestamp_seconds(expression[, ..., format_n])") - .with_argument( - "expression", - "Expression to operate on. Can be a constant, column, or function, and any combination of arithmetic operators." - ) - .with_argument( - "format_n", - "Optional [Chrono format](https://docs.rs/chrono/latest/chrono/format/strftime/index.html) strings to use to parse the expression. Formats will be tried in the order they appear with the first successful one being returned. If none of the formats successfully parse the expression an error will be returned.", - ) - .with_sql_example(r#"```sql -> select to_timestamp_seconds('2023-01-31T09:26:56.123456789-05:00'); -+-------------------------------------------------------------------+ -| to_timestamp_seconds(Utf8("2023-01-31T09:26:56.123456789-05:00")) | -+-------------------------------------------------------------------+ -| 2023-01-31T14:26:56 | -+-------------------------------------------------------------------+ -> select to_timestamp_seconds('03:59:00.123456789 05-17-2023', '%c', '%+', '%H:%M:%S%.f %m-%d-%Y'); -+----------------------------------------------------------------------------------------------------------------+ -| to_timestamp_seconds(Utf8("03:59:00.123456789 05-17-2023"),Utf8("%c"),Utf8("%+"),Utf8("%H:%M:%S%.f %m-%d-%Y")) | -+----------------------------------------------------------------------------------------------------------------+ -| 2023-05-17T03:59:00 | -+----------------------------------------------------------------------------------------------------------------+ -``` -Additional examples can be found [here](https://github.com/apache/datafusion/blob/main/datafusion-examples/examples/to_timestamp.rs) -"#) - .build() - }) -} - impl ScalarUDFImpl for ToTimestampMillisFunc { fn as_any(&self) -> &dyn Any { self @@ -377,46 +450,10 @@ impl ScalarUDFImpl for ToTimestampMillisFunc { } } fn documentation(&self) -> Option<&Documentation> { - Some(get_to_timestamp_millis_doc()) + self.doc() } } -static TO_TIMESTAMP_MILLIS_DOC: OnceLock = OnceLock::new(); - -fn get_to_timestamp_millis_doc() -> &'static Documentation { - TO_TIMESTAMP_MILLIS_DOC.get_or_init(|| { - Documentation::builder( - DOC_SECTION_DATETIME, - "Converts a value to a timestamp (`YYYY-MM-DDT00:00:00.000Z`). Supports strings, integer, and unsigned integer types as input. Strings are parsed as RFC3339 (e.g. '2023-07-20T05:44:00') if no [Chrono formats](https://docs.rs/chrono/latest/chrono/format/strftime/index.html) are provided. Integers and unsigned integers are interpreted as milliseconds since the unix epoch (`1970-01-01T00:00:00Z`). Returns the corresponding timestamp.", - "to_timestamp_millis(expression[, ..., format_n])") - .with_argument( - "expression", - "Expression to operate on. Can be a constant, column, or function, and any combination of arithmetic operators." - ) - .with_argument( - "format_n", - "Optional [Chrono format](https://docs.rs/chrono/latest/chrono/format/strftime/index.html) strings to use to parse the expression. Formats will be tried in the order they appear with the first successful one being returned. If none of the formats successfully parse the expression an error will be returned.", - ) - .with_sql_example(r#"```sql -> select to_timestamp_millis('2023-01-31T09:26:56.123456789-05:00'); -+------------------------------------------------------------------+ -| to_timestamp_millis(Utf8("2023-01-31T09:26:56.123456789-05:00")) | -+------------------------------------------------------------------+ -| 2023-01-31T14:26:56.123 | -+------------------------------------------------------------------+ -> select to_timestamp_millis('03:59:00.123456789 05-17-2023', '%c', '%+', '%H:%M:%S%.f %m-%d-%Y'); -+---------------------------------------------------------------------------------------------------------------+ -| to_timestamp_millis(Utf8("03:59:00.123456789 05-17-2023"),Utf8("%c"),Utf8("%+"),Utf8("%H:%M:%S%.f %m-%d-%Y")) | -+---------------------------------------------------------------------------------------------------------------+ -| 2023-05-17T03:59:00.123 | -+---------------------------------------------------------------------------------------------------------------+ -``` -Additional examples can be found [here](https://github.com/apache/datafusion/blob/main/datafusion-examples/examples/to_timestamp.rs) -"#) - .build() - }) -} - impl ScalarUDFImpl for ToTimestampMicrosFunc { fn as_any(&self) -> &dyn Any { self @@ -470,46 +507,10 @@ impl ScalarUDFImpl for ToTimestampMicrosFunc { } } fn documentation(&self) -> Option<&Documentation> { - Some(get_to_timestamp_micros_doc()) + self.doc() } } -static TO_TIMESTAMP_MICROS_DOC: OnceLock = OnceLock::new(); - -fn get_to_timestamp_micros_doc() -> &'static Documentation { - TO_TIMESTAMP_MICROS_DOC.get_or_init(|| { - Documentation::builder( - DOC_SECTION_DATETIME, - "Converts a value to a timestamp (`YYYY-MM-DDT00:00:00.000000Z`). Supports strings, integer, and unsigned integer types as input. Strings are parsed as RFC3339 (e.g. '2023-07-20T05:44:00') if no [Chrono format](https://docs.rs/chrono/latest/chrono/format/strftime/index.html)s are provided. Integers and unsigned integers are interpreted as microseconds since the unix epoch (`1970-01-01T00:00:00Z`) Returns the corresponding timestamp.", - "to_timestamp_micros(expression[, ..., format_n])") - .with_argument( - "expression", - "Expression to operate on. Can be a constant, column, or function, and any combination of arithmetic operators." - ) - .with_argument( - "format_n", - "Optional [Chrono format](https://docs.rs/chrono/latest/chrono/format/strftime/index.html) strings to use to parse the expression. Formats will be tried in the order they appear with the first successful one being returned. If none of the formats successfully parse the expression an error will be returned.", - ) - .with_sql_example(r#"```sql -> select to_timestamp_micros('2023-01-31T09:26:56.123456789-05:00'); -+------------------------------------------------------------------+ -| to_timestamp_micros(Utf8("2023-01-31T09:26:56.123456789-05:00")) | -+------------------------------------------------------------------+ -| 2023-01-31T14:26:56.123456 | -+------------------------------------------------------------------+ -> select to_timestamp_micros('03:59:00.123456789 05-17-2023', '%c', '%+', '%H:%M:%S%.f %m-%d-%Y'); -+---------------------------------------------------------------------------------------------------------------+ -| to_timestamp_micros(Utf8("03:59:00.123456789 05-17-2023"),Utf8("%c"),Utf8("%+"),Utf8("%H:%M:%S%.f %m-%d-%Y")) | -+---------------------------------------------------------------------------------------------------------------+ -| 2023-05-17T03:59:00.123456 | -+---------------------------------------------------------------------------------------------------------------+ -``` -Additional examples can be found [here](https://github.com/apache/datafusion/blob/main/datafusion-examples/examples/to_timestamp.rs) -"#) - .build() - }) -} - impl ScalarUDFImpl for ToTimestampNanosFunc { fn as_any(&self) -> &dyn Any { self @@ -563,46 +564,10 @@ impl ScalarUDFImpl for ToTimestampNanosFunc { } } fn documentation(&self) -> Option<&Documentation> { - Some(get_to_timestamp_nanos_doc()) + self.doc() } } -static TO_TIMESTAMP_NANOS_DOC: OnceLock = OnceLock::new(); - -fn get_to_timestamp_nanos_doc() -> &'static Documentation { - TO_TIMESTAMP_NANOS_DOC.get_or_init(|| { - Documentation::builder( - DOC_SECTION_DATETIME, - "Converts a value to a timestamp (`YYYY-MM-DDT00:00:00.000000000Z`). Supports strings, integer, and unsigned integer types as input. Strings are parsed as RFC3339 (e.g. '2023-07-20T05:44:00') if no [Chrono format](https://docs.rs/chrono/latest/chrono/format/strftime/index.html)s are provided. Integers and unsigned integers are interpreted as nanoseconds since the unix epoch (`1970-01-01T00:00:00Z`). Returns the corresponding timestamp.", - "to_timestamp_nanos(expression[, ..., format_n])") - .with_argument( - "expression", - "Expression to operate on. Can be a constant, column, or function, and any combination of arithmetic operators." - ) - .with_argument( - "format_n", - "Optional [Chrono format](https://docs.rs/chrono/latest/chrono/format/strftime/index.html) strings to use to parse the expression. Formats will be tried in the order they appear with the first successful one being returned. If none of the formats successfully parse the expression an error will be returned.", - ) - .with_sql_example(r#"```sql -> select to_timestamp_nanos('2023-01-31T09:26:56.123456789-05:00'); -+-----------------------------------------------------------------+ -| to_timestamp_nanos(Utf8("2023-01-31T09:26:56.123456789-05:00")) | -+-----------------------------------------------------------------+ -| 2023-01-31T14:26:56.123456789 | -+-----------------------------------------------------------------+ -> select to_timestamp_nanos('03:59:00.123456789 05-17-2023', '%c', '%+', '%H:%M:%S%.f %m-%d-%Y'); -+--------------------------------------------------------------------------------------------------------------+ -| to_timestamp_nanos(Utf8("03:59:00.123456789 05-17-2023"),Utf8("%c"),Utf8("%+"),Utf8("%H:%M:%S%.f %m-%d-%Y")) | -+--------------------------------------------------------------------------------------------------------------+ -| 2023-05-17T03:59:00.123456789 | -+---------------------------------------------------------------------------------------------------------------+ -``` -Additional examples can be found [here](https://github.com/apache/datafusion/blob/main/datafusion-examples/examples/to_timestamp.rs) -"#) - .build() - }) -} - /// Returns the return type for the to_timestamp_* function, preserving /// the timezone if it exists. fn return_type_for(arg: &DataType, unit: TimeUnit) -> DataType { diff --git a/datafusion/functions/src/datetime/to_unixtime.rs b/datafusion/functions/src/datetime/to_unixtime.rs index 9e6453a597c3..6776981bc74a 100644 --- a/datafusion/functions/src/datetime/to_unixtime.rs +++ b/datafusion/functions/src/datetime/to_unixtime.rs @@ -19,13 +19,41 @@ use super::to_timestamp::ToTimestampSecondsFunc; use crate::datetime::common::*; use arrow::datatypes::{DataType, TimeUnit}; use datafusion_common::{exec_err, Result}; -use datafusion_expr::scalar_doc_sections::DOC_SECTION_DATETIME; use datafusion_expr::{ ColumnarValue, Documentation, ScalarUDFImpl, Signature, Volatility, }; +use datafusion_macros::user_doc; use std::any::Any; -use std::sync::OnceLock; +#[user_doc( + doc_section(label = "Time and Date Functions"), + description = "Converts a value to seconds since the unix epoch (`1970-01-01T00:00:00Z`). Supports strings, dates, timestamps and double types as input. Strings are parsed as RFC3339 (e.g. '2023-07-20T05:44:00') if no [Chrono formats](https://docs.rs/chrono/latest/chrono/format/strftime/index.html) are provided.", + syntax_example = "to_unixtime(expression[, ..., format_n])", + sql_example = r#" +```sql +> select to_unixtime('2020-09-08T12:00:00+00:00'); ++------------------------------------------------+ +| to_unixtime(Utf8("2020-09-08T12:00:00+00:00")) | ++------------------------------------------------+ +| 1599566400 | ++------------------------------------------------+ +> select to_unixtime('01-14-2023 01:01:30+05:30', '%q', '%d-%m-%Y %H/%M/%S', '%+', '%m-%d-%Y %H:%M:%S%#z'); ++-----------------------------------------------------------------------------------------------------------------------------+ +| to_unixtime(Utf8("01-14-2023 01:01:30+05:30"),Utf8("%q"),Utf8("%d-%m-%Y %H/%M/%S"),Utf8("%+"),Utf8("%m-%d-%Y %H:%M:%S%#z")) | ++-----------------------------------------------------------------------------------------------------------------------------+ +| 1673638290 | ++-----------------------------------------------------------------------------------------------------------------------------+ +``` +"#, + argument( + name = "expression", + description = "Expression to operate on. Can be a constant, column, or function, and any combination of arithmetic operators." + ), + argument( + name = "format_n", + description = "Optional [Chrono format](https://docs.rs/chrono/latest/chrono/format/strftime/index.html) strings to use to parse the expression. Formats will be tried in the order they appear with the first successful one being returned. If none of the formats successfully parse the expression an error will be returned." + ) +)] #[derive(Debug)] pub struct ToUnixtimeFunc { signature: Signature, @@ -93,40 +121,6 @@ impl ScalarUDFImpl for ToUnixtimeFunc { } } fn documentation(&self) -> Option<&Documentation> { - Some(get_to_unixtime_doc()) + self.doc() } } - -static DOCUMENTATION: OnceLock = OnceLock::new(); - -fn get_to_unixtime_doc() -> &'static Documentation { - DOCUMENTATION.get_or_init(|| { - Documentation::builder( - DOC_SECTION_DATETIME, - "Converts a value to seconds since the unix epoch (`1970-01-01T00:00:00Z`). Supports strings, dates, timestamps and double types as input. Strings are parsed as RFC3339 (e.g. '2023-07-20T05:44:00') if no [Chrono formats](https://docs.rs/chrono/latest/chrono/format/strftime/index.html) are provided.", - "to_unixtime(expression[, ..., format_n])") - .with_argument( - "expression", - "Expression to operate on. Can be a constant, column, or function, and any combination of arithmetic operators." - ).with_argument( - "format_n", - "Optional [Chrono format](https://docs.rs/chrono/latest/chrono/format/strftime/index.html) strings to use to parse the expression. Formats will be tried in the order they appear with the first successful one being returned. If none of the formats successfully parse the expression an error will be returned.") - .with_sql_example(r#" -```sql -> select to_unixtime('2020-09-08T12:00:00+00:00'); -+------------------------------------------------+ -| to_unixtime(Utf8("2020-09-08T12:00:00+00:00")) | -+------------------------------------------------+ -| 1599566400 | -+------------------------------------------------+ -> select to_unixtime('01-14-2023 01:01:30+05:30', '%q', '%d-%m-%Y %H/%M/%S', '%+', '%m-%d-%Y %H:%M:%S%#z'); -+-----------------------------------------------------------------------------------------------------------------------------+ -| to_unixtime(Utf8("01-14-2023 01:01:30+05:30"),Utf8("%q"),Utf8("%d-%m-%Y %H/%M/%S"),Utf8("%+"),Utf8("%m-%d-%Y %H:%M:%S%#z")) | -+-----------------------------------------------------------------------------------------------------------------------------+ -| 1673638290 | -+-----------------------------------------------------------------------------------------------------------------------------+ -``` -"#) - .build() - }) -} diff --git a/docs/source/user-guide/sql/scalar_functions.md b/docs/source/user-guide/sql/scalar_functions.md index c4501fff8f78..ac0978683c36 100644 --- a/docs/source/user-guide/sql/scalar_functions.md +++ b/docs/source/user-guide/sql/scalar_functions.md @@ -2231,17 +2231,17 @@ to_date('2017-05-31', '%Y-%m-%d') ```sql > select to_date('2023-01-31'); -+-----------------------------+ ++-------------------------------+ | to_date(Utf8("2023-01-31")) | -+-----------------------------+ -| 2023-01-31 | -+-----------------------------+ ++-------------------------------+ +| 2023-01-31 | ++-------------------------------+ > select to_date('2023/01/31', '%Y-%m-%d', '%Y/%m/%d'); -+---------------------------------------------------------------+ ++---------------------------------------------------------------------+ | to_date(Utf8("2023/01/31"),Utf8("%Y-%m-%d"),Utf8("%Y/%m/%d")) | -+---------------------------------------------------------------+ -| 2023-01-31 | -+---------------------------------------------------------------+ ++---------------------------------------------------------------------+ +| 2023-01-31 | ++---------------------------------------------------------------------+ ``` Additional examples can be found [here](https://github.com/apache/datafusion/blob/main/datafusion-examples/examples/to_date.rs) From aafec07e086463fc7ed72c704e9f7e367460618a Mon Sep 17 00:00:00 2001 From: Bruce Ritchie Date: Wed, 1 Jan 2025 07:51:54 -0500 Subject: [PATCH 04/17] Add sqlite test files, progress bar, and automatic postgres container management into sqllogictests (#13936) * Fix md5 return_type to only return Utf8 as per current code impl. * Add support for sqlite test files to sqllogictest * Force version 0.24.0 of sqllogictest dependency until issue with labels is fixed. * Removed workaround for bug that was fixed. * Git submodule update ... err update, link to sqlite tests. * Git submodule update * Readd submodule --------- Co-authored-by: Andrew Lamb --- .gitmodules | 4 + datafusion-testing | 1 + datafusion/sqllogictest/Cargo.toml | 9 +- datafusion/sqllogictest/README.md | 46 +- datafusion/sqllogictest/bin/sqllogictests.rs | 522 +++++++++++++++--- .../src/engines/datafusion_engine/runner.rs | 65 ++- .../src/engines/postgres_engine/mod.rs | 63 ++- .../contributor-guide/getting_started.md | 2 +- 8 files changed, 611 insertions(+), 101 deletions(-) create mode 160000 datafusion-testing diff --git a/.gitmodules b/.gitmodules index ec5d6208b8dd..037accdbe424 100644 --- a/.gitmodules +++ b/.gitmodules @@ -4,3 +4,7 @@ [submodule "testing"] path = testing url = https://github.com/apache/arrow-testing +[submodule "datafusion-testing"] + path = datafusion-testing + url = https://github.com/apache/datafusion-testing.git + branch = main diff --git a/datafusion-testing b/datafusion-testing new file mode 160000 index 000000000000..e2e320c9477a --- /dev/null +++ b/datafusion-testing @@ -0,0 +1 @@ +Subproject commit e2e320c9477a6d8ab09662eae255887733c0e304 diff --git a/datafusion/sqllogictest/Cargo.toml b/datafusion/sqllogictest/Cargo.toml index 5c7d909d5c43..1bb88a8bd44f 100644 --- a/datafusion/sqllogictest/Cargo.toml +++ b/datafusion/sqllogictest/Cargo.toml @@ -45,9 +45,11 @@ datafusion-common = { workspace = true, default-features = true } datafusion-common-runtime = { workspace = true, default-features = true } futures = { workspace = true } half = { workspace = true, default-features = true } +indicatif = "0.17" itertools = { workspace = true } log = { workspace = true } object_store = { workspace = true } +once_cell = { version = "1.20", optional = true } postgres-protocol = { version = "0.6.7", optional = true } postgres-types = { version = "0.2.8", features = ["derive", "with-chrono-0_4"], optional = true } rust_decimal = { version = "1.36.0", features = ["tokio-pg"] } @@ -56,6 +58,8 @@ rust_decimal = { version = "1.36.0", features = ["tokio-pg"] } sqllogictest = "=0.24.0" sqlparser = { workspace = true } tempfile = { workspace = true } +testcontainers = { version = "0.23", features = ["default"], optional = true } +testcontainers-modules = { version = "0.11", features = ["postgres"], optional = true } thiserror = "2.0.0" tokio = { workspace = true } tokio-postgres = { version = "0.7.12", optional = true } @@ -65,9 +69,12 @@ avro = ["datafusion/avro"] postgres = [ "bytes", "chrono", - "tokio-postgres", + "once_cell", "postgres-types", "postgres-protocol", + "testcontainers", + "testcontainers-modules", + "tokio-postgres", ] [dev-dependencies] diff --git a/datafusion/sqllogictest/README.md b/datafusion/sqllogictest/README.md index 885e92fee270..4a7dc09d7dd1 100644 --- a/datafusion/sqllogictest/README.md +++ b/datafusion/sqllogictest/README.md @@ -28,13 +28,14 @@ This crate is a submodule of DataFusion that contains an implementation of [sqll ## Overview This crate uses [sqllogictest-rs](https://github.com/risinglightdb/sqllogictest-rs) to parse and run `.slt` files in the -[`test_files`](test_files) directory of this crate. +[`test_files`](test_files) directory of this crate or the [`data/sqlite`](sqlite) +directory of the datafusion-testing crate. ## Testing setup 1. `rustup update stable` DataFusion uses the latest stable release of rust 2. `git submodule init` -3. `git submodule update` +3. `git submodule update --init --remote --recursive` ## Running tests: TLDR Examples @@ -160,7 +161,7 @@ cargo test --test sqllogictests -- information Test files that start with prefix `pg_compat_` verify compatibility with Postgres by running the same script files both with DataFusion and with Postgres -In order to run the sqllogictests running against a previously running Postgres instance, do: +In order to have the sqllogictest run against an existing running Postgres instance, do: ```shell PG_COMPAT=true PG_URI="postgresql://postgres@127.0.0.1/postgres" cargo test --features=postgres --test sqllogictests @@ -172,7 +173,7 @@ The environment variables: 2. `PG_URI` contains a `libpq` style connection string, whose format is described in [the docs](https://docs.rs/tokio-postgres/latest/tokio_postgres/config/struct.Config.html#url) -One way to create a suitable a posgres container in docker is to use +One way to create a suitable a postgres container in docker is to use the [Official Image](https://hub.docker.com/_/postgres) with a command such as the following. Note the collation **must** be set to `C` otherwise `ORDER BY` will not match DataFusion and the tests will diff. @@ -185,6 +186,15 @@ docker run \ postgres ``` +If you do not want to create a new postgres database and you have docker +installed you can skip providing a PG_URI env variable and the sqllogictest +runner will automatically create a temporary postgres docker container. +For example: + +```shell +PG_COMPAT=true cargo test --features=postgres --test sqllogictests +``` + ## Running Tests: `tpch` Test files in `tpch` directory runs against the `TPCH` data set (SF = @@ -205,6 +215,34 @@ Then you need to add `INCLUDE_TPCH=true` to run tpch tests: INCLUDE_TPCH=true cargo test --test sqllogictests ``` +## Running Tests: `sqlite` + +Test files in `data/sqlite` directory of the datafusion-testing crate were +sourced from the [sqlite test suite](https://www.sqlite.org/sqllogictest/dir?ci=tip) and have been cleansed and updated to +run within DataFusion's sqllogictest runner. + +To run the sqlite tests you need to increase the rust stack size and add +`INCLUDE_SQLITE=true` to run the sqlite tests: + +```shell +export RUST_MIN_STACK=30485760; +INCLUDE_SQLITE=true cargo test --test sqllogictests +``` + +Note that there are well over 5 million queries in these tests and running the +sqlite tests will take a long time. You may wish to run them in release-nonlto mode: + +```shell +INCLUDE_SQLITE=true cargo test --profile release-nonlto --test sqllogictests +``` + +The sqlite tests can also be run with the postgres runner to verify compatibility: + +```shell +export RUST_MIN_STACK=30485760; +PG_COMPAT=true INCLUDE_SQLITE=true cargo test --features=postgres --test sqllogictests +``` + ## Updating tests: Completion Mode In test script completion mode, `sqllogictests` reads a prototype script and runs the statements and queries against the diff --git a/datafusion/sqllogictest/bin/sqllogictests.rs b/datafusion/sqllogictest/bin/sqllogictests.rs index 066cc8ee9824..498539c1674a 100644 --- a/datafusion/sqllogictest/bin/sqllogictests.rs +++ b/datafusion/sqllogictest/bin/sqllogictests.rs @@ -16,57 +16,129 @@ // under the License. use clap::Parser; +use datafusion_common::instant::Instant; use datafusion_common::utils::get_available_parallelism; +use datafusion_common::{exec_datafusion_err, exec_err, DataFusionError, Result}; +use datafusion_common_runtime::SpawnedTask; use datafusion_sqllogictest::{DataFusion, TestContext}; use futures::stream::StreamExt; +use indicatif::{ + HumanDuration, MultiProgress, ProgressBar, ProgressDrawTarget, ProgressStyle, +}; use itertools::Itertools; -use log::info; -use sqllogictest::{strict_column_validator, Normalizer}; +use log::Level::{Info, Warn}; +use log::{info, log_enabled, warn}; +#[cfg(feature = "postgres")] +use once_cell::sync::Lazy; +use sqllogictest::{ + parse_file, strict_column_validator, AsyncDB, Condition, Normalizer, Record, + Validator, +}; +#[cfg(feature = "postgres")] +use std::env::set_var; use std::ffi::OsStr; use std::fs; +#[cfg(feature = "postgres")] +use std::future::Future; use std::path::{Path, PathBuf}; - -use datafusion_common::{exec_datafusion_err, exec_err, DataFusionError, Result}; -use datafusion_common_runtime::SpawnedTask; +#[cfg(feature = "postgres")] +use std::{env, thread}; +#[cfg(feature = "postgres")] +use testcontainers::core::IntoContainerPort; +#[cfg(feature = "postgres")] +use testcontainers::runners::AsyncRunner; +#[cfg(feature = "postgres")] +use testcontainers::ImageExt; +#[cfg(feature = "postgres")] +use tokio::sync::mpsc::{UnboundedReceiver, UnboundedSender}; +#[cfg(feature = "postgres")] +use tokio::sync::{mpsc, Mutex}; +#[cfg(feature = "postgres")] +use ContainerCommands::{FetchHost, FetchPort}; const TEST_DIRECTORY: &str = "test_files/"; +const DATAFUSION_TESTING_TEST_DIRECTORY: &str = "../../datafusion-testing/data/"; const PG_COMPAT_FILE_PREFIX: &str = "pg_compat_"; +const SQLITE_PREFIX: &str = "sqlite"; pub fn main() -> Result<()> { tokio::runtime::Builder::new_multi_thread() .enable_all() - .build() - .unwrap() + .build()? .block_on(run_tests()) } +// Trailing whitespace from lines in SLT will typically be removed, but do not fail if it is not +// If particular test wants to cover trailing whitespace on a value, +// it should project additional non-whitespace column on the right. #[allow(clippy::ptr_arg)] -fn normalizer(s: &String) -> String { - // Trailing whitespace from lines in SLT will typically be removed, but do not fail if it is not - // If particular test wants to cover trailing whitespace on a value, - // it should project additional non-whitespace column on the right. - s.trim_end().to_owned() +fn value_normalizer(s: &String) -> String { + s.trim_end().to_string() } -fn value_validator( +fn sqlite_value_validator( normalizer: Normalizer, actual: &[Vec], expected: &[String], ) -> bool { - let expected = expected.iter().map(normalizer).collect::>(); - let actual = actual + let normalized_expected = expected.iter().map(normalizer).collect::>(); + let normalized_actual = actual + .iter() + .map(|strs| strs.iter().map(normalizer).join(" ")) + .collect_vec(); + + if log_enabled!(Info) && normalized_actual != normalized_expected { + info!("sqlite validation failed. actual vs expected:"); + for i in 0..normalized_actual.len() { + info!("[{i}] {}", normalized_actual[i]); + info!( + "[{i}] {}", + if normalized_expected.len() >= i { + &normalized_expected[i] + } else { + "No more results" + } + ); + } + } + + normalized_actual == normalized_expected +} + +fn df_value_validator( + normalizer: Normalizer, + actual: &[Vec], + expected: &[String], +) -> bool { + let normalized_expected = expected.iter().map(normalizer).collect::>(); + let normalized_actual = actual .iter() .map(|strs| strs.iter().join(" ")) - // Editors do not preserve trailing whitespace, so expected may or may not lack it included - .map(|str| normalizer(&str)) - .collect::>(); - actual == expected + .map(|str| str.trim_end().to_string()) + .collect_vec(); + + if log_enabled!(Warn) && normalized_actual != normalized_expected { + warn!("df validation failed. actual vs expected:"); + for i in 0..normalized_actual.len() { + warn!("[{i}] {}", normalized_actual[i]); + warn!( + "[{i}] {}", + if normalized_expected.len() >= i { + &normalized_expected[i] + } else { + "No more results" + } + ); + } + } + + normalized_actual == normalized_expected } /// Sets up an empty directory at test_files/scratch/ /// creating it if needed and clearing any file contents if it exists /// This allows tests for inserting to external tables or copy to -/// to persist data to disk and have consistent state when running +/// persist data to disk and have consistent state when running /// a new test fn setup_scratch_dir(name: &Path) -> Result<()> { // go from copy.slt --> copy @@ -97,23 +169,89 @@ async fn run_tests() -> Result<()> { } options.warn_on_ignored(); + #[cfg(feature = "postgres")] + let start_pg_database = options.postgres_runner && !is_pg_uri_set(); + #[cfg(feature = "postgres")] + if start_pg_database { + info!("Starting postgres db ..."); + + thread::spawn(|| { + execute_blocking(start_postgres( + &POSTGRES_IN, + &POSTGRES_HOST, + &POSTGRES_PORT, + &POSTGRES_STOPPED, + )) + }); + + POSTGRES_IN.tx.send(FetchHost).unwrap(); + let db_host = POSTGRES_HOST.rx.lock().await.recv().await.unwrap(); + + POSTGRES_IN.tx.send(FetchPort).unwrap(); + let db_port = POSTGRES_PORT.rx.lock().await.recv().await.unwrap(); + + let pg_uri = format!("postgresql://postgres:postgres@{db_host}:{db_port}/test"); + info!("Postgres uri is {pg_uri}"); + + set_var("PG_URI", pg_uri); + } + // Run all tests in parallel, reporting failures at the end // // Doing so is safe because each slt file runs with its own // `SessionContext` and should not have side effects (like // modifying shared state like `/tmp/`) + let m = MultiProgress::with_draw_target(ProgressDrawTarget::stderr_with_hz(1)); + let m_style = ProgressStyle::with_template( + "[{elapsed_precise}] {bar:40.cyan/blue} {pos:>7}/{len:7} {msg}", + ) + .unwrap() + .progress_chars("##-"); + + let start = Instant::now(); + let errors: Vec<_> = futures::stream::iter(read_test_files(&options)?) .map(|test_file| { + let validator = if options.include_sqlite + && test_file.relative_path.starts_with(SQLITE_PREFIX) + { + sqlite_value_validator + } else { + df_value_validator + }; + + let m_clone = m.clone(); + let m_style_clone = m_style.clone(); + SpawnedTask::spawn(async move { - let file_path = test_file.relative_path.clone(); - let start = datafusion::common::instant::Instant::now(); match (options.postgres_runner, options.complete) { - (false, false) => run_test_file(test_file).await?, - (false, true) => run_complete_file(test_file).await?, - (true, false) => run_test_file_with_postgres(test_file).await?, - (true, true) => run_complete_file_with_postgres(test_file).await?, + (false, false) => { + run_test_file(test_file, validator, m_clone, m_style_clone) + .await? + } + (false, true) => { + run_complete_file(test_file, validator, m_clone, m_style_clone) + .await? + } + (true, false) => { + run_test_file_with_postgres( + test_file, + validator, + m_clone, + m_style_clone, + ) + .await? + } + (true, true) => { + run_complete_file_with_postgres( + test_file, + validator, + m_clone, + m_style_clone, + ) + .await? + } } - println!("Executed {:?}. Took {:?}", file_path, start.elapsed()); Ok(()) as Result<()> }) .join() @@ -136,6 +274,15 @@ async fn run_tests() -> Result<()> { .collect() .await; + m.println(format!("Completed in {}", HumanDuration(start.elapsed())))?; + + #[cfg(feature = "postgres")] + if start_pg_database { + println!("Stopping postgres db ..."); + POSTGRES_IN.tx.send(ContainerCommands::Stop).unwrap_or(()); + POSTGRES_STOPPED.rx.lock().await.recv().await; + } + // report on any errors if !errors.is_empty() { for e in &errors { @@ -147,60 +294,148 @@ async fn run_tests() -> Result<()> { } } -async fn run_test_file(test_file: TestFile) -> Result<()> { +#[cfg(feature = "postgres")] +fn is_pg_uri_set() -> bool { + match env::var("PG_URI") { + Ok(_) => true, + Err(_) => false, + } +} + +async fn run_test_file( + test_file: TestFile, + validator: Validator, + mp: MultiProgress, + mp_style: ProgressStyle, +) -> Result<()> { let TestFile { path, relative_path, } = test_file; - info!("Running with DataFusion runner: {}", path.display()); let Some(test_ctx) = TestContext::try_new_for_test_file(&relative_path).await else { info!("Skipping: {}", path.display()); return Ok(()); }; setup_scratch_dir(&relative_path)?; + + let count: u64 = get_record_count(&path, "Datafusion".to_string()); + let pb = mp.add(ProgressBar::new(count)); + + pb.set_style(mp_style); + pb.set_message(format!("{:?}", &relative_path)); + let mut runner = sqllogictest::Runner::new(|| async { Ok(DataFusion::new( test_ctx.session_ctx().clone(), relative_path.clone(), + pb.clone(), )) }); + runner.add_label("Datafusion"); runner.with_column_validator(strict_column_validator); - runner.with_normalizer(normalizer); - runner.with_validator(value_validator); - runner + runner.with_normalizer(value_normalizer); + runner.with_validator(validator); + + let res = runner .run_file_async(path) .await - .map_err(|e| DataFusionError::External(Box::new(e))) + .map_err(|e| DataFusionError::External(Box::new(e))); + + pb.finish_and_clear(); + + res +} + +fn get_record_count(path: &PathBuf, label: String) -> u64 { + let records: Vec::ColumnType>> = + parse_file(path).unwrap(); + let mut count: u64 = 0; + + records.iter().for_each(|rec| match rec { + Record::Query { conditions, .. } => { + if conditions.is_empty() + || !conditions.contains(&Condition::SkipIf { + label: label.clone(), + }) + || conditions.contains(&Condition::OnlyIf { + label: label.clone(), + }) + { + count += 1; + } + } + Record::Statement { conditions, .. } => { + if conditions.is_empty() + || !conditions.contains(&Condition::SkipIf { + label: label.clone(), + }) + || conditions.contains(&Condition::OnlyIf { + label: label.clone(), + }) + { + count += 1; + } + } + _ => {} + }); + + count } #[cfg(feature = "postgres")] -async fn run_test_file_with_postgres(test_file: TestFile) -> Result<()> { +async fn run_test_file_with_postgres( + test_file: TestFile, + validator: Validator, + mp: MultiProgress, + mp_style: ProgressStyle, +) -> Result<()> { use datafusion_sqllogictest::Postgres; let TestFile { path, relative_path, } = test_file; - info!("Running with Postgres runner: {}", path.display()); setup_scratch_dir(&relative_path)?; - let mut runner = - sqllogictest::Runner::new(|| Postgres::connect(relative_path.clone())); + + let count: u64 = get_record_count(&path, "postgresql".to_string()); + let pb = mp.add(ProgressBar::new(count)); + + pb.set_style(mp_style); + pb.set_message(format!("{:?}", &relative_path)); + + let mut runner = sqllogictest::Runner::new(|| { + Postgres::connect(relative_path.clone(), pb.clone()) + }); + runner.add_label("postgres"); runner.with_column_validator(strict_column_validator); - runner.with_normalizer(normalizer); - runner.with_validator(value_validator); + runner.with_normalizer(value_normalizer); + runner.with_validator(validator); runner .run_file_async(path) .await .map_err(|e| DataFusionError::External(Box::new(e)))?; + + pb.finish_and_clear(); + Ok(()) } #[cfg(not(feature = "postgres"))] -async fn run_test_file_with_postgres(_test_file: TestFile) -> Result<()> { +async fn run_test_file_with_postgres( + _test_file: TestFile, + _validator: Validator, + _mp: MultiProgress, + _mp_style: ProgressStyle, +) -> Result<()> { use datafusion_common::plan_err; plan_err!("Can not run with postgres as postgres feature is not enabled") } -async fn run_complete_file(test_file: TestFile) -> Result<()> { +async fn run_complete_file( + test_file: TestFile, + validator: Validator, + mp: MultiProgress, + mp_style: ProgressStyle, +) -> Result<()> { let TestFile { path, relative_path, @@ -213,30 +448,48 @@ async fn run_complete_file(test_file: TestFile) -> Result<()> { return Ok(()); }; setup_scratch_dir(&relative_path)?; + + let count: u64 = get_record_count(&path, "Datafusion".to_string()); + let pb = mp.add(ProgressBar::new(count)); + + pb.set_style(mp_style); + pb.set_message(format!("{:?}", &relative_path)); + let mut runner = sqllogictest::Runner::new(|| async { Ok(DataFusion::new( test_ctx.session_ctx().clone(), relative_path.clone(), + pb.clone(), )) }); + let col_separator = " "; - runner + let res = runner .update_test_file( path, col_separator, - value_validator, - normalizer, + validator, + value_normalizer, strict_column_validator, ) .await // Can't use e directly because it isn't marked Send, so turn it into a string. .map_err(|e| { DataFusionError::Execution(format!("Error completing {relative_path:?}: {e}")) - }) + }); + + pb.finish_and_clear(); + + res } #[cfg(feature = "postgres")] -async fn run_complete_file_with_postgres(test_file: TestFile) -> Result<()> { +async fn run_complete_file_with_postgres( + test_file: TestFile, + validator: Validator, + mp: MultiProgress, + mp_style: ProgressStyle, +) -> Result<()> { use datafusion_sqllogictest::Postgres; let TestFile { path, @@ -247,26 +500,48 @@ async fn run_complete_file_with_postgres(test_file: TestFile) -> Result<()> { path.display() ); setup_scratch_dir(&relative_path)?; - let mut runner = - sqllogictest::Runner::new(|| Postgres::connect(relative_path.clone())); + + let count: u64 = get_record_count(&path, "postgresql".to_string()); + let pb = mp.add(ProgressBar::new(count)); + + pb.set_style(mp_style); + pb.set_message(format!("{:?}", &relative_path)); + + let mut runner = sqllogictest::Runner::new(|| { + Postgres::connect(relative_path.clone(), pb.clone()) + }); + runner.add_label("postgres"); + runner.with_column_validator(strict_column_validator); + runner.with_normalizer(value_normalizer); + runner.with_validator(validator); + let col_separator = " "; - runner + let res = runner .update_test_file( path, col_separator, - value_validator, - normalizer, + validator, + value_normalizer, strict_column_validator, ) .await // Can't use e directly because it isn't marked Send, so turn it into a string. .map_err(|e| { DataFusionError::Execution(format!("Error completing {relative_path:?}: {e}")) - }) + }); + + pb.finish_and_clear(); + + res } #[cfg(not(feature = "postgres"))] -async fn run_complete_file_with_postgres(_test_file: TestFile) -> Result<()> { +async fn run_complete_file_with_postgres( + _test_file: TestFile, + _validator: Validator, + _mp: MultiProgress, + _mp_style: ProgressStyle, +) -> Result<()> { use datafusion_common::plan_err; plan_err!("Can not run with postgres as postgres feature is not enabled") } @@ -282,11 +557,14 @@ struct TestFile { impl TestFile { fn new(path: PathBuf) -> Self { - let relative_path = PathBuf::from( - path.to_string_lossy() - .strip_prefix(TEST_DIRECTORY) - .unwrap_or(""), - ); + let p = path.to_string_lossy(); + let relative_path = PathBuf::from(if p.starts_with(TEST_DIRECTORY) { + p.strip_prefix(TEST_DIRECTORY).unwrap() + } else if p.starts_with(DATAFUSION_TESTING_TEST_DIRECTORY) { + p.strip_prefix(DATAFUSION_TESTING_TEST_DIRECTORY).unwrap() + } else { + "" + }); Self { path, @@ -298,6 +576,14 @@ impl TestFile { self.path.extension() == Some(OsStr::new("slt")) } + fn check_sqlite(&self, options: &Options) -> bool { + if !self.relative_path.starts_with(SQLITE_PREFIX) { + return true; + } + + options.include_sqlite + } + fn check_tpch(&self, options: &Options) -> bool { if !self.relative_path.starts_with("tpch") { return true; @@ -310,15 +596,29 @@ impl TestFile { fn read_test_files<'a>( options: &'a Options, ) -> Result + 'a>> { - Ok(Box::new( - read_dir_recursive(TEST_DIRECTORY)? + let mut paths = read_dir_recursive(TEST_DIRECTORY)? + .into_iter() + .map(TestFile::new) + .filter(|f| options.check_test_file(&f.relative_path)) + .filter(|f| f.is_slt_file()) + .filter(|f| f.check_tpch(options)) + .filter(|f| f.check_sqlite(options)) + .filter(|f| options.check_pg_compat_file(f.path.as_path())) + .collect::>(); + if options.include_sqlite { + let mut sqlite_paths = read_dir_recursive(DATAFUSION_TESTING_TEST_DIRECTORY)? .into_iter() .map(TestFile::new) .filter(|f| options.check_test_file(&f.relative_path)) .filter(|f| f.is_slt_file()) - .filter(|f| f.check_tpch(options)) - .filter(|f| options.check_pg_compat_file(f.path.as_path())), - )) + .filter(|f| f.check_sqlite(options)) + .filter(|f| options.check_pg_compat_file(f.path.as_path())) + .collect::>(); + + paths.append(&mut sqlite_paths) + } + + Ok(Box::new(paths.into_iter())) } fn read_dir_recursive>(path: P) -> Result> { @@ -350,7 +650,7 @@ fn read_dir_recursive_impl(dst: &mut Vec, path: &Path) -> Result<()> { /// Parsed command line options /// -/// This structure attempts to mimic the command line options of the built in rust test runner +/// This structure attempts to mimic the command line options of the built-in rust test runner /// accepted by IDEs such as CLion that pass arguments /// /// See for more details @@ -367,6 +667,9 @@ struct Options { )] postgres_runner: bool, + #[clap(long, env = "INCLUDE_SQLITE", help = "Include sqlite files")] + include_sqlite: bool, + #[clap(long, env = "INCLUDE_TPCH", help = "Include tpch files")] include_tpch: bool, @@ -431,10 +734,13 @@ impl Options { .any(|filter| relative_path.to_string_lossy().contains(filter)) } - /// Postgres runner executes only tests in files with specific names + /// Postgres runner executes only tests in files with specific names or in + /// specific folders fn check_pg_compat_file(&self, path: &Path) -> bool { let file_name = path.file_name().unwrap().to_str().unwrap().to_string(); - !self.postgres_runner || file_name.starts_with(PG_COMPAT_FILE_PREFIX) + !self.postgres_runner + || file_name.starts_with(PG_COMPAT_FILE_PREFIX) + || (self.include_sqlite && path.to_string_lossy().contains(SQLITE_PREFIX)) } /// Logs warning messages to stdout if any ignored options are passed @@ -452,3 +758,87 @@ impl Options { } } } + +#[cfg(feature = "postgres")] +pub async fn start_postgres( + in_channel: &Channel, + host_channel: &Channel, + port_channel: &Channel, + stopped_channel: &Channel<()>, +) { + info!("Starting postgres test container with user postgres/postgres and db test"); + + let container = testcontainers_modules::postgres::Postgres::default() + .with_user("postgres") + .with_password("postgres") + .with_db_name("test") + .with_mapped_port(16432, 5432.tcp()) + .with_tag("17-alpine") + .start() + .await + .unwrap(); + // uncomment this if you are running docker in docker + // let host = "host.docker.internal".to_string(); + let host = container.get_host().await.unwrap().to_string(); + let port = container.get_host_port_ipv4(5432).await.unwrap(); + + let mut rx = in_channel.rx.lock().await; + while let Some(command) = rx.recv().await { + match command { + FetchHost => host_channel.tx.send(host.clone()).unwrap(), + FetchPort => port_channel.tx.send(port).unwrap(), + ContainerCommands::Stop => { + container.stop().await.unwrap(); + stopped_channel.tx.send(()).unwrap(); + rx.close(); + } + } + } +} + +#[cfg(feature = "postgres")] +#[derive(Debug)] +pub enum ContainerCommands { + FetchHost, + FetchPort, + Stop, +} + +#[cfg(feature = "postgres")] +pub struct Channel { + pub tx: UnboundedSender, + pub rx: Mutex>, +} + +#[cfg(feature = "postgres")] +pub fn channel() -> Channel { + let (tx, rx) = mpsc::unbounded_channel(); + Channel { + tx, + rx: Mutex::new(rx), + } +} + +#[cfg(feature = "postgres")] +pub fn execute_blocking(f: F) { + tokio::runtime::Builder::new_current_thread() + .enable_all() + .build() + .unwrap() + .block_on(f); +} + +#[cfg(feature = "postgres")] +pub struct HostPort { + pub host: String, + pub port: u16, +} + +#[cfg(feature = "postgres")] +static POSTGRES_IN: Lazy> = Lazy::new(channel); +#[cfg(feature = "postgres")] +static POSTGRES_HOST: Lazy> = Lazy::new(channel); +#[cfg(feature = "postgres")] +static POSTGRES_PORT: Lazy> = Lazy::new(channel); +#[cfg(feature = "postgres")] +static POSTGRES_STOPPED: Lazy> = Lazy::new(channel); diff --git a/datafusion/sqllogictest/src/engines/datafusion_engine/runner.rs b/datafusion/sqllogictest/src/engines/datafusion_engine/runner.rs index 5c24b49cfe86..e696058484a9 100644 --- a/datafusion/sqllogictest/src/engines/datafusion_engine/runner.rs +++ b/datafusion/sqllogictest/src/engines/datafusion_engine/runner.rs @@ -18,26 +18,49 @@ use std::sync::Arc; use std::{path::PathBuf, time::Duration}; +use super::{error::Result, normalize, DFSqlLogicTestError}; use arrow::record_batch::RecordBatch; use async_trait::async_trait; use datafusion::physical_plan::common::collect; use datafusion::physical_plan::execute_stream; use datafusion::prelude::SessionContext; -use log::info; +use indicatif::ProgressBar; +use log::Level::{Debug, Info}; +use log::{debug, log_enabled, warn}; use sqllogictest::DBOutput; - -use super::{error::Result, normalize, DFSqlLogicTestError}; +use tokio::time::Instant; use crate::engines::output::{DFColumnType, DFOutput}; pub struct DataFusion { ctx: SessionContext, relative_path: PathBuf, + pb: ProgressBar, } impl DataFusion { - pub fn new(ctx: SessionContext, relative_path: PathBuf) -> Self { - Self { ctx, relative_path } + pub fn new(ctx: SessionContext, relative_path: PathBuf, pb: ProgressBar) -> Self { + Self { + ctx, + relative_path, + pb, + } + } + + fn update_slow_count(&self) { + let msg = self.pb.message(); + let split: Vec<&str> = msg.split(" ").collect(); + let mut current_count = 0; + + if split.len() > 2 { + // third match will be current slow count + current_count = split[2].parse::().unwrap(); + } + + current_count += 1; + + self.pb + .set_message(format!("{} - {} took > 500 ms", split[0], current_count)); } } @@ -47,12 +70,32 @@ impl sqllogictest::AsyncDB for DataFusion { type ColumnType = DFColumnType; async fn run(&mut self, sql: &str) -> Result { - info!( - "[{}] Running query: \"{}\"", - self.relative_path.display(), - sql - ); - run_query(&self.ctx, sql).await + if log_enabled!(Debug) { + debug!( + "[{}] Running query: \"{}\"", + self.relative_path.display(), + sql + ); + } + + let start = Instant::now(); + let result = run_query(&self.ctx, sql).await; + let duration = start.elapsed(); + + if duration.gt(&Duration::from_millis(500)) { + self.update_slow_count(); + } + + self.pb.inc(1); + + if log_enabled!(Info) && duration.gt(&Duration::from_secs(2)) { + warn!( + "[{}] Running query took more than 2 sec ({duration:?}): \"{sql}\"", + self.relative_path.display() + ); + } + + result } /// Engine name of current database. diff --git a/datafusion/sqllogictest/src/engines/postgres_engine/mod.rs b/datafusion/sqllogictest/src/engines/postgres_engine/mod.rs index a490488cd764..1439695d62c6 100644 --- a/datafusion/sqllogictest/src/engines/postgres_engine/mod.rs +++ b/datafusion/sqllogictest/src/engines/postgres_engine/mod.rs @@ -15,22 +15,24 @@ // specific language governing permissions and limitations // under the License. -/// Postgres engine implementation for sqllogictest. -use std::path::{Path, PathBuf}; -use std::str::FromStr; - use async_trait::async_trait; use bytes::Bytes; use futures::{SinkExt, StreamExt}; -use log::debug; +use log::{debug, info}; use sqllogictest::DBOutput; +/// Postgres engine implementation for sqllogictest. +use std::path::{Path, PathBuf}; +use std::str::FromStr; +use std::time::Duration; use tokio::task::JoinHandle; use super::conversion::*; use crate::engines::output::{DFColumnType, DFOutput}; use chrono::{NaiveDate, NaiveDateTime, NaiveTime}; +use indicatif::ProgressBar; use postgres_types::Type; use rust_decimal::Decimal; +use tokio::time::Instant; use tokio_postgres::{Column, Row}; use types::PgRegtype; @@ -55,6 +57,7 @@ pub struct Postgres { join_handle: JoinHandle<()>, /// Relative test file path relative_path: PathBuf, + pb: ProgressBar, } impl Postgres { @@ -71,11 +74,11 @@ impl Postgres { /// ``` /// /// See https://docs.rs/tokio-postgres/latest/tokio_postgres/config/struct.Config.html#url for format - pub async fn connect(relative_path: PathBuf) -> Result { + pub async fn connect(relative_path: PathBuf, pb: ProgressBar) -> Result { let uri = std::env::var("PG_URI").map_or(PG_URI.to_string(), std::convert::identity); - debug!("Using postgres connection string: {uri}"); + info!("Using postgres connection string: {uri}"); let config = tokio_postgres::Config::from_str(&uri)?; @@ -113,6 +116,7 @@ impl Postgres { client, join_handle, relative_path, + pb, }) } @@ -181,6 +185,22 @@ impl Postgres { tx.commit().await?; Ok(DBOutput::StatementComplete(0)) } + + fn update_slow_count(&self) { + let msg = self.pb.message(); + let split: Vec<&str> = msg.split(" ").collect(); + let mut current_count = 0; + + if split.len() > 2 { + // second match will be current slow count + current_count += split[2].parse::().unwrap(); + } + + current_count += 1; + + self.pb + .set_message(format!("{} - {} took > 500 ms", split[0], current_count)); + } } /// remove single quotes from the start and end of the string @@ -194,16 +214,13 @@ fn no_quotes(t: &str) -> &str { /// return a schema name fn schema_name(relative_path: &Path) -> String { relative_path - .file_name() - .map(|name| { - name.to_string_lossy() - .chars() - .filter(|ch| ch.is_ascii_alphabetic()) - .collect::() - .trim_start_matches("pg_") - .to_string() - }) - .unwrap_or_else(|| "default_schema".to_string()) + .to_string_lossy() + .to_string() + .chars() + .filter(|ch| ch.is_ascii_alphanumeric()) + .collect::() + .trim_start_matches("pg_") + .to_string() } impl Drop for Postgres { @@ -221,7 +238,7 @@ impl sqllogictest::AsyncDB for Postgres { &mut self, sql: &str, ) -> Result, Self::Error> { - println!( + debug!( "[{}] Running query: \"{}\"", self.relative_path.display(), sql @@ -242,14 +259,24 @@ impl sqllogictest::AsyncDB for Postgres { }; if lower_sql.starts_with("copy") { + self.pb.inc(1); return self.run_copy_command(sql).await; } if !is_query_sql { self.client.execute(sql, &[]).await?; + self.pb.inc(1); return Ok(DBOutput::StatementComplete(0)); } + let start = Instant::now(); let rows = self.client.query(sql, &[]).await?; + let duration = start.elapsed(); + + if duration.gt(&Duration::from_millis(500)) { + self.update_slow_count(); + } + + self.pb.inc(1); let types: Vec = if rows.is_empty() { self.client diff --git a/docs/source/contributor-guide/getting_started.md b/docs/source/contributor-guide/getting_started.md index 696d6d3a0fe2..5d85e07f3eaa 100644 --- a/docs/source/contributor-guide/getting_started.md +++ b/docs/source/contributor-guide/getting_started.md @@ -74,7 +74,7 @@ Testing setup: - `rustup update stable` DataFusion uses the latest stable release of rust - `git submodule init` -- `git submodule update` +- `git submodule update --init --remote --recursive` Formatting instructions: From b54e648a7ee82e1ef292ef36df7d49902171b94f Mon Sep 17 00:00:00 2001 From: wiedld Date: Wed, 1 Jan 2025 08:03:11 -0500 Subject: [PATCH 05/17] Supporting writing schema metadata when writing Parquet in parallel (#13866) * refactor: make ParquetSink tests a bit more readable * chore(11770): add new ParquetOptions.skip_arrow_metadata * test(11770): demonstrate that the single threaded ParquetSink is already writing the arrow schema in the kv_meta, and allow disablement * refactor(11770): replace with new method, since the kv_metadata is inherent to TableParquetOptions and therefore we should explicitly make the API apparant that you have to include the arrow schema or not * fix(11770): fix parallel ParquetSink to encode arrow schema into the file metadata, based on the ParquetOptions * refactor(11770): provide deprecation warning for TryFrom * test(11770): update tests with new default to include arrow schema * refactor: including partitioning of arrow schema inserted into kv_metdata * test: update tests for new config prop, as well as the new file partition offsets based upon larger metadata * chore: avoid cloning in tests, and update code docs * refactor: return to the WriterPropertiesBuilder::TryFrom, and separately add the arrow_schema to the kv_metadata on the TableParquetOptions * refactor: require the arrow_schema key to be present in the kv_metadata, if is required by the configuration * chore: update configs.md * test: update tests to handle the (default) required arrow schema in the kv_metadata * chore: add reference to arrow-rs upstream PR --- datafusion-cli/Cargo.lock | 2 + datafusion/common/Cargo.toml | 2 + datafusion/common/src/config.rs | 20 ++ datafusion/common/src/file_options/mod.rs | 15 +- .../common/src/file_options/parquet_writer.rs | 113 ++++++- .../src/datasource/file_format/parquet.rs | 279 +++++++++++++----- .../proto/datafusion_common.proto | 1 + datafusion/proto-common/src/from_proto/mod.rs | 1 + .../proto-common/src/generated/pbjson.rs | 18 ++ .../proto-common/src/generated/prost.rs | 3 + datafusion/proto-common/src/to_proto/mod.rs | 1 + .../src/generated/datafusion_proto_common.rs | 3 + .../proto/src/logical_plan/file_formats.rs | 2 + .../test_files/information_schema.slt | 2 + .../test_files/repartition_scan.slt | 8 +- docs/source/user-guide/configs.md | 1 + 16 files changed, 380 insertions(+), 91 deletions(-) diff --git a/datafusion-cli/Cargo.lock b/datafusion-cli/Cargo.lock index 92dcf24708c6..4e2837e48178 100644 --- a/datafusion-cli/Cargo.lock +++ b/datafusion-cli/Cargo.lock @@ -1335,7 +1335,9 @@ dependencies = [ "arrow", "arrow-array", "arrow-buffer", + "arrow-ipc", "arrow-schema", + "base64 0.22.1", "half", "hashbrown 0.14.5", "indexmap", diff --git a/datafusion/common/Cargo.toml b/datafusion/common/Cargo.toml index b331a55a98d0..feba589082b0 100644 --- a/datafusion/common/Cargo.toml +++ b/datafusion/common/Cargo.toml @@ -53,7 +53,9 @@ apache-avro = { version = "0.17", default-features = false, features = [ arrow = { workspace = true } arrow-array = { workspace = true } arrow-buffer = { workspace = true } +arrow-ipc = { workspace = true } arrow-schema = { workspace = true } +base64 = "0.22.1" half = { workspace = true } hashbrown = { workspace = true } indexmap = { workspace = true } diff --git a/datafusion/common/src/config.rs b/datafusion/common/src/config.rs index 942aa308e200..4da6921ba53c 100644 --- a/datafusion/common/src/config.rs +++ b/datafusion/common/src/config.rs @@ -436,6 +436,12 @@ config_namespace! { /// valid values are "1.0" and "2.0" pub writer_version: String, default = "1.0".to_string() + /// (writing) Skip encoding the embedded arrow metadata in the KV_meta + /// + /// This is analogous to the `ArrowWriterOptions::with_skip_arrow_metadata`. + /// Refer to + pub skip_arrow_metadata: bool, default = false + /// (writing) Sets default parquet compression codec. /// Valid values are: uncompressed, snappy, gzip(level), /// lzo, brotli(level), lz4, zstd(level), and lz4_raw. @@ -1496,6 +1502,20 @@ impl TableParquetOptions { pub fn new() -> Self { Self::default() } + + /// Set whether the encoding of the arrow metadata should occur + /// during the writing of parquet. + /// + /// Default is to encode the arrow schema in the file kv_metadata. + pub fn with_skip_arrow_metadata(self, skip: bool) -> Self { + Self { + global: ParquetOptions { + skip_arrow_metadata: skip, + ..self.global + }, + ..self + } + } } impl ConfigField for TableParquetOptions { diff --git a/datafusion/common/src/file_options/mod.rs b/datafusion/common/src/file_options/mod.rs index 77781457d0d2..02667e016571 100644 --- a/datafusion/common/src/file_options/mod.rs +++ b/datafusion/common/src/file_options/mod.rs @@ -30,7 +30,6 @@ pub mod parquet_writer; mod tests { use std::collections::HashMap; - use super::parquet_writer::ParquetWriterOptions; use crate::{ config::{ConfigFileType, TableOptions}, file_options::{csv_writer::CsvWriterOptions, json_writer::JsonWriterOptions}, @@ -40,7 +39,7 @@ mod tests { use parquet::{ basic::{Compression, Encoding, ZstdLevel}, - file::properties::{EnabledStatistics, WriterVersion}, + file::properties::{EnabledStatistics, WriterPropertiesBuilder, WriterVersion}, schema::types::ColumnPath, }; @@ -79,8 +78,10 @@ mod tests { table_config.set_config_format(ConfigFileType::PARQUET); table_config.alter_with_string_hash_map(&option_map)?; - let parquet_options = ParquetWriterOptions::try_from(&table_config.parquet)?; - let properties = parquet_options.writer_options(); + let properties = WriterPropertiesBuilder::try_from( + &table_config.parquet.with_skip_arrow_metadata(true), + )? + .build(); // Verify the expected options propagated down to parquet crate WriterProperties struct assert_eq!(properties.max_row_group_size(), 123); @@ -184,8 +185,10 @@ mod tests { table_config.set_config_format(ConfigFileType::PARQUET); table_config.alter_with_string_hash_map(&option_map)?; - let parquet_options = ParquetWriterOptions::try_from(&table_config.parquet)?; - let properties = parquet_options.writer_options(); + let properties = WriterPropertiesBuilder::try_from( + &table_config.parquet.with_skip_arrow_metadata(true), + )? + .build(); let col1 = ColumnPath::from(vec!["col1".to_owned()]); let col2_nested = ColumnPath::from(vec!["col2".to_owned(), "nested".to_owned()]); diff --git a/datafusion/common/src/file_options/parquet_writer.rs b/datafusion/common/src/file_options/parquet_writer.rs index dd9d67d6bb47..46bce06470f3 100644 --- a/datafusion/common/src/file_options/parquet_writer.rs +++ b/datafusion/common/src/file_options/parquet_writer.rs @@ -17,18 +17,25 @@ //! Options related to how parquet files should be written +use base64::Engine; +use std::sync::Arc; + use crate::{ config::{ParquetOptions, TableParquetOptions}, - DataFusionError, Result, + DataFusionError, Result, _internal_datafusion_err, }; +use arrow_schema::Schema; use parquet::{ + arrow::ARROW_SCHEMA_META_KEY, basic::{BrotliLevel, GzipLevel, ZstdLevel}, - file::properties::{ - EnabledStatistics, WriterProperties, WriterPropertiesBuilder, WriterVersion, - DEFAULT_MAX_STATISTICS_SIZE, DEFAULT_STATISTICS_ENABLED, + file::{ + metadata::KeyValue, + properties::{ + EnabledStatistics, WriterProperties, WriterPropertiesBuilder, WriterVersion, + DEFAULT_MAX_STATISTICS_SIZE, DEFAULT_STATISTICS_ENABLED, + }, }, - format::KeyValue, schema::types::ColumnPath, }; @@ -51,6 +58,17 @@ impl ParquetWriterOptions { } } +impl TableParquetOptions { + /// Add the arrow schema to the parquet kv_metadata. + /// If already exists, then overwrites. + pub fn arrow_schema(&mut self, schema: &Arc) { + self.key_value_metadata.insert( + ARROW_SCHEMA_META_KEY.into(), + Some(encode_arrow_schema(schema)), + ); + } +} + impl TryFrom<&TableParquetOptions> for ParquetWriterOptions { type Error = DataFusionError; @@ -79,6 +97,14 @@ impl TryFrom<&TableParquetOptions> for WriterPropertiesBuilder { let mut builder = global.into_writer_properties_builder()?; + // check that the arrow schema is present in the kv_metadata, if configured to do so + if !global.skip_arrow_metadata + && !key_value_metadata.contains_key(ARROW_SCHEMA_META_KEY) + { + return Err(_internal_datafusion_err!("arrow schema was not added to the kv_metadata, even though it is required by configuration settings")); + } + + // add kv_meta, if any if !key_value_metadata.is_empty() { builder = builder.set_key_value_metadata(Some( key_value_metadata @@ -140,11 +166,38 @@ impl TryFrom<&TableParquetOptions> for WriterPropertiesBuilder { } } +/// Encodes the Arrow schema into the IPC format, and base64 encodes it +/// +/// TODO: use extern parquet's private method, once publicly available. +/// Refer to +fn encode_arrow_schema(schema: &Arc) -> String { + let options = arrow_ipc::writer::IpcWriteOptions::default(); + let mut dictionary_tracker = arrow_ipc::writer::DictionaryTracker::new(true); + let data_gen = arrow_ipc::writer::IpcDataGenerator::default(); + let mut serialized_schema = data_gen.schema_to_bytes_with_dictionary_tracker( + schema, + &mut dictionary_tracker, + &options, + ); + + // manually prepending the length to the schema as arrow uses the legacy IPC format + // TODO: change after addressing ARROW-9777 + let schema_len = serialized_schema.ipc_message.len(); + let mut len_prefix_schema = Vec::with_capacity(schema_len + 8); + len_prefix_schema.append(&mut vec![255u8, 255, 255, 255]); + len_prefix_schema.append((schema_len as u32).to_le_bytes().to_vec().as_mut()); + len_prefix_schema.append(&mut serialized_schema.ipc_message); + + base64::prelude::BASE64_STANDARD.encode(&len_prefix_schema) +} + impl ParquetOptions { /// Convert the global session options, [`ParquetOptions`], into a single write action's [`WriterPropertiesBuilder`]. /// /// The returned [`WriterPropertiesBuilder`] can then be further modified with additional options /// applied per column; a customization which is not applicable for [`ParquetOptions`]. + /// + /// Note that this method does not include the key_value_metadata from [`TableParquetOptions`]. pub fn into_writer_properties_builder(&self) -> Result { let ParquetOptions { data_pagesize_limit, @@ -177,6 +230,7 @@ impl ParquetOptions { bloom_filter_on_read: _, // reads not used for writer props schema_force_view_types: _, binary_as_string: _, // not used for writer props + skip_arrow_metadata: _, } = self; let mut builder = WriterProperties::builder() @@ -444,6 +498,7 @@ mod tests { bloom_filter_on_read: defaults.bloom_filter_on_read, schema_force_view_types: defaults.schema_force_view_types, binary_as_string: defaults.binary_as_string, + skip_arrow_metadata: defaults.skip_arrow_metadata, } } @@ -546,19 +601,55 @@ mod tests { bloom_filter_on_read: global_options_defaults.bloom_filter_on_read, schema_force_view_types: global_options_defaults.schema_force_view_types, binary_as_string: global_options_defaults.binary_as_string, + skip_arrow_metadata: global_options_defaults.skip_arrow_metadata, }, column_specific_options, key_value_metadata, } } + #[test] + fn table_parquet_opts_to_writer_props_skip_arrow_metadata() { + // TableParquetOptions, all props set to default + let mut table_parquet_opts = TableParquetOptions::default(); + assert!( + !table_parquet_opts.global.skip_arrow_metadata, + "default false, to not skip the arrow schema requirement" + ); + + // see errors without the schema added, using default settings + let should_error = WriterPropertiesBuilder::try_from(&table_parquet_opts); + assert!( + should_error.is_err(), + "should error without the required arrow schema in kv_metadata", + ); + + // succeeds if we permit skipping the arrow schema + table_parquet_opts = table_parquet_opts.with_skip_arrow_metadata(true); + let should_succeed = WriterPropertiesBuilder::try_from(&table_parquet_opts); + assert!( + should_succeed.is_ok(), + "should work with the arrow schema skipped by config", + ); + + // Set the arrow schema back to required + table_parquet_opts = table_parquet_opts.with_skip_arrow_metadata(false); + // add the arrow schema to the kv_meta + table_parquet_opts.arrow_schema(&Arc::new(Schema::empty())); + let should_succeed = WriterPropertiesBuilder::try_from(&table_parquet_opts); + assert!( + should_succeed.is_ok(), + "should work with the arrow schema included in TableParquetOptions", + ); + } + #[test] fn table_parquet_opts_to_writer_props() { // ParquetOptions, all props set to non-default let parquet_options = parquet_options_with_non_defaults(); // TableParquetOptions, using ParquetOptions for global settings - let key = "foo".to_string(); + let key = ARROW_SCHEMA_META_KEY.to_string(); let value = Some("bar".into()); let table_parquet_opts = TableParquetOptions { global: parquet_options.clone(), @@ -585,7 +676,7 @@ mod tests { #[test] fn test_defaults_match() { // ensure the global settings are the same - let default_table_writer_opts = TableParquetOptions::default(); + let mut default_table_writer_opts = TableParquetOptions::default(); let default_parquet_opts = ParquetOptions::default(); assert_eq!( default_table_writer_opts.global, @@ -593,6 +684,10 @@ mod tests { "should have matching defaults for TableParquetOptions.global and ParquetOptions", ); + // selectively skip the arrow_schema metadata, since the WriterProperties default has an empty kv_meta (no arrow schema) + default_table_writer_opts = + default_table_writer_opts.with_skip_arrow_metadata(true); + // WriterProperties::default, a.k.a. using extern parquet's defaults let default_writer_props = WriterProperties::new(); @@ -640,6 +735,7 @@ mod tests { session_config_from_writer_props(&default_writer_props); from_extern_parquet.global.created_by = same_created_by; from_extern_parquet.global.compression = Some("zstd(3)".into()); + from_extern_parquet.global.skip_arrow_metadata = true; assert_eq!( default_table_writer_opts, @@ -653,6 +749,7 @@ mod tests { // the TableParquetOptions::default, with only the bloom filter turned on let mut default_table_writer_opts = TableParquetOptions::default(); default_table_writer_opts.global.bloom_filter_on_write = true; + default_table_writer_opts.arrow_schema(&Arc::new(Schema::empty())); // add the required arrow schema let from_datafusion_defaults = WriterPropertiesBuilder::try_from(&default_table_writer_opts) .unwrap() @@ -681,6 +778,7 @@ mod tests { let mut default_table_writer_opts = TableParquetOptions::default(); default_table_writer_opts.global.bloom_filter_on_write = true; default_table_writer_opts.global.bloom_filter_fpp = Some(0.42); + default_table_writer_opts.arrow_schema(&Arc::new(Schema::empty())); // add the required arrow schema let from_datafusion_defaults = WriterPropertiesBuilder::try_from(&default_table_writer_opts) .unwrap() @@ -713,6 +811,7 @@ mod tests { let mut default_table_writer_opts = TableParquetOptions::default(); default_table_writer_opts.global.bloom_filter_on_write = true; default_table_writer_opts.global.bloom_filter_ndv = Some(42); + default_table_writer_opts.arrow_schema(&Arc::new(Schema::empty())); // add the required arrow schema let from_datafusion_defaults = WriterPropertiesBuilder::try_from(&default_table_writer_opts) .unwrap() diff --git a/datafusion/core/src/datasource/file_format/parquet.rs b/datafusion/core/src/datasource/file_format/parquet.rs index 383fd6575234..8f64bea39df7 100644 --- a/datafusion/core/src/datasource/file_format/parquet.rs +++ b/datafusion/core/src/datasource/file_format/parquet.rs @@ -45,7 +45,6 @@ use crate::physical_plan::{ use arrow::compute::sum; use datafusion_common::config::{ConfigField, ConfigFileType, TableParquetOptions}; -use datafusion_common::file_options::parquet_writer::ParquetWriterOptions; use datafusion_common::parsers::CompressionTypeVariant; use datafusion_common::stats::Precision; use datafusion_common::{ @@ -68,13 +67,13 @@ use log::debug; use object_store::buffered::BufWriter; use parquet::arrow::arrow_writer::{ compute_leaves, get_column_writers, ArrowColumnChunk, ArrowColumnWriter, - ArrowLeafColumn, + ArrowLeafColumn, ArrowWriterOptions, }; use parquet::arrow::{ arrow_to_parquet_schema, parquet_to_arrow_schema, AsyncArrowWriter, }; use parquet::file::metadata::{ParquetMetaData, ParquetMetaDataReader, RowGroupMetaData}; -use parquet::file::properties::WriterProperties; +use parquet::file::properties::{WriterProperties, WriterPropertiesBuilder}; use parquet::file::writer::SerializedFileWriter; use parquet::format::FileMetaData; use tokio::io::{AsyncWrite, AsyncWriteExt}; @@ -750,6 +749,28 @@ impl ParquetSink { } } + /// Create writer properties based upon configuration settings, + /// including partitioning and the inclusion of arrow schema metadata. + fn create_writer_props(&self) -> Result { + let schema = if self.parquet_options.global.allow_single_file_parallelism { + // If parallelizing writes, we may be also be doing hive style partitioning + // into multiple files which impacts the schema per file. + // Refer to `self.get_writer_schema()` + &self.get_writer_schema() + } else { + self.config.output_schema() + }; + + // TODO: avoid this clone in follow up PR, where the writer properties & schema + // are calculated once on `ParquetSink::new` + let mut parquet_opts = self.parquet_options.clone(); + if !self.parquet_options.global.skip_arrow_metadata { + parquet_opts.arrow_schema(schema); + } + + Ok(WriterPropertiesBuilder::try_from(&parquet_opts)?.build()) + } + /// Creates an AsyncArrowWriter which serializes a parquet file to an ObjectStore /// AsyncArrowWriters are used when individual parquet file serialization is not parallelized async fn create_async_arrow_writer( @@ -759,10 +780,14 @@ impl ParquetSink { parquet_props: WriterProperties, ) -> Result> { let buf_writer = BufWriter::new(object_store, location.clone()); - let writer = AsyncArrowWriter::try_new( + let options = ArrowWriterOptions::new() + .with_properties(parquet_props) + .with_skip_arrow_metadata(self.parquet_options.global.skip_arrow_metadata); + + let writer = AsyncArrowWriter::try_new_with_options( buf_writer, self.get_writer_schema(), - Some(parquet_props), + options, )?; Ok(writer) } @@ -788,7 +813,7 @@ impl DataSink for ParquetSink { data: SendableRecordBatchStream, context: &Arc, ) -> Result { - let parquet_props = ParquetWriterOptions::try_from(&self.parquet_options)?; + let parquet_props = self.create_writer_props()?; let object_store = context .runtime_env() @@ -832,7 +857,7 @@ impl DataSink for ParquetSink { .create_async_arrow_writer( &path, Arc::clone(&object_store), - parquet_props.writer_options().clone(), + parquet_props.clone(), ) .await?; let mut reservation = @@ -867,7 +892,7 @@ impl DataSink for ParquetSink { writer, rx, schema, - props.writer_options(), + &props, parallel_options_clone, pool, ) @@ -2335,42 +2360,74 @@ mod tests { async fn parquet_sink_write() -> Result<()> { let parquet_sink = create_written_parquet_sink("file:///").await?; - // assert written - let mut written = parquet_sink.written(); - let written = written.drain(); - assert_eq!( - written.len(), - 1, - "expected a single parquet files to be written, instead found {}", - written.len() - ); + // assert written to proper path + let (path, file_metadata) = get_written(parquet_sink)?; + let path_parts = path.parts().collect::>(); + assert_eq!(path_parts.len(), 1, "should not have path prefix"); // check the file metadata - let ( - path, - FileMetaData { - num_rows, - schema, - key_value_metadata, - .. + let expected_kv_meta = vec![ + // default is to include arrow schema + KeyValue { + key: "ARROW:schema".to_string(), + value: Some(ENCODED_ARROW_SCHEMA.to_string()), + }, + KeyValue { + key: "my-data".to_string(), + value: Some("stuff".to_string()), + }, + KeyValue { + key: "my-data-bool-key".to_string(), + value: None, }, - ) = written.take(1).next().unwrap(); + ]; + assert_file_metadata(file_metadata, &expected_kv_meta); + + Ok(()) + } + + #[tokio::test] + async fn parquet_sink_parallel_write() -> Result<()> { + let opts = ParquetOptions { + allow_single_file_parallelism: true, + maximum_parallel_row_group_writers: 2, + maximum_buffered_record_batches_per_stream: 2, + ..Default::default() + }; + + let parquet_sink = + create_written_parquet_sink_using_config("file:///", opts).await?; + + // assert written to proper path + let (path, file_metadata) = get_written(parquet_sink)?; let path_parts = path.parts().collect::>(); assert_eq!(path_parts.len(), 1, "should not have path prefix"); - assert_eq!(num_rows, 2, "file metadata to have 2 rows"); - assert!( - schema.iter().any(|col_schema| col_schema.name == "a"), - "output file metadata should contain col a" - ); - assert!( - schema.iter().any(|col_schema| col_schema.name == "b"), - "output file metadata should contain col b" - ); + // check the file metadata + let expected_kv_meta = vec![ + // default is to include arrow schema + KeyValue { + key: "ARROW:schema".to_string(), + value: Some(ENCODED_ARROW_SCHEMA.to_string()), + }, + KeyValue { + key: "my-data".to_string(), + value: Some("stuff".to_string()), + }, + KeyValue { + key: "my-data-bool-key".to_string(), + value: None, + }, + ]; + assert_file_metadata(file_metadata, &expected_kv_meta); - let mut key_value_metadata = key_value_metadata.unwrap(); - key_value_metadata.sort_by(|a, b| a.key.cmp(&b.key)); - let expected_metadata = vec![ + Ok(()) + } + + #[tokio::test] + async fn parquet_sink_write_insert_schema_into_metadata() -> Result<()> { + // expected kv metadata without schema + let expected_without = vec![ KeyValue { key: "my-data".to_string(), value: Some("stuff".to_string()), @@ -2380,7 +2437,63 @@ mod tests { value: None, }, ]; - assert_eq!(key_value_metadata, expected_metadata); + // expected kv metadata with schema + let expected_with = [ + vec![KeyValue { + key: "ARROW:schema".to_string(), + value: Some(ENCODED_ARROW_SCHEMA.to_string()), + }], + expected_without.clone(), + ] + .concat(); + + // single threaded write, skip insert + let opts = ParquetOptions { + allow_single_file_parallelism: false, + skip_arrow_metadata: true, + ..Default::default() + }; + let parquet_sink = + create_written_parquet_sink_using_config("file:///", opts).await?; + let (_, file_metadata) = get_written(parquet_sink)?; + assert_file_metadata(file_metadata, &expected_without); + + // single threaded write, do not skip insert + let opts = ParquetOptions { + allow_single_file_parallelism: false, + skip_arrow_metadata: false, + ..Default::default() + }; + let parquet_sink = + create_written_parquet_sink_using_config("file:///", opts).await?; + let (_, file_metadata) = get_written(parquet_sink)?; + assert_file_metadata(file_metadata, &expected_with); + + // multithreaded write, skip insert + let opts = ParquetOptions { + allow_single_file_parallelism: true, + maximum_parallel_row_group_writers: 2, + maximum_buffered_record_batches_per_stream: 2, + skip_arrow_metadata: true, + ..Default::default() + }; + let parquet_sink = + create_written_parquet_sink_using_config("file:///", opts).await?; + let (_, file_metadata) = get_written(parquet_sink)?; + assert_file_metadata(file_metadata, &expected_without); + + // multithreaded write, do not skip insert + let opts = ParquetOptions { + allow_single_file_parallelism: true, + maximum_parallel_row_group_writers: 2, + maximum_buffered_record_batches_per_stream: 2, + skip_arrow_metadata: false, + ..Default::default() + }; + let parquet_sink = + create_written_parquet_sink_using_config("file:///", opts).await?; + let (_, file_metadata) = get_written(parquet_sink)?; + assert_file_metadata(file_metadata, &expected_with); Ok(()) } @@ -2391,18 +2504,8 @@ mod tests { let file_path = format!("file:///path/to/{}", filename); let parquet_sink = create_written_parquet_sink(file_path.as_str()).await?; - // assert written - let mut written = parquet_sink.written(); - let written = written.drain(); - assert_eq!( - written.len(), - 1, - "expected a single parquet file to be written, instead found {}", - written.len() - ); - - let (path, ..) = written.take(1).next().unwrap(); - + // assert written to proper path + let (path, _) = get_written(parquet_sink)?; let path_parts = path.parts().collect::>(); assert_eq!( path_parts.len(), @@ -2420,18 +2523,8 @@ mod tests { let file_path = "file:///path/to"; let parquet_sink = create_written_parquet_sink(file_path).await?; - // assert written - let mut written = parquet_sink.written(); - let written = written.drain(); - assert_eq!( - written.len(), - 1, - "expected a single parquet file to be written, instead found {}", - written.len() - ); - - let (path, ..) = written.take(1).next().unwrap(); - + // assert written to proper path + let (path, _) = get_written(parquet_sink)?; let path_parts = path.parts().collect::>(); assert_eq!( path_parts.len(), @@ -2449,18 +2542,8 @@ mod tests { let file_path = "file:///path/to/"; let parquet_sink = create_written_parquet_sink(file_path).await?; - // assert written - let mut written = parquet_sink.written(); - let written = written.drain(); - assert_eq!( - written.len(), - 1, - "expected a single parquet file to be written, instead found {}", - written.len() - ); - - let (path, ..) = written.take(1).next().unwrap(); - + // assert written to proper path + let (path, _) = get_written(parquet_sink)?; let path_parts = path.parts().collect::>(); assert_eq!( path_parts.len(), @@ -2474,6 +2557,17 @@ mod tests { } async fn create_written_parquet_sink(table_path: &str) -> Result> { + create_written_parquet_sink_using_config(table_path, ParquetOptions::default()) + .await + } + + static ENCODED_ARROW_SCHEMA: &str = "/////5QAAAAQAAAAAAAKAAwACgAJAAQACgAAABAAAAAAAQQACAAIAAAABAAIAAAABAAAAAIAAAA8AAAABAAAANz///8UAAAADAAAAAAAAAUMAAAAAAAAAMz///8BAAAAYgAAABAAFAAQAAAADwAEAAAACAAQAAAAGAAAAAwAAAAAAAAFEAAAAAAAAAAEAAQABAAAAAEAAABhAAAA"; + + async fn create_written_parquet_sink_using_config( + table_path: &str, + global: ParquetOptions, + ) -> Result> { + // schema should match the ENCODED_ARROW_SCHEMA bove let field_a = Field::new("a", DataType::Utf8, false); let field_b = Field::new("b", DataType::Utf8, false); let schema = Arc::new(Schema::new(vec![field_a, field_b])); @@ -2495,6 +2589,7 @@ mod tests { ("my-data".to_string(), Some("stuff".to_string())), ("my-data-bool-key".to_string(), None), ]), + global, ..Default::default() }, )); @@ -2519,6 +2614,42 @@ mod tests { Ok(parquet_sink) } + fn get_written(parquet_sink: Arc) -> Result<(Path, FileMetaData)> { + let mut written = parquet_sink.written(); + let written = written.drain(); + assert_eq!( + written.len(), + 1, + "expected a single parquet files to be written, instead found {}", + written.len() + ); + + let (path, file_metadata) = written.take(1).next().unwrap(); + Ok((path, file_metadata)) + } + + fn assert_file_metadata(file_metadata: FileMetaData, expected_kv: &Vec) { + let FileMetaData { + num_rows, + schema, + key_value_metadata, + .. + } = file_metadata; + assert_eq!(num_rows, 2, "file metadata to have 2 rows"); + assert!( + schema.iter().any(|col_schema| col_schema.name == "a"), + "output file metadata should contain col a" + ); + assert!( + schema.iter().any(|col_schema| col_schema.name == "b"), + "output file metadata should contain col b" + ); + + let mut key_value_metadata = key_value_metadata.unwrap(); + key_value_metadata.sort_by(|a, b| a.key.cmp(&b.key)); + assert_eq!(&key_value_metadata, expected_kv); + } + #[tokio::test] async fn parquet_sink_write_partitions() -> Result<()> { let field_a = Field::new("a", DataType::Utf8, false); diff --git a/datafusion/proto-common/proto/datafusion_common.proto b/datafusion/proto-common/proto/datafusion_common.proto index 69626f97fd80..6a7dc1604b0a 100644 --- a/datafusion/proto-common/proto/datafusion_common.proto +++ b/datafusion/proto-common/proto/datafusion_common.proto @@ -497,6 +497,7 @@ message ParquetOptions { bool bloom_filter_on_write = 27; // default = false bool schema_force_view_types = 28; // default = false bool binary_as_string = 29; // default = false + bool skip_arrow_metadata = 30; // default = false oneof metadata_size_hint_opt { uint64 metadata_size_hint = 4; diff --git a/datafusion/proto-common/src/from_proto/mod.rs b/datafusion/proto-common/src/from_proto/mod.rs index eb6976aa0c06..ca8306275b11 100644 --- a/datafusion/proto-common/src/from_proto/mod.rs +++ b/datafusion/proto-common/src/from_proto/mod.rs @@ -962,6 +962,7 @@ impl TryFrom<&protobuf::ParquetOptions> for ParquetOptions { maximum_buffered_record_batches_per_stream: value.maximum_buffered_record_batches_per_stream as usize, schema_force_view_types: value.schema_force_view_types, binary_as_string: value.binary_as_string, + skip_arrow_metadata: value.skip_arrow_metadata, }) } } diff --git a/datafusion/proto-common/src/generated/pbjson.rs b/datafusion/proto-common/src/generated/pbjson.rs index e88c1497af08..e9f9de09d4d1 100644 --- a/datafusion/proto-common/src/generated/pbjson.rs +++ b/datafusion/proto-common/src/generated/pbjson.rs @@ -4940,6 +4940,9 @@ impl serde::Serialize for ParquetOptions { if self.binary_as_string { len += 1; } + if self.skip_arrow_metadata { + len += 1; + } if self.dictionary_page_size_limit != 0 { len += 1; } @@ -5033,6 +5036,9 @@ impl serde::Serialize for ParquetOptions { if self.binary_as_string { struct_ser.serialize_field("binaryAsString", &self.binary_as_string)?; } + if self.skip_arrow_metadata { + struct_ser.serialize_field("skipArrowMetadata", &self.skip_arrow_metadata)?; + } if self.dictionary_page_size_limit != 0 { #[allow(clippy::needless_borrow)] #[allow(clippy::needless_borrows_for_generic_args)] @@ -5161,6 +5167,8 @@ impl<'de> serde::Deserialize<'de> for ParquetOptions { "schemaForceViewTypes", "binary_as_string", "binaryAsString", + "skip_arrow_metadata", + "skipArrowMetadata", "dictionary_page_size_limit", "dictionaryPageSizeLimit", "data_page_row_count_limit", @@ -5204,6 +5212,7 @@ impl<'de> serde::Deserialize<'de> for ParquetOptions { BloomFilterOnWrite, SchemaForceViewTypes, BinaryAsString, + SkipArrowMetadata, DictionaryPageSizeLimit, DataPageRowCountLimit, MaxRowGroupSize, @@ -5253,6 +5262,7 @@ impl<'de> serde::Deserialize<'de> for ParquetOptions { "bloomFilterOnWrite" | "bloom_filter_on_write" => Ok(GeneratedField::BloomFilterOnWrite), "schemaForceViewTypes" | "schema_force_view_types" => Ok(GeneratedField::SchemaForceViewTypes), "binaryAsString" | "binary_as_string" => Ok(GeneratedField::BinaryAsString), + "skipArrowMetadata" | "skip_arrow_metadata" => Ok(GeneratedField::SkipArrowMetadata), "dictionaryPageSizeLimit" | "dictionary_page_size_limit" => Ok(GeneratedField::DictionaryPageSizeLimit), "dataPageRowCountLimit" | "data_page_row_count_limit" => Ok(GeneratedField::DataPageRowCountLimit), "maxRowGroupSize" | "max_row_group_size" => Ok(GeneratedField::MaxRowGroupSize), @@ -5300,6 +5310,7 @@ impl<'de> serde::Deserialize<'de> for ParquetOptions { let mut bloom_filter_on_write__ = None; let mut schema_force_view_types__ = None; let mut binary_as_string__ = None; + let mut skip_arrow_metadata__ = None; let mut dictionary_page_size_limit__ = None; let mut data_page_row_count_limit__ = None; let mut max_row_group_size__ = None; @@ -5413,6 +5424,12 @@ impl<'de> serde::Deserialize<'de> for ParquetOptions { } binary_as_string__ = Some(map_.next_value()?); } + GeneratedField::SkipArrowMetadata => { + if skip_arrow_metadata__.is_some() { + return Err(serde::de::Error::duplicate_field("skipArrowMetadata")); + } + skip_arrow_metadata__ = Some(map_.next_value()?); + } GeneratedField::DictionaryPageSizeLimit => { if dictionary_page_size_limit__.is_some() { return Err(serde::de::Error::duplicate_field("dictionaryPageSizeLimit")); @@ -5515,6 +5532,7 @@ impl<'de> serde::Deserialize<'de> for ParquetOptions { bloom_filter_on_write: bloom_filter_on_write__.unwrap_or_default(), schema_force_view_types: schema_force_view_types__.unwrap_or_default(), binary_as_string: binary_as_string__.unwrap_or_default(), + skip_arrow_metadata: skip_arrow_metadata__.unwrap_or_default(), dictionary_page_size_limit: dictionary_page_size_limit__.unwrap_or_default(), data_page_row_count_limit: data_page_row_count_limit__.unwrap_or_default(), max_row_group_size: max_row_group_size__.unwrap_or_default(), diff --git a/datafusion/proto-common/src/generated/prost.rs b/datafusion/proto-common/src/generated/prost.rs index 6b8509775847..3263c1c755af 100644 --- a/datafusion/proto-common/src/generated/prost.rs +++ b/datafusion/proto-common/src/generated/prost.rs @@ -763,6 +763,9 @@ pub struct ParquetOptions { /// default = false #[prost(bool, tag = "29")] pub binary_as_string: bool, + /// default = false + #[prost(bool, tag = "30")] + pub skip_arrow_metadata: bool, #[prost(uint64, tag = "12")] pub dictionary_page_size_limit: u64, #[prost(uint64, tag = "18")] diff --git a/datafusion/proto-common/src/to_proto/mod.rs b/datafusion/proto-common/src/to_proto/mod.rs index a7cea607cb6d..79faaba864f3 100644 --- a/datafusion/proto-common/src/to_proto/mod.rs +++ b/datafusion/proto-common/src/to_proto/mod.rs @@ -833,6 +833,7 @@ impl TryFrom<&ParquetOptions> for protobuf::ParquetOptions { maximum_buffered_record_batches_per_stream: value.maximum_buffered_record_batches_per_stream as u64, schema_force_view_types: value.schema_force_view_types, binary_as_string: value.binary_as_string, + skip_arrow_metadata: value.skip_arrow_metadata, }) } } diff --git a/datafusion/proto/src/generated/datafusion_proto_common.rs b/datafusion/proto/src/generated/datafusion_proto_common.rs index 6b8509775847..3263c1c755af 100644 --- a/datafusion/proto/src/generated/datafusion_proto_common.rs +++ b/datafusion/proto/src/generated/datafusion_proto_common.rs @@ -763,6 +763,9 @@ pub struct ParquetOptions { /// default = false #[prost(bool, tag = "29")] pub binary_as_string: bool, + /// default = false + #[prost(bool, tag = "30")] + pub skip_arrow_metadata: bool, #[prost(uint64, tag = "12")] pub dictionary_page_size_limit: u64, #[prost(uint64, tag = "18")] diff --git a/datafusion/proto/src/logical_plan/file_formats.rs b/datafusion/proto/src/logical_plan/file_formats.rs index 62405b2fef21..772e6d23426a 100644 --- a/datafusion/proto/src/logical_plan/file_formats.rs +++ b/datafusion/proto/src/logical_plan/file_formats.rs @@ -410,6 +410,7 @@ impl TableParquetOptionsProto { maximum_buffered_record_batches_per_stream: global_options.global.maximum_buffered_record_batches_per_stream as u64, schema_force_view_types: global_options.global.schema_force_view_types, binary_as_string: global_options.global.binary_as_string, + skip_arrow_metadata: global_options.global.skip_arrow_metadata, }), column_specific_options: column_specific_options.into_iter().map(|(column_name, options)| { ParquetColumnSpecificOptions { @@ -501,6 +502,7 @@ impl From<&ParquetOptionsProto> for ParquetOptions { maximum_buffered_record_batches_per_stream: proto.maximum_buffered_record_batches_per_stream as usize, schema_force_view_types: proto.schema_force_view_types, binary_as_string: proto.binary_as_string, + skip_arrow_metadata: proto.skip_arrow_metadata, } } } diff --git a/datafusion/sqllogictest/test_files/information_schema.slt b/datafusion/sqllogictest/test_files/information_schema.slt index 7d70cd9db53e..46618b32d77a 100644 --- a/datafusion/sqllogictest/test_files/information_schema.slt +++ b/datafusion/sqllogictest/test_files/information_schema.slt @@ -215,6 +215,7 @@ datafusion.execution.parquet.pruning true datafusion.execution.parquet.pushdown_filters false datafusion.execution.parquet.reorder_filters false datafusion.execution.parquet.schema_force_view_types true +datafusion.execution.parquet.skip_arrow_metadata false datafusion.execution.parquet.skip_metadata true datafusion.execution.parquet.statistics_enabled page datafusion.execution.parquet.write_batch_size 1024 @@ -308,6 +309,7 @@ datafusion.execution.parquet.pruning true (reading) If true, the parquet reader datafusion.execution.parquet.pushdown_filters false (reading) If true, filter expressions are be applied during the parquet decoding operation to reduce the number of rows decoded. This optimization is sometimes called "late materialization". datafusion.execution.parquet.reorder_filters false (reading) If true, filter expressions evaluated during the parquet decoding operation will be reordered heuristically to minimize the cost of evaluation. If false, the filters are applied in the same order as written in the query datafusion.execution.parquet.schema_force_view_types true (reading) If true, parquet reader will read columns of `Utf8/Utf8Large` with `Utf8View`, and `Binary/BinaryLarge` with `BinaryView`. +datafusion.execution.parquet.skip_arrow_metadata false (writing) Skip encoding the embedded arrow metadata in the KV_meta This is analogous to the `ArrowWriterOptions::with_skip_arrow_metadata`. Refer to datafusion.execution.parquet.skip_metadata true (reading) If true, the parquet reader skip the optional embedded metadata that may be in the file Schema. This setting can help avoid schema conflicts when querying multiple parquet files with schemas containing compatible types but different metadata datafusion.execution.parquet.statistics_enabled page (writing) Sets if statistics are enabled for any column Valid values are: "none", "chunk", and "page" These values are not case sensitive. If NULL, uses default parquet writer setting datafusion.execution.parquet.write_batch_size 1024 (writing) Sets write_batch_size in bytes diff --git a/datafusion/sqllogictest/test_files/repartition_scan.slt b/datafusion/sqllogictest/test_files/repartition_scan.slt index a1db84b87850..9ba96e985fe5 100644 --- a/datafusion/sqllogictest/test_files/repartition_scan.slt +++ b/datafusion/sqllogictest/test_files/repartition_scan.slt @@ -61,7 +61,7 @@ logical_plan physical_plan 01)CoalesceBatchesExec: target_batch_size=8192 02)--FilterExec: column1@0 != 42 -03)----ParquetExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:0..88], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:88..176], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:176..264], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:264..351]]}, projection=[column1], predicate=column1@0 != 42, pruning_predicate=column1_null_count@2 != column1_row_count@3 AND (column1_min@0 != 42 OR 42 != column1_max@1), required_guarantees=[column1 not in (42)] +03)----ParquetExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:0..137], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:137..274], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:274..411], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:411..547]]}, projection=[column1], predicate=column1@0 != 42, pruning_predicate=column1_null_count@2 != column1_row_count@3 AND (column1_min@0 != 42 OR 42 != column1_max@1), required_guarantees=[column1 not in (42)] # disable round robin repartitioning statement ok @@ -77,7 +77,7 @@ logical_plan physical_plan 01)CoalesceBatchesExec: target_batch_size=8192 02)--FilterExec: column1@0 != 42 -03)----ParquetExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:0..88], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:88..176], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:176..264], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:264..351]]}, projection=[column1], predicate=column1@0 != 42, pruning_predicate=column1_null_count@2 != column1_row_count@3 AND (column1_min@0 != 42 OR 42 != column1_max@1), required_guarantees=[column1 not in (42)] +03)----ParquetExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:0..137], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:137..274], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:274..411], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:411..547]]}, projection=[column1], predicate=column1@0 != 42, pruning_predicate=column1_null_count@2 != column1_row_count@3 AND (column1_min@0 != 42 OR 42 != column1_max@1), required_guarantees=[column1 not in (42)] # enable round robin repartitioning again statement ok @@ -102,7 +102,7 @@ physical_plan 02)--SortExec: expr=[column1@0 ASC NULLS LAST], preserve_partitioning=[true] 03)----CoalesceBatchesExec: target_batch_size=8192 04)------FilterExec: column1@0 != 42 -05)--------ParquetExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/1.parquet:0..174], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/1.parquet:174..342, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:0..6], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:6..180], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:180..351]]}, projection=[column1], predicate=column1@0 != 42, pruning_predicate=column1_null_count@2 != column1_row_count@3 AND (column1_min@0 != 42 OR 42 != column1_max@1), required_guarantees=[column1 not in (42)] +05)--------ParquetExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/1.parquet:0..272], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/1.parquet:272..538, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:0..6], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:6..278], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:278..547]]}, projection=[column1], predicate=column1@0 != 42, pruning_predicate=column1_null_count@2 != column1_row_count@3 AND (column1_min@0 != 42 OR 42 != column1_max@1), required_guarantees=[column1 not in (42)] ## Read the files as though they are ordered @@ -138,7 +138,7 @@ physical_plan 01)SortPreservingMergeExec: [column1@0 ASC NULLS LAST] 02)--CoalesceBatchesExec: target_batch_size=8192 03)----FilterExec: column1@0 != 42 -04)------ParquetExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/1.parquet:0..171], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:0..175], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:175..351], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/1.parquet:171..342]]}, projection=[column1], output_ordering=[column1@0 ASC NULLS LAST], predicate=column1@0 != 42, pruning_predicate=column1_null_count@2 != column1_row_count@3 AND (column1_min@0 != 42 OR 42 != column1_max@1), required_guarantees=[column1 not in (42)] +04)------ParquetExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/1.parquet:0..269], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:0..273], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:273..547], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/1.parquet:269..538]]}, projection=[column1], output_ordering=[column1@0 ASC NULLS LAST], predicate=column1@0 != 42, pruning_predicate=column1_null_count@2 != column1_row_count@3 AND (column1_min@0 != 42 OR 42 != column1_max@1), required_guarantees=[column1 not in (42)] # Cleanup statement ok diff --git a/docs/source/user-guide/configs.md b/docs/source/user-guide/configs.md index 329b9a95c8f9..1c39064c15d7 100644 --- a/docs/source/user-guide/configs.md +++ b/docs/source/user-guide/configs.md @@ -61,6 +61,7 @@ Environment variables are read during `SessionConfig` initialisation so they mus | datafusion.execution.parquet.data_pagesize_limit | 1048576 | (writing) Sets best effort maximum size of data page in bytes | | datafusion.execution.parquet.write_batch_size | 1024 | (writing) Sets write_batch_size in bytes | | datafusion.execution.parquet.writer_version | 1.0 | (writing) Sets parquet writer version valid values are "1.0" and "2.0" | +| datafusion.execution.parquet.skip_arrow_metadata | false | (writing) Skip encoding the embedded arrow metadata in the KV_meta This is analogous to the `ArrowWriterOptions::with_skip_arrow_metadata`. Refer to | | datafusion.execution.parquet.compression | zstd(3) | (writing) Sets default parquet compression codec. Valid values are: uncompressed, snappy, gzip(level), lzo, brotli(level), lz4, zstd(level), and lz4_raw. These values are not case sensitive. If NULL, uses default parquet writer setting Note that this default setting is not the same as the default parquet writer setting. | | datafusion.execution.parquet.dictionary_enabled | true | (writing) Sets if dictionary encoding is enabled. If NULL, uses default parquet writer setting | | datafusion.execution.parquet.dictionary_page_size_limit | 1048576 | (writing) Sets best effort maximum dictionary page size, in bytes | From ab9ff56ed2d3a81ced3458c2f3f8955e53cf6e57 Mon Sep 17 00:00:00 2001 From: Raz Luvaton <16746759+rluvaton@users.noreply.github.com> Date: Wed, 1 Jan 2025 16:22:18 +0200 Subject: [PATCH 06/17] chore: Create devcontainer.json (#13520) * Create devcontainer.json * update devcontainer * remove useless features --- .devcontainer/Dockerfile | 13 +++++++++++++ .devcontainer/devcontainer.json | 16 ++++++++++++++++ 2 files changed, 29 insertions(+) create mode 100644 .devcontainer/Dockerfile create mode 100644 .devcontainer/devcontainer.json diff --git a/.devcontainer/Dockerfile b/.devcontainer/Dockerfile new file mode 100644 index 000000000000..9dd627b01abe --- /dev/null +++ b/.devcontainer/Dockerfile @@ -0,0 +1,13 @@ +FROM rust:bookworm + +RUN apt-get update && export DEBIAN_FRONTEND=noninteractive \ + # Remove imagemagick due to https://security-tracker.debian.org/tracker/CVE-2019-10131 + && apt-get purge -y imagemagick imagemagick-6-common + +# Add protoc +# https://datafusion.apache.org/contributor-guide/getting_started.html#protoc-installation +RUN curl -LO https://github.com/protocolbuffers/protobuf/releases/download/v25.1/protoc-25.1-linux-x86_64.zip \ + && unzip protoc-25.1-linux-x86_64.zip -d $HOME/.local \ + && rm protoc-25.1-linux-x86_64.zip + +ENV PATH="$PATH:$HOME/.local/bin" \ No newline at end of file diff --git a/.devcontainer/devcontainer.json b/.devcontainer/devcontainer.json new file mode 100644 index 000000000000..1af22306ed8c --- /dev/null +++ b/.devcontainer/devcontainer.json @@ -0,0 +1,16 @@ +{ + "build": { + "dockerfile": "./Dockerfile", + "context": "." + }, + "customizations": { + "vscode": { + "extensions": [ + "rust-lang.rust-analyzer" + ] + } + }, + "features": { + "ghcr.io/devcontainers/features/rust:1": "latest" + } +} From 259443d66f609e7b440537292ad530dcbed31f96 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Wed, 1 Jan 2025 22:26:04 -0500 Subject: [PATCH 07/17] Minor: consolidate ConfigExtension example into API docs (#13954) * Update examples README.md * Minor: consolidate ConfigExtension example into API docs * more docs * Remove update * clippy * Fix issue with ExtensionsOptions docs --- .../examples/config_extension.rs | 52 ------------------- datafusion/common/src/config.rs | 48 ++++++++++++++++- 2 files changed, 46 insertions(+), 54 deletions(-) delete mode 100644 datafusion-examples/examples/config_extension.rs diff --git a/datafusion-examples/examples/config_extension.rs b/datafusion-examples/examples/config_extension.rs deleted file mode 100644 index b9f83f91ce56..000000000000 --- a/datafusion-examples/examples/config_extension.rs +++ /dev/null @@ -1,52 +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. - -//! This example demonstrates how to extend the DataFusion configs with custom extensions. - -use datafusion::{ - common::{config::ConfigExtension, extensions_options}, - config::ConfigOptions, -}; - -extensions_options! { - /// My own config options. - pub struct MyConfig { - /// Should "foo" be replaced by "bar"? - pub foo_to_bar: bool, default = true - - /// How many "baz" should be created? - pub baz_count: usize, default = 1337 - } -} - -impl ConfigExtension for MyConfig { - const PREFIX: &'static str = "my_config"; -} - -fn main() { - // set up config struct and register extension - let mut config = ConfigOptions::default(); - config.extensions.insert(MyConfig::default()); - - // overwrite config default - config.set("my_config.baz_count", "42").unwrap(); - - // check config state - let my_config = config.extensions.get::().unwrap(); - assert!(my_config.foo_to_bar,); - assert_eq!(my_config.baz_count, 42,); -} diff --git a/datafusion/common/src/config.rs b/datafusion/common/src/config.rs index 4da6921ba53c..8d2742aaafe5 100644 --- a/datafusion/common/src/config.rs +++ b/datafusion/common/src/config.rs @@ -895,8 +895,48 @@ impl ConfigOptions { } } -/// [`ConfigExtension`] provides a mechanism to store third-party configuration within DataFusion +/// [`ConfigExtension`] provides a mechanism to store third-party configuration +/// within DataFusion [`ConfigOptions`] /// +/// This mechanism can be used to pass configuration to user defined functions +/// or optimizer passes +/// +/// # Example +/// ``` +/// use datafusion_common::{ +/// config::ConfigExtension, extensions_options, +/// config::ConfigOptions, +/// }; +/// // Define a new configuration struct using the `extensions_options` macro +/// extensions_options! { +/// /// My own config options. +/// pub struct MyConfig { +/// /// Should "foo" be replaced by "bar"? +/// pub foo_to_bar: bool, default = true +/// +/// /// How many "baz" should be created? +/// pub baz_count: usize, default = 1337 +/// } +/// } +/// +/// impl ConfigExtension for MyConfig { +/// const PREFIX: &'static str = "my_config"; +/// } +/// +/// // set up config struct and register extension +/// let mut config = ConfigOptions::default(); +/// config.extensions.insert(MyConfig::default()); +/// +/// // overwrite config default +/// config.set("my_config.baz_count", "42").unwrap(); +/// +/// // check config state +/// let my_config = config.extensions.get::().unwrap(); +/// assert!(my_config.foo_to_bar,); +/// assert_eq!(my_config.baz_count, 42,); +/// ``` +/// +/// # Note: /// Unfortunately associated constants are not currently object-safe, and so this /// extends the object-safe [`ExtensionOptions`] pub trait ConfigExtension: ExtensionOptions { @@ -906,7 +946,9 @@ pub trait ConfigExtension: ExtensionOptions { const PREFIX: &'static str; } -/// An object-safe API for storing arbitrary configuration +/// An object-safe API for storing arbitrary configuration. +/// +/// See [`ConfigExtension`] for user defined configuration pub trait ExtensionOptions: Send + Sync + fmt::Debug + 'static { /// Return `self` as [`Any`] /// @@ -1114,6 +1156,8 @@ pub trait Visit { /// - ``: Default value matching the field type like `42`. /// /// # Example +/// See also a full example on the [`ConfigExtension`] documentation +/// /// ``` /// use datafusion_common::extensions_options; /// From 846adf319d44d1e4088f2d4f2464bb3db2c04359 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Thu, 2 Jan 2025 07:44:54 -0500 Subject: [PATCH 08/17] Parallelize pruning utf8 fuzz test (#13947) --- datafusion/core/tests/fuzz_cases/pruning.rs | 387 +++++++++++++------- 1 file changed, 263 insertions(+), 124 deletions(-) diff --git a/datafusion/core/tests/fuzz_cases/pruning.rs b/datafusion/core/tests/fuzz_cases/pruning.rs index ad35850a5f18..3725e6d908e6 100644 --- a/datafusion/core/tests/fuzz_cases/pruning.rs +++ b/datafusion/core/tests/fuzz_cases/pruning.rs @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -use std::sync::Arc; +use std::sync::{Arc, OnceLock}; use arrow_array::{Array, RecordBatch, StringArray}; use arrow_schema::{DataType, Field, Schema}; @@ -38,151 +38,266 @@ use parquet::{ file::properties::{EnabledStatistics, WriterProperties}, }; use rand::seq::SliceRandom; +use tokio::sync::Mutex; use url::Url; #[tokio::test] -async fn test_fuzz_utf8() { - // Fuzz testing for UTF8 predicate pruning - // The basic idea is that query results should always be the same with or without stats/pruning - // If we get this right we at least guarantee that there are no incorrect results - // There may still be suboptimal pruning or stats but that's something we can try to catch - // with more targeted tests. - - // Since we know where the edge cases might be we don't do random black box fuzzing. - // Instead we fuzz on specific pre-defined axis: - // - // - Which characters are in each value. We want to make sure to include characters that when - // incremented, truncated or otherwise manipulated might cause issues. - // - The values in each row group. This impacts which min/max stats are generated for each rg. - // We'll generate combinations of the characters with lengths ranging from 1 to 4. - // - Truncation of statistics to 1, 2 or 3 characters as well as no truncation. - - let mut rng = rand::thread_rng(); - - let characters = [ - "z", - "0", - "~", - "ß", - "℣", - "%", // this one is useful for like/not like tests since it will result in randomly inserted wildcards - "_", // this one is useful for like/not like tests since it will result in randomly inserted wildcards - "\u{7F}", - "\u{7FF}", - "\u{FF}", - "\u{10FFFF}", - "\u{D7FF}", - "\u{FDCF}", - // null character - "\u{0}", - ]; - - let value_lengths = [1, 2, 3]; - - // generate all combinations of characters with lengths ranging from 1 to 4 - let mut values = vec![]; - for length in &value_lengths { - values.extend( - characters - .iter() - .cloned() - .combinations(*length) - // now get all permutations of each combination - .flat_map(|c| c.into_iter().permutations(*length)) - // and join them into strings - .map(|c| c.join("")), - ); - } +async fn test_utf8_eq() { + Utf8Test::new(|value| col("a").eq(lit(value))).run().await; +} + +#[tokio::test] +async fn test_utf8_not_eq() { + Utf8Test::new(|value| col("a").not_eq(lit(value))) + .run() + .await; +} + +#[tokio::test] +async fn test_utf8_lt() { + Utf8Test::new(|value| col("a").lt(lit(value))).run().await; +} + +#[tokio::test] +async fn test_utf8_lt_eq() { + Utf8Test::new(|value| col("a").lt_eq(lit(value))) + .run() + .await; +} + +#[tokio::test] +async fn test_utf8_gt() { + Utf8Test::new(|value| col("a").gt(lit(value))).run().await; +} + +#[tokio::test] +async fn test_utf8_gt_eq() { + Utf8Test::new(|value| col("a").gt_eq(lit(value))) + .run() + .await; +} + +#[tokio::test] +async fn test_utf8_like() { + Utf8Test::new(|value| col("a").like(lit(value))).run().await; +} - println!("Generated {} values", values.len()); +#[tokio::test] +async fn test_utf8_not_like() { + Utf8Test::new(|value| col("a").not_like(lit(value))) + .run() + .await; +} - // randomly pick 100 values - values.shuffle(&mut rng); - values.truncate(100); +#[tokio::test] +async fn test_utf8_like_prefix() { + Utf8Test::new(|value| col("a").like(lit(format!("%{}", value)))) + .run() + .await; +} + +#[tokio::test] +async fn test_utf8_like_suffix() { + Utf8Test::new(|value| col("a").like(lit(format!("{}%", value)))) + .run() + .await; +} + +#[tokio::test] +async fn test_utf8_not_like_prefix() { + Utf8Test::new(|value| col("a").not_like(lit(format!("%{}", value)))) + .run() + .await; +} - let mut row_groups = vec![]; - // generate all combinations of values for row groups (1 or 2 values per rg, more is unnecessary since we only get min/max stats out) - for rg_length in [1, 2] { - row_groups.extend(values.iter().cloned().combinations(rg_length)); +#[tokio::test] +async fn test_utf8_not_like_suffix() { + Utf8Test::new(|value| col("a").not_like(lit(format!("{}%", value)))) + .run() + .await; +} + +/// Fuzz testing for UTF8 predicate pruning +/// The basic idea is that query results should always be the same with or without stats/pruning +/// If we get this right we at least guarantee that there are no incorrect results +/// There may still be suboptimal pruning or stats but that's something we can try to catch +/// with more targeted tests. +// +/// Since we know where the edge cases might be we don't do random black box fuzzing. +/// Instead we fuzz on specific pre-defined axis: +/// +/// - Which characters are in each value. We want to make sure to include characters that when +/// incremented, truncated or otherwise manipulated might cause issues. +/// - The values in each row group. This impacts which min/max stats are generated for each rg. +/// We'll generate combinations of the characters with lengths ranging from 1 to 4. +/// - Truncation of statistics to 1, 2 or 3 characters as well as no truncation. +struct Utf8Test { + /// Test queries the parquet files with this predicate both with and without + /// pruning enabled + predicate_generator: Box Expr + 'static>, +} + +impl Utf8Test { + /// Create a new test with the given predicate generator + fn new Expr + 'static>(f: F) -> Self { + Self { + predicate_generator: Box::new(f), + } } - println!("Generated {} row groups", row_groups.len()); + /// Run the test by evaluating the predicate on the test files with and + /// without pruning enable + async fn run(&self) { + let ctx = SessionContext::new(); + + let mut predicates = vec![]; + for value in Self::values() { + predicates.push((self.predicate_generator)(value)); + } - // Randomly pick 100 row groups (combinations of said values) - row_groups.shuffle(&mut rng); - row_groups.truncate(100); + let store = Self::memory_store(); + ctx.register_object_store(&Url::parse("memory://").unwrap(), Arc::clone(store)); - let schema = Arc::new(Schema::new(vec![Field::new("a", DataType::Utf8, false)])); - let df_schema = DFSchema::try_from(schema.clone()).unwrap(); + let files = Self::test_files().await; + let schema = Self::schema(); + let df_schema = DFSchema::try_from(Arc::clone(&schema)).unwrap(); - let store = InMemory::new(); - let mut files = vec![]; - for (idx, truncation_length) in [Some(1), Some(2), None].iter().enumerate() { - // parquet files only support 32767 row groups per file, so chunk up into multiple files so we don't error if running on a large number of row groups - for (rg_idx, row_groups) in row_groups.chunks(32766).enumerate() { - let buf = write_parquet_file( - *truncation_length, + println!("Testing {} predicates", predicates.len()); + for predicate in predicates { + // println!("Testing predicate {:?}", predicate); + let phys_expr_predicate = ctx + .create_physical_expr(predicate.clone(), &df_schema) + .unwrap(); + let expected = execute_with_predicate( + &files, + Arc::clone(&phys_expr_predicate), + false, schema.clone(), - row_groups.to_vec(), + &ctx, ) .await; - let filename = format!("test_fuzz_utf8_{idx}_{rg_idx}.parquet"); - files.push((filename.clone(), buf.len())); - let payload = PutPayload::from(buf); - let path = Path::from(filename); - store.put(&path, payload).await.unwrap(); + let with_pruning = execute_with_predicate( + &files, + phys_expr_predicate, + true, + schema.clone(), + &ctx, + ) + .await; + assert_eq!(expected, with_pruning); } } - println!("Generated {} parquet files", files.len()); - - let ctx = SessionContext::new(); - - ctx.register_object_store(&Url::parse("memory://").unwrap(), Arc::new(store)); - - let mut predicates = vec![]; - for value in values { - predicates.push(col("a").eq(lit(value.clone()))); - predicates.push(col("a").not_eq(lit(value.clone()))); - predicates.push(col("a").lt(lit(value.clone()))); - predicates.push(col("a").lt_eq(lit(value.clone()))); - predicates.push(col("a").gt(lit(value.clone()))); - predicates.push(col("a").gt_eq(lit(value.clone()))); - predicates.push(col("a").like(lit(value.clone()))); - predicates.push(col("a").not_like(lit(value.clone()))); - predicates.push(col("a").like(lit(format!("%{}", value.clone())))); - predicates.push(col("a").like(lit(format!("{}%", value.clone())))); - predicates.push(col("a").not_like(lit(format!("%{}", value.clone())))); - predicates.push(col("a").not_like(lit(format!("{}%", value.clone())))); + /// all combinations of interesting charactes with lengths ranging from 1 to 4 + fn values() -> &'static [String] { + VALUES.get_or_init(|| { + let mut rng = rand::thread_rng(); + + let characters = [ + "z", + "0", + "~", + "ß", + "℣", + "%", // this one is useful for like/not like tests since it will result in randomly inserted wildcards + "_", // this one is useful for like/not like tests since it will result in randomly inserted wildcards + "\u{7F}", + "\u{7FF}", + "\u{FF}", + "\u{10FFFF}", + "\u{D7FF}", + "\u{FDCF}", + // null character + "\u{0}", + ]; + let value_lengths = [1, 2, 3]; + let mut values = vec![]; + for length in &value_lengths { + values.extend( + characters + .iter() + .cloned() + .combinations(*length) + // now get all permutations of each combination + .flat_map(|c| c.into_iter().permutations(*length)) + // and join them into strings + .map(|c| c.join("")), + ); + } + println!("Generated {} values", values.len()); + // randomly pick 100 values + values.shuffle(&mut rng); + values.truncate(100); + values + }) } - for predicate in predicates { - println!("Testing predicate {:?}", predicate); - let phys_expr_predicate = ctx - .create_physical_expr(predicate.clone(), &df_schema) - .unwrap(); - let expected = execute_with_predicate( - &files, - phys_expr_predicate.clone(), - false, - schema.clone(), - &ctx, - ) - .await; - let with_pruning = execute_with_predicate( - &files, - phys_expr_predicate, - true, - schema.clone(), - &ctx, - ) - .await; - assert_eq!(expected, with_pruning); + /// return the in memory object store + fn memory_store() -> &'static Arc { + MEMORY_STORE.get_or_init(|| Arc::new(InMemory::new())) + } + + /// return the schema of the created test files + fn schema() -> Arc { + let schema = SCHEMA.get_or_init(|| { + Arc::new(Schema::new(vec![Field::new("a", DataType::Utf8, false)])) + }); + Arc::clone(schema) + } + + /// Return a list of test files with UTF8 data and combinations of + /// [`Self::values`] + async fn test_files() -> Vec { + let files_mutex = TESTFILES.get_or_init(|| Mutex::new(vec![])); + let mut files = files_mutex.lock().await; + if !files.is_empty() { + return (*files).clone(); + } + + let mut rng = rand::thread_rng(); + let values = Self::values(); + + let mut row_groups = vec![]; + // generate all combinations of values for row groups (1 or 2 values per rg, more is unnecessary since we only get min/max stats out) + for rg_length in [1, 2] { + row_groups.extend(values.iter().cloned().combinations(rg_length)); + } + + println!("Generated {} row groups", row_groups.len()); + + // Randomly pick 100 row groups (combinations of said values) + row_groups.shuffle(&mut rng); + row_groups.truncate(100); + + let schema = Self::schema(); + + let store = Self::memory_store(); + for (idx, truncation_length) in [Some(1), Some(2), None].iter().enumerate() { + // parquet files only support 32767 row groups per file, so chunk up into multiple files so we don't error if running on a large number of row groups + for (rg_idx, row_groups) in row_groups.chunks(32766).enumerate() { + let buf = write_parquet_file( + *truncation_length, + Arc::clone(&schema), + row_groups.to_vec(), + ) + .await; + let filename = format!("test_fuzz_utf8_{idx}_{rg_idx}.parquet"); + let size = buf.len(); + let path = Path::from(filename); + let payload = PutPayload::from(buf); + store.put(&path, payload).await.unwrap(); + + files.push(TestFile { path, size }); + } + } + + println!("Generated {} parquet files", files.len()); + files.clone() } } async fn execute_with_predicate( - files: &[(String, usize)], + files: &[TestFile], predicate: Arc, prune_stats: bool, schema: Arc, @@ -193,7 +308,12 @@ async fn execute_with_predicate( .with_file_group( files .iter() - .map(|(path, size)| PartitionedFile::new(path.clone(), *size as u64)) + .map(|test_file| { + PartitionedFile::new( + test_file.path.clone(), + test_file.size as u64, + ) + }) .collect(), ); let mut builder = ParquetExecBuilder::new(scan); @@ -245,3 +365,22 @@ async fn write_parquet_file( } buf.into_inner().freeze() } + +/// The string values for [Utf8Test::values] +static VALUES: OnceLock> = OnceLock::new(); +/// The schema for the [Utf8Test::schema] +static SCHEMA: OnceLock> = OnceLock::new(); + +/// The InMemory object store +static MEMORY_STORE: OnceLock> = OnceLock::new(); + +/// List of in memory parquet files with UTF8 data +// Use a mutex rather than OnceLock to allow for async initialization +static TESTFILES: OnceLock>> = OnceLock::new(); + +/// Holds a temporary parquet file path and its size +#[derive(Debug, Clone)] +struct TestFile { + path: Path, + size: usize, +} From 38ccb0071045be1fae672ce2561c001f5d505efb Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Thu, 2 Jan 2025 18:20:57 +0300 Subject: [PATCH 09/17] Add swap_inputs to SMJ (#13984) --- .../src/physical_optimizer/join_selection.rs | 4 ++- .../src/joins/sort_merge_join.rs | 36 +++++++++++++++++-- 2 files changed, 37 insertions(+), 3 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/join_selection.rs b/datafusion/core/src/physical_optimizer/join_selection.rs index d7a2f1740141..736c3fbd0184 100644 --- a/datafusion/core/src/physical_optimizer/join_selection.rs +++ b/datafusion/core/src/physical_optimizer/join_selection.rs @@ -552,7 +552,9 @@ fn hash_join_swap_subrule( /// [`JoinType::Full`], [`JoinType::Right`], [`JoinType::RightAnti`] and /// [`JoinType::RightSemi`] can not run with an unbounded left side, even if /// we swap join sides. Therefore, we do not consider them here. -fn swap_join_according_to_unboundedness( +/// This function is crate public as it is useful for downstream projects +/// to implement, or experiment with, their own join selection rules. +pub(crate) fn swap_join_according_to_unboundedness( hash_join: &HashJoinExec, ) -> Result> { let partition_mode = hash_join.partition_mode(); diff --git a/datafusion/physical-plan/src/joins/sort_merge_join.rs b/datafusion/physical-plan/src/joins/sort_merge_join.rs index 438d9818475d..54bd63084ece 100644 --- a/datafusion/physical-plan/src/joins/sort_merge_join.rs +++ b/datafusion/physical-plan/src/joins/sort_merge_join.rs @@ -58,7 +58,8 @@ use crate::execution_plan::{boundedness_from_children, EmissionType}; use crate::expressions::PhysicalSortExpr; use crate::joins::utils::{ build_join_schema, check_join_is_valid, estimate_join_statistics, - symmetric_join_output_partitioning, JoinFilter, JoinOn, JoinOnRef, + reorder_output_after_swap, symmetric_join_output_partitioning, JoinFilter, JoinOn, + JoinOnRef, }; use crate::metrics::{Count, ExecutionPlanMetricsSet, MetricBuilder, MetricsSet}; use crate::spill::spill_record_batches; @@ -73,7 +74,7 @@ use futures::{Stream, StreamExt}; /// Join execution plan that executes equi-join predicates on multiple partitions using Sort-Merge /// join algorithm and applies an optional filter post join. Can be used to join arbitrarily large /// inputs where one or both of the inputs don't fit in the available memory. -/// +/// /// # Join Expressions /// /// Equi-join predicate (e.g. ` = `) expressions are represented by [`Self::on`]. @@ -311,6 +312,37 @@ impl SortMergeJoinExec { boundedness_from_children([left, right]), ) } + + pub fn swap_inputs(&self) -> Result> { + let left = self.left(); + let right = self.right(); + let new_join = SortMergeJoinExec::try_new( + Arc::clone(right), + Arc::clone(left), + self.on() + .iter() + .map(|(l, r)| (Arc::clone(r), Arc::clone(l))) + .collect::>(), + self.filter().as_ref().map(JoinFilter::swap), + self.join_type().swap(), + self.sort_options.clone(), + self.null_equals_null, + )?; + + // TODO: OR this condition with having a built-in projection (like + // ordinary hash join) when we support it. + if matches!( + self.join_type(), + JoinType::LeftSemi + | JoinType::RightSemi + | JoinType::LeftAnti + | JoinType::RightAnti + ) { + Ok(Arc::new(new_join)) + } else { + reorder_output_after_swap(Arc::new(new_join), &left.schema(), &right.schema()) + } + } } impl DisplayAs for SortMergeJoinExec { From 04f56bdb17ac3a652c09f1e49c6a10cfc58bad57 Mon Sep 17 00:00:00 2001 From: Raz Luvaton <16746759+rluvaton@users.noreply.github.com> Date: Thu, 2 Jan 2025 20:20:28 +0200 Subject: [PATCH 10/17] fix(datafusion-functions-nested): `arrow-distinct` now work with null rows (#13966) * added failing test * fix(datafusion-functions-nested): `arrow-distinct` now work with null rows * Update datafusion/functions-nested/src/set_ops.rs Co-authored-by: Andrew Lamb * Update set_ops.rs --------- Co-authored-by: Andrew Lamb --- datafusion/functions-nested/src/set_ops.rs | 12 +++++++++--- datafusion/sqllogictest/test_files/array.slt | 7 +++++++ 2 files changed, 16 insertions(+), 3 deletions(-) diff --git a/datafusion/functions-nested/src/set_ops.rs b/datafusion/functions-nested/src/set_ops.rs index 202330715ba0..893fc933d0c8 100644 --- a/datafusion/functions-nested/src/set_ops.rs +++ b/datafusion/functions-nested/src/set_ops.rs @@ -516,11 +516,16 @@ fn general_array_distinct( let mut new_arrays = Vec::with_capacity(array.len()); let converter = RowConverter::new(vec![SortField::new(dt)])?; // distinct for each list in ListArray - for arr in array.iter().flatten() { + for arr in array.iter() { + let last_offset: OffsetSize = offsets.last().copied().unwrap(); + let Some(arr) = arr else { + // Add same offset for null + offsets.push(last_offset); + continue; + }; let values = converter.convert_columns(&[arr])?; // sort elements in list and remove duplicates let rows = values.iter().sorted().dedup().collect::>(); - let last_offset: OffsetSize = offsets.last().copied().unwrap(); offsets.push(last_offset + OffsetSize::usize_as(rows.len())); let arrays = converter.convert_rows(rows)?; let array = match arrays.first() { @@ -538,6 +543,7 @@ fn general_array_distinct( Arc::clone(field), offsets, values, - None, + // Keep the list nulls + array.nulls().cloned(), )?)) } diff --git a/datafusion/sqllogictest/test_files/array.slt b/datafusion/sqllogictest/test_files/array.slt index a023e06030a2..dcceeebaf413 100644 --- a/datafusion/sqllogictest/test_files/array.slt +++ b/datafusion/sqllogictest/test_files/array.slt @@ -5674,6 +5674,13 @@ select array_distinct([sum(a)]) from t1 where a > 100 group by b; statement ok drop table t1; +query ? +select array_distinct(a) from values ([1, 2, 3]), (null), ([1, 3, 1]) as X(a); +---- +[1, 2, 3] +NULL +[1, 3] + query ? select array_distinct([]); ---- From 032d7220374b702f013f90eafcff78d87893227a Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Thu, 2 Jan 2025 13:22:44 -0500 Subject: [PATCH 11/17] Update release instructions for 44.0.0 (#13959) * Update release instructions for 44.0.0 * update macros and order * add functions-table --- datafusion/functions-table/README.md | 26 ++++++++++++++++++++++++++ dev/release/README.md | 5 ++++- 2 files changed, 30 insertions(+), 1 deletion(-) create mode 100644 datafusion/functions-table/README.md diff --git a/datafusion/functions-table/README.md b/datafusion/functions-table/README.md new file mode 100644 index 000000000000..c4e7a5aff999 --- /dev/null +++ b/datafusion/functions-table/README.md @@ -0,0 +1,26 @@ + + +# DataFusion Table Function Library + +[DataFusion][df] is an extensible query execution framework, written in Rust, that uses Apache Arrow as its in-memory format. + +This crate contains table functions that can be used in DataFusion queries. + +[df]: https://crates.io/crates/datafusion diff --git a/dev/release/README.md b/dev/release/README.md index 5dd9b4fc59fd..0c0f6ffb21a5 100644 --- a/dev/release/README.md +++ b/dev/release/README.md @@ -274,13 +274,16 @@ Verify that the Cargo.toml in the tarball contains the correct version (cd datafusion/physical-expr-common && cargo publish) (cd datafusion/functions-aggregate-common && cargo publish) (cd datafusion/functions-window-common && cargo publish) +(cd datafusion/doc && cargo publish) +(cd datafusion/macros && cargo publish) (cd datafusion/expr && cargo publish) (cd datafusion/execution && cargo publish) -(cd datafusion/physical-expr && cargo publish) (cd datafusion/functions && cargo publish) +(cd datafusion/physical-expr && cargo publish) (cd datafusion/functions-aggregate && cargo publish) (cd datafusion/functions-window && cargo publish) (cd datafusion/functions-nested && cargo publish) +(cd datafusion/functions-table && cargo publish) (cd datafusion/sql && cargo publish) (cd datafusion/optimizer && cargo publish) (cd datafusion/common-runtime && cargo publish) From f1af933df2cfa62b9d20f9da07973aeea7ff9b7f Mon Sep 17 00:00:00 2001 From: Bruce Ritchie Date: Thu, 2 Jan 2025 13:25:01 -0500 Subject: [PATCH 12/17] Add datafusion python 43.1.0 blog post to doc. (#13974) --- docs/source/user-guide/concepts-readings-events.md | 2 ++ 1 file changed, 2 insertions(+) diff --git a/docs/source/user-guide/concepts-readings-events.md b/docs/source/user-guide/concepts-readings-events.md index ee9e5e9a844b..609dcadf2a8d 100644 --- a/docs/source/user-guide/concepts-readings-events.md +++ b/docs/source/user-guide/concepts-readings-events.md @@ -124,6 +124,8 @@ This is a list of DataFusion related blog posts, articles, and other resources. ## 📅 Release Notes & Updates +- **2024-09-14** [Apache DataFusion Python 43.1.0 Released](https://datafusion.apache.org/blog/2024/12/14/datafusion-python-43.1.0/) + - **2024-08-24** [Apache DataFusion Python 40.1.0 Released, Significant usability updates](https://datafusion.apache.org/blog/2024/08/20/python-datafusion-40.0.0/) - **2024-07-24** [DataFusion 40.0.0 Release](https://datafusion.apache.org/blog/2024/07/24/datafusion-40.0.0/) From 8fc26c2feae5171174889edc6003802b1b863bda Mon Sep 17 00:00:00 2001 From: Andrew Kane Date: Thu, 2 Jan 2025 13:25:23 -0500 Subject: [PATCH 13/17] Include license and notice files in more crates (#13985) --- datafusion/core/Cargo.toml | 2 +- datafusion/doc/LICENSE.txt | 1 + datafusion/doc/NOTICE.txt | 1 + datafusion/functions-table/LICENSE.txt | 1 + datafusion/functions-table/NOTICE.txt | 1 + datafusion/macros/LICENSE.txt | 1 + datafusion/macros/NOTICE.txt | 1 + 7 files changed, 7 insertions(+), 1 deletion(-) create mode 120000 datafusion/doc/LICENSE.txt create mode 120000 datafusion/doc/NOTICE.txt create mode 120000 datafusion/functions-table/LICENSE.txt create mode 120000 datafusion/functions-table/NOTICE.txt create mode 120000 datafusion/macros/LICENSE.txt create mode 120000 datafusion/macros/NOTICE.txt diff --git a/datafusion/core/Cargo.toml b/datafusion/core/Cargo.toml index 64ad8f2ba152..7a458f8e9bbb 100644 --- a/datafusion/core/Cargo.toml +++ b/datafusion/core/Cargo.toml @@ -19,7 +19,7 @@ name = "datafusion" description = "DataFusion is an in-memory query engine that uses Apache Arrow as the memory model" keywords = ["arrow", "query", "sql"] -include = ["benches/*.rs", "src/**/*.rs", "Cargo.toml"] +include = ["benches/*.rs", "src/**/*.rs", "Cargo.toml", "LICENSE.txt", "NOTICE.txt"] readme = "../../README.md" version = { workspace = true } edition = { workspace = true } diff --git a/datafusion/doc/LICENSE.txt b/datafusion/doc/LICENSE.txt new file mode 120000 index 000000000000..1ef648f64b34 --- /dev/null +++ b/datafusion/doc/LICENSE.txt @@ -0,0 +1 @@ +../../LICENSE.txt \ No newline at end of file diff --git a/datafusion/doc/NOTICE.txt b/datafusion/doc/NOTICE.txt new file mode 120000 index 000000000000..fb051c92b10b --- /dev/null +++ b/datafusion/doc/NOTICE.txt @@ -0,0 +1 @@ +../../NOTICE.txt \ No newline at end of file diff --git a/datafusion/functions-table/LICENSE.txt b/datafusion/functions-table/LICENSE.txt new file mode 120000 index 000000000000..1ef648f64b34 --- /dev/null +++ b/datafusion/functions-table/LICENSE.txt @@ -0,0 +1 @@ +../../LICENSE.txt \ No newline at end of file diff --git a/datafusion/functions-table/NOTICE.txt b/datafusion/functions-table/NOTICE.txt new file mode 120000 index 000000000000..fb051c92b10b --- /dev/null +++ b/datafusion/functions-table/NOTICE.txt @@ -0,0 +1 @@ +../../NOTICE.txt \ No newline at end of file diff --git a/datafusion/macros/LICENSE.txt b/datafusion/macros/LICENSE.txt new file mode 120000 index 000000000000..1ef648f64b34 --- /dev/null +++ b/datafusion/macros/LICENSE.txt @@ -0,0 +1 @@ +../../LICENSE.txt \ No newline at end of file diff --git a/datafusion/macros/NOTICE.txt b/datafusion/macros/NOTICE.txt new file mode 120000 index 000000000000..fb051c92b10b --- /dev/null +++ b/datafusion/macros/NOTICE.txt @@ -0,0 +1 @@ +../../NOTICE.txt \ No newline at end of file From 858c020423ac7fadd210fbc3983eb91e86951a40 Mon Sep 17 00:00:00 2001 From: Bruce Ritchie Date: Thu, 2 Jan 2025 13:29:26 -0500 Subject: [PATCH 14/17] Extract postgres container from sqllogictest, update datafusion-testing pin (#13971) * Add support for sqlite test files to sqllogictest * Removed workaround for bug that was fixed. * Refactor sqllogictest to extract postgres functionality into a separate file. Removed dependency on once_cell in favour of LazyLock. * Add missing license header. --- datafusion-testing | 2 +- datafusion/sqllogictest/Cargo.toml | 2 - .../sqllogictest/bin/postgres_container.rs | 151 ++++++++++++++++++ datafusion/sqllogictest/bin/sqllogictests.rs | 148 +---------------- .../src/engines/postgres_engine/mod.rs | 1 - 5 files changed, 160 insertions(+), 144 deletions(-) create mode 100644 datafusion/sqllogictest/bin/postgres_container.rs diff --git a/datafusion-testing b/datafusion-testing index e2e320c9477a..5cc59ceceeeb 160000 --- a/datafusion-testing +++ b/datafusion-testing @@ -1 +1 @@ -Subproject commit e2e320c9477a6d8ab09662eae255887733c0e304 +Subproject commit 5cc59ceceeebeea6b39861210b6d1cd27e66648a diff --git a/datafusion/sqllogictest/Cargo.toml b/datafusion/sqllogictest/Cargo.toml index 1bb88a8bd44f..3104846eda73 100644 --- a/datafusion/sqllogictest/Cargo.toml +++ b/datafusion/sqllogictest/Cargo.toml @@ -49,7 +49,6 @@ indicatif = "0.17" itertools = { workspace = true } log = { workspace = true } object_store = { workspace = true } -once_cell = { version = "1.20", optional = true } postgres-protocol = { version = "0.6.7", optional = true } postgres-types = { version = "0.2.8", features = ["derive", "with-chrono-0_4"], optional = true } rust_decimal = { version = "1.36.0", features = ["tokio-pg"] } @@ -69,7 +68,6 @@ avro = ["datafusion/avro"] postgres = [ "bytes", "chrono", - "once_cell", "postgres-types", "postgres-protocol", "testcontainers", diff --git a/datafusion/sqllogictest/bin/postgres_container.rs b/datafusion/sqllogictest/bin/postgres_container.rs new file mode 100644 index 000000000000..210b9b3e361c --- /dev/null +++ b/datafusion/sqllogictest/bin/postgres_container.rs @@ -0,0 +1,151 @@ +// 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. + +#![cfg(feature = "postgres")] + +use crate::Options; +use datafusion_common::Result; +use log::info; +use std::env::set_var; +use std::future::Future; +use std::sync::LazyLock; +use std::{env, thread}; +use testcontainers::core::IntoContainerPort; +use testcontainers::runners::AsyncRunner; +use testcontainers::ImageExt; +use testcontainers_modules::postgres; +use tokio::sync::mpsc::{UnboundedReceiver, UnboundedSender}; +use tokio::sync::{mpsc, Mutex}; +use ContainerCommands::{FetchHost, FetchPort}; + +#[derive(Debug)] +pub enum ContainerCommands { + FetchHost, + FetchPort, + Stop, +} + +pub struct Channel { + pub tx: UnboundedSender, + pub rx: Mutex>, +} + +pub fn channel() -> Channel { + let (tx, rx) = mpsc::unbounded_channel(); + Channel { + tx, + rx: Mutex::new(rx), + } +} + +pub fn execute_blocking(f: F) { + tokio::runtime::Builder::new_current_thread() + .enable_all() + .build() + .unwrap() + .block_on(f); +} + +static POSTGRES_IN: LazyLock> = LazyLock::new(channel); +static POSTGRES_HOST: LazyLock> = LazyLock::new(channel); +static POSTGRES_PORT: LazyLock> = LazyLock::new(channel); +static POSTGRES_STOPPED: LazyLock> = LazyLock::new(channel); + +pub async fn initialize_postgres_container(options: &Options) -> Result<()> { + let start_pg_database = options.postgres_runner && !is_pg_uri_set(); + if start_pg_database { + info!("Starting postgres db ..."); + + thread::spawn(|| { + execute_blocking(start_postgres( + &POSTGRES_IN, + &POSTGRES_HOST, + &POSTGRES_PORT, + &POSTGRES_STOPPED, + )) + }); + + POSTGRES_IN.tx.send(FetchHost).unwrap(); + let db_host = POSTGRES_HOST.rx.lock().await.recv().await.unwrap(); + + POSTGRES_IN.tx.send(FetchPort).unwrap(); + let db_port = POSTGRES_PORT.rx.lock().await.recv().await.unwrap(); + + let pg_uri = format!("postgresql://postgres:postgres@{db_host}:{db_port}/test"); + info!("Postgres uri is {pg_uri}"); + + set_var("PG_URI", pg_uri); + } else { + // close receiver + POSTGRES_IN.rx.lock().await.close(); + } + + Ok(()) +} + +pub async fn terminate_postgres_container() -> Result<()> { + if !POSTGRES_IN.tx.is_closed() { + println!("Stopping postgres db ..."); + POSTGRES_IN.tx.send(ContainerCommands::Stop).unwrap_or(()); + POSTGRES_STOPPED.rx.lock().await.recv().await; + } + + Ok(()) +} + +async fn start_postgres( + in_channel: &Channel, + host_channel: &Channel, + port_channel: &Channel, + stopped_channel: &Channel<()>, +) { + info!("Starting postgres test container with user postgres/postgres and db test"); + + let container = postgres::Postgres::default() + .with_user("postgres") + .with_password("postgres") + .with_db_name("test") + .with_mapped_port(16432, 5432.tcp()) + .with_tag("17-alpine") + .start() + .await + .unwrap(); + // uncomment this if you are running docker in docker + let host = "host.docker.internal".to_string(); + // let host = container.get_host().await.unwrap().to_string(); + let port = container.get_host_port_ipv4(5432).await.unwrap(); + + let mut rx = in_channel.rx.lock().await; + while let Some(command) = rx.recv().await { + match command { + FetchHost => host_channel.tx.send(host.clone()).unwrap(), + FetchPort => port_channel.tx.send(port).unwrap(), + ContainerCommands::Stop => { + container.stop().await.unwrap(); + stopped_channel.tx.send(()).unwrap(); + rx.close(); + } + } + } +} + +fn is_pg_uri_set() -> bool { + match env::var("PG_URI") { + Ok(_) => true, + Err(_) => false, + } +} diff --git a/datafusion/sqllogictest/bin/sqllogictests.rs b/datafusion/sqllogictest/bin/sqllogictests.rs index 498539c1674a..f6b35bf3771c 100644 --- a/datafusion/sqllogictest/bin/sqllogictests.rs +++ b/datafusion/sqllogictest/bin/sqllogictests.rs @@ -28,33 +28,21 @@ use indicatif::{ use itertools::Itertools; use log::Level::{Info, Warn}; use log::{info, log_enabled, warn}; -#[cfg(feature = "postgres")] -use once_cell::sync::Lazy; use sqllogictest::{ parse_file, strict_column_validator, AsyncDB, Condition, Normalizer, Record, Validator, }; + #[cfg(feature = "postgres")] -use std::env::set_var; +use crate::postgres_container::{ + initialize_postgres_container, terminate_postgres_container, +}; use std::ffi::OsStr; use std::fs; -#[cfg(feature = "postgres")] -use std::future::Future; use std::path::{Path, PathBuf}; + #[cfg(feature = "postgres")] -use std::{env, thread}; -#[cfg(feature = "postgres")] -use testcontainers::core::IntoContainerPort; -#[cfg(feature = "postgres")] -use testcontainers::runners::AsyncRunner; -#[cfg(feature = "postgres")] -use testcontainers::ImageExt; -#[cfg(feature = "postgres")] -use tokio::sync::mpsc::{UnboundedReceiver, UnboundedSender}; -#[cfg(feature = "postgres")] -use tokio::sync::{mpsc, Mutex}; -#[cfg(feature = "postgres")] -use ContainerCommands::{FetchHost, FetchPort}; +mod postgres_container; const TEST_DIRECTORY: &str = "test_files/"; const DATAFUSION_TESTING_TEST_DIRECTORY: &str = "../../datafusion-testing/data/"; @@ -170,31 +158,7 @@ async fn run_tests() -> Result<()> { options.warn_on_ignored(); #[cfg(feature = "postgres")] - let start_pg_database = options.postgres_runner && !is_pg_uri_set(); - #[cfg(feature = "postgres")] - if start_pg_database { - info!("Starting postgres db ..."); - - thread::spawn(|| { - execute_blocking(start_postgres( - &POSTGRES_IN, - &POSTGRES_HOST, - &POSTGRES_PORT, - &POSTGRES_STOPPED, - )) - }); - - POSTGRES_IN.tx.send(FetchHost).unwrap(); - let db_host = POSTGRES_HOST.rx.lock().await.recv().await.unwrap(); - - POSTGRES_IN.tx.send(FetchPort).unwrap(); - let db_port = POSTGRES_PORT.rx.lock().await.recv().await.unwrap(); - - let pg_uri = format!("postgresql://postgres:postgres@{db_host}:{db_port}/test"); - info!("Postgres uri is {pg_uri}"); - - set_var("PG_URI", pg_uri); - } + initialize_postgres_container(&options).await?; // Run all tests in parallel, reporting failures at the end // @@ -277,11 +241,7 @@ async fn run_tests() -> Result<()> { m.println(format!("Completed in {}", HumanDuration(start.elapsed())))?; #[cfg(feature = "postgres")] - if start_pg_database { - println!("Stopping postgres db ..."); - POSTGRES_IN.tx.send(ContainerCommands::Stop).unwrap_or(()); - POSTGRES_STOPPED.rx.lock().await.recv().await; - } + terminate_postgres_container().await?; // report on any errors if !errors.is_empty() { @@ -294,14 +254,6 @@ async fn run_tests() -> Result<()> { } } -#[cfg(feature = "postgres")] -fn is_pg_uri_set() -> bool { - match env::var("PG_URI") { - Ok(_) => true, - Err(_) => false, - } -} - async fn run_test_file( test_file: TestFile, validator: Validator, @@ -758,87 +710,3 @@ impl Options { } } } - -#[cfg(feature = "postgres")] -pub async fn start_postgres( - in_channel: &Channel, - host_channel: &Channel, - port_channel: &Channel, - stopped_channel: &Channel<()>, -) { - info!("Starting postgres test container with user postgres/postgres and db test"); - - let container = testcontainers_modules::postgres::Postgres::default() - .with_user("postgres") - .with_password("postgres") - .with_db_name("test") - .with_mapped_port(16432, 5432.tcp()) - .with_tag("17-alpine") - .start() - .await - .unwrap(); - // uncomment this if you are running docker in docker - // let host = "host.docker.internal".to_string(); - let host = container.get_host().await.unwrap().to_string(); - let port = container.get_host_port_ipv4(5432).await.unwrap(); - - let mut rx = in_channel.rx.lock().await; - while let Some(command) = rx.recv().await { - match command { - FetchHost => host_channel.tx.send(host.clone()).unwrap(), - FetchPort => port_channel.tx.send(port).unwrap(), - ContainerCommands::Stop => { - container.stop().await.unwrap(); - stopped_channel.tx.send(()).unwrap(); - rx.close(); - } - } - } -} - -#[cfg(feature = "postgres")] -#[derive(Debug)] -pub enum ContainerCommands { - FetchHost, - FetchPort, - Stop, -} - -#[cfg(feature = "postgres")] -pub struct Channel { - pub tx: UnboundedSender, - pub rx: Mutex>, -} - -#[cfg(feature = "postgres")] -pub fn channel() -> Channel { - let (tx, rx) = mpsc::unbounded_channel(); - Channel { - tx, - rx: Mutex::new(rx), - } -} - -#[cfg(feature = "postgres")] -pub fn execute_blocking(f: F) { - tokio::runtime::Builder::new_current_thread() - .enable_all() - .build() - .unwrap() - .block_on(f); -} - -#[cfg(feature = "postgres")] -pub struct HostPort { - pub host: String, - pub port: u16, -} - -#[cfg(feature = "postgres")] -static POSTGRES_IN: Lazy> = Lazy::new(channel); -#[cfg(feature = "postgres")] -static POSTGRES_HOST: Lazy> = Lazy::new(channel); -#[cfg(feature = "postgres")] -static POSTGRES_PORT: Lazy> = Lazy::new(channel); -#[cfg(feature = "postgres")] -static POSTGRES_STOPPED: Lazy> = Lazy::new(channel); diff --git a/datafusion/sqllogictest/src/engines/postgres_engine/mod.rs b/datafusion/sqllogictest/src/engines/postgres_engine/mod.rs index 1439695d62c6..6391f666b422 100644 --- a/datafusion/sqllogictest/src/engines/postgres_engine/mod.rs +++ b/datafusion/sqllogictest/src/engines/postgres_engine/mod.rs @@ -215,7 +215,6 @@ fn no_quotes(t: &str) -> &str { fn schema_name(relative_path: &Path) -> String { relative_path .to_string_lossy() - .to_string() .chars() .filter(|ch| ch.is_ascii_alphanumeric()) .collect::() From 63c4d1ba6eefbcab3d07de925303c87d86a913c6 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Thu, 2 Jan 2025 13:30:05 -0500 Subject: [PATCH 15/17] Update rstest requirement from 0.23.0 to 0.24.0 (#13977) Updates the requirements on [rstest](https://github.com/la10736/rstest) to permit the latest version. - [Release notes](https://github.com/la10736/rstest/releases) - [Changelog](https://github.com/la10736/rstest/blob/master/CHANGELOG.md) - [Commits](https://github.com/la10736/rstest/compare/v0.23.0...v0.23.0) --- updated-dependencies: - dependency-name: rstest dependency-type: direct:production ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- Cargo.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/Cargo.toml b/Cargo.toml index b26f8163bc86..814ed09cd3be 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -145,7 +145,7 @@ prost-derive = "0.13.1" rand = "0.8" recursive = "0.1.1" regex = "1.8" -rstest = "0.23.0" +rstest = "0.24.0" serde_json = "1" sqlparser = { version = "0.53.0", features = ["visitor"] } tempfile = "3" From 264f4c51fc97981435f1a1827de934472d60edf8 Mon Sep 17 00:00:00 2001 From: Bruce Ritchie Date: Thu, 2 Jan 2025 13:34:50 -0500 Subject: [PATCH 16/17] Move hash collision test to run only when merging to main. (#13973) --- .github/workflows/hash_collisions.yml | 54 +++++++++++++++++++++++++++ .github/workflows/rust.yml | 21 ----------- 2 files changed, 54 insertions(+), 21 deletions(-) create mode 100644 .github/workflows/hash_collisions.yml diff --git a/.github/workflows/hash_collisions.yml b/.github/workflows/hash_collisions.yml new file mode 100644 index 000000000000..908bec9d134a --- /dev/null +++ b/.github/workflows/hash_collisions.yml @@ -0,0 +1,54 @@ +# 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. + +name: Rust Hash Collisions + +concurrency: + group: ${{ github.repository }}-${{ github.head_ref || github.sha }}-${{ github.workflow }} + cancel-in-progress: true + +# https://docs.github.com/en/actions/writing-workflows/choosing-when-your-workflow-runs/events-that-trigger-workflows#running-your-pull_request-workflow-when-a-pull-request-merges +# +# this job is intended to only run on merge to main branch +on: + pull_request: + branches: + - main + types: + - closed + +jobs: + # Check answers are correct when hash values collide + hash-collisions: + name: cargo test hash collisions (amd64) + runs-on: ubuntu-latest + container: + image: amd64/rust + if: github.event.pull_request.merged == true + steps: + - uses: actions/checkout@v4 + with: + submodules: true + fetch-depth: 1 + - name: Setup Rust toolchain + uses: ./.github/actions/setup-builder + with: + rust-version: stable + - name: Run tests + run: | + cd datafusion + cargo test --profile ci --exclude datafusion-examples --exclude datafusion-benchmarks --exclude datafusion-sqllogictest --workspace --lib --tests --features=force_hash_collisions,avro diff --git a/.github/workflows/rust.yml b/.github/workflows/rust.yml index f5b6eece9dc8..7ac0dfa78215 100644 --- a/.github/workflows/rust.yml +++ b/.github/workflows/rust.yml @@ -541,27 +541,6 @@ jobs: - name: Run clippy run: ci/scripts/rust_clippy.sh - # Check answers are correct when hash values collide - hash-collisions: - name: cargo test hash collisions (amd64) - needs: linux-build-lib - runs-on: ubuntu-latest - container: - image: amd64/rust - steps: - - uses: actions/checkout@v4 - with: - submodules: true - fetch-depth: 1 - - name: Setup Rust toolchain - uses: ./.github/actions/setup-builder - with: - rust-version: stable - - name: Run tests - run: | - cd datafusion - cargo test --profile ci --exclude datafusion-examples --exclude datafusion-benchmarks --exclude datafusion-sqllogictest --workspace --lib --tests --features=force_hash_collisions,avro - cargo-toml-formatting-checks: name: check Cargo.toml formatting needs: linux-build-lib From 63265fdbbc3371bff9fad4a5dd0b901d3b59c397 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Fri, 3 Jan 2025 09:49:21 +0800 Subject: [PATCH 17/17] Update itertools requirement from 0.13 to 0.14 (#13965) * Update itertools requirement from 0.13 to 0.14 Updates the requirements on [itertools](https://github.com/rust-itertools/itertools) to permit the latest version. - [Changelog](https://github.com/rust-itertools/itertools/blob/master/CHANGELOG.md) - [Commits](https://github.com/rust-itertools/itertools/compare/v0.13.0...v0.13.0) --- updated-dependencies: - dependency-name: itertools dependency-type: direct:production ... Signed-off-by: dependabot[bot] * Fix build * Simplify * Update CLI lock --------- Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> Co-authored-by: jonahgao --- Cargo.toml | 2 +- datafusion-cli/Cargo.lock | 31 ++++++++++++------- .../enforce_distribution.rs | 4 +-- .../src/physical_optimizer/sanity_checker.rs | 12 +++---- 4 files changed, 29 insertions(+), 20 deletions(-) diff --git a/Cargo.toml b/Cargo.toml index 814ed09cd3be..ba8bda42d521 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -129,7 +129,7 @@ futures = "0.3" half = { version = "2.2.1", default-features = false } hashbrown = { version = "0.14.5", features = ["raw"] } indexmap = "2.0.0" -itertools = "0.13" +itertools = "0.14" log = "^0.4" object_store = { version = "0.11.0", default-features = false } parking_lot = "0.12" diff --git a/datafusion-cli/Cargo.lock b/datafusion-cli/Cargo.lock index 4e2837e48178..e8b6e99d2dc4 100644 --- a/datafusion-cli/Cargo.lock +++ b/datafusion-cli/Cargo.lock @@ -1263,7 +1263,7 @@ dependencies = [ "flate2", "futures", "glob", - "itertools", + "itertools 0.14.0", "log", "num-traits", "object_store", @@ -1406,7 +1406,7 @@ version = "44.0.0" dependencies = [ "arrow", "datafusion-common", - "itertools", + "itertools 0.14.0", ] [[package]] @@ -1427,7 +1427,7 @@ dependencies = [ "datafusion-macros", "hashbrown 0.14.5", "hex", - "itertools", + "itertools 0.14.0", "log", "md-5", "rand", @@ -1485,7 +1485,7 @@ dependencies = [ "datafusion-functions-aggregate", "datafusion-macros", "datafusion-physical-expr-common", - "itertools", + "itertools 0.14.0", "log", "paste", ] @@ -1545,7 +1545,7 @@ dependencies = [ "datafusion-expr", "datafusion-physical-expr", "indexmap", - "itertools", + "itertools 0.14.0", "log", "recursive", "regex", @@ -1569,7 +1569,7 @@ dependencies = [ "half", "hashbrown 0.14.5", "indexmap", - "itertools", + "itertools 0.14.0", "log", "paste", "petgraph", @@ -1584,7 +1584,7 @@ dependencies = [ "datafusion-common", "datafusion-expr-common", "hashbrown 0.14.5", - "itertools", + "itertools 0.14.0", ] [[package]] @@ -1597,7 +1597,7 @@ dependencies = [ "datafusion-expr-common", "datafusion-physical-expr", "datafusion-physical-plan", - "itertools", + "itertools 0.14.0", "log", "recursive", ] @@ -1625,7 +1625,7 @@ dependencies = [ "half", "hashbrown 0.14.5", "indexmap", - "itertools", + "itertools 0.14.0", "log", "parking_lot", "pin-project-lite", @@ -2432,6 +2432,15 @@ dependencies = [ "either", ] +[[package]] +name = "itertools" +version = "0.14.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2b192c782037fadd9cfa75548310488aabdbf3d2da73885b31bd0abd03351285" +dependencies = [ + "either", +] + [[package]] name = "itoa" version = "1.0.14" @@ -2812,7 +2821,7 @@ dependencies = [ "futures", "humantime", "hyper 1.5.2", - "itertools", + "itertools 0.13.0", "md-5", "parking_lot", "percent-encoding", @@ -4393,7 +4402,7 @@ version = "0.1.9" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "cf221c93e13a30d793f7645a0e7762c55d169dbb0a49671918a2319d289b10bb" dependencies = [ - "windows-sys 0.48.0", + "windows-sys 0.59.0", ] [[package]] diff --git a/datafusion/core/src/physical_optimizer/enforce_distribution.rs b/datafusion/core/src/physical_optimizer/enforce_distribution.rs index 3c8d08ee32d4..c44200a492eb 100644 --- a/datafusion/core/src/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/src/physical_optimizer/enforce_distribution.rs @@ -1208,7 +1208,7 @@ fn ensure_distribution( // We store the updated children in `new_children`. let children = izip!( children.into_iter(), - plan.required_input_ordering().iter(), + plan.required_input_ordering(), plan.maintains_input_order(), repartition_status_flags.into_iter() ) @@ -1275,7 +1275,7 @@ fn ensure_distribution( let ordering_satisfied = child .plan .equivalence_properties() - .ordering_satisfy_requirement(required_input_ordering); + .ordering_satisfy_requirement(&required_input_ordering); if (!ordering_satisfied || !order_preserving_variants_desirable) && child.data { diff --git a/datafusion/core/src/physical_optimizer/sanity_checker.rs b/datafusion/core/src/physical_optimizer/sanity_checker.rs index b6d22320d086..f4b0f7c6069b 100644 --- a/datafusion/core/src/physical_optimizer/sanity_checker.rs +++ b/datafusion/core/src/physical_optimizer/sanity_checker.rs @@ -131,18 +131,18 @@ pub fn check_plan_sanity( check_finiteness_requirements(Arc::clone(&plan), optimizer_options)?; for ((idx, child), sort_req, dist_req) in izip!( - plan.children().iter().enumerate(), - plan.required_input_ordering().iter(), - plan.required_input_distribution().iter() + plan.children().into_iter().enumerate(), + plan.required_input_ordering(), + plan.required_input_distribution(), ) { let child_eq_props = child.equivalence_properties(); if let Some(sort_req) = sort_req { - if !child_eq_props.ordering_satisfy_requirement(sort_req) { + if !child_eq_props.ordering_satisfy_requirement(&sort_req) { let plan_str = get_plan_string(&plan); return plan_err!( "Plan: {:?} does not satisfy order requirements: {}. Child-{} order: {}", plan_str, - format_physical_sort_requirement_list(sort_req), + format_physical_sort_requirement_list(&sort_req), idx, child_eq_props.oeq_class ); @@ -151,7 +151,7 @@ pub fn check_plan_sanity( if !child .output_partitioning() - .satisfy(dist_req, child_eq_props) + .satisfy(&dist_req, child_eq_props) { let plan_str = get_plan_string(&plan); return plan_err!(