From 74fc6f8c8e113d4d23b11a8969c92c956629c5a6 Mon Sep 17 00:00:00 2001 From: Huaijin Date: Sat, 28 Oct 2023 05:06:38 +0800 Subject: [PATCH 01/43] fix (#7946) --- datafusion-examples/Cargo.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion-examples/Cargo.toml b/datafusion-examples/Cargo.toml index e5146c7fd94e..8d504f834bc5 100644 --- a/datafusion-examples/Cargo.toml +++ b/datafusion-examples/Cargo.toml @@ -36,7 +36,7 @@ arrow-schema = { workspace = true } async-trait = "0.1.41" bytes = "1.4" dashmap = "5.4" -datafusion = { path = "../datafusion/core" } +datafusion = { path = "../datafusion/core", features = ["avro"] } datafusion-common = { path = "../datafusion/common" } datafusion-expr = { path = "../datafusion/expr" } datafusion-optimizer = { path = "../datafusion/optimizer" } From 46ae9a4756aaa56440748c0c529fa312aca23e96 Mon Sep 17 00:00:00 2001 From: Jeffrey <22608443+Jefffrey@users.noreply.github.com> Date: Sat, 28 Oct 2023 20:40:13 +1100 Subject: [PATCH 02/43] Add simple exclude all columns test to sqllogictest (#7945) * Add simple exclude all columns test to sqllogictest * Add more exclude test cases --- datafusion/sqllogictest/test_files/select.slt | 20 +++++++++++++++++++ 1 file changed, 20 insertions(+) diff --git a/datafusion/sqllogictest/test_files/select.slt b/datafusion/sqllogictest/test_files/select.slt index 1d427479763a..98ea061c731b 100644 --- a/datafusion/sqllogictest/test_files/select.slt +++ b/datafusion/sqllogictest/test_files/select.slt @@ -848,6 +848,26 @@ statement error DataFusion error: Error during planning: EXCLUDE or EXCEPT conta SELECT * EXCLUDE(a, a) FROM table1 +# if EXCEPT all the columns, query should still succeed but return empty +statement ok +SELECT * EXCEPT(a, b, c, d) +FROM table1 + +# EXCLUDE order shouldn't matter +query II +SELECT * EXCLUDE(b, a) +FROM table1 +ORDER BY c +LIMIT 5 +---- +100 1000 +200 2000 + +# EXCLUDE with out of order but duplicate columns should error +statement error DataFusion error: Error during planning: EXCLUDE or EXCEPT contains duplicate column names +SELECT * EXCLUDE(d, b, c, a, a, b, c, d) +FROM table1 + # run below query in multi partitions statement ok set datafusion.execution.target_partitions = 2; From 250e71694b4f9789444e52b8cc12476dcbf35ac6 Mon Sep 17 00:00:00 2001 From: Devin D'Angelo Date: Sat, 28 Oct 2023 06:04:47 -0400 Subject: [PATCH 03/43] Support Partitioning Data by Dictionary Encoded String Array Types (#7896) * support dictionary encoded string columns for partition cols * remove debug prints * cargo fmt * generic dictionary cast and dict encoded test * updates from review * force retry checks * try checks again --- datafusion/common/src/dfschema.rs | 91 +++++++++++++++++++ .../src/datasource/file_format/write/demux.rs | 18 +++- .../core/src/datasource/listing/table.rs | 5 +- datafusion/core/src/datasource/memory.rs | 5 +- .../test_files/insert_to_external.slt | 38 +++++++- 5 files changed, 153 insertions(+), 4 deletions(-) diff --git a/datafusion/common/src/dfschema.rs b/datafusion/common/src/dfschema.rs index e16acbfedc81..d8cd103a4777 100644 --- a/datafusion/common/src/dfschema.rs +++ b/datafusion/common/src/dfschema.rs @@ -391,11 +391,33 @@ impl DFSchema { }) } + /// Returns true if the two schemas have the same qualified named + /// fields with logically equivalent data types. Returns false otherwise. + /// + /// Use [DFSchema]::equivalent_names_and_types for stricter semantic type + /// equivalence checking. + pub fn logically_equivalent_names_and_types(&self, other: &Self) -> bool { + if self.fields().len() != other.fields().len() { + return false; + } + let self_fields = self.fields().iter(); + let other_fields = other.fields().iter(); + self_fields.zip(other_fields).all(|(f1, f2)| { + f1.qualifier() == f2.qualifier() + && f1.name() == f2.name() + && Self::datatype_is_logically_equal(f1.data_type(), f2.data_type()) + }) + } + /// Returns true if the two schemas have the same qualified named /// fields with the same data types. Returns false otherwise. /// /// This is a specialized version of Eq that ignores differences /// in nullability and metadata. + /// + /// Use [DFSchema]::logically_equivalent_names_and_types for a weaker + /// logical type checking, which for example would consider a dictionary + /// encoded UTF8 array to be equivalent to a plain UTF8 array. pub fn equivalent_names_and_types(&self, other: &Self) -> bool { if self.fields().len() != other.fields().len() { return false; @@ -409,6 +431,46 @@ impl DFSchema { }) } + /// Checks if two [`DataType`]s are logically equal. This is a notably weaker constraint + /// than datatype_is_semantically_equal in that a Dictionary type is logically + /// equal to a plain V type, but not semantically equal. Dictionary is also + /// logically equal to Dictionary. + fn datatype_is_logically_equal(dt1: &DataType, dt2: &DataType) -> bool { + // check nested fields + match (dt1, dt2) { + (DataType::Dictionary(_, v1), DataType::Dictionary(_, v2)) => { + v1.as_ref() == v2.as_ref() + } + (DataType::Dictionary(_, v1), othertype) => v1.as_ref() == othertype, + (othertype, DataType::Dictionary(_, v1)) => v1.as_ref() == othertype, + (DataType::List(f1), DataType::List(f2)) + | (DataType::LargeList(f1), DataType::LargeList(f2)) + | (DataType::FixedSizeList(f1, _), DataType::FixedSizeList(f2, _)) + | (DataType::Map(f1, _), DataType::Map(f2, _)) => { + Self::field_is_logically_equal(f1, f2) + } + (DataType::Struct(fields1), DataType::Struct(fields2)) => { + let iter1 = fields1.iter(); + let iter2 = fields2.iter(); + fields1.len() == fields2.len() && + // all fields have to be the same + iter1 + .zip(iter2) + .all(|(f1, f2)| Self::field_is_logically_equal(f1, f2)) + } + (DataType::Union(fields1, _), DataType::Union(fields2, _)) => { + let iter1 = fields1.iter(); + let iter2 = fields2.iter(); + fields1.len() == fields2.len() && + // all fields have to be the same + iter1 + .zip(iter2) + .all(|((t1, f1), (t2, f2))| t1 == t2 && Self::field_is_logically_equal(f1, f2)) + } + _ => dt1 == dt2, + } + } + /// Returns true of two [`DataType`]s are semantically equal (same /// name and type), ignoring both metadata and nullability. /// @@ -456,6 +518,11 @@ impl DFSchema { } } + fn field_is_logically_equal(f1: &Field, f2: &Field) -> bool { + f1.name() == f2.name() + && Self::datatype_is_logically_equal(f1.data_type(), f2.data_type()) + } + fn field_is_semantically_equal(f1: &Field, f2: &Field) -> bool { f1.name() == f2.name() && Self::datatype_is_semantically_equal(f1.data_type(), f2.data_type()) @@ -786,6 +853,13 @@ pub trait SchemaExt { /// /// It works the same as [`DFSchema::equivalent_names_and_types`]. fn equivalent_names_and_types(&self, other: &Self) -> bool; + + /// Returns true if the two schemas have the same qualified named + /// fields with logically equivalent data types. Returns false otherwise. + /// + /// Use [DFSchema]::equivalent_names_and_types for stricter semantic type + /// equivalence checking. + fn logically_equivalent_names_and_types(&self, other: &Self) -> bool; } impl SchemaExt for Schema { @@ -805,6 +879,23 @@ impl SchemaExt for Schema { ) }) } + + fn logically_equivalent_names_and_types(&self, other: &Self) -> bool { + if self.fields().len() != other.fields().len() { + return false; + } + + self.fields() + .iter() + .zip(other.fields().iter()) + .all(|(f1, f2)| { + f1.name() == f2.name() + && DFSchema::datatype_is_logically_equal( + f1.data_type(), + f2.data_type(), + ) + }) + } } #[cfg(test)] diff --git a/datafusion/core/src/datasource/file_format/write/demux.rs b/datafusion/core/src/datasource/file_format/write/demux.rs index 67dd1f940676..27c65dd459ec 100644 --- a/datafusion/core/src/datasource/file_format/write/demux.rs +++ b/datafusion/core/src/datasource/file_format/write/demux.rs @@ -29,7 +29,7 @@ use crate::physical_plan::SendableRecordBatchStream; use arrow_array::builder::UInt64Builder; use arrow_array::cast::AsArray; -use arrow_array::{RecordBatch, StructArray}; +use arrow_array::{downcast_dictionary_array, RecordBatch, StringArray, StructArray}; use arrow_schema::{DataType, Schema}; use datafusion_common::cast::as_string_array; use datafusion_common::DataFusionError; @@ -338,6 +338,22 @@ fn compute_partition_keys_by_row<'a>( partition_values.push(array.value(i)); } } + DataType::Dictionary(_, _) => { + downcast_dictionary_array!( + col_array => { + let array = col_array.downcast_dict::() + .ok_or(DataFusionError::Execution(format!("it is not yet supported to write to hive partitions with datatype {}", + dtype)))?; + + for val in array.values() { + partition_values.push( + val.ok_or(DataFusionError::Execution(format!("Cannot partition by null value for column {}", col)))? + ); + } + }, + _ => unreachable!(), + ) + } _ => { return Err(DataFusionError::NotImplemented(format!( "it is not yet supported to write to hive partitions with datatype {}", diff --git a/datafusion/core/src/datasource/listing/table.rs b/datafusion/core/src/datasource/listing/table.rs index 822a78a5522a..d26d417bd8b2 100644 --- a/datafusion/core/src/datasource/listing/table.rs +++ b/datafusion/core/src/datasource/listing/table.rs @@ -824,7 +824,10 @@ impl TableProvider for ListingTable { overwrite: bool, ) -> Result> { // Check that the schema of the plan matches the schema of this table. - if !self.schema().equivalent_names_and_types(&input.schema()) { + if !self + .schema() + .logically_equivalent_names_and_types(&input.schema()) + { return plan_err!( // Return an error if schema of the input query does not match with the table schema. "Inserting query must have the same schema with the table." diff --git a/datafusion/core/src/datasource/memory.rs b/datafusion/core/src/datasource/memory.rs index a2f8e225e121..6bcaa97a408f 100644 --- a/datafusion/core/src/datasource/memory.rs +++ b/datafusion/core/src/datasource/memory.rs @@ -209,7 +209,10 @@ impl TableProvider for MemTable { ) -> Result> { // Create a physical plan from the logical plan. // Check that the schema of the plan matches the schema of this table. - if !self.schema().equivalent_names_and_types(&input.schema()) { + if !self + .schema() + .logically_equivalent_names_and_types(&input.schema()) + { return plan_err!( "Inserting query must have the same schema with the table." ); diff --git a/datafusion/sqllogictest/test_files/insert_to_external.slt b/datafusion/sqllogictest/test_files/insert_to_external.slt index b2206e987864..8b01a14568e7 100644 --- a/datafusion/sqllogictest/test_files/insert_to_external.slt +++ b/datafusion/sqllogictest/test_files/insert_to_external.slt @@ -40,8 +40,44 @@ STORED AS CSV WITH HEADER ROW LOCATION '../../testing/data/csv/aggregate_test_100.csv' -# test_insert_into +statement ok +create table dictionary_encoded_values as values +('a', arrow_cast('foo', 'Dictionary(Int32, Utf8)')), ('b', arrow_cast('bar', 'Dictionary(Int32, Utf8)')); + +query TTT +describe dictionary_encoded_values; +---- +column1 Utf8 YES +column2 Dictionary(Int32, Utf8) YES + +statement ok +CREATE EXTERNAL TABLE dictionary_encoded_parquet_partitioned( + a varchar, + b varchar, +) +STORED AS parquet +LOCATION 'test_files/scratch/insert_to_external/parquet_types_partitioned' +PARTITIONED BY (b) +OPTIONS( +create_local_path 'true', +insert_mode 'append_new_files', +); + +query TT +insert into dictionary_encoded_parquet_partitioned +select * from dictionary_encoded_values +---- +2 + +query TT +select * from dictionary_encoded_parquet_partitioned order by (a); +---- +a foo +b bar + + +# test_insert_into statement ok set datafusion.execution.target_partitions = 8; From d28c79d1809fd545bd4f3e15951267b7b1ec222c Mon Sep 17 00:00:00 2001 From: Jay Zhan Date: Sat, 28 Oct 2023 18:06:29 +0800 Subject: [PATCH 04/43] Minor: Remove array() in array_expression (#7961) * remove array Signed-off-by: jayzhan211 * cleanup others Signed-off-by: jayzhan211 * clippy Signed-off-by: jayzhan211 * cleanup cast Signed-off-by: jayzhan211 * fmt Signed-off-by: jayzhan211 * cleanup cast Signed-off-by: jayzhan211 --------- Signed-off-by: jayzhan211 --- .../physical-expr/src/array_expressions.rs | 391 ++++++++---------- 1 file changed, 172 insertions(+), 219 deletions(-) diff --git a/datafusion/physical-expr/src/array_expressions.rs b/datafusion/physical-expr/src/array_expressions.rs index af4612272676..7077f8b59860 100644 --- a/datafusion/physical-expr/src/array_expressions.rs +++ b/datafusion/physical-expr/src/array_expressions.rs @@ -25,12 +25,14 @@ use arrow::buffer::OffsetBuffer; use arrow::compute; use arrow::datatypes::{DataType, Field, UInt64Type}; use arrow_buffer::NullBuffer; -use datafusion_common::cast::{as_generic_string_array, as_int64_array, as_list_array}; + +use datafusion_common::cast::{ + as_generic_string_array, as_int64_array, as_list_array, as_string_array, +}; use datafusion_common::utils::wrap_into_list_array; use datafusion_common::{ exec_err, internal_err, not_impl_err, plan_err, DataFusionError, Result, }; -use datafusion_expr::ColumnarValue; use itertools::Itertools; @@ -395,21 +397,10 @@ fn array_array(args: &[ArrayRef], data_type: DataType) -> Result { Ok(res) } -/// Convert one or more [`ColumnarValue`] of the same type into a -/// `ListArray` -/// -/// See [`array_array`] for more details. -fn array(values: &[ColumnarValue]) -> Result { - let arrays: Vec = values - .iter() - .map(|x| match x { - ColumnarValue::Array(array) => array.clone(), - ColumnarValue::Scalar(scalar) => scalar.to_array(), - }) - .collect(); - +/// `make_array` SQL function +pub fn make_array(arrays: &[ArrayRef]) -> Result { let mut data_type = DataType::Null; - for arg in &arrays { + for arg in arrays { let arg_data_type = arg.data_type(); if !arg_data_type.equals_datatype(&DataType::Null) { data_type = arg_data_type.clone(); @@ -423,19 +414,10 @@ fn array(values: &[ColumnarValue]) -> Result { let array = new_null_array(&DataType::Null, arrays.len()); Ok(Arc::new(wrap_into_list_array(array))) } - data_type => array_array(arrays.as_slice(), data_type), + data_type => array_array(arrays, data_type), } } -/// `make_array` SQL function -pub fn make_array(arrays: &[ArrayRef]) -> Result { - let values: Vec = arrays - .iter() - .map(|x| ColumnarValue::Array(x.clone())) - .collect(); - array(values.as_slice()) -} - fn return_empty(return_null: bool, data_type: DataType) -> Arc { if return_null { new_null_array(&data_type, 1) @@ -654,7 +636,7 @@ pub fn array_append(args: &[ArrayRef]) -> Result { check_datatypes("array_append", &[arr.values(), element])?; let res = match arr.value_type() { DataType::List(_) => concat_internal(args)?, - DataType::Null => return array(&[ColumnarValue::Array(args[1].clone())]), + DataType::Null => return make_array(&[element.to_owned()]), data_type => { macro_rules! array_function { ($ARRAY_TYPE:ident) => { @@ -728,7 +710,7 @@ pub fn array_prepend(args: &[ArrayRef]) -> Result { check_datatypes("array_prepend", &[element, arr.values()])?; let res = match arr.value_type() { DataType::List(_) => concat_internal(args)?, - DataType::Null => return array(&[ColumnarValue::Array(args[0].clone())]), + DataType::Null => return make_array(&[element.to_owned()]), data_type => { macro_rules! array_function { ($ARRAY_TYPE:ident) => { @@ -1479,15 +1461,13 @@ macro_rules! to_string { pub fn array_to_string(args: &[ArrayRef]) -> Result { let arr = &args[0]; - let delimiters = as_generic_string_array::(&args[1])?; + let delimiters = as_string_array(&args[1])?; let delimiters: Vec> = delimiters.iter().collect(); let mut null_string = String::from(""); let mut with_null_string = false; if args.len() == 3 { - null_string = as_generic_string_array::(&args[2])? - .value(0) - .to_string(); + null_string = as_string_array(&args[2])?.value(0).to_string(); with_null_string = true; } @@ -1941,29 +1921,23 @@ pub fn string_to_array(args: &[ArrayRef]) -> Result() - .unwrap() + as_int64_array(&result.value(0)) + .expect("failed to cast to primitive array") .values() ) } @@ -1972,29 +1946,23 @@ mod tests { fn test_nested_array() { // make_array([1, 3, 5], [2, 4, 6]) = [[1, 3, 5], [2, 4, 6]] let args = [ - ColumnarValue::Array(Arc::new(Int64Array::from(vec![1, 2]))), - ColumnarValue::Array(Arc::new(Int64Array::from(vec![3, 4]))), - ColumnarValue::Array(Arc::new(Int64Array::from(vec![5, 6]))), + Arc::new(Int64Array::from(vec![1, 2])) as ArrayRef, + Arc::new(Int64Array::from(vec![3, 4])), + Arc::new(Int64Array::from(vec![5, 6])), ]; - let array = array(&args).expect("failed to initialize function array"); + let array = make_array(&args).expect("failed to initialize function array"); let result = as_list_array(&array).expect("failed to initialize function array"); assert_eq!(result.len(), 2); assert_eq!( &[1, 3, 5], - result - .value(0) - .as_any() - .downcast_ref::() - .unwrap() + as_int64_array(&result.value(0)) + .expect("failed to cast to primitive array") .values() ); assert_eq!( &[2, 4, 6], - result - .value(1) - .as_any() - .downcast_ref::() - .unwrap() + as_int64_array(&result.value(1)) + .expect("failed to cast to primitive array") .values() ); } @@ -2002,7 +1970,7 @@ mod tests { #[test] fn test_array_element() { // array_element([1, 2, 3, 4], 1) = 1 - let list_array = return_array().into_array(1); + let list_array = return_array(); let arr = array_element(&[list_array, Arc::new(Int64Array::from_value(1, 1))]) .expect("failed to initialize function array_element"); let result = @@ -2011,7 +1979,7 @@ mod tests { assert_eq!(result, &Int64Array::from_value(1, 1)); // array_element([1, 2, 3, 4], 3) = 3 - let list_array = return_array().into_array(1); + let list_array = return_array(); let arr = array_element(&[list_array, Arc::new(Int64Array::from_value(3, 1))]) .expect("failed to initialize function array_element"); let result = @@ -2020,7 +1988,7 @@ mod tests { assert_eq!(result, &Int64Array::from_value(3, 1)); // array_element([1, 2, 3, 4], 0) = NULL - let list_array = return_array().into_array(1); + let list_array = return_array(); let arr = array_element(&[list_array, Arc::new(Int64Array::from_value(0, 1))]) .expect("failed to initialize function array_element"); let result = @@ -2029,7 +1997,7 @@ mod tests { assert_eq!(result, &Int64Array::from(vec![None])); // array_element([1, 2, 3, 4], NULL) = NULL - let list_array = return_array().into_array(1); + let list_array = return_array(); let arr = array_element(&[list_array, Arc::new(Int64Array::from(vec![None]))]) .expect("failed to initialize function array_element"); let result = @@ -2038,7 +2006,7 @@ mod tests { assert_eq!(result, &Int64Array::from(vec![None])); // array_element([1, 2, 3, 4], -1) = 4 - let list_array = return_array().into_array(1); + let list_array = return_array(); let arr = array_element(&[list_array, Arc::new(Int64Array::from_value(-1, 1))]) .expect("failed to initialize function array_element"); let result = @@ -2047,7 +2015,7 @@ mod tests { assert_eq!(result, &Int64Array::from_value(4, 1)); // array_element([1, 2, 3, 4], -3) = 2 - let list_array = return_array().into_array(1); + let list_array = return_array(); let arr = array_element(&[list_array, Arc::new(Int64Array::from_value(-3, 1))]) .expect("failed to initialize function array_element"); let result = @@ -2056,7 +2024,7 @@ mod tests { assert_eq!(result, &Int64Array::from_value(2, 1)); // array_element([1, 2, 3, 4], 10) = NULL - let list_array = return_array().into_array(1); + let list_array = return_array(); let arr = array_element(&[list_array, Arc::new(Int64Array::from_value(10, 1))]) .expect("failed to initialize function array_element"); let result = @@ -2068,7 +2036,7 @@ mod tests { #[test] fn test_nested_array_element() { // array_element([[1, 2, 3, 4], [5, 6, 7, 8]], 2) = [5, 6, 7, 8] - let list_array = return_nested_array().into_array(1); + let list_array = return_nested_array(); let arr = array_element(&[list_array, Arc::new(Int64Array::from_value(2, 1))]) .expect("failed to initialize function array_element"); let result = @@ -2088,7 +2056,7 @@ mod tests { #[test] fn test_array_pop_back() { // array_pop_back([1, 2, 3, 4]) = [1, 2, 3] - let list_array = return_array().into_array(1); + let list_array = return_array(); let arr = array_pop_back(&[list_array]) .expect("failed to initialize function array_pop_back"); let result = @@ -2167,7 +2135,7 @@ mod tests { ); // array_pop_back([1, NULL, 3, NULL]) = [1, NULL, 3] - let list_array = return_array_with_nulls().into_array(1); + let list_array = return_array_with_nulls(); let arr = array_pop_back(&[list_array]) .expect("failed to initialize function array_pop_back"); let result = @@ -2185,7 +2153,7 @@ mod tests { #[test] fn test_nested_array_pop_back() { // array_pop_back([[1, 2, 3, 4], [5, 6, 7, 8]]) = [[1, 2, 3, 4]] - let list_array = return_nested_array().into_array(1); + let list_array = return_nested_array(); let arr = array_pop_back(&[list_array]) .expect("failed to initialize function array_slice"); let result = @@ -2233,7 +2201,7 @@ mod tests { #[test] fn test_array_slice() { // array_slice([1, 2, 3, 4], 1, 3) = [1, 2, 3] - let list_array = return_array().into_array(1); + let list_array = return_array(); let arr = array_slice(&[ list_array, Arc::new(Int64Array::from_value(1, 1)), @@ -2254,7 +2222,7 @@ mod tests { ); // array_slice([1, 2, 3, 4], 2, 2) = [2] - let list_array = return_array().into_array(1); + let list_array = return_array(); let arr = array_slice(&[ list_array, Arc::new(Int64Array::from_value(2, 1)), @@ -2275,7 +2243,7 @@ mod tests { ); // array_slice([1, 2, 3, 4], 0, 0) = [] - let list_array = return_array().into_array(1); + let list_array = return_array(); let arr = array_slice(&[ list_array, Arc::new(Int64Array::from_value(0, 1)), @@ -2293,7 +2261,7 @@ mod tests { .is_empty()); // array_slice([1, 2, 3, 4], 0, 6) = [1, 2, 3, 4] - let list_array = return_array().into_array(1); + let list_array = return_array(); let arr = array_slice(&[ list_array, Arc::new(Int64Array::from_value(0, 1)), @@ -2314,7 +2282,7 @@ mod tests { ); // array_slice([1, 2, 3, 4], -2, -2) = [] - let list_array = return_array().into_array(1); + let list_array = return_array(); let arr = array_slice(&[ list_array, Arc::new(Int64Array::from_value(-2, 1)), @@ -2332,7 +2300,7 @@ mod tests { .is_empty()); // array_slice([1, 2, 3, 4], -3, -1) = [2, 3] - let list_array = return_array().into_array(1); + let list_array = return_array(); let arr = array_slice(&[ list_array, Arc::new(Int64Array::from_value(-3, 1)), @@ -2353,7 +2321,7 @@ mod tests { ); // array_slice([1, 2, 3, 4], -3, 2) = [2] - let list_array = return_array().into_array(1); + let list_array = return_array(); let arr = array_slice(&[ list_array, Arc::new(Int64Array::from_value(-3, 1)), @@ -2374,7 +2342,7 @@ mod tests { ); // array_slice([1, 2, 3, 4], 2, 11) = [2, 3, 4] - let list_array = return_array().into_array(1); + let list_array = return_array(); let arr = array_slice(&[ list_array, Arc::new(Int64Array::from_value(2, 1)), @@ -2395,7 +2363,7 @@ mod tests { ); // array_slice([1, 2, 3, 4], 3, 1) = [] - let list_array = return_array().into_array(1); + let list_array = return_array(); let arr = array_slice(&[ list_array, Arc::new(Int64Array::from_value(3, 1)), @@ -2413,7 +2381,7 @@ mod tests { .is_empty()); // array_slice([1, 2, 3, 4], -7, -2) = NULL - let list_array = return_array().into_array(1); + let list_array = return_array(); let arr = array_slice(&[ list_array, Arc::new(Int64Array::from_value(-7, 1)), @@ -2434,7 +2402,7 @@ mod tests { #[test] fn test_nested_array_slice() { // array_slice([[1, 2, 3, 4], [5, 6, 7, 8]], 1, 1) = [[1, 2, 3, 4]] - let list_array = return_nested_array().into_array(1); + let list_array = return_nested_array(); let arr = array_slice(&[ list_array, Arc::new(Int64Array::from_value(1, 1)), @@ -2459,7 +2427,7 @@ mod tests { ); // array_slice([[1, 2, 3, 4], [5, 6, 7, 8]], -1, -1) = [] - let list_array = return_nested_array().into_array(1); + let list_array = return_nested_array(); let arr = array_slice(&[ list_array, Arc::new(Int64Array::from_value(-1, 1)), @@ -2477,7 +2445,7 @@ mod tests { .is_empty()); // array_slice([[1, 2, 3, 4], [5, 6, 7, 8]], -1, 2) = [[5, 6, 7, 8]] - let list_array = return_nested_array().into_array(1); + let list_array = return_nested_array(); let arr = array_slice(&[ list_array, Arc::new(Int64Array::from_value(-1, 1)), @@ -2588,7 +2556,7 @@ mod tests { #[test] fn test_nested_array_concat() { // array_concat([1, 2, 3, 4], [1, 2, 3, 4]) = [1, 2, 3, 4, 1, 2, 3, 4] - let list_array = return_array().into_array(1); + let list_array = return_array(); let arr = array_concat(&[list_array.clone(), list_array.clone()]) .expect("failed to initialize function array_concat"); let result = @@ -2605,8 +2573,8 @@ mod tests { ); // array_concat([[1, 2, 3, 4], [5, 6, 7, 8]], [1, 2, 3, 4]) = [[1, 2, 3, 4], [5, 6, 7, 8], [1, 2, 3, 4]] - let list_nested_array = return_nested_array().into_array(1); - let list_array = return_array().into_array(1); + let list_nested_array = return_nested_array(); + let list_array = return_array(); let arr = array_concat(&[list_nested_array, list_array]) .expect("failed to initialize function array_concat"); let result = @@ -2630,7 +2598,7 @@ mod tests { #[test] fn test_array_position() { // array_position([1, 2, 3, 4], 3) = 3 - let list_array = return_array().into_array(1); + let list_array = return_array(); let array = array_position(&[list_array, Arc::new(Int64Array::from_value(3, 1))]) .expect("failed to initialize function array_position"); let result = as_uint64_array(&array) @@ -2642,7 +2610,7 @@ mod tests { #[test] fn test_array_positions() { // array_positions([1, 2, 3, 4], 3) = [3] - let list_array = return_array().into_array(1); + let list_array = return_array(); let array = array_positions(&[list_array, Arc::new(Int64Array::from_value(3, 1))]) .expect("failed to initialize function array_position"); @@ -2664,7 +2632,7 @@ mod tests { #[test] fn test_array_remove() { // array_remove([3, 1, 2, 3, 2, 3], 3) = [1, 2, 3, 2, 3] - let list_array = return_array_with_repeating_elements().into_array(1); + let list_array = return_array_with_repeating_elements(); let array = array_remove(&[list_array, Arc::new(Int64Array::from_value(3, 1))]) .expect("failed to initialize function array_remove"); let result = @@ -2688,8 +2656,8 @@ mod tests { // [[1, 2, 3, 4], [5, 6, 7, 8], [1, 2, 3, 4], [9, 10, 11, 12], [5, 6, 7, 8]], // [1, 2, 3, 4], // ) = [[5, 6, 7, 8], [1, 2, 3, 4], [9, 10, 11, 12], [5, 6, 7, 8]] - let list_array = return_nested_array_with_repeating_elements().into_array(1); - let element_array = return_array().into_array(1); + let list_array = return_nested_array_with_repeating_elements(); + let element_array = return_array(); let array = array_remove(&[list_array, element_array]) .expect("failed to initialize function array_remove"); let result = @@ -2717,7 +2685,7 @@ mod tests { #[test] fn test_array_remove_n() { // array_remove_n([3, 1, 2, 3, 2, 3], 3, 2) = [1, 2, 2, 3] - let list_array = return_array_with_repeating_elements().into_array(1); + let list_array = return_array_with_repeating_elements(); let array = array_remove_n(&[ list_array, Arc::new(Int64Array::from_value(3, 1)), @@ -2746,8 +2714,8 @@ mod tests { // [1, 2, 3, 4], // 3, // ) = [[5, 6, 7, 8], [9, 10, 11, 12], [5, 6, 7, 8]] - let list_array = return_nested_array_with_repeating_elements().into_array(1); - let element_array = return_array().into_array(1); + let list_array = return_nested_array_with_repeating_elements(); + let element_array = return_array(); let array = array_remove_n(&[ list_array, element_array, @@ -2778,7 +2746,7 @@ mod tests { #[test] fn test_array_remove_all() { // array_remove_all([3, 1, 2, 3, 2, 3], 3) = [1, 2, 2] - let list_array = return_array_with_repeating_elements().into_array(1); + let list_array = return_array_with_repeating_elements(); let array = array_remove_all(&[list_array, Arc::new(Int64Array::from_value(3, 1))]) .expect("failed to initialize function array_remove_all"); @@ -2803,8 +2771,8 @@ mod tests { // [[1, 2, 3, 4], [5, 6, 7, 8], [1, 2, 3, 4], [9, 10, 11, 12], [5, 6, 7, 8]], // [1, 2, 3, 4], // ) = [[5, 6, 7, 8], [9, 10, 11, 12], [5, 6, 7, 8]] - let list_array = return_nested_array_with_repeating_elements().into_array(1); - let element_array = return_array().into_array(1); + let list_array = return_nested_array_with_repeating_elements(); + let element_array = return_array(); let array = array_remove_all(&[list_array, element_array]) .expect("failed to initialize function array_remove_all"); let result = as_list_array(&array) @@ -2831,7 +2799,7 @@ mod tests { #[test] fn test_array_replace() { // array_replace([3, 1, 2, 3, 2, 3], 3, 4) = [4, 1, 2, 3, 2, 3] - let list_array = return_array_with_repeating_elements().into_array(1); + let list_array = return_array_with_repeating_elements(); let array = array_replace(&[ list_array, Arc::new(Int64Array::from_value(3, 1)), @@ -2860,9 +2828,9 @@ mod tests { // [1, 2, 3, 4], // [11, 12, 13, 14], // ) = [[11, 12, 13, 14], [5, 6, 7, 8], [1, 2, 3, 4], [9, 10, 11, 12], [5, 6, 7, 8]] - let list_array = return_nested_array_with_repeating_elements().into_array(1); - let from_array = return_array().into_array(1); - let to_array = return_extra_array().into_array(1); + let list_array = return_nested_array_with_repeating_elements(); + let from_array = return_array(); + let to_array = return_extra_array(); let array = array_replace(&[list_array, from_array, to_array]) .expect("failed to initialize function array_replace"); let result = @@ -2891,7 +2859,7 @@ mod tests { #[test] fn test_array_replace_n() { // array_replace_n([3, 1, 2, 3, 2, 3], 3, 4, 2) = [4, 1, 2, 4, 2, 3] - let list_array = return_array_with_repeating_elements().into_array(1); + let list_array = return_array_with_repeating_elements(); let array = array_replace_n(&[ list_array, Arc::new(Int64Array::from_value(3, 1)), @@ -2922,9 +2890,9 @@ mod tests { // [11, 12, 13, 14], // 2, // ) = [[11, 12, 13, 14], [5, 6, 7, 8], [11, 12, 13, 14], [9, 10, 11, 12], [5, 6, 7, 8]] - let list_array = return_nested_array_with_repeating_elements().into_array(1); - let from_array = return_array().into_array(1); - let to_array = return_extra_array().into_array(1); + let list_array = return_nested_array_with_repeating_elements(); + let from_array = return_array(); + let to_array = return_extra_array(); let array = array_replace_n(&[ list_array, from_array, @@ -2958,7 +2926,7 @@ mod tests { #[test] fn test_array_replace_all() { // array_replace_all([3, 1, 2, 3, 2, 3], 3, 4) = [4, 1, 2, 4, 2, 4] - let list_array = return_array_with_repeating_elements().into_array(1); + let list_array = return_array_with_repeating_elements(); let array = array_replace_all(&[ list_array, Arc::new(Int64Array::from_value(3, 1)), @@ -2987,9 +2955,9 @@ mod tests { // [1, 2, 3, 4], // [11, 12, 13, 14], // ) = [[11, 12, 13, 14], [5, 6, 7, 8], [11, 12, 13, 14], [9, 10, 11, 12], [5, 6, 7, 8]] - let list_array = return_nested_array_with_repeating_elements().into_array(1); - let from_array = return_array().into_array(1); - let to_array = return_extra_array().into_array(1); + let list_array = return_nested_array_with_repeating_elements(); + let from_array = return_array(); + let to_array = return_extra_array(); let array = array_replace_all(&[list_array, from_array, to_array]) .expect("failed to initialize function array_replace_all"); let result = as_list_array(&array) @@ -3041,7 +3009,7 @@ mod tests { #[test] fn test_nested_array_repeat() { // array_repeat([1, 2, 3, 4], 3) = [[1, 2, 3, 4], [1, 2, 3, 4], [1, 2, 3, 4]] - let element = return_array().into_array(1); + let element = return_array(); let array = array_repeat(&[element, Arc::new(Int64Array::from_value(3, 1))]) .expect("failed to initialize function array_repeat"); let result = @@ -3067,25 +3035,25 @@ mod tests { #[test] fn test_array_to_string() { // array_to_string([1, 2, 3, 4], ',') = 1,2,3,4 - let list_array = return_array().into_array(1); + let list_array = return_array(); let array = array_to_string(&[list_array, Arc::new(StringArray::from(vec![Some(",")]))]) .expect("failed to initialize function array_to_string"); - let result = as_generic_string_array::(&array) + let result = as_string_array(&array) .expect("failed to initialize function array_to_string"); assert_eq!(result.len(), 1); assert_eq!("1,2,3,4", result.value(0)); // array_to_string([1, NULL, 3, NULL], ',', '*') = 1,*,3,* - let list_array = return_array_with_nulls().into_array(1); + let list_array = return_array_with_nulls(); let array = array_to_string(&[ list_array, Arc::new(StringArray::from(vec![Some(",")])), Arc::new(StringArray::from(vec![Some("*")])), ]) .expect("failed to initialize function array_to_string"); - let result = as_generic_string_array::(&array) + let result = as_string_array(&array) .expect("failed to initialize function array_to_string"); assert_eq!(result.len(), 1); @@ -3095,25 +3063,25 @@ mod tests { #[test] fn test_nested_array_to_string() { // array_to_string([[1, 2, 3, 4], [5, 6, 7, 8]], '-') = 1-2-3-4-5-6-7-8 - let list_array = return_nested_array().into_array(1); + let list_array = return_nested_array(); let array = array_to_string(&[list_array, Arc::new(StringArray::from(vec![Some("-")]))]) .expect("failed to initialize function array_to_string"); - let result = as_generic_string_array::(&array) + let result = as_string_array(&array) .expect("failed to initialize function array_to_string"); assert_eq!(result.len(), 1); assert_eq!("1-2-3-4-5-6-7-8", result.value(0)); // array_to_string([[1, NULL, 3, NULL], [NULL, 6, 7, NULL]], '-', '*') = 1-*-3-*-*-6-7-* - let list_array = return_nested_array_with_nulls().into_array(1); + let list_array = return_nested_array_with_nulls(); let array = array_to_string(&[ list_array, Arc::new(StringArray::from(vec![Some("-")])), Arc::new(StringArray::from(vec![Some("*")])), ]) .expect("failed to initialize function array_to_string"); - let result = as_generic_string_array::(&array) + let result = as_string_array(&array) .expect("failed to initialize function array_to_string"); assert_eq!(result.len(), 1); @@ -3123,7 +3091,7 @@ mod tests { #[test] fn test_cardinality() { // cardinality([1, 2, 3, 4]) = 4 - let list_array = return_array().into_array(1); + let list_array = return_array(); let arr = cardinality(&[list_array]) .expect("failed to initialize function cardinality"); let result = @@ -3135,7 +3103,7 @@ mod tests { #[test] fn test_nested_cardinality() { // cardinality([[1, 2, 3, 4], [5, 6, 7, 8]]) = 8 - let list_array = return_nested_array().into_array(1); + let list_array = return_nested_array(); let arr = cardinality(&[list_array]) .expect("failed to initialize function cardinality"); let result = @@ -3147,7 +3115,7 @@ mod tests { #[test] fn test_array_length() { // array_length([1, 2, 3, 4]) = 4 - let list_array = return_array().into_array(1); + let list_array = return_array(); let arr = array_length(&[list_array.clone()]) .expect("failed to initialize function array_ndims"); let result = @@ -3166,7 +3134,7 @@ mod tests { #[test] fn test_nested_array_length() { - let list_array = return_nested_array().into_array(1); + let list_array = return_nested_array(); // array_length([[1, 2, 3, 4], [5, 6, 7, 8]]) = 2 let arr = array_length(&[list_array.clone()]) @@ -3206,7 +3174,7 @@ mod tests { #[test] fn test_array_dims() { // array_dims([1, 2, 3, 4]) = [4] - let list_array = return_array().into_array(1); + let list_array = return_array(); let array = array_dims(&[list_array]).expect("failed to initialize function array_dims"); @@ -3227,7 +3195,7 @@ mod tests { #[test] fn test_nested_array_dims() { // array_dims([[1, 2, 3, 4], [5, 6, 7, 8]]) = [2, 4] - let list_array = return_nested_array().into_array(1); + let list_array = return_nested_array(); let array = array_dims(&[list_array]).expect("failed to initialize function array_dims"); @@ -3248,7 +3216,7 @@ mod tests { #[test] fn test_array_ndims() { // array_ndims([1, 2, 3, 4]) = 1 - let list_array = return_array().into_array(1); + let list_array = return_array(); let array = array_ndims(&[list_array]) .expect("failed to initialize function array_ndims"); @@ -3261,7 +3229,7 @@ mod tests { #[test] fn test_nested_array_ndims() { // array_ndims([[1, 2, 3, 4], [5, 6, 7, 8]]) = 2 - let list_array = return_nested_array().into_array(1); + let list_array = return_nested_array(); let array = array_ndims(&[list_array]) .expect("failed to initialize function array_ndims"); @@ -3285,152 +3253,137 @@ mod tests { assert_eq!(array.unwrap_err().strip_backtrace(), "Error during planning: array_append received incompatible types: '[Int64, Utf8]'."); } - fn return_array() -> ColumnarValue { + fn return_array() -> ArrayRef { // Returns: [1, 2, 3, 4] let args = [ - ColumnarValue::Scalar(ScalarValue::Int64(Some(1))), - ColumnarValue::Scalar(ScalarValue::Int64(Some(2))), - ColumnarValue::Scalar(ScalarValue::Int64(Some(3))), - ColumnarValue::Scalar(ScalarValue::Int64(Some(4))), + Arc::new(Int64Array::from(vec![Some(1)])) as ArrayRef, + Arc::new(Int64Array::from(vec![Some(2)])) as ArrayRef, + Arc::new(Int64Array::from(vec![Some(3)])) as ArrayRef, + Arc::new(Int64Array::from(vec![Some(4)])) as ArrayRef, ]; - let result = array(&args).expect("failed to initialize function array"); - ColumnarValue::Array(result.clone()) + make_array(&args).expect("failed to initialize function array") } - fn return_extra_array() -> ColumnarValue { + fn return_extra_array() -> ArrayRef { // Returns: [11, 12, 13, 14] let args = [ - ColumnarValue::Scalar(ScalarValue::Int64(Some(11))), - ColumnarValue::Scalar(ScalarValue::Int64(Some(12))), - ColumnarValue::Scalar(ScalarValue::Int64(Some(13))), - ColumnarValue::Scalar(ScalarValue::Int64(Some(14))), + Arc::new(Int64Array::from(vec![Some(11)])) as ArrayRef, + Arc::new(Int64Array::from(vec![Some(12)])) as ArrayRef, + Arc::new(Int64Array::from(vec![Some(13)])) as ArrayRef, + Arc::new(Int64Array::from(vec![Some(14)])) as ArrayRef, ]; - let result = array(&args).expect("failed to initialize function array"); - ColumnarValue::Array(result.clone()) + make_array(&args).expect("failed to initialize function array") } - fn return_nested_array() -> ColumnarValue { + fn return_nested_array() -> ArrayRef { // Returns: [[1, 2, 3, 4], [5, 6, 7, 8]] let args = [ - ColumnarValue::Scalar(ScalarValue::Int64(Some(1))), - ColumnarValue::Scalar(ScalarValue::Int64(Some(2))), - ColumnarValue::Scalar(ScalarValue::Int64(Some(3))), - ColumnarValue::Scalar(ScalarValue::Int64(Some(4))), + Arc::new(Int64Array::from(vec![Some(1)])) as ArrayRef, + Arc::new(Int64Array::from(vec![Some(2)])) as ArrayRef, + Arc::new(Int64Array::from(vec![Some(3)])) as ArrayRef, + Arc::new(Int64Array::from(vec![Some(4)])) as ArrayRef, ]; - let arr1 = array(&args).expect("failed to initialize function array"); + let arr1 = make_array(&args).expect("failed to initialize function array"); let args = [ - ColumnarValue::Scalar(ScalarValue::Int64(Some(5))), - ColumnarValue::Scalar(ScalarValue::Int64(Some(6))), - ColumnarValue::Scalar(ScalarValue::Int64(Some(7))), - ColumnarValue::Scalar(ScalarValue::Int64(Some(8))), + Arc::new(Int64Array::from(vec![Some(5)])) as ArrayRef, + Arc::new(Int64Array::from(vec![Some(6)])) as ArrayRef, + Arc::new(Int64Array::from(vec![Some(7)])) as ArrayRef, + Arc::new(Int64Array::from(vec![Some(8)])) as ArrayRef, ]; - let arr2 = array(&args).expect("failed to initialize function array"); + let arr2 = make_array(&args).expect("failed to initialize function array"); - let args = [ColumnarValue::Array(arr1), ColumnarValue::Array(arr2)]; - let result = array(&args).expect("failed to initialize function array"); - ColumnarValue::Array(result.clone()) + make_array(&[arr1, arr2]).expect("failed to initialize function array") } - fn return_array_with_nulls() -> ColumnarValue { + fn return_array_with_nulls() -> ArrayRef { // Returns: [1, NULL, 3, NULL] let args = [ - ColumnarValue::Scalar(ScalarValue::Int64(Some(1))), - ColumnarValue::Scalar(ScalarValue::Null), - ColumnarValue::Scalar(ScalarValue::Int64(Some(3))), - ColumnarValue::Scalar(ScalarValue::Null), + Arc::new(Int64Array::from(vec![Some(1)])) as ArrayRef, + Arc::new(Int64Array::from(vec![None])) as ArrayRef, + Arc::new(Int64Array::from(vec![Some(3)])) as ArrayRef, + Arc::new(Int64Array::from(vec![None])) as ArrayRef, ]; - let result = array(&args).expect("failed to initialize function array"); - ColumnarValue::Array(result.clone()) + make_array(&args).expect("failed to initialize function array") } - fn return_nested_array_with_nulls() -> ColumnarValue { + fn return_nested_array_with_nulls() -> ArrayRef { // Returns: [[1, NULL, 3, NULL], [NULL, 6, 7, NULL]] let args = [ - ColumnarValue::Scalar(ScalarValue::Int64(Some(1))), - ColumnarValue::Scalar(ScalarValue::Null), - ColumnarValue::Scalar(ScalarValue::Int64(Some(3))), - ColumnarValue::Scalar(ScalarValue::Null), + Arc::new(Int64Array::from(vec![Some(1)])) as ArrayRef, + Arc::new(Int64Array::from(vec![None])) as ArrayRef, + Arc::new(Int64Array::from(vec![Some(3)])) as ArrayRef, + Arc::new(Int64Array::from(vec![None])) as ArrayRef, ]; - let arr1 = array(&args).expect("failed to initialize function array"); + let arr1 = make_array(&args).expect("failed to initialize function array"); let args = [ - ColumnarValue::Scalar(ScalarValue::Null), - ColumnarValue::Scalar(ScalarValue::Int64(Some(6))), - ColumnarValue::Scalar(ScalarValue::Int64(Some(7))), - ColumnarValue::Scalar(ScalarValue::Null), + Arc::new(Int64Array::from(vec![None])) as ArrayRef, + Arc::new(Int64Array::from(vec![Some(6)])) as ArrayRef, + Arc::new(Int64Array::from(vec![Some(7)])) as ArrayRef, + Arc::new(Int64Array::from(vec![None])) as ArrayRef, ]; - let arr2 = array(&args).expect("failed to initialize function array"); + let arr2 = make_array(&args).expect("failed to initialize function array"); - let args = [ColumnarValue::Array(arr1), ColumnarValue::Array(arr2)]; - let result = array(&args).expect("failed to initialize function array"); - ColumnarValue::Array(result.clone()) + make_array(&[arr1, arr2]).expect("failed to initialize function array") } - fn return_array_with_repeating_elements() -> ColumnarValue { + fn return_array_with_repeating_elements() -> ArrayRef { // Returns: [3, 1, 2, 3, 2, 3] let args = [ - ColumnarValue::Scalar(ScalarValue::Int64(Some(3))), - ColumnarValue::Scalar(ScalarValue::Int64(Some(1))), - ColumnarValue::Scalar(ScalarValue::Int64(Some(2))), - ColumnarValue::Scalar(ScalarValue::Int64(Some(3))), - ColumnarValue::Scalar(ScalarValue::Int64(Some(2))), - ColumnarValue::Scalar(ScalarValue::Int64(Some(3))), + Arc::new(Int64Array::from(vec![Some(3)])) as ArrayRef, + Arc::new(Int64Array::from(vec![Some(1)])) as ArrayRef, + Arc::new(Int64Array::from(vec![Some(2)])) as ArrayRef, + Arc::new(Int64Array::from(vec![Some(3)])) as ArrayRef, + Arc::new(Int64Array::from(vec![Some(2)])) as ArrayRef, + Arc::new(Int64Array::from(vec![Some(3)])) as ArrayRef, ]; - let result = array(&args).expect("failed to initialize function array"); - ColumnarValue::Array(result.clone()) + make_array(&args).expect("failed to initialize function array") } - fn return_nested_array_with_repeating_elements() -> ColumnarValue { + fn return_nested_array_with_repeating_elements() -> ArrayRef { // Returns: [[1, 2, 3, 4], [5, 6, 7, 8], [1, 2, 3, 4], [9, 10, 11, 12], [5, 6, 7, 8]] let args = [ - ColumnarValue::Scalar(ScalarValue::Int64(Some(1))), - ColumnarValue::Scalar(ScalarValue::Int64(Some(2))), - ColumnarValue::Scalar(ScalarValue::Int64(Some(3))), - ColumnarValue::Scalar(ScalarValue::Int64(Some(4))), + Arc::new(Int64Array::from(vec![Some(1)])) as ArrayRef, + Arc::new(Int64Array::from(vec![Some(2)])) as ArrayRef, + Arc::new(Int64Array::from(vec![Some(3)])) as ArrayRef, + Arc::new(Int64Array::from(vec![Some(4)])) as ArrayRef, ]; - let arr1 = array(&args).expect("failed to initialize function array"); + let arr1 = make_array(&args).expect("failed to initialize function array"); let args = [ - ColumnarValue::Scalar(ScalarValue::Int64(Some(5))), - ColumnarValue::Scalar(ScalarValue::Int64(Some(6))), - ColumnarValue::Scalar(ScalarValue::Int64(Some(7))), - ColumnarValue::Scalar(ScalarValue::Int64(Some(8))), + Arc::new(Int64Array::from(vec![Some(5)])) as ArrayRef, + Arc::new(Int64Array::from(vec![Some(6)])) as ArrayRef, + Arc::new(Int64Array::from(vec![Some(7)])) as ArrayRef, + Arc::new(Int64Array::from(vec![Some(8)])) as ArrayRef, ]; - let arr2 = array(&args).expect("failed to initialize function array"); + let arr2 = make_array(&args).expect("failed to initialize function array"); let args = [ - ColumnarValue::Scalar(ScalarValue::Int64(Some(1))), - ColumnarValue::Scalar(ScalarValue::Int64(Some(2))), - ColumnarValue::Scalar(ScalarValue::Int64(Some(3))), - ColumnarValue::Scalar(ScalarValue::Int64(Some(4))), + Arc::new(Int64Array::from(vec![Some(1)])) as ArrayRef, + Arc::new(Int64Array::from(vec![Some(2)])) as ArrayRef, + Arc::new(Int64Array::from(vec![Some(3)])) as ArrayRef, + Arc::new(Int64Array::from(vec![Some(4)])) as ArrayRef, ]; - let arr3 = array(&args).expect("failed to initialize function array"); + let arr3 = make_array(&args).expect("failed to initialize function array"); let args = [ - ColumnarValue::Scalar(ScalarValue::Int64(Some(9))), - ColumnarValue::Scalar(ScalarValue::Int64(Some(10))), - ColumnarValue::Scalar(ScalarValue::Int64(Some(11))), - ColumnarValue::Scalar(ScalarValue::Int64(Some(12))), + Arc::new(Int64Array::from(vec![Some(9)])) as ArrayRef, + Arc::new(Int64Array::from(vec![Some(10)])) as ArrayRef, + Arc::new(Int64Array::from(vec![Some(11)])) as ArrayRef, + Arc::new(Int64Array::from(vec![Some(12)])) as ArrayRef, ]; - let arr4 = array(&args).expect("failed to initialize function array"); + let arr4 = make_array(&args).expect("failed to initialize function array"); let args = [ - ColumnarValue::Scalar(ScalarValue::Int64(Some(5))), - ColumnarValue::Scalar(ScalarValue::Int64(Some(6))), - ColumnarValue::Scalar(ScalarValue::Int64(Some(7))), - ColumnarValue::Scalar(ScalarValue::Int64(Some(8))), + Arc::new(Int64Array::from(vec![Some(5)])) as ArrayRef, + Arc::new(Int64Array::from(vec![Some(6)])) as ArrayRef, + Arc::new(Int64Array::from(vec![Some(7)])) as ArrayRef, + Arc::new(Int64Array::from(vec![Some(8)])) as ArrayRef, ]; - let arr5 = array(&args).expect("failed to initialize function array"); + let arr5 = make_array(&args).expect("failed to initialize function array"); - let args = [ - ColumnarValue::Array(arr1), - ColumnarValue::Array(arr2), - ColumnarValue::Array(arr3), - ColumnarValue::Array(arr4), - ColumnarValue::Array(arr5), - ]; - let result = array(&args).expect("failed to initialize function array"); - ColumnarValue::Array(result.clone()) + make_array(&[arr1, arr2, arr3, arr4, arr5]) + .expect("failed to initialize function array") } } From b02fe5bbd7854bf186c54c542b80ab75ac323c38 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Sat, 28 Oct 2023 11:40:34 -0400 Subject: [PATCH 05/43] Minor: simplify update code (#7943) --- datafusion/sql/src/statement.rs | 22 +++++++--------------- 1 file changed, 7 insertions(+), 15 deletions(-) diff --git a/datafusion/sql/src/statement.rs b/datafusion/sql/src/statement.rs index b5196c086638..80a27db6e63d 100644 --- a/datafusion/sql/src/statement.rs +++ b/datafusion/sql/src/statement.rs @@ -36,7 +36,6 @@ use datafusion_common::{ Result, SchemaReference, TableReference, ToDFSchema, }; use datafusion_expr::dml::{CopyOptions, CopyTo}; -use datafusion_expr::expr::Placeholder; use datafusion_expr::expr_rewriter::normalize_col_with_schemas_and_ambiguity_check; use datafusion_expr::logical_plan::builder::project; use datafusion_expr::logical_plan::DdlStatement; @@ -1025,20 +1024,13 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { // Projection let mut exprs = vec![]; for (col_name, expr, dt) in values_and_types.into_iter() { - let expr = self.sql_to_expr(expr, &table_schema, &mut planner_context)?; - let expr = match expr { - datafusion_expr::Expr::Placeholder(Placeholder { - ref id, - ref data_type, - }) => match data_type { - None => datafusion_expr::Expr::Placeholder(Placeholder::new( - id.clone(), - Some(dt.clone()), - )), - Some(_) => expr, - }, - _ => expr, - }; + let mut expr = self.sql_to_expr(expr, &table_schema, &mut planner_context)?; + // Update placeholder's datatype to the type of the target column + if let datafusion_expr::Expr::Placeholder(placeholder) = &mut expr { + placeholder.data_type = + placeholder.data_type.take().or_else(|| Some(dt.clone())); + } + // Cast to target column type, if necessary let expr = expr.cast_to(dt, source.schema())?.alias(col_name); exprs.push(expr); } From 9ee055a3f59ec08de3432a3ec3de8ff55d29975a Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Sat, 28 Oct 2023 09:45:16 -0600 Subject: [PATCH 06/43] Add some initial content about creating logical plans (#7952) --- Cargo.toml | 1 + .../src/datasource/default_table_source.rs | 6 +- datafusion/core/src/prelude.rs | 2 +- dev/update_datafusion_versions.py | 1 + docs/Cargo.toml | 32 +++++ .../building-logical-plans.md | 129 +++++++++++++++++- docs/src/lib.rs | 19 +++ docs/src/library_logical_plan.rs | 78 +++++++++++ 8 files changed, 264 insertions(+), 4 deletions(-) create mode 100644 docs/Cargo.toml create mode 100644 docs/src/lib.rs create mode 100644 docs/src/library_logical_plan.rs diff --git a/Cargo.toml b/Cargo.toml index 71088e7fc7ad..77e3c6038ea7 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -32,6 +32,7 @@ members = [ "datafusion/substrait", "datafusion/wasmtest", "datafusion-examples", + "docs", "test-utils", "benchmarks", ] diff --git a/datafusion/core/src/datasource/default_table_source.rs b/datafusion/core/src/datasource/default_table_source.rs index f93faa50a9b9..00a9c123ceee 100644 --- a/datafusion/core/src/datasource/default_table_source.rs +++ b/datafusion/core/src/datasource/default_table_source.rs @@ -26,10 +26,12 @@ use arrow::datatypes::SchemaRef; use datafusion_common::{internal_err, Constraints, DataFusionError}; use datafusion_expr::{Expr, TableProviderFilterPushDown, TableSource}; -/// DataFusion default table source, wrapping TableProvider +/// DataFusion default table source, wrapping TableProvider. /// /// This structure adapts a `TableProvider` (physical plan trait) to the `TableSource` -/// (logical plan trait) +/// (logical plan trait) and is necessary because the logical plan is contained in +/// the `datafusion_expr` crate, and is not aware of table providers, which exist in +/// the core `datafusion` crate. pub struct DefaultTableSource { /// table provider pub table_provider: Arc, diff --git a/datafusion/core/src/prelude.rs b/datafusion/core/src/prelude.rs index 7689468e5d13..5cd8b3870f81 100644 --- a/datafusion/core/src/prelude.rs +++ b/datafusion/core/src/prelude.rs @@ -13,7 +13,7 @@ // "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.pub}, +// under the License. //! DataFusion "prelude" to simplify importing common types. //! diff --git a/dev/update_datafusion_versions.py b/dev/update_datafusion_versions.py index 7cbe39fdfb66..19701b813671 100755 --- a/dev/update_datafusion_versions.py +++ b/dev/update_datafusion_versions.py @@ -43,6 +43,7 @@ 'datafusion-wasmtest': 'datafusion/wasmtest/Cargo.toml', 'datafusion-benchmarks': 'benchmarks/Cargo.toml', 'datafusion-examples': 'datafusion-examples/Cargo.toml', + 'datafusion-docs': 'docs/Cargo.toml', } def update_workspace_version(new_version: str): diff --git a/docs/Cargo.toml b/docs/Cargo.toml new file mode 100644 index 000000000000..9caa0bde3608 --- /dev/null +++ b/docs/Cargo.toml @@ -0,0 +1,32 @@ +# 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. + +[package] +name = "datafusion-docs-tests" +description = "DataFusion Documentation Tests" +publish = false +version = { workspace = true } +edition = { workspace = true } +readme = { workspace = true } +homepage = { workspace = true } +repository = { workspace = true } +license = { workspace = true } +authors = { workspace = true } +rust-version = "1.70" + +[dependencies] +datafusion = { path = "../datafusion/core", version = "32.0.0", default-features = false } diff --git a/docs/source/library-user-guide/building-logical-plans.md b/docs/source/library-user-guide/building-logical-plans.md index 406f4881129c..fe922d8eaeb1 100644 --- a/docs/source/library-user-guide/building-logical-plans.md +++ b/docs/source/library-user-guide/building-logical-plans.md @@ -19,4 +19,131 @@ # Building Logical Plans -Coming Soon +A logical plan is a structured representation of a database query that describes the high-level operations and +transformations needed to retrieve data from a database or data source. It abstracts away specific implementation +details and focuses on the logical flow of the query, including operations like filtering, sorting, and joining tables. + +This logical plan serves as an intermediate step before generating an optimized physical execution plan. This is +explained in more detail in the [Query Planning and Execution Overview] section of the [Architecture Guide]. + +## Building Logical Plans Manually + +DataFusion's [LogicalPlan] is an enum containing variants representing all the supported operators, and also +contains an `Extension` variant that allows projects building on DataFusion to add custom logical operators. + +It is possible to create logical plans by directly creating instances of the [LogicalPlan] enum as follows, but is is +much easier to use the [LogicalPlanBuilder], which is described in the next section. + +Here is an example of building a logical plan directly: + + + +```rust +// create a logical table source +let schema = Schema::new(vec![ + Field::new("id", DataType::Int32, true), + Field::new("name", DataType::Utf8, true), +]); +let table_source = LogicalTableSource::new(SchemaRef::new(schema)); + +// create a TableScan plan +let projection = None; // optional projection +let filters = vec![]; // optional filters to push down +let fetch = None; // optional LIMIT +let table_scan = LogicalPlan::TableScan(TableScan::try_new( + "person", + Arc::new(table_source), + projection, + filters, + fetch, +)?); + +// create a Filter plan that evaluates `id > 500` that wraps the TableScan +let filter_expr = col("id").gt(lit(500)); +let plan = LogicalPlan::Filter(Filter::try_new(filter_expr, Arc::new(table_scan))?); + +// print the plan +println!("{}", plan.display_indent_schema()); +``` + +This example produces the following plan: + +``` +Filter: person.id > Int32(500) [id:Int32;N, name:Utf8;N] + TableScan: person [id:Int32;N, name:Utf8;N] +``` + +## Building Logical Plans with LogicalPlanBuilder + +DataFusion logical plans can be created using the [LogicalPlanBuilder] struct. There is also a [DataFrame] API which is +a higher-level API that delegates to [LogicalPlanBuilder]. + +The following associated functions can be used to create a new builder: + +- `empty` - create an empty plan with no fields +- `values` - create a plan from a set of literal values +- `scan` - create a plan representing a table scan +- `scan_with_filters` - create a plan representing a table scan with filters + +Once the builder is created, transformation methods can be called to declare that further operations should be +performed on the plan. Note that all we are doing at this stage is building up the logical plan structure. No query +execution will be performed. + +Here are some examples of transformation methods, but for a full list, refer to the [LogicalPlanBuilder] API documentation. + +- `filter` +- `limit` +- `sort` +- `distinct` +- `join` + +The following example demonstrates building the same simple query plan as the previous example, with a table scan followed by a filter. + + + +```rust +// create a logical table source +let schema = Schema::new(vec![ + Field::new("id", DataType::Int32, true), + Field::new("name", DataType::Utf8, true), +]); +let table_source = LogicalTableSource::new(SchemaRef::new(schema)); + +// optional projection +let projection = None; + +// create a LogicalPlanBuilder for a table scan +let builder = LogicalPlanBuilder::scan("person", Arc::new(table_source), projection)?; + +// perform a filter operation and build the plan +let plan = builder + .filter(col("id").gt(lit(500)))? // WHERE id > 500 + .build()?; + +// print the plan +println!("{}", plan.display_indent_schema()); +``` + +This example produces the following plan: + +``` +Filter: person.id > Int32(500) [id:Int32;N, name:Utf8;N] + TableScan: person [id:Int32;N, name:Utf8;N] +``` + +## Table Sources + +The previous example used a [LogicalTableSource], which is used for tests and documentation in DataFusion, and is also +suitable if you are using DataFusion to build logical plans but do not use DataFusion's physical planner. However, if you +want to use a [TableSource] that can be executed in DataFusion then you will need to use [DefaultTableSource], which is a +wrapper for a [TableProvider]. + +[query planning and execution overview]: https://docs.rs/datafusion/latest/datafusion/index.html#query-planning-and-execution-overview +[architecture guide]: https://docs.rs/datafusion/latest/datafusion/index.html#architecture +[logicalplan]: https://docs.rs/datafusion-expr/latest/datafusion_expr/logical_plan/enum.LogicalPlan.html +[logicalplanbuilder]: https://docs.rs/datafusion-expr/latest/datafusion_expr/logical_plan/builder/struct.LogicalPlanBuilder.html +[dataframe]: using-the-dataframe-api.md +[logicaltablesource]: https://docs.rs/datafusion-expr/latest/datafusion_expr/logical_plan/builder/struct.LogicalTableSource.html +[defaulttablesource]: https://docs.rs/datafusion/latest/datafusion/datasource/default_table_source/struct.DefaultTableSource.html +[tableprovider]: https://docs.rs/datafusion/latest/datafusion/datasource/provider/trait.TableProvider.html +[tablesource]: https://docs.rs/datafusion-expr/latest/datafusion_expr/trait.TableSource.html diff --git a/docs/src/lib.rs b/docs/src/lib.rs new file mode 100644 index 000000000000..f73132468ec9 --- /dev/null +++ b/docs/src/lib.rs @@ -0,0 +1,19 @@ +// 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(test)] +mod library_logical_plan; diff --git a/docs/src/library_logical_plan.rs b/docs/src/library_logical_plan.rs new file mode 100644 index 000000000000..355003941570 --- /dev/null +++ b/docs/src/library_logical_plan.rs @@ -0,0 +1,78 @@ +// 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 datafusion::arrow::datatypes::{DataType, Field, Schema, SchemaRef}; +use datafusion::error::Result; +use datafusion::logical_expr::builder::LogicalTableSource; +use datafusion::logical_expr::{Filter, LogicalPlan, LogicalPlanBuilder, TableScan}; +use datafusion::prelude::*; +use std::sync::Arc; + +#[test] +fn plan_1() -> Result<()> { + // create a logical table source + let schema = Schema::new(vec![ + Field::new("id", DataType::Int32, true), + Field::new("name", DataType::Utf8, true), + ]); + let table_source = LogicalTableSource::new(SchemaRef::new(schema)); + + // create a TableScan plan + let projection = None; // optional projection + let filters = vec![]; // optional filters to push down + let fetch = None; // optional LIMIT + let table_scan = LogicalPlan::TableScan(TableScan::try_new( + "person", + Arc::new(table_source), + projection, + filters, + fetch, + )?); + + // create a Filter plan that evaluates `id > 500` and wraps the TableScan + let filter_expr = col("id").gt(lit(500)); + let plan = LogicalPlan::Filter(Filter::try_new(filter_expr, Arc::new(table_scan))?); + + // print the plan + println!("{}", plan.display_indent_schema()); + + Ok(()) +} + +#[test] +fn plan_builder_1() -> Result<()> { + // create a logical table source + let schema = Schema::new(vec![ + Field::new("id", DataType::Int32, true), + Field::new("name", DataType::Utf8, true), + ]); + let table_source = LogicalTableSource::new(SchemaRef::new(schema)); + + // optional projection + let projection = None; + + // create a LogicalPlanBuilder for a table scan + let builder = LogicalPlanBuilder::scan("person", Arc::new(table_source), projection)?; + + // perform a filter that evaluates `id > 500`, and build the plan + let plan = builder.filter(col("id").gt(lit(500)))?.build()?; + + // print the plan + println!("{}", plan.display_indent_schema()); + + Ok(()) +} From d24228a6b9552f2dce166836e60c1b928116933f Mon Sep 17 00:00:00 2001 From: yi wang <48236141+my-vegetable-has-exploded@users.noreply.github.com> Date: Sun, 29 Oct 2023 19:34:29 +0800 Subject: [PATCH 07/43] Minor: Change from `&mut SessionContext` to `&SessionContext` in substrait (#7965) * Lower &mut SessionContext in substrait * rm mut ctx in tests --- .../substrait/src/logical_plan/consumer.rs | 4 +-- .../substrait/src/physical_plan/consumer.rs | 2 +- .../tests/cases/roundtrip_logical_plan.rs | 30 +++++++++---------- .../tests/cases/roundtrip_physical_plan.rs | 4 +-- datafusion/substrait/tests/cases/serialize.rs | 4 +-- 5 files changed, 22 insertions(+), 22 deletions(-) diff --git a/datafusion/substrait/src/logical_plan/consumer.rs b/datafusion/substrait/src/logical_plan/consumer.rs index ae65a2c7d94a..a15121652452 100644 --- a/datafusion/substrait/src/logical_plan/consumer.rs +++ b/datafusion/substrait/src/logical_plan/consumer.rs @@ -177,7 +177,7 @@ fn split_eq_and_noneq_join_predicate_with_nulls_equality( /// Convert Substrait Plan to DataFusion DataFrame pub async fn from_substrait_plan( - ctx: &mut SessionContext, + ctx: &SessionContext, plan: &Plan, ) -> Result { // Register function extension @@ -219,7 +219,7 @@ pub async fn from_substrait_plan( /// Convert Substrait Rel to DataFusion DataFrame #[async_recursion] pub async fn from_substrait_rel( - ctx: &mut SessionContext, + ctx: &SessionContext, rel: &Rel, extensions: &HashMap, ) -> Result { diff --git a/datafusion/substrait/src/physical_plan/consumer.rs b/datafusion/substrait/src/physical_plan/consumer.rs index 7788ba0a69de..1dab1f9d5e39 100644 --- a/datafusion/substrait/src/physical_plan/consumer.rs +++ b/datafusion/substrait/src/physical_plan/consumer.rs @@ -38,7 +38,7 @@ use substrait::proto::{ /// Convert Substrait Rel to DataFusion ExecutionPlan #[async_recursion] pub async fn from_substrait_rel( - _ctx: &mut SessionContext, + _ctx: &SessionContext, rel: &Rel, _extensions: &HashMap, ) -> Result> { diff --git a/datafusion/substrait/tests/cases/roundtrip_logical_plan.rs b/datafusion/substrait/tests/cases/roundtrip_logical_plan.rs index 32416125de24..ca2b4d48c460 100644 --- a/datafusion/substrait/tests/cases/roundtrip_logical_plan.rs +++ b/datafusion/substrait/tests/cases/roundtrip_logical_plan.rs @@ -606,7 +606,7 @@ async fn new_test_grammar() -> Result<()> { #[tokio::test] async fn extension_logical_plan() -> Result<()> { - let mut ctx = create_context().await?; + let ctx = create_context().await?; let validation_bytes = "MockUserDefinedLogicalPlan".as_bytes().to_vec(); let ext_plan = LogicalPlan::Extension(Extension { node: Arc::new(MockUserDefinedLogicalPlan { @@ -617,7 +617,7 @@ async fn extension_logical_plan() -> Result<()> { }); let proto = to_substrait_plan(&ext_plan, &ctx)?; - let plan2 = from_substrait_plan(&mut ctx, &proto).await?; + let plan2 = from_substrait_plan(&ctx, &proto).await?; let plan1str = format!("{ext_plan:?}"); let plan2str = format!("{plan2:?}"); @@ -712,11 +712,11 @@ async fn verify_post_join_filter_value(proto: Box) -> Result<()> { } async fn assert_expected_plan(sql: &str, expected_plan_str: &str) -> Result<()> { - let mut ctx = create_context().await?; + let ctx = create_context().await?; let df = ctx.sql(sql).await?; let plan = df.into_optimized_plan()?; let proto = to_substrait_plan(&plan, &ctx)?; - let plan2 = from_substrait_plan(&mut ctx, &proto).await?; + let plan2 = from_substrait_plan(&ctx, &proto).await?; let plan2 = ctx.state().optimize(&plan2)?; let plan2str = format!("{plan2:?}"); assert_eq!(expected_plan_str, &plan2str); @@ -724,11 +724,11 @@ async fn assert_expected_plan(sql: &str, expected_plan_str: &str) -> Result<()> } async fn roundtrip_fill_na(sql: &str) -> Result<()> { - let mut ctx = create_context().await?; + let ctx = create_context().await?; let df = ctx.sql(sql).await?; let plan1 = df.into_optimized_plan()?; let proto = to_substrait_plan(&plan1, &ctx)?; - let plan2 = from_substrait_plan(&mut ctx, &proto).await?; + let plan2 = from_substrait_plan(&ctx, &proto).await?; let plan2 = ctx.state().optimize(&plan2)?; // Format plan string and replace all None's with 0 @@ -743,15 +743,15 @@ async fn test_alias(sql_with_alias: &str, sql_no_alias: &str) -> Result<()> { // Since we ignore the SubqueryAlias in the producer, the result should be // the same as producing a Substrait plan from the same query without aliases // sql_with_alias -> substrait -> logical plan = sql_no_alias -> substrait -> logical plan - let mut ctx = create_context().await?; + let ctx = create_context().await?; let df_a = ctx.sql(sql_with_alias).await?; let proto_a = to_substrait_plan(&df_a.into_optimized_plan()?, &ctx)?; - let plan_with_alias = from_substrait_plan(&mut ctx, &proto_a).await?; + let plan_with_alias = from_substrait_plan(&ctx, &proto_a).await?; let df = ctx.sql(sql_no_alias).await?; let proto = to_substrait_plan(&df.into_optimized_plan()?, &ctx)?; - let plan = from_substrait_plan(&mut ctx, &proto).await?; + let plan = from_substrait_plan(&ctx, &proto).await?; println!("{plan_with_alias:#?}"); println!("{plan:#?}"); @@ -763,11 +763,11 @@ async fn test_alias(sql_with_alias: &str, sql_no_alias: &str) -> Result<()> { } async fn roundtrip(sql: &str) -> Result<()> { - let mut ctx = create_context().await?; + let ctx = create_context().await?; let df = ctx.sql(sql).await?; let plan = df.into_optimized_plan()?; let proto = to_substrait_plan(&plan, &ctx)?; - let plan2 = from_substrait_plan(&mut ctx, &proto).await?; + let plan2 = from_substrait_plan(&ctx, &proto).await?; let plan2 = ctx.state().optimize(&plan2)?; println!("{plan:#?}"); @@ -780,11 +780,11 @@ async fn roundtrip(sql: &str) -> Result<()> { } async fn roundtrip_verify_post_join_filter(sql: &str) -> Result<()> { - let mut ctx = create_context().await?; + let ctx = create_context().await?; let df = ctx.sql(sql).await?; let plan = df.into_optimized_plan()?; let proto = to_substrait_plan(&plan, &ctx)?; - let plan2 = from_substrait_plan(&mut ctx, &proto).await?; + let plan2 = from_substrait_plan(&ctx, &proto).await?; let plan2 = ctx.state().optimize(&plan2)?; println!("{plan:#?}"); @@ -799,11 +799,11 @@ async fn roundtrip_verify_post_join_filter(sql: &str) -> Result<()> { } async fn roundtrip_all_types(sql: &str) -> Result<()> { - let mut ctx = create_all_type_context().await?; + let ctx = create_all_type_context().await?; let df = ctx.sql(sql).await?; let plan = df.into_optimized_plan()?; let proto = to_substrait_plan(&plan, &ctx)?; - let plan2 = from_substrait_plan(&mut ctx, &proto).await?; + let plan2 = from_substrait_plan(&ctx, &proto).await?; let plan2 = ctx.state().optimize(&plan2)?; println!("{plan:#?}"); diff --git a/datafusion/substrait/tests/cases/roundtrip_physical_plan.rs b/datafusion/substrait/tests/cases/roundtrip_physical_plan.rs index 3e5e757e4c39..b64dd2c138fc 100644 --- a/datafusion/substrait/tests/cases/roundtrip_physical_plan.rs +++ b/datafusion/substrait/tests/cases/roundtrip_physical_plan.rs @@ -62,10 +62,10 @@ async fn parquet_exec() -> Result<()> { let substrait_rel = producer::to_substrait_rel(parquet_exec.as_ref(), &mut extension_info)?; - let mut ctx = SessionContext::new(); + let ctx = SessionContext::new(); let parquet_exec_roundtrip = - consumer::from_substrait_rel(&mut ctx, substrait_rel.as_ref(), &HashMap::new()) + consumer::from_substrait_rel(&ctx, substrait_rel.as_ref(), &HashMap::new()) .await?; let expected = format!("{}", displayable(parquet_exec.as_ref()).indent(true)); diff --git a/datafusion/substrait/tests/cases/serialize.rs b/datafusion/substrait/tests/cases/serialize.rs index d6dc5d7e58f2..f6736ca22279 100644 --- a/datafusion/substrait/tests/cases/serialize.rs +++ b/datafusion/substrait/tests/cases/serialize.rs @@ -30,7 +30,7 @@ mod tests { #[tokio::test] async fn serialize_simple_select() -> Result<()> { - let mut ctx = create_context().await?; + let ctx = create_context().await?; let path = "tests/simple_select.bin"; let sql = "SELECT a, b FROM data"; // Test reference @@ -42,7 +42,7 @@ mod tests { // Read substrait plan from file let proto = serializer::deserialize(path).await?; // Check plan equality - let plan = from_substrait_plan(&mut ctx, &proto).await?; + let plan = from_substrait_plan(&ctx, &proto).await?; let plan_str_ref = format!("{plan_ref:?}"); let plan_str = format!("{plan:?}"); assert_eq!(plan_str_ref, plan_str); From f388a2bbce4b50fa41a81613929880af598ccd04 Mon Sep 17 00:00:00 2001 From: Jeffrey <22608443+Jefffrey@users.noreply.github.com> Date: Sun, 29 Oct 2023 23:49:36 +1100 Subject: [PATCH 08/43] Fix crate READMEs (#7964) --- datafusion-examples/Cargo.toml | 2 +- datafusion/common/Cargo.toml | 2 +- datafusion/expr/Cargo.toml | 2 +- datafusion/optimizer/Cargo.toml | 2 +- datafusion/physical-expr/Cargo.toml | 2 +- datafusion/proto/Cargo.toml | 2 +- datafusion/sql/Cargo.toml | 2 +- datafusion/sqllogictest/Cargo.toml | 2 +- datafusion/substrait/Cargo.toml | 2 +- datafusion/wasmtest/Cargo.toml | 2 +- 10 files changed, 10 insertions(+), 10 deletions(-) diff --git a/datafusion-examples/Cargo.toml b/datafusion-examples/Cargo.toml index 8d504f834bc5..7350c4ab981f 100644 --- a/datafusion-examples/Cargo.toml +++ b/datafusion-examples/Cargo.toml @@ -20,9 +20,9 @@ name = "datafusion-examples" description = "DataFusion usage examples" keywords = ["arrow", "query", "sql"] publish = false +readme = "README.md" version = { workspace = true } edition = { workspace = true } -readme = { workspace = true } homepage = { workspace = true } repository = { workspace = true } license = { workspace = true } diff --git a/datafusion/common/Cargo.toml b/datafusion/common/Cargo.toml index 490fbeacad85..87087c50a2d2 100644 --- a/datafusion/common/Cargo.toml +++ b/datafusion/common/Cargo.toml @@ -19,9 +19,9 @@ name = "datafusion-common" description = "Common functionality for DataFusion query engine" keywords = ["arrow", "query", "sql"] +readme = "README.md" version = { workspace = true } edition = { workspace = true } -readme = { workspace = true } homepage = { workspace = true } repository = { workspace = true } license = { workspace = true } diff --git a/datafusion/expr/Cargo.toml b/datafusion/expr/Cargo.toml index c5cf6a1ac11f..a7919a557ad7 100644 --- a/datafusion/expr/Cargo.toml +++ b/datafusion/expr/Cargo.toml @@ -19,9 +19,9 @@ name = "datafusion-expr" description = "Logical plan and expression representation for DataFusion query engine" keywords = ["datafusion", "logical", "plan", "expressions"] +readme = "README.md" version = { workspace = true } edition = { workspace = true } -readme = { workspace = true } homepage = { workspace = true } repository = { workspace = true } license = { workspace = true } diff --git a/datafusion/optimizer/Cargo.toml b/datafusion/optimizer/Cargo.toml index bf786686f474..dce05058b826 100644 --- a/datafusion/optimizer/Cargo.toml +++ b/datafusion/optimizer/Cargo.toml @@ -19,9 +19,9 @@ name = "datafusion-optimizer" description = "DataFusion Query Optimizer" keywords = [ "datafusion", "query", "optimizer" ] +readme = "README.md" version = { workspace = true } edition = { workspace = true } -readme = { workspace = true } homepage = { workspace = true } repository = { workspace = true } license = { workspace = true } diff --git a/datafusion/physical-expr/Cargo.toml b/datafusion/physical-expr/Cargo.toml index 6269f27310a6..f7c0221756fd 100644 --- a/datafusion/physical-expr/Cargo.toml +++ b/datafusion/physical-expr/Cargo.toml @@ -19,9 +19,9 @@ name = "datafusion-physical-expr" description = "Physical expression implementation for DataFusion query engine" keywords = ["arrow", "query", "sql"] +readme = "README.md" version = { workspace = true } edition = { workspace = true } -readme = { workspace = true } homepage = { workspace = true } repository = { workspace = true } license = { workspace = true } diff --git a/datafusion/proto/Cargo.toml b/datafusion/proto/Cargo.toml index 32e10e58a7d7..5e52dadc0b18 100644 --- a/datafusion/proto/Cargo.toml +++ b/datafusion/proto/Cargo.toml @@ -19,9 +19,9 @@ name = "datafusion-proto" description = "Protobuf serialization of DataFusion logical plan expressions" keywords = ["arrow", "query", "sql"] +readme = "README.md" version = { workspace = true } edition = { workspace = true } -readme = { workspace = true } homepage = { workspace = true } repository = { workspace = true } license = { workspace = true } diff --git a/datafusion/sql/Cargo.toml b/datafusion/sql/Cargo.toml index a00a7f021352..fe60feb6ab1f 100644 --- a/datafusion/sql/Cargo.toml +++ b/datafusion/sql/Cargo.toml @@ -19,9 +19,9 @@ name = "datafusion-sql" description = "DataFusion SQL Query Planner" keywords = ["datafusion", "sql", "parser", "planner"] +readme = "README.md" version = { workspace = true } edition = { workspace = true } -readme = { workspace = true } homepage = { workspace = true } repository = { workspace = true } license = { workspace = true } diff --git a/datafusion/sqllogictest/Cargo.toml b/datafusion/sqllogictest/Cargo.toml index 454f99942f52..f1a730351417 100644 --- a/datafusion/sqllogictest/Cargo.toml +++ b/datafusion/sqllogictest/Cargo.toml @@ -21,7 +21,7 @@ edition = { workspace = true } homepage = { workspace = true } license = { workspace = true } name = "datafusion-sqllogictest" -readme = { workspace = true } +readme = "README.md" repository = { workspace = true } rust-version = { workspace = true } version = { workspace = true } diff --git a/datafusion/substrait/Cargo.toml b/datafusion/substrait/Cargo.toml index 7c4ff868cfcd..67d31caca260 100644 --- a/datafusion/substrait/Cargo.toml +++ b/datafusion/substrait/Cargo.toml @@ -18,9 +18,9 @@ [package] name = "datafusion-substrait" description = "DataFusion Substrait Producer and Consumer" +readme = "README.md" version = { workspace = true } edition = { workspace = true } -readme = { workspace = true } homepage = { workspace = true } repository = { workspace = true } license = { workspace = true } diff --git a/datafusion/wasmtest/Cargo.toml b/datafusion/wasmtest/Cargo.toml index e1a9a5d41a5a..3195c989e223 100644 --- a/datafusion/wasmtest/Cargo.toml +++ b/datafusion/wasmtest/Cargo.toml @@ -18,9 +18,9 @@ [package] name = "datafusion-wasmtest" description = "Test library to compile datafusion crates to wasm" +readme = "README.md" version = { workspace = true } edition = { workspace = true } -readme = { workspace = true } homepage = { workspace = true } repository = { workspace = true } license = { workspace = true } From 4a91ce91e2e82ed33142405a00f619cc714a5a30 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Sun, 29 Oct 2023 08:50:10 -0400 Subject: [PATCH 09/43] Minor: Improve `HashJoinExec` documentation (#7953) * Minor: Improve `HashJoinExec` documentation * Apply suggestions from code review Co-authored-by: Liang-Chi Hsieh --------- Co-authored-by: Liang-Chi Hsieh --- .../physical-plan/src/joins/hash_join.rs | 138 ++++++++++++++++-- 1 file changed, 126 insertions(+), 12 deletions(-) diff --git a/datafusion/physical-plan/src/joins/hash_join.rs b/datafusion/physical-plan/src/joins/hash_join.rs index 9aa776fe054c..dc0e81a6f36e 100644 --- a/datafusion/physical-plan/src/joins/hash_join.rs +++ b/datafusion/physical-plan/src/joins/hash_join.rs @@ -15,8 +15,7 @@ // specific language governing permissions and limitations // under the License. -//! Defines the join plan for executing partitions in parallel and then joining the results -//! into a set of partitions. +//! [`HashJoinExec`] Partitioned Hash Join Operator use std::fmt; use std::mem::size_of; @@ -78,29 +77,140 @@ use futures::{ready, Stream, StreamExt, TryStreamExt}; type JoinLeftData = (JoinHashMap, RecordBatch, MemoryReservation); -/// Join execution plan executes partitions in parallel and combines them into a set of -/// partitions. +/// Join execution plan: Evaluates eqijoin predicates in parallel on multiple +/// partitions using a hash table and an optional filter list to apply post +/// join. /// -/// Filter expression expected to contain non-equality predicates that can not be pushed -/// down to any of join inputs. -/// In case of outer join, filter applied to only matched rows. +/// # Join Expressions +/// +/// This implementation is optimized for evaluating eqijoin predicates ( +/// ` = `) expressions, which are represented as a list of `Columns` +/// in [`Self::on`]. +/// +/// Non-equality predicates, which can not pushed down to a join inputs (e.g. +/// ` != `) are known as "filter expressions" and are evaluated +/// after the equijoin predicates. +/// +/// # "Build Side" vs "Probe Side" +/// +/// HashJoin takes two inputs, which are referred to as the "build" and the +/// "probe". The build side is the first child, and the probe side is the second +/// child. +/// +/// The two inputs are treated differently and it is VERY important that the +/// *smaller* input is placed on the build side to minimize the work of creating +/// the hash table. +/// +/// ```text +/// ā”Œā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā” +/// ā”‚ HashJoin ā”‚ +/// ā”‚ ā”‚ +/// ā””ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”˜ +/// ā”‚ ā”‚ +/// ā”Œā”€ā”€ā”€ā”€ā”€ā”˜ ā””ā”€ā”€ā”€ā”€ā”€ā” +/// ā–¼ ā–¼ +/// ā”Œā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā” ā”Œā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā” +/// ā”‚ Input ā”‚ ā”‚ Input ā”‚ +/// ā”‚ [0] ā”‚ ā”‚ [1] ā”‚ +/// ā””ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”˜ ā””ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”˜ +/// +/// "build side" "probe side" +/// ``` +/// +/// Execution proceeds in 2 stages: +/// +/// 1. the **build phase** where a hash table is created from the tuples of the +/// build side. +/// +/// 2. the **probe phase** where the tuples of the probe side are streamed +/// through, checking for matches of the join keys in the hash table. +/// +/// ```text +/// ā”Œā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā” ā”Œā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā” +/// ā”‚ ā”Œā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā” ā”‚ ā”‚ ā”Œā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā” ā”‚ +/// ā”‚ ā”‚ Hash ā”‚ ā”‚ ā”‚ ā”‚ Hash ā”‚ ā”‚ +/// ā”‚ ā”‚ Table ā”‚ ā”‚ ā”‚ ā”‚ Table ā”‚ ā”‚ +/// ā”‚ ā”‚(keys areā”‚ ā”‚ ā”‚ ā”‚(keys areā”‚ ā”‚ +/// ā”‚ ā”‚equi joinā”‚ ā”‚ ā”‚ ā”‚equi joinā”‚ ā”‚ Stage 2: batches from +/// Stage 1: the ā”‚ ā”‚columns) ā”‚ ā”‚ ā”‚ ā”‚columns) ā”‚ ā”‚ the probe side are +/// *entire* build ā”‚ ā”‚ ā”‚ ā”‚ ā”‚ ā”‚ ā”‚ ā”‚ streamed through, and +/// side is read ā”‚ ā””ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”˜ ā”‚ ā”‚ ā””ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”˜ ā”‚ checked against the +/// into the hash ā”‚ ā–² ā”‚ ā”‚ ā–² ā”‚ contents of the hash +/// table ā”‚ HashJoin ā”‚ ā”‚ HashJoin ā”‚ table +/// ā””ā”€ā”€ā”€ā”€ā”€ā”€ā”¼ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”˜ ā””ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”¼ā”€ā”€ā”€ā”€ā”€ā”˜ +/// ā”€ ā”€ ā”€ ā”€ ā”€ ā”€ ā”€ ā”€ ā”€ ā”€ ā”€ ā”€ ā”€ +/// ā”‚ ā”‚ +/// +/// ā”‚ ā”‚ +/// ā”Œā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā” ā”Œā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā” +/// ā”‚RecordBatch ā”‚ ā”‚RecordBatch ā”‚ +/// ā””ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”˜ ā””ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”˜ +/// ā”Œā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā” ā”Œā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā” +/// ā”‚RecordBatch ā”‚ ā”‚RecordBatch ā”‚ +/// ā””ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”˜ ā””ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”˜ +/// ... ... +/// ā”Œā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā” ā”Œā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā” +/// ā”‚RecordBatch ā”‚ ā”‚RecordBatch ā”‚ +/// ā””ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”˜ ā””ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”˜ +/// +/// build side probe side +/// +/// ``` +/// +/// # Example "Optimal" Plans +/// +/// The differences in the inputs means that for classic "Star Schema Query", +/// the optimal plan will be a **"Right Deep Tree"** . A Star Schema Query is +/// one where there is one large table and several smaller "dimension" tables, +/// joined on `Foreign Key = Primary Key` predicates. +/// +/// A "Right Deep Tree" looks like this large table as the probe side on the +/// lowest join: +/// +/// ```text +/// ā”Œā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā” +/// ā”‚ HashJoin ā”‚ +/// ā”‚ ā”‚ +/// ā””ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”˜ +/// ā”‚ ā”‚ +/// ā”Œā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”˜ ā””ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā” +/// ā–¼ ā–¼ +/// ā”Œā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā” ā”Œā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā” +/// ā”‚ small table 1 ā”‚ ā”‚ HashJoin ā”‚ +/// ā”‚ "dimension" ā”‚ ā”‚ ā”‚ +/// ā””ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”˜ ā””ā”€ā”€ā”€ā”¬ā”€ā”€ā”€ā”¬ā”€ā”€ā”€ā”˜ +/// ā”Œā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”˜ ā””ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā” +/// ā”‚ ā”‚ +/// ā–¼ ā–¼ +/// ā”Œā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā” ā”Œā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā” +/// ā”‚ small table 2 ā”‚ ā”‚ HashJoin ā”‚ +/// ā”‚ "dimension" ā”‚ ā”‚ ā”‚ +/// ā””ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”˜ ā””ā”€ā”€ā”€ā”¬ā”€ā”€ā”€ā”¬ā”€ā”€ā”€ā”˜ +/// ā”Œā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”˜ ā””ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā” +/// ā”‚ ā”‚ +/// ā–¼ ā–¼ +/// ā”Œā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā” ā”Œā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā” +/// ā”‚ small table 3 ā”‚ ā”‚ large table ā”‚ +/// ā”‚ "dimension" ā”‚ ā”‚ "fact" ā”‚ +/// ā””ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”˜ ā””ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”€ā”˜ +/// ``` #[derive(Debug)] pub struct HashJoinExec { /// left (build) side which gets hashed pub left: Arc, /// right (probe) side which are filtered by the hash table pub right: Arc, - /// Set of common columns used to join on + /// Set of equijoin columns from the relations: `(left_col, right_col)` pub on: Vec<(Column, Column)>, /// Filters which are applied while finding matching rows pub filter: Option, - /// How the join is performed + /// How the join is performed (`OUTER`, `INNER`, etc) pub join_type: JoinType, - /// The schema once the join is applied + /// The output schema for the join schema: SchemaRef, /// Build-side data left_fut: OnceAsync, - /// Shares the `RandomState` for the hashing algorithm + /// Shared the `RandomState` for the hashing algorithm random_state: RandomState, /// Output order output_order: Option>, @@ -110,12 +220,16 @@ pub struct HashJoinExec { metrics: ExecutionPlanMetricsSet, /// Information of index and left / right placement of columns column_indices: Vec, - /// If null_equals_null is true, null == null else null != null + /// Null matching behavior: If `null_equals_null` is true, rows that have + /// `null`s in both left and right equijoin columns will be matched. + /// Otherwise, rows that have `null`s in the join columns will not be + /// matched and thus will not appear in the output. pub null_equals_null: bool, } impl HashJoinExec { /// Tries to create a new [HashJoinExec]. + /// /// # Error /// This function errors when it is not possible to join the left and right sides on keys `on`. pub fn try_new( From 9b45967edc6dba312ea223464dad3e66604d2095 Mon Sep 17 00:00:00 2001 From: Alex Huang Date: Sun, 29 Oct 2023 22:45:34 +0100 Subject: [PATCH 10/43] chore: clean useless clone baesd on clippy (#7973) --- .../core/src/physical_optimizer/combine_partial_final_agg.rs | 2 +- datafusion/core/src/physical_optimizer/enforce_distribution.rs | 2 +- datafusion/core/src/physical_optimizer/topk_aggregation.rs | 2 +- datafusion/physical-expr/src/array_expressions.rs | 2 +- datafusion/substrait/src/logical_plan/producer.rs | 2 +- 5 files changed, 5 insertions(+), 5 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/combine_partial_final_agg.rs b/datafusion/core/src/physical_optimizer/combine_partial_final_agg.rs index 838ae613683e..2c4e929788df 100644 --- a/datafusion/core/src/physical_optimizer/combine_partial_final_agg.rs +++ b/datafusion/core/src/physical_optimizer/combine_partial_final_agg.rs @@ -93,7 +93,7 @@ impl PhysicalOptimizerRule for CombinePartialFinalAggregate { input_agg_exec.filter_expr().to_vec(), input_agg_exec.order_by_expr().to_vec(), input_agg_exec.input().clone(), - input_agg_exec.input_schema().clone(), + input_agg_exec.input_schema(), ) .ok() .map(Arc::new) diff --git a/datafusion/core/src/physical_optimizer/enforce_distribution.rs b/datafusion/core/src/physical_optimizer/enforce_distribution.rs index 7b91dce32aa9..12df9efbbca6 100644 --- a/datafusion/core/src/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/src/physical_optimizer/enforce_distribution.rs @@ -554,7 +554,7 @@ fn reorder_aggregate_keys( agg_exec.filter_expr().to_vec(), agg_exec.order_by_expr().to_vec(), partial_agg, - agg_exec.input_schema().clone(), + agg_exec.input_schema(), )?); // Need to create a new projection to change the expr ordering back diff --git a/datafusion/core/src/physical_optimizer/topk_aggregation.rs b/datafusion/core/src/physical_optimizer/topk_aggregation.rs index 572e796a8ba7..e0a8da82e35f 100644 --- a/datafusion/core/src/physical_optimizer/topk_aggregation.rs +++ b/datafusion/core/src/physical_optimizer/topk_aggregation.rs @@ -75,7 +75,7 @@ impl TopKAggregation { aggr.filter_expr().to_vec(), aggr.order_by_expr().to_vec(), aggr.input().clone(), - aggr.input_schema().clone(), + aggr.input_schema(), ) .expect("Unable to copy Aggregate!") .with_limit(Some(limit)); diff --git a/datafusion/physical-expr/src/array_expressions.rs b/datafusion/physical-expr/src/array_expressions.rs index 7077f8b59860..84fd301b84de 100644 --- a/datafusion/physical-expr/src/array_expressions.rs +++ b/datafusion/physical-expr/src/array_expressions.rs @@ -811,7 +811,7 @@ fn concat_internal(args: &[ArrayRef]) -> Result { } } // Assume all arrays have the same data type - let data_type = list_arrays[0].value_type().clone(); + let data_type = list_arrays[0].value_type(); let buffer = valid.finish(); let elements = arrays diff --git a/datafusion/substrait/src/logical_plan/producer.rs b/datafusion/substrait/src/logical_plan/producer.rs index 757bddf9fe58..e3c6f94d43d5 100644 --- a/datafusion/substrait/src/logical_plan/producer.rs +++ b/datafusion/substrait/src/logical_plan/producer.rs @@ -326,7 +326,7 @@ pub fn to_substrait_rel( left: Some(left), right: Some(right), r#type: join_type as i32, - expression: join_expr.clone(), + expression: join_expr, post_join_filter: None, advanced_extension: None, }))), From 806a9631d70a1e4be5021e017304ee2ce5d4b7fc Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Mon, 30 Oct 2023 09:17:02 -0400 Subject: [PATCH 11/43] Add README.md to `core`, `execution` and `physical-plan` crates (#7970) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit * Add README.md to `core`, `execution` and `physical-plan` crates * prettier * Update datafusion/physical-plan/README.md * Update datafusion/wasmtest/README.md --------- Co-authored-by: DaniĆ«l Heres --- datafusion/core/README.md | 26 ++++++++++++++++++++++++++ datafusion/execution/README.md | 26 ++++++++++++++++++++++++++ datafusion/physical-plan/README.md | 27 +++++++++++++++++++++++++++ datafusion/wasmtest/README.md | 11 +++++++++-- 4 files changed, 88 insertions(+), 2 deletions(-) create mode 100644 datafusion/core/README.md create mode 100644 datafusion/execution/README.md create mode 100644 datafusion/physical-plan/README.md diff --git a/datafusion/core/README.md b/datafusion/core/README.md new file mode 100644 index 000000000000..c83e33991a07 --- /dev/null +++ b/datafusion/core/README.md @@ -0,0 +1,26 @@ + + +# DataFusion Common + +[DataFusion](df) is an extensible query execution framework, written in Rust, that uses Apache Arrow as its in-memory format. + +This crate contains the main entrypoints and high level DataFusion APIs such as SessionContext, and DataFrame and ListingTable. + +[df]: https://crates.io/crates/datafusion diff --git a/datafusion/execution/README.md b/datafusion/execution/README.md new file mode 100644 index 000000000000..40b3ffaa3ed3 --- /dev/null +++ b/datafusion/execution/README.md @@ -0,0 +1,26 @@ + + +# DataFusion Common + +[DataFusion](df) is an extensible query execution framework, written in Rust, that uses Apache Arrow as its in-memory format. + +This crate is a submodule of DataFusion that provides execution runtime such as the memory pools and disk manager. + +[df]: https://crates.io/crates/datafusion diff --git a/datafusion/physical-plan/README.md b/datafusion/physical-plan/README.md new file mode 100644 index 000000000000..3632ea7d2a26 --- /dev/null +++ b/datafusion/physical-plan/README.md @@ -0,0 +1,27 @@ + + +# DataFusion Common + +[DataFusion](df) is an extensible query execution framework, written in Rust, that uses Apache Arrow as its in-memory format. + +This crate is a submodule of DataFusion that contains the `ExecutionPlan` trait and the various implementations of that +trait for built in operators such as filters, projections, joins, aggregations, etc. + +[df]: https://crates.io/crates/datafusion diff --git a/datafusion/wasmtest/README.md b/datafusion/wasmtest/README.md index 5dc7bb2de45d..21e505f38e02 100644 --- a/datafusion/wasmtest/README.md +++ b/datafusion/wasmtest/README.md @@ -17,9 +17,16 @@ under the License. --> -## wasmtest +# DataFusion wasmtest + +[DataFusion](df) is an extensible query execution framework, written in Rust, that uses Apache Arrow as its in-memory format. + +This crate is a submodule of DataFusion used to verify that various DataFusion crates compile successfully to the +`wasm32-unknown-unknown` target with wasm-pack. -Library crate to verify that various DataFusion crates compile successfully to the `wasm32-unknown-unknown` target with wasm-pack. +[df]: https://crates.io/crates/datafusion + +## wasmtest Some of DataFusion's downstream projects compile to WASM to run in the browser. Doing so requires special care that certain library dependencies are not included in DataFusion. From fdb54549301ccbede0b48788ff0fa057dd799d63 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Mon, 30 Oct 2023 13:17:04 -0400 Subject: [PATCH 12/43] Move source repartitioning into `ExecutionPlan::repartition` (#7936) * Move source repartitioning into ExecutionPlan::repartition * cleanup * update test * update test * refine docs * fix merge --- .../core/src/datasource/physical_plan/csv.rs | 58 +++++------ .../core/src/datasource/physical_plan/mod.rs | 96 +++++++++---------- .../src/datasource/physical_plan/parquet.rs | 41 ++++---- .../enforce_distribution.rs | 28 ++---- datafusion/physical-plan/src/lib.rs | 30 +++++- 5 files changed, 130 insertions(+), 123 deletions(-) diff --git a/datafusion/core/src/datasource/physical_plan/csv.rs b/datafusion/core/src/datasource/physical_plan/csv.rs index 8117e101ea99..82163da64af8 100644 --- a/datafusion/core/src/datasource/physical_plan/csv.rs +++ b/datafusion/core/src/datasource/physical_plan/csv.rs @@ -46,6 +46,7 @@ use datafusion_physical_expr::{ }; use bytes::{Buf, Bytes}; +use datafusion_common::config::ConfigOptions; use futures::{ready, StreamExt, TryStreamExt}; use object_store::{GetOptions, GetResultPayload, ObjectStore}; use tokio::io::AsyncWriteExt; @@ -117,34 +118,6 @@ impl CsvExec { pub fn escape(&self) -> Option { self.escape } - - /// Redistribute files across partitions according to their size - /// See comments on `repartition_file_groups()` for more detail. - /// - /// Return `None` if can't get repartitioned(empty/compressed file). - pub fn get_repartitioned( - &self, - target_partitions: usize, - repartition_file_min_size: usize, - ) -> Option { - // Parallel execution on compressed CSV file is not supported yet. - if self.file_compression_type.is_compressed() { - return None; - } - - let repartitioned_file_groups_option = FileScanConfig::repartition_file_groups( - self.base_config.file_groups.clone(), - target_partitions, - repartition_file_min_size, - ); - - if let Some(repartitioned_file_groups) = repartitioned_file_groups_option { - let mut new_plan = self.clone(); - new_plan.base_config.file_groups = repartitioned_file_groups; - return Some(new_plan); - } - None - } } impl DisplayAs for CsvExec { @@ -205,6 +178,35 @@ impl ExecutionPlan for CsvExec { Ok(self) } + /// Redistribute files across partitions according to their size + /// See comments on `repartition_file_groups()` for more detail. + /// + /// Return `None` if can't get repartitioned(empty/compressed file). + fn repartitioned( + &self, + target_partitions: usize, + config: &ConfigOptions, + ) -> Result>> { + let repartition_file_min_size = config.optimizer.repartition_file_min_size; + // Parallel execution on compressed CSV file is not supported yet. + if self.file_compression_type.is_compressed() { + return Ok(None); + } + + let repartitioned_file_groups_option = FileScanConfig::repartition_file_groups( + self.base_config.file_groups.clone(), + target_partitions, + repartition_file_min_size, + ); + + if let Some(repartitioned_file_groups) = repartitioned_file_groups_option { + let mut new_plan = self.clone(); + new_plan.base_config.file_groups = repartitioned_file_groups; + return Ok(Some(Arc::new(new_plan))); + } + Ok(None) + } + fn execute( &self, partition: usize, diff --git a/datafusion/core/src/datasource/physical_plan/mod.rs b/datafusion/core/src/datasource/physical_plan/mod.rs index 6643e4127dbd..ea0a9698ff5c 100644 --- a/datafusion/core/src/datasource/physical_plan/mod.rs +++ b/datafusion/core/src/datasource/physical_plan/mod.rs @@ -527,6 +527,7 @@ mod tests { }; use arrow_schema::Field; use chrono::Utc; + use datafusion_common::config::ConfigOptions; use crate::physical_plan::{DefaultDisplay, VerboseDisplay}; @@ -828,11 +829,7 @@ mod tests { None, ); - let partitioned_file = parquet_exec - .get_repartitioned(4, 0) - .base_config() - .file_groups - .clone(); + let partitioned_file = repartition_with_size(&parquet_exec, 4, 0); assert!(partitioned_file[0][0].range.is_none()); } @@ -893,13 +890,8 @@ mod tests { None, ); - let actual = file_groups_to_vec( - parquet_exec - .get_repartitioned(n_partition, 10) - .base_config() - .file_groups - .clone(), - ); + let actual = + repartition_with_size_to_vec(&parquet_exec, n_partition, 10); assert_eq!(expected, &actual); } @@ -927,13 +919,7 @@ mod tests { None, ); - let actual = file_groups_to_vec( - parquet_exec - .get_repartitioned(4, 10) - .base_config() - .file_groups - .clone(), - ); + let actual = repartition_with_size_to_vec(&parquet_exec, 4, 10); let expected = vec![ (0, "a".to_string(), 0, 31), (1, "a".to_string(), 31, 62), @@ -964,13 +950,7 @@ mod tests { None, ); - let actual = file_groups_to_vec( - parquet_exec - .get_repartitioned(96, 5) - .base_config() - .file_groups - .clone(), - ); + let actual = repartition_with_size_to_vec(&parquet_exec, 96, 5); let expected = vec![ (0, "a".to_string(), 0, 1), (1, "a".to_string(), 1, 2), @@ -1007,13 +987,7 @@ mod tests { None, ); - let actual = file_groups_to_vec( - parquet_exec - .get_repartitioned(3, 10) - .base_config() - .file_groups - .clone(), - ); + let actual = repartition_with_size_to_vec(&parquet_exec, 3, 10); let expected = vec![ (0, "a".to_string(), 0, 34), (1, "a".to_string(), 34, 40), @@ -1046,13 +1020,7 @@ mod tests { None, ); - let actual = file_groups_to_vec( - parquet_exec - .get_repartitioned(2, 10) - .base_config() - .file_groups - .clone(), - ); + let actual = repartition_with_size_to_vec(&parquet_exec, 2, 10); let expected = vec![ (0, "a".to_string(), 0, 40), (0, "b".to_string(), 0, 10), @@ -1086,11 +1054,7 @@ mod tests { None, ); - let actual = parquet_exec - .get_repartitioned(65, 10) - .base_config() - .file_groups - .clone(); + let actual = repartition_with_size(&parquet_exec, 65, 10); assert_eq!(2, actual.len()); } @@ -1115,17 +1079,47 @@ mod tests { None, ); - let actual = parquet_exec - .get_repartitioned(65, 500) + let actual = repartition_with_size(&parquet_exec, 65, 500); + assert_eq!(1, actual.len()); + } + + /// Calls `ParquetExec.repartitioned` with the specified + /// `target_partitions` and `repartition_file_min_size`, returning the + /// resulting `PartitionedFile`s + fn repartition_with_size( + parquet_exec: &ParquetExec, + target_partitions: usize, + repartition_file_min_size: usize, + ) -> Vec> { + let mut config = ConfigOptions::new(); + config.optimizer.repartition_file_min_size = repartition_file_min_size; + + parquet_exec + .repartitioned(target_partitions, &config) + .unwrap() // unwrap Result + .unwrap() // unwrap Option + .as_any() + .downcast_ref::() + .unwrap() .base_config() .file_groups - .clone(); - assert_eq!(1, actual.len()); + .clone() } - fn file_groups_to_vec( - file_groups: Vec>, + /// Calls `repartition_with_size` and returns a tuple for each output `PartitionedFile`: + /// + /// `(partition index, file path, start, end)` + fn repartition_with_size_to_vec( + parquet_exec: &ParquetExec, + target_partitions: usize, + repartition_file_min_size: usize, ) -> Vec<(usize, String, i64, i64)> { + let file_groups = repartition_with_size( + parquet_exec, + target_partitions, + repartition_file_min_size, + ); + file_groups .iter() .enumerate() diff --git a/datafusion/core/src/datasource/physical_plan/parquet.rs b/datafusion/core/src/datasource/physical_plan/parquet.rs index 3a2459bec817..f6e999f60249 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet.rs @@ -259,26 +259,6 @@ impl ParquetExec { self.enable_bloom_filter .unwrap_or(config_options.execution.parquet.bloom_filter_enabled) } - - /// Redistribute files across partitions according to their size - /// See comments on `get_file_groups_repartitioned()` for more detail. - pub fn get_repartitioned( - &self, - target_partitions: usize, - repartition_file_min_size: usize, - ) -> Self { - let repartitioned_file_groups_option = FileScanConfig::repartition_file_groups( - self.base_config.file_groups.clone(), - target_partitions, - repartition_file_min_size, - ); - - let mut new_plan = self.clone(); - if let Some(repartitioned_file_groups) = repartitioned_file_groups_option { - new_plan.base_config.file_groups = repartitioned_file_groups; - } - new_plan - } } impl DisplayAs for ParquetExec { @@ -349,6 +329,27 @@ impl ExecutionPlan for ParquetExec { Ok(self) } + /// Redistribute files across partitions according to their size + /// See comments on `get_file_groups_repartitioned()` for more detail. + fn repartitioned( + &self, + target_partitions: usize, + config: &ConfigOptions, + ) -> Result>> { + let repartition_file_min_size = config.optimizer.repartition_file_min_size; + let repartitioned_file_groups_option = FileScanConfig::repartition_file_groups( + self.base_config.file_groups.clone(), + target_partitions, + repartition_file_min_size, + ); + + let mut new_plan = self.clone(); + if let Some(repartitioned_file_groups) = repartitioned_file_groups_option { + new_plan.base_config.file_groups = repartitioned_file_groups; + } + Ok(Some(Arc::new(new_plan))) + } + fn execute( &self, partition_index: usize, diff --git a/datafusion/core/src/physical_optimizer/enforce_distribution.rs b/datafusion/core/src/physical_optimizer/enforce_distribution.rs index 12df9efbbca6..6de39db7d52a 100644 --- a/datafusion/core/src/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/src/physical_optimizer/enforce_distribution.rs @@ -26,9 +26,6 @@ use std::fmt::Formatter; use std::sync::Arc; use crate::config::ConfigOptions; -use crate::datasource::physical_plan::CsvExec; -#[cfg(feature = "parquet")] -use crate::datasource::physical_plan::ParquetExec; use crate::error::Result; use crate::physical_optimizer::utils::{ add_sort_above, get_children_exectrees, get_plan_string, is_coalesce_partitions, @@ -1188,7 +1185,6 @@ fn ensure_distribution( // When `false`, round robin repartition will not be added to increase parallelism let enable_round_robin = config.optimizer.enable_round_robin_repartition; let repartition_file_scans = config.optimizer.repartition_file_scans; - let repartition_file_min_size = config.optimizer.repartition_file_min_size; let batch_size = config.execution.batch_size; let is_unbounded = unbounded_output(&dist_context.plan); // Use order preserving variants either of the conditions true @@ -1265,25 +1261,13 @@ fn ensure_distribution( // Unless partitioning doesn't increase the partition count, it is not beneficial: && child.output_partitioning().partition_count() < target_partitions { - // When `repartition_file_scans` is set, leverage source operators - // (`ParquetExec`, `CsvExec` etc.) to increase parallelism at the source. + // When `repartition_file_scans` is set, attempt to increase + // parallelism at the source. if repartition_file_scans { - #[cfg(feature = "parquet")] - if let Some(parquet_exec) = - child.as_any().downcast_ref::() + if let Some(new_child) = + child.repartitioned(target_partitions, config)? { - child = Arc::new(parquet_exec.get_repartitioned( - target_partitions, - repartition_file_min_size, - )); - } - if let Some(csv_exec) = child.as_any().downcast_ref::() { - if let Some(csv_exec) = csv_exec.get_repartitioned( - target_partitions, - repartition_file_min_size, - ) { - child = Arc::new(csv_exec); - } + child = new_child; } } // Increase parallelism by adding round-robin repartitioning @@ -1644,8 +1628,8 @@ mod tests { use crate::datasource::file_format::file_compression_type::FileCompressionType; use crate::datasource::listing::PartitionedFile; use crate::datasource::object_store::ObjectStoreUrl; - use crate::datasource::physical_plan::FileScanConfig; use crate::datasource::physical_plan::ParquetExec; + use crate::datasource::physical_plan::{CsvExec, FileScanConfig}; use crate::physical_optimizer::enforce_sorting::EnforceSorting; use crate::physical_optimizer::output_requirements::OutputRequirements; use crate::physical_plan::aggregates::{ diff --git a/datafusion/physical-plan/src/lib.rs b/datafusion/physical-plan/src/lib.rs index b2f81579f8e8..3ada2fa163fd 100644 --- a/datafusion/physical-plan/src/lib.rs +++ b/datafusion/physical-plan/src/lib.rs @@ -76,6 +76,7 @@ pub use crate::metrics::Metric; pub use crate::topk::TopK; pub use crate::visitor::{accept, visit_execution_plan, ExecutionPlanVisitor}; +use datafusion_common::config::ConfigOptions; pub use datafusion_common::hash_utils; pub use datafusion_common::utils::project_schema; pub use datafusion_common::{internal_err, ColumnStatistics, Statistics}; @@ -209,7 +210,32 @@ pub trait ExecutionPlan: Debug + DisplayAs + Send + Sync { children: Vec>, ) -> Result>; - /// creates an iterator + /// If supported, attempt to increase the partitioning of this `ExecutionPlan` to + /// produce `target_partitions` partitions. + /// + /// If the `ExecutionPlan` does not support changing its partitioning, + /// returns `Ok(None)` (the default). + /// + /// It is the `ExecutionPlan` can increase its partitioning, but not to the + /// `target_partitions`, it may return an ExecutionPlan with fewer + /// partitions. This might happen, for example, if each new partition would + /// be too small to be efficiently processed individually. + /// + /// The DataFusion optimizer attempts to use as many threads as possible by + /// repartitioning its inputs to match the target number of threads + /// available (`target_partitions`). Some data sources, such as the built in + /// CSV and Parquet readers, implement this method as they are able to read + /// from their input files in parallel, regardless of how the source data is + /// split amongst files. + fn repartitioned( + &self, + _target_partitions: usize, + _config: &ConfigOptions, + ) -> Result>> { + Ok(None) + } + + /// Begin execution of `partition`, returning a stream of [`RecordBatch`]es. fn execute( &self, partition: usize, @@ -217,7 +243,7 @@ pub trait ExecutionPlan: Debug + DisplayAs + Send + Sync { ) -> Result; /// Return a snapshot of the set of [`Metric`]s for this - /// [`ExecutionPlan`]. + /// [`ExecutionPlan`]. If no `Metric`s are available, return None. /// /// While the values of the metrics in the returned /// [`MetricsSet`]s may change as execution progresses, the From 3fd8a2082e5910c2c1cf8e17b9596914633626db Mon Sep 17 00:00:00 2001 From: Jonah Gao Date: Tue, 31 Oct 2023 01:34:05 +0800 Subject: [PATCH 13/43] minor: fix broken links in README.md (#7986) * minor: fix broken links in README.md * fix proto link --- datafusion/common/README.md | 2 +- datafusion/core/README.md | 2 +- datafusion/execution/README.md | 2 +- datafusion/expr/README.md | 2 +- datafusion/optimizer/README.md | 2 +- datafusion/physical-expr/README.md | 2 +- datafusion/physical-plan/README.md | 2 +- datafusion/proto/README.md | 2 +- datafusion/sql/README.md | 2 +- datafusion/wasmtest/README.md | 2 +- 10 files changed, 10 insertions(+), 10 deletions(-) diff --git a/datafusion/common/README.md b/datafusion/common/README.md index 9bccf3f18b7f..524ab4420d2a 100644 --- a/datafusion/common/README.md +++ b/datafusion/common/README.md @@ -19,7 +19,7 @@ # DataFusion Common -[DataFusion](df) is an extensible query execution framework, written in Rust, that uses Apache Arrow as its in-memory format. +[DataFusion][df] is an extensible query execution framework, written in Rust, that uses Apache Arrow as its in-memory format. This crate is a submodule of DataFusion that provides common data types and utilities. diff --git a/datafusion/core/README.md b/datafusion/core/README.md index c83e33991a07..5a9493d086cd 100644 --- a/datafusion/core/README.md +++ b/datafusion/core/README.md @@ -19,7 +19,7 @@ # DataFusion Common -[DataFusion](df) is an extensible query execution framework, written in Rust, that uses Apache Arrow as its in-memory format. +[DataFusion][df] is an extensible query execution framework, written in Rust, that uses Apache Arrow as its in-memory format. This crate contains the main entrypoints and high level DataFusion APIs such as SessionContext, and DataFrame and ListingTable. diff --git a/datafusion/execution/README.md b/datafusion/execution/README.md index 40b3ffaa3ed3..67aac6be82b3 100644 --- a/datafusion/execution/README.md +++ b/datafusion/execution/README.md @@ -19,7 +19,7 @@ # DataFusion Common -[DataFusion](df) is an extensible query execution framework, written in Rust, that uses Apache Arrow as its in-memory format. +[DataFusion][df] is an extensible query execution framework, written in Rust, that uses Apache Arrow as its in-memory format. This crate is a submodule of DataFusion that provides execution runtime such as the memory pools and disk manager. diff --git a/datafusion/expr/README.md b/datafusion/expr/README.md index bcce30be39d9..b086f930e871 100644 --- a/datafusion/expr/README.md +++ b/datafusion/expr/README.md @@ -19,7 +19,7 @@ # DataFusion Logical Plan and Expressions -[DataFusion](df) is an extensible query execution framework, written in Rust, that uses Apache Arrow as its in-memory format. +[DataFusion][df] is an extensible query execution framework, written in Rust, that uses Apache Arrow as its in-memory format. This crate is a submodule of DataFusion that provides data types and utilities for logical plans and expressions. diff --git a/datafusion/optimizer/README.md b/datafusion/optimizer/README.md index c8baae03efa2..b8e5b93e6692 100644 --- a/datafusion/optimizer/README.md +++ b/datafusion/optimizer/README.md @@ -19,7 +19,7 @@ # DataFusion Query Optimizer -[DataFusion](df) is an extensible query execution framework, written in Rust, that uses Apache Arrow as its in-memory +[DataFusion][df] is an extensible query execution framework, written in Rust, that uses Apache Arrow as its in-memory format. DataFusion has modular design, allowing individual crates to be re-used in other projects. diff --git a/datafusion/physical-expr/README.md b/datafusion/physical-expr/README.md index a887d3eb29fe..424256c77e7e 100644 --- a/datafusion/physical-expr/README.md +++ b/datafusion/physical-expr/README.md @@ -19,7 +19,7 @@ # DataFusion Physical Expressions -[DataFusion](df) is an extensible query execution framework, written in Rust, that uses Apache Arrow as its in-memory format. +[DataFusion][df] is an extensible query execution framework, written in Rust, that uses Apache Arrow as its in-memory format. This crate is a submodule of DataFusion that provides data types and utilities for physical expressions. diff --git a/datafusion/physical-plan/README.md b/datafusion/physical-plan/README.md index 3632ea7d2a26..366a6b555150 100644 --- a/datafusion/physical-plan/README.md +++ b/datafusion/physical-plan/README.md @@ -19,7 +19,7 @@ # DataFusion Common -[DataFusion](df) is an extensible query execution framework, written in Rust, that uses Apache Arrow as its in-memory format. +[DataFusion][df] is an extensible query execution framework, written in Rust, that uses Apache Arrow as its in-memory format. This crate is a submodule of DataFusion that contains the `ExecutionPlan` trait and the various implementations of that trait for built in operators such as filters, projections, joins, aggregations, etc. diff --git a/datafusion/proto/README.md b/datafusion/proto/README.md index fd66d54aa2de..171aadb744d6 100644 --- a/datafusion/proto/README.md +++ b/datafusion/proto/README.md @@ -19,7 +19,7 @@ # DataFusion Proto -[DataFusion](df) is an extensible query execution framework, written in Rust, that uses Apache Arrow as its in-memory format. +[DataFusion][df] is an extensible query execution framework, written in Rust, that uses Apache Arrow as its in-memory format. This crate is a submodule of DataFusion that provides a protocol buffer format for representing query plans and expressions. diff --git a/datafusion/sql/README.md b/datafusion/sql/README.md index 2ad994e4eba5..256fa774b410 100644 --- a/datafusion/sql/README.md +++ b/datafusion/sql/README.md @@ -20,7 +20,7 @@ # DataFusion SQL Query Planner This crate provides a general purpose SQL query planner that can parse SQL and translate queries into logical -plans. Although this crate is used by the [DataFusion](df) query engine, it was designed to be easily usable from any +plans. Although this crate is used by the [DataFusion][df] query engine, it was designed to be easily usable from any project that requires a SQL query planner and does not make any assumptions about how the resulting logical plan will be translated to a physical plan. For example, there is no concept of row-based versus columnar execution in the logical plan. diff --git a/datafusion/wasmtest/README.md b/datafusion/wasmtest/README.md index 21e505f38e02..d26369a18ab9 100644 --- a/datafusion/wasmtest/README.md +++ b/datafusion/wasmtest/README.md @@ -19,7 +19,7 @@ # DataFusion wasmtest -[DataFusion](df) is an extensible query execution framework, written in Rust, that uses Apache Arrow as its in-memory format. +[DataFusion][df] is an extensible query execution framework, written in Rust, that uses Apache Arrow as its in-memory format. This crate is a submodule of DataFusion used to verify that various DataFusion crates compile successfully to the `wasm32-unknown-unknown` target with wasm-pack. From 7ee2c0bd8d33412dbc8fd3298254904e78b5156f Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Mon, 30 Oct 2023 13:37:10 -0400 Subject: [PATCH 14/43] Minor: Upate the `sqllogictest` crate README (#7971) * Minor: Upate the sqllogictest crate README * prettier * Apply suggestions from code review Co-authored-by: Jonah Gao Co-authored-by: jakevin --------- Co-authored-by: Jonah Gao Co-authored-by: jakevin --- datafusion/sqllogictest/README.md | 41 ++++++++++++++++++------------- 1 file changed, 24 insertions(+), 17 deletions(-) diff --git a/datafusion/sqllogictest/README.md b/datafusion/sqllogictest/README.md index 3e94859d35a7..0349ed852f46 100644 --- a/datafusion/sqllogictest/README.md +++ b/datafusion/sqllogictest/README.md @@ -17,19 +17,26 @@ under the License. --> -#### Overview +# DataFusion sqllogictest -This is the Datafusion implementation of [sqllogictest](https://www.sqlite.org/sqllogictest/doc/trunk/about.wiki). We -use [sqllogictest-rs](https://github.com/risinglightdb/sqllogictest-rs) as a parser/runner of `.slt` files -in [`test_files`](test_files). +[DataFusion][df] is an extensible query execution framework, written in Rust, that uses Apache Arrow as its in-memory format. -#### Testing setup +This crate is a submodule of DataFusion that contains an implementation of [sqllogictest](https://www.sqlite.org/sqllogictest/doc/trunk/about.wiki). + +[df]: https://crates.io/crates/datafusion + +## 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. + +## Testing setup 1. `rustup update stable` DataFusion uses the latest stable release of rust 2. `git submodule init` 3. `git submodule update` -#### Running tests: TLDR Examples +## Running tests: TLDR Examples ```shell # Run all tests @@ -56,7 +63,7 @@ cargo test --test sqllogictests -- ddl --complete RUST_LOG=debug cargo test --test sqllogictests -- ddl ``` -#### Cookbook: Adding Tests +## Cookbook: Adding Tests 1. Add queries @@ -95,11 +102,11 @@ SELECT * from foo; Assuming it looks good, check it in! -#### Reference +# Reference -#### Running tests: Validation Mode +## Running tests: Validation Mode -In this model, `sqllogictests` runs the statements and queries in a `.slt` file, comparing the expected output in the +In this mode, `sqllogictests` runs the statements and queries in a `.slt` file, comparing the expected output in the file to the output produced by that run. For example, to run all tests suites in validation mode @@ -115,10 +122,10 @@ sqllogictests also supports `cargo test` style substring matches on file names t cargo test --test sqllogictests -- information ``` -#### Running tests: Postgres compatibility +## Running tests: Postgres compatibility Test files that start with prefix `pg_compat_` verify compatibility -with Postgres by running the same script files both with DataFusion and with Posgres +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: @@ -145,7 +152,7 @@ docker run \ postgres ``` -#### Running Tests: `tpch` +## Running Tests: `tpch` Test files in `tpch` directory runs against the `TPCH` data set (SF = 0.1), which must be generated before running. You can use following @@ -165,7 +172,7 @@ Then you need to add `INCLUDE_TPCH=true` to run tpch tests: INCLUDE_TPCH=true cargo test --test sqllogictests ``` -#### Updating tests: Completion Mode +## Updating tests: Completion Mode In test script completion mode, `sqllogictests` reads a prototype script and runs the statements and queries against the database engine. The output is a full script that is a copy of the prototype script with result inserted. @@ -177,7 +184,7 @@ You can update the tests / generate expected output by passing the `--complete` cargo test --test sqllogictests -- ddl --complete ``` -#### Running tests: `scratchdir` +## Running tests: `scratchdir` The DataFusion sqllogictest runner automatically creates a directory named `test_files/scratch/`, creating it if needed and @@ -190,7 +197,7 @@ Tests that need to write temporary files should write (only) to this directory to ensure they do not interfere with others concurrently running tests. -#### `.slt` file format +## `.slt` file format [`sqllogictest`] was originally written for SQLite to verify the correctness of SQL queries against the SQLite engine. The format is designed @@ -247,7 +254,7 @@ query > :warning: It is encouraged to either apply `order by`, or use `rowsort` for queries without explicit `order by` > clauses. -##### Example +### Example ```sql # group_by_distinct From bb1d7f9343532d5fa8df871ff42000fbe836d7d7 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=BC=A0=E6=9E=97=E4=BC=9F?= Date: Tue, 31 Oct 2023 01:40:34 +0800 Subject: [PATCH 15/43] Improve MemoryCatalogProvider default impl block placement (#7975) --- datafusion/core/src/catalog/mod.rs | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/datafusion/core/src/catalog/mod.rs b/datafusion/core/src/catalog/mod.rs index fe5bdc0ec6a9..ce27d57da00d 100644 --- a/datafusion/core/src/catalog/mod.rs +++ b/datafusion/core/src/catalog/mod.rs @@ -93,12 +93,6 @@ impl CatalogList for MemoryCatalogList { } } -impl Default for MemoryCatalogProvider { - fn default() -> Self { - Self::new() - } -} - /// Represents a catalog, comprising a number of named schemas. pub trait CatalogProvider: Sync + Send { /// Returns the catalog provider as [`Any`] @@ -161,6 +155,12 @@ impl MemoryCatalogProvider { } } +impl Default for MemoryCatalogProvider { + fn default() -> Self { + Self::new() + } +} + impl CatalogProvider for MemoryCatalogProvider { fn as_any(&self) -> &dyn Any { self From 448dff519c2c46189b5909b542f15df36766544b Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Mon, 30 Oct 2023 13:23:00 -0700 Subject: [PATCH 16/43] Fix `ScalarValue` handling of NULL values for ListArray (#7969) * Fix try_from_array data type for NULL value in ListArray * Fix * Explicitly assert the datatype * For review --- datafusion/common/src/scalar.rs | 125 +++++++++++++++---- datafusion/sqllogictest/test_files/array.slt | 11 ++ 2 files changed, 111 insertions(+), 25 deletions(-) diff --git a/datafusion/common/src/scalar.rs b/datafusion/common/src/scalar.rs index be24e2b933b5..f9b0cbdf22de 100644 --- a/datafusion/common/src/scalar.rs +++ b/datafusion/common/src/scalar.rs @@ -1312,10 +1312,11 @@ impl ScalarValue { Arc::new(ListArray::from_iter_primitive::<$ARRAY_TY, _, _>( scalars.into_iter().map(|x| match x { ScalarValue::List(arr) => { - if arr.as_any().downcast_ref::().is_some() { + // `ScalarValue::List` contains a single element `ListArray`. + let list_arr = as_list_array(&arr); + if list_arr.is_null(0) { None } else { - let list_arr = as_list_array(&arr); let primitive_arr = list_arr.values().as_primitive::<$ARRAY_TY>(); Some( @@ -1339,12 +1340,14 @@ impl ScalarValue { for scalar in scalars.into_iter() { match scalar { ScalarValue::List(arr) => { - if arr.as_any().downcast_ref::().is_some() { + // `ScalarValue::List` contains a single element `ListArray`. + let list_arr = as_list_array(&arr); + + if list_arr.is_null(0) { builder.append(false); continue; } - let list_arr = as_list_array(&arr); let string_arr = $STRING_ARRAY(list_arr.values()); for v in string_arr.iter() { @@ -1699,15 +1702,16 @@ impl ScalarValue { for scalar in scalars { if let ScalarValue::List(arr) = scalar { - // i.e. NullArray(1) - if arr.as_any().downcast_ref::().is_some() { + // `ScalarValue::List` contains a single element `ListArray`. + let list_arr = as_list_array(&arr); + + if list_arr.is_null(0) { // Repeat previous offset index offsets.push(0); // Element is null valid.append(false); } else { - let list_arr = as_list_array(&arr); let arr = list_arr.values().to_owned(); offsets.push(arr.len()); elements.push(arr); @@ -2234,28 +2238,20 @@ impl ScalarValue { } DataType::Utf8 => typed_cast!(array, index, StringArray, Utf8), DataType::LargeUtf8 => typed_cast!(array, index, LargeStringArray, LargeUtf8), - DataType::List(nested_type) => { + DataType::List(_) => { let list_array = as_list_array(array); - let arr = match list_array.is_null(index) { - true => new_null_array(nested_type.data_type(), 0), - false => { - let nested_array = list_array.value(index); - Arc::new(wrap_into_list_array(nested_array)) - } - }; + let nested_array = list_array.value(index); + // Produces a single element `ListArray` with the value at `index`. + let arr = Arc::new(wrap_into_list_array(nested_array)); ScalarValue::List(arr) } // TODO: There is no test for FixedSizeList now, add it later - DataType::FixedSizeList(nested_type, _len) => { + DataType::FixedSizeList(_, _) => { let list_array = as_fixed_size_list_array(array)?; - let arr = match list_array.is_null(index) { - true => new_null_array(nested_type.data_type(), 0), - false => { - let nested_array = list_array.value(index); - Arc::new(wrap_into_list_array(nested_array)) - } - }; + let nested_array = list_array.value(index); + // Produces a single element `ListArray` with the value at `index`. + let arr = Arc::new(wrap_into_list_array(nested_array)); ScalarValue::List(arr) } @@ -2944,8 +2940,15 @@ impl TryFrom<&DataType> for ScalarValue { index_type.clone(), Box::new(value_type.as_ref().try_into()?), ), - DataType::List(_) => ScalarValue::List(new_null_array(&DataType::Null, 0)), - + // `ScalaValue::List` contains single element `ListArray`. + DataType::List(field) => ScalarValue::List(new_null_array( + &DataType::List(Arc::new(Field::new( + "item", + field.data_type().clone(), + true, + ))), + 1, + )), DataType::Struct(fields) => ScalarValue::Struct(None, fields.clone()), DataType::Null => ScalarValue::Null, _ => { @@ -3885,6 +3888,78 @@ mod tests { ); } + #[test] + fn scalar_try_from_array_list_array_null() { + let list = ListArray::from_iter_primitive::(vec![ + Some(vec![Some(1), Some(2)]), + None, + ]); + + let non_null_list_scalar = ScalarValue::try_from_array(&list, 0).unwrap(); + let null_list_scalar = ScalarValue::try_from_array(&list, 1).unwrap(); + + let data_type = + DataType::List(Arc::new(Field::new("item", DataType::Int32, true))); + + assert_eq!(non_null_list_scalar.data_type(), data_type.clone()); + assert_eq!(null_list_scalar.data_type(), data_type); + } + + #[test] + fn scalar_try_from_list() { + let data_type = + DataType::List(Arc::new(Field::new("item", DataType::Int32, true))); + let data_type = &data_type; + let scalar: ScalarValue = data_type.try_into().unwrap(); + + let expected = ScalarValue::List(new_null_array( + &DataType::List(Arc::new(Field::new("item", DataType::Int32, true))), + 1, + )); + + assert_eq!(expected, scalar) + } + + #[test] + fn scalar_try_from_list_of_list() { + let data_type = DataType::List(Arc::new(Field::new( + "item", + DataType::List(Arc::new(Field::new("item", DataType::Int32, true))), + true, + ))); + let data_type = &data_type; + let scalar: ScalarValue = data_type.try_into().unwrap(); + + let expected = ScalarValue::List(new_null_array( + &DataType::List(Arc::new(Field::new( + "item", + DataType::List(Arc::new(Field::new("item", DataType::Int32, true))), + true, + ))), + 1, + )); + + assert_eq!(expected, scalar) + } + + #[test] + fn scalar_try_from_not_equal_list_nested_list() { + let list_data_type = + DataType::List(Arc::new(Field::new("item", DataType::Int32, true))); + let data_type = &list_data_type; + let list_scalar: ScalarValue = data_type.try_into().unwrap(); + + let nested_list_data_type = DataType::List(Arc::new(Field::new( + "item", + DataType::List(Arc::new(Field::new("item", DataType::Int32, true))), + true, + ))); + let data_type = &nested_list_data_type; + let nested_list_scalar: ScalarValue = data_type.try_into().unwrap(); + + assert_ne!(list_scalar, nested_list_scalar); + } + #[test] fn scalar_try_from_dict_datatype() { let data_type = diff --git a/datafusion/sqllogictest/test_files/array.slt b/datafusion/sqllogictest/test_files/array.slt index 621cb4a8f4c0..b5601a22226c 100644 --- a/datafusion/sqllogictest/test_files/array.slt +++ b/datafusion/sqllogictest/test_files/array.slt @@ -209,6 +209,17 @@ AS VALUES (make_array([28, 29, 30], [31, 32, 33], [34, 35, 36], [28, 29, 30], [31, 32, 33], [34, 35, 36], [28, 29, 30], [31, 32, 33], [34, 35, 36], [28, 29, 30]), [28, 29, 30], [37, 38, 39], 10) ; +query TTT +select arrow_typeof(column1), arrow_typeof(column2), arrow_typeof(column3) from arrays; +---- +List(Field { name: "item", data_type: List(Field { name: "item", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }) List(Field { name: "item", data_type: Float64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }) List(Field { name: "item", data_type: Utf8, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }) +List(Field { name: "item", data_type: List(Field { name: "item", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }) List(Field { name: "item", data_type: Float64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }) List(Field { name: "item", data_type: Utf8, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }) +List(Field { name: "item", data_type: List(Field { name: "item", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }) List(Field { name: "item", data_type: Float64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }) List(Field { name: "item", data_type: Utf8, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }) +List(Field { name: "item", data_type: List(Field { name: "item", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }) List(Field { name: "item", data_type: Float64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }) List(Field { name: "item", data_type: Utf8, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }) +List(Field { name: "item", data_type: List(Field { name: "item", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }) List(Field { name: "item", data_type: Float64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }) List(Field { name: "item", data_type: Utf8, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }) +List(Field { name: "item", data_type: List(Field { name: "item", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }) List(Field { name: "item", data_type: Float64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }) List(Field { name: "item", data_type: Utf8, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }) +List(Field { name: "item", data_type: List(Field { name: "item", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }) List(Field { name: "item", data_type: Float64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }) List(Field { name: "item", data_type: Utf8, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }) + # arrays table query ??? select column1, column2, column3 from arrays; From 0d4dc3601b390c50640e841e70c522cd733e02f4 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Berkay=20=C5=9Eahin?= <124376117+berkaysynnada@users.noreply.github.com> Date: Tue, 31 Oct 2023 00:01:09 +0300 Subject: [PATCH 17/43] Refactor of Ordering and Prunability Traversals and States (#7985) * simplify ExprOrdering * Comment improvements * Move map/transform comment up --------- Co-authored-by: Mehmet Ozan Kabak --- .../physical-expr/src/sort_properties.rs | 59 ++++++------ datafusion/physical-expr/src/utils.rs | 90 ++++++++++++++++++- 2 files changed, 120 insertions(+), 29 deletions(-) diff --git a/datafusion/physical-expr/src/sort_properties.rs b/datafusion/physical-expr/src/sort_properties.rs index 097f491cb979..8ae3379218fb 100644 --- a/datafusion/physical-expr/src/sort_properties.rs +++ b/datafusion/physical-expr/src/sort_properties.rs @@ -155,37 +155,36 @@ impl Neg for SortProperties { #[derive(Debug)] pub struct ExprOrdering { pub expr: Arc, - pub state: Option, - pub children_states: Option>, + pub state: SortProperties, + pub children_states: Vec, } impl ExprOrdering { + /// Creates a new [`ExprOrdering`] with [`SortProperties::Unordered`] states + /// for `expr` and its children. pub fn new(expr: Arc) -> Self { + let size = expr.children().len(); Self { expr, - state: None, - children_states: None, + state: SortProperties::Unordered, + children_states: vec![SortProperties::Unordered; size], } } - pub fn children(&self) -> Vec { + /// Updates this [`ExprOrdering`]'s children states with the given states. + pub fn with_new_children(mut self, children_states: Vec) -> Self { + self.children_states = children_states; + self + } + + /// Creates new [`ExprOrdering`] objects for each child of the expression. + pub fn children_expr_orderings(&self) -> Vec { self.expr .children() .into_iter() .map(ExprOrdering::new) .collect() } - - pub fn new_with_children( - children_states: Vec, - parent_expr: Arc, - ) -> Self { - Self { - expr: parent_expr, - state: None, - children_states: Some(children_states), - } - } } impl TreeNode for ExprOrdering { @@ -193,7 +192,7 @@ impl TreeNode for ExprOrdering { where F: FnMut(&Self) -> Result, { - for child in self.children() { + for child in self.children_expr_orderings() { match op(&child)? { VisitRecursion::Continue => {} VisitRecursion::Skip => return Ok(VisitRecursion::Continue), @@ -207,17 +206,20 @@ impl TreeNode for ExprOrdering { where F: FnMut(Self) -> Result, { - let children = self.children(); - if children.is_empty() { + if self.children_states.is_empty() { Ok(self) } else { - Ok(ExprOrdering::new_with_children( - children + let child_expr_orderings = self.children_expr_orderings(); + // After mapping over the children, the function `F` applies to the + // current object and updates its state. + Ok(self.with_new_children( + child_expr_orderings .into_iter() + // Update children states after this transformation: .map(transform) - .map_ok(|c| c.state.unwrap_or(SortProperties::Unordered)) + // Extract the state (i.e. sort properties) information: + .map_ok(|c| c.state) .collect::>>()?, - self.expr, )) } } @@ -248,13 +250,13 @@ pub fn update_ordering( // a BinaryExpr like a + b), and there is an ordering equivalence of // it (let's say like c + d), we actually can find it at this step. if sort_expr.expr.eq(&node.expr) { - node.state = Some(SortProperties::Ordered(sort_expr.options)); + node.state = SortProperties::Ordered(sort_expr.options); return Ok(Transformed::Yes(node)); } - if let Some(children_sort_options) = &node.children_states { + if !node.expr.children().is_empty() { // We have an intermediate (non-leaf) node, account for its children: - node.state = Some(node.expr.get_ordering(children_sort_options)); + node.state = node.expr.get_ordering(&node.children_states); } else if let Some(column) = node.expr.as_any().downcast_ref::() { // We have a Column, which is one of the two possible leaf node types: node.state = get_indices_of_matching_sort_exprs_with_order_eq( @@ -268,10 +270,11 @@ pub fn update_ordering( descending: sort_options[0].descending, nulls_first: sort_options[0].nulls_first, }) - }); + }) + .unwrap_or(SortProperties::Unordered); } else { // We have a Literal, which is the other possible leaf node type: - node.state = Some(node.expr.get_ordering(&[])); + node.state = node.expr.get_ordering(&[]); } Ok(Transformed::Yes(node)) } diff --git a/datafusion/physical-expr/src/utils.rs b/datafusion/physical-expr/src/utils.rs index b2a6bb5ca6d2..b38117d206cc 100644 --- a/datafusion/physical-expr/src/utils.rs +++ b/datafusion/physical-expr/src/utils.rs @@ -773,7 +773,7 @@ pub fn find_orderings_of_exprs( &input_ordering_equal_properties, ) })?; - if let Some(SortProperties::Ordered(sort_options)) = transformed.state { + if let SortProperties::Ordered(sort_options) = transformed.state { orderings.push(Some(PhysicalSortExpr { expr: Arc::new(Column::new(name, index)), options: sort_options, @@ -1836,4 +1836,92 @@ mod tests { Ok(()) } + + #[test] + fn test_find_orderings_of_exprs() -> Result<()> { + let schema = Schema::new(vec![ + Field::new("a", DataType::Int32, true), + Field::new("b", DataType::Int32, true), + Field::new("c", DataType::Int32, true), + Field::new("d", DataType::Int32, true), + ]); + + let mut eq = EquivalenceProperties::new(Arc::new(schema.clone())); + let col_a = &col("a", &schema)?; + let col_b = &col("b", &schema)?; + let col_c = &col("c", &schema)?; + let col_d = &col("d", &schema)?; + let option_asc = SortOptions { + descending: false, + nulls_first: false, + }; + // b=a (e.g they are aliases) + eq.add_equal_conditions((&Column::new("b", 1), &Column::new("a", 0))); + let mut oeq = OrderingEquivalenceProperties::new(Arc::new(schema.clone())); + // [b ASC], [d ASC] + oeq.add_equal_conditions(( + &vec![PhysicalSortExpr { + expr: col_b.clone(), + options: option_asc, + }], + &vec![PhysicalSortExpr { + expr: col_d.clone(), + options: option_asc, + }], + )); + + let orderings = find_orderings_of_exprs( + &[ + // d + b + ( + Arc::new(BinaryExpr::new( + col_d.clone(), + Operator::Plus, + col_b.clone(), + )), + "d+b".to_string(), + ), + // b as b_new + (col_b.clone(), "b_new".to_string()), + // a as a_new + (col_a.clone(), "a_new".to_string()), + // a + c + ( + Arc::new(BinaryExpr::new( + col_a.clone(), + Operator::Plus, + col_c.clone(), + )), + "a+c".to_string(), + ), + ], + Some(&[PhysicalSortExpr { + expr: col_b.clone(), + options: option_asc, + }]), + eq, + oeq, + )?; + + assert_eq!( + vec![ + Some(PhysicalSortExpr { + expr: Arc::new(Column::new("d+b", 0)), + options: option_asc, + }), + Some(PhysicalSortExpr { + expr: Arc::new(Column::new("b_new", 1)), + options: option_asc, + }), + Some(PhysicalSortExpr { + expr: Arc::new(Column::new("a_new", 2)), + options: option_asc, + }), + None, + ], + orderings + ); + + Ok(()) + } } From 3d78bf471f8b2a66c4298e1aca230bc87e7b4c25 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Tue, 31 Oct 2023 00:05:03 -0700 Subject: [PATCH 18/43] Keep output as scalar for scalar function if all inputs are scalar (#7967) * Keep output as scalar for scalar function if all inputs are scalar * Add end-to-end tests --- datafusion/physical-expr/src/functions.rs | 11 ++++- datafusion/physical-expr/src/planner.rs | 34 +++++++++++++ datafusion/sqllogictest/test_files/scalar.slt | 48 +++++++++++++++++++ 3 files changed, 92 insertions(+), 1 deletion(-) diff --git a/datafusion/physical-expr/src/functions.rs b/datafusion/physical-expr/src/functions.rs index 8422862043ae..b66bac41014d 100644 --- a/datafusion/physical-expr/src/functions.rs +++ b/datafusion/physical-expr/src/functions.rs @@ -357,6 +357,8 @@ where ColumnarValue::Array(a) => Some(a.len()), }); + let is_scalar = len.is_none(); + let inferred_length = len.unwrap_or(1); let args = args .iter() @@ -373,7 +375,14 @@ where .collect::>(); let result = (inner)(&args); - result.map(ColumnarValue::Array) + + if is_scalar { + // If all inputs are scalar, keeps output as scalar + let result = result.and_then(|arr| ScalarValue::try_from_array(&arr, 0)); + result.map(ColumnarValue::Scalar) + } else { + result.map(ColumnarValue::Array) + } }) } diff --git a/datafusion/physical-expr/src/planner.rs b/datafusion/physical-expr/src/planner.rs index 9a74c2ca64d1..64c1d0be0455 100644 --- a/datafusion/physical-expr/src/planner.rs +++ b/datafusion/physical-expr/src/planner.rs @@ -448,3 +448,37 @@ pub fn create_physical_expr( } } } + +#[cfg(test)] +mod tests { + use super::*; + use arrow_array::{ArrayRef, BooleanArray, RecordBatch, StringArray}; + use arrow_schema::{DataType, Field, Schema}; + use datafusion_common::{DFSchema, Result}; + use datafusion_expr::{col, left, Literal}; + + #[test] + fn test_create_physical_expr_scalar_input_output() -> Result<()> { + let expr = col("letter").eq(left("APACHE".lit(), 1i64.lit())); + + let schema = Schema::new(vec![Field::new("letter", DataType::Utf8, false)]); + let df_schema = DFSchema::try_from_qualified_schema("data", &schema)?; + let p = create_physical_expr(&expr, &df_schema, &schema, &ExecutionProps::new())?; + + let batch = RecordBatch::try_new( + Arc::new(schema), + vec![Arc::new(StringArray::from_iter_values(vec![ + "A", "B", "C", "D", + ]))], + )?; + let result = p.evaluate(&batch)?; + let result = result.into_array(4); + + assert_eq!( + &result, + &(Arc::new(BooleanArray::from(vec![true, false, false, false,])) as ArrayRef) + ); + + Ok(()) + } +} diff --git a/datafusion/sqllogictest/test_files/scalar.slt b/datafusion/sqllogictest/test_files/scalar.slt index e5c1a828492a..ecb7fe13fcf4 100644 --- a/datafusion/sqllogictest/test_files/scalar.slt +++ b/datafusion/sqllogictest/test_files/scalar.slt @@ -1878,3 +1878,51 @@ query T SELECT CONCAT('Hello', 'World') ---- HelloWorld + +statement ok +CREATE TABLE simple_string( + letter STRING, + letter2 STRING +) as VALUES + ('A', 'APACHE'), + ('B', 'APACHE'), + ('C', 'APACHE'), + ('D', 'APACHE') +; + +query TT +EXPLAIN SELECT letter, letter = LEFT('APACHE', 1) FROM simple_string; +---- +logical_plan +Projection: simple_string.letter, simple_string.letter = Utf8("A") AS simple_string.letter = left(Utf8("APACHE"),Int64(1)) +--TableScan: simple_string projection=[letter] +physical_plan +ProjectionExec: expr=[letter@0 as letter, letter@0 = A as simple_string.letter = left(Utf8("APACHE"),Int64(1))] +--MemoryExec: partitions=1, partition_sizes=[1] + +query TB +SELECT letter, letter = LEFT('APACHE', 1) FROM simple_string; + ---- +---- +A true +B false +C false +D false + +query TT +EXPLAIN SELECT letter, letter = LEFT(letter2, 1) FROM simple_string; +---- +logical_plan +Projection: simple_string.letter, simple_string.letter = left(simple_string.letter2, Int64(1)) +--TableScan: simple_string projection=[letter, letter2] +physical_plan +ProjectionExec: expr=[letter@0 as letter, letter@0 = left(letter2@1, 1) as simple_string.letter = left(simple_string.letter2,Int64(1))] +--MemoryExec: partitions=1, partition_sizes=[1] + +query TB +SELECT letter, letter = LEFT(letter2, 1) FROM simple_string; +---- +A true +B false +C false +D false From d8e413c0b92b86593ed9801a034bd62bdb9ddc0b Mon Sep 17 00:00:00 2001 From: Jeffrey <22608443+Jefffrey@users.noreply.github.com> Date: Tue, 31 Oct 2023 19:11:42 +1100 Subject: [PATCH 19/43] Fix crate READMEs for core, execution, physical-plan (#7990) --- datafusion/core/Cargo.toml | 2 +- datafusion/execution/Cargo.toml | 2 +- datafusion/physical-plan/Cargo.toml | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/datafusion/core/Cargo.toml b/datafusion/core/Cargo.toml index 5f9d28bd620b..a9614934673b 100644 --- a/datafusion/core/Cargo.toml +++ b/datafusion/core/Cargo.toml @@ -20,9 +20,9 @@ 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"] +readme = "README.md" version = { workspace = true } edition = { workspace = true } -readme = { workspace = true } homepage = { workspace = true } repository = { workspace = true } license = { workspace = true } diff --git a/datafusion/execution/Cargo.toml b/datafusion/execution/Cargo.toml index 6ae8bccdae38..54a2a6d8f3df 100644 --- a/datafusion/execution/Cargo.toml +++ b/datafusion/execution/Cargo.toml @@ -19,9 +19,9 @@ name = "datafusion-execution" description = "Execution configuration support for DataFusion query engine" keywords = ["arrow", "query", "sql"] +readme = "README.md" version = { workspace = true } edition = { workspace = true } -readme = { workspace = true } homepage = { workspace = true } repository = { workspace = true } license = { workspace = true } diff --git a/datafusion/physical-plan/Cargo.toml b/datafusion/physical-plan/Cargo.toml index 2dfcf12e350a..033d2aa187dd 100644 --- a/datafusion/physical-plan/Cargo.toml +++ b/datafusion/physical-plan/Cargo.toml @@ -19,9 +19,9 @@ name = "datafusion-physical-plan" description = "Physical (ExecutionPlan) implementations for DataFusion query engine" keywords = ["arrow", "query", "sql"] +readme = "README.md" version = { workspace = true } edition = { workspace = true } -readme = { workspace = true } homepage = { workspace = true } repository = { workspace = true } license = { workspace = true } From 8fcc5e03147db93d688c2dfa112653191476f2bf Mon Sep 17 00:00:00 2001 From: jakevin Date: Tue, 31 Oct 2023 17:43:52 +0800 Subject: [PATCH 20/43] Update sqlparser requirement from 0.38.0 to 0.39.0 (#7983) * chore: Update sqlparser requirement from 0.38.0 to 0.39.0 * support FILTER Aggregates --- Cargo.toml | 2 +- datafusion-cli/Cargo.lock | 363 ++++++++++++++-------------- datafusion/sql/src/expr/function.rs | 7 +- datafusion/sql/src/expr/mod.rs | 35 ++- datafusion/sql/src/planner.rs | 16 +- datafusion/sql/src/set_expr.rs | 3 + datafusion/sql/src/statement.rs | 15 ++ 7 files changed, 252 insertions(+), 189 deletions(-) diff --git a/Cargo.toml b/Cargo.toml index 77e3c6038ea7..5aa830ddff0f 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -55,7 +55,7 @@ arrow-buffer = { version = "48.0.0", default-features = false } arrow-flight = { version = "48.0.0", features = ["flight-sql-experimental"] } arrow-schema = { version = "48.0.0", default-features = false } parquet = { version = "48.0.0", features = ["arrow", "async", "object_store"] } -sqlparser = { version = "0.38.0", features = ["visitor"] } +sqlparser = { version = "0.39.0", features = ["visitor"] } chrono = { version = "0.4.31", default-features = false } [profile.release] diff --git a/datafusion-cli/Cargo.lock b/datafusion-cli/Cargo.lock index b83088f94c57..268f9e28427a 100644 --- a/datafusion-cli/Cargo.lock +++ b/datafusion-cli/Cargo.lock @@ -25,15 +25,16 @@ checksum = "aae1277d39aeec15cb388266ecc24b11c80469deae6067e17a1a7aa9e5c1f234" [[package]] name = "ahash" -version = "0.8.3" +version = "0.8.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2c99f64d1e06488f620f932677e24bc6e2897582980441ae90a671415bd7ec2f" +checksum = "91429305e9f0a25f6205c5b8e0d2db09e0708a7a6df0f42212bb56c32c8ac97a" dependencies = [ "cfg-if", "const-random", "getrandom", "once_cell", "version_check", + "zerocopy", ] [[package]] @@ -106,8 +107,8 @@ dependencies = [ "serde", "serde_json", "snap", - "strum 0.25.0", - "strum_macros 0.25.2", + "strum", + "strum_macros", "thiserror", "typed-builder", "uuid", @@ -177,7 +178,7 @@ dependencies = [ "chrono", "chrono-tz", "half", - "hashbrown 0.14.1", + "hashbrown 0.14.2", "num", ] @@ -302,7 +303,7 @@ dependencies = [ "arrow-data", "arrow-schema", "half", - "hashbrown 0.14.1", + "hashbrown 0.14.2", ] [[package]] @@ -358,9 +359,9 @@ dependencies = [ [[package]] name = "async-compression" -version = "0.4.3" +version = "0.4.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bb42b2197bf15ccb092b62c74515dbd8b86d0effd934795f6687c93b6e679a2c" +checksum = "f658e2baef915ba0f26f1f7c42bfb8e12f532a01f449a090ded75ae7a07e9ba2" dependencies = [ "bzip2", "flate2", @@ -370,15 +371,15 @@ dependencies = [ "pin-project-lite", "tokio", "xz2", - "zstd 0.12.4", - "zstd-safe 6.0.6", + "zstd 0.13.0", + "zstd-safe 7.0.0", ] [[package]] name = "async-trait" -version = "0.1.73" +version = "0.1.74" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bc00ceb34980c03614e35a3a4e218276a0a824e911d07651cd0d858a51e8c0f0" +checksum = "a66537f1bb974b254c98ed142ff995236e81b9d0fe4db0575f46612cb15eb0f9" dependencies = [ "proc-macro2", "quote", @@ -709,9 +710,9 @@ dependencies = [ [[package]] name = "base64" -version = "0.21.4" +version = "0.21.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9ba43ea6f343b788c8764558649e08df62f86c6ef251fdaeb1ffd010a9ae50a2" +checksum = "35636a1494ede3b646cc98f74f8e62c773a38a659ebc777a2cf26b9b74171df9" [[package]] name = "base64-simd" @@ -731,9 +732,9 @@ checksum = "bef38d45163c2f1dde094a7dfd33ccf595c92905c8f8f4fdc18d06fb1037718a" [[package]] name = "bitflags" -version = "2.4.0" +version = "2.4.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b4682ae6287fcf752ecaabbfcc7b6f9b72aa33933dc23a554d853aea8eea8635" +checksum = "327762f6e5a765692301e5bb513e0d9fef63be86bbc14528052b1cd3e6f03e07" [[package]] name = "blake2" @@ -779,9 +780,9 @@ dependencies = [ [[package]] name = "brotli-decompressor" -version = "2.5.0" +version = "2.5.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "da74e2b81409b1b743f8f0c62cc6254afefb8b8e50bbfe3735550f7aeefa3448" +checksum = "4e2e4afe60d7dd600fdd3de8d0f08c2b7ec039712e3b6137ff98b7004e82de4f" dependencies = [ "alloc-no-stdlib", "alloc-stdlib", @@ -878,9 +879,9 @@ dependencies = [ [[package]] name = "chrono-tz" -version = "0.8.3" +version = "0.8.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f1369bc6b9e9a7dfdae2055f6ec151fe9c554a9d23d357c0237cee2e25eaabb7" +checksum = "e23185c0e21df6ed832a12e2bda87c7d1def6842881fb634a8511ced741b0d76" dependencies = [ "chrono", "chrono-tz-build", @@ -889,9 +890,9 @@ dependencies = [ [[package]] name = "chrono-tz-build" -version = "0.2.0" +version = "0.2.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e2f5ebdc942f57ed96d560a6d1a459bae5851102a25d5bf89dc04ae453e31ecf" +checksum = "433e39f13c9a060046954e0592a8d0a4bcb1040125cbf91cb8ee58964cfb350f" dependencies = [ "parse-zoneinfo", "phf", @@ -950,34 +951,32 @@ dependencies = [ [[package]] name = "comfy-table" -version = "7.0.1" +version = "7.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9ab77dbd8adecaf3f0db40581631b995f312a8a5ae3aa9993188bb8f23d83a5b" +checksum = "7c64043d6c7b7a4c58e39e7efccfdea7b93d885a795d0c054a69dbbf4dd52686" dependencies = [ - "strum 0.24.1", - "strum_macros 0.24.3", + "strum", + "strum_macros", "unicode-width", ] [[package]] name = "const-random" -version = "0.1.15" +version = "0.1.16" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "368a7a772ead6ce7e1de82bfb04c485f3db8ec744f72925af5735e29a22cc18e" +checksum = "11df32a13d7892ec42d51d3d175faba5211ffe13ed25d4fb348ac9e9ce835593" dependencies = [ "const-random-macro", - "proc-macro-hack", ] [[package]] name = "const-random-macro" -version = "0.1.15" +version = "0.1.16" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9d7d6ab3c3a2282db210df5f02c4dab6e0a7057af0fb7ebd4070f30fe05c0ddb" +checksum = "f9d839f2a20b0aee515dc581a6172f2321f96cab76c1a38a4c584a194955390e" dependencies = [ "getrandom", "once_cell", - "proc-macro-hack", "tiny-keccak", ] @@ -1014,9 +1013,9 @@ dependencies = [ [[package]] name = "cpufeatures" -version = "0.2.9" +version = "0.2.11" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a17b76ff3a4162b0b27f354a0c87015ddad39d35f9c0c36607a3bdd175dde1f1" +checksum = "ce420fe07aecd3e67c5f910618fe65e94158f6dcc0adf44e00d69ce2bdfe0fd0" dependencies = [ "libc", ] @@ -1090,7 +1089,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "978747c1d849a7d2ee5e8adc0159961c48fb7e5db2f06af6723b80123bb53856" dependencies = [ "cfg-if", - "hashbrown 0.14.1", + "hashbrown 0.14.2", "lock_api", "once_cell", "parking_lot_core", @@ -1122,7 +1121,7 @@ dependencies = [ "futures", "glob", "half", - "hashbrown 0.14.1", + "hashbrown 0.14.2", "indexmap 2.0.2", "itertools", "log", @@ -1197,7 +1196,7 @@ dependencies = [ "datafusion-common", "datafusion-expr", "futures", - "hashbrown 0.14.1", + "hashbrown 0.14.2", "log", "object_store", "parking_lot", @@ -1215,8 +1214,8 @@ dependencies = [ "arrow-array", "datafusion-common", "sqlparser", - "strum 0.25.0", - "strum_macros 0.25.2", + "strum", + "strum_macros", ] [[package]] @@ -1229,7 +1228,7 @@ dependencies = [ "datafusion-common", "datafusion-expr", "datafusion-physical-expr", - "hashbrown 0.14.1", + "hashbrown 0.14.2", "itertools", "log", "regex-syntax", @@ -1251,7 +1250,7 @@ dependencies = [ "datafusion-common", "datafusion-expr", "half", - "hashbrown 0.14.1", + "hashbrown 0.14.2", "hex", "indexmap 2.0.2", "itertools", @@ -1284,7 +1283,7 @@ dependencies = [ "datafusion-physical-expr", "futures", "half", - "hashbrown 0.14.1", + "hashbrown 0.14.2", "indexmap 2.0.2", "itertools", "log", @@ -1310,9 +1309,12 @@ dependencies = [ [[package]] name = "deranged" -version = "0.3.8" +version = "0.3.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f2696e8a945f658fd14dc3b87242e6b80cd0f36ff04ea560fa39082368847946" +checksum = "0f32d04922c60427da6f9fef14d042d9edddef64cb9d4ce0d64d0685fbeb1fd3" +dependencies = [ + "powerfmt", +] [[package]] name = "difflib" @@ -1482,9 +1484,9 @@ dependencies = [ [[package]] name = "flate2" -version = "1.0.27" +version = "1.0.28" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c6c98ee8095e9d1dcbf2fcc6d95acccb90d1c81db1e44725c6a984b1dbdfb010" +checksum = "46303f565772937ffe1d394a4fac6f411c6013172fadde9dcdb1e147a086940e" dependencies = [ "crc32fast", "miniz_oxide", @@ -1516,9 +1518,9 @@ dependencies = [ [[package]] name = "futures" -version = "0.3.28" +version = "0.3.29" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "23342abe12aba583913b2e62f22225ff9c950774065e4bfb61a19cd9770fec40" +checksum = "da0290714b38af9b4a7b094b8a37086d1b4e61f2df9122c3cad2577669145335" dependencies = [ "futures-channel", "futures-core", @@ -1531,9 +1533,9 @@ dependencies = [ [[package]] name = "futures-channel" -version = "0.3.28" +version = "0.3.29" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "955518d47e09b25bbebc7a18df10b81f0c766eaf4c4f1cccef2fca5f2a4fb5f2" +checksum = "ff4dd66668b557604244583e3e1e1eada8c5c2e96a6d0d6653ede395b78bbacb" dependencies = [ "futures-core", "futures-sink", @@ -1541,15 +1543,15 @@ dependencies = [ [[package]] name = "futures-core" -version = "0.3.28" +version = "0.3.29" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4bca583b7e26f571124fe5b7561d49cb2868d79116cfa0eefce955557c6fee8c" +checksum = "eb1d22c66e66d9d72e1758f0bd7d4fd0bee04cad842ee34587d68c07e45d088c" [[package]] name = "futures-executor" -version = "0.3.28" +version = "0.3.29" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ccecee823288125bd88b4d7f565c9e58e41858e47ab72e8ea2d64e93624386e0" +checksum = "0f4fb8693db0cf099eadcca0efe2a5a22e4550f98ed16aba6c48700da29597bc" dependencies = [ "futures-core", "futures-task", @@ -1558,15 +1560,15 @@ dependencies = [ [[package]] name = "futures-io" -version = "0.3.28" +version = "0.3.29" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4fff74096e71ed47f8e023204cfd0aa1289cd54ae5430a9523be060cdb849964" +checksum = "8bf34a163b5c4c52d0478a4d757da8fb65cabef42ba90515efee0f6f9fa45aaa" [[package]] name = "futures-macro" -version = "0.3.28" +version = "0.3.29" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "89ca545a94061b6365f2c7355b4b32bd20df3ff95f02da9329b34ccc3bd6ee72" +checksum = "53b153fd91e4b0147f4aced87be237c98248656bb01050b96bf3ee89220a8ddb" dependencies = [ "proc-macro2", "quote", @@ -1575,15 +1577,15 @@ dependencies = [ [[package]] name = "futures-sink" -version = "0.3.28" +version = "0.3.29" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f43be4fe21a13b9781a69afa4985b0f6ee0e1afab2c6f454a8cf30e2b2237b6e" +checksum = "e36d3378ee38c2a36ad710c5d30c2911d752cb941c00c72dbabfb786a7970817" [[package]] name = "futures-task" -version = "0.3.28" +version = "0.3.29" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "76d3d132be6c0e6aa1534069c705a74a5997a356c0dc2f86a47765e5617c5b65" +checksum = "efd193069b0ddadc69c46389b740bbccdd97203899b48d09c5f7969591d6bae2" [[package]] name = "futures-timer" @@ -1593,9 +1595,9 @@ checksum = "e64b03909df88034c26dc1547e8970b91f98bdb65165d6a4e9110d94263dbb2c" [[package]] name = "futures-util" -version = "0.3.28" +version = "0.3.29" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "26b01e40b772d54cf6c6d721c1d1abd0647a0106a12ecaa1c186273392a69533" +checksum = "a19526d624e703a3179b3d322efec918b6246ea0fa51d41124525f00f1cc8104" dependencies = [ "futures-channel", "futures-core", @@ -1689,9 +1691,9 @@ dependencies = [ [[package]] name = "hashbrown" -version = "0.14.1" +version = "0.14.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7dfda62a12f55daeae5015f81b0baea145391cb4520f86c248fc615d72640d12" +checksum = "f93e7192158dbcda357bdec5fb5788eebf8bbac027f3f33e719d29135ae84156" dependencies = [ "ahash", "allocator-api2", @@ -1790,7 +1792,7 @@ dependencies = [ "httpdate", "itoa", "pin-project-lite", - "socket2 0.4.9", + "socket2 0.4.10", "tokio", "tower-service", "tracing", @@ -1814,30 +1816,30 @@ dependencies = [ [[package]] name = "hyper-rustls" -version = "0.24.1" +version = "0.24.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8d78e1e73ec14cf7375674f74d7dde185c8206fd9dea6fb6295e8a98098aaa97" +checksum = "ec3efd23720e2049821a693cbc7e65ea87c72f1c58ff2f9522ff332b1491e590" dependencies = [ "futures-util", "http", "hyper", - "rustls 0.21.7", + "rustls 0.21.8", "tokio", "tokio-rustls 0.24.1", ] [[package]] name = "iana-time-zone" -version = "0.1.57" +version = "0.1.58" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2fad5b825842d2b38bd206f3e81d6957625fd7f0a361e345c30e01a0ae2dd613" +checksum = "8326b86b6cff230b97d0d312a6c40a60726df3332e721f72a1b035f451663b20" dependencies = [ "android_system_properties", "core-foundation-sys", "iana-time-zone-haiku", "js-sys", "wasm-bindgen", - "windows", + "windows-core", ] [[package]] @@ -1876,7 +1878,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "8adf3ddd720272c6ea8bf59463c04e0f93d0bbf7c5439b691bca2987e0270897" dependencies = [ "equivalent", - "hashbrown 0.14.1", + "hashbrown 0.14.2", ] [[package]] @@ -1896,9 +1898,9 @@ checksum = "8bb03732005da905c88227371639bf1ad885cc712789c011c31c5fb3ab3ccf02" [[package]] name = "ipnet" -version = "2.8.0" +version = "2.9.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "28b29a3cd74f0f4598934efe3aeba42bae0eb4680554128851ebbecb02af14e6" +checksum = "8f518f335dce6725a761382244631d86cf0ccb2863413590b31338feb467f9c3" [[package]] name = "itertools" @@ -2057,9 +2059,9 @@ checksum = "da2479e8c062e40bf0066ffa0bc823de0a9368974af99c9f6df941d2c231e03f" [[package]] name = "lock_api" -version = "0.4.10" +version = "0.4.11" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c1cc9717a20b1bb222f333e6a92fd32f7d8a18ddc5a3191a11af45dcbf4dcd16" +checksum = "3c168f8615b12bc01f9c17e2eb0cc07dcae1940121185446edc3744920e8ef45" dependencies = [ "autocfg", "scopeguard", @@ -2133,9 +2135,9 @@ dependencies = [ [[package]] name = "mio" -version = "0.8.8" +version = "0.8.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "927a765cd3fc26206e66b296465fa9d3e5ab003e651c1b3c060e7956d96b19d2" +checksum = "3dce281c5e46beae905d4de1870d8b1509a9142b62eedf18b443b011ca8343d0" dependencies = [ "libc", "wasi", @@ -2317,9 +2319,9 @@ dependencies = [ [[package]] name = "os_str_bytes" -version = "6.5.1" +version = "6.6.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4d5d9eb14b174ee9aa2ef96dc2b94637a2d4b6e7cb873c7e171f0c20c6cf3eac" +checksum = "e2355d85b9a3786f481747ced0e0ff2ba35213a1f9bd406ed906554d7af805a1" [[package]] name = "outref" @@ -2339,13 +2341,13 @@ dependencies = [ [[package]] name = "parking_lot_core" -version = "0.9.8" +version = "0.9.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "93f00c865fe7cabf650081affecd3871070f26767e7b2070a3ffae14c654b447" +checksum = "4c42a9226546d68acdd9c0a280d17ce19bfe27a46bf68784e4066115788d008e" dependencies = [ "cfg-if", "libc", - "redox_syscall 0.3.5", + "redox_syscall 0.4.1", "smallvec", "windows-targets", ] @@ -2370,7 +2372,7 @@ dependencies = [ "chrono", "flate2", "futures", - "hashbrown 0.14.1", + "hashbrown 0.14.2", "lz4_flex", "num", "num-bigint", @@ -2491,6 +2493,12 @@ version = "0.3.27" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "26072860ba924cbfa98ea39c8c19b4dd6a4a25423dbdf219c1eca91aa0cf6964" +[[package]] +name = "powerfmt" +version = "0.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "439ee305def115ba05938db6eb1644ff94165c5ab5e9420d1c1bcedbba909391" + [[package]] name = "ppv-lite86" version = "0.2.17" @@ -2552,12 +2560,6 @@ dependencies = [ "version_check", ] -[[package]] -name = "proc-macro-hack" -version = "0.5.20+deprecated" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "dc375e1527247fe1a97d8b7156678dfe7c1af2fc075c9a4db3690ecd2a148068" - [[package]] name = "proc-macro2" version = "1.0.69" @@ -2643,9 +2645,9 @@ dependencies = [ [[package]] name = "redox_syscall" -version = "0.3.5" +version = "0.4.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "567664f262709473930a4bf9e51bf2ebf3348f2e748ccc50dea20646858f8f29" +checksum = "4722d768eff46b75989dd134e5c353f0d6296e5aaa3132e776cbdb56be7731aa" dependencies = [ "bitflags 1.3.2", ] @@ -2663,9 +2665,9 @@ dependencies = [ [[package]] name = "regex" -version = "1.10.0" +version = "1.10.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d119d7c7ca818f8a53c300863d4f87566aac09943aef5b355bb83969dae75d87" +checksum = "380b951a9c5e80ddfd6136919eef32310721aa4aacd4889a8d39124b026ab343" dependencies = [ "aho-corasick", "memchr", @@ -2675,9 +2677,9 @@ dependencies = [ [[package]] name = "regex-automata" -version = "0.4.1" +version = "0.4.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "465c6fc0621e4abc4187a2bda0937bfd4f722c2730b29562e19689ea796c9a4b" +checksum = "5f804c7828047e88b2d32e2d7fe5a105da8ee3264f01902f796c8e067dc2483f" dependencies = [ "aho-corasick", "memchr", @@ -2686,15 +2688,15 @@ dependencies = [ [[package]] name = "regex-lite" -version = "0.1.3" +version = "0.1.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9a6ebcd15653947e6140f59a9811a06ed061d18a5c35dfca2e2e4c5525696878" +checksum = "30b661b2f27137bdbc16f00eda72866a92bb28af1753ffbd56744fb6e2e9cd8e" [[package]] name = "regex-syntax" -version = "0.8.1" +version = "0.8.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "56d84fdd47036b038fc80dd333d10b6aab10d5d31f4a366e20014def75328d33" +checksum = "c08c74e62047bb2de4ff487b251e4a92e24f48745648451635cec7d591162d9f" [[package]] name = "reqwest" @@ -2711,7 +2713,7 @@ dependencies = [ "http", "http-body", "hyper", - "hyper-rustls 0.24.1", + "hyper-rustls 0.24.2", "ipnet", "js-sys", "log", @@ -2719,7 +2721,7 @@ dependencies = [ "once_cell", "percent-encoding", "pin-project-lite", - "rustls 0.21.7", + "rustls 0.21.8", "rustls-pemfile", "serde", "serde_json", @@ -2755,9 +2757,9 @@ dependencies = [ [[package]] name = "ring" -version = "0.17.3" +version = "0.17.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9babe80d5c16becf6594aa32ad2be8fe08498e7ae60b77de8df700e67f191d7e" +checksum = "fb0205304757e5d899b9c2e448b867ffd03ae7f988002e47cd24954391394d0b" dependencies = [ "cc", "getrandom", @@ -2816,11 +2818,11 @@ dependencies = [ [[package]] name = "rustix" -version = "0.38.18" +version = "0.38.21" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5a74ee2d7c2581cd139b42447d7d9389b889bdaad3a73f1ebb16f2a3237bb19c" +checksum = "2b426b0506e5d50a7d8dafcf2e81471400deb602392c7dd110815afb4eaf02a3" dependencies = [ - "bitflags 2.4.0", + "bitflags 2.4.1", "errno", "libc", "linux-raw-sys", @@ -2841,12 +2843,12 @@ dependencies = [ [[package]] name = "rustls" -version = "0.21.7" +version = "0.21.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cd8d6c9f025a446bc4d18ad9632e69aec8f287aa84499ee335599fabd20c3fd8" +checksum = "446e14c5cda4f3f30fe71863c34ec70f5ac79d6087097ad0bb433e1be5edf04c" dependencies = [ "log", - "ring 0.16.20", + "ring 0.17.5", "rustls-webpki", "sct", ] @@ -2874,12 +2876,12 @@ dependencies = [ [[package]] name = "rustls-webpki" -version = "0.101.6" +version = "0.101.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3c7d5dece342910d9ba34d259310cae3e0154b873b35408b787b59bce53d34fe" +checksum = "8b6275d1ee7a1cd780b64aca7726599a1dbc893b1e64144529e55c3c2f745765" dependencies = [ - "ring 0.16.20", - "untrusted 0.7.1", + "ring 0.17.5", + "untrusted 0.9.0", ] [[package]] @@ -2943,12 +2945,12 @@ checksum = "94143f37725109f92c262ed2cf5e59bce7498c01bcc1502d7b9afe439a4e9f49" [[package]] name = "sct" -version = "0.7.0" +version = "0.7.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d53dcdb7c9f8158937a7981b48accfd39a43af418591a5d008c7b22b5e1b7ca4" +checksum = "da046153aa2352493d6cb7da4b6e5c0c057d8a1d0a9aa8560baffdd945acd414" dependencies = [ - "ring 0.16.20", - "untrusted 0.7.1", + "ring 0.17.5", + "untrusted 0.9.0", ] [[package]] @@ -2988,18 +2990,18 @@ checksum = "a3f0bf26fd526d2a95683cd0f87bf103b8539e2ca1ef48ce002d67aad59aa0b4" [[package]] name = "serde" -version = "1.0.188" +version = "1.0.190" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cf9e0fcba69a370eed61bcf2b728575f726b50b55cba78064753d708ddc7549e" +checksum = "91d3c334ca1ee894a2c6f6ad698fe8c435b76d504b13d436f0685d648d6d96f7" dependencies = [ "serde_derive", ] [[package]] name = "serde_derive" -version = "1.0.188" +version = "1.0.190" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4eca7ac642d82aa35b60049a6eccb4be6be75e599bd2e9adb5f875a737654af2" +checksum = "67c5609f394e5c2bd7fc51efda478004ea80ef42fee983d5c67a65e34f32c0e3" dependencies = [ "proc-macro2", "quote", @@ -3008,9 +3010,9 @@ dependencies = [ [[package]] name = "serde_json" -version = "1.0.107" +version = "1.0.108" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6b420ce6e3d8bd882e9b243c6eed35dbc9a6110c9769e74b584e0d68d1f20c65" +checksum = "3d1c7e3eac408d115102c4c24ad393e0821bb3a5df4d506a80f85f7a742a526b" dependencies = [ "itoa", "ryu", @@ -3091,9 +3093,9 @@ checksum = "5e9f0ab6ef7eb7353d9119c170a436d1bf248eea575ac42d19d12f4e34130831" [[package]] name = "socket2" -version = "0.4.9" +version = "0.4.10" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "64a4a911eed85daf18834cfaa86a79b7d266ff93ff5ba14005426219480ed662" +checksum = "9f7916fc008ca5542385b89a3d3ce689953c143e9304a9bf8beec1de48994c0d" dependencies = [ "libc", "winapi", @@ -3101,9 +3103,9 @@ dependencies = [ [[package]] name = "socket2" -version = "0.5.4" +version = "0.5.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4031e820eb552adee9295814c0ced9e5cf38ddf1e8b7d566d6de8e2538ea989e" +checksum = "7b5fac59a5cb5dd637972e5fca70daf0523c9067fcdc4842f053dae04a18f8e9" dependencies = [ "libc", "windows-sys", @@ -3123,9 +3125,9 @@ checksum = "6980e8d7511241f8acf4aebddbb1ff938df5eebe98691418c4468d0b72a96a67" [[package]] name = "sqlparser" -version = "0.38.0" +version = "0.39.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0272b7bb0a225320170c99901b4b5fb3a4384e255a7f2cc228f61e2ba3893e75" +checksum = "743b4dc2cbde11890ccb254a8fc9d537fa41b36da00de2a1c5e9848c9bc42bd7" dependencies = [ "log", "sqlparser_derive", @@ -3160,39 +3162,20 @@ version = "0.10.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "73473c0e59e6d5812c5dfe2a064a6444949f089e20eec9a2e5506596494e4623" -[[package]] -name = "strum" -version = "0.24.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "063e6045c0e62079840579a7e47a355ae92f60eb74daaf156fb1e84ba164e63f" - [[package]] name = "strum" version = "0.25.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "290d54ea6f91c969195bdbcd7442c8c2a2ba87da8bf60a7ee86a235d4bc1e125" dependencies = [ - "strum_macros 0.25.2", + "strum_macros", ] [[package]] name = "strum_macros" -version = "0.24.3" +version = "0.25.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1e385be0d24f186b4ce2f9982191e7101bb737312ad61c1f2f984f34bcf85d59" -dependencies = [ - "heck", - "proc-macro2", - "quote", - "rustversion", - "syn 1.0.109", -] - -[[package]] -name = "strum_macros" -version = "0.25.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ad8d03b598d3d0fff69bf533ee3ef19b8eeb342729596df84bcc7e1f96ec4059" +checksum = "23dc1fa9ac9c169a78ba62f0b841814b7abae11bdd047b9c58f893439e309ea0" dependencies = [ "heck", "proc-macro2", @@ -3252,13 +3235,13 @@ dependencies = [ [[package]] name = "tempfile" -version = "3.8.0" +version = "3.8.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cb94d2f3cc536af71caac6b6fcebf65860b347e7ce0cc9ebe8f70d3e521054ef" +checksum = "7ef1adac450ad7f4b3c28589471ade84f25f731a7a0fe30d71dfa9f60fd808e5" dependencies = [ "cfg-if", "fastrand 2.0.1", - "redox_syscall 0.3.5", + "redox_syscall 0.4.1", "rustix", "windows-sys", ] @@ -3286,18 +3269,18 @@ checksum = "222a222a5bfe1bba4a77b45ec488a741b3cb8872e5e499451fd7d0129c9c7c3d" [[package]] name = "thiserror" -version = "1.0.49" +version = "1.0.50" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1177e8c6d7ede7afde3585fd2513e611227efd6481bd78d2e82ba1ce16557ed4" +checksum = "f9a7210f5c9a7156bb50aa36aed4c95afb51df0df00713949448cf9e97d382d2" dependencies = [ "thiserror-impl", ] [[package]] name = "thiserror-impl" -version = "1.0.49" +version = "1.0.50" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "10712f02019e9288794769fba95cd6847df9874d49d871d062172f9dd41bc4cc" +checksum = "266b2e40bc00e5a6c09c3584011e08b06f123c00362c92b975ba9843aaaa14b8" dependencies = [ "proc-macro2", "quote", @@ -3317,11 +3300,12 @@ dependencies = [ [[package]] name = "time" -version = "0.3.29" +version = "0.3.30" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "426f806f4089c493dcac0d24c29c01e2c38baf8e30f1b716ee37e83d200b18fe" +checksum = "c4a34ab300f2dee6e562c10a046fc05e358b29f9bf92277f30c3c8d82275f6f5" dependencies = [ "deranged", + "powerfmt", "serde", "time-core", "time-macros", @@ -3379,7 +3363,7 @@ dependencies = [ "num_cpus", "parking_lot", "pin-project-lite", - "socket2 0.5.4", + "socket2 0.5.5", "tokio-macros", "windows-sys", ] @@ -3412,7 +3396,7 @@ version = "0.24.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "c28327cf380ac148141087fbfb9de9d7bd4e84ab5d2c28fbc911d753de8a7081" dependencies = [ - "rustls 0.21.7", + "rustls 0.21.8", "tokio", ] @@ -3429,9 +3413,9 @@ dependencies = [ [[package]] name = "tokio-util" -version = "0.7.9" +version = "0.7.10" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1d68074620f57a0b21594d9735eb2e98ab38b17f80d3fcb189fca266771ca60d" +checksum = "5419f34732d9eb6ee4c3578b7989078579b7f039cbbb9ca2c4da015749371e15" dependencies = [ "bytes", "futures-core", @@ -3471,11 +3455,10 @@ checksum = "b6bc1c9ce2b5135ac7f93c72918fc37feb872bdc6a5533a8b85eb4b86bfdae52" [[package]] name = "tracing" -version = "0.1.37" +version = "0.1.40" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8ce8c33a8d48bd45d624a6e523445fd21ec13d3653cd51f681abf67418f54eb8" +checksum = "c3523ab5a71916ccf420eebdf5521fcef02141234bbc0b8a49f2fdc4544364ef" dependencies = [ - "cfg-if", "log", "pin-project-lite", "tracing-attributes", @@ -3484,9 +3467,9 @@ dependencies = [ [[package]] name = "tracing-attributes" -version = "0.1.26" +version = "0.1.27" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5f4f31f56159e98206da9efd823404b79b6ef3143b4a7ab76e67b1751b25a4ab" +checksum = "34704c8d6ebcbc939824180af020566b01a7c01f80641264eba0999f6c2b6be7" dependencies = [ "proc-macro2", "quote", @@ -3495,9 +3478,9 @@ dependencies = [ [[package]] name = "tracing-core" -version = "0.1.31" +version = "0.1.32" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0955b8137a1df6f1a2e9a37d8a6656291ff0297c1a97c24e0d8425fe2312f79a" +checksum = "c06d3da6113f116aaee68e4d601191614c9053067f9ab7f6edbcb161237daa54" dependencies = [ "once_cell", ] @@ -3614,9 +3597,9 @@ checksum = "711b9620af191e0cdc7468a8d14e709c3dcdb115b36f838e601583af800a370a" [[package]] name = "uuid" -version = "1.4.1" +version = "1.5.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "79daa5ed5740825c40b389c5e50312b9c86df53fccd33f281df655642b43869d" +checksum = "88ad59a7560b41a70d191093a945f0b87bc1deeda46fb237479708a1d6b6cdfc" dependencies = [ "getrandom", "serde", @@ -3763,7 +3746,7 @@ version = "0.22.4" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "ed63aea5ce73d0ff405984102c42de94fc55a6b75765d621c65262469b3c9b53" dependencies = [ - "ring 0.17.3", + "ring 0.17.5", "untrusted 0.9.0", ] @@ -3805,10 +3788,10 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "712e227841d057c1ee1cd2fb22fa7e5a5461ae8e48fa2ca79ec42cfc1931183f" [[package]] -name = "windows" -version = "0.48.0" +name = "windows-core" +version = "0.51.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e686886bc078bc1b0b600cac0147aadb815089b6e4da64016cbd754b6342700f" +checksum = "f1f8cf84f35d2db49a46868f947758c7a1138116f7fac3bc844f43ade1292e64" dependencies = [ "windows-targets", ] @@ -3904,6 +3887,26 @@ dependencies = [ "lzma-sys", ] +[[package]] +name = "zerocopy" +version = "0.7.20" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "dd66a62464e3ffd4e37bd09950c2b9dd6c4f8767380fabba0d523f9a775bc85a" +dependencies = [ + "zerocopy-derive", +] + +[[package]] +name = "zerocopy-derive" +version = "0.7.20" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "255c4596d41e6916ced49cfafea18727b24d67878fa180ddfd69b9df34fd1726" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.38", +] + [[package]] name = "zeroize" version = "1.6.0" diff --git a/datafusion/sql/src/expr/function.rs b/datafusion/sql/src/expr/function.rs index 373388277351..8af5fef84ac1 100644 --- a/datafusion/sql/src/expr/function.rs +++ b/datafusion/sql/src/expr/function.rs @@ -146,9 +146,14 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { let order_by = (!order_by.is_empty()).then_some(order_by); let args = self.function_args_to_expr(function.args, schema, planner_context)?; + let filter: Option> = function + .filter + .map(|e| self.sql_expr_to_logical_expr(*e, schema, planner_context)) + .transpose()? + .map(Box::new); return Ok(Expr::AggregateFunction(expr::AggregateFunction::new( - fun, args, distinct, None, order_by, + fun, args, distinct, filter, order_by, ))); }; diff --git a/datafusion/sql/src/expr/mod.rs b/datafusion/sql/src/expr/mod.rs index 26184834874a..1cf0fc133f04 100644 --- a/datafusion/sql/src/expr/mod.rs +++ b/datafusion/sql/src/expr/mod.rs @@ -222,7 +222,9 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { planner_context, ), - SQLExpr::Cast { expr, data_type } => Ok(Expr::Cast(Cast::new( + SQLExpr::Cast { + expr, data_type, .. + } => Ok(Expr::Cast(Cast::new( Box::new(self.sql_expr_to_logical_expr( *expr, schema, @@ -231,7 +233,9 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { self.convert_data_type(&data_type)?, ))), - SQLExpr::TryCast { expr, data_type } => Ok(Expr::TryCast(TryCast::new( + SQLExpr::TryCast { + expr, data_type, .. + } => Ok(Expr::TryCast(TryCast::new( Box::new(self.sql_expr_to_logical_expr( *expr, schema, @@ -412,6 +416,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { expr, trim_where, trim_what, + .. } => self.sql_trim_to_expr( *expr, trim_where, @@ -477,10 +482,36 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { self.parse_array_agg(array_agg, schema, planner_context) } + SQLExpr::Struct { values, fields } => { + self.parse_struct(values, fields, schema, planner_context) + } + _ => not_impl_err!("Unsupported ast node in sqltorel: {sql:?}"), } } + fn parse_struct( + &self, + values: Vec, + fields: Vec, + input_schema: &DFSchema, + planner_context: &mut PlannerContext, + ) -> Result { + if !fields.is_empty() { + return not_impl_err!("Struct fields are not supported yet"); + } + let args = values + .into_iter() + .map(|value| { + self.sql_expr_to_logical_expr(value, input_schema, planner_context) + }) + .collect::>>()?; + Ok(Expr::ScalarFunction(ScalarFunction::new( + BuiltinScalarFunction::Struct, + args, + ))) + } + fn parse_array_agg( &self, array_agg: ArrayAgg, diff --git a/datafusion/sql/src/planner.rs b/datafusion/sql/src/planner.rs index f7d8307d33a0..ca5e260aee05 100644 --- a/datafusion/sql/src/planner.rs +++ b/datafusion/sql/src/planner.rs @@ -24,8 +24,8 @@ use arrow_schema::*; use datafusion_common::field_not_found; use datafusion_common::internal_err; use datafusion_expr::WindowUDF; -use sqlparser::ast::ExactNumberInfo; use sqlparser::ast::TimezoneInfo; +use sqlparser::ast::{ArrayElemTypeDef, ExactNumberInfo}; use sqlparser::ast::{ColumnDef as SQLColumnDef, ColumnOption}; use sqlparser::ast::{DataType as SQLDataType, Ident, ObjectName, TableAlias}; @@ -297,14 +297,15 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { pub(crate) fn convert_data_type(&self, sql_type: &SQLDataType) -> Result { match sql_type { - SQLDataType::Array(Some(inner_sql_type)) => { + SQLDataType::Array(ArrayElemTypeDef::AngleBracket(inner_sql_type)) + | SQLDataType::Array(ArrayElemTypeDef::SquareBracket(inner_sql_type)) => { let data_type = self.convert_simple_data_type(inner_sql_type)?; Ok(DataType::List(Arc::new(Field::new( "field", data_type, true, )))) } - SQLDataType::Array(None) => { + SQLDataType::Array(ArrayElemTypeDef::None) => { not_impl_err!("Arrays with unspecified type is not supported") } other => self.convert_simple_data_type(other), @@ -330,7 +331,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { SQLDataType::Char(_) | SQLDataType::Varchar(_) | SQLDataType::Text - | SQLDataType::String => Ok(DataType::Utf8), + | SQLDataType::String(_) => Ok(DataType::Utf8), SQLDataType::Timestamp(None, tz_info) => { let tz = if matches!(tz_info, TimezoneInfo::Tz) || matches!(tz_info, TimezoneInfo::WithTimeZone) @@ -400,7 +401,12 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { | SQLDataType::Dec(_) | SQLDataType::BigNumeric(_) | SQLDataType::BigDecimal(_) - | SQLDataType::Clob(_) => not_impl_err!( + | SQLDataType::Clob(_) + | SQLDataType::Bytes(_) + | SQLDataType::Int64 + | SQLDataType::Float64 + | SQLDataType::Struct(_) + => not_impl_err!( "Unsupported SQL type {sql_type:?}" ), } diff --git a/datafusion/sql/src/set_expr.rs b/datafusion/sql/src/set_expr.rs index e771a5ba3de4..7300d49be0f5 100644 --- a/datafusion/sql/src/set_expr.rs +++ b/datafusion/sql/src/set_expr.rs @@ -44,6 +44,9 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { SetQuantifier::AllByName => { return not_impl_err!("UNION ALL BY NAME not implemented") } + SetQuantifier::DistinctByName => { + return not_impl_err!("UNION DISTINCT BY NAME not implemented") + } }; let left_plan = self.set_expr_to_plan(*left, planner_context)?; diff --git a/datafusion/sql/src/statement.rs b/datafusion/sql/src/statement.rs index 80a27db6e63d..834bddbf20a3 100644 --- a/datafusion/sql/src/statement.rs +++ b/datafusion/sql/src/statement.rs @@ -430,6 +430,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { table, on, returning, + ignore, } => { if or.is_some() { plan_err!("Inserts with or clauses not supported")?; @@ -449,6 +450,9 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { if returning.is_some() { plan_err!("Insert-returning clause not supported")?; } + if ignore { + plan_err!("Insert-ignore clause not supported")?; + } let _ = into; // optional keyword doesn't change behavior self.insert_to_plan(table_name, columns, source, overwrite) } @@ -471,6 +475,8 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { selection, returning, from, + order_by, + limit, } => { if !tables.is_empty() { plan_err!("DELETE not supported")?; @@ -483,6 +489,15 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { if returning.is_some() { plan_err!("Delete-returning clause not yet supported")?; } + + if !order_by.is_empty() { + plan_err!("Delete-order-by clause not yet supported")?; + } + + if limit.is_some() { + plan_err!("Delete-limit clause not yet supported")?; + } + let table_name = self.get_delete_target(from)?; self.delete_to_plan(table_name, selection) } From 27e64ae496b12f6e702f21b7e48f6ff8dd01c825 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Tue, 31 Oct 2023 10:12:42 -0400 Subject: [PATCH 21/43] Fix panic in multiple distinct aggregates by fixing `ScalarValue::new_list` (#7989) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit * Fix panic in multiple distinct aggregates by fixing ScalarValue::new_list * Update datafusion/common/src/scalar.rs Co-authored-by: DaniĆ«l Heres --------- Co-authored-by: DaniĆ«l Heres --- Cargo.toml | 1 + datafusion/common/src/scalar.rs | 204 +----------------- .../sqllogictest/test_files/aggregate.slt | 35 ++- 3 files changed, 38 insertions(+), 202 deletions(-) diff --git a/Cargo.toml b/Cargo.toml index 5aa830ddff0f..22d5f2f64464 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -75,3 +75,4 @@ opt-level = 3 overflow-checks = false panic = 'unwind' rpath = false + diff --git a/datafusion/common/src/scalar.rs b/datafusion/common/src/scalar.rs index f9b0cbdf22de..b3c11740abf9 100644 --- a/datafusion/common/src/scalar.rs +++ b/datafusion/common/src/scalar.rs @@ -600,117 +600,6 @@ macro_rules! typed_cast { }}; } -macro_rules! build_timestamp_list { - ($TIME_UNIT:expr, $TIME_ZONE:expr, $VALUES:expr, $SIZE:expr) => {{ - match $VALUES { - // the return on the macro is necessary, to short-circuit and return ArrayRef - None => { - return new_null_array( - &DataType::List(Arc::new(Field::new( - "item", - DataType::Timestamp($TIME_UNIT, $TIME_ZONE), - true, - ))), - $SIZE, - ) - } - Some(values) => match $TIME_UNIT { - TimeUnit::Second => { - build_values_list_tz!( - TimestampSecondBuilder, - TimestampSecond, - values, - $SIZE, - $TIME_ZONE - ) - } - TimeUnit::Millisecond => build_values_list_tz!( - TimestampMillisecondBuilder, - TimestampMillisecond, - values, - $SIZE, - $TIME_ZONE - ), - TimeUnit::Microsecond => build_values_list_tz!( - TimestampMicrosecondBuilder, - TimestampMicrosecond, - values, - $SIZE, - $TIME_ZONE - ), - TimeUnit::Nanosecond => build_values_list_tz!( - TimestampNanosecondBuilder, - TimestampNanosecond, - values, - $SIZE, - $TIME_ZONE - ), - }, - } - }}; -} - -macro_rules! new_builder { - (StringBuilder, $len:expr) => { - StringBuilder::new() - }; - (LargeStringBuilder, $len:expr) => { - LargeStringBuilder::new() - }; - ($el:ident, $len:expr) => {{ - <$el>::with_capacity($len) - }}; -} - -macro_rules! build_values_list { - ($VALUE_BUILDER_TY:ident, $SCALAR_TY:ident, $VALUES:expr, $SIZE:expr) => {{ - let builder = new_builder!($VALUE_BUILDER_TY, $VALUES.len()); - let mut builder = ListBuilder::new(builder); - - for _ in 0..$SIZE { - for scalar_value in $VALUES { - match scalar_value { - ScalarValue::$SCALAR_TY(Some(v)) => { - builder.values().append_value(v.clone()); - } - ScalarValue::$SCALAR_TY(None) => { - builder.values().append_null(); - } - _ => panic!("Incompatible ScalarValue for list"), - }; - } - builder.append(true); - } - - builder.finish() - }}; -} - -macro_rules! build_values_list_tz { - ($VALUE_BUILDER_TY:ident, $SCALAR_TY:ident, $VALUES:expr, $SIZE:expr, $TIME_ZONE:expr) => {{ - let mut builder = ListBuilder::new( - $VALUE_BUILDER_TY::with_capacity($VALUES.len()).with_timezone_opt($TIME_ZONE), - ); - - for _ in 0..$SIZE { - for scalar_value in $VALUES { - match scalar_value { - ScalarValue::$SCALAR_TY(Some(v), _) => { - builder.values().append_value(v.clone()); - } - ScalarValue::$SCALAR_TY(None, _) => { - builder.values().append_null(); - } - _ => panic!("Incompatible ScalarValue for list"), - }; - } - builder.append(true); - } - - builder.finish() - }}; -} - macro_rules! build_array_from_option { ($DATA_TYPE:ident, $ARRAY_TYPE:ident, $EXPR:expr, $SIZE:expr) => {{ match $EXPR { @@ -1198,7 +1087,8 @@ impl ScalarValue { } /// Converts an iterator of references [`ScalarValue`] into an [`ArrayRef`] - /// corresponding to those values. For example, + /// corresponding to those values. For example, an iterator of + /// [`ScalarValue::Int32`] would be converted to an [`Int32Array`]. /// /// Returns an error if the iterator is empty or if the /// [`ScalarValue`]s are not all the same type @@ -1657,41 +1547,6 @@ impl ScalarValue { Ok(array) } - /// This function does not contains nulls but empty array instead. - fn iter_to_array_list_without_nulls( - values: &[ScalarValue], - data_type: &DataType, - ) -> Result> { - let mut elements: Vec = vec![]; - let mut offsets = vec![]; - - if values.is_empty() { - offsets.push(0); - } else { - let arr = ScalarValue::iter_to_array(values.to_vec())?; - offsets.push(arr.len()); - elements.push(arr); - } - - // Concatenate element arrays to create single flat array - let flat_array = if elements.is_empty() { - new_empty_array(data_type) - } else { - let element_arrays: Vec<&dyn Array> = - elements.iter().map(|a| a.as_ref()).collect(); - arrow::compute::concat(&element_arrays)? - }; - - let list_array = ListArray::new( - Arc::new(Field::new("item", flat_array.data_type().to_owned(), true)), - OffsetBuffer::::from_lengths(offsets), - flat_array, - None, - ); - - Ok(list_array) - } - /// This function build with nulls with nulls buffer. fn iter_to_array_list( scalars: impl IntoIterator, @@ -1780,7 +1635,8 @@ impl ScalarValue { .unwrap() } - /// Converts `Vec` to ListArray, simplified version of ScalarValue::to_array + /// Converts `Vec` where each element has type corresponding to + /// `data_type`, to a [`ListArray`]. /// /// Example /// ``` @@ -1806,52 +1662,12 @@ impl ScalarValue { /// assert_eq!(result, &expected); /// ``` pub fn new_list(values: &[ScalarValue], data_type: &DataType) -> ArrayRef { - Arc::new(match data_type { - DataType::Boolean => build_values_list!(BooleanBuilder, Boolean, values, 1), - DataType::Int8 => build_values_list!(Int8Builder, Int8, values, 1), - DataType::Int16 => build_values_list!(Int16Builder, Int16, values, 1), - DataType::Int32 => build_values_list!(Int32Builder, Int32, values, 1), - DataType::Int64 => build_values_list!(Int64Builder, Int64, values, 1), - DataType::UInt8 => build_values_list!(UInt8Builder, UInt8, values, 1), - DataType::UInt16 => build_values_list!(UInt16Builder, UInt16, values, 1), - DataType::UInt32 => build_values_list!(UInt32Builder, UInt32, values, 1), - DataType::UInt64 => build_values_list!(UInt64Builder, UInt64, values, 1), - DataType::Utf8 => build_values_list!(StringBuilder, Utf8, values, 1), - DataType::LargeUtf8 => { - build_values_list!(LargeStringBuilder, LargeUtf8, values, 1) - } - DataType::Float32 => build_values_list!(Float32Builder, Float32, values, 1), - DataType::Float64 => build_values_list!(Float64Builder, Float64, values, 1), - DataType::Timestamp(unit, tz) => { - let values = Some(values); - build_timestamp_list!(unit.clone(), tz.clone(), values, 1) - } - DataType::List(_) | DataType::Struct(_) => { - ScalarValue::iter_to_array_list_without_nulls(values, data_type).unwrap() - } - DataType::Decimal128(precision, scale) => { - let mut vals = vec![]; - for value in values.iter() { - if let ScalarValue::Decimal128(v, _, _) = value { - vals.push(v.to_owned()) - } - } - - let arr = Decimal128Array::from(vals) - .with_precision_and_scale(*precision, *scale) - .unwrap(); - wrap_into_list_array(Arc::new(arr)) - } - - DataType::Null => { - let arr = new_null_array(&DataType::Null, values.len()); - wrap_into_list_array(arr) - } - _ => panic!( - "Unsupported data type {:?} for ScalarValue::list_to_array", - data_type - ), - }) + let values = if values.is_empty() { + new_empty_array(data_type) + } else { + Self::iter_to_array(values.iter().cloned()).unwrap() + }; + Arc::new(wrap_into_list_array(values)) } /// Converts a scalar value into an array of `size` rows. diff --git a/datafusion/sqllogictest/test_files/aggregate.slt b/datafusion/sqllogictest/test_files/aggregate.slt index 777b634e93b1..6217f12279a9 100644 --- a/datafusion/sqllogictest/test_files/aggregate.slt +++ b/datafusion/sqllogictest/test_files/aggregate.slt @@ -2020,14 +2020,6 @@ statement ok drop table t; - - -statement error DataFusion error: Execution error: Table 't_source' doesn't exist\. -drop table t_source; - -statement error DataFusion error: Execution error: Table 't' doesn't exist\. -drop table t; - query I select median(a) from (select 1 as a where 1=0); ---- @@ -2199,6 +2191,26 @@ NULL 1 10.1 10.1 10.1 10.1 0 NULL statement ok set datafusion.sql_parser.dialect = 'Generic'; +## Multiple distinct aggregates and dictionaries +statement ok +create table dict_test as values (1, arrow_cast('foo', 'Dictionary(Int32, Utf8)')), (2, arrow_cast('bar', 'Dictionary(Int32, Utf8)')); + +query I? +select * from dict_test; +---- +1 foo +2 bar + +query II +select count(distinct column1), count(distinct column2) from dict_test group by column1; +---- +1 1 +1 1 + +statement ok +drop table dict_test; + + # Prepare the table with dictionary values for testing statement ok CREATE TABLE value(x bigint) AS VALUES (1), (2), (3), (1), (3), (4), (5), (2); @@ -2282,6 +2294,13 @@ select max(x_dict) from value_dict group by x_dict % 2 order by max(x_dict); 4 5 +statement ok +drop table value + +statement ok +drop table value_dict + + # bool aggregation statement ok CREATE TABLE value_bool(x boolean, g int) AS VALUES (NULL, 0), (false, 0), (true, 0), (false, 1), (true, 2), (NULL, 3); From 747cb502c47dec4c78f03f0d0ef2aabb447f8cfa Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Marko=20Milenkovi=C4=87?= Date: Tue, 31 Oct 2023 14:25:09 +0000 Subject: [PATCH 22/43] MemoryReservation exposes MemoryConsumer (#8000) ... as a getter method. --- datafusion/execution/src/memory_pool/mod.rs | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/datafusion/execution/src/memory_pool/mod.rs b/datafusion/execution/src/memory_pool/mod.rs index bbd8f4be4f1c..71483e200af6 100644 --- a/datafusion/execution/src/memory_pool/mod.rs +++ b/datafusion/execution/src/memory_pool/mod.rs @@ -157,6 +157,11 @@ impl MemoryReservation { self.size } + /// Returns [MemoryConsumer] for this [MemoryReservation] + pub fn consumer(&self) -> &MemoryConsumer { + &self.registration.consumer + } + /// Frees all bytes from this reservation back to the underlying /// pool, returning the number of bytes freed. pub fn free(&mut self) -> usize { From 656c6a93fadcec7bc43a8a881dfaf55388b0b5c6 Mon Sep 17 00:00:00 2001 From: Jonah Gao Date: Tue, 31 Oct 2023 22:28:19 +0800 Subject: [PATCH 23/43] fix: generate logical plan for `UPDATE SET FROM` statement (#7984) --- datafusion/sql/src/statement.rs | 68 ++++++++++--------- datafusion/sqllogictest/test_files/update.slt | 36 ++++++++++ 2 files changed, 72 insertions(+), 32 deletions(-) diff --git a/datafusion/sql/src/statement.rs b/datafusion/sql/src/statement.rs index 834bddbf20a3..9d9c55361a5e 100644 --- a/datafusion/sql/src/statement.rs +++ b/datafusion/sql/src/statement.rs @@ -978,10 +978,9 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { // Do a table lookup to verify the table exists let table_name = self.object_name_to_table_reference(table_name)?; let table_source = self.context_provider.get_table_source(table_name.clone())?; - let arrow_schema = (*table_source.schema()).clone(); let table_schema = Arc::new(DFSchema::try_from_qualified_schema( table_name.clone(), - &arrow_schema, + &table_source.schema(), )?); // Overwrite with assignment expressions @@ -1000,21 +999,10 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { }) .collect::>>()?; - let values_and_types = table_schema - .fields() - .iter() - .map(|f| { - let col_name = f.name(); - let val = assign_map.remove(col_name).unwrap_or_else(|| { - ast::Expr::Identifier(ast::Ident::from(col_name.as_str())) - }); - (col_name, val, f.data_type()) - }) - .collect::>(); - - // Build scan - let from = from.unwrap_or(table); - let scan = self.plan_from_tables(vec![from], &mut planner_context)?; + // Build scan, join with from table if it exists. + let mut input_tables = vec![table]; + input_tables.extend(from); + let scan = self.plan_from_tables(input_tables, &mut planner_context)?; // Filter let source = match predicate_expr { @@ -1022,33 +1010,49 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { Some(predicate_expr) => { let filter_expr = self.sql_to_expr( predicate_expr, - &table_schema, + scan.schema(), &mut planner_context, )?; let mut using_columns = HashSet::new(); expr_to_columns(&filter_expr, &mut using_columns)?; let filter_expr = normalize_col_with_schemas_and_ambiguity_check( filter_expr, - &[&[&table_schema]], + &[&[&scan.schema()]], &[using_columns], )?; LogicalPlan::Filter(Filter::try_new(filter_expr, Arc::new(scan))?) } }; - // Projection - let mut exprs = vec![]; - for (col_name, expr, dt) in values_and_types.into_iter() { - let mut expr = self.sql_to_expr(expr, &table_schema, &mut planner_context)?; - // Update placeholder's datatype to the type of the target column - if let datafusion_expr::Expr::Placeholder(placeholder) = &mut expr { - placeholder.data_type = - placeholder.data_type.take().or_else(|| Some(dt.clone())); - } - // Cast to target column type, if necessary - let expr = expr.cast_to(dt, source.schema())?.alias(col_name); - exprs.push(expr); - } + // Build updated values for each column, using the previous value if not modified + let exprs = table_schema + .fields() + .iter() + .map(|field| { + let expr = match assign_map.remove(field.name()) { + Some(new_value) => { + let mut expr = self.sql_to_expr( + new_value, + source.schema(), + &mut planner_context, + )?; + // Update placeholder's datatype to the type of the target column + if let datafusion_expr::Expr::Placeholder(placeholder) = &mut expr + { + placeholder.data_type = placeholder + .data_type + .take() + .or_else(|| Some(field.data_type().clone())); + } + // Cast to target column type, if necessary + expr.cast_to(field.data_type(), source.schema())? + } + None => datafusion_expr::Expr::Column(field.qualified_column()), + }; + Ok(expr.alias(field.name())) + }) + .collect::>>()?; + let source = project(source, exprs)?; let plan = LogicalPlan::Dml(DmlStatement { diff --git a/datafusion/sqllogictest/test_files/update.slt b/datafusion/sqllogictest/test_files/update.slt index 4542a262390c..cb8c6a4fac28 100644 --- a/datafusion/sqllogictest/test_files/update.slt +++ b/datafusion/sqllogictest/test_files/update.slt @@ -41,3 +41,39 @@ logical_plan Dml: op=[Update] table=[t1] --Projection: CAST(t1.c + CAST(Int64(1) AS Float64) AS Int32) AS a, CAST(t1.a AS Utf8) AS b, t1.c + Float64(1) AS c, CAST(t1.b AS Int32) AS d ----TableScan: t1 + +statement ok +create table t2(a int, b varchar, c double, d int); + +## set from subquery +query TT +explain update t1 set b = (select max(b) from t2 where t1.a = t2.a) +---- +logical_plan +Dml: op=[Update] table=[t1] +--Projection: t1.a AS a, () AS b, t1.c AS c, t1.d AS d +----Subquery: +------Projection: MAX(t2.b) +--------Aggregate: groupBy=[[]], aggr=[[MAX(t2.b)]] +----------Filter: outer_ref(t1.a) = t2.a +------------TableScan: t2 +----TableScan: t1 + +# set from other table +query TT +explain update t1 set b = t2.b, c = t2.a, d = 1 from t2 where t1.a = t2.a and t1.b > 'foo' and t2.c > 1.0; +---- +logical_plan +Dml: op=[Update] table=[t1] +--Projection: t1.a AS a, t2.b AS b, CAST(t2.a AS Float64) AS c, CAST(Int64(1) AS Int32) AS d +----Filter: t1.a = t2.a AND t1.b > Utf8("foo") AND t2.c > Float64(1) +------CrossJoin: +--------TableScan: t1 +--------TableScan: t2 + +statement ok +create table t3(a int, b varchar, c double, d int); + +# set from mutiple tables, sqlparser only supports from one table +query error DataFusion error: SQL error: ParserError\("Expected end of statement, found: ,"\) +explain update t1 set b = t2.b, c = t3.a, d = 1 from t2, t3 where t1.a = t2.a and t1.a = t3.a; \ No newline at end of file From 318578314676ef09f5a6b3408c7a91414344642a Mon Sep 17 00:00:00 2001 From: jokercurry <982458633@qq.com> Date: Wed, 1 Nov 2023 17:05:37 +0800 Subject: [PATCH 24/43] Create temporary files for reading or writing (#8005) * Create temporary files for reading or writing * nit * addr comment --------- Co-authored-by: zhongjingxiong --- datafusion-examples/examples/dataframe.rs | 36 +++++++++++++---------- 1 file changed, 21 insertions(+), 15 deletions(-) diff --git a/datafusion-examples/examples/dataframe.rs b/datafusion-examples/examples/dataframe.rs index 26fddcd226a9..ea01c53b1c62 100644 --- a/datafusion-examples/examples/dataframe.rs +++ b/datafusion-examples/examples/dataframe.rs @@ -18,7 +18,9 @@ use datafusion::arrow::datatypes::{DataType, Field, Schema}; use datafusion::error::Result; use datafusion::prelude::*; -use std::fs; +use std::fs::File; +use std::io::Write; +use tempfile::tempdir; /// This example demonstrates executing a simple query against an Arrow data source (Parquet) and /// fetching results, using the DataFrame trait @@ -41,12 +43,19 @@ async fn main() -> Result<()> { // print the results df.show().await?; + // create a csv file waiting to be written + let dir = tempdir()?; + let file_path = dir.path().join("example.csv"); + let file = File::create(&file_path)?; + write_csv_file(file); + // Reading CSV file with inferred schema example - let csv_df = example_read_csv_file_with_inferred_schema().await; + let csv_df = + example_read_csv_file_with_inferred_schema(file_path.to_str().unwrap()).await; csv_df.show().await?; // Reading CSV file with defined schema - let csv_df = example_read_csv_file_with_schema().await; + let csv_df = example_read_csv_file_with_schema(file_path.to_str().unwrap()).await; csv_df.show().await?; // Reading PARQUET file and print describe @@ -59,31 +68,28 @@ async fn main() -> Result<()> { } // Function to create an test CSV file -fn create_csv_file(path: String) { +fn write_csv_file(mut file: File) { // Create the data to put into the csv file with headers let content = r#"id,time,vote,unixtime,rating a1,"10 6, 2013",3,1381017600,5.0 a2,"08 9, 2013",2,1376006400,4.5"#; // write the data - fs::write(path, content).expect("Problem with writing file!"); + file.write_all(content.as_ref()) + .expect("Problem with writing file!"); } // Example to read data from a csv file with inferred schema -async fn example_read_csv_file_with_inferred_schema() -> DataFrame { - let path = "example.csv"; - // Create a csv file using the predefined function - create_csv_file(path.to_string()); +async fn example_read_csv_file_with_inferred_schema(file_path: &str) -> DataFrame { // Create a session context let ctx = SessionContext::new(); // Register a lazy DataFrame using the context - ctx.read_csv(path, CsvReadOptions::default()).await.unwrap() + ctx.read_csv(file_path, CsvReadOptions::default()) + .await + .unwrap() } // Example to read csv file with a defined schema for the csv file -async fn example_read_csv_file_with_schema() -> DataFrame { - let path = "example.csv"; - // Create a csv file using the predefined function - create_csv_file(path.to_string()); +async fn example_read_csv_file_with_schema(file_path: &str) -> DataFrame { // Create a session context let ctx = SessionContext::new(); // Define the schema @@ -101,5 +107,5 @@ async fn example_read_csv_file_with_schema() -> DataFrame { ..Default::default() }; // Register a lazy DataFrame by using the context and option provider - ctx.read_csv(path, csv_read_option).await.unwrap() + ctx.read_csv(file_path, csv_read_option).await.unwrap() } From aef95edf9bf1324be8642146c882b8d4c89a3785 Mon Sep 17 00:00:00 2001 From: Weston Pace Date: Wed, 1 Nov 2023 02:05:50 -0700 Subject: [PATCH 25/43] doc: minor fix to SortExec::with_fetch comment (#8011) --- datafusion/physical-plan/src/sorts/sort.rs | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/datafusion/physical-plan/src/sorts/sort.rs b/datafusion/physical-plan/src/sorts/sort.rs index c7d676493f04..08fa2c25d792 100644 --- a/datafusion/physical-plan/src/sorts/sort.rs +++ b/datafusion/physical-plan/src/sorts/sort.rs @@ -735,7 +735,13 @@ impl SortExec { self } - /// Whether this `SortExec` preserves partitioning of the children + /// Modify how many rows to include in the result + /// + /// If None, then all rows will be returned, in sorted order. + /// If Some, then only the top `fetch` rows will be returned. + /// This can reduce the memory pressure required by the sort + /// operation since rows that are not going to be included + /// can be dropped. pub fn with_fetch(mut self, fetch: Option) -> Self { self.fetch = fetch; self From 69ba82f03e3fa76a18a712fa228402f6d9324879 Mon Sep 17 00:00:00 2001 From: jokercurry <982458633@qq.com> Date: Wed, 1 Nov 2023 19:16:50 +0800 Subject: [PATCH 26/43] Fix: dataframe_subquery example Optimizer rule `common_sub_expression_eliminate` failed (#8016) * Fix: Optimizer rule 'common_sub_expression_eliminate' failed * nit * nit * nit --------- Co-authored-by: zhongjingxiong --- datafusion-examples/examples/dataframe_subquery.rs | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/datafusion-examples/examples/dataframe_subquery.rs b/datafusion-examples/examples/dataframe_subquery.rs index 94049e59b3ab..9fb61008b9f6 100644 --- a/datafusion-examples/examples/dataframe_subquery.rs +++ b/datafusion-examples/examples/dataframe_subquery.rs @@ -15,6 +15,7 @@ // specific language governing permissions and limitations // under the License. +use arrow_schema::DataType; use std::sync::Arc; use datafusion::error::Result; @@ -38,7 +39,7 @@ async fn main() -> Result<()> { Ok(()) } -//select c1,c2 from t1 where (select avg(t2.c2) from t2 where t1.c1 = t2.c1)>0 limit 10; +//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? @@ -46,7 +47,7 @@ async fn where_scalar_subquery(ctx: &SessionContext) -> Result<()> { scalar_subquery(Arc::new( ctx.table("t2") .await? - .filter(col("t1.c1").eq(col("t2.c1")))? + .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(), @@ -60,7 +61,7 @@ async fn where_scalar_subquery(ctx: &SessionContext) -> Result<()> { Ok(()) } -//SELECT t1.c1, t1.c2 FROM t1 WHERE t1.c2 in (select max(t2.c2) from t2 where t2.c1 > 0 ) limit 10 +//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? @@ -82,14 +83,14 @@ async fn where_in_subquery(ctx: &SessionContext) -> Result<()> { Ok(()) } -//SELECT t1.c1, t1.c2 FROM t1 WHERE EXISTS (select t2.c2 from t2 where t1.c1 = t2.c1) limit 10 +//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(col("t1.c1").eq(col("t2.c1")))? + .filter(out_ref_col(DataType::Utf8, "t1.c1").eq(col("t2.c1")))? .select(vec![col("t2.c2")])? .into_unoptimized_plan(), )))? From 4e60cddb1caa8f3f552a6f9d4aa6fd3a3653fb58 Mon Sep 17 00:00:00 2001 From: Raphael Taylor-Davies <1781103+tustvold@users.noreply.github.com> Date: Wed, 1 Nov 2023 12:14:52 +0000 Subject: [PATCH 27/43] Percent Decode URL Paths (#8009) (#8012) * Treat ListingTableUrl as URL-encoded (#8009) * Update lockfile * Review feedback --- datafusion-cli/Cargo.lock | 1 - datafusion/core/Cargo.toml | 1 - datafusion/core/src/datasource/listing/url.rs | 48 +++++++++++++++---- 3 files changed, 38 insertions(+), 12 deletions(-) diff --git a/datafusion-cli/Cargo.lock b/datafusion-cli/Cargo.lock index 268f9e28427a..dc828f018fd5 100644 --- a/datafusion-cli/Cargo.lock +++ b/datafusion-cli/Cargo.lock @@ -1130,7 +1130,6 @@ dependencies = [ "object_store", "parking_lot", "parquet", - "percent-encoding", "pin-project-lite", "rand", "sqlparser", diff --git a/datafusion/core/Cargo.toml b/datafusion/core/Cargo.toml index a9614934673b..f1542fca0026 100644 --- a/datafusion/core/Cargo.toml +++ b/datafusion/core/Cargo.toml @@ -82,7 +82,6 @@ num_cpus = "1.13.0" object_store = "0.7.0" parking_lot = "0.12" parquet = { workspace = true, optional = true } -percent-encoding = "2.2.0" pin-project-lite = "^0.2.7" rand = "0.8" sqlparser = { workspace = true } diff --git a/datafusion/core/src/datasource/listing/url.rs b/datafusion/core/src/datasource/listing/url.rs index 4d1ca4853a73..9197e37adbd5 100644 --- a/datafusion/core/src/datasource/listing/url.rs +++ b/datafusion/core/src/datasource/listing/url.rs @@ -27,7 +27,6 @@ use itertools::Itertools; use log::debug; use object_store::path::Path; use object_store::{ObjectMeta, ObjectStore}; -use percent_encoding; use std::sync::Arc; use url::Url; @@ -46,6 +45,16 @@ pub struct ListingTableUrl { impl ListingTableUrl { /// Parse a provided string as a `ListingTableUrl` /// + /// # URL Encoding + /// + /// URL paths are expected to be URL-encoded. That is, the URL for a file named `bar%2Efoo` + /// would be `file:///bar%252Efoo`, as per the [URL] specification. + /// + /// It should be noted that some tools, such as the AWS CLI, take a different approach and + /// instead interpret the URL path verbatim. For example the object `bar%2Efoo` would be + /// addressed as `s3://BUCKET/bar%252Efoo` using [`ListingTableUrl`] but `s3://BUCKET/bar%2Efoo` + /// when using the aws-cli. + /// /// # Paths without a Scheme /// /// If no scheme is provided, or the string is an absolute filesystem path @@ -77,6 +86,7 @@ impl ListingTableUrl { /// filter when listing files from object storage /// /// [file URI]: https://en.wikipedia.org/wiki/File_URI_scheme + /// [URL]: https://url.spec.whatwg.org/ pub fn parse(s: impl AsRef) -> Result { let s = s.as_ref(); @@ -86,7 +96,7 @@ impl ListingTableUrl { } match Url::parse(s) { - Ok(url) => Ok(Self::new(url, None)), + Ok(url) => Self::try_new(url, None), Err(url::ParseError::RelativeUrlWithoutBase) => Self::parse_path(s), Err(e) => Err(DataFusionError::External(Box::new(e))), } @@ -138,15 +148,13 @@ impl ListingTableUrl { .map_err(|_| DataFusionError::Internal(format!("Can not open path: {s}")))?; // TODO: Currently we do not have an IO-related error variant that accepts () // or a string. Once we have such a variant, change the error type above. - Ok(Self::new(url, glob)) + Self::try_new(url, glob) } /// Creates a new [`ListingTableUrl`] from a url and optional glob expression - fn new(url: Url, glob: Option) -> Self { - let decoded_path = - percent_encoding::percent_decode_str(url.path()).decode_utf8_lossy(); - let prefix = Path::from(decoded_path.as_ref()); - Self { url, prefix, glob } + fn try_new(url: Url, glob: Option) -> Result { + let prefix = Path::from_url_path(url.path())?; + Ok(Self { url, prefix, glob }) } /// Returns the URL scheme @@ -286,6 +294,7 @@ fn split_glob_expression(path: &str) -> Option<(&str, &str)> { #[cfg(test)] mod tests { use super::*; + use tempfile::tempdir; #[test] fn test_prefix_path() { @@ -317,8 +326,27 @@ mod tests { let url = ListingTableUrl::parse("file:///foo/bar?").unwrap(); assert_eq!(url.prefix.as_ref(), "foo/bar"); - let url = ListingTableUrl::parse("file:///foo/šŸ˜ŗ").unwrap(); - assert_eq!(url.prefix.as_ref(), "foo/%F0%9F%98%BA"); + let err = ListingTableUrl::parse("file:///foo/šŸ˜ŗ").unwrap_err(); + assert_eq!(err.to_string(), "Object Store error: Encountered object with invalid path: Error parsing Path \"/foo/šŸ˜ŗ\": Encountered illegal character sequence \"šŸ˜ŗ\" whilst parsing path segment \"šŸ˜ŗ\""); + + let url = ListingTableUrl::parse("file:///foo/bar%2Efoo").unwrap(); + assert_eq!(url.prefix.as_ref(), "foo/bar.foo"); + + let url = ListingTableUrl::parse("file:///foo/bar%2Efoo").unwrap(); + assert_eq!(url.prefix.as_ref(), "foo/bar.foo"); + + let url = ListingTableUrl::parse("file:///foo/bar%252Ffoo").unwrap(); + assert_eq!(url.prefix.as_ref(), "foo/bar%2Ffoo"); + + let url = ListingTableUrl::parse("file:///foo/a%252Fb.txt").unwrap(); + assert_eq!(url.prefix.as_ref(), "foo/a%2Fb.txt"); + + let dir = tempdir().unwrap(); + let path = dir.path().join("bar%2Ffoo"); + std::fs::File::create(&path).unwrap(); + + let url = ListingTableUrl::parse(path.to_str().unwrap()).unwrap(); + assert!(url.prefix.as_ref().ends_with("bar%2Ffoo"), "{}", url.prefix); } #[test] From e98625cb0ecab970c9b13e8a4c2a49c37d2505b2 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=BC=A0=E6=9E=97=E4=BC=9F?= Date: Thu, 2 Nov 2023 00:18:23 +0800 Subject: [PATCH 28/43] Minor: Extract common deps into workspace (#7982) * Improve datafusion-* * More common crates * Extract async-trait * Extract more * Fix cli --------- Co-authored-by: Andrew Lamb --- Cargo.toml | 32 ++++++++++++++++++ benchmarks/Cargo.toml | 10 +++--- datafusion-examples/Cargo.toml | 20 +++++------ datafusion/common/Cargo.toml | 2 +- datafusion/core/Cargo.toml | 52 ++++++++++++++--------------- datafusion/execution/Cargo.toml | 20 +++++------ datafusion/expr/Cargo.toml | 6 ++-- datafusion/optimizer/Cargo.toml | 12 +++---- datafusion/physical-expr/Cargo.toml | 14 ++++---- datafusion/physical-plan/Cargo.toml | 22 ++++++------ datafusion/proto/Cargo.toml | 8 ++--- datafusion/sql/Cargo.toml | 10 +++--- datafusion/sqllogictest/Cargo.toml | 22 ++++++------ datafusion/substrait/Cargo.toml | 6 ++-- datafusion/wasmtest/Cargo.toml | 10 +++--- test-utils/Cargo.toml | 2 +- 16 files changed, 140 insertions(+), 108 deletions(-) diff --git a/Cargo.toml b/Cargo.toml index 22d5f2f64464..1a2f4a84af38 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -54,9 +54,41 @@ arrow-array = { version = "48.0.0", default-features = false, features = ["chron arrow-buffer = { version = "48.0.0", default-features = false } arrow-flight = { version = "48.0.0", features = ["flight-sql-experimental"] } arrow-schema = { version = "48.0.0", default-features = false } +async-trait = "0.1.73" +bigdecimal = "0.4.1" +bytes = "1.4" +ctor = "0.2.0" +datafusion = { path = "datafusion/core" } +datafusion-common = { path = "datafusion/common" } +datafusion-expr = { path = "datafusion/expr" } +datafusion-sql = { path = "datafusion/sql" } +datafusion-optimizer = { path = "datafusion/optimizer" } +datafusion-physical-expr = { path = "datafusion/physical-expr" } +datafusion-physical-plan = { path = "datafusion/physical-plan" } +datafusion-execution = { path = "datafusion/execution" } +datafusion-proto = { path = "datafusion/proto" } +datafusion-sqllogictest = { path = "datafusion/sqllogictest" } +datafusion-substrait = { path = "datafusion/substrait" } +dashmap = "5.4.0" +doc-comment = "0.3" +env_logger = "0.10" +futures = "0.3" +half = "2.2.1" +indexmap = "2.0.0" +itertools = "0.11" +log = "^0.4" +num_cpus = "1.13.0" +object_store = "0.7.0" +parking_lot = "0.12" parquet = { version = "48.0.0", features = ["arrow", "async", "object_store"] } +rand = "0.8" +rstest = "0.18.0" +serde_json = "1" sqlparser = { version = "0.39.0", features = ["visitor"] } +tempfile = "3" +thiserror = "1.0.44" chrono = { version = "0.4.31", default-features = false } +url = "2.2" [profile.release] codegen-units = 1 diff --git a/benchmarks/Cargo.toml b/benchmarks/Cargo.toml index 0def335521ce..ce0a4267fc28 100644 --- a/benchmarks/Cargo.toml +++ b/benchmarks/Cargo.toml @@ -36,14 +36,14 @@ snmalloc = ["snmalloc-rs"] arrow = { workspace = true } datafusion = { path = "../datafusion/core", version = "32.0.0" } datafusion-common = { path = "../datafusion/common", version = "32.0.0" } -env_logger = "0.10" -futures = "0.3" -log = "^0.4" +env_logger = { workspace = true } +futures = { workspace = true } +log = { workspace = true } mimalloc = { version = "0.1", optional = true, default-features = false } -num_cpus = "1.13.0" +num_cpus = { workspace = true } parquet = { workspace = true } serde = { version = "1.0.136", features = ["derive"] } -serde_json = "1.0.78" +serde_json = { workspace = true } snmalloc-rs = { version = "0.3", optional = true } structopt = { version = "0.3", default-features = false } test-utils = { path = "../test-utils/", version = "0.1.0" } diff --git a/datafusion-examples/Cargo.toml b/datafusion-examples/Cargo.toml index 7350c4ab981f..57691520a401 100644 --- a/datafusion-examples/Cargo.toml +++ b/datafusion-examples/Cargo.toml @@ -33,26 +33,26 @@ rust-version = { workspace = true } arrow = { workspace = true } arrow-flight = { workspace = true } arrow-schema = { workspace = true } -async-trait = "0.1.41" -bytes = "1.4" -dashmap = "5.4" +async-trait = { workspace = true } +bytes = { workspace = true } +dashmap = { workspace = true } datafusion = { path = "../datafusion/core", features = ["avro"] } datafusion-common = { path = "../datafusion/common" } datafusion-expr = { path = "../datafusion/expr" } datafusion-optimizer = { path = "../datafusion/optimizer" } datafusion-sql = { path = "../datafusion/sql" } -env_logger = "0.10" -futures = "0.3" -log = "0.4" +env_logger = { workspace = true } +futures = { workspace = true } +log = { workspace = true } mimalloc = { version = "0.1", default-features = false } -num_cpus = "1.13.0" +num_cpus = { workspace = true } object_store = { version = "0.7.0", features = ["aws", "http"] } prost = { version = "0.12", default-features = false } prost-derive = { version = "0.11", default-features = false } serde = { version = "1.0.136", features = ["derive"] } -serde_json = "1.0.82" -tempfile = "3" +serde_json = { workspace = true } +tempfile = { workspace = true } tokio = { version = "1.0", features = ["macros", "rt", "rt-multi-thread", "sync", "parking_lot"] } tonic = "0.10" -url = "2.2" +url = { workspace = true } uuid = "1.2" diff --git a/datafusion/common/Cargo.toml b/datafusion/common/Cargo.toml index 87087c50a2d2..d04db86b7830 100644 --- a/datafusion/common/Cargo.toml +++ b/datafusion/common/Cargo.toml @@ -46,7 +46,7 @@ arrow-buffer = { workspace = true } arrow-schema = { workspace = true } chrono = { workspace = true } half = { version = "2.1", default-features = false } -num_cpus = "1.13.0" +num_cpus = { workspace = true } object_store = { version = "0.7.0", default-features = false, optional = true } parquet = { workspace = true, optional = true } pyo3 = { version = "0.20.0", optional = true } diff --git a/datafusion/core/Cargo.toml b/datafusion/core/Cargo.toml index f1542fca0026..4015ba439e67 100644 --- a/datafusion/core/Cargo.toml +++ b/datafusion/core/Cargo.toml @@ -57,61 +57,61 @@ arrow = { workspace = true } arrow-array = { workspace = true } arrow-schema = { workspace = true } async-compression = { version = "0.4.0", features = ["bzip2", "gzip", "xz", "zstd", "futures-io", "tokio"], optional = true } -async-trait = "0.1.73" -bytes = "1.4" +async-trait = { workspace = true } +bytes = { workspace = true } bzip2 = { version = "0.4.3", optional = true } chrono = { workspace = true } -dashmap = "5.4.0" +dashmap = { workspace = true } datafusion-common = { path = "../common", version = "32.0.0", features = ["object_store"], default-features = false } -datafusion-execution = { path = "../execution", version = "32.0.0" } -datafusion-expr = { path = "../expr", version = "32.0.0" } +datafusion-execution = { workspace = true } +datafusion-expr = { workspace = true } datafusion-optimizer = { path = "../optimizer", version = "32.0.0", default-features = false } datafusion-physical-expr = { path = "../physical-expr", version = "32.0.0", default-features = false } -datafusion-physical-plan = { path = "../physical-plan", version = "32.0.0", default-features = false } -datafusion-sql = { path = "../sql", version = "32.0.0" } +datafusion-physical-plan = { workspace = true } +datafusion-sql = { workspace = true } flate2 = { version = "1.0.24", optional = true } -futures = "0.3" +futures = { workspace = true } glob = "0.3.0" half = { version = "2.1", default-features = false } hashbrown = { version = "0.14", features = ["raw"] } -indexmap = "2.0.0" -itertools = "0.11" -log = "^0.4" +indexmap = { workspace = true } +itertools = { workspace = true } +log = { workspace = true } num-traits = { version = "0.2", optional = true } -num_cpus = "1.13.0" -object_store = "0.7.0" -parking_lot = "0.12" +num_cpus = { workspace = true } +object_store = { workspace = true } +parking_lot = { workspace = true } parquet = { workspace = true, optional = true } pin-project-lite = "^0.2.7" -rand = "0.8" +rand = { workspace = true } sqlparser = { workspace = true } -tempfile = "3" +tempfile = { workspace = true } tokio = { version = "1.28", features = ["macros", "rt", "rt-multi-thread", "sync", "fs", "parking_lot"] } tokio-util = { version = "0.7.4", features = ["io"] } -url = "2.2" +url = { workspace = true } uuid = { version = "1.0", features = ["v4"] } xz2 = { version = "0.1", optional = true } zstd = { version = "0.13", optional = true, default-features = false } [dev-dependencies] -async-trait = "0.1.53" -bigdecimal = "0.4.1" +async-trait = { workspace = true } +bigdecimal = { workspace = true } criterion = { version = "0.5", features = ["async_tokio"] } csv = "1.1.6" -ctor = "0.2.0" -doc-comment = "0.3" -env_logger = "0.10" -half = "2.2.1" +ctor = { workspace = true } +doc-comment = { workspace = true } +env_logger = { workspace = true } +half = { workspace = true } postgres-protocol = "0.6.4" postgres-types = { version = "0.2.4", features = ["derive", "with-chrono-0_4"] } rand = { version = "0.8", features = ["small_rng"] } rand_distr = "0.4.3" regex = "1.5.4" -rstest = "0.18.0" +rstest = { workspace = true } rust_decimal = { version = "1.27.0", features = ["tokio-pg"] } -serde_json = "1" +serde_json = { workspace = true } test-utils = { path = "../../test-utils" } -thiserror = "1.0.37" +thiserror = { workspace = true } tokio-postgres = "0.7.7" [target.'cfg(not(target_os = "windows"))'.dev-dependencies] nix = { version = "0.27.1", features = ["fs"] } diff --git a/datafusion/execution/Cargo.toml b/datafusion/execution/Cargo.toml index 54a2a6d8f3df..e9bb87e9f8ac 100644 --- a/datafusion/execution/Cargo.toml +++ b/datafusion/execution/Cargo.toml @@ -35,14 +35,14 @@ path = "src/lib.rs" [dependencies] arrow = { workspace = true } chrono = { version = "0.4", default-features = false } -dashmap = "5.4.0" -datafusion-common = { path = "../common", version = "32.0.0" } -datafusion-expr = { path = "../expr", version = "32.0.0" } -futures = "0.3" +dashmap = { workspace = true } +datafusion-common = { workspace = true } +datafusion-expr = { workspace = true } +futures = { workspace = true } hashbrown = { version = "0.14", features = ["raw"] } -log = "^0.4" -object_store = "0.7.0" -parking_lot = "0.12" -rand = "0.8" -tempfile = "3" -url = "2.2" +log = { workspace = true } +object_store = { workspace = true } +parking_lot = { workspace = true } +rand = { workspace = true } +tempfile = { workspace = true } +url = { workspace = true } diff --git a/datafusion/expr/Cargo.toml b/datafusion/expr/Cargo.toml index a7919a557ad7..5b1b42153877 100644 --- a/datafusion/expr/Cargo.toml +++ b/datafusion/expr/Cargo.toml @@ -38,11 +38,11 @@ path = "src/lib.rs" ahash = { version = "0.8", default-features = false, features = ["runtime-rng"] } arrow = { workspace = true } arrow-array = { workspace = true } -datafusion-common = { path = "../common", version = "32.0.0", default-features = false } +datafusion-common = { workspace = true } sqlparser = { workspace = true } strum = { version = "0.25.0", features = ["derive"] } strum_macros = "0.25.0" [dev-dependencies] -ctor = "0.2.0" -env_logger = "0.10" +ctor = { workspace = true } +env_logger = { workspace = true } diff --git a/datafusion/optimizer/Cargo.toml b/datafusion/optimizer/Cargo.toml index dce05058b826..797dd17a26b5 100644 --- a/datafusion/optimizer/Cargo.toml +++ b/datafusion/optimizer/Cargo.toml @@ -40,17 +40,17 @@ unicode_expressions = ["datafusion-physical-expr/unicode_expressions"] [dependencies] arrow = { workspace = true } -async-trait = "0.1.41" +async-trait = { workspace = true } chrono = { workspace = true } -datafusion-common = { path = "../common", version = "32.0.0", default-features = false } -datafusion-expr = { path = "../expr", version = "32.0.0" } +datafusion-common = { workspace = true } +datafusion-expr = { workspace = true } datafusion-physical-expr = { path = "../physical-expr", version = "32.0.0", default-features = false } hashbrown = { version = "0.14", features = ["raw"] } -itertools = "0.11" -log = "^0.4" +itertools = { workspace = true } +log = { workspace = true } regex-syntax = "0.8.0" [dev-dependencies] -ctor = "0.2.0" +ctor = { workspace = true } datafusion-sql = { path = "../sql", version = "32.0.0" } env_logger = "0.10.0" diff --git a/datafusion/physical-expr/Cargo.toml b/datafusion/physical-expr/Cargo.toml index f7c0221756fd..4be625e384b9 100644 --- a/datafusion/physical-expr/Cargo.toml +++ b/datafusion/physical-expr/Cargo.toml @@ -49,19 +49,19 @@ base64 = { version = "0.21", optional = true } blake2 = { version = "^0.10.2", optional = true } blake3 = { version = "1.0", optional = true } chrono = { workspace = true } -datafusion-common = { path = "../common", version = "32.0.0", default-features = false } -datafusion-expr = { path = "../expr", version = "32.0.0" } +datafusion-common = { workspace = true } +datafusion-expr = { workspace = true } half = { version = "2.1", default-features = false } hashbrown = { version = "0.14", features = ["raw"] } hex = { version = "0.4", optional = true } -indexmap = "2.0.0" +indexmap = { workspace = true } itertools = { version = "0.11", features = ["use_std"] } libc = "0.2.140" -log = "^0.4" +log = { workspace = true } md-5 = { version = "^0.10.0", optional = true } paste = "^1.0" petgraph = "0.6.2" -rand = "0.8" +rand = { workspace = true } regex = { version = "1.8", optional = true } sha2 = { version = "^0.10.1", optional = true } unicode-segmentation = { version = "^1.7.1", optional = true } @@ -69,8 +69,8 @@ uuid = { version = "^1.2", features = ["v4"] } [dev-dependencies] criterion = "0.5" -rand = "0.8" -rstest = "0.18.0" +rand = { workspace = true } +rstest = { workspace = true } [[bench]] harness = false diff --git a/datafusion/physical-plan/Cargo.toml b/datafusion/physical-plan/Cargo.toml index 033d2aa187dd..82c8f49a764f 100644 --- a/datafusion/physical-plan/Cargo.toml +++ b/datafusion/physical-plan/Cargo.toml @@ -38,26 +38,26 @@ arrow = { workspace = true } arrow-array = { workspace = true } arrow-buffer = { workspace = true } arrow-schema = { workspace = true } -async-trait = "0.1.41" +async-trait = { workspace = true } chrono = { version = "0.4.23", default-features = false } -datafusion-common = { path = "../common", version = "32.0.0", default-features = false } -datafusion-execution = { path = "../execution", version = "32.0.0" } -datafusion-expr = { path = "../expr", version = "32.0.0" } -datafusion-physical-expr = { path = "../physical-expr", version = "32.0.0" } -futures = "0.3" +datafusion-common = { workspace = true } +datafusion-execution = { workspace = true } +datafusion-expr = { workspace = true } +datafusion-physical-expr = { workspace = true } +futures = { workspace = true } half = { version = "2.1", default-features = false } hashbrown = { version = "0.14", features = ["raw"] } -indexmap = "2.0.0" +indexmap = { workspace = true } itertools = { version = "0.11", features = ["use_std"] } -log = "^0.4" +log = { workspace = true } once_cell = "1.18.0" -parking_lot = "0.12" +parking_lot = { workspace = true } pin-project-lite = "^0.2.7" -rand = "0.8" +rand = { workspace = true } tokio = { version = "1.28", features = ["sync", "fs", "parking_lot"] } uuid = { version = "^1.2", features = ["v4"] } [dev-dependencies] -rstest = "0.18.0" +rstest = { workspace = true } termtree = "0.4.1" tokio = { version = "1.28", features = ["macros", "rt", "rt-multi-thread", "sync", "fs", "parking_lot"] } diff --git a/datafusion/proto/Cargo.toml b/datafusion/proto/Cargo.toml index 5e52dadc0b18..72a4df66ebd7 100644 --- a/datafusion/proto/Cargo.toml +++ b/datafusion/proto/Cargo.toml @@ -44,14 +44,14 @@ parquet = ["datafusion/parquet", "datafusion-common/parquet"] arrow = { workspace = true } chrono = { workspace = true } datafusion = { path = "../core", version = "32.0.0" } -datafusion-common = { path = "../common", version = "32.0.0", default-features = false } -datafusion-expr = { path = "../expr", version = "32.0.0" } +datafusion-common = { workspace = true } +datafusion-expr = { workspace = true } object_store = { version = "0.7.0" } pbjson = { version = "0.5", optional = true } prost = "0.12.0" serde = { version = "1.0", optional = true } -serde_json = { version = "1.0", optional = true } +serde_json = { workspace = true, optional = true } [dev-dependencies] -doc-comment = "0.3" +doc-comment = { workspace = true } tokio = "1.18" diff --git a/datafusion/sql/Cargo.toml b/datafusion/sql/Cargo.toml index fe60feb6ab1f..b91a2ac1fbd7 100644 --- a/datafusion/sql/Cargo.toml +++ b/datafusion/sql/Cargo.toml @@ -39,13 +39,13 @@ unicode_expressions = [] [dependencies] arrow = { workspace = true } arrow-schema = { workspace = true } -datafusion-common = { path = "../common", version = "32.0.0", default-features = false } -datafusion-expr = { path = "../expr", version = "32.0.0" } -log = "^0.4" +datafusion-common = { workspace = true } +datafusion-expr = { workspace = true } +log = { workspace = true } sqlparser = { workspace = true } [dev-dependencies] -ctor = "0.2.0" -env_logger = "0.10" +ctor = { workspace = true } +env_logger = { workspace = true } paste = "^1.0" rstest = "0.18" diff --git a/datafusion/sqllogictest/Cargo.toml b/datafusion/sqllogictest/Cargo.toml index f1a730351417..07debf179529 100644 --- a/datafusion/sqllogictest/Cargo.toml +++ b/datafusion/sqllogictest/Cargo.toml @@ -32,24 +32,24 @@ path = "src/lib.rs" [dependencies] arrow = { workspace = true } -async-trait = "0.1.41" -bigdecimal = "0.4.1" +async-trait = { workspace = true } +bigdecimal = { workspace = true } bytes = { version = "1.4.0", optional = true } chrono = { workspace = true, optional = true } datafusion = { path = "../core", version = "32.0.0" } -datafusion-common = { path = "../common", version = "32.0.0", default-features = false } +datafusion-common = { workspace = true } futures = { version = "0.3.28" } -half = "2.2.1" -itertools = "0.11" -log = "^0.4" -object_store = "0.7.0" +half = { workspace = true } +itertools = { workspace = true } +log = { workspace = true } +object_store = { workspace = true } postgres-protocol = { version = "0.6.4", optional = true } postgres-types = { version = "0.2.4", optional = true } rust_decimal = { version = "1.27.0" } sqllogictest = "0.17.0" sqlparser = { workspace = true } -tempfile = "3" -thiserror = "1.0.44" +tempfile = { workspace = true } +thiserror = { workspace = true } tokio = { version = "1.0" } tokio-postgres = { version = "0.7.7", optional = true } @@ -58,8 +58,8 @@ avro = ["datafusion/avro"] postgres = ["bytes", "chrono", "tokio-postgres", "postgres-types", "postgres-protocol"] [dev-dependencies] -env_logger = "0.10" -num_cpus = "1.13.0" +env_logger = { workspace = true } +num_cpus = { workspace = true } [[test]] harness = false diff --git a/datafusion/substrait/Cargo.toml b/datafusion/substrait/Cargo.toml index 67d31caca260..585cb6e43d15 100644 --- a/datafusion/substrait/Cargo.toml +++ b/datafusion/substrait/Cargo.toml @@ -30,9 +30,9 @@ rust-version = "1.70" [dependencies] async-recursion = "1.0" chrono = { workspace = true } -datafusion = { version = "32.0.0", path = "../core" } -itertools = "0.11" -object_store = "0.7.0" +datafusion = { workspace = true } +itertools = { workspace = true } +object_store = { workspace = true } prost = "0.12" prost-types = "0.12" substrait = "0.18.0" diff --git a/datafusion/wasmtest/Cargo.toml b/datafusion/wasmtest/Cargo.toml index 3195c989e223..882b02bcc84b 100644 --- a/datafusion/wasmtest/Cargo.toml +++ b/datafusion/wasmtest/Cargo.toml @@ -38,11 +38,11 @@ crate-type = ["cdylib", "rlib",] # code size when deploying. console_error_panic_hook = { version = "0.1.1", optional = true } -datafusion-common = { path = "../common", version = "32.0.0", default-features = false } -datafusion-expr = { path = "../expr" } -datafusion-optimizer = { path = "../optimizer" } -datafusion-physical-expr = { path = "../physical-expr" } -datafusion-sql = { path = "../sql" } +datafusion-common = { workspace = true } +datafusion-expr = { workspace = true } +datafusion-optimizer = { workspace = true } +datafusion-physical-expr = { workspace = true } +datafusion-sql = { workspace = true } # getrandom must be compiled with js feature getrandom = { version = "0.2.8", features = ["js"] } diff --git a/test-utils/Cargo.toml b/test-utils/Cargo.toml index 5ab10e42cf68..b9c4db17c098 100644 --- a/test-utils/Cargo.toml +++ b/test-utils/Cargo.toml @@ -26,4 +26,4 @@ edition = { workspace = true } arrow = { workspace = true } datafusion-common = { path = "../datafusion/common" } env_logger = "0.10.0" -rand = "0.8" +rand = { workspace = true } From 7d1cf9174c7b92af38e732cfafdd67acb243a5c3 Mon Sep 17 00:00:00 2001 From: Ruihang Xia Date: Thu, 2 Nov 2023 00:19:17 +0800 Subject: [PATCH 29/43] minor: change some plan_err to exec_err (#7996) * minor: change some plan_err to exec_err Signed-off-by: Ruihang Xia * change unreachable code to internal error Signed-off-by: Ruihang Xia --------- Signed-off-by: Ruihang Xia --- datafusion/physical-expr/src/expressions/column.rs | 3 +-- datafusion/physical-expr/src/expressions/no_op.rs | 4 ++-- datafusion/physical-expr/src/sort_expr.rs | 4 ++-- 3 files changed, 5 insertions(+), 6 deletions(-) diff --git a/datafusion/physical-expr/src/expressions/column.rs b/datafusion/physical-expr/src/expressions/column.rs index b7b5895db6d3..62da8ff9ed44 100644 --- a/datafusion/physical-expr/src/expressions/column.rs +++ b/datafusion/physical-expr/src/expressions/column.rs @@ -28,7 +28,6 @@ use arrow::{ datatypes::{DataType, Schema}, record_batch::RecordBatch, }; -use datafusion_common::plan_err; use datafusion_common::{internal_err, DataFusionError, Result}; use datafusion_expr::ColumnarValue; @@ -176,7 +175,7 @@ impl PhysicalExpr for UnKnownColumn { /// Evaluate the expression fn evaluate(&self, _batch: &RecordBatch) -> Result { - plan_err!("UnKnownColumn::evaluate() should not be called") + internal_err!("UnKnownColumn::evaluate() should not be called") } fn children(&self) -> Vec> { diff --git a/datafusion/physical-expr/src/expressions/no_op.rs b/datafusion/physical-expr/src/expressions/no_op.rs index 497fb42fe4df..95e6879a6c2d 100644 --- a/datafusion/physical-expr/src/expressions/no_op.rs +++ b/datafusion/physical-expr/src/expressions/no_op.rs @@ -28,7 +28,7 @@ use arrow::{ use crate::physical_expr::down_cast_any_ref; use crate::PhysicalExpr; -use datafusion_common::{plan_err, DataFusionError, Result}; +use datafusion_common::{internal_err, DataFusionError, Result}; use datafusion_expr::ColumnarValue; /// A place holder expression, can not be evaluated. @@ -65,7 +65,7 @@ impl PhysicalExpr for NoOp { } fn evaluate(&self, _batch: &RecordBatch) -> Result { - plan_err!("NoOp::evaluate() should not be called") + internal_err!("NoOp::evaluate() should not be called") } fn children(&self) -> Vec> { diff --git a/datafusion/physical-expr/src/sort_expr.rs b/datafusion/physical-expr/src/sort_expr.rs index 74179ba5947c..3b3221289d31 100644 --- a/datafusion/physical-expr/src/sort_expr.rs +++ b/datafusion/physical-expr/src/sort_expr.rs @@ -25,7 +25,7 @@ use crate::PhysicalExpr; use arrow::compute::kernels::sort::{SortColumn, SortOptions}; use arrow::record_batch::RecordBatch; -use datafusion_common::plan_err; +use datafusion_common::exec_err; use datafusion_common::{DataFusionError, Result}; use datafusion_expr::ColumnarValue; @@ -66,7 +66,7 @@ impl PhysicalSortExpr { let array_to_sort = match value_to_sort { ColumnarValue::Array(array) => array, ColumnarValue::Scalar(scalar) => { - return plan_err!( + return exec_err!( "Sort operation is not applicable to scalar value {scalar}" ); } From 7788b90694dc57fb88cc5a6da8cbfd024077e05b Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Wed, 1 Nov 2023 12:19:50 -0400 Subject: [PATCH 30/43] Minor: error on unsupported RESPECT NULLs syntax (#7998) * Minor: error on unsupported RESPECT NULLs syntax * fix clippy * Update datafusion/sql/tests/sql_integration.rs Co-authored-by: Liang-Chi Hsieh --------- Co-authored-by: Liang-Chi Hsieh --- datafusion/sql/src/expr/function.rs | 68 ++++++++++++------------- datafusion/sql/tests/sql_integration.rs | 10 ++++ 2 files changed, 43 insertions(+), 35 deletions(-) diff --git a/datafusion/sql/src/expr/function.rs b/datafusion/sql/src/expr/function.rs index 8af5fef84ac1..c58b8319ceb7 100644 --- a/datafusion/sql/src/expr/function.rs +++ b/datafusion/sql/src/expr/function.rs @@ -36,44 +36,57 @@ use super::arrow_cast::ARROW_CAST_NAME; impl<'a, S: ContextProvider> SqlToRel<'a, S> { pub(super) fn sql_function_to_expr( &self, - mut function: SQLFunction, + function: SQLFunction, schema: &DFSchema, planner_context: &mut PlannerContext, ) -> Result { - let name = if function.name.0.len() > 1 { + let SQLFunction { + name, + args, + over, + distinct, + filter, + null_treatment, + special: _, // true if not called with trailing parens + order_by, + } = function; + + if let Some(null_treatment) = null_treatment { + return not_impl_err!("Null treatment in aggregate functions is not supported: {null_treatment}"); + } + + let name = if name.0.len() > 1 { // DF doesn't handle compound identifiers // (e.g. "foo.bar") for function names yet - function.name.to_string() + name.to_string() } else { - crate::utils::normalize_ident(function.name.0[0].clone()) + crate::utils::normalize_ident(name.0[0].clone()) }; // user-defined function (UDF) should have precedence in case it has the same name as a scalar built-in function if let Some(fm) = self.context_provider.get_function_meta(&name) { - let args = - self.function_args_to_expr(function.args, schema, planner_context)?; + let args = self.function_args_to_expr(args, schema, planner_context)?; return Ok(Expr::ScalarUDF(ScalarUDF::new(fm, args))); } // next, scalar built-in if let Ok(fun) = BuiltinScalarFunction::from_str(&name) { - let args = - self.function_args_to_expr(function.args, schema, planner_context)?; + let args = self.function_args_to_expr(args, schema, planner_context)?; return Ok(Expr::ScalarFunction(ScalarFunction::new(fun, args))); }; // If function is a window function (it has an OVER clause), // it shouldn't have ordering requirement as function argument // required ordering should be defined in OVER clause. - let is_function_window = function.over.is_some(); - if !function.order_by.is_empty() && is_function_window { + let is_function_window = over.is_some(); + if !order_by.is_empty() && is_function_window { return plan_err!( "Aggregate ORDER BY is not implemented for window functions" ); } // then, window function - if let Some(WindowType::WindowSpec(window)) = function.over.take() { + if let Some(WindowType::WindowSpec(window)) = over { let partition_by = window .partition_by .into_iter() @@ -97,11 +110,8 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { if let Ok(fun) = self.find_window_func(&name) { let expr = match fun { WindowFunction::AggregateFunction(aggregate_fun) => { - let args = self.function_args_to_expr( - function.args, - schema, - planner_context, - )?; + let args = + self.function_args_to_expr(args, schema, planner_context)?; Expr::WindowFunction(expr::WindowFunction::new( WindowFunction::AggregateFunction(aggregate_fun), @@ -113,11 +123,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { } _ => Expr::WindowFunction(expr::WindowFunction::new( fun, - self.function_args_to_expr( - function.args, - schema, - planner_context, - )?, + self.function_args_to_expr(args, schema, planner_context)?, partition_by, order_by, window_frame, @@ -128,8 +134,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { } else { // User defined aggregate functions (UDAF) have precedence in case it has the same name as a scalar built-in function if let Some(fm) = self.context_provider.get_aggregate_meta(&name) { - let args = - self.function_args_to_expr(function.args, schema, planner_context)?; + let args = self.function_args_to_expr(args, schema, planner_context)?; return Ok(Expr::AggregateUDF(expr::AggregateUDF::new( fm, args, None, None, ))); @@ -137,17 +142,11 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { // next, aggregate built-ins if let Ok(fun) = AggregateFunction::from_str(&name) { - let distinct = function.distinct; - let order_by = self.order_by_to_sort_expr( - &function.order_by, - schema, - planner_context, - )?; + let order_by = + self.order_by_to_sort_expr(&order_by, schema, planner_context)?; let order_by = (!order_by.is_empty()).then_some(order_by); - let args = - self.function_args_to_expr(function.args, schema, planner_context)?; - let filter: Option> = function - .filter + let args = self.function_args_to_expr(args, schema, planner_context)?; + let filter: Option> = filter .map(|e| self.sql_expr_to_logical_expr(*e, schema, planner_context)) .transpose()? .map(Box::new); @@ -159,8 +158,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { // Special case arrow_cast (as its type is dependent on its argument value) if name == ARROW_CAST_NAME { - let args = - self.function_args_to_expr(function.args, schema, planner_context)?; + let args = self.function_args_to_expr(args, schema, planner_context)?; return super::arrow_cast::create_arrow_cast(args, schema); } } diff --git a/datafusion/sql/tests/sql_integration.rs b/datafusion/sql/tests/sql_integration.rs index 2446ee0a5841..ff6dca7eef2a 100644 --- a/datafusion/sql/tests/sql_integration.rs +++ b/datafusion/sql/tests/sql_integration.rs @@ -1287,6 +1287,16 @@ fn select_simple_aggregate_repeated_aggregate_with_unique_aliases() { ); } +#[test] +fn select_simple_aggregate_respect_nulls() { + let sql = "SELECT MIN(age) RESPECT NULLS FROM person"; + let err = logical_plan(sql).expect_err("query should have failed"); + + assert_contains!( + err.strip_backtrace(), + "This feature is not implemented: Null treatment in aggregate functions is not supported: RESPECT NULLS" + ); +} #[test] fn select_from_typed_string_values() { quick_test( From 94dac76e79380aad9adc781e0c89c2db56d96301 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Marko=20Milenkovi=C4=87?= Date: Wed, 1 Nov 2023 16:20:38 +0000 Subject: [PATCH 31/43] GroupedHashAggregateStream breaks spill batch (#8004) ... into smaller chunks to decrease memory required for merging. --- datafusion/physical-plan/src/aggregates/mod.rs | 2 +- datafusion/physical-plan/src/aggregates/row_hash.rs | 11 ++++++++++- 2 files changed, 11 insertions(+), 2 deletions(-) diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 4c612223178c..da152a6264af 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -2155,7 +2155,7 @@ mod tests { spill: bool, ) -> Result<()> { let task_ctx = if spill { - new_spill_ctx(2, 2812) + new_spill_ctx(2, 2886) } else { Arc::new(TaskContext::default()) }; diff --git a/datafusion/physical-plan/src/aggregates/row_hash.rs b/datafusion/physical-plan/src/aggregates/row_hash.rs index d773533ad6a3..7b660885845b 100644 --- a/datafusion/physical-plan/src/aggregates/row_hash.rs +++ b/datafusion/physical-plan/src/aggregates/row_hash.rs @@ -673,7 +673,16 @@ impl GroupedHashAggregateStream { let spillfile = self.runtime.disk_manager.create_tmp_file("HashAggSpill")?; let mut writer = IPCWriter::new(spillfile.path(), &emit.schema())?; // TODO: slice large `sorted` and write to multiple files in parallel - writer.write(&sorted)?; + let mut offset = 0; + let total_rows = sorted.num_rows(); + + while offset < total_rows { + let length = std::cmp::min(total_rows - offset, self.batch_size); + let batch = sorted.slice(offset, length); + offset += batch.num_rows(); + writer.write(&batch)?; + } + writer.finish()?; self.spill_state.spills.push(spillfile); Ok(()) From 35e8e33930f8a60f406edcb3bbba4b3a4c0800a7 Mon Sep 17 00:00:00 2001 From: Raphael Taylor-Davies <1781103+tustvold@users.noreply.github.com> Date: Wed, 1 Nov 2023 16:47:55 +0000 Subject: [PATCH 32/43] Minor: Add implementation examples to ExecutionPlan::execute (#8013) * Add implementation examples to ExecutionPlan::execute * Review feedback --- datafusion/physical-plan/src/lib.rs | 104 ++++++++++++++++++++++++++++ 1 file changed, 104 insertions(+) diff --git a/datafusion/physical-plan/src/lib.rs b/datafusion/physical-plan/src/lib.rs index 3ada2fa163fd..8ae2a8686674 100644 --- a/datafusion/physical-plan/src/lib.rs +++ b/datafusion/physical-plan/src/lib.rs @@ -236,6 +236,110 @@ pub trait ExecutionPlan: Debug + DisplayAs + Send + Sync { } /// Begin execution of `partition`, returning a stream of [`RecordBatch`]es. + /// + /// # Implementation Examples + /// + /// ## Return Precomputed Batch + /// + /// We can return a precomputed batch as a stream + /// + /// ``` + /// # use std::sync::Arc; + /// # use arrow_array::RecordBatch; + /// # use arrow_schema::SchemaRef; + /// # use datafusion_common::Result; + /// # use datafusion_execution::{SendableRecordBatchStream, TaskContext}; + /// # use datafusion_physical_plan::memory::MemoryStream; + /// # use datafusion_physical_plan::stream::RecordBatchStreamAdapter; + /// struct MyPlan { + /// batch: RecordBatch, + /// } + /// + /// impl MyPlan { + /// fn execute( + /// &self, + /// partition: usize, + /// context: Arc + /// ) -> Result { + /// let fut = futures::future::ready(Ok(self.batch.clone())); + /// let stream = futures::stream::once(fut); + /// Ok(Box::pin(RecordBatchStreamAdapter::new(self.batch.schema(), stream))) + /// } + /// } + /// ``` + /// + /// ## Async Compute Batch + /// + /// We can also lazily compute a RecordBatch when the returned stream is polled + /// + /// ``` + /// # use std::sync::Arc; + /// # use arrow_array::RecordBatch; + /// # use arrow_schema::SchemaRef; + /// # use datafusion_common::Result; + /// # use datafusion_execution::{SendableRecordBatchStream, TaskContext}; + /// # use datafusion_physical_plan::memory::MemoryStream; + /// # use datafusion_physical_plan::stream::RecordBatchStreamAdapter; + /// struct MyPlan { + /// schema: SchemaRef, + /// } + /// + /// async fn get_batch() -> Result { + /// todo!() + /// } + /// + /// impl MyPlan { + /// fn execute( + /// &self, + /// partition: usize, + /// context: Arc + /// ) -> Result { + /// let fut = get_batch(); + /// let stream = futures::stream::once(fut); + /// Ok(Box::pin(RecordBatchStreamAdapter::new(self.schema.clone(), stream))) + /// } + /// } + /// ``` + /// + /// ## Async Compute Batch Stream + /// + /// We can lazily compute a RecordBatch stream when the returned stream is polled + /// flattening the result into a single stream + /// + /// ``` + /// # use std::sync::Arc; + /// # use arrow_array::RecordBatch; + /// # use arrow_schema::SchemaRef; + /// # use futures::TryStreamExt; + /// # use datafusion_common::Result; + /// # use datafusion_execution::{SendableRecordBatchStream, TaskContext}; + /// # use datafusion_physical_plan::memory::MemoryStream; + /// # use datafusion_physical_plan::stream::RecordBatchStreamAdapter; + /// struct MyPlan { + /// schema: SchemaRef, + /// } + /// + /// async fn get_batch_stream() -> Result { + /// todo!() + /// } + /// + /// impl MyPlan { + /// fn execute( + /// &self, + /// partition: usize, + /// context: Arc + /// ) -> Result { + /// // A future that yields a stream + /// let fut = get_batch_stream(); + /// // Use TryStreamExt::try_flatten to flatten the stream of streams + /// let stream = futures::stream::once(fut).try_flatten(); + /// Ok(Box::pin(RecordBatchStreamAdapter::new(self.schema.clone(), stream))) + /// } + /// } + /// ``` + /// + /// See [`futures::stream::StreamExt`] and [`futures::stream::TryStreamExt`] for further + /// combinators that can be used with streams fn execute( &self, partition: usize, From 06f2475b304ad173fe7dfe31656928747460588e Mon Sep 17 00:00:00 2001 From: Jay Zhan Date: Thu, 2 Nov 2023 03:04:38 +0800 Subject: [PATCH 33/43] address comment (#7993) Signed-off-by: jayzhan211 --- datafusion/common/src/scalar.rs | 20 ++++---- datafusion/common/src/utils.rs | 46 ++++++++++++++++++- .../physical-expr/src/aggregate/array_agg.rs | 4 +- .../src/aggregate/array_agg_distinct.rs | 4 +- .../physical-expr/src/array_expressions.rs | 4 +- 5 files changed, 60 insertions(+), 18 deletions(-) diff --git a/datafusion/common/src/scalar.rs b/datafusion/common/src/scalar.rs index b3c11740abf9..0d701eaad283 100644 --- a/datafusion/common/src/scalar.rs +++ b/datafusion/common/src/scalar.rs @@ -30,7 +30,7 @@ use crate::cast::{ }; use crate::error::{DataFusionError, Result, _internal_err, _not_impl_err}; use crate::hash_utils::create_hashes; -use crate::utils::wrap_into_list_array; +use crate::utils::array_into_list_array; use arrow::buffer::{NullBuffer, OffsetBuffer}; use arrow::compute::kernels::numeric::*; use arrow::datatypes::{i256, FieldRef, Fields, SchemaBuilder}; @@ -1667,7 +1667,7 @@ impl ScalarValue { } else { Self::iter_to_array(values.iter().cloned()).unwrap() }; - Arc::new(wrap_into_list_array(values)) + Arc::new(array_into_list_array(values)) } /// Converts a scalar value into an array of `size` rows. @@ -2058,7 +2058,7 @@ impl ScalarValue { let list_array = as_list_array(array); let nested_array = list_array.value(index); // Produces a single element `ListArray` with the value at `index`. - let arr = Arc::new(wrap_into_list_array(nested_array)); + let arr = Arc::new(array_into_list_array(nested_array)); ScalarValue::List(arr) } @@ -2067,7 +2067,7 @@ impl ScalarValue { let list_array = as_fixed_size_list_array(array)?; let nested_array = list_array.value(index); // Produces a single element `ListArray` with the value at `index`. - let arr = Arc::new(wrap_into_list_array(nested_array)); + let arr = Arc::new(array_into_list_array(nested_array)); ScalarValue::List(arr) } @@ -3052,7 +3052,7 @@ mod tests { let array = ScalarValue::new_list(scalars.as_slice(), &DataType::Utf8); - let expected = wrap_into_list_array(Arc::new(StringArray::from(vec![ + let expected = array_into_list_array(Arc::new(StringArray::from(vec![ "rust", "arrow", "data-fusion", @@ -3091,9 +3091,9 @@ mod tests { #[test] fn iter_to_array_string_test() { let arr1 = - wrap_into_list_array(Arc::new(StringArray::from(vec!["foo", "bar", "baz"]))); + array_into_list_array(Arc::new(StringArray::from(vec!["foo", "bar", "baz"]))); let arr2 = - wrap_into_list_array(Arc::new(StringArray::from(vec!["rust", "world"]))); + array_into_list_array(Arc::new(StringArray::from(vec!["rust", "world"]))); let scalars = vec![ ScalarValue::List(Arc::new(arr1)), @@ -4335,13 +4335,13 @@ mod tests { // Define list-of-structs scalars let nl0_array = ScalarValue::iter_to_array(vec![s0.clone(), s1.clone()]).unwrap(); - let nl0 = ScalarValue::List(Arc::new(wrap_into_list_array(nl0_array))); + let nl0 = ScalarValue::List(Arc::new(array_into_list_array(nl0_array))); let nl1_array = ScalarValue::iter_to_array(vec![s2.clone()]).unwrap(); - let nl1 = ScalarValue::List(Arc::new(wrap_into_list_array(nl1_array))); + let nl1 = ScalarValue::List(Arc::new(array_into_list_array(nl1_array))); let nl2_array = ScalarValue::iter_to_array(vec![s1.clone()]).unwrap(); - let nl2 = ScalarValue::List(Arc::new(wrap_into_list_array(nl2_array))); + let nl2 = ScalarValue::List(Arc::new(array_into_list_array(nl2_array))); // iter_to_array for list-of-struct let array = ScalarValue::iter_to_array(vec![nl0, nl1, nl2]).unwrap(); diff --git a/datafusion/common/src/utils.rs b/datafusion/common/src/utils.rs index b2f71e86f21e..f031f7880436 100644 --- a/datafusion/common/src/utils.rs +++ b/datafusion/common/src/utils.rs @@ -17,6 +17,7 @@ //! This module provides the bisect function, which implements binary search. +use crate::error::_internal_err; use crate::{DataFusionError, Result, ScalarValue}; use arrow::array::{ArrayRef, PrimitiveArray}; use arrow::buffer::OffsetBuffer; @@ -24,7 +25,7 @@ use arrow::compute; use arrow::compute::{partition, SortColumn, SortOptions}; use arrow::datatypes::{Field, SchemaRef, UInt32Type}; use arrow::record_batch::RecordBatch; -use arrow_array::ListArray; +use arrow_array::{Array, ListArray}; use sqlparser::ast::Ident; use sqlparser::dialect::GenericDialect; use sqlparser::parser::Parser; @@ -338,7 +339,7 @@ pub fn longest_consecutive_prefix>( /// Wrap an array into a single element `ListArray`. /// For example `[1, 2, 3]` would be converted into `[[1, 2, 3]]` -pub fn wrap_into_list_array(arr: ArrayRef) -> ListArray { +pub fn array_into_list_array(arr: ArrayRef) -> ListArray { let offsets = OffsetBuffer::from_lengths([arr.len()]); ListArray::new( Arc::new(Field::new("item", arr.data_type().to_owned(), true)), @@ -348,6 +349,47 @@ pub fn wrap_into_list_array(arr: ArrayRef) -> ListArray { ) } +/// Wrap arrays into a single element `ListArray`. +/// +/// Example: +/// ``` +/// use arrow::array::{Int32Array, ListArray, ArrayRef}; +/// use arrow::datatypes::{Int32Type, Field}; +/// use std::sync::Arc; +/// +/// let arr1 = Arc::new(Int32Array::from(vec![1, 2, 3])) as ArrayRef; +/// let arr2 = Arc::new(Int32Array::from(vec![4, 5, 6])) as ArrayRef; +/// +/// let list_arr = datafusion_common::utils::arrays_into_list_array([arr1, arr2]).unwrap(); +/// +/// let expected = ListArray::from_iter_primitive::( +/// vec![ +/// Some(vec![Some(1), Some(2), Some(3)]), +/// Some(vec![Some(4), Some(5), Some(6)]), +/// ] +/// ); +/// +/// assert_eq!(list_arr, expected); +pub fn arrays_into_list_array( + arr: impl IntoIterator, +) -> Result { + let arr = arr.into_iter().collect::>(); + if arr.is_empty() { + return _internal_err!("Cannot wrap empty array into list array"); + } + + let lens = arr.iter().map(|x| x.len()).collect::>(); + // Assume data type is consistent + let data_type = arr[0].data_type().to_owned(); + let values = arr.iter().map(|x| x.as_ref()).collect::>(); + Ok(ListArray::new( + Arc::new(Field::new("item", data_type, true)), + OffsetBuffer::from_lengths(lens), + arrow::compute::concat(values.as_slice())?, + None, + )) +} + /// An extension trait for smart pointers. Provides an interface to get a /// raw pointer to the data (with metadata stripped away). /// diff --git a/datafusion/physical-expr/src/aggregate/array_agg.rs b/datafusion/physical-expr/src/aggregate/array_agg.rs index 834925b8d554..4dccbfef07f8 100644 --- a/datafusion/physical-expr/src/aggregate/array_agg.rs +++ b/datafusion/physical-expr/src/aggregate/array_agg.rs @@ -24,7 +24,7 @@ use arrow::array::ArrayRef; use arrow::datatypes::{DataType, Field}; use arrow_array::Array; use datafusion_common::cast::as_list_array; -use datafusion_common::utils::wrap_into_list_array; +use datafusion_common::utils::array_into_list_array; use datafusion_common::Result; use datafusion_common::ScalarValue; use datafusion_expr::Accumulator; @@ -161,7 +161,7 @@ impl Accumulator for ArrayAggAccumulator { } let concated_array = arrow::compute::concat(&element_arrays)?; - let list_array = wrap_into_list_array(concated_array); + let list_array = array_into_list_array(concated_array); Ok(ScalarValue::List(Arc::new(list_array))) } diff --git a/datafusion/physical-expr/src/aggregate/array_agg_distinct.rs b/datafusion/physical-expr/src/aggregate/array_agg_distinct.rs index 21143ce54a20..9b391b0c42cf 100644 --- a/datafusion/physical-expr/src/aggregate/array_agg_distinct.rs +++ b/datafusion/physical-expr/src/aggregate/array_agg_distinct.rs @@ -185,7 +185,7 @@ mod tests { use arrow_array::types::Int32Type; use arrow_array::{Array, ListArray}; use arrow_buffer::OffsetBuffer; - use datafusion_common::utils::wrap_into_list_array; + use datafusion_common::utils::array_into_list_array; use datafusion_common::{internal_err, DataFusionError}; // arrow::compute::sort cann't sort ListArray directly, so we need to sort the inner primitive array and wrap it back into ListArray. @@ -201,7 +201,7 @@ mod tests { }; let arr = arrow::compute::sort(&arr, None).unwrap(); - let list_arr = wrap_into_list_array(arr); + let list_arr = array_into_list_array(arr); ScalarValue::List(Arc::new(list_arr)) } diff --git a/datafusion/physical-expr/src/array_expressions.rs b/datafusion/physical-expr/src/array_expressions.rs index 84fd301b84de..18d8c60fe7bd 100644 --- a/datafusion/physical-expr/src/array_expressions.rs +++ b/datafusion/physical-expr/src/array_expressions.rs @@ -29,7 +29,7 @@ use arrow_buffer::NullBuffer; use datafusion_common::cast::{ as_generic_string_array, as_int64_array, as_list_array, as_string_array, }; -use datafusion_common::utils::wrap_into_list_array; +use datafusion_common::utils::array_into_list_array; use datafusion_common::{ exec_err, internal_err, not_impl_err, plan_err, DataFusionError, Result, }; @@ -412,7 +412,7 @@ pub fn make_array(arrays: &[ArrayRef]) -> Result { // Either an empty array or all nulls: DataType::Null => { let array = new_null_array(&DataType::Null, arrays.len()); - Ok(Arc::new(wrap_into_list_array(array))) + Ok(Arc::new(array_into_list_array(array))) } data_type => array_array(arrays, data_type), } From 5634cce35f6ceb13cf7367b91de9e50d751e442a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Marko=20Milenkovi=C4=87?= Date: Wed, 1 Nov 2023 20:33:28 +0000 Subject: [PATCH 34/43] GroupedHashAggregateStream should register spillable consumer (#8002) --- datafusion/physical-plan/src/aggregates/row_hash.rs | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/datafusion/physical-plan/src/aggregates/row_hash.rs b/datafusion/physical-plan/src/aggregates/row_hash.rs index 7b660885845b..f1eece4315e4 100644 --- a/datafusion/physical-plan/src/aggregates/row_hash.rs +++ b/datafusion/physical-plan/src/aggregates/row_hash.rs @@ -328,7 +328,9 @@ impl GroupedHashAggregateStream { .collect(); let name = format!("GroupedHashAggregateStream[{partition}]"); - let reservation = MemoryConsumer::new(name).register(context.memory_pool()); + let reservation = MemoryConsumer::new(name) + .with_can_spill(true) + .register(context.memory_pool()); let group_ordering = agg .aggregation_ordering From 7f3f4657185c0a7c398ced8d1cfdbaf34c19eeb5 Mon Sep 17 00:00:00 2001 From: Huaijin Date: Thu, 2 Nov 2023 19:56:03 +0800 Subject: [PATCH 35/43] fix: single_distinct_aggretation_to_group_by fail (#7997) * fix: single_distinct_aggretation_to_group_by faile * fix * move test to groupby.slt --- .../src/single_distinct_to_groupby.rs | 2 +- .../sqllogictest/test_files/groupby.slt | 58 +++++++++++++++++++ 2 files changed, 59 insertions(+), 1 deletion(-) diff --git a/datafusion/optimizer/src/single_distinct_to_groupby.rs b/datafusion/optimizer/src/single_distinct_to_groupby.rs index 8e0f93cb5781..be76c069f0b7 100644 --- a/datafusion/optimizer/src/single_distinct_to_groupby.rs +++ b/datafusion/optimizer/src/single_distinct_to_groupby.rs @@ -74,7 +74,7 @@ fn is_single_distinct_agg(plan: &LogicalPlan) -> Result { distinct_count += 1; } for e in args { - fields_set.insert(e.display_name()?); + fields_set.insert(e.canonical_name()); } } } diff --git a/datafusion/sqllogictest/test_files/groupby.slt b/datafusion/sqllogictest/test_files/groupby.slt index 5cb3ac2f8135..ef6fee69abbf 100644 --- a/datafusion/sqllogictest/test_files/groupby.slt +++ b/datafusion/sqllogictest/test_files/groupby.slt @@ -3733,3 +3733,61 @@ AggregateExec: mode=FinalPartitioned, gby=[c@0 as c, b@1 as b], aggr=[SUM(multip --------AggregateExec: mode=Partial, gby=[c@1 as c, b@0 as b], aggr=[SUM(multiple_ordered_table_with_pk.d)], ordering_mode=PartiallyOrdered ----------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 ------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[b, c, d], output_ordering=[c@1 ASC NULLS LAST], has_header=true + +# Tests for single distinct to group by optimization rule +statement ok +CREATE TABLE t(x int) AS VALUES (1), (2), (1); + +statement ok +create table t1(x bigint,y int) as values (9223372036854775807,2), (9223372036854775806,2); + +query II +SELECT SUM(DISTINCT x), MAX(DISTINCT x) from t GROUP BY x ORDER BY x; +---- +1 1 +2 2 + +query II +SELECT MAX(DISTINCT x), SUM(DISTINCT x) from t GROUP BY x ORDER BY x; +---- +1 1 +2 2 + +query TT +EXPLAIN SELECT SUM(DISTINCT CAST(x AS DOUBLE)), MAX(DISTINCT x) FROM t1 GROUP BY y; +---- +logical_plan +Projection: SUM(DISTINCT t1.x), MAX(DISTINCT t1.x) +--Aggregate: groupBy=[[t1.y]], aggr=[[SUM(DISTINCT CAST(t1.x AS Float64)), MAX(DISTINCT t1.x)]] +----TableScan: t1 projection=[x, y] +physical_plan +ProjectionExec: expr=[SUM(DISTINCT t1.x)@1 as SUM(DISTINCT t1.x), MAX(DISTINCT t1.x)@2 as MAX(DISTINCT t1.x)] +--AggregateExec: mode=FinalPartitioned, gby=[y@0 as y], aggr=[SUM(DISTINCT t1.x), MAX(DISTINCT t1.x)] +----CoalesceBatchesExec: target_batch_size=2 +------RepartitionExec: partitioning=Hash([y@0], 8), input_partitions=8 +--------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 +----------AggregateExec: mode=Partial, gby=[y@1 as y], aggr=[SUM(DISTINCT t1.x), MAX(DISTINCT t1.x)] +------------MemoryExec: partitions=1, partition_sizes=[1] + +query TT +EXPLAIN SELECT SUM(DISTINCT CAST(x AS DOUBLE)), MAX(DISTINCT CAST(x AS DOUBLE)) FROM t1 GROUP BY y; +---- +logical_plan +Projection: SUM(alias1) AS SUM(DISTINCT t1.x), MAX(alias1) AS MAX(DISTINCT t1.x) +--Aggregate: groupBy=[[t1.y]], aggr=[[SUM(alias1), MAX(alias1)]] +----Aggregate: groupBy=[[t1.y, CAST(t1.x AS Float64)t1.x AS t1.x AS alias1]], aggr=[[]] +------Projection: CAST(t1.x AS Float64) AS CAST(t1.x AS Float64)t1.x, t1.y +--------TableScan: t1 projection=[x, y] +physical_plan +ProjectionExec: expr=[SUM(alias1)@1 as SUM(DISTINCT t1.x), MAX(alias1)@2 as MAX(DISTINCT t1.x)] +--AggregateExec: mode=FinalPartitioned, gby=[y@0 as y], aggr=[SUM(alias1), MAX(alias1)] +----CoalesceBatchesExec: target_batch_size=2 +------RepartitionExec: partitioning=Hash([y@0], 8), input_partitions=8 +--------AggregateExec: mode=Partial, gby=[y@0 as y], aggr=[SUM(alias1), MAX(alias1)] +----------AggregateExec: mode=FinalPartitioned, gby=[y@0 as y, alias1@1 as alias1], aggr=[] +------------CoalesceBatchesExec: target_batch_size=2 +--------------RepartitionExec: partitioning=Hash([y@0, alias1@1], 8), input_partitions=8 +----------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 +------------------AggregateExec: mode=Partial, gby=[y@1 as y, CAST(t1.x AS Float64)t1.x@0 as alias1], aggr=[] +--------------------ProjectionExec: expr=[CAST(x@0 AS Float64) as CAST(t1.x AS Float64)t1.x, y@1 as y] +----------------------MemoryExec: partitions=1, partition_sizes=[1] From 436a4fa348f1fcdf47bb2624deb24f8b5266df8c Mon Sep 17 00:00:00 2001 From: Jeffrey <22608443+Jefffrey@users.noreply.github.com> Date: Fri, 3 Nov 2023 00:48:36 +1100 Subject: [PATCH 36/43] Read only enough bytes to infer Arrow IPC file schema via stream (#7962) * Read only enough bytes to infer Arrow IPC file schema via stream * Error checking for collect bytes func * Update datafusion/core/src/datasource/file_format/arrow.rs Co-authored-by: Andrew Lamb --------- Co-authored-by: Andrew Lamb --- .../core/src/datasource/file_format/arrow.rs | 197 +++++++++++++++++- 1 file changed, 186 insertions(+), 11 deletions(-) diff --git a/datafusion/core/src/datasource/file_format/arrow.rs b/datafusion/core/src/datasource/file_format/arrow.rs index 16ae4411d1bf..2777805078c7 100644 --- a/datafusion/core/src/datasource/file_format/arrow.rs +++ b/datafusion/core/src/datasource/file_format/arrow.rs @@ -20,7 +20,7 @@ //! Works with files following the [Arrow IPC format](https://arrow.apache.org/docs/format/Columnar.html#ipc-file-format) use std::any::Any; -use std::io::{Read, Seek}; +use std::borrow::Cow; use std::sync::Arc; use crate::datasource::file_format::FileFormat; @@ -29,13 +29,18 @@ use crate::error::Result; use crate::execution::context::SessionState; use crate::physical_plan::ExecutionPlan; +use arrow::ipc::convert::fb_to_schema; use arrow::ipc::reader::FileReader; -use arrow_schema::{Schema, SchemaRef}; +use arrow::ipc::root_as_message; +use arrow_schema::{ArrowError, Schema, SchemaRef}; +use bytes::Bytes; use datafusion_common::{FileType, Statistics}; use datafusion_physical_expr::PhysicalExpr; use async_trait::async_trait; +use futures::stream::BoxStream; +use futures::StreamExt; use object_store::{GetResultPayload, ObjectMeta, ObjectStore}; /// Arrow `FileFormat` implementation. @@ -59,13 +64,11 @@ impl FileFormat for ArrowFormat { let r = store.as_ref().get(&object.location).await?; let schema = match r.payload { GetResultPayload::File(mut file, _) => { - read_arrow_schema_from_reader(&mut file)? + let reader = FileReader::try_new(&mut file, None)?; + reader.schema() } - GetResultPayload::Stream(_) => { - // TODO: Fetching entire file to get schema is potentially wasteful - let data = r.bytes().await?; - let mut cursor = std::io::Cursor::new(&data); - read_arrow_schema_from_reader(&mut cursor)? + GetResultPayload::Stream(stream) => { + infer_schema_from_file_stream(stream).await? } }; schemas.push(schema.as_ref().clone()); @@ -99,7 +102,179 @@ impl FileFormat for ArrowFormat { } } -fn read_arrow_schema_from_reader(reader: R) -> Result { - let reader = FileReader::try_new(reader, None)?; - Ok(reader.schema()) +const ARROW_MAGIC: [u8; 6] = [b'A', b'R', b'R', b'O', b'W', b'1']; +const CONTINUATION_MARKER: [u8; 4] = [0xff; 4]; + +/// Custom implementation of inferring schema. Should eventually be moved upstream to arrow-rs. +/// See https://github.com/apache/arrow-rs/issues/5021 +async fn infer_schema_from_file_stream( + mut stream: BoxStream<'static, object_store::Result>, +) -> Result { + // Expected format: + // - 6 bytes + // - 2 bytes + // - 4 bytes, not present below v0.15.0 + // - 4 bytes + // + // + + // So in first read we need at least all known sized sections, + // which is 6 + 2 + 4 + 4 = 16 bytes. + let bytes = collect_at_least_n_bytes(&mut stream, 16, None).await?; + + // Files should start with these magic bytes + if bytes[0..6] != ARROW_MAGIC { + return Err(ArrowError::ParseError( + "Arrow file does not contian correct header".to_string(), + ))?; + } + + // Since continuation marker bytes added in later versions + let (meta_len, rest_of_bytes_start_index) = if bytes[8..12] == CONTINUATION_MARKER { + (&bytes[12..16], 16) + } else { + (&bytes[8..12], 12) + }; + + let meta_len = [meta_len[0], meta_len[1], meta_len[2], meta_len[3]]; + let meta_len = i32::from_le_bytes(meta_len); + + // Read bytes for Schema message + let block_data = if bytes[rest_of_bytes_start_index..].len() < meta_len as usize { + // Need to read more bytes to decode Message + let mut block_data = Vec::with_capacity(meta_len as usize); + // In case we had some spare bytes in our initial read chunk + block_data.extend_from_slice(&bytes[rest_of_bytes_start_index..]); + let size_to_read = meta_len as usize - block_data.len(); + let block_data = + collect_at_least_n_bytes(&mut stream, size_to_read, Some(block_data)).await?; + Cow::Owned(block_data) + } else { + // Already have the bytes we need + let end_index = meta_len as usize + rest_of_bytes_start_index; + let block_data = &bytes[rest_of_bytes_start_index..end_index]; + Cow::Borrowed(block_data) + }; + + // Decode Schema message + let message = root_as_message(&block_data).map_err(|err| { + ArrowError::ParseError(format!("Unable to read IPC message as metadata: {err:?}")) + })?; + let ipc_schema = message.header_as_schema().ok_or_else(|| { + ArrowError::IpcError("Unable to read IPC message as schema".to_string()) + })?; + let schema = fb_to_schema(ipc_schema); + + Ok(Arc::new(schema)) +} + +async fn collect_at_least_n_bytes( + stream: &mut BoxStream<'static, object_store::Result>, + n: usize, + extend_from: Option>, +) -> Result> { + let mut buf = extend_from.unwrap_or_else(|| Vec::with_capacity(n)); + // If extending existing buffer then ensure we read n additional bytes + let n = n + buf.len(); + while let Some(bytes) = stream.next().await.transpose()? { + buf.extend_from_slice(&bytes); + if buf.len() >= n { + break; + } + } + if buf.len() < n { + return Err(ArrowError::ParseError( + "Unexpected end of byte stream for Arrow IPC file".to_string(), + ))?; + } + Ok(buf) +} + +#[cfg(test)] +mod tests { + use chrono::DateTime; + use object_store::{chunked::ChunkedStore, memory::InMemory, path::Path}; + + use crate::execution::context::SessionContext; + + use super::*; + + #[tokio::test] + async fn test_infer_schema_stream() -> Result<()> { + let mut bytes = std::fs::read("tests/data/example.arrow")?; + bytes.truncate(bytes.len() - 20); // mangle end to show we don't need to read whole file + let location = Path::parse("example.arrow")?; + let in_memory_store: Arc = Arc::new(InMemory::new()); + in_memory_store.put(&location, bytes.into()).await?; + + let session_ctx = SessionContext::new(); + let state = session_ctx.state(); + let object_meta = ObjectMeta { + location, + last_modified: DateTime::default(), + size: usize::MAX, + e_tag: None, + }; + + let arrow_format = ArrowFormat {}; + let expected = vec!["f0: Int64", "f1: Utf8", "f2: Boolean"]; + + // Test chunk sizes where too small so we keep having to read more bytes + // And when large enough that first read contains all we need + for chunk_size in [7, 3000] { + let store = Arc::new(ChunkedStore::new(in_memory_store.clone(), chunk_size)); + let inferred_schema = arrow_format + .infer_schema( + &state, + &(store.clone() as Arc), + &[object_meta.clone()], + ) + .await?; + let actual_fields = inferred_schema + .fields() + .iter() + .map(|f| format!("{}: {:?}", f.name(), f.data_type())) + .collect::>(); + assert_eq!(expected, actual_fields); + } + + Ok(()) + } + + #[tokio::test] + async fn test_infer_schema_short_stream() -> Result<()> { + let mut bytes = std::fs::read("tests/data/example.arrow")?; + bytes.truncate(20); // should cause error that file shorter than expected + let location = Path::parse("example.arrow")?; + let in_memory_store: Arc = Arc::new(InMemory::new()); + in_memory_store.put(&location, bytes.into()).await?; + + let session_ctx = SessionContext::new(); + let state = session_ctx.state(); + let object_meta = ObjectMeta { + location, + last_modified: DateTime::default(), + size: usize::MAX, + e_tag: None, + }; + + let arrow_format = ArrowFormat {}; + + let store = Arc::new(ChunkedStore::new(in_memory_store.clone(), 7)); + let err = arrow_format + .infer_schema( + &state, + &(store.clone() as Arc), + &[object_meta.clone()], + ) + .await; + + assert!(err.is_err()); + assert_eq!( + "Arrow error: Parser error: Unexpected end of byte stream for Arrow IPC file", + err.unwrap_err().to_string() + ); + + Ok(()) + } } From d2671cd17ed854251adc54ca1ba87776593be61b Mon Sep 17 00:00:00 2001 From: Huaijin Date: Fri, 3 Nov 2023 02:53:52 +0800 Subject: [PATCH 37/43] Minor: remove a strange char (#8030) --- datafusion/execution/src/memory_pool/mod.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/execution/src/memory_pool/mod.rs b/datafusion/execution/src/memory_pool/mod.rs index 71483e200af6..55555014f2ef 100644 --- a/datafusion/execution/src/memory_pool/mod.rs +++ b/datafusion/execution/src/memory_pool/mod.rs @@ -235,7 +235,7 @@ impl MemoryReservation { } } - /// Returns a new empty [`MemoryReservation`] with the same [`MemoryConsumer`] + /// Returns a new empty [`MemoryReservation`] with the same [`MemoryConsumer`] pub fn new_empty(&self) -> Self { Self { size: 0, From b0891372a75d1e19fdadb37312686b53a8d03090 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Thu, 2 Nov 2023 14:54:41 -0400 Subject: [PATCH 38/43] Minor: Improve documentation for Filter Pushdown (#8023) * Minor: Improve documentation for Fulter Pushdown * Update datafusion/optimizer/src/push_down_filter.rs Co-authored-by: jakevin * Apply suggestions from code review * Update datafusion/optimizer/src/push_down_filter.rs Co-authored-by: Alex Huang --------- Co-authored-by: jakevin Co-authored-by: Alex Huang --- datafusion/optimizer/src/push_down_filter.rs | 101 +++++++++++++++---- 1 file changed, 82 insertions(+), 19 deletions(-) diff --git a/datafusion/optimizer/src/push_down_filter.rs b/datafusion/optimizer/src/push_down_filter.rs index 8c2eb96a48d8..ae986b3c84dd 100644 --- a/datafusion/optimizer/src/push_down_filter.rs +++ b/datafusion/optimizer/src/push_down_filter.rs @@ -12,7 +12,8 @@ // specific language governing permissions and limitations // under the License. -//! Push Down Filter optimizer rule ensures that filters are applied as early as possible in the plan +//! [`PushDownFilter`] Moves filters so they are applied as early as possible in +//! the plan. use crate::optimizer::ApplyOrder; use crate::utils::{conjunction, split_conjunction, split_conjunction_owned}; @@ -33,31 +34,93 @@ use itertools::Itertools; use std::collections::{HashMap, HashSet}; use std::sync::Arc; -/// Push Down Filter optimizer rule pushes filter clauses down the plan +/// Optimizer rule for pushing (moving) filter expressions down in a plan so +/// they are applied as early as possible. +/// /// # Introduction -/// A filter-commutative operation is an operation whose result of filter(op(data)) = op(filter(data)). -/// An example of a filter-commutative operation is a projection; a counter-example is `limit`. /// -/// The filter-commutative property is column-specific. An aggregate grouped by A on SUM(B) -/// can commute with a filter that depends on A only, but does not commute with a filter that depends -/// on SUM(B). +/// The goal of this rule is to improve query performance by eliminating +/// redundant work. +/// +/// For example, given a plan that sorts all values where `a > 10`: +/// +/// ```text +/// Filter (a > 10) +/// Sort (a, b) +/// ``` +/// +/// A better plan is to filter the data *before* the Sort, which sorts fewer +/// rows and therefore does less work overall: +/// +/// ```text +/// Sort (a, b) +/// Filter (a > 10) <-- Filter is moved before the sort +/// ``` +/// +/// However it is not always possible to push filters down. For example, given a +/// plan that finds the top 3 values and then keeps only those that are greater +/// than 10, if the filter is pushed below the limit it would produce a +/// different result. +/// +/// ```text +/// Filter (a > 10) <-- can not move this Filter before the limit +/// Limit (fetch=3) +/// Sort (a, b) +/// ``` +/// +/// +/// More formally, a filter-commutative operation is an operation `op` that +/// satisfies `filter(op(data)) = op(filter(data))`. +/// +/// The filter-commutative property is plan and column-specific. A filter on `a` +/// can be pushed through a `Aggregate(group_by = [a], agg=[SUM(b))`. However, a +/// filter on `SUM(b)` can not be pushed through the same aggregate. +/// +/// # Handling Conjunctions +/// +/// It is possible to only push down **part** of a filter expression if is +/// connected with `AND`s (more formally if it is a "conjunction"). +/// +/// For example, given the following plan: +/// +/// ```text +/// Filter(a > 10 AND SUM(b) < 5) +/// Aggregate(group_by = [a], agg = [SUM(b)) +/// ``` +/// +/// The `a > 10` is commutative with the `Aggregate` but `SUM(b) < 5` is not. +/// Therefore it is possible to only push part of the expression, resulting in: +/// +/// ```text +/// Filter(SUM(b) < 5) +/// Aggregate(group_by = [a], agg = [SUM(b)) +/// Filter(a > 10) +/// ``` +/// +/// # Handling Column Aliases /// -/// This optimizer commutes filters with filter-commutative operations to push the filters -/// the closest possible to the scans, re-writing the filter expressions by every -/// projection that changes the filter's expression. +/// This optimizer must sometimes handle re-writing filter expressions when they +/// pushed, for example if there is a projection that aliases `a+1` to `"b"`: /// -/// Filter: b Gt Int64(10) -/// Projection: a AS b +/// ```text +/// Filter (b > 10) +/// Projection: [a+1 AS "b"] <-- changes the name of `a+1` to `b` +/// ``` /// -/// is optimized to +/// To apply the filter prior to the `Projection`, all references to `b` must be +/// rewritten to `a+1`: /// -/// Projection: a AS b -/// Filter: a Gt Int64(10) <--- changed from b to a +/// ```text +/// Projection: a AS "b" +/// Filter: (a + 1 > 10) <--- changed from b to a + 1 +/// ``` +/// # Implementation Notes /// -/// This performs a single pass through the plan. When it passes through a filter, it stores that filter, -/// and when it reaches a node that does not commute with it, it adds the filter to that place. -/// When it passes through a projection, it re-writes the filter's expression taking into account that projection. -/// When multiple filters would have been written, it `AND` their expressions into a single expression. +/// This implementation performs a single pass through the plan, "pushing" down +/// filters. When it passes through a filter, it stores that filter, and when it +/// reaches a plan node that does not commute with that filter, it adds the +/// filter to that place. When it passes through a projection, it re-writes the +/// filter's expression taking into account that projection. #[derive(Default)] pub struct PushDownFilter {} From 8682be57f5208a9d1097729802e18f35c09392cd Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Thu, 2 Nov 2023 14:55:02 -0400 Subject: [PATCH 39/43] Minor: Improve `ExecutionPlan` documentation (#8019) * Minor: Improve `ExecutionPlan` documentation * Add link to Partitioning --- datafusion/physical-plan/src/lib.rs | 113 +++++++++++++++++++--------- 1 file changed, 79 insertions(+), 34 deletions(-) diff --git a/datafusion/physical-plan/src/lib.rs b/datafusion/physical-plan/src/lib.rs index 8ae2a8686674..ed795a1cb333 100644 --- a/datafusion/physical-plan/src/lib.rs +++ b/datafusion/physical-plan/src/lib.rs @@ -91,16 +91,27 @@ pub use datafusion_physical_expr::{ pub use crate::stream::EmptyRecordBatchStream; pub use datafusion_execution::{RecordBatchStream, SendableRecordBatchStream}; -/// `ExecutionPlan` represent nodes in the DataFusion Physical Plan. +/// Represent nodes in the DataFusion Physical Plan. /// -/// Each `ExecutionPlan` is partition-aware and is responsible for -/// creating the actual `async` [`SendableRecordBatchStream`]s -/// of [`RecordBatch`] that incrementally compute the operator's -/// output from its input partition. +/// Calling [`execute`] produces an `async` [`SendableRecordBatchStream`] of +/// [`RecordBatch`] that incrementally computes a partition of the +/// `ExecutionPlan`'s output from its input. See [`Partitioning`] for more +/// details on partitioning. +/// +/// Methods such as [`schema`] and [`output_partitioning`] communicate +/// properties of this output to the DataFusion optimizer, and methods such as +/// [`required_input_distribution`] and [`required_input_ordering`] express +/// requirements of the `ExecutionPlan` from its input. /// /// [`ExecutionPlan`] can be displayed in a simplified form using the /// return value from [`displayable`] in addition to the (normally /// quite verbose) `Debug` output. +/// +/// [`execute`]: ExecutionPlan::execute +/// [`schema`]: ExecutionPlan::schema +/// [`output_partitioning`]: ExecutionPlan::output_partitioning +/// [`required_input_distribution`]: ExecutionPlan::required_input_distribution +/// [`required_input_ordering`]: ExecutionPlan::required_input_ordering pub trait ExecutionPlan: Debug + DisplayAs + Send + Sync { /// Returns the execution plan as [`Any`] so that it can be /// downcast to a specific implementation. @@ -109,7 +120,8 @@ pub trait ExecutionPlan: Debug + DisplayAs + Send + Sync { /// Get the schema for this execution plan fn schema(&self) -> SchemaRef; - /// Specifies the output partitioning scheme of this plan + /// Specifies how the output of this `ExecutionPlan` is split into + /// partitions. fn output_partitioning(&self) -> Partitioning; /// Specifies whether this plan generates an infinite stream of records. @@ -123,7 +135,7 @@ pub trait ExecutionPlan: Debug + DisplayAs + Send + Sync { } } - /// If the output of this operator within each partition is sorted, + /// If the output of this `ExecutionPlan` within each partition is sorted, /// returns `Some(keys)` with the description of how it was sorted. /// /// For example, Sort, (obviously) produces sorted output as does @@ -131,17 +143,19 @@ pub trait ExecutionPlan: Debug + DisplayAs + Send + Sync { /// produces sorted output if its input was sorted as it does not /// reorder the input rows, /// - /// It is safe to return `None` here if your operator does not + /// It is safe to return `None` here if your `ExecutionPlan` does not /// have any particular output order here fn output_ordering(&self) -> Option<&[PhysicalSortExpr]>; /// Specifies the data distribution requirements for all the - /// children for this operator, By default it's [[Distribution::UnspecifiedDistribution]] for each child, + /// children for this `ExecutionPlan`, By default it's [[Distribution::UnspecifiedDistribution]] for each child, fn required_input_distribution(&self) -> Vec { vec![Distribution::UnspecifiedDistribution; self.children().len()] } - /// Specifies the ordering requirements for all of the children + /// Specifies the ordering required for all of the children of this + /// `ExecutionPlan`. + /// /// For each child, it's the local ordering requirement within /// each partition rather than the global ordering /// @@ -152,7 +166,7 @@ pub trait ExecutionPlan: Debug + DisplayAs + Send + Sync { vec![None; self.children().len()] } - /// Returns `false` if this operator's implementation may reorder + /// Returns `false` if this `ExecutionPlan`'s implementation may reorder /// rows within or between partitions. /// /// For example, Projection, Filter, and Limit maintain the order @@ -166,19 +180,21 @@ pub trait ExecutionPlan: Debug + DisplayAs + Send + Sync { /// The default implementation returns `false` /// /// WARNING: if you override this default, you *MUST* ensure that - /// the operator's maintains the ordering invariant or else + /// the `ExecutionPlan`'s maintains the ordering invariant or else /// DataFusion may produce incorrect results. fn maintains_input_order(&self) -> Vec { vec![false; self.children().len()] } - /// Specifies whether the operator benefits from increased parallelization - /// at its input for each child. If set to `true`, this indicates that the - /// operator would benefit from partitioning its corresponding child - /// (and thus from more parallelism). For operators that do very little work - /// the overhead of extra parallelism may outweigh any benefits + /// Specifies whether the `ExecutionPlan` benefits from increased + /// parallelization at its input for each child. /// - /// The default implementation returns `true` unless this operator + /// If returns `true`, the `ExecutionPlan` would benefit from partitioning + /// its corresponding child (and thus from more parallelism). For + /// `ExecutionPlan` that do very little work the overhead of extra + /// parallelism may outweigh any benefits + /// + /// The default implementation returns `true` unless this `ExecutionPlan` /// has signalled it requires a single child input partition. fn benefits_from_input_partitioning(&self) -> Vec { // By default try to maximize parallelism with more CPUs if @@ -199,12 +215,14 @@ pub trait ExecutionPlan: Debug + DisplayAs + Send + Sync { OrderingEquivalenceProperties::new(self.schema()) } - /// Get a list of child execution plans that provide the input for this plan. The returned list - /// will be empty for leaf nodes, will contain a single value for unary nodes, or two - /// values for binary nodes (such as joins). + /// Get a list of `ExecutionPlan` that provide input for this plan. The + /// returned list will be empty for leaf nodes such as scans, will contain a + /// single value for unary nodes, or two values for binary nodes (such as + /// joins). fn children(&self) -> Vec>; - /// Returns a new plan where all children were replaced by new plans. + /// Returns a new `ExecutionPlan` where all existing children were replaced + /// by the `children`, oi order fn with_new_children( self: Arc, children: Vec>, @@ -235,13 +253,40 @@ pub trait ExecutionPlan: Debug + DisplayAs + Send + Sync { Ok(None) } - /// Begin execution of `partition`, returning a stream of [`RecordBatch`]es. + /// Begin execution of `partition`, returning a [`Stream`] of + /// [`RecordBatch`]es. + /// + /// # Notes + /// + /// The `execute` method itself is not `async` but it returns an `async` + /// [`futures::stream::Stream`]. This `Stream` should incrementally compute + /// the output, `RecordBatch` by `RecordBatch` (in a streaming fashion). + /// Most `ExecutionPlan`s should not do any work before the first + /// `RecordBatch` is requested from the stream. + /// + /// [`RecordBatchStreamAdapter`] can be used to convert an `async` + /// [`Stream`] into a [`SendableRecordBatchStream`]. + /// + /// Using `async` `Streams` allows for network I/O during execution and + /// takes advantage of Rust's built in support for `async` continuations and + /// crate ecosystem. + /// + /// [`Stream`]: futures::stream::Stream + /// [`StreamExt`]: futures::stream::StreamExt + /// [`TryStreamExt`]: futures::stream::TryStreamExt + /// [`RecordBatchStreamAdapter`]: crate::stream::RecordBatchStreamAdapter /// /// # Implementation Examples /// - /// ## Return Precomputed Batch + /// While `async` `Stream`s have a non trivial learning curve, the + /// [`futures`] crate provides [`StreamExt`] and [`TryStreamExt`] + /// which help simplify many common operations. /// - /// We can return a precomputed batch as a stream + /// Here are some common patterns: + /// + /// ## Return Precomputed `RecordBatch` + /// + /// We can return a precomputed `RecordBatch` as a `Stream`: /// /// ``` /// # use std::sync::Arc; @@ -261,6 +306,7 @@ pub trait ExecutionPlan: Debug + DisplayAs + Send + Sync { /// partition: usize, /// context: Arc /// ) -> Result { + /// // use functions from futures crate convert the batch into a stream /// let fut = futures::future::ready(Ok(self.batch.clone())); /// let stream = futures::stream::once(fut); /// Ok(Box::pin(RecordBatchStreamAdapter::new(self.batch.schema(), stream))) @@ -268,9 +314,9 @@ pub trait ExecutionPlan: Debug + DisplayAs + Send + Sync { /// } /// ``` /// - /// ## Async Compute Batch + /// ## Lazily (async) Compute `RecordBatch` /// - /// We can also lazily compute a RecordBatch when the returned stream is polled + /// We can also lazily compute a `RecordBatch` when the returned `Stream` is polled /// /// ``` /// # use std::sync::Arc; @@ -284,6 +330,7 @@ pub trait ExecutionPlan: Debug + DisplayAs + Send + Sync { /// schema: SchemaRef, /// } /// + /// /// Returns a single batch when the returned stream is polled /// async fn get_batch() -> Result { /// todo!() /// } @@ -301,10 +348,10 @@ pub trait ExecutionPlan: Debug + DisplayAs + Send + Sync { /// } /// ``` /// - /// ## Async Compute Batch Stream + /// ## Lazily (async) create a Stream /// - /// We can lazily compute a RecordBatch stream when the returned stream is polled - /// flattening the result into a single stream + /// If you need to to create the return `Stream` using an `async` function, + /// you can do so by flattening the result: /// /// ``` /// # use std::sync::Arc; @@ -319,6 +366,7 @@ pub trait ExecutionPlan: Debug + DisplayAs + Send + Sync { /// schema: SchemaRef, /// } /// + /// /// async function that returns a stream /// async fn get_batch_stream() -> Result { /// todo!() /// } @@ -337,9 +385,6 @@ pub trait ExecutionPlan: Debug + DisplayAs + Send + Sync { /// } /// } /// ``` - /// - /// See [`futures::stream::StreamExt`] and [`futures::stream::TryStreamExt`] for further - /// combinators that can be used with streams fn execute( &self, partition: usize, @@ -372,7 +417,7 @@ pub trait ExecutionPlan: Debug + DisplayAs + Send + Sync { /// Indicate whether a data exchange is needed for the input of `plan`, which will be very helpful /// especially for the distributed engine to judge whether need to deal with shuffling. /// Currently there are 3 kinds of execution plan which needs data exchange -/// 1. RepartitionExec for changing the partition number between two operators +/// 1. RepartitionExec for changing the partition number between two `ExecutionPlan`s /// 2. CoalescePartitionsExec for collapsing all of the partitions into one without ordering guarantee /// 3. SortPreservingMergeExec for collapsing all of the sorted partitions into one with ordering guarantee pub fn need_data_exchange(plan: Arc) -> bool { From 0fa4ce963af7f87afc24ab6eb474c337ddd0018d Mon Sep 17 00:00:00 2001 From: Ruihang Xia Date: Fri, 3 Nov 2023 02:59:54 +0800 Subject: [PATCH 40/43] fix: clippy warnings from nightly rust 1.75 (#8025) Signed-off-by: Ruihang Xia --- datafusion/common/src/error.rs | 2 +- datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs | 1 + datafusion/core/tests/fuzz_cases/window_fuzz.rs | 1 + datafusion/optimizer/src/simplify_expressions/utils.rs | 2 +- 4 files changed, 4 insertions(+), 2 deletions(-) diff --git a/datafusion/common/src/error.rs b/datafusion/common/src/error.rs index adf58e282ed9..9114c669ab8b 100644 --- a/datafusion/common/src/error.rs +++ b/datafusion/common/src/error.rs @@ -369,7 +369,7 @@ impl From for io::Error { } impl DataFusionError { - const BACK_TRACE_SEP: &str = "\n\nbacktrace: "; + const BACK_TRACE_SEP: &'static str = "\n\nbacktrace: "; /// Get deepest underlying [`DataFusionError`] /// diff --git a/datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs b/datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs index 50d3610deed3..798f7ae8baf6 100644 --- a/datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs @@ -36,6 +36,7 @@ use datafusion_physical_expr::{AggregateExpr, PhysicalSortExpr}; use test_utils::add_empty_batches; #[cfg(test)] +#[allow(clippy::items_after_test_module)] mod tests { use super::*; diff --git a/datafusion/core/tests/fuzz_cases/window_fuzz.rs b/datafusion/core/tests/fuzz_cases/window_fuzz.rs index db940a9794a1..66f7374a51fb 100644 --- a/datafusion/core/tests/fuzz_cases/window_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/window_fuzz.rs @@ -45,6 +45,7 @@ use rand::rngs::StdRng; use rand::{Rng, SeedableRng}; #[cfg(test)] +#[allow(clippy::items_after_test_module)] mod tests { use super::*; diff --git a/datafusion/optimizer/src/simplify_expressions/utils.rs b/datafusion/optimizer/src/simplify_expressions/utils.rs index 28c61427c5ef..17e5d97c3006 100644 --- a/datafusion/optimizer/src/simplify_expressions/utils.rs +++ b/datafusion/optimizer/src/simplify_expressions/utils.rs @@ -525,8 +525,8 @@ pub fn simpl_concat_ws(delimiter: &Expr, args: &[Expr]) -> Result { d => Ok(concat_ws( d.clone(), args.iter() + .filter(|&x| !is_null(x)) .cloned() - .filter(|x| !is_null(x)) .collect::>(), )), } From 661d211a62b39a32e3aba29df55fc6ccd9fa1d22 Mon Sep 17 00:00:00 2001 From: Jay Zhan Date: Fri, 3 Nov 2023 04:03:58 +0800 Subject: [PATCH 41/43] Minor: Avoid recomputing compute_array_ndims in align_array_dimensions (#7963) * Refactor align_array_dimensions Signed-off-by: jayzhan211 * address comment Signed-off-by: jayzhan211 * remove unwrap Signed-off-by: jayzhan211 * address comment Signed-off-by: jayzhan211 * fix rebase Signed-off-by: jayzhan211 --------- Signed-off-by: jayzhan211 --- .../physical-expr/src/array_expressions.rs | 75 ++++++++++++++----- 1 file changed, 56 insertions(+), 19 deletions(-) diff --git a/datafusion/physical-expr/src/array_expressions.rs b/datafusion/physical-expr/src/array_expressions.rs index 18d8c60fe7bd..687502e79fed 100644 --- a/datafusion/physical-expr/src/array_expressions.rs +++ b/datafusion/physical-expr/src/array_expressions.rs @@ -725,35 +725,31 @@ pub fn array_prepend(args: &[ArrayRef]) -> Result { } fn align_array_dimensions(args: Vec) -> Result> { - // Find the maximum number of dimensions - let max_ndim: u64 = (*args + let args_ndim = args .iter() - .map(|arr| compute_array_ndims(Some(arr.clone()))) - .collect::>>>()? - .iter() - .max() - .unwrap()) - .unwrap(); + .map(|arg| compute_array_ndims(Some(arg.to_owned()))) + .collect::>>()? + .into_iter() + .map(|x| x.unwrap_or(0)) + .collect::>(); + let max_ndim = args_ndim.iter().max().unwrap_or(&0); // Align the dimensions of the arrays let aligned_args: Result> = args .into_iter() - .map(|array| { - let ndim = compute_array_ndims(Some(array.clone()))?.unwrap(); + .zip(args_ndim.iter()) + .map(|(array, ndim)| { if ndim < max_ndim { let mut aligned_array = array.clone(); for _ in 0..(max_ndim - ndim) { - let data_type = aligned_array.as_ref().data_type().clone(); - let offsets: Vec = - (0..downcast_arg!(aligned_array, ListArray).offsets().len()) - .map(|i| i as i32) - .collect(); - let field = Arc::new(Field::new("item", data_type, true)); + let data_type = aligned_array.data_type().to_owned(); + let array_lengths = vec![1; aligned_array.len()]; + let offsets = OffsetBuffer::::from_lengths(array_lengths); aligned_array = Arc::new(ListArray::try_new( - field, - OffsetBuffer::new(offsets.into()), - Arc::new(aligned_array.clone()), + Arc::new(Field::new("item", data_type, true)), + offsets, + aligned_array, None, )?) } @@ -1923,6 +1919,47 @@ mod tests { use arrow::datatypes::Int64Type; use datafusion_common::cast::as_uint64_array; + #[test] + fn test_align_array_dimensions() { + let array1d_1 = + Arc::new(ListArray::from_iter_primitive::(vec![ + Some(vec![Some(1), Some(2), Some(3)]), + Some(vec![Some(4), Some(5)]), + ])); + let array1d_2 = + Arc::new(ListArray::from_iter_primitive::(vec![ + Some(vec![Some(6), Some(7), Some(8)]), + ])); + + let array2d_1 = Arc::new(array_into_list_array(array1d_1.clone())) as ArrayRef; + let array2d_2 = Arc::new(array_into_list_array(array1d_2.clone())) as ArrayRef; + + let res = + align_array_dimensions(vec![array1d_1.to_owned(), array2d_2.to_owned()]) + .unwrap(); + + let expected = as_list_array(&array2d_1).unwrap(); + let expected_dim = compute_array_ndims(Some(array2d_1.to_owned())).unwrap(); + assert_ne!(as_list_array(&res[0]).unwrap(), expected); + assert_eq!( + compute_array_ndims(Some(res[0].clone())).unwrap(), + expected_dim + ); + + let array3d_1 = Arc::new(array_into_list_array(array2d_1)) as ArrayRef; + let array3d_2 = array_into_list_array(array2d_2.to_owned()); + let res = + align_array_dimensions(vec![array1d_1, Arc::new(array3d_2.clone())]).unwrap(); + + let expected = as_list_array(&array3d_1).unwrap(); + let expected_dim = compute_array_ndims(Some(array3d_1.to_owned())).unwrap(); + assert_ne!(as_list_array(&res[0]).unwrap(), expected); + assert_eq!( + compute_array_ndims(Some(res[0].clone())).unwrap(), + expected_dim + ); + } + #[test] fn test_array() { // make_array(1, 2, 3) = [1, 2, 3] From b2a16681838826ec409c93a26bd7a46262e9a5ef Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Fri, 3 Nov 2023 04:48:44 -0400 Subject: [PATCH 42/43] Minor: fix doc check (#8037) --- datafusion/core/src/datasource/file_format/arrow.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/datafusion/core/src/datasource/file_format/arrow.rs b/datafusion/core/src/datasource/file_format/arrow.rs index 2777805078c7..a9bd7d0e27bb 100644 --- a/datafusion/core/src/datasource/file_format/arrow.rs +++ b/datafusion/core/src/datasource/file_format/arrow.rs @@ -105,8 +105,8 @@ impl FileFormat for ArrowFormat { const ARROW_MAGIC: [u8; 6] = [b'A', b'R', b'R', b'O', b'W', b'1']; const CONTINUATION_MARKER: [u8; 4] = [0xff; 4]; -/// Custom implementation of inferring schema. Should eventually be moved upstream to arrow-rs. -/// See https://github.com/apache/arrow-rs/issues/5021 +/// Custom implementation of inferring schema. Should eventually be moved upstream to arrow-rs. +/// See async fn infer_schema_from_file_stream( mut stream: BoxStream<'static, object_store::Result>, ) -> Result { From 8c42d94851daa6f43053cd903222adfd1d81cb82 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Fri, 3 Nov 2023 05:19:24 -0400 Subject: [PATCH 43/43] Minor: remove uncessary #cfg test (#8036) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit * Minor: remove uncessary #cfg test * fmt * Update datafusion/core/src/datasource/file_format/arrow.rs Co-authored-by: Ruihang Xia --------- Co-authored-by: DaniĆ«l Heres Co-authored-by: Ruihang Xia --- .../core/tests/fuzz_cases/aggregate_fuzz.rs | 60 +++--- .../core/tests/fuzz_cases/window_fuzz.rs | 192 +++++++++--------- 2 files changed, 120 insertions(+), 132 deletions(-) diff --git a/datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs b/datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs index 798f7ae8baf6..821f236af87b 100644 --- a/datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs @@ -35,39 +35,33 @@ use datafusion_physical_expr::expressions::{col, Sum}; use datafusion_physical_expr::{AggregateExpr, PhysicalSortExpr}; use test_utils::add_empty_batches; -#[cfg(test)] -#[allow(clippy::items_after_test_module)] -mod tests { - use super::*; - - #[tokio::test(flavor = "multi_thread", worker_threads = 8)] - async fn aggregate_test() { - let test_cases = vec![ - vec!["a"], - vec!["b", "a"], - vec!["c", "a"], - vec!["c", "b", "a"], - vec!["d", "a"], - vec!["d", "b", "a"], - vec!["d", "c", "a"], - vec!["d", "c", "b", "a"], - ]; - let n = 300; - let distincts = vec![10, 20]; - for distinct in distincts { - let mut handles = Vec::new(); - for i in 0..n { - let test_idx = i % test_cases.len(); - let group_by_columns = test_cases[test_idx].clone(); - let job = tokio::spawn(run_aggregate_test( - make_staggered_batches::(1000, distinct, i as u64), - group_by_columns, - )); - handles.push(job); - } - for job in handles { - job.await.unwrap(); - } +#[tokio::test(flavor = "multi_thread", worker_threads = 8)] +async fn aggregate_test() { + let test_cases = vec![ + vec!["a"], + vec!["b", "a"], + vec!["c", "a"], + vec!["c", "b", "a"], + vec!["d", "a"], + vec!["d", "b", "a"], + vec!["d", "c", "a"], + vec!["d", "c", "b", "a"], + ]; + let n = 300; + let distincts = vec![10, 20]; + for distinct in distincts { + let mut handles = Vec::new(); + for i in 0..n { + let test_idx = i % test_cases.len(); + let group_by_columns = test_cases[test_idx].clone(); + let job = tokio::spawn(run_aggregate_test( + make_staggered_batches::(1000, distinct, i as u64), + group_by_columns, + )); + handles.push(job); + } + for job in handles { + job.await.unwrap(); } } } diff --git a/datafusion/core/tests/fuzz_cases/window_fuzz.rs b/datafusion/core/tests/fuzz_cases/window_fuzz.rs index 66f7374a51fb..9f70321318fd 100644 --- a/datafusion/core/tests/fuzz_cases/window_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/window_fuzz.rs @@ -44,108 +44,102 @@ use hashbrown::HashMap; use rand::rngs::StdRng; use rand::{Rng, SeedableRng}; -#[cfg(test)] -#[allow(clippy::items_after_test_module)] -mod tests { - use super::*; - - use datafusion_physical_plan::windows::PartitionSearchMode::{ - Linear, PartiallySorted, Sorted, - }; +use datafusion_physical_plan::windows::PartitionSearchMode::{ + Linear, PartiallySorted, Sorted, +}; - #[tokio::test(flavor = "multi_thread", worker_threads = 16)] - async fn window_bounded_window_random_comparison() -> Result<()> { - // make_staggered_batches gives result sorted according to a, b, c - // In the test cases first entry represents partition by columns - // Second entry represents order by columns. - // Third entry represents search mode. - // In sorted mode physical plans are in the form for WindowAggExec - //``` - // WindowAggExec - // MemoryExec] - // ``` - // and in the form for BoundedWindowAggExec - // ``` - // BoundedWindowAggExec - // MemoryExec - // ``` - // In Linear and PartiallySorted mode physical plans are in the form for WindowAggExec - //``` - // WindowAggExec - // SortExec(required by window function) - // MemoryExec] - // ``` - // and in the form for BoundedWindowAggExec - // ``` - // BoundedWindowAggExec - // MemoryExec - // ``` - let test_cases = vec![ - (vec!["a"], vec!["a"], Sorted), - (vec!["a"], vec!["b"], Sorted), - (vec!["a"], vec!["a", "b"], Sorted), - (vec!["a"], vec!["b", "c"], Sorted), - (vec!["a"], vec!["a", "b", "c"], Sorted), - (vec!["b"], vec!["a"], Linear), - (vec!["b"], vec!["a", "b"], Linear), - (vec!["b"], vec!["a", "c"], Linear), - (vec!["b"], vec!["a", "b", "c"], Linear), - (vec!["c"], vec!["a"], Linear), - (vec!["c"], vec!["a", "b"], Linear), - (vec!["c"], vec!["a", "c"], Linear), - (vec!["c"], vec!["a", "b", "c"], Linear), - (vec!["b", "a"], vec!["a"], Sorted), - (vec!["b", "a"], vec!["b"], Sorted), - (vec!["b", "a"], vec!["c"], Sorted), - (vec!["b", "a"], vec!["a", "b"], Sorted), - (vec!["b", "a"], vec!["b", "c"], Sorted), - (vec!["b", "a"], vec!["a", "c"], Sorted), - (vec!["b", "a"], vec!["a", "b", "c"], Sorted), - (vec!["c", "b"], vec!["a"], Linear), - (vec!["c", "b"], vec!["a", "b"], Linear), - (vec!["c", "b"], vec!["a", "c"], Linear), - (vec!["c", "b"], vec!["a", "b", "c"], Linear), - (vec!["c", "a"], vec!["a"], PartiallySorted(vec![1])), - (vec!["c", "a"], vec!["b"], PartiallySorted(vec![1])), - (vec!["c", "a"], vec!["c"], PartiallySorted(vec![1])), - (vec!["c", "a"], vec!["a", "b"], PartiallySorted(vec![1])), - (vec!["c", "a"], vec!["b", "c"], PartiallySorted(vec![1])), - (vec!["c", "a"], vec!["a", "c"], PartiallySorted(vec![1])), - ( - vec!["c", "a"], - vec!["a", "b", "c"], - PartiallySorted(vec![1]), - ), - (vec!["c", "b", "a"], vec!["a"], Sorted), - (vec!["c", "b", "a"], vec!["b"], Sorted), - (vec!["c", "b", "a"], vec!["c"], Sorted), - (vec!["c", "b", "a"], vec!["a", "b"], Sorted), - (vec!["c", "b", "a"], vec!["b", "c"], Sorted), - (vec!["c", "b", "a"], vec!["a", "c"], Sorted), - (vec!["c", "b", "a"], vec!["a", "b", "c"], Sorted), - ]; - let n = 300; - let n_distincts = vec![10, 20]; - for n_distinct in n_distincts { - let mut handles = Vec::new(); - for i in 0..n { - let idx = i % test_cases.len(); - let (pb_cols, ob_cols, search_mode) = test_cases[idx].clone(); - let job = tokio::spawn(run_window_test( - make_staggered_batches::(1000, n_distinct, i as u64), - i as u64, - pb_cols, - ob_cols, - search_mode, - )); - handles.push(job); - } - for job in handles { - job.await.unwrap()?; - } +#[tokio::test(flavor = "multi_thread", worker_threads = 16)] +async fn window_bounded_window_random_comparison() -> Result<()> { + // make_staggered_batches gives result sorted according to a, b, c + // In the test cases first entry represents partition by columns + // Second entry represents order by columns. + // Third entry represents search mode. + // In sorted mode physical plans are in the form for WindowAggExec + //``` + // WindowAggExec + // MemoryExec] + // ``` + // and in the form for BoundedWindowAggExec + // ``` + // BoundedWindowAggExec + // MemoryExec + // ``` + // In Linear and PartiallySorted mode physical plans are in the form for WindowAggExec + //``` + // WindowAggExec + // SortExec(required by window function) + // MemoryExec] + // ``` + // and in the form for BoundedWindowAggExec + // ``` + // BoundedWindowAggExec + // MemoryExec + // ``` + let test_cases = vec![ + (vec!["a"], vec!["a"], Sorted), + (vec!["a"], vec!["b"], Sorted), + (vec!["a"], vec!["a", "b"], Sorted), + (vec!["a"], vec!["b", "c"], Sorted), + (vec!["a"], vec!["a", "b", "c"], Sorted), + (vec!["b"], vec!["a"], Linear), + (vec!["b"], vec!["a", "b"], Linear), + (vec!["b"], vec!["a", "c"], Linear), + (vec!["b"], vec!["a", "b", "c"], Linear), + (vec!["c"], vec!["a"], Linear), + (vec!["c"], vec!["a", "b"], Linear), + (vec!["c"], vec!["a", "c"], Linear), + (vec!["c"], vec!["a", "b", "c"], Linear), + (vec!["b", "a"], vec!["a"], Sorted), + (vec!["b", "a"], vec!["b"], Sorted), + (vec!["b", "a"], vec!["c"], Sorted), + (vec!["b", "a"], vec!["a", "b"], Sorted), + (vec!["b", "a"], vec!["b", "c"], Sorted), + (vec!["b", "a"], vec!["a", "c"], Sorted), + (vec!["b", "a"], vec!["a", "b", "c"], Sorted), + (vec!["c", "b"], vec!["a"], Linear), + (vec!["c", "b"], vec!["a", "b"], Linear), + (vec!["c", "b"], vec!["a", "c"], Linear), + (vec!["c", "b"], vec!["a", "b", "c"], Linear), + (vec!["c", "a"], vec!["a"], PartiallySorted(vec![1])), + (vec!["c", "a"], vec!["b"], PartiallySorted(vec![1])), + (vec!["c", "a"], vec!["c"], PartiallySorted(vec![1])), + (vec!["c", "a"], vec!["a", "b"], PartiallySorted(vec![1])), + (vec!["c", "a"], vec!["b", "c"], PartiallySorted(vec![1])), + (vec!["c", "a"], vec!["a", "c"], PartiallySorted(vec![1])), + ( + vec!["c", "a"], + vec!["a", "b", "c"], + PartiallySorted(vec![1]), + ), + (vec!["c", "b", "a"], vec!["a"], Sorted), + (vec!["c", "b", "a"], vec!["b"], Sorted), + (vec!["c", "b", "a"], vec!["c"], Sorted), + (vec!["c", "b", "a"], vec!["a", "b"], Sorted), + (vec!["c", "b", "a"], vec!["b", "c"], Sorted), + (vec!["c", "b", "a"], vec!["a", "c"], Sorted), + (vec!["c", "b", "a"], vec!["a", "b", "c"], Sorted), + ]; + let n = 300; + let n_distincts = vec![10, 20]; + for n_distinct in n_distincts { + let mut handles = Vec::new(); + for i in 0..n { + let idx = i % test_cases.len(); + let (pb_cols, ob_cols, search_mode) = test_cases[idx].clone(); + let job = tokio::spawn(run_window_test( + make_staggered_batches::(1000, n_distinct, i as u64), + i as u64, + pb_cols, + ob_cols, + search_mode, + )); + handles.push(job); + } + for job in handles { + job.await.unwrap()?; } - Ok(()) } + Ok(()) } fn get_random_function(