From 780639329f1664cce184165a9d4653319b893cb0 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Fri, 23 Aug 2024 00:12:24 -0700 Subject: [PATCH 001/154] fix: Produce buffered null join row only if all joined rows are failed on join filter in SMJ full join (#12090) * Add test case * fix * Update test * fix * Remove duplicate * fix clippy * fix clippy again * For review --- .../src/joins/sort_merge_join.rs | 48 +++++++++++-------- .../test_files/sort_merge_join.slt | 27 ++++++++++- 2 files changed, 53 insertions(+), 22 deletions(-) diff --git a/datafusion/physical-plan/src/joins/sort_merge_join.rs b/datafusion/physical-plan/src/joins/sort_merge_join.rs index 96d5ba728a30..b5994d16d2ee 100644 --- a/datafusion/physical-plan/src/joins/sort_merge_join.rs +++ b/datafusion/physical-plan/src/joins/sort_merge_join.rs @@ -22,7 +22,7 @@ use std::any::Any; use std::cmp::Ordering; -use std::collections::VecDeque; +use std::collections::{HashMap, VecDeque}; use std::fmt::Formatter; use std::fs::File; use std::io::BufReader; @@ -595,8 +595,10 @@ struct BufferedBatch { /// Size estimation used for reserving / releasing memory pub size_estimation: usize, /// The indices of buffered batch that failed the join filter. + /// This is a map between buffered row index and a boolean value indicating whether all joined row + /// of the buffered row failed the join filter. /// When dequeuing the buffered batch, we need to produce null joined rows for these indices. - pub join_filter_failed_idxs: HashSet, + pub join_filter_failed_map: HashMap, /// Current buffered batch number of rows. Equal to batch.num_rows() /// but if batch is spilled to disk this property is preferable /// and less expensive @@ -637,7 +639,7 @@ impl BufferedBatch { join_arrays, null_joined: vec![], size_estimation, - join_filter_failed_idxs: HashSet::new(), + join_filter_failed_map: HashMap::new(), num_rows, spill_file: None, } @@ -1229,11 +1231,19 @@ impl SMJStream { } buffered_batch.null_joined.clear(); - // For buffered rows which are joined with streamed side but doesn't satisfy the join filter + // For buffered row which is joined with streamed side rows but all joined rows + // don't satisfy the join filter if output_not_matched_filter { + let not_matched_buffered_indices = buffered_batch + .join_filter_failed_map + .iter() + .filter_map(|(idx, failed)| if *failed { Some(*idx) } else { None }) + .collect::>(); + let buffered_indices = UInt64Array::from_iter_values( - buffered_batch.join_filter_failed_idxs.iter().copied(), + not_matched_buffered_indices.iter().copied(), ); + if let Some(record_batch) = produce_buffered_null_batch( &self.schema, &self.streamed_schema, @@ -1242,7 +1252,7 @@ impl SMJStream { )? { self.output_record_batches.push(record_batch); } - buffered_batch.join_filter_failed_idxs.clear(); + buffered_batch.join_filter_failed_map.clear(); } } Ok(()) @@ -1459,24 +1469,20 @@ impl SMJStream { // If it is joined with streamed side, but doesn't match the join filter, // we need to output it with nulls as streamed side. if matches!(self.join_type, JoinType::Full) { + let buffered_batch = &mut self.buffered_data.batches + [chunk.buffered_batch_idx.unwrap()]; + for i in 0..pre_mask.len() { - let buffered_batch = &mut self.buffered_data.batches - [chunk.buffered_batch_idx.unwrap()]; let buffered_index = buffered_indices.value(i); - if !pre_mask.value(i) { - // For a buffered row that is joined with streamed side but doesn't satisfy the join filter, - buffered_batch - .join_filter_failed_idxs - .insert(buffered_index); - } else if buffered_batch - .join_filter_failed_idxs - .contains(&buffered_index) - { - buffered_batch - .join_filter_failed_idxs - .remove(&buffered_index); - } + buffered_batch.join_filter_failed_map.insert( + buffered_index, + *buffered_batch + .join_filter_failed_map + .get(&buffered_index) + .unwrap_or(&true) + && !pre_mask.value(i), + ); } } } diff --git a/datafusion/sqllogictest/test_files/sort_merge_join.slt b/datafusion/sqllogictest/test_files/sort_merge_join.slt index ea3088e69674..2c28a5feadba 100644 --- a/datafusion/sqllogictest/test_files/sort_merge_join.slt +++ b/datafusion/sqllogictest/test_files/sort_merge_join.slt @@ -134,7 +134,6 @@ Alice 100 NULL NULL Alice 50 Alice 2 Bob 1 NULL NULL NULL NULL Alice 1 -NULL NULL Alice 2 query TITI rowsort SELECT * FROM t1 FULL JOIN t2 ON t1.a = t2.a AND t1.b > t2.b + 50 @@ -624,6 +623,32 @@ NULL NULL 7 9 NULL NULL 8 10 NULL NULL 9 11 +query IIII +select * from ( +with t as ( + select id_a id_a_1, id_a % 5 id_a_2 from (select unnest(make_array(5, 6, 7, 8, 9, 0, 1, 2, 3, 4)) id_a) +), t1 as ( + select id_b % 10 id_b_1, id_b + 2 id_b_2 from (select unnest(make_array(0, 1, 2, 3, 4, 5, 6, 7, 8, 9)) id_b) +) +select * from t full join t1 on t.id_a_2 = t1.id_b_1 and t.id_a_1 > t1.id_b_2 +) order by 1, 2, 3, 4 +---- +0 0 NULL NULL +1 1 NULL NULL +2 2 NULL NULL +3 3 NULL NULL +4 4 NULL NULL +5 0 0 2 +6 1 1 3 +7 2 2 4 +8 3 3 5 +9 4 4 6 +NULL NULL 5 7 +NULL NULL 6 8 +NULL NULL 7 9 +NULL NULL 8 10 +NULL NULL 9 11 + # return sql params back to default values statement ok set datafusion.optimizer.prefer_hash_join = true; From 3de50c833b0e4bbb688b02240d11a1d5092c0d36 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Dani=C3=ABl=20Heres?= Date: Fri, 23 Aug 2024 10:24:59 +0200 Subject: [PATCH 002/154] [minor] Use Vec instead of primitive builders (#12121) * Use vec instead of builder * Compile * Use vec instead of builder * Revert --- .../src/aggregate/groups_accumulator.rs | 8 +++--- .../physical-plan/src/joins/hash_join.rs | 25 +++++++----------- .../src/joins/nested_loop_join.rs | 25 ++++++++---------- .../src/joins/symmetric_hash_join.rs | 10 +++---- datafusion/physical-plan/src/joins/utils.rs | 26 ++++++++----------- .../physical-plan/src/repartition/mod.rs | 14 +++++----- 6 files changed, 46 insertions(+), 62 deletions(-) diff --git a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs index 3984b02c5fbb..1c97d22ec79c 100644 --- a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs +++ b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs @@ -24,7 +24,7 @@ pub mod nulls; pub mod prim_op; use arrow::{ - array::{ArrayRef, AsArray, BooleanArray, PrimitiveArray, UInt32Builder}, + array::{ArrayRef, AsArray, BooleanArray, PrimitiveArray}, compute, datatypes::UInt32Type, }; @@ -170,7 +170,7 @@ impl GroupsAccumulatorAdapter { let mut groups_with_rows = vec![]; // batch_indices holds indices into values, each group is contiguous - let mut batch_indices = UInt32Builder::with_capacity(0); + let mut batch_indices = vec![]; // offsets[i] is index into batch_indices where the rows for // group_index i starts @@ -184,11 +184,11 @@ impl GroupsAccumulatorAdapter { } groups_with_rows.push(group_index); - batch_indices.append_slice(indices); + batch_indices.extend_from_slice(indices); offset_so_far += indices.len(); offsets.push(offset_so_far); } - let batch_indices = batch_indices.finish(); + let batch_indices = batch_indices.into(); // reorder the values and opt_filter by batch_indices so that // all values for each group are contiguous, then invoke the diff --git a/datafusion/physical-plan/src/joins/hash_join.rs b/datafusion/physical-plan/src/joins/hash_join.rs index e40a07cf6220..7fac23ad5557 100644 --- a/datafusion/physical-plan/src/joins/hash_join.rs +++ b/datafusion/physical-plan/src/joins/hash_join.rs @@ -49,8 +49,7 @@ use crate::{ }; use arrow::array::{ - Array, ArrayRef, BooleanArray, BooleanBufferBuilder, PrimitiveArray, UInt32Array, - UInt64Array, + Array, ArrayRef, BooleanArray, BooleanBufferBuilder, UInt32Array, UInt64Array, }; use arrow::compute::kernels::cmp::{eq, not_distinct}; use arrow::compute::{and, concat_batches, take, FilterBuilder}; @@ -1204,13 +1203,11 @@ fn lookup_join_hashmap( }) .collect::>>()?; - let (mut probe_builder, mut build_builder, next_offset) = build_hashmap + let (probe_indices, build_indices, next_offset) = build_hashmap .get_matched_indices_with_limit_offset(hashes_buffer, None, limit, offset); - let build_indices: UInt64Array = - PrimitiveArray::new(build_builder.finish().into(), None); - let probe_indices: UInt32Array = - PrimitiveArray::new(probe_builder.finish().into(), None); + let build_indices: UInt64Array = build_indices.into(); + let probe_indices: UInt32Array = probe_indices.into(); let (build_indices, probe_indices) = equal_rows_arr( &build_indices, @@ -1566,7 +1563,7 @@ mod tests { test::build_table_i32, test::exec::MockExec, }; - use arrow::array::{Date32Array, Int32Array, UInt32Builder, UInt64Builder}; + use arrow::array::{Date32Array, Int32Array}; use arrow::datatypes::{DataType, Field}; use arrow_array::StructArray; use arrow_buffer::NullBuffer; @@ -3169,17 +3166,13 @@ mod tests { (0, None), )?; - let mut left_ids = UInt64Builder::with_capacity(0); - left_ids.append_value(0); - left_ids.append_value(1); + let left_ids: UInt64Array = vec![0, 1].into(); - let mut right_ids = UInt32Builder::with_capacity(0); - right_ids.append_value(0); - right_ids.append_value(1); + let right_ids: UInt32Array = vec![0, 1].into(); - assert_eq!(left_ids.finish(), l); + assert_eq!(left_ids, l); - assert_eq!(right_ids.finish(), r); + assert_eq!(right_ids, r); Ok(()) } diff --git a/datafusion/physical-plan/src/joins/nested_loop_join.rs b/datafusion/physical-plan/src/joins/nested_loop_join.rs index 04a025c93288..18de2de03192 100644 --- a/datafusion/physical-plan/src/joins/nested_loop_join.rs +++ b/datafusion/physical-plan/src/joins/nested_loop_join.rs @@ -40,13 +40,12 @@ use crate::{ RecordBatchStream, SendableRecordBatchStream, }; -use arrow::array::{ - BooleanBufferBuilder, UInt32Array, UInt32Builder, UInt64Array, UInt64Builder, -}; +use arrow::array::{BooleanBufferBuilder, UInt32Array, UInt64Array}; use arrow::compute::concat_batches; -use arrow::datatypes::{Schema, SchemaRef}; +use arrow::datatypes::{Schema, SchemaRef, UInt64Type}; use arrow::record_batch::RecordBatch; use arrow::util::bit_util; +use arrow_array::PrimitiveArray; use datafusion_common::{exec_datafusion_err, JoinSide, Result, Statistics}; use datafusion_execution::memory_pool::{MemoryConsumer, MemoryReservation}; use datafusion_execution::TaskContext; @@ -573,23 +572,21 @@ fn join_left_and_right_batch( ) })?; - let mut left_indices_builder = UInt64Builder::new(); - let mut right_indices_builder = UInt32Builder::new(); + let mut left_indices_builder: Vec = vec![]; + let mut right_indices_builder: Vec = vec![]; for (left_side, right_side) in indices { - left_indices_builder - .append_values(left_side.values(), &vec![true; left_side.len()]); - right_indices_builder - .append_values(right_side.values(), &vec![true; right_side.len()]); + left_indices_builder.extend(left_side.values()); + right_indices_builder.extend(right_side.values()); } - let left_side = left_indices_builder.finish(); - let right_side = right_indices_builder.finish(); + let left_side: PrimitiveArray = left_indices_builder.into(); + let right_side = right_indices_builder.into(); // set the left bitmap // and only full join need the left bitmap if need_produce_result_in_final(join_type) { let mut bitmap = visited_left_side.lock(); - left_side.iter().flatten().for_each(|x| { - bitmap.set_bit(x as usize, true); + left_side.values().iter().for_each(|x| { + bitmap.set_bit(*x as usize, true); }); } // adjust the two side indices base on the join type diff --git a/datafusion/physical-plan/src/joins/symmetric_hash_join.rs b/datafusion/physical-plan/src/joins/symmetric_hash_join.rs index 1bf2ef2fd5f7..7dab664502e9 100644 --- a/datafusion/physical-plan/src/joins/symmetric_hash_join.rs +++ b/datafusion/physical-plan/src/joins/symmetric_hash_join.rs @@ -929,13 +929,11 @@ fn lookup_join_hashmap( let (mut matched_probe, mut matched_build) = build_hashmap .get_matched_indices(hash_values.iter().enumerate().rev(), deleted_offset); - matched_probe.as_slice_mut().reverse(); - matched_build.as_slice_mut().reverse(); + matched_probe.reverse(); + matched_build.reverse(); - let build_indices: UInt64Array = - PrimitiveArray::new(matched_build.finish().into(), None); - let probe_indices: UInt32Array = - PrimitiveArray::new(matched_probe.finish().into(), None); + let build_indices: UInt64Array = matched_build.into(); + let probe_indices: UInt32Array = matched_probe.into(); let (build_indices, probe_indices) = equal_rows_arr( &build_indices, diff --git a/datafusion/physical-plan/src/joins/utils.rs b/datafusion/physical-plan/src/joins/utils.rs index 80d8815bdebc..8fdbf7041e2f 100644 --- a/datafusion/physical-plan/src/joins/utils.rs +++ b/datafusion/physical-plan/src/joins/utils.rs @@ -31,7 +31,7 @@ use crate::{ use arrow::array::{ downcast_array, new_null_array, Array, BooleanBufferBuilder, UInt32Array, - UInt32BufferBuilder, UInt32Builder, UInt64Array, UInt64BufferBuilder, + UInt32Builder, UInt64Array, }; use arrow::compute; use arrow::datatypes::{Field, Schema, SchemaBuilder, UInt32Type, UInt64Type}; @@ -163,8 +163,8 @@ macro_rules! chain_traverse { } else { i }; - $match_indices.append(match_row_idx); - $input_indices.append($input_idx as u32); + $match_indices.push(match_row_idx); + $input_indices.push($input_idx as u32); $remaining_output -= 1; // Follow the chain to get the next index value let next = $next_chain[match_row_idx as usize]; @@ -238,9 +238,9 @@ pub trait JoinHashMapType { &self, iter: impl Iterator, deleted_offset: Option, - ) -> (UInt32BufferBuilder, UInt64BufferBuilder) { - let mut input_indices = UInt32BufferBuilder::new(0); - let mut match_indices = UInt64BufferBuilder::new(0); + ) -> (Vec, Vec) { + let mut input_indices = vec![]; + let mut match_indices = vec![]; let hash_map = self.get_map(); let next_chain = self.get_list(); @@ -261,8 +261,8 @@ pub trait JoinHashMapType { } else { i }; - match_indices.append(match_row_idx); - input_indices.append(row_idx as u32); + match_indices.push(match_row_idx); + input_indices.push(row_idx as u32); // Follow the chain to get the next index value let next = next_chain[match_row_idx as usize]; if next == 0 { @@ -289,13 +289,9 @@ pub trait JoinHashMapType { deleted_offset: Option, limit: usize, offset: JoinHashMapOffset, - ) -> ( - UInt32BufferBuilder, - UInt64BufferBuilder, - Option, - ) { - let mut input_indices = UInt32BufferBuilder::new(0); - let mut match_indices = UInt64BufferBuilder::new(0); + ) -> (Vec, Vec, Option) { + let mut input_indices = vec![]; + let mut match_indices = vec![]; let mut remaining_output = limit; diff --git a/datafusion/physical-plan/src/repartition/mod.rs b/datafusion/physical-plan/src/repartition/mod.rs index 656d82215bbe..5a3fcb5029e1 100644 --- a/datafusion/physical-plan/src/repartition/mod.rs +++ b/datafusion/physical-plan/src/repartition/mod.rs @@ -38,9 +38,10 @@ use crate::sorts::streaming_merge; use crate::stream::RecordBatchStreamAdapter; use crate::{DisplayFormatType, ExecutionPlan, Partitioning, PlanProperties, Statistics}; -use arrow::array::{ArrayRef, UInt64Builder}; -use arrow::datatypes::SchemaRef; +use arrow::array::ArrayRef; +use arrow::datatypes::{SchemaRef, UInt64Type}; use arrow::record_batch::RecordBatch; +use arrow_array::PrimitiveArray; use datafusion_common::utils::transpose; use datafusion_common::{arrow_datafusion_err, not_impl_err, DataFusionError, Result}; use datafusion_common_runtime::SpawnedTask; @@ -275,12 +276,11 @@ impl BatchPartitioner { create_hashes(&arrays, random_state, hash_buffer)?; let mut indices: Vec<_> = (0..*partitions) - .map(|_| UInt64Builder::with_capacity(batch.num_rows())) + .map(|_| Vec::with_capacity(batch.num_rows())) .collect(); for (index, hash) in hash_buffer.iter().enumerate() { - indices[(*hash % *partitions as u64) as usize] - .append_value(index as u64); + indices[(*hash % *partitions as u64) as usize].push(index as u64); } // Finished building index-arrays for output partitions @@ -291,8 +291,8 @@ impl BatchPartitioner { let it = indices .into_iter() .enumerate() - .filter_map(|(partition, mut indices)| { - let indices = indices.finish(); + .filter_map(|(partition, indices)| { + let indices: PrimitiveArray = indices.into(); (!indices.is_empty()).then_some((partition, indices)) }) .map(move |(partition, indices)| { From 7be9897237b0003dc66b70831c48f0a8b11c3e56 Mon Sep 17 00:00:00 2001 From: wiedld Date: Fri, 23 Aug 2024 09:06:41 -0700 Subject: [PATCH 003/154] Fix thread panic when "unreachable" SpawnedTask code is reachable. (#12086) * test: demonstrate that the unreachable in SpawnedTask is reachable * chore: use workspace tokio and add feature * fix(12089): SpawnedTask will no longer panic during shutdown * chore(12089): add new error type for JoinError * refactor(12089): handle join error when using SpawnedTask::join_unwind * Revert "chore: use workspace tokio and add feature" This reverts commit 3010288d50a7dd3435f14a74bae6cc28b67f91ad. * refactor(12089): update test to avoid the looping and global (to package tests) panic hook manipulation * refactor(12089): make single conditional for unwind vs no-unwind, and update test for cancellation error --- datafusion-cli/Cargo.lock | 2 + datafusion/common-runtime/Cargo.toml | 4 ++ datafusion/common-runtime/src/common.rs | 42 +++++++++++++++++-- datafusion/common/Cargo.toml | 1 + datafusion/common/src/error.rs | 8 ++++ .../core/src/datasource/file_format/arrow.rs | 5 ++- .../src/datasource/file_format/parquet.rs | 18 ++++++-- .../file_format/write/orchestration.rs | 4 +- datafusion/core/src/datasource/stream.rs | 5 ++- 9 files changed, 77 insertions(+), 12 deletions(-) diff --git a/datafusion-cli/Cargo.lock b/datafusion-cli/Cargo.lock index e35eb3906b9a..456dc295d449 100644 --- a/datafusion-cli/Cargo.lock +++ b/datafusion-cli/Cargo.lock @@ -1295,12 +1295,14 @@ dependencies = [ "parquet", "paste", "sqlparser", + "tokio", ] [[package]] name = "datafusion-common-runtime" version = "41.0.0" dependencies = [ + "log", "tokio", ] diff --git a/datafusion/common-runtime/Cargo.toml b/datafusion/common-runtime/Cargo.toml index c10436087675..a21c72cd9f83 100644 --- a/datafusion/common-runtime/Cargo.toml +++ b/datafusion/common-runtime/Cargo.toml @@ -36,4 +36,8 @@ name = "datafusion_common_runtime" path = "src/lib.rs" [dependencies] +log = { workspace = true } tokio = { workspace = true } + +[dev-dependencies] +tokio = { version = "1.36", features = ["rt", "rt-multi-thread", "time"] } diff --git a/datafusion/common-runtime/src/common.rs b/datafusion/common-runtime/src/common.rs index 2f7ddb972f42..698a846b4844 100644 --- a/datafusion/common-runtime/src/common.rs +++ b/datafusion/common-runtime/src/common.rs @@ -60,8 +60,8 @@ impl SpawnedTask { } /// Joins the task and unwinds the panic if it happens. - pub async fn join_unwind(self) -> R { - self.join().await.unwrap_or_else(|e| { + pub async fn join_unwind(self) -> Result { + self.join().await.map_err(|e| { // `JoinError` can be caused either by panic or cancellation. We have to handle panics: if e.is_panic() { std::panic::resume_unwind(e.into_panic()); @@ -69,9 +69,43 @@ impl SpawnedTask { // Cancellation may be caused by two reasons: // 1. Abort is called, but since we consumed `self`, it's not our case (`JoinHandle` not accessible outside). // 2. The runtime is shutting down. - // So we consider this branch as unreachable. - unreachable!("SpawnedTask was cancelled unexpectedly"); + log::warn!("SpawnedTask was polled during shutdown"); + e } }) } } + +#[cfg(test)] +mod tests { + use super::*; + + use std::future::{pending, Pending}; + + use tokio::runtime::Runtime; + + #[tokio::test] + async fn runtime_shutdown() { + let rt = Runtime::new().unwrap(); + let task = rt + .spawn(async { + SpawnedTask::spawn(async { + let fut: Pending<()> = pending(); + fut.await; + unreachable!("should never return"); + }) + }) + .await + .unwrap(); + + // caller shutdown their DF runtime (e.g. timeout, error in caller, etc) + rt.shutdown_background(); + + // race condition + // poll occurs during shutdown (buffered stream poll calls, etc) + assert!(matches!( + task.join_unwind().await, + Err(e) if e.is_cancelled() + )); + } +} diff --git a/datafusion/common/Cargo.toml b/datafusion/common/Cargo.toml index 8435d0632576..79e20ba1215c 100644 --- a/datafusion/common/Cargo.toml +++ b/datafusion/common/Cargo.toml @@ -63,6 +63,7 @@ parquet = { workspace = true, optional = true, default-features = true } paste = "1.0.15" pyo3 = { version = "0.21.0", optional = true } sqlparser = { workspace = true } +tokio = { workspace = true } [target.'cfg(target_family = "wasm")'.dependencies] instant = { version = "0.1", features = ["wasm-bindgen"] } diff --git a/datafusion/common/src/error.rs b/datafusion/common/src/error.rs index 27a25d0c9dd5..05988d6c6da4 100644 --- a/datafusion/common/src/error.rs +++ b/datafusion/common/src/error.rs @@ -34,6 +34,7 @@ use arrow::error::ArrowError; #[cfg(feature = "parquet")] use parquet::errors::ParquetError; use sqlparser::parser::ParserError; +use tokio::task::JoinError; /// Result type for operations that could result in an [DataFusionError] pub type Result = result::Result; @@ -112,6 +113,10 @@ pub enum DataFusionError { /// SQL method, opened a CSV file that is broken, or tried to divide an /// integer by zero. Execution(String), + /// [`JoinError`] during execution of the query. + /// + /// This error can unoccur for unjoined tasks, such as execution shutdown. + ExecutionJoin(JoinError), /// Error when resources (such as memory of scratch disk space) are exhausted. /// /// This error is thrown when a consumer cannot acquire additional memory @@ -306,6 +311,7 @@ impl Error for DataFusionError { DataFusionError::Plan(_) => None, DataFusionError::SchemaError(e, _) => Some(e), DataFusionError::Execution(_) => None, + DataFusionError::ExecutionJoin(e) => Some(e), DataFusionError::ResourcesExhausted(_) => None, DataFusionError::External(e) => Some(e.as_ref()), DataFusionError::Context(_, e) => Some(e.as_ref()), @@ -418,6 +424,7 @@ impl DataFusionError { DataFusionError::Configuration(_) => "Invalid or Unsupported Configuration: ", DataFusionError::SchemaError(_, _) => "Schema error: ", DataFusionError::Execution(_) => "Execution error: ", + DataFusionError::ExecutionJoin(_) => "ExecutionJoin error: ", DataFusionError::ResourcesExhausted(_) => "Resources exhausted: ", DataFusionError::External(_) => "External error: ", DataFusionError::Context(_, _) => "", @@ -453,6 +460,7 @@ impl DataFusionError { Cow::Owned(format!("{desc}{backtrace}")) } DataFusionError::Execution(ref desc) => Cow::Owned(desc.to_string()), + DataFusionError::ExecutionJoin(ref desc) => Cow::Owned(desc.to_string()), DataFusionError::ResourcesExhausted(ref desc) => Cow::Owned(desc.to_string()), DataFusionError::External(ref desc) => Cow::Owned(desc.to_string()), #[cfg(feature = "object_store")] diff --git a/datafusion/core/src/datasource/file_format/arrow.rs b/datafusion/core/src/datasource/file_format/arrow.rs index 8b6a8800119d..95f76195e63d 100644 --- a/datafusion/core/src/datasource/file_format/arrow.rs +++ b/datafusion/core/src/datasource/file_format/arrow.rs @@ -341,7 +341,10 @@ impl DataSink for ArrowFileSink { } } - demux_task.join_unwind().await?; + demux_task + .join_unwind() + .await + .map_err(DataFusionError::ExecutionJoin)??; Ok(row_count as u64) } } diff --git a/datafusion/core/src/datasource/file_format/parquet.rs b/datafusion/core/src/datasource/file_format/parquet.rs index f233f3842c8c..83f77ca9371a 100644 --- a/datafusion/core/src/datasource/file_format/parquet.rs +++ b/datafusion/core/src/datasource/file_format/parquet.rs @@ -836,7 +836,10 @@ impl DataSink for ParquetSink { } } - demux_task.join_unwind().await?; + demux_task + .join_unwind() + .await + .map_err(DataFusionError::ExecutionJoin)??; Ok(row_count as u64) } @@ -942,7 +945,10 @@ fn spawn_rg_join_and_finalize_task( let num_cols = column_writer_tasks.len(); let mut finalized_rg = Vec::with_capacity(num_cols); for task in column_writer_tasks.into_iter() { - let (writer, _col_reservation) = task.join_unwind().await?; + let (writer, _col_reservation) = task + .join_unwind() + .await + .map_err(DataFusionError::ExecutionJoin)??; let encoded_size = writer.get_estimated_total_bytes(); rg_reservation.grow(encoded_size); finalized_rg.push(writer.close()?); @@ -1070,7 +1076,8 @@ async fn concatenate_parallel_row_groups( while let Some(task) = serialize_rx.recv().await { let result = task.join_unwind().await; let mut rg_out = parquet_writer.next_row_group()?; - let (serialized_columns, mut rg_reservation, _cnt) = result?; + let (serialized_columns, mut rg_reservation, _cnt) = + result.map_err(DataFusionError::ExecutionJoin)??; for chunk in serialized_columns { chunk.append_to_row_group(&mut rg_out)?; rg_reservation.free(); @@ -1134,7 +1141,10 @@ async fn output_single_parquet_file_parallelized( ) .await?; - launch_serialization_task.join_unwind().await?; + launch_serialization_task + .join_unwind() + .await + .map_err(DataFusionError::ExecutionJoin)??; Ok(file_metadata) } diff --git a/datafusion/core/src/datasource/file_format/write/orchestration.rs b/datafusion/core/src/datasource/file_format/write/orchestration.rs index 1d32063ee9f3..6f27e6f3889f 100644 --- a/datafusion/core/src/datasource/file_format/write/orchestration.rs +++ b/datafusion/core/src/datasource/file_format/write/orchestration.rs @@ -298,8 +298,8 @@ pub(crate) async fn stateless_multipart_put( write_coordinator_task.join_unwind(), demux_task.join_unwind() ); - r1?; - r2?; + r1.map_err(DataFusionError::ExecutionJoin)??; + r2.map_err(DataFusionError::ExecutionJoin)??; let total_count = rx_row_cnt.await.map_err(|_| { internal_datafusion_err!("Did not receive row count from write coordinator") diff --git a/datafusion/core/src/datasource/stream.rs b/datafusion/core/src/datasource/stream.rs index 682565aea909..b53fe8663178 100644 --- a/datafusion/core/src/datasource/stream.rs +++ b/datafusion/core/src/datasource/stream.rs @@ -438,6 +438,9 @@ impl DataSink for StreamWrite { } } drop(sender); - write_task.join_unwind().await + write_task + .join_unwind() + .await + .map_err(DataFusionError::ExecutionJoin)? } } From 8fd9d69b3996a0bfd86c8b0cd4473a7b98555d87 Mon Sep 17 00:00:00 2001 From: Huaijin Date: Sat, 24 Aug 2024 00:09:47 +0800 Subject: [PATCH 004/154] fix: single partition in SortPreservingMergeExec with fetch (#12109) --- .../src/sorts/sort_preserving_merge.rs | 97 +++++++++++++++++-- 1 file changed, 91 insertions(+), 6 deletions(-) diff --git a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs index 7ba1d77aea4e..f34701d0d713 100644 --- a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs +++ b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs @@ -22,6 +22,7 @@ use std::sync::Arc; use crate::common::spawn_buffered; use crate::expressions::PhysicalSortExpr; +use crate::limit::LimitStream; use crate::metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet}; use crate::sorts::streaming_merge; use crate::{ @@ -238,12 +239,23 @@ impl ExecutionPlan for SortPreservingMergeExec { 0 => internal_err!( "SortPreservingMergeExec requires at least one input partition" ), - 1 => { - // bypass if there is only one partition to merge (no metrics in this case either) - let result = self.input.execute(0, context); - debug!("Done getting stream for SortPreservingMergeExec::execute with 1 input"); - result - } + 1 => match self.fetch { + Some(fetch) => { + let stream = self.input.execute(0, context)?; + debug!("Done getting stream for SortPreservingMergeExec::execute with 1 input with {fetch}"); + Ok(Box::pin(LimitStream::new( + stream, + 0, + Some(fetch), + BaselineMetrics::new(&self.metrics, partition), + ))) + } + None => { + let stream = self.input.execute(0, context); + debug!("Done getting stream for SortPreservingMergeExec::execute with 1 input without fetch"); + stream + } + }, _ => { let receivers = (0..input_partitions) .map(|partition| { @@ -817,6 +829,79 @@ mod tests { ); } + #[tokio::test] + async fn test_sort_merge_single_partition_with_fetch() { + let task_ctx = Arc::new(TaskContext::default()); + let a: ArrayRef = Arc::new(Int32Array::from(vec![1, 2, 7, 9, 3])); + let b: ArrayRef = Arc::new(StringArray::from(vec!["a", "b", "c", "d", "e"])); + let batch = RecordBatch::try_from_iter(vec![("a", a), ("b", b)]).unwrap(); + let schema = batch.schema(); + + let sort = vec![PhysicalSortExpr { + expr: col("b", &schema).unwrap(), + options: SortOptions { + descending: false, + nulls_first: true, + }, + }]; + let exec = MemoryExec::try_new(&[vec![batch]], schema, None).unwrap(); + let merge = Arc::new( + SortPreservingMergeExec::new(sort, Arc::new(exec)).with_fetch(Some(2)), + ); + + let collected = collect(merge, task_ctx).await.unwrap(); + assert_eq!(collected.len(), 1); + + assert_batches_eq!( + &[ + "+---+---+", + "| a | b |", + "+---+---+", + "| 1 | a |", + "| 2 | b |", + "+---+---+", + ], + collected.as_slice() + ); + } + + #[tokio::test] + async fn test_sort_merge_single_partition_without_fetch() { + let task_ctx = Arc::new(TaskContext::default()); + let a: ArrayRef = Arc::new(Int32Array::from(vec![1, 2, 7, 9, 3])); + let b: ArrayRef = Arc::new(StringArray::from(vec!["a", "b", "c", "d", "e"])); + let batch = RecordBatch::try_from_iter(vec![("a", a), ("b", b)]).unwrap(); + let schema = batch.schema(); + + let sort = vec![PhysicalSortExpr { + expr: col("b", &schema).unwrap(), + options: SortOptions { + descending: false, + nulls_first: true, + }, + }]; + let exec = MemoryExec::try_new(&[vec![batch]], schema, None).unwrap(); + let merge = Arc::new(SortPreservingMergeExec::new(sort, Arc::new(exec))); + + let collected = collect(merge, task_ctx).await.unwrap(); + assert_eq!(collected.len(), 1); + + assert_batches_eq!( + &[ + "+---+---+", + "| a | b |", + "+---+---+", + "| 1 | a |", + "| 2 | b |", + "| 7 | c |", + "| 9 | d |", + "| 3 | e |", + "+---+---+", + ], + collected.as_slice() + ); + } + #[tokio::test] async fn test_async() -> Result<()> { let task_ctx = Arc::new(TaskContext::default()); From 2b6341c48a69528bb7793b11f46a29a1826e8c2d Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=BC=A0=E6=9E=97=E4=BC=9F?= Date: Sat, 24 Aug 2024 08:39:36 +0800 Subject: [PATCH 005/154] Improve CombinePartialFinalAggregate code (#12128) --- .../combine_partial_final_agg.rs | 107 +++++++++--------- 1 file changed, 51 insertions(+), 56 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 f65a4c837a60..8cbb187f7bd2 100644 --- a/datafusion/core/src/physical_optimizer/combine_partial_final_agg.rs +++ b/datafusion/core/src/physical_optimizer/combine_partial_final_agg.rs @@ -51,62 +51,57 @@ impl PhysicalOptimizerRule for CombinePartialFinalAggregate { _config: &ConfigOptions, ) -> Result> { plan.transform_down(|plan| { - let transformed = - plan.as_any() - .downcast_ref::() - .and_then(|agg_exec| { - if matches!( - agg_exec.mode(), - AggregateMode::Final | AggregateMode::FinalPartitioned - ) { - agg_exec - .input() - .as_any() - .downcast_ref::() - .and_then(|input_agg_exec| { - if matches!( - input_agg_exec.mode(), - AggregateMode::Partial - ) && can_combine( - ( - agg_exec.group_expr(), - agg_exec.aggr_expr(), - agg_exec.filter_expr(), - ), - ( - input_agg_exec.group_expr(), - input_agg_exec.aggr_expr(), - input_agg_exec.filter_expr(), - ), - ) { - let mode = - if agg_exec.mode() == &AggregateMode::Final { - AggregateMode::Single - } else { - AggregateMode::SinglePartitioned - }; - AggregateExec::try_new( - mode, - input_agg_exec.group_expr().clone(), - input_agg_exec.aggr_expr().to_vec(), - input_agg_exec.filter_expr().to_vec(), - input_agg_exec.input().clone(), - input_agg_exec.input_schema(), - ) - .map(|combined_agg| { - combined_agg.with_limit(agg_exec.limit()) - }) - .ok() - .map(Arc::new) - } else { - None - } - }) - } else { - None - } - }); - + // Check if the plan is AggregateExec + let Some(agg_exec) = plan.as_any().downcast_ref::() else { + return Ok(Transformed::no(plan)); + }; + + if !matches!( + agg_exec.mode(), + AggregateMode::Final | AggregateMode::FinalPartitioned + ) { + return Ok(Transformed::no(plan)); + } + + // Check if the input is AggregateExec + let Some(input_agg_exec) = + agg_exec.input().as_any().downcast_ref::() + else { + return Ok(Transformed::no(plan)); + }; + + let transformed = if matches!(input_agg_exec.mode(), AggregateMode::Partial) + && can_combine( + ( + agg_exec.group_expr(), + agg_exec.aggr_expr(), + agg_exec.filter_expr(), + ), + ( + input_agg_exec.group_expr(), + input_agg_exec.aggr_expr(), + input_agg_exec.filter_expr(), + ), + ) { + let mode = if agg_exec.mode() == &AggregateMode::Final { + AggregateMode::Single + } else { + AggregateMode::SinglePartitioned + }; + AggregateExec::try_new( + mode, + input_agg_exec.group_expr().clone(), + input_agg_exec.aggr_expr().to_vec(), + input_agg_exec.filter_expr().to_vec(), + input_agg_exec.input().clone(), + input_agg_exec.input_schema(), + ) + .map(|combined_agg| combined_agg.with_limit(agg_exec.limit())) + .ok() + .map(Arc::new) + } else { + None + }; Ok(if let Some(transformed) = transformed { Transformed::yes(transformed) } else { From 3b90e3e5479d4354992cbe465d50e56c033d8fc2 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=BC=A0=E6=9E=97=E4=BC=9F?= Date: Sat, 24 Aug 2024 08:45:40 +0800 Subject: [PATCH 006/154] Use `LexRequirement` alias as much as possible (#12130) --- datafusion-examples/examples/custom_file_format.rs | 5 +++-- datafusion/core/src/datasource/file_format/arrow.rs | 5 +++-- datafusion/core/src/datasource/file_format/csv.rs | 5 +++-- datafusion/core/src/datasource/file_format/json.rs | 5 +++-- datafusion/core/src/datasource/file_format/mod.rs | 6 ++++-- .../core/src/datasource/file_format/parquet.rs | 5 +++-- .../src/physical_optimizer/enforce_distribution.rs | 3 ++- .../core/src/physical_optimizer/sort_pushdown.rs | 11 ++++++----- .../core/src/physical_optimizer/test_utils.rs | 6 ++++-- datafusion/physical-expr-common/src/sort_expr.rs | 2 +- datafusion/physical-expr/src/equivalence/mod.rs | 2 +- .../physical-optimizer/src/output_requirements.rs | 2 +- datafusion/physical-plan/src/execution_plan.rs | 7 +++---- datafusion/physical-plan/src/insert.rs | 13 ++++++------- .../physical-plan/src/joins/sort_merge_join.rs | 3 ++- .../physical-plan/src/joins/symmetric_hash_join.rs | 3 ++- .../src/sorts/sort_preserving_merge.rs | 3 ++- .../src/windows/bounded_window_agg_exec.rs | 5 +++-- datafusion/physical-plan/src/windows/mod.rs | 7 ++++--- .../physical-plan/src/windows/window_agg_exec.rs | 4 ++-- 20 files changed, 58 insertions(+), 44 deletions(-) diff --git a/datafusion-examples/examples/custom_file_format.rs b/datafusion-examples/examples/custom_file_format.rs index 8612a1cc4430..1d9b587f15b9 100644 --- a/datafusion-examples/examples/custom_file_format.rs +++ b/datafusion-examples/examples/custom_file_format.rs @@ -23,6 +23,7 @@ use arrow::{ }; use arrow_schema::{DataType, Field, Schema, SchemaRef}; use datafusion::execution::session_state::SessionStateBuilder; +use datafusion::physical_expr::LexRequirement; use datafusion::{ datasource::{ file_format::{ @@ -38,7 +39,7 @@ use datafusion::{ prelude::SessionContext, }; use datafusion_common::{GetExt, Statistics}; -use datafusion_physical_expr::{PhysicalExpr, PhysicalSortRequirement}; +use datafusion_physical_expr::PhysicalExpr; use object_store::{ObjectMeta, ObjectStore}; use tempfile::tempdir; @@ -123,7 +124,7 @@ impl FileFormat for TSVFileFormat { input: Arc, state: &SessionState, conf: FileSinkConfig, - order_requirements: Option>, + order_requirements: Option, ) -> Result> { self.csv_file_format .create_writer_physical_plan(input, state, conf, order_requirements) diff --git a/datafusion/core/src/datasource/file_format/arrow.rs b/datafusion/core/src/datasource/file_format/arrow.rs index 95f76195e63d..6ee4280956e8 100644 --- a/datafusion/core/src/datasource/file_format/arrow.rs +++ b/datafusion/core/src/datasource/file_format/arrow.rs @@ -47,12 +47,13 @@ use datafusion_common::{ not_impl_err, DataFusionError, GetExt, Statistics, DEFAULT_ARROW_EXTENSION, }; use datafusion_execution::{SendableRecordBatchStream, TaskContext}; -use datafusion_physical_expr::{PhysicalExpr, PhysicalSortRequirement}; +use datafusion_physical_expr::PhysicalExpr; use datafusion_physical_plan::insert::{DataSink, DataSinkExec}; use datafusion_physical_plan::metrics::MetricsSet; use async_trait::async_trait; use bytes::Bytes; +use datafusion_physical_expr_common::sort_expr::LexRequirement; use futures::stream::BoxStream; use futures::StreamExt; use object_store::{GetResultPayload, ObjectMeta, ObjectStore}; @@ -178,7 +179,7 @@ impl FileFormat for ArrowFormat { input: Arc, _state: &SessionState, conf: FileSinkConfig, - order_requirements: Option>, + order_requirements: Option, ) -> Result> { if conf.overwrite { return not_impl_err!("Overwrites are not implemented yet for Arrow format"); diff --git a/datafusion/core/src/datasource/file_format/csv.rs b/datafusion/core/src/datasource/file_format/csv.rs index 24d55ea54068..d1ce2afcccf3 100644 --- a/datafusion/core/src/datasource/file_format/csv.rs +++ b/datafusion/core/src/datasource/file_format/csv.rs @@ -46,11 +46,12 @@ use datafusion_common::{ exec_err, not_impl_err, DataFusionError, GetExt, DEFAULT_CSV_EXTENSION, }; use datafusion_execution::TaskContext; -use datafusion_physical_expr::{PhysicalExpr, PhysicalSortRequirement}; +use datafusion_physical_expr::PhysicalExpr; use datafusion_physical_plan::metrics::MetricsSet; use async_trait::async_trait; use bytes::{Buf, Bytes}; +use datafusion_physical_expr_common::sort_expr::LexRequirement; use futures::stream::BoxStream; use futures::{pin_mut, Stream, StreamExt, TryStreamExt}; use object_store::{delimited::newline_delimited_stream, ObjectMeta, ObjectStore}; @@ -371,7 +372,7 @@ impl FileFormat for CsvFormat { input: Arc, state: &SessionState, conf: FileSinkConfig, - order_requirements: Option>, + order_requirements: Option, ) -> Result> { if conf.overwrite { return not_impl_err!("Overwrites are not implemented yet for CSV"); diff --git a/datafusion/core/src/datasource/file_format/json.rs b/datafusion/core/src/datasource/file_format/json.rs index 7c579e890c8c..4471d7d6cb31 100644 --- a/datafusion/core/src/datasource/file_format/json.rs +++ b/datafusion/core/src/datasource/file_format/json.rs @@ -46,12 +46,13 @@ use datafusion_common::config::{ConfigField, ConfigFileType, JsonOptions}; use datafusion_common::file_options::json_writer::JsonWriterOptions; use datafusion_common::{not_impl_err, GetExt, DEFAULT_JSON_EXTENSION}; use datafusion_execution::TaskContext; -use datafusion_physical_expr::{PhysicalExpr, PhysicalSortRequirement}; +use datafusion_physical_expr::PhysicalExpr; use datafusion_physical_plan::metrics::MetricsSet; use datafusion_physical_plan::ExecutionPlan; use async_trait::async_trait; use bytes::{Buf, Bytes}; +use datafusion_physical_expr_common::sort_expr::LexRequirement; use object_store::{GetResultPayload, ObjectMeta, ObjectStore}; #[derive(Default)] @@ -249,7 +250,7 @@ impl FileFormat for JsonFormat { input: Arc, _state: &SessionState, conf: FileSinkConfig, - order_requirements: Option>, + order_requirements: Option, ) -> Result> { if conf.overwrite { return not_impl_err!("Overwrites are not implemented yet for Json"); diff --git a/datafusion/core/src/datasource/file_format/mod.rs b/datafusion/core/src/datasource/file_format/mod.rs index a324a4578424..d21464b74b53 100644 --- a/datafusion/core/src/datasource/file_format/mod.rs +++ b/datafusion/core/src/datasource/file_format/mod.rs @@ -45,12 +45,14 @@ use crate::physical_plan::{ExecutionPlan, Statistics}; use arrow_schema::{DataType, Field, Schema}; use datafusion_common::file_options::file_type::FileType; use datafusion_common::{internal_err, not_impl_err, GetExt}; -use datafusion_physical_expr::{PhysicalExpr, PhysicalSortRequirement}; +use datafusion_physical_expr::PhysicalExpr; use async_trait::async_trait; +use datafusion_physical_expr_common::sort_expr::LexRequirement; use file_compression_type::FileCompressionType; use object_store::{ObjectMeta, ObjectStore}; use std::fmt::Debug; + /// Factory for creating [`FileFormat`] instances based on session and command level options /// /// Users can provide their own `FileFormatFactory` to support arbitrary file formats @@ -132,7 +134,7 @@ pub trait FileFormat: Send + Sync + fmt::Debug { _input: Arc, _state: &SessionState, _conf: FileSinkConfig, - _order_requirements: Option>, + _order_requirements: Option, ) -> Result> { not_impl_err!("Writer not implemented for this format") } diff --git a/datafusion/core/src/datasource/file_format/parquet.rs b/datafusion/core/src/datasource/file_format/parquet.rs index 83f77ca9371a..23e765f0f2cd 100644 --- a/datafusion/core/src/datasource/file_format/parquet.rs +++ b/datafusion/core/src/datasource/file_format/parquet.rs @@ -51,7 +51,7 @@ use datafusion_common_runtime::SpawnedTask; use datafusion_execution::memory_pool::{MemoryConsumer, MemoryPool, MemoryReservation}; use datafusion_execution::TaskContext; use datafusion_functions_aggregate::min_max::{MaxAccumulator, MinAccumulator}; -use datafusion_physical_expr::{PhysicalExpr, PhysicalSortRequirement}; +use datafusion_physical_expr::PhysicalExpr; use datafusion_physical_plan::metrics::MetricsSet; use async_trait::async_trait; @@ -76,6 +76,7 @@ use tokio::sync::mpsc::{self, Receiver, Sender}; use tokio::task::JoinSet; use crate::datasource::physical_plan::parquet::ParquetExecBuilder; +use datafusion_physical_expr_common::sort_expr::LexRequirement; use futures::{StreamExt, TryStreamExt}; use object_store::path::Path; use object_store::{ObjectMeta, ObjectStore}; @@ -376,7 +377,7 @@ impl FileFormat for ParquetFormat { input: Arc, _state: &SessionState, conf: FileSinkConfig, - order_requirements: Option>, + order_requirements: Option, ) -> Result> { if conf.overwrite { return not_impl_err!("Overwrites are not implemented yet for Parquet"); diff --git a/datafusion/core/src/physical_optimizer/enforce_distribution.rs b/datafusion/core/src/physical_optimizer/enforce_distribution.rs index 2ee5624c83dd..77ddd098c7be 100644 --- a/datafusion/core/src/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/src/physical_optimizer/enforce_distribution.rs @@ -1419,6 +1419,7 @@ pub(crate) mod tests { expressions, expressions::binary, expressions::lit, LexOrdering, PhysicalSortExpr, PhysicalSortRequirement, }; + use datafusion_physical_expr_common::sort_expr::LexRequirement; use datafusion_physical_plan::PlanProperties; /// Models operators like BoundedWindowExec that require an input @@ -1489,7 +1490,7 @@ pub(crate) mod tests { } // model that it requires the output ordering of its input - fn required_input_ordering(&self) -> Vec>> { + fn required_input_ordering(&self) -> Vec> { if self.expr.is_empty() { vec![None] } else { diff --git a/datafusion/core/src/physical_optimizer/sort_pushdown.rs b/datafusion/core/src/physical_optimizer/sort_pushdown.rs index 17d63a06a6f8..9ab6802d18f1 100644 --- a/datafusion/core/src/physical_optimizer/sort_pushdown.rs +++ b/datafusion/core/src/physical_optimizer/sort_pushdown.rs @@ -36,6 +36,7 @@ use datafusion_physical_expr::expressions::Column; use datafusion_physical_expr::{ LexRequirementRef, PhysicalSortExpr, PhysicalSortRequirement, }; +use datafusion_physical_expr_common::sort_expr::LexRequirement; /// This is a "data class" we use within the [`EnforceSorting`] rule to push /// down [`SortExec`] in the plan. In some cases, we can reduce the total @@ -46,7 +47,7 @@ use datafusion_physical_expr::{ /// [`EnforceSorting`]: crate::physical_optimizer::enforce_sorting::EnforceSorting #[derive(Default, Clone)] pub struct ParentRequirements { - ordering_requirement: Option>, + ordering_requirement: Option, fetch: Option, } @@ -159,7 +160,7 @@ fn pushdown_sorts_helper( fn pushdown_requirement_to_children( plan: &Arc, parent_required: LexRequirementRef, -) -> Result>>>> { +) -> Result>>> { let maintains_input_order = plan.maintains_input_order(); if is_window(plan) { let required_input_ordering = plan.required_input_ordering(); @@ -345,7 +346,7 @@ fn try_pushdown_requirements_to_join( parent_required: LexRequirementRef, sort_expr: Vec, push_side: JoinSide, -) -> Result>>>> { +) -> Result>>> { let left_eq_properties = smj.left().equivalence_properties(); let right_eq_properties = smj.right().equivalence_properties(); let mut smj_required_orderings = smj.required_input_ordering(); @@ -460,7 +461,7 @@ fn expr_source_side( fn shift_right_required( parent_required: LexRequirementRef, left_columns_len: usize, -) -> Result> { +) -> Result { let new_right_required = parent_required .iter() .filter_map(|r| { @@ -486,7 +487,7 @@ enum RequirementsCompatibility { /// Requirements satisfy Satisfy, /// Requirements compatible - Compatible(Option>), + Compatible(Option), /// Requirements not compatible NonCompatible, } diff --git a/datafusion/core/src/physical_optimizer/test_utils.rs b/datafusion/core/src/physical_optimizer/test_utils.rs index 90853c347672..98f1a7c21a39 100644 --- a/datafusion/core/src/physical_optimizer/test_utils.rs +++ b/datafusion/core/src/physical_optimizer/test_utils.rs @@ -56,7 +56,9 @@ use datafusion_physical_plan::{ use async_trait::async_trait; use datafusion_execution::{SendableRecordBatchStream, TaskContext}; -use datafusion_physical_expr_common::sort_expr::PhysicalSortRequirement; +use datafusion_physical_expr_common::sort_expr::{ + LexRequirement, PhysicalSortRequirement, +}; async fn register_current_csv( ctx: &SessionContext, @@ -416,7 +418,7 @@ impl ExecutionPlan for RequirementsTestExec { self.input.properties() } - fn required_input_ordering(&self) -> Vec>> { + fn required_input_ordering(&self) -> Vec> { let requirement = PhysicalSortRequirement::from_sort_exprs(&self.required_input_ordering); vec![Some(requirement)] diff --git a/datafusion/physical-expr-common/src/sort_expr.rs b/datafusion/physical-expr-common/src/sort_expr.rs index 9dc54d2eb2d0..745ec543c31a 100644 --- a/datafusion/physical-expr-common/src/sort_expr.rs +++ b/datafusion/physical-expr-common/src/sort_expr.rs @@ -221,7 +221,7 @@ impl PhysicalSortRequirement { /// [`ExecutionPlan::required_input_ordering`]: https://docs.rs/datafusion/latest/datafusion/physical_plan/trait.ExecutionPlan.html#method.required_input_ordering pub fn from_sort_exprs<'a>( ordering: impl IntoIterator, - ) -> Vec { + ) -> LexRequirement { ordering .into_iter() .cloned() diff --git a/datafusion/physical-expr/src/equivalence/mod.rs b/datafusion/physical-expr/src/equivalence/mod.rs index b9228282b081..d862eda5018e 100644 --- a/datafusion/physical-expr/src/equivalence/mod.rs +++ b/datafusion/physical-expr/src/equivalence/mod.rs @@ -239,7 +239,7 @@ mod tests { // Convert each tuple to PhysicalSortRequirement pub fn convert_to_sort_reqs( in_data: &[(&Arc, Option)], - ) -> Vec { + ) -> LexRequirement { in_data .iter() .map(|(expr, options)| { diff --git a/datafusion/physical-optimizer/src/output_requirements.rs b/datafusion/physical-optimizer/src/output_requirements.rs index fdfdd349e36e..4f6f91a2348f 100644 --- a/datafusion/physical-optimizer/src/output_requirements.rs +++ b/datafusion/physical-optimizer/src/output_requirements.rs @@ -165,7 +165,7 @@ impl ExecutionPlan for OutputRequirementExec { vec![&self.input] } - fn required_input_ordering(&self) -> Vec>> { + fn required_input_ordering(&self) -> Vec> { vec![self.order_requirement.clone()] } diff --git a/datafusion/physical-plan/src/execution_plan.rs b/datafusion/physical-plan/src/execution_plan.rs index a6a15e46860c..e1182719293d 100644 --- a/datafusion/physical-plan/src/execution_plan.rs +++ b/datafusion/physical-plan/src/execution_plan.rs @@ -36,9 +36,8 @@ pub use datafusion_physical_expr::window::WindowExpr; pub use datafusion_physical_expr::{ expressions, functions, udf, AggregateExpr, Distribution, Partitioning, PhysicalExpr, }; -use datafusion_physical_expr::{ - EquivalenceProperties, LexOrdering, PhysicalSortExpr, PhysicalSortRequirement, -}; +use datafusion_physical_expr::{EquivalenceProperties, LexOrdering, PhysicalSortExpr}; +use datafusion_physical_expr_common::sort_expr::LexRequirement; use crate::coalesce_partitions::CoalescePartitionsExec; use crate::display::DisplayableExecutionPlan; @@ -125,7 +124,7 @@ pub trait ExecutionPlan: Debug + DisplayAs + Send + Sync { /// NOTE that checking `!is_empty()` does **not** check for a /// required input ordering. Instead, the correct check is that at /// least one entry must be `Some` - fn required_input_ordering(&self) -> Vec>> { + fn required_input_ordering(&self) -> Vec> { vec![None; self.children().len()] } diff --git a/datafusion/physical-plan/src/insert.rs b/datafusion/physical-plan/src/insert.rs index 5cd864125e29..5dc27bc239d2 100644 --- a/datafusion/physical-plan/src/insert.rs +++ b/datafusion/physical-plan/src/insert.rs @@ -35,11 +35,10 @@ use arrow_array::{ArrayRef, UInt64Array}; use arrow_schema::{DataType, Field, Schema}; use datafusion_common::{internal_err, Result}; use datafusion_execution::TaskContext; -use datafusion_physical_expr::{ - Distribution, EquivalenceProperties, PhysicalSortRequirement, -}; +use datafusion_physical_expr::{Distribution, EquivalenceProperties}; use async_trait::async_trait; +use datafusion_physical_expr_common::sort_expr::LexRequirement; use futures::StreamExt; /// `DataSink` implements writing streams of [`RecordBatch`]es to @@ -90,7 +89,7 @@ pub struct DataSinkExec { /// Schema describing the structure of the output data. count_schema: SchemaRef, /// Optional required sort order for output data. - sort_order: Option>, + sort_order: Option, cache: PlanProperties, } @@ -106,7 +105,7 @@ impl DataSinkExec { input: Arc, sink: Arc, sink_schema: SchemaRef, - sort_order: Option>, + sort_order: Option, ) -> Self { let count_schema = make_count_schema(); let cache = Self::create_schema(&input, count_schema); @@ -131,7 +130,7 @@ impl DataSinkExec { } /// Optional sort order for output data - pub fn sort_order(&self) -> &Option> { + pub fn sort_order(&self) -> &Option { &self.sort_order } @@ -189,7 +188,7 @@ impl ExecutionPlan for DataSinkExec { vec![Distribution::SinglePartition; self.children().len()] } - fn required_input_ordering(&self) -> Vec>> { + fn required_input_ordering(&self) -> Vec> { // The required input ordering is set externally (e.g. by a `ListingTable`). // Otherwise, there is no specific requirement (i.e. `sort_expr` is `None`). vec![self.sort_order.as_ref().cloned()] diff --git a/datafusion/physical-plan/src/joins/sort_merge_join.rs b/datafusion/physical-plan/src/joins/sort_merge_join.rs index b5994d16d2ee..56cd699bf6e7 100644 --- a/datafusion/physical-plan/src/joins/sort_merge_join.rs +++ b/datafusion/physical-plan/src/joins/sort_merge_join.rs @@ -51,6 +51,7 @@ use datafusion_execution::runtime_env::RuntimeEnv; use datafusion_execution::TaskContext; use datafusion_physical_expr::equivalence::join_equivalence_properties; use datafusion_physical_expr::{PhysicalExprRef, PhysicalSortRequirement}; +use datafusion_physical_expr_common::sort_expr::LexRequirement; use crate::expressions::PhysicalSortExpr; use crate::joins::utils::{ @@ -288,7 +289,7 @@ impl ExecutionPlan for SortMergeJoinExec { ] } - fn required_input_ordering(&self) -> Vec>> { + fn required_input_ordering(&self) -> Vec> { vec![ Some(PhysicalSortRequirement::from_sort_exprs( &self.left_sort_exprs, diff --git a/datafusion/physical-plan/src/joins/symmetric_hash_join.rs b/datafusion/physical-plan/src/joins/symmetric_hash_join.rs index 7dab664502e9..ac718a95e9f4 100644 --- a/datafusion/physical-plan/src/joins/symmetric_hash_join.rs +++ b/datafusion/physical-plan/src/joins/symmetric_hash_join.rs @@ -72,6 +72,7 @@ use datafusion_physical_expr::intervals::cp_solver::ExprIntervalGraph; use datafusion_physical_expr::{PhysicalExprRef, PhysicalSortRequirement}; use ahash::RandomState; +use datafusion_physical_expr_common::sort_expr::LexRequirement; use futures::{ready, Stream, StreamExt}; use hashbrown::HashSet; use parking_lot::Mutex; @@ -410,7 +411,7 @@ impl ExecutionPlan for SymmetricHashJoinExec { } } - fn required_input_ordering(&self) -> Vec>> { + fn required_input_ordering(&self) -> Vec> { vec![ self.left_sort_exprs .as_ref() diff --git a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs index f34701d0d713..131fa71217cc 100644 --- a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs +++ b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs @@ -35,6 +35,7 @@ use datafusion_execution::memory_pool::MemoryConsumer; use datafusion_execution::TaskContext; use datafusion_physical_expr::PhysicalSortRequirement; +use datafusion_physical_expr_common::sort_expr::LexRequirement; use log::{debug, trace}; /// Sort preserving merge execution plan @@ -187,7 +188,7 @@ impl ExecutionPlan for SortPreservingMergeExec { vec![false] } - fn required_input_ordering(&self) -> Vec>> { + fn required_input_ordering(&self) -> Vec> { vec![Some(PhysicalSortRequirement::from_sort_exprs(&self.expr))] } diff --git a/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs b/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs index efb5dea1ec6e..084436ee376d 100644 --- a/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs +++ b/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs @@ -59,7 +59,8 @@ use datafusion_expr::ColumnarValue; use datafusion_physical_expr::window::{ PartitionBatches, PartitionKey, PartitionWindowAggStates, WindowState, }; -use datafusion_physical_expr::{PhysicalExpr, PhysicalSortRequirement}; +use datafusion_physical_expr::PhysicalExpr; +use datafusion_physical_expr_common::sort_expr::LexRequirement; use futures::stream::Stream; use futures::{ready, StreamExt}; use hashbrown::raw::RawTable; @@ -253,7 +254,7 @@ impl ExecutionPlan for BoundedWindowAggExec { vec![&self.input] } - fn required_input_ordering(&self) -> Vec>> { + fn required_input_ordering(&self) -> Vec> { let partition_bys = self.window_expr()[0].partition_by(); let order_keys = self.window_expr()[0].order_by(); if self.input_order_mode != InputOrderMode::Sorted diff --git a/datafusion/physical-plan/src/windows/mod.rs b/datafusion/physical-plan/src/windows/mod.rs index 63f4ffcfaacc..d607bb79b44e 100644 --- a/datafusion/physical-plan/src/windows/mod.rs +++ b/datafusion/physical-plan/src/windows/mod.rs @@ -56,6 +56,7 @@ use datafusion_physical_expr::expressions::Column; pub use datafusion_physical_expr::window::{ BuiltInWindowExpr, PlainAggregateWindowExpr, WindowExpr, }; +use datafusion_physical_expr_common::sort_expr::LexRequirement; pub use window_agg_exec::WindowAggExec; /// Build field from window function and add it into schema @@ -401,7 +402,7 @@ pub(crate) fn calc_requirements< >( partition_by_exprs: impl IntoIterator, orderby_sort_exprs: impl IntoIterator, -) -> Option> { +) -> Option { let mut sort_reqs = partition_by_exprs .into_iter() .map(|partition_by| { @@ -571,7 +572,7 @@ pub fn get_window_mode( input: &Arc, ) -> Option<(bool, InputOrderMode)> { let input_eqs = input.equivalence_properties().clone(); - let mut partition_by_reqs: Vec = vec![]; + let mut partition_by_reqs: LexRequirement = vec![]; let (_, indices) = input_eqs.find_longest_permutation(partitionby_exprs); partition_by_reqs.extend(indices.iter().map(|&idx| PhysicalSortRequirement { expr: Arc::clone(&partitionby_exprs[idx]), @@ -728,7 +729,7 @@ mod tests { orderbys.push(PhysicalSortExpr { expr, options }); } - let mut expected: Option> = None; + let mut expected: Option = None; for (col_name, reqs) in expected_params { let options = reqs.map(|(descending, nulls_first)| SortOptions { descending, diff --git a/datafusion/physical-plan/src/windows/window_agg_exec.rs b/datafusion/physical-plan/src/windows/window_agg_exec.rs index d2f7090fca17..afe9700ed08c 100644 --- a/datafusion/physical-plan/src/windows/window_agg_exec.rs +++ b/datafusion/physical-plan/src/windows/window_agg_exec.rs @@ -43,7 +43,7 @@ use datafusion_common::stats::Precision; use datafusion_common::utils::{evaluate_partition_ranges, transpose}; use datafusion_common::{internal_err, Result}; use datafusion_execution::TaskContext; -use datafusion_physical_expr::PhysicalSortRequirement; +use datafusion_physical_expr_common::sort_expr::LexRequirement; use futures::{ready, Stream, StreamExt}; /// Window execution plan @@ -191,7 +191,7 @@ impl ExecutionPlan for WindowAggExec { vec![true] } - fn required_input_ordering(&self) -> Vec>> { + fn required_input_ordering(&self) -> Vec> { let partition_bys = self.window_expr()[0].partition_by(); let order_keys = self.window_expr()[0].order_by(); if self.ordered_partition_by_indices.len() < partition_bys.len() { From 932adabcc04faf65b8c8670b2385dc13a0f849f4 Mon Sep 17 00:00:00 2001 From: Jay Zhan Date: Sat, 24 Aug 2024 09:21:35 +0800 Subject: [PATCH 007/154] `array_has` avoid row converter for string type (#12097) * first draft Signed-off-by: jayzhan211 * avoid row converter for string Signed-off-by: jayzhan211 * cleanup Signed-off-by: jayzhan211 * string view Signed-off-by: jayzhan211 * trigger ci Signed-off-by: jayzhan211 * refactor Signed-off-by: jayzhan211 * cleanup Signed-off-by: jayzhan211 * array_has_all Signed-off-by: jayzhan211 * array_has_any Signed-off-by: jayzhan211 * cleanup Signed-off-by: jayzhan211 * cleanup Signed-off-by: jayzhan211 * fmt Signed-off-by: jayzhan211 * rm unused import Signed-off-by: jayzhan211 * cleanup Signed-off-by: jayzhan211 --------- Signed-off-by: jayzhan211 --- datafusion/common/src/utils/mod.rs | 1 + datafusion/common/src/utils/string_utils.rs | 31 ++ datafusion/functions-nested/src/array_has.rs | 303 ++++++++++++------- 3 files changed, 227 insertions(+), 108 deletions(-) create mode 100644 datafusion/common/src/utils/string_utils.rs diff --git a/datafusion/common/src/utils/mod.rs b/datafusion/common/src/utils/mod.rs index d7059e882e55..839f890bf077 100644 --- a/datafusion/common/src/utils/mod.rs +++ b/datafusion/common/src/utils/mod.rs @@ -20,6 +20,7 @@ pub mod expr; pub mod memory; pub mod proxy; +pub mod string_utils; use crate::error::{_internal_datafusion_err, _internal_err}; use crate::{arrow_datafusion_err, DataFusionError, Result, ScalarValue}; diff --git a/datafusion/common/src/utils/string_utils.rs b/datafusion/common/src/utils/string_utils.rs new file mode 100644 index 000000000000..a2231e6786a7 --- /dev/null +++ b/datafusion/common/src/utils/string_utils.rs @@ -0,0 +1,31 @@ +// 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. + +//! Utilities for working with strings + +use arrow::{array::AsArray, datatypes::DataType}; +use arrow_array::Array; + +/// Convenient function to convert an Arrow string array to a vector of strings +pub fn string_array_to_vec(array: &dyn Array) -> Vec> { + match array.data_type() { + DataType::Utf8 => array.as_string::().iter().collect(), + DataType::LargeUtf8 => array.as_string::().iter().collect(), + DataType::Utf8View => array.as_string_view().iter().collect(), + _ => unreachable!(), + } +} diff --git a/datafusion/functions-nested/src/array_has.rs b/datafusion/functions-nested/src/array_has.rs index fe1df2579932..d7c194304887 100644 --- a/datafusion/functions-nested/src/array_has.rs +++ b/datafusion/functions-nested/src/array_has.rs @@ -19,14 +19,16 @@ use arrow::array::{Array, ArrayRef, BooleanArray, OffsetSizeTrait}; use arrow::datatypes::DataType; -use arrow::row::{RowConverter, SortField}; +use arrow::row::{RowConverter, Rows, SortField}; +use arrow_array::GenericListArray; use datafusion_common::cast::as_generic_list_array; +use datafusion_common::utils::string_utils::string_array_to_vec; use datafusion_common::{exec_err, Result}; use datafusion_expr::{ColumnarValue, ScalarUDFImpl, Signature, Volatility}; use itertools::Itertools; -use crate::utils::check_datatypes; +use crate::utils::make_scalar_function; use std::any::Any; use std::sync::Arc; @@ -93,33 +95,52 @@ impl ScalarUDFImpl for ArrayHas { } fn invoke(&self, args: &[ColumnarValue]) -> Result { - let args = ColumnarValue::values_to_arrays(args)?; + make_scalar_function(array_has_inner)(args) + } - if args.len() != 2 { - return exec_err!("array_has needs two arguments"); - } + fn aliases(&self) -> &[String] { + &self.aliases + } +} - let array_type = args[0].data_type(); - - match array_type { - DataType::List(_) => general_array_has_dispatch::( - &args[0], - &args[1], - ComparisonType::Single, - ) - .map(ColumnarValue::Array), - DataType::LargeList(_) => general_array_has_dispatch::( - &args[0], - &args[1], - ComparisonType::Single, - ) - .map(ColumnarValue::Array), - _ => exec_err!("array_has does not support type '{array_type:?}'."), +fn array_has_inner(args: &[ArrayRef]) -> Result { + match args[0].data_type() { + DataType::List(_) => array_has_dispatch::(&args[0], &args[1]), + DataType::LargeList(_) => array_has_dispatch::(&args[0], &args[1]), + _ => exec_err!( + "array_has does not support type '{:?}'.", + args[0].data_type() + ), + } +} + +fn array_has_all_inner(args: &[ArrayRef]) -> Result { + match args[0].data_type() { + DataType::List(_) => { + array_has_all_and_any_dispatch::(&args[0], &args[1], ComparisonType::All) + } + DataType::LargeList(_) => { + array_has_all_and_any_dispatch::(&args[0], &args[1], ComparisonType::All) } + _ => exec_err!( + "array_has does not support type '{:?}'.", + args[0].data_type() + ), } +} - fn aliases(&self) -> &[String] { - &self.aliases +fn array_has_any_inner(args: &[ArrayRef]) -> Result { + match args[0].data_type() { + DataType::List(_) => { + array_has_all_and_any_dispatch::(&args[0], &args[1], ComparisonType::Any) + } + DataType::LargeList(_) => { + array_has_all_and_any_dispatch::(&args[0], &args[1], ComparisonType::Any) + } + _ => exec_err!( + "array_has does not support type '{:?}'.", + args[0].data_type() + ), } } @@ -161,24 +182,7 @@ impl ScalarUDFImpl for ArrayHasAll { } fn invoke(&self, args: &[ColumnarValue]) -> Result { - let args = ColumnarValue::values_to_arrays(args)?; - if args.len() != 2 { - return exec_err!("array_has_all needs two arguments"); - } - - let array_type = args[0].data_type(); - - match array_type { - DataType::List(_) => { - general_array_has_dispatch::(&args[0], &args[1], ComparisonType::All) - .map(ColumnarValue::Array) - } - DataType::LargeList(_) => { - general_array_has_dispatch::(&args[0], &args[1], ComparisonType::All) - .map(ColumnarValue::Array) - } - _ => exec_err!("array_has_all does not support type '{array_type:?}'."), - } + make_scalar_function(array_has_all_inner)(args) } fn aliases(&self) -> &[String] { @@ -224,25 +228,7 @@ impl ScalarUDFImpl for ArrayHasAny { } fn invoke(&self, args: &[ColumnarValue]) -> Result { - let args = ColumnarValue::values_to_arrays(args)?; - - if args.len() != 2 { - return exec_err!("array_has_any needs two arguments"); - } - - let array_type = args[0].data_type(); - - match array_type { - DataType::List(_) => { - general_array_has_dispatch::(&args[0], &args[1], ComparisonType::Any) - .map(ColumnarValue::Array) - } - DataType::LargeList(_) => { - general_array_has_dispatch::(&args[0], &args[1], ComparisonType::Any) - .map(ColumnarValue::Array) - } - _ => exec_err!("array_has_any does not support type '{array_type:?}'."), - } + make_scalar_function(array_has_any_inner)(args) } fn aliases(&self) -> &[String] { @@ -251,75 +237,176 @@ impl ScalarUDFImpl for ArrayHasAny { } /// Represents the type of comparison for array_has. -#[derive(Debug, PartialEq)] +#[derive(Debug, PartialEq, Clone, Copy)] enum ComparisonType { // array_has_all All, // array_has_any Any, - // array_has - Single, } -fn general_array_has_dispatch( +fn array_has_dispatch( + haystack: &ArrayRef, + needle: &ArrayRef, +) -> Result { + let haystack = as_generic_list_array::(haystack)?; + match needle.data_type() { + DataType::Utf8 | DataType::LargeUtf8 | DataType::Utf8View => { + array_has_string_internal::(haystack, needle) + } + _ => general_array_has::(haystack, needle), + } +} + +fn array_has_all_and_any_dispatch( haystack: &ArrayRef, needle: &ArrayRef, comparison_type: ComparisonType, ) -> Result { - let array = if comparison_type == ComparisonType::Single { - let arr = as_generic_list_array::(haystack)?; - check_datatypes("array_has", &[arr.values(), needle])?; - arr - } else { - check_datatypes("array_has", &[haystack, needle])?; - as_generic_list_array::(haystack)? - }; + let haystack = as_generic_list_array::(haystack)?; + let needle = as_generic_list_array::(needle)?; + match needle.data_type() { + DataType::Utf8 | DataType::LargeUtf8 | DataType::Utf8View => { + array_has_all_and_any_string_internal::(haystack, needle, comparison_type) + } + _ => general_array_has_for_all_and_any::(haystack, needle, comparison_type), + } +} - let mut boolean_builder = BooleanArray::builder(array.len()); +fn array_has_string_internal( + haystack: &GenericListArray, + needle: &ArrayRef, +) -> Result { + let mut boolean_builder = BooleanArray::builder(haystack.len()); + for (arr, element) in haystack.iter().zip(string_array_to_vec(needle).into_iter()) { + match (arr, element) { + (Some(arr), Some(element)) => { + boolean_builder.append_value( + string_array_to_vec(&arr) + .into_iter() + .flatten() + .any(|x| x == element), + ); + } + (_, _) => { + boolean_builder.append_null(); + } + } + } + + Ok(Arc::new(boolean_builder.finish())) +} +fn general_array_has( + array: &GenericListArray, + needle: &ArrayRef, +) -> Result { + let mut boolean_builder = BooleanArray::builder(array.len()); let converter = RowConverter::new(vec![SortField::new(array.value_type())])?; + let sub_arr_values = converter.convert_columns(&[Arc::clone(needle)])?; + + for (row_idx, arr) in array.iter().enumerate() { + if let Some(arr) = arr { + let arr_values = converter.convert_columns(&[arr])?; + boolean_builder.append_value( + arr_values + .iter() + .dedup() + .any(|x| x == sub_arr_values.row(row_idx)), + ); + } else { + boolean_builder.append_null(); + } + } - let element = Arc::clone(needle); - let sub_array = if comparison_type != ComparisonType::Single { - as_generic_list_array::(needle)? - } else { - array - }; - for (row_idx, (arr, sub_arr)) in array.iter().zip(sub_array.iter()).enumerate() { + Ok(Arc::new(boolean_builder.finish())) +} + +// String comparison for array_has_all and array_has_any +fn array_has_all_and_any_string_internal( + array: &GenericListArray, + needle: &GenericListArray, + comparison_type: ComparisonType, +) -> Result { + let mut boolean_builder = BooleanArray::builder(array.len()); + for (arr, sub_arr) in array.iter().zip(needle.iter()) { match (arr, sub_arr) { (Some(arr), Some(sub_arr)) => { - let arr_values = converter.convert_columns(&[arr])?; - let sub_arr_values = if comparison_type != ComparisonType::Single { - converter.convert_columns(&[sub_arr])? - } else { - converter.convert_columns(&[Arc::clone(&element)])? - }; - - let mut res = match comparison_type { - ComparisonType::All => sub_arr_values - .iter() - .dedup() - .all(|elem| arr_values.iter().dedup().any(|x| x == elem)), - ComparisonType::Any => sub_arr_values - .iter() - .dedup() - .any(|elem| arr_values.iter().dedup().any(|x| x == elem)), - ComparisonType::Single => arr_values - .iter() - .dedup() - .any(|x| x == sub_arr_values.row(row_idx)), - }; - - if comparison_type == ComparisonType::Any { - res |= res; - } - boolean_builder.append_value(res); + let haystack_array = string_array_to_vec(&arr); + let needle_array = string_array_to_vec(&sub_arr); + boolean_builder.append_value(array_has_string_kernel( + haystack_array, + needle_array, + comparison_type, + )); } - // respect null input (_, _) => { boolean_builder.append_null(); } } } + Ok(Arc::new(boolean_builder.finish())) } + +fn array_has_string_kernel( + haystack: Vec>, + needle: Vec>, + comparison_type: ComparisonType, +) -> bool { + match comparison_type { + ComparisonType::All => needle + .iter() + .dedup() + .all(|x| haystack.iter().dedup().any(|y| y == x)), + ComparisonType::Any => needle + .iter() + .dedup() + .any(|x| haystack.iter().dedup().any(|y| y == x)), + } +} + +// General row comparison for array_has_all and array_has_any +fn general_array_has_for_all_and_any( + haystack: &GenericListArray, + needle: &GenericListArray, + comparison_type: ComparisonType, +) -> Result { + let mut boolean_builder = BooleanArray::builder(haystack.len()); + let converter = RowConverter::new(vec![SortField::new(haystack.value_type())])?; + + for (arr, sub_arr) in haystack.iter().zip(needle.iter()) { + if let (Some(arr), Some(sub_arr)) = (arr, sub_arr) { + let arr_values = converter.convert_columns(&[arr])?; + let sub_arr_values = converter.convert_columns(&[sub_arr])?; + boolean_builder.append_value(general_array_has_all_and_any_kernel( + arr_values, + sub_arr_values, + comparison_type, + )); + } else { + boolean_builder.append_null(); + } + } + + Ok(Arc::new(boolean_builder.finish())) +} + +fn general_array_has_all_and_any_kernel( + haystack_rows: Rows, + needle_rows: Rows, + comparison_type: ComparisonType, +) -> bool { + match comparison_type { + ComparisonType::All => needle_rows.iter().all(|needle_row| { + haystack_rows + .iter() + .any(|haystack_row| haystack_row == needle_row) + }), + ComparisonType::Any => needle_rows.iter().any(|needle_row| { + haystack_rows + .iter() + .any(|haystack_row| haystack_row == needle_row) + }), + } +} From 31adb083e000641d568fe6b3f611d602031e27be Mon Sep 17 00:00:00 2001 From: Jax Liu Date: Sat, 24 Aug 2024 13:14:21 +0800 Subject: [PATCH 008/154] Throw `not_impl_error` for `approx_percentile_cont` parameters validation (#12133) * handle the get scalar value as not_impl_err * simplify the error handling --- .../src/approx_percentile_cont.rs | 12 +++++++----- datafusion/sqllogictest/test_files/aggregate.slt | 6 ++++++ 2 files changed, 13 insertions(+), 5 deletions(-) diff --git a/datafusion/functions-aggregate/src/approx_percentile_cont.rs b/datafusion/functions-aggregate/src/approx_percentile_cont.rs index 89d827e86859..e860783dbe0c 100644 --- a/datafusion/functions-aggregate/src/approx_percentile_cont.rs +++ b/datafusion/functions-aggregate/src/approx_percentile_cont.rs @@ -31,8 +31,8 @@ use arrow::{ use arrow_schema::{Field, Schema}; use datafusion_common::{ - downcast_value, internal_err, not_impl_err, plan_err, DataFusionError, Result, - ScalarValue, + downcast_value, internal_err, not_impl_datafusion_err, not_impl_err, plan_err, + DataFusionError, Result, ScalarValue, }; use datafusion_expr::function::{AccumulatorArgs, StateFieldsArgs}; use datafusion_expr::type_coercion::aggregates::{INTEGERS, NUMERICS}; @@ -154,7 +154,8 @@ fn get_scalar_value(expr: &Arc) -> Result { } fn validate_input_percentile_expr(expr: &Arc) -> Result { - let percentile = match get_scalar_value(expr)? { + let percentile = match get_scalar_value(expr) + .map_err(|_| not_impl_datafusion_err!("Percentile value for 'APPROX_PERCENTILE_CONT' must be a literal, got: {expr}"))? { ScalarValue::Float32(Some(value)) => { value as f64 } @@ -179,7 +180,8 @@ fn validate_input_percentile_expr(expr: &Arc) -> Result { } fn validate_input_max_size_expr(expr: &Arc) -> Result { - let max_size = match get_scalar_value(expr)? { + let max_size = match get_scalar_value(expr) + .map_err(|_| not_impl_datafusion_err!("Tdigest max_size value for 'APPROX_PERCENTILE_CONT' must be a literal, got: {expr}"))? { ScalarValue::UInt8(Some(q)) => q as usize, ScalarValue::UInt16(Some(q)) => q as usize, ScalarValue::UInt32(Some(q)) => q as usize, @@ -193,7 +195,7 @@ fn validate_input_max_size_expr(expr: &Arc) -> Result { "Tdigest max_size value for 'APPROX_PERCENTILE_CONT' must be UInt > 0 literal (got data type {}).", sv.data_type() ) - } + }, }; Ok(max_size) diff --git a/datafusion/sqllogictest/test_files/aggregate.slt b/datafusion/sqllogictest/test_files/aggregate.slt index 09fc397bf915..c3609e59f265 100644 --- a/datafusion/sqllogictest/test_files/aggregate.slt +++ b/datafusion/sqllogictest/test_files/aggregate.slt @@ -98,6 +98,12 @@ SELECT approx_percentile_cont(c3, 0.95, 111.1) FROM aggregate_test_100 statement error DataFusion error: Error during planning: Error during planning: Coercion from \[Float64, Float64, Float64\] to the signature OneOf(.*) failed(.|\n)* SELECT approx_percentile_cont(c12, 0.95, 111.1) FROM aggregate_test_100 +statement error DataFusion error: This feature is not implemented: Percentile value for 'APPROX_PERCENTILE_CONT' must be a literal +SELECT approx_percentile_cont(c12, c12) FROM aggregate_test_100 + +statement error DataFusion error: This feature is not implemented: Tdigest max_size value for 'APPROX_PERCENTILE_CONT' must be a literal +SELECT approx_percentile_cont(c12, 0.95, c5) FROM aggregate_test_100 + # array agg can use order by query ? SELECT array_agg(c13 ORDER BY c13) From 6f183043e9c28a55311e187e9e78caad330976af Mon Sep 17 00:00:00 2001 From: Jonah Gao Date: Sat, 24 Aug 2024 17:28:09 +0800 Subject: [PATCH 009/154] fix: set `supports_retract_batch` to false for `ApproxPercentileAccumulator` (#12132) --- datafusion/functions-aggregate/src/approx_percentile_cont.rs | 4 ---- datafusion/sqllogictest/test_files/aggregate.slt | 5 +++++ 2 files changed, 5 insertions(+), 4 deletions(-) diff --git a/datafusion/functions-aggregate/src/approx_percentile_cont.rs b/datafusion/functions-aggregate/src/approx_percentile_cont.rs index e860783dbe0c..867dfed485c7 100644 --- a/datafusion/functions-aggregate/src/approx_percentile_cont.rs +++ b/datafusion/functions-aggregate/src/approx_percentile_cont.rs @@ -460,10 +460,6 @@ impl Accumulator for ApproxPercentileAccumulator { + self.return_type.size() - std::mem::size_of_val(&self.return_type) } - - fn supports_retract_batch(&self) -> bool { - true - } } #[cfg(test)] diff --git a/datafusion/sqllogictest/test_files/aggregate.slt b/datafusion/sqllogictest/test_files/aggregate.slt index c3609e59f265..b826dd31f4d1 100644 --- a/datafusion/sqllogictest/test_files/aggregate.slt +++ b/datafusion/sqllogictest/test_files/aggregate.slt @@ -104,6 +104,11 @@ SELECT approx_percentile_cont(c12, c12) FROM aggregate_test_100 statement error DataFusion error: This feature is not implemented: Tdigest max_size value for 'APPROX_PERCENTILE_CONT' must be a literal SELECT approx_percentile_cont(c12, 0.95, c5) FROM aggregate_test_100 +# Not supported over sliding windows +query error This feature is not implemented: Aggregate can not be used as a sliding accumulator because `retract_batch` is not implemented +SELECT approx_percentile_cont(c3, 0.5) OVER (ROWS BETWEEN 4 PRECEDING AND CURRENT ROW) +FROM aggregate_test_100 + # array agg can use order by query ? SELECT array_agg(c13 ORDER BY c13) From 2b3d45666701d6bb053494f9f1d4c4b3186368e0 Mon Sep 17 00:00:00 2001 From: Yongting You <2010youy01@gmail.com> Date: Sat, 24 Aug 2024 17:37:25 +0800 Subject: [PATCH 010/154] minor: Add comments for `GroupedHashAggregateStream` struct (#12127) * Add doc for Aggr struct * Fix format --- .../physical-plan/src/aggregates/row_hash.rs | 124 ++++++++++++------ 1 file changed, 83 insertions(+), 41 deletions(-) diff --git a/datafusion/physical-plan/src/aggregates/row_hash.rs b/datafusion/physical-plan/src/aggregates/row_hash.rs index b3221752d034..05f4ec621813 100644 --- a/datafusion/physical-plan/src/aggregates/row_hash.rs +++ b/datafusion/physical-plan/src/aggregates/row_hash.rs @@ -76,35 +76,45 @@ use super::AggregateExec; /// This encapsulates the spilling state struct SpillState { - /// If data has previously been spilled, the locations of the - /// spill files (in Arrow IPC format) - spills: Vec, - + // ======================================================================== + // PROPERTIES: + // These fields are initialized at the start and remain constant throughout + // the execution. + // ======================================================================== /// Sorting expression for spilling batches spill_expr: Vec, /// Schema for spilling batches spill_schema: SchemaRef, - /// true when streaming merge is in progress - is_stream_merging: bool, - /// aggregate_arguments for merging spilled data merging_aggregate_arguments: Vec>>, /// GROUP BY expressions for merging spilled data merging_group_by: PhysicalGroupBy, + + // ======================================================================== + // STATES: + // Fields changes during execution. Can be buffer, or state flags that + // influence the exeuction in parent `GroupedHashAggregateStream` + // ======================================================================== + /// If data has previously been spilled, the locations of the + /// spill files (in Arrow IPC format) + spills: Vec, + + /// true when streaming merge is in progress + is_stream_merging: bool, } /// Tracks if the aggregate should skip partial aggregations /// /// See "partial aggregation" discussion on [`GroupedHashAggregateStream`] struct SkipAggregationProbe { - /// Number of processed input rows (updated during probing) - input_rows: usize, - /// Number of total group values for `input_rows` (updated during probing) - num_groups: usize, - + // ======================================================================== + // PROPERTIES: + // These fields are initialized at the start and remain constant throughout + // the execution. + // ======================================================================== /// Aggregation ratio check performed when the number of input rows exceeds /// this threshold (from `SessionConfig`) probe_rows_threshold: usize, @@ -113,6 +123,16 @@ struct SkipAggregationProbe { /// is skipped and input rows are directly converted to output probe_ratio_threshold: f64, + // ======================================================================== + // STATES: + // Fields changes during execution. Can be buffer, or state flags that + // influence the exeuction in parent `GroupedHashAggregateStream` + // ======================================================================== + /// Number of processed input rows (updated during probing) + input_rows: usize, + /// Number of total group values for `input_rows` (updated during probing) + num_groups: usize, + /// Flag indicating further data aggregation may be skipped (decision made /// when probing complete) should_skip: bool, @@ -316,17 +336,15 @@ impl SkipAggregationProbe { /// └─────────────────┘ └─────────────────┘ /// ``` pub(crate) struct GroupedHashAggregateStream { + // ======================================================================== + // PROPERTIES: + // These fields are initialized at the start and remain constant throughout + // the execution. + // ======================================================================== schema: SchemaRef, input: SendableRecordBatchStream, mode: AggregateMode, - /// Accumulators, one for each `AggregateExpr` in the query - /// - /// For example, if the query has aggregates, `SUM(x)`, - /// `COUNT(y)`, there will be two accumulators, each one - /// specialized for that particular aggregate and its input types - accumulators: Vec>, - /// Arguments to pass to each accumulator. /// /// The arguments in `accumulator[i]` is passed `aggregate_arguments[i]` @@ -347,9 +365,30 @@ pub(crate) struct GroupedHashAggregateStream { /// GROUP BY expressions group_by: PhysicalGroupBy, - /// The memory reservation for this grouping - reservation: MemoryReservation, + /// max rows in output RecordBatches + batch_size: usize, + + /// Optional soft limit on the number of `group_values` in a batch + /// If the number of `group_values` in a single batch exceeds this value, + /// the `GroupedHashAggregateStream` operation immediately switches to + /// output mode and emits all groups. + group_values_soft_limit: Option, + // ======================================================================== + // STATE FLAGS: + // These fields will be updated during the execution. And control the flow of + // the execution. + // ======================================================================== + /// Tracks if this stream is generating input or output + exec_state: ExecutionState, + + /// Have we seen the end of the input + input_done: bool, + + // ======================================================================== + // STATE BUFFERS: + // These fields will accumulate intermediate results during the execution. + // ======================================================================== /// An interning store of group keys group_values: Box, @@ -357,38 +396,41 @@ pub(crate) struct GroupedHashAggregateStream { /// processed. Reused across batches here to avoid reallocations current_group_indices: Vec, - /// Tracks if this stream is generating input or output - exec_state: ExecutionState, - - /// Execution metrics - baseline_metrics: BaselineMetrics, - - /// max rows in output RecordBatches - batch_size: usize, + /// Accumulators, one for each `AggregateExpr` in the query + /// + /// For example, if the query has aggregates, `SUM(x)`, + /// `COUNT(y)`, there will be two accumulators, each one + /// specialized for that particular aggregate and its input types + accumulators: Vec>, + // ======================================================================== + // TASK-SPECIFIC STATES: + // Inner states groups together properties, states for a specific task. + // ======================================================================== /// Optional ordering information, that might allow groups to be /// emitted from the hash table prior to seeing the end of the /// input group_ordering: GroupOrdering, - /// Have we seen the end of the input - input_done: bool, - - /// The [`RuntimeEnv`] associated with the [`TaskContext`] argument - runtime: Arc, - /// The spill state object spill_state: SpillState, - /// Optional soft limit on the number of `group_values` in a batch - /// If the number of `group_values` in a single batch exceeds this value, - /// the `GroupedHashAggregateStream` operation immediately switches to - /// output mode and emits all groups. - group_values_soft_limit: Option, - /// Optional probe for skipping data aggregation, if supported by /// current stream. skip_aggregation_probe: Option, + + // ======================================================================== + // EXECUTION RESOURCES: + // Fields related to managing execution resources and monitoring performance. + // ======================================================================== + /// The memory reservation for this grouping + reservation: MemoryReservation, + + /// Execution metrics + baseline_metrics: BaselineMetrics, + + /// The [`RuntimeEnv`] associated with the [`TaskContext`] argument + runtime: Arc, } impl GroupedHashAggregateStream { From e4c3d64a6683fbea6c0244c9668926fee24f7784 Mon Sep 17 00:00:00 2001 From: Trent Hauck Date: Sat, 24 Aug 2024 02:55:28 -0700 Subject: [PATCH 011/154] feat: support upper and lower for stringview (#12138) --- datafusion/functions/src/string/common.rs | 23 ++++++++++++- datafusion/functions/src/string/lower.rs | 2 +- datafusion/functions/src/string/upper.rs | 2 +- .../sqllogictest/test_files/string_view.slt | 34 ++++++++++++++++--- 4 files changed, 53 insertions(+), 8 deletions(-) diff --git a/datafusion/functions/src/string/common.rs b/datafusion/functions/src/string/common.rs index 6f23a5ddd236..a5dc22b4d9e4 100644 --- a/datafusion/functions/src/string/common.rs +++ b/datafusion/functions/src/string/common.rs @@ -23,7 +23,7 @@ use std::sync::Arc; use arrow::array::{ new_null_array, Array, ArrayAccessor, ArrayDataBuilder, ArrayIter, ArrayRef, GenericStringArray, GenericStringBuilder, OffsetSizeTrait, StringArray, - StringViewArray, + StringBuilder, StringViewArray, }; use arrow::buffer::{Buffer, MutableBuffer, NullBuffer}; use arrow::datatypes::DataType; @@ -214,6 +214,23 @@ where i64, _, >(array, op)?)), + DataType::Utf8View => { + let string_array = as_string_view_array(array)?; + let mut string_builder = StringBuilder::with_capacity( + string_array.len(), + string_array.get_array_memory_size(), + ); + + for str in string_array.iter() { + if let Some(str) = str { + string_builder.append_value(op(str)); + } else { + string_builder.append_null(); + } + } + + Ok(ColumnarValue::Array(Arc::new(string_builder.finish()))) + } other => exec_err!("Unsupported data type {other:?} for function {name}"), }, ColumnarValue::Scalar(scalar) => match scalar { @@ -225,6 +242,10 @@ where let result = a.as_ref().map(|x| op(x)); Ok(ColumnarValue::Scalar(ScalarValue::LargeUtf8(result))) } + ScalarValue::Utf8View(a) => { + let result = a.as_ref().map(|x| op(x)); + Ok(ColumnarValue::Scalar(ScalarValue::Utf8(result))) + } other => exec_err!("Unsupported data type {other:?} for function {name}"), }, } diff --git a/datafusion/functions/src/string/lower.rs b/datafusion/functions/src/string/lower.rs index 29ca682c380b..ca324e69c0d2 100644 --- a/datafusion/functions/src/string/lower.rs +++ b/datafusion/functions/src/string/lower.rs @@ -43,7 +43,7 @@ impl LowerFunc { Self { signature: Signature::uniform( 1, - vec![Utf8, LargeUtf8], + vec![Utf8, LargeUtf8, Utf8View], Volatility::Immutable, ), } diff --git a/datafusion/functions/src/string/upper.rs b/datafusion/functions/src/string/upper.rs index da31948fbcfa..593e33ab6bb4 100644 --- a/datafusion/functions/src/string/upper.rs +++ b/datafusion/functions/src/string/upper.rs @@ -40,7 +40,7 @@ impl UpperFunc { Self { signature: Signature::uniform( 1, - vec![Utf8, LargeUtf8], + vec![Utf8, LargeUtf8, Utf8View], Volatility::Immutable, ), } diff --git a/datafusion/sqllogictest/test_files/string_view.slt b/datafusion/sqllogictest/test_files/string_view.slt index 3b3d7b88a4a1..83c75b8df38c 100644 --- a/datafusion/sqllogictest/test_files/string_view.slt +++ b/datafusion/sqllogictest/test_files/string_view.slt @@ -460,8 +460,6 @@ Xiangpeng Raphael NULL - - ### Initcap query TT @@ -501,7 +499,7 @@ SELECT INITCAP(column1_large_utf8_lower) as c3 FROM test_lowercase; ---- -Andrew Andrew Andrew +Andrew Andrew Andrew Xiangpeng Xiangpeng Xiangpeng Raphael Raphael Raphael NULL NULL NULL @@ -828,16 +826,42 @@ logical_plan 02)--TableScan: test projection=[column1_utf8view, column2_utf8view] ## Ensure no casts for LOWER -## TODO https://github.com/apache/datafusion/issues/11855 query TT EXPLAIN SELECT LOWER(column1_utf8view) as c1 FROM test; ---- logical_plan -01)Projection: lower(CAST(test.column1_utf8view AS Utf8)) AS c1 +01)Projection: lower(test.column1_utf8view) AS c1 +02)--TableScan: test projection=[column1_utf8view] + +query T +SELECT LOWER(column1_utf8view) as c1 +FROM test; +---- +andrew +xiangpeng +raphael +NULL + +## Ensure no casts for UPPER +query TT +EXPLAIN SELECT + UPPER(column1_utf8view) as c1 +FROM test; +---- +logical_plan +01)Projection: upper(test.column1_utf8view) AS c1 02)--TableScan: test projection=[column1_utf8view] +query T +SELECT UPPER(column1_utf8view) as c1 +FROM test; +---- +ANDREW +XIANGPENG +RAPHAEL +NULL ## Ensure no casts for LPAD query TT From a58416c2eae47d19f0ef4756768f1e978d7c7dd3 Mon Sep 17 00:00:00 2001 From: iamthinh Date: Sat, 24 Aug 2024 04:00:38 -0700 Subject: [PATCH 012/154] fix concat dictionary(int32, utf8) bug (#12143) * fix concat bug * clippy fix --- .../expr-common/src/type_coercion/binary.rs | 3 ++ .../sqllogictest/test_files/type_coercion.slt | 29 +++++++++++++++++++ 2 files changed, 32 insertions(+) diff --git a/datafusion/expr-common/src/type_coercion/binary.rs b/datafusion/expr-common/src/type_coercion/binary.rs index 401762ad4d36..552bc89b38f4 100644 --- a/datafusion/expr-common/src/type_coercion/binary.rs +++ b/datafusion/expr-common/src/type_coercion/binary.rs @@ -936,6 +936,9 @@ fn string_concat_coercion(lhs_type: &DataType, rhs_type: &DataType) -> Option { string_concat_internal_coercion(from_type, &LargeUtf8) } + (Dictionary(_, lhs_value_type), Dictionary(_, rhs_value_type)) => { + string_coercion(lhs_value_type, rhs_value_type).or(None) + } _ => None, }) } diff --git a/datafusion/sqllogictest/test_files/type_coercion.slt b/datafusion/sqllogictest/test_files/type_coercion.slt index e420c0cc7155..0f9399cede2e 100644 --- a/datafusion/sqllogictest/test_files/type_coercion.slt +++ b/datafusion/sqllogictest/test_files/type_coercion.slt @@ -50,6 +50,35 @@ select interval '1 month' - '2023-05-01'::date; query error DataFusion error: Error during planning: Cannot coerce arithmetic expression Interval\(MonthDayNano\) \- Timestamp\(Nanosecond, None\) to valid types SELECT interval '1 month' - '2023-05-01 12:30:00'::timestamp; +# dictionary(int32, utf8) -> utf8 +query T +select arrow_cast('foo', 'Dictionary(Int32, Utf8)') || arrow_cast('bar', 'Dictionary(Int32, Utf8)'); +---- +foobar + +# dictionary(int32, largeUtf8) -> largeUtf8 +query T +select arrow_cast('foo', 'Dictionary(Int32, LargeUtf8)') || arrow_cast('bar', 'Dictionary(Int32, LargeUtf8)'); +---- +foobar + +#################################### +## Concat column dictionary test ## +#################################### +statement ok +create table t as values (arrow_cast('foo', 'Dictionary(Int32, Utf8)'), arrow_cast('bar', 'Dictionary(Int32, Utf8)')); + +query T +select column1 || column2 from t; +---- +foobar + +statement ok +DROP TABLE t + +####################################### +## Concat column dictionary test end ## +####################################### #################################### ## Test type coercion with UNIONs ## From 14d640433eac9314f37edc1d12ff3260a4778923 Mon Sep 17 00:00:00 2001 From: Jay Zhan Date: Sat, 24 Aug 2024 22:03:17 +0800 Subject: [PATCH 013/154] `array_has` with eq kernel (#12125) * first draft Signed-off-by: jayzhan211 * avoid row converter for string Signed-off-by: jayzhan211 * cleanup Signed-off-by: jayzhan211 * string view Signed-off-by: jayzhan211 * trigger ci Signed-off-by: jayzhan211 * refactor Signed-off-by: jayzhan211 * cleanup Signed-off-by: jayzhan211 * array_has_all Signed-off-by: jayzhan211 * array_has_any Signed-off-by: jayzhan211 * cleanup Signed-off-by: jayzhan211 * cleanup Signed-off-by: jayzhan211 * fmt Signed-off-by: jayzhan211 * rm unused import Signed-off-by: jayzhan211 * cleanup Signed-off-by: jayzhan211 * backup Signed-off-by: jayzhan211 * add bench Signed-off-by: jayzhan211 * new approach Signed-off-by: jayzhan211 * general scalar wins Signed-off-by: jayzhan211 * cleanup Signed-off-by: jayzhan211 * cleanup Signed-off-by: jayzhan211 * revert query Signed-off-by: jayzhan211 * cleanup Signed-off-by: jayzhan211 * cleanup Signed-off-by: jayzhan211 * reuse slice and fix typo Signed-off-by: jayzhan211 --------- Signed-off-by: jayzhan211 --- datafusion-cli/Cargo.lock | 1 + datafusion/functions-nested/Cargo.toml | 1 + datafusion/functions-nested/src/array_has.rs | 188 ++++++++++++------- datafusion/sqllogictest/test_files/array.slt | 14 +- 4 files changed, 130 insertions(+), 74 deletions(-) diff --git a/datafusion-cli/Cargo.lock b/datafusion-cli/Cargo.lock index 456dc295d449..50333d17ca8d 100644 --- a/datafusion-cli/Cargo.lock +++ b/datafusion-cli/Cargo.lock @@ -1424,6 +1424,7 @@ dependencies = [ "datafusion-expr", "datafusion-functions", "datafusion-functions-aggregate", + "datafusion-physical-expr-common", "itertools", "log", "paste", diff --git a/datafusion/functions-nested/Cargo.toml b/datafusion/functions-nested/Cargo.toml index 5e1a15233cb5..bdfb07031b8c 100644 --- a/datafusion/functions-nested/Cargo.toml +++ b/datafusion/functions-nested/Cargo.toml @@ -50,6 +50,7 @@ datafusion-execution = { workspace = true } datafusion-expr = { workspace = true } datafusion-functions = { workspace = true } datafusion-functions-aggregate = { workspace = true } +datafusion-physical-expr-common = { workspace = true } itertools = { workspace = true, features = ["use_std"] } log = { workspace = true } paste = "1.0.14" diff --git a/datafusion/functions-nested/src/array_has.rs b/datafusion/functions-nested/src/array_has.rs index d7c194304887..9b4357d0d14f 100644 --- a/datafusion/functions-nested/src/array_has.rs +++ b/datafusion/functions-nested/src/array_has.rs @@ -20,12 +20,13 @@ use arrow::array::{Array, ArrayRef, BooleanArray, OffsetSizeTrait}; use arrow::datatypes::DataType; use arrow::row::{RowConverter, Rows, SortField}; -use arrow_array::GenericListArray; +use arrow_array::{Datum, GenericListArray, Scalar}; use datafusion_common::cast::as_generic_list_array; use datafusion_common::utils::string_utils::string_array_to_vec; -use datafusion_common::{exec_err, Result}; -use datafusion_expr::{ColumnarValue, ScalarUDFImpl, Signature, Volatility}; +use datafusion_common::{exec_err, Result, ScalarValue}; +use datafusion_expr::{ColumnarValue, Operator, ScalarUDFImpl, Signature, Volatility}; +use datafusion_physical_expr_common::datum::compare_op_for_nested; use itertools::Itertools; use crate::utils::make_scalar_function; @@ -95,7 +96,45 @@ impl ScalarUDFImpl for ArrayHas { } fn invoke(&self, args: &[ColumnarValue]) -> Result { - make_scalar_function(array_has_inner)(args) + // Always return null if the second argumet is null + // i.e. array_has(array, null) -> null + if let ColumnarValue::Scalar(s) = &args[1] { + if s.is_null() { + return Ok(ColumnarValue::Scalar(ScalarValue::Boolean(None))); + } + } + + // first, identify if any of the arguments is an Array. If yes, store its `len`, + // as any scalar will need to be converted to an array of len `len`. + let len = args + .iter() + .fold(Option::::None, |acc, arg| match arg { + ColumnarValue::Scalar(_) => acc, + ColumnarValue::Array(a) => Some(a.len()), + }); + + let is_scalar = len.is_none(); + + let result = match args[1] { + ColumnarValue::Array(_) => { + let args = ColumnarValue::values_to_arrays(args)?; + array_has_inner_for_array(&args[0], &args[1]) + } + ColumnarValue::Scalar(_) => { + let haystack = args[0].to_owned().into_array(1)?; + let needle = args[1].to_owned().into_array(1)?; + let needle = Scalar::new(needle); + array_has_inner_for_scalar(&haystack, &needle) + } + }; + + 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) + } } fn aliases(&self) -> &[String] { @@ -103,17 +142,86 @@ impl ScalarUDFImpl for ArrayHas { } } -fn array_has_inner(args: &[ArrayRef]) -> Result { - match args[0].data_type() { - DataType::List(_) => array_has_dispatch::(&args[0], &args[1]), - DataType::LargeList(_) => array_has_dispatch::(&args[0], &args[1]), +fn array_has_inner_for_scalar( + haystack: &ArrayRef, + needle: &dyn Datum, +) -> Result { + match haystack.data_type() { + DataType::List(_) => array_has_dispatch_for_scalar::(haystack, needle), + DataType::LargeList(_) => array_has_dispatch_for_scalar::(haystack, needle), _ => exec_err!( "array_has does not support type '{:?}'.", - args[0].data_type() + haystack.data_type() + ), + } +} + +fn array_has_inner_for_array(haystack: &ArrayRef, needle: &ArrayRef) -> Result { + match haystack.data_type() { + DataType::List(_) => array_has_dispatch_for_array::(haystack, needle), + DataType::LargeList(_) => array_has_dispatch_for_array::(haystack, needle), + _ => exec_err!( + "array_has does not support type '{:?}'.", + haystack.data_type() ), } } +fn array_has_dispatch_for_array( + haystack: &ArrayRef, + needle: &ArrayRef, +) -> Result { + let haystack = as_generic_list_array::(haystack)?; + let mut boolean_builder = BooleanArray::builder(haystack.len()); + + for (i, arr) in haystack.iter().enumerate() { + if arr.is_none() || needle.is_null(i) { + boolean_builder.append_null(); + continue; + } + let arr = arr.unwrap(); + let needle_row = Scalar::new(needle.slice(i, 1)); + let eq_array = compare_op_for_nested(Operator::Eq, &arr, &needle_row)?; + let is_contained = eq_array.true_count() > 0; + boolean_builder.append_value(is_contained) + } + + Ok(Arc::new(boolean_builder.finish())) +} + +fn array_has_dispatch_for_scalar( + haystack: &ArrayRef, + needle: &dyn Datum, +) -> Result { + let haystack = as_generic_list_array::(haystack)?; + let values = haystack.values(); + let offsets = haystack.value_offsets(); + // If first argument is empty list (second argument is non-null), return false + // i.e. array_has([], non-null element) -> false + if values.len() == 0 { + return Ok(Arc::new(BooleanArray::from(vec![Some(false)]))); + } + let eq_array = compare_op_for_nested(Operator::Eq, values, needle)?; + let mut final_contained = vec![None; haystack.len()]; + for (i, offset) in offsets.windows(2).enumerate() { + let start = offset[0].to_usize().unwrap(); + let end = offset[1].to_usize().unwrap(); + let length = end - start; + // For non-nested list, length is 0 for null + if length == 0 { + continue; + } + let sliced_array = eq_array.slice(start, length); + // For nested list, check number of nulls + if sliced_array.null_count() == length { + continue; + } + final_contained[i] = Some(sliced_array.true_count() > 0); + } + + Ok(Arc::new(BooleanArray::from(final_contained))) +} + fn array_has_all_inner(args: &[ArrayRef]) -> Result { match args[0].data_type() { DataType::List(_) => { @@ -245,19 +353,6 @@ enum ComparisonType { Any, } -fn array_has_dispatch( - haystack: &ArrayRef, - needle: &ArrayRef, -) -> Result { - let haystack = as_generic_list_array::(haystack)?; - match needle.data_type() { - DataType::Utf8 | DataType::LargeUtf8 | DataType::Utf8View => { - array_has_string_internal::(haystack, needle) - } - _ => general_array_has::(haystack, needle), - } -} - fn array_has_all_and_any_dispatch( haystack: &ArrayRef, needle: &ArrayRef, @@ -273,55 +368,6 @@ fn array_has_all_and_any_dispatch( } } -fn array_has_string_internal( - haystack: &GenericListArray, - needle: &ArrayRef, -) -> Result { - let mut boolean_builder = BooleanArray::builder(haystack.len()); - for (arr, element) in haystack.iter().zip(string_array_to_vec(needle).into_iter()) { - match (arr, element) { - (Some(arr), Some(element)) => { - boolean_builder.append_value( - string_array_to_vec(&arr) - .into_iter() - .flatten() - .any(|x| x == element), - ); - } - (_, _) => { - boolean_builder.append_null(); - } - } - } - - Ok(Arc::new(boolean_builder.finish())) -} - -fn general_array_has( - array: &GenericListArray, - needle: &ArrayRef, -) -> Result { - let mut boolean_builder = BooleanArray::builder(array.len()); - let converter = RowConverter::new(vec![SortField::new(array.value_type())])?; - let sub_arr_values = converter.convert_columns(&[Arc::clone(needle)])?; - - for (row_idx, arr) in array.iter().enumerate() { - if let Some(arr) = arr { - let arr_values = converter.convert_columns(&[arr])?; - boolean_builder.append_value( - arr_values - .iter() - .dedup() - .any(|x| x == sub_arr_values.row(row_idx)), - ); - } else { - boolean_builder.append_null(); - } - } - - Ok(Arc::new(boolean_builder.finish())) -} - // String comparison for array_has_all and array_has_any fn array_has_all_and_any_string_internal( array: &GenericListArray, diff --git a/datafusion/sqllogictest/test_files/array.slt b/datafusion/sqllogictest/test_files/array.slt index 249241a51aea..c80fd7e92417 100644 --- a/datafusion/sqllogictest/test_files/array.slt +++ b/datafusion/sqllogictest/test_files/array.slt @@ -4972,11 +4972,19 @@ NULL 1 1 ## array_has/array_has_all/array_has_any -query BB +# If lhs is empty, return false +query B +select array_has([], 1); +---- +false + +# If rhs is Null, we returns Null +query BBB select array_has([], null), - array_has([1, 2, 3], null); + array_has([1, 2, 3], null), + array_has([null, 1], null); ---- -false false +NULL NULL NULL #TODO: array_has_all and array_has_any cannot handle NULL #query BBBB From 696ac49aeaedfd4346051c725222aa85cd2b3c81 Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Sat, 24 Aug 2024 23:02:59 +0200 Subject: [PATCH 014/154] Check for overflow in substring with negative start (#12141) --- datafusion/functions/src/unicode/substr.rs | 47 +++++++++++++++++----- 1 file changed, 37 insertions(+), 10 deletions(-) diff --git a/datafusion/functions/src/unicode/substr.rs b/datafusion/functions/src/unicode/substr.rs index 9fd8c75eab23..db218f9127ae 100644 --- a/datafusion/functions/src/unicode/substr.rs +++ b/datafusion/functions/src/unicode/substr.rs @@ -25,7 +25,7 @@ use arrow::array::{ use arrow::datatypes::DataType; use datafusion_common::cast::as_int64_array; -use datafusion_common::{exec_err, Result}; +use datafusion_common::{exec_datafusion_err, exec_err, Result}; use datafusion_expr::TypeSignature::Exact; use datafusion_expr::{ColumnarValue, ScalarUDFImpl, Signature, Volatility}; @@ -144,19 +144,23 @@ where let result = iter .zip(start_array.iter()) .zip(count_array.iter()) - .map(|((string, start), count)| match (string, start, count) { - (Some(string), Some(start), Some(count)) => { - if count < 0 { - exec_err!( + .map(|((string, start), count)| { + match (string, start, count) { + (Some(string), Some(start), Some(count)) => { + if count < 0 { + exec_err!( "negative substring length not allowed: substr(, {start}, {count})" ) - } else { - let skip = max(0, start - 1); - let count = max(0, count + (if start < 1 {start - 1} else {0})); - Ok(Some(string.chars().skip(skip as usize).take(count as usize).collect::())) + } else { + let skip = max(0, start.checked_sub(1).ok_or_else( + || exec_datafusion_err!("negative overflow when calculating skip value") + )?); + let count = max(0, count + (if start < 1 { start - 1 } else { 0 })); + Ok(Some(string.chars().skip(skip as usize).take(count as usize).collect::())) + } } + _ => Ok(None), } - _ => Ok(None), }) .collect::>>()?; @@ -482,6 +486,29 @@ mod tests { Utf8, StringArray ); + test_function!( + SubstrFunc::new(), + &[ + ColumnarValue::Scalar(ScalarValue::from("abc")), + ColumnarValue::Scalar(ScalarValue::from(-9223372036854775808i64)), + ], + Ok(Some("abc")), + &str, + Utf8, + StringArray + ); + test_function!( + SubstrFunc::new(), + &[ + ColumnarValue::Scalar(ScalarValue::from("overflow")), + ColumnarValue::Scalar(ScalarValue::from(-9223372036854775808i64)), + ColumnarValue::Scalar(ScalarValue::from(1i64)), + ], + exec_err!("negative overflow when calculating skip value"), + &str, + Utf8, + StringArray + ); Ok(()) } From 90420861deada38190416d5ae7e77cfa1dd2fba4 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Sat, 24 Aug 2024 17:04:42 -0400 Subject: [PATCH 015/154] Minor: add test for panic propagation (#12134) --- datafusion/common-runtime/src/common.rs | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/datafusion/common-runtime/src/common.rs b/datafusion/common-runtime/src/common.rs index 698a846b4844..30f7526bc0b2 100644 --- a/datafusion/common-runtime/src/common.rs +++ b/datafusion/common-runtime/src/common.rs @@ -108,4 +108,14 @@ mod tests { Err(e) if e.is_cancelled() )); } + + #[tokio::test] + #[should_panic(expected = "foo")] + async fn panic_resume() { + // this should panic w/o an `unwrap` + SpawnedTask::spawn(async { panic!("foo") }) + .join_unwind() + .await + .ok(); + } } From 23ccca9cd9bdf1791985f2b4abb6d5d8a4fc5ece Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Sat, 24 Aug 2024 17:09:31 -0400 Subject: [PATCH 016/154] Add example for configuring SessionContext (#12139) --- datafusion/core/src/execution/context/mod.rs | 42 +++++++++++++++++++- datafusion/core/src/execution/mod.rs | 2 + 2 files changed, 43 insertions(+), 1 deletion(-) diff --git a/datafusion/core/src/execution/context/mod.rs b/datafusion/core/src/execution/context/mod.rs index 972a6f643733..49cbf9ebde9d 100644 --- a/datafusion/core/src/execution/context/mod.rs +++ b/datafusion/core/src/execution/context/mod.rs @@ -128,6 +128,9 @@ where /// the state of the connection between a user and an instance of the /// DataFusion engine. /// +/// See examples below for how to use the `SessionContext` to execute queries +/// and how to configure the session. +/// /// # Overview /// /// [`SessionContext`] provides the following functionality: @@ -200,7 +203,38 @@ where /// # } /// ``` /// -/// # `SessionContext`, `SessionState`, and `TaskContext` +/// # Example: Configuring `SessionContext` +/// +/// The `SessionContext` can be configured by creating a [`SessionState`] using +/// [`SessionStateBuilder`]: +/// +/// ``` +/// # use std::sync::Arc; +/// # use datafusion::prelude::*; +/// # use datafusion::execution::SessionStateBuilder; +/// # use datafusion_execution::runtime_env::{RuntimeConfig, RuntimeEnv}; +/// // Configure a 4k batch size +/// let config = SessionConfig::new() .with_batch_size(4 * 1024); +/// +/// // configure a memory limit of 1GB with 20% slop +/// let runtime_env = RuntimeEnv::new( +/// RuntimeConfig::new() +/// .with_memory_limit(1024 * 1024 * 1024, 0.80) +/// ).unwrap(); +/// +/// // Create a SessionState using the config and runtime_env +/// let state = SessionStateBuilder::new() +/// .with_config(config) +/// .with_runtime_env(Arc::new(runtime_env)) +/// // include support for built in functions and configurations +/// .with_default_features() +/// .build(); +/// +/// // Create a SessionContext +/// let ctx = SessionContext::from(state); +/// ``` +/// +/// # Relationship between `SessionContext`, `SessionState`, and `TaskContext` /// /// The state required to optimize, and evaluate queries is /// broken into three levels to allow tailoring @@ -1427,6 +1461,12 @@ impl From<&SessionContext> for TaskContext { } } +impl From for SessionContext { + fn from(state: SessionState) -> Self { + Self::new_with_state(state) + } +} + /// A planner used to add extensions to DataFusion logical and physical plans. #[async_trait] pub trait QueryPlanner { diff --git a/datafusion/core/src/execution/mod.rs b/datafusion/core/src/execution/mod.rs index a1b3eab25f33..10aa16ffe47a 100644 --- a/datafusion/core/src/execution/mod.rs +++ b/datafusion/core/src/execution/mod.rs @@ -19,6 +19,8 @@ pub mod context; pub mod session_state; +pub use session_state::{SessionState, SessionStateBuilder}; + mod session_state_defaults; pub use session_state_defaults::SessionStateDefaults; From 83d3c5a945b8331eceb8383f7d33495b3476919d Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Sun, 25 Aug 2024 06:57:06 -0400 Subject: [PATCH 017/154] Add query with STDDEV and VAR to Clickbench extended (#12146) --- benchmarks/queries/clickbench/README.md | 15 +++++++++++++++ benchmarks/queries/clickbench/extended.sql | 3 ++- 2 files changed, 17 insertions(+), 1 deletion(-) diff --git a/benchmarks/queries/clickbench/README.md b/benchmarks/queries/clickbench/README.md index 560b54181d5f..c6bd8fe9d6f5 100644 --- a/benchmarks/queries/clickbench/README.md +++ b/benchmarks/queries/clickbench/README.md @@ -58,6 +58,21 @@ LIMIT 10; ``` +### Q3: What is the income distribution for users in specific regions + +**Question**: "What regions and social networks have the highest variance of parameter price + +**Important Query Properties**: STDDEV and VAR aggregation functions, GROUP BY multiple small ints + +```sql +SELECT "SocialSourceNetworkID", "RegionID", COUNT(*), AVG("Age"), AVG("ParamPrice"), STDDEV("ParamPrice") as s, VAR("ParamPrice") +FROM 'hits.parquet' +GROUP BY "SocialSourceNetworkID", "RegionID" +HAVING s IS NOT NULL +ORDER BY s DESC +LIMIT 10; +``` + ## Data Notes Here are some interesting statistics about the data used in the queries diff --git a/benchmarks/queries/clickbench/extended.sql b/benchmarks/queries/clickbench/extended.sql index 0a2999fceb49..2f814ad8450a 100644 --- a/benchmarks/queries/clickbench/extended.sql +++ b/benchmarks/queries/clickbench/extended.sql @@ -1,3 +1,4 @@ SELECT COUNT(DISTINCT "SearchPhrase"), COUNT(DISTINCT "MobilePhone"), COUNT(DISTINCT "MobilePhoneModel") FROM hits; SELECT COUNT(DISTINCT "HitColor"), COUNT(DISTINCT "BrowserCountry"), COUNT(DISTINCT "BrowserLanguage") FROM hits; -SELECT "BrowserCountry", COUNT(DISTINCT "SocialNetwork"), COUNT(DISTINCT "HitColor"), COUNT(DISTINCT "BrowserLanguage"), COUNT(DISTINCT "SocialAction") FROM hits GROUP BY 1 ORDER BY 2 DESC LIMIT 10; \ No newline at end of file +SELECT "BrowserCountry", COUNT(DISTINCT "SocialNetwork"), COUNT(DISTINCT "HitColor"), COUNT(DISTINCT "BrowserLanguage"), COUNT(DISTINCT "SocialAction") FROM hits GROUP BY 1 ORDER BY 2 DESC LIMIT 10; +SELECT "SocialSourceNetworkID", "RegionID", COUNT(*), AVG("Age"), AVG("ParamPrice"), STDDEV("ParamPrice") as s, VAR("ParamPrice") FROM hits GROUP BY "SocialSourceNetworkID", "RegionID" HAVING s IS NOT NULL ORDER BY s DESC LIMIT 10; From 945902dd5d440bdc360cab60ef31cd0c3bceec41 Mon Sep 17 00:00:00 2001 From: Jonah Gao Date: Sun, 25 Aug 2024 19:23:41 +0800 Subject: [PATCH 018/154] fix: preserve expression names when replacing placeholders (#12126) * fix: preserve expression names when replacing placeholders * Add comment --- datafusion/core/src/dataframe/mod.rs | 30 ++++++++++++- datafusion/expr/src/expr_rewriter/mod.rs | 51 ++++++++++++++++++++++ datafusion/expr/src/logical_plan/plan.rs | 23 ++++++---- datafusion/optimizer/src/utils.rs | 55 ++---------------------- datafusion/sql/tests/sql_integration.rs | 15 ++++--- 5 files changed, 107 insertions(+), 67 deletions(-) diff --git a/datafusion/core/src/dataframe/mod.rs b/datafusion/core/src/dataframe/mod.rs index 42203e5fe84e..a38e7f45a6f1 100644 --- a/datafusion/core/src/dataframe/mod.rs +++ b/datafusion/core/src/dataframe/mod.rs @@ -1707,7 +1707,7 @@ mod tests { use crate::test_util::{register_aggregate_csv, test_table, test_table_with_name}; use arrow::array::{self, Int32Array}; - use datafusion_common::{Constraint, Constraints, ScalarValue}; + use datafusion_common::{assert_batches_eq, Constraint, Constraints, ScalarValue}; use datafusion_common_runtime::SpawnedTask; use datafusion_expr::expr::WindowFunction; use datafusion_expr::{ @@ -3699,4 +3699,32 @@ mod tests { assert!(result.is_err()); Ok(()) } + + // Test issue: https://github.com/apache/datafusion/issues/12065 + #[tokio::test] + async fn filtered_aggr_with_param_values() -> Result<()> { + let cfg = SessionConfig::new() + .set("datafusion.sql_parser.dialect", "PostgreSQL".into()); + let ctx = SessionContext::new_with_config(cfg); + register_aggregate_csv(&ctx, "table1").await?; + + let df = ctx + .sql("select count (c2) filter (where c3 > $1) from table1") + .await? + .with_param_values(ParamValues::List(vec![ScalarValue::from(10u64)])); + + let df_results = df?.collect().await?; + assert_batches_eq!( + &[ + "+------------------------------------------------+", + "| count(table1.c2) FILTER (WHERE table1.c3 > $1) |", + "+------------------------------------------------+", + "| 54 |", + "+------------------------------------------------+", + ], + &df_results + ); + + Ok(()) + } } diff --git a/datafusion/expr/src/expr_rewriter/mod.rs b/datafusion/expr/src/expr_rewriter/mod.rs index c26970cb053a..768c4aabc840 100644 --- a/datafusion/expr/src/expr_rewriter/mod.rs +++ b/datafusion/expr/src/expr_rewriter/mod.rs @@ -279,6 +279,57 @@ where expr.alias_if_changed(original_name) } +/// Handles ensuring the name of rewritten expressions is not changed. +/// +/// For example, if an expression `1 + 2` is rewritten to `3`, the name of the +/// expression should be preserved: `3 as "1 + 2"` +/// +/// See for details +pub struct NamePreserver { + use_alias: bool, +} + +/// If the name of an expression is remembered, it will be preserved when +/// rewriting the expression +pub struct SavedName(Option); + +impl NamePreserver { + /// Create a new NamePreserver for rewriting the `expr` that is part of the specified plan + pub fn new(plan: &LogicalPlan) -> Self { + Self { + use_alias: !matches!(plan, LogicalPlan::Filter(_) | LogicalPlan::Join(_)), + } + } + + /// Create a new NamePreserver for rewriting the `expr`s in `Projection` + /// + /// This will use aliases + pub fn new_for_projection() -> Self { + Self { use_alias: true } + } + + pub fn save(&self, expr: &Expr) -> Result { + let original_name = if self.use_alias { + Some(expr.name_for_alias()?) + } else { + None + }; + + Ok(SavedName(original_name)) + } +} + +impl SavedName { + /// Ensures the name of the rewritten expression is preserved + pub fn restore(self, expr: Expr) -> Result { + let Self(original_name) = self; + match original_name { + Some(name) => expr.alias_if_changed(name), + None => Ok(expr), + } + } +} + #[cfg(test)] mod test { use std::ops::Add; diff --git a/datafusion/expr/src/logical_plan/plan.rs b/datafusion/expr/src/logical_plan/plan.rs index ca7d04b9b03e..3ede7f25b753 100644 --- a/datafusion/expr/src/logical_plan/plan.rs +++ b/datafusion/expr/src/logical_plan/plan.rs @@ -26,7 +26,7 @@ use super::dml::CopyTo; use super::DdlStatement; use crate::builder::{change_redundant_column, unnest_with_options}; use crate::expr::{Placeholder, Sort as SortExpr, WindowFunction}; -use crate::expr_rewriter::{create_col_from_scalar_expr, normalize_cols}; +use crate::expr_rewriter::{create_col_from_scalar_expr, normalize_cols, NamePreserver}; use crate::logical_plan::display::{GraphvizVisitor, IndentVisitor}; use crate::logical_plan::extension::UserDefinedLogicalNode; use crate::logical_plan::{DmlStatement, Statement}; @@ -1339,15 +1339,20 @@ impl LogicalPlan { ) -> Result { self.transform_up_with_subqueries(|plan| { let schema = Arc::clone(plan.schema()); + let name_preserver = NamePreserver::new(&plan); plan.map_expressions(|e| { - e.infer_placeholder_types(&schema)?.transform_up(|e| { - if let Expr::Placeholder(Placeholder { id, .. }) = e { - let value = param_values.get_placeholders_with_values(&id)?; - Ok(Transformed::yes(Expr::Literal(value))) - } else { - Ok(Transformed::no(e)) - } - }) + let original_name = name_preserver.save(&e)?; + let transformed_expr = + e.infer_placeholder_types(&schema)?.transform_up(|e| { + if let Expr::Placeholder(Placeholder { id, .. }) = e { + let value = param_values.get_placeholders_with_values(&id)?; + Ok(Transformed::yes(Expr::Literal(value))) + } else { + Ok(Transformed::no(e)) + } + })?; + // Preserve name to avoid breaking column references to this expression + transformed_expr.map_data(|expr| original_name.restore(expr)) }) }) .map(|res| res.data) diff --git a/datafusion/optimizer/src/utils.rs b/datafusion/optimizer/src/utils.rs index 05b1744d90c5..45cef55bf272 100644 --- a/datafusion/optimizer/src/utils.rs +++ b/datafusion/optimizer/src/utils.rs @@ -28,6 +28,10 @@ use datafusion_expr::{logical_plan::LogicalPlan, Expr, Operator}; use log::{debug, trace}; +/// Re-export of `NamesPreserver` for backwards compatibility, +/// as it was initially placed here and then moved elsewhere. +pub use datafusion_expr::expr_rewriter::NamePreserver; + /// Convenience rule for writing optimizers: recursively invoke /// optimize on plan's children and then return a node of the same /// type. Useful for optimizer rules which want to leave the type @@ -294,54 +298,3 @@ pub fn only_or_err(slice: &[T]) -> Result<&T> { pub fn merge_schema(inputs: Vec<&LogicalPlan>) -> DFSchema { expr_utils::merge_schema(inputs) } - -/// Handles ensuring the name of rewritten expressions is not changed. -/// -/// For example, if an expression `1 + 2` is rewritten to `3`, the name of the -/// expression should be preserved: `3 as "1 + 2"` -/// -/// See for details -pub struct NamePreserver { - use_alias: bool, -} - -/// If the name of an expression is remembered, it will be preserved when -/// rewriting the expression -pub struct SavedName(Option); - -impl NamePreserver { - /// Create a new NamePreserver for rewriting the `expr` that is part of the specified plan - pub fn new(plan: &LogicalPlan) -> Self { - Self { - use_alias: !matches!(plan, LogicalPlan::Filter(_) | LogicalPlan::Join(_)), - } - } - - /// Create a new NamePreserver for rewriting the `expr`s in `Projection` - /// - /// This will use aliases - pub fn new_for_projection() -> Self { - Self { use_alias: true } - } - - pub fn save(&self, expr: &Expr) -> Result { - let original_name = if self.use_alias { - Some(expr.name_for_alias()?) - } else { - None - }; - - Ok(SavedName(original_name)) - } -} - -impl SavedName { - /// Ensures the name of the rewritten expression is preserved - pub fn restore(self, expr: Expr) -> Result { - let Self(original_name) = self; - match original_name { - Some(name) => expr.alias_if_changed(name), - None => Ok(expr), - } - } -} diff --git a/datafusion/sql/tests/sql_integration.rs b/datafusion/sql/tests/sql_integration.rs index 5685e09c9c9f..5a203703e967 100644 --- a/datafusion/sql/tests/sql_integration.rs +++ b/datafusion/sql/tests/sql_integration.rs @@ -3813,7 +3813,7 @@ fn test_prepare_statement_to_plan_params_as_constants() { /////////////////// // replace params with values let param_values = vec![ScalarValue::Int32(Some(10))]; - let expected_plan = "Projection: Int32(10)\n EmptyRelation"; + let expected_plan = "Projection: Int32(10) AS $1\n EmptyRelation"; prepare_stmt_replace_params_quick_test(plan, param_values, expected_plan); @@ -3829,7 +3829,8 @@ fn test_prepare_statement_to_plan_params_as_constants() { /////////////////// // replace params with values let param_values = vec![ScalarValue::Int32(Some(10))]; - let expected_plan = "Projection: Int64(1) + Int32(10)\n EmptyRelation"; + let expected_plan = + "Projection: Int64(1) + Int32(10) AS Int64(1) + $1\n EmptyRelation"; prepare_stmt_replace_params_quick_test(plan, param_values, expected_plan); @@ -3848,7 +3849,9 @@ fn test_prepare_statement_to_plan_params_as_constants() { ScalarValue::Int32(Some(10)), ScalarValue::Float64(Some(10.0)), ]; - let expected_plan = "Projection: Int64(1) + Int32(10) + Float64(10)\n EmptyRelation"; + let expected_plan = + "Projection: Int64(1) + Int32(10) + Float64(10) AS Int64(1) + $1 + $2\ + \n EmptyRelation"; prepare_stmt_replace_params_quick_test(plan, param_values, expected_plan); } @@ -4063,7 +4066,7 @@ fn test_prepare_statement_insert_infer() { \n Projection: column1 AS id, column2 AS first_name, column3 AS last_name, \ CAST(NULL AS Int32) AS age, CAST(NULL AS Utf8) AS state, CAST(NULL AS Float64) AS salary, \ CAST(NULL AS Timestamp(Nanosecond, None)) AS birth_date, CAST(NULL AS Int32) AS 😀\ - \n Values: (UInt32(1), Utf8(\"Alan\"), Utf8(\"Turing\"))"; + \n Values: (UInt32(1) AS $1, Utf8(\"Alan\") AS $2, Utf8(\"Turing\") AS $3)"; let plan = plan.replace_params_with_values(¶m_values).unwrap(); prepare_stmt_replace_params_quick_test(plan, param_values, expected_plan); @@ -4144,7 +4147,7 @@ fn test_prepare_statement_to_plan_multi_params() { ScalarValue::from("xyz"), ]; let expected_plan = - "Projection: person.id, person.age, Utf8(\"xyz\")\ + "Projection: person.id, person.age, Utf8(\"xyz\") AS $6\ \n Filter: person.age IN ([Int32(10), Int32(20)]) AND person.salary > Float64(100) AND person.salary < Float64(200) OR person.first_name < Utf8(\"abc\")\ \n TableScan: person"; @@ -4213,7 +4216,7 @@ fn test_prepare_statement_to_plan_value_list() { let expected_plan = "Projection: *\ \n SubqueryAlias: t\ \n Projection: column1 AS num, column2 AS letter\ - \n Values: (Int64(1), Utf8(\"a\")), (Int64(2), Utf8(\"b\"))"; + \n Values: (Int64(1), Utf8(\"a\") AS $1), (Int64(2), Utf8(\"b\") AS $2)"; prepare_stmt_replace_params_quick_test(plan, param_values, expected_plan); } From 0c75ddde9255a149c20847a59a2b39b7a80b5051 Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Mon, 26 Aug 2024 16:18:57 +0200 Subject: [PATCH 019/154] Use Result.unwrap_or_else where applicable (#12166) --- datafusion/proto/tests/cases/roundtrip_logical_plan.rs | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/datafusion/proto/tests/cases/roundtrip_logical_plan.rs b/datafusion/proto/tests/cases/roundtrip_logical_plan.rs index 09c5f0f8bd3d..4f58185798f7 100644 --- a/datafusion/proto/tests/cases/roundtrip_logical_plan.rs +++ b/datafusion/proto/tests/cases/roundtrip_logical_plan.rs @@ -104,10 +104,8 @@ fn roundtrip_json_test(_proto: &protobuf::LogicalExprNode) {} fn roundtrip_expr_test(initial_struct: Expr, ctx: SessionContext) { let extension_codec = DefaultLogicalExtensionCodec {}; let proto: protobuf::LogicalExprNode = - match serialize_expr(&initial_struct, &extension_codec) { - Ok(p) => p, - Err(e) => panic!("Error serializing expression: {:?}", e), - }; + serialize_expr(&initial_struct, &extension_codec) + .unwrap_or_else(|e| panic!("Error serializing expression: {:?}", e)); let round_trip: Expr = from_proto::parse_expr(&proto, &ctx, &extension_codec).unwrap(); From 533ddcb6f43dc1b11368bdc33924afd87f439e19 Mon Sep 17 00:00:00 2001 From: wiedld Date: Mon, 26 Aug 2024 09:01:02 -0700 Subject: [PATCH 020/154] Provide documentation of expose APIs to enable handling of type coercion at UNION plan construction. (#12142) * chore(12105): enable union type-coercion by two approaches, using newly pub interfaces * chore(12105): update documentation to delineate btwn the interfaces involved in type coercion * chore((12105): update union() logical plan construction docs, to address type coercion --- datafusion/expr/src/logical_plan/builder.rs | 12 +++++++++- .../optimizer/src/analyzer/type_coercion.rs | 22 ++++++++++++++----- 2 files changed, 28 insertions(+), 6 deletions(-) diff --git a/datafusion/expr/src/logical_plan/builder.rs b/datafusion/expr/src/logical_plan/builder.rs index 559908bcfdfa..fdd07da023e0 100644 --- a/datafusion/expr/src/logical_plan/builder.rs +++ b/datafusion/expr/src/logical_plan/builder.rs @@ -1331,7 +1331,17 @@ pub fn validate_unique_names<'a>( }) } -/// Union two logical plans. +/// Union two [`LogicalPlan`]s. +/// +/// Constructs the UNION plan, but does not perform type-coercion. Therefore the +/// subtree expressions will not be properly typed until the optimizer pass. +/// +/// If a properly typed UNION plan is needed, refer to [`TypeCoercionRewriter::coerce_union`] +/// or alternatively, merge the union input schema using [`coerce_union_schema`] and +/// apply the expression rewrite with [`coerce_plan_expr_for_schema`]. +/// +/// [`TypeCoercionRewriter::coerce_union`]: https://docs.rs/datafusion-optimizer/latest/datafusion_optimizer/analyzer/type_coercion/struct.TypeCoercionRewriter.html#method.coerce_union +/// [`coerce_union_schema`]: https://docs.rs/datafusion-optimizer/latest/datafusion_optimizer/analyzer/type_coercion/fn.coerce_union_schema.html pub fn union(left_plan: LogicalPlan, right_plan: LogicalPlan) -> Result { // Temporarily use the schema from the left input and later rely on the analyzer to // coerce the two schemas into a common one. diff --git a/datafusion/optimizer/src/analyzer/type_coercion.rs b/datafusion/optimizer/src/analyzer/type_coercion.rs index 68ab2e13005f..315284c50839 100644 --- a/datafusion/optimizer/src/analyzer/type_coercion.rs +++ b/datafusion/optimizer/src/analyzer/type_coercion.rs @@ -56,6 +56,8 @@ use datafusion_expr::{ Projection, ScalarUDF, Union, WindowFrame, WindowFrameBound, WindowFrameUnits, }; +/// Performs type coercion by determining the schema +/// and performing the expression rewrites. #[derive(Default)] pub struct TypeCoercion {} @@ -128,16 +130,23 @@ fn analyze_internal( .map_data(|plan| plan.recompute_schema()) } -pub(crate) struct TypeCoercionRewriter<'a> { +/// Rewrite expressions to apply type coercion. +pub struct TypeCoercionRewriter<'a> { pub(crate) schema: &'a DFSchema, } impl<'a> TypeCoercionRewriter<'a> { + /// Create a new [`TypeCoercionRewriter`] with a provided schema + /// representing both the inputs and output of the [`LogicalPlan`] node. fn new(schema: &'a DFSchema) -> Self { Self { schema } } - fn coerce_plan(&mut self, plan: LogicalPlan) -> Result { + /// Coerce the [`LogicalPlan`]. + /// + /// Refer to [`TypeCoercionRewriter::coerce_join`] and [`TypeCoercionRewriter::coerce_union`] + /// for type-coercion approach. + pub fn coerce_plan(&mut self, plan: LogicalPlan) -> Result { match plan { LogicalPlan::Join(join) => self.coerce_join(join), LogicalPlan::Union(union) => Self::coerce_union(union), @@ -153,7 +162,7 @@ impl<'a> TypeCoercionRewriter<'a> { /// /// For example, on_exprs like `t1.a = t2.b AND t1.x = t2.y` will be stored /// as a list of `(t1.a, t2.b), (t1.x, t2.y)` - fn coerce_join(&mut self, mut join: Join) -> Result { + pub fn coerce_join(&mut self, mut join: Join) -> Result { join.on = join .on .into_iter() @@ -176,7 +185,7 @@ impl<'a> TypeCoercionRewriter<'a> { /// Coerce the union’s inputs to a common schema compatible with all inputs. /// This occurs after wildcard expansion and the coercion of the input expressions. - fn coerce_union(union_plan: Union) -> Result { + pub fn coerce_union(union_plan: Union) -> Result { let union_schema = Arc::new(coerce_union_schema(&union_plan.inputs)?); let new_inputs = union_plan .inputs @@ -809,7 +818,10 @@ fn coerce_case_expression(case: Case, schema: &DFSchema) -> Result { } /// Get a common schema that is compatible with all inputs of UNION. -fn coerce_union_schema(inputs: &[Arc]) -> Result { +/// +/// This method presumes that the wildcard expansion is unneeded, or has already +/// been applied. +pub fn coerce_union_schema(inputs: &[Arc]) -> Result { let base_schema = inputs[0].schema(); let mut union_datatypes = base_schema .fields() From da3f6afec51e4272622cb74ec62a11d13df17267 Mon Sep 17 00:00:00 2001 From: Albert Skalt <133099191+askalt@users.noreply.github.com> Date: Mon, 26 Aug 2024 22:10:16 +0300 Subject: [PATCH 021/154] Add ability to return LogicalPlan by value from TableProvider (#12113) This patch changes the `get_logical_plan(...)` method signature. Now it returns a `Cow` to allow an implementation to return plan by value. --- datafusion/catalog/src/table.rs | 5 ++- datafusion/core/src/dataframe/mod.rs | 5 ++- .../core/src/datasource/cte_worktable.rs | 4 +- .../src/datasource/default_table_source.rs | 4 +- datafusion/core/src/datasource/view.rs | 6 +-- datafusion/expr/src/table_source.rs | 4 +- .../src/analyzer/inline_table_scan.rs | 43 +++++++++---------- 7 files changed, 36 insertions(+), 35 deletions(-) diff --git a/datafusion/catalog/src/table.rs b/datafusion/catalog/src/table.rs index 792315642a00..69fa81faf8e2 100644 --- a/datafusion/catalog/src/table.rs +++ b/datafusion/catalog/src/table.rs @@ -16,6 +16,7 @@ // under the License. use std::any::Any; +use std::borrow::Cow; use std::sync::Arc; use crate::session::Session; @@ -56,8 +57,8 @@ pub trait TableProvider: Sync + Send { None } - /// Get the [`LogicalPlan`] of this table, if available - fn get_logical_plan(&self) -> Option<&LogicalPlan> { + /// Get the [`LogicalPlan`] of this table, if available. + fn get_logical_plan(&self) -> Option> { None } diff --git a/datafusion/core/src/dataframe/mod.rs b/datafusion/core/src/dataframe/mod.rs index a38e7f45a6f1..c516c7985d54 100644 --- a/datafusion/core/src/dataframe/mod.rs +++ b/datafusion/core/src/dataframe/mod.rs @@ -21,6 +21,7 @@ mod parquet; use std::any::Any; +use std::borrow::Cow; use std::collections::HashMap; use std::sync::Arc; @@ -1648,8 +1649,8 @@ impl TableProvider for DataFrameTableProvider { self } - fn get_logical_plan(&self) -> Option<&LogicalPlan> { - Some(&self.plan) + fn get_logical_plan(&self) -> Option> { + Some(Cow::Borrowed(&self.plan)) } fn supports_filters_pushdown( diff --git a/datafusion/core/src/datasource/cte_worktable.rs b/datafusion/core/src/datasource/cte_worktable.rs index d7d224828dda..d2da15c64f52 100644 --- a/datafusion/core/src/datasource/cte_worktable.rs +++ b/datafusion/core/src/datasource/cte_worktable.rs @@ -17,8 +17,8 @@ //! CteWorkTable implementation used for recursive queries -use std::any::Any; use std::sync::Arc; +use std::{any::Any, borrow::Cow}; use arrow::datatypes::SchemaRef; use async_trait::async_trait; @@ -63,7 +63,7 @@ impl TableProvider for CteWorkTable { self } - fn get_logical_plan(&self) -> Option<&LogicalPlan> { + fn get_logical_plan(&self) -> Option> { None } diff --git a/datafusion/core/src/datasource/default_table_source.rs b/datafusion/core/src/datasource/default_table_source.rs index 977e681d6641..b4a5a76fc9ff 100644 --- a/datafusion/core/src/datasource/default_table_source.rs +++ b/datafusion/core/src/datasource/default_table_source.rs @@ -17,8 +17,8 @@ //! Default TableSource implementation used in DataFusion physical plans -use std::any::Any; use std::sync::Arc; +use std::{any::Any, borrow::Cow}; use crate::datasource::TableProvider; @@ -70,7 +70,7 @@ impl TableSource for DefaultTableSource { self.table_provider.supports_filters_pushdown(filter) } - fn get_logical_plan(&self) -> Option<&datafusion_expr::LogicalPlan> { + fn get_logical_plan(&self) -> Option> { self.table_provider.get_logical_plan() } diff --git a/datafusion/core/src/datasource/view.rs b/datafusion/core/src/datasource/view.rs index a81942bf769e..947714c1e4f9 100644 --- a/datafusion/core/src/datasource/view.rs +++ b/datafusion/core/src/datasource/view.rs @@ -17,7 +17,7 @@ //! View data source which uses a LogicalPlan as it's input. -use std::{any::Any, sync::Arc}; +use std::{any::Any, borrow::Cow, sync::Arc}; use crate::{ error::Result, @@ -90,8 +90,8 @@ impl TableProvider for ViewTable { self } - fn get_logical_plan(&self) -> Option<&LogicalPlan> { - Some(&self.logical_plan) + fn get_logical_plan(&self) -> Option> { + Some(Cow::Borrowed(&self.logical_plan)) } fn schema(&self) -> SchemaRef { diff --git a/datafusion/expr/src/table_source.rs b/datafusion/expr/src/table_source.rs index 2de3cc923315..8b8d2dfcf2df 100644 --- a/datafusion/expr/src/table_source.rs +++ b/datafusion/expr/src/table_source.rs @@ -22,7 +22,7 @@ use crate::{Expr, LogicalPlan}; use arrow::datatypes::SchemaRef; use datafusion_common::{Constraints, Result}; -use std::any::Any; +use std::{any::Any, borrow::Cow}; /// Indicates how a filter expression is handled by /// [`TableProvider::scan`]. @@ -122,7 +122,7 @@ pub trait TableSource: Sync + Send { } /// Get the Logical plan of this table provider, if available. - fn get_logical_plan(&self) -> Option<&LogicalPlan> { + fn get_logical_plan(&self) -> Option> { None } diff --git a/datafusion/optimizer/src/analyzer/inline_table_scan.rs b/datafusion/optimizer/src/analyzer/inline_table_scan.rs index b69b8410da49..d5b3648725b9 100644 --- a/datafusion/optimizer/src/analyzer/inline_table_scan.rs +++ b/datafusion/optimizer/src/analyzer/inline_table_scan.rs @@ -24,7 +24,7 @@ use datafusion_common::config::ConfigOptions; use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; use datafusion_common::{Column, Result}; use datafusion_expr::expr::WildcardOptions; -use datafusion_expr::{logical_plan::LogicalPlan, Expr, LogicalPlanBuilder, TableScan}; +use datafusion_expr::{logical_plan::LogicalPlan, Expr, LogicalPlanBuilder}; /// Analyzed rule that inlines TableScan that provide a [`LogicalPlan`] /// (DataFrame / ViewTable) @@ -56,24 +56,23 @@ fn analyze_internal(plan: LogicalPlan) -> Result> { match plan { // Match only on scans without filter / projection / fetch // Views and DataFrames won't have those added - // during the early stage of planning - LogicalPlan::TableScan(TableScan { - table_name, - source, - projection, - filters, - .. - }) if filters.is_empty() && source.get_logical_plan().is_some() => { - let sub_plan = source.get_logical_plan().unwrap(); - let projection_exprs = generate_projection_expr(&projection, sub_plan)?; - LogicalPlanBuilder::from(sub_plan.clone()) - .project(projection_exprs)? - // Ensures that the reference to the inlined table remains the - // same, meaning we don't have to change any of the parent nodes - // that reference this table. - .alias(table_name)? - .build() - .map(Transformed::yes) + // during the early stage of planning. + LogicalPlan::TableScan(table_scan) if table_scan.filters.is_empty() => { + if let Some(sub_plan) = table_scan.source.get_logical_plan() { + let sub_plan = sub_plan.into_owned(); + let projection_exprs = + generate_projection_expr(&table_scan.projection, &sub_plan)?; + LogicalPlanBuilder::from(sub_plan) + .project(projection_exprs)? + // Ensures that the reference to the inlined table remains the + // same, meaning we don't have to change any of the parent nodes + // that reference this table. + .alias(table_scan.table_name)? + .build() + .map(Transformed::yes) + } else { + Ok(Transformed::no(LogicalPlan::TableScan(table_scan))) + } } _ => Ok(Transformed::no(plan)), } @@ -104,7 +103,7 @@ fn generate_projection_expr( #[cfg(test)] mod tests { - use std::{sync::Arc, vec}; + use std::{borrow::Cow, sync::Arc, vec}; use crate::analyzer::inline_table_scan::InlineTableScan; use crate::test::assert_analyzed_plan_eq; @@ -167,8 +166,8 @@ mod tests { Arc::new(Schema::new(vec![Field::new("a", DataType::Int64, false)])) } - fn get_logical_plan(&self) -> Option<&LogicalPlan> { - Some(&self.plan) + fn get_logical_plan(&self) -> Option> { + Some(Cow::Borrowed(&self.plan)) } } From 1b875f456a1d916d2c6826531dc1b3f46c42706f Mon Sep 17 00:00:00 2001 From: Emil Ejbyfeldt Date: Mon, 26 Aug 2024 21:11:41 +0200 Subject: [PATCH 022/154] Implement groups accumulator for stddev and variance (#12095) * Add more stddev/var tests cases Add tests cases for stddev_samp/pop and var_smap/pop the includes a group_by clause. * Implement GroupsAccumulator for stddev and variance * Add cast to support all numeric types * Improve documenataion and comments --- .../groups_accumulator/accumulate.rs | 268 +++++++++-------- datafusion/functions-aggregate/src/stddev.rs | 83 +++++- .../functions-aggregate/src/variance.rs | 275 ++++++++++++++++-- .../sqllogictest/test_files/aggregate.slt | 79 +++++ 4 files changed, 555 insertions(+), 150 deletions(-) diff --git a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/accumulate.rs b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/accumulate.rs index 455fc5fec450..a0475fe8e446 100644 --- a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/accumulate.rs +++ b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/accumulate.rs @@ -91,36 +91,9 @@ impl NullState { /// * `opt_filter`: if present, only rows for which is Some(true) are included /// * `value_fn`: function invoked for (group_index, value) where value is non null /// - /// # Example + /// See [`accumulate`], for more details on how value_fn is called /// - /// ```text - /// ┌─────────┐ ┌─────────┐ ┌ ─ ─ ─ ─ ┐ - /// │ ┌─────┐ │ │ ┌─────┐ │ ┌─────┐ - /// │ │ 2 │ │ │ │ 200 │ │ │ │ t │ │ - /// │ ├─────┤ │ │ ├─────┤ │ ├─────┤ - /// │ │ 2 │ │ │ │ 100 │ │ │ │ f │ │ - /// │ ├─────┤ │ │ ├─────┤ │ ├─────┤ - /// │ │ 0 │ │ │ │ 200 │ │ │ │ t │ │ - /// │ ├─────┤ │ │ ├─────┤ │ ├─────┤ - /// │ │ 1 │ │ │ │ 200 │ │ │ │NULL │ │ - /// │ ├─────┤ │ │ ├─────┤ │ ├─────┤ - /// │ │ 0 │ │ │ │ 300 │ │ │ │ t │ │ - /// │ └─────┘ │ │ └─────┘ │ └─────┘ - /// └─────────┘ └─────────┘ └ ─ ─ ─ ─ ┘ - /// - /// group_indices values opt_filter - /// ``` - /// - /// In the example above, `value_fn` is invoked for each (group_index, - /// value) pair where `opt_filter[i]` is true and values is non null - /// - /// ```text - /// value_fn(2, 200) - /// value_fn(0, 200) - /// value_fn(0, 300) - /// ``` - /// - /// It also sets + /// When value_fn is called it also sets /// /// 1. `self.seen_values[group_index]` to true for all rows that had a non null vale pub fn accumulate( @@ -134,105 +107,14 @@ impl NullState { T: ArrowPrimitiveType + Send, F: FnMut(usize, T::Native) + Send, { - let data: &[T::Native] = values.values(); - assert_eq!(data.len(), group_indices.len()); - // ensure the seen_values is big enough (start everything at // "not seen" valid) let seen_values = initialize_builder(&mut self.seen_values, total_num_groups, false); - - match (values.null_count() > 0, opt_filter) { - // no nulls, no filter, - (false, None) => { - let iter = group_indices.iter().zip(data.iter()); - for (&group_index, &new_value) in iter { - seen_values.set_bit(group_index, true); - value_fn(group_index, new_value); - } - } - // nulls, no filter - (true, None) => { - let nulls = values.nulls().unwrap(); - // This is based on (ahem, COPY/PASTE) arrow::compute::aggregate::sum - // iterate over in chunks of 64 bits for more efficient null checking - let group_indices_chunks = group_indices.chunks_exact(64); - let data_chunks = data.chunks_exact(64); - let bit_chunks = nulls.inner().bit_chunks(); - - let group_indices_remainder = group_indices_chunks.remainder(); - let data_remainder = data_chunks.remainder(); - - group_indices_chunks - .zip(data_chunks) - .zip(bit_chunks.iter()) - .for_each(|((group_index_chunk, data_chunk), mask)| { - // index_mask has value 1 << i in the loop - let mut index_mask = 1; - group_index_chunk.iter().zip(data_chunk.iter()).for_each( - |(&group_index, &new_value)| { - // valid bit was set, real value - let is_valid = (mask & index_mask) != 0; - if is_valid { - seen_values.set_bit(group_index, true); - value_fn(group_index, new_value); - } - index_mask <<= 1; - }, - ) - }); - - // handle any remaining bits (after the initial 64) - let remainder_bits = bit_chunks.remainder_bits(); - group_indices_remainder - .iter() - .zip(data_remainder.iter()) - .enumerate() - .for_each(|(i, (&group_index, &new_value))| { - let is_valid = remainder_bits & (1 << i) != 0; - if is_valid { - seen_values.set_bit(group_index, true); - value_fn(group_index, new_value); - } - }); - } - // no nulls, but a filter - (false, Some(filter)) => { - assert_eq!(filter.len(), group_indices.len()); - // The performance with a filter could be improved by - // iterating over the filter in chunks, rather than a single - // iterator. TODO file a ticket - group_indices - .iter() - .zip(data.iter()) - .zip(filter.iter()) - .for_each(|((&group_index, &new_value), filter_value)| { - if let Some(true) = filter_value { - seen_values.set_bit(group_index, true); - value_fn(group_index, new_value); - } - }) - } - // both null values and filters - (true, Some(filter)) => { - assert_eq!(filter.len(), group_indices.len()); - // The performance with a filter could be improved by - // iterating over the filter in chunks, rather than using - // iterators. TODO file a ticket - filter - .iter() - .zip(group_indices.iter()) - .zip(values.iter()) - .for_each(|((filter_value, &group_index), new_value)| { - if let Some(true) = filter_value { - if let Some(new_value) = new_value { - seen_values.set_bit(group_index, true); - value_fn(group_index, new_value) - } - } - }) - } - } + accumulate(group_indices, values, opt_filter, |group_index, value| { + seen_values.set_bit(group_index, true); + value_fn(group_index, value); + }); } /// Invokes `value_fn(group_index, value)` for each non null, non @@ -351,6 +233,144 @@ impl NullState { } } +/// Invokes `value_fn(group_index, value)` for each non null, non +/// filtered value of `value`, +/// +/// # Arguments: +/// +/// * `group_indices`: To which groups do the rows in `values` belong, (aka group_index) +/// * `values`: the input arguments to the accumulator +/// * `opt_filter`: if present, only rows for which is Some(true) are included +/// * `value_fn`: function invoked for (group_index, value) where value is non null +/// +/// # Example +/// +/// ```text +/// ┌─────────┐ ┌─────────┐ ┌ ─ ─ ─ ─ ┐ +/// │ ┌─────┐ │ │ ┌─────┐ │ ┌─────┐ +/// │ │ 2 │ │ │ │ 200 │ │ │ │ t │ │ +/// │ ├─────┤ │ │ ├─────┤ │ ├─────┤ +/// │ │ 2 │ │ │ │ 100 │ │ │ │ f │ │ +/// │ ├─────┤ │ │ ├─────┤ │ ├─────┤ +/// │ │ 0 │ │ │ │ 200 │ │ │ │ t │ │ +/// │ ├─────┤ │ │ ├─────┤ │ ├─────┤ +/// │ │ 1 │ │ │ │ 200 │ │ │ │NULL │ │ +/// │ ├─────┤ │ │ ├─────┤ │ ├─────┤ +/// │ │ 0 │ │ │ │ 300 │ │ │ │ t │ │ +/// │ └─────┘ │ │ └─────┘ │ └─────┘ +/// └─────────┘ └─────────┘ └ ─ ─ ─ ─ ┘ +/// +/// group_indices values opt_filter +/// ``` +/// +/// In the example above, `value_fn` is invoked for each (group_index, +/// value) pair where `opt_filter[i]` is true and values is non null +/// +/// ```text +/// value_fn(2, 200) +/// value_fn(0, 200) +/// value_fn(0, 300) +/// ``` +pub fn accumulate( + group_indices: &[usize], + values: &PrimitiveArray, + opt_filter: Option<&BooleanArray>, + mut value_fn: F, +) where + T: ArrowPrimitiveType + Send, + F: FnMut(usize, T::Native) + Send, +{ + let data: &[T::Native] = values.values(); + assert_eq!(data.len(), group_indices.len()); + + match (values.null_count() > 0, opt_filter) { + // no nulls, no filter, + (false, None) => { + let iter = group_indices.iter().zip(data.iter()); + for (&group_index, &new_value) in iter { + value_fn(group_index, new_value); + } + } + // nulls, no filter + (true, None) => { + let nulls = values.nulls().unwrap(); + // This is based on (ahem, COPY/PASTE) arrow::compute::aggregate::sum + // iterate over in chunks of 64 bits for more efficient null checking + let group_indices_chunks = group_indices.chunks_exact(64); + let data_chunks = data.chunks_exact(64); + let bit_chunks = nulls.inner().bit_chunks(); + + let group_indices_remainder = group_indices_chunks.remainder(); + let data_remainder = data_chunks.remainder(); + + group_indices_chunks + .zip(data_chunks) + .zip(bit_chunks.iter()) + .for_each(|((group_index_chunk, data_chunk), mask)| { + // index_mask has value 1 << i in the loop + let mut index_mask = 1; + group_index_chunk.iter().zip(data_chunk.iter()).for_each( + |(&group_index, &new_value)| { + // valid bit was set, real value + let is_valid = (mask & index_mask) != 0; + if is_valid { + value_fn(group_index, new_value); + } + index_mask <<= 1; + }, + ) + }); + + // handle any remaining bits (after the initial 64) + let remainder_bits = bit_chunks.remainder_bits(); + group_indices_remainder + .iter() + .zip(data_remainder.iter()) + .enumerate() + .for_each(|(i, (&group_index, &new_value))| { + let is_valid = remainder_bits & (1 << i) != 0; + if is_valid { + value_fn(group_index, new_value); + } + }); + } + // no nulls, but a filter + (false, Some(filter)) => { + assert_eq!(filter.len(), group_indices.len()); + // The performance with a filter could be improved by + // iterating over the filter in chunks, rather than a single + // iterator. TODO file a ticket + group_indices + .iter() + .zip(data.iter()) + .zip(filter.iter()) + .for_each(|((&group_index, &new_value), filter_value)| { + if let Some(true) = filter_value { + value_fn(group_index, new_value); + } + }) + } + // both null values and filters + (true, Some(filter)) => { + assert_eq!(filter.len(), group_indices.len()); + // The performance with a filter could be improved by + // iterating over the filter in chunks, rather than using + // iterators. TODO file a ticket + filter + .iter() + .zip(group_indices.iter()) + .zip(values.iter()) + .for_each(|((filter_value, &group_index), new_value)| { + if let Some(true) = filter_value { + if let Some(new_value) = new_value { + value_fn(group_index, new_value) + } + } + }) + } + } +} + /// This function is called to update the accumulator state per row /// when the value is not needed (e.g. COUNT) /// diff --git a/datafusion/functions-aggregate/src/stddev.rs b/datafusion/functions-aggregate/src/stddev.rs index 180f4ad3cf37..3534fb5b4d26 100644 --- a/datafusion/functions-aggregate/src/stddev.rs +++ b/datafusion/functions-aggregate/src/stddev.rs @@ -19,17 +19,21 @@ use std::any::Any; use std::fmt::{Debug, Formatter}; +use std::sync::Arc; +use arrow::array::Float64Array; use arrow::{array::ArrayRef, datatypes::DataType, datatypes::Field}; use datafusion_common::{internal_err, not_impl_err, Result}; use datafusion_common::{plan_err, ScalarValue}; use datafusion_expr::function::{AccumulatorArgs, StateFieldsArgs}; use datafusion_expr::utils::format_state_name; -use datafusion_expr::{Accumulator, AggregateUDFImpl, Signature, Volatility}; +use datafusion_expr::{ + Accumulator, AggregateUDFImpl, GroupsAccumulator, Signature, Volatility, +}; use datafusion_functions_aggregate_common::stats::StatsType; -use crate::variance::VarianceAccumulator; +use crate::variance::{VarianceAccumulator, VarianceGroupsAccumulator}; make_udaf_expr_and_func!( Stddev, @@ -118,6 +122,17 @@ impl AggregateUDFImpl for Stddev { fn aliases(&self) -> &[String] { &self.alias } + + fn groups_accumulator_supported(&self, acc_args: AccumulatorArgs) -> bool { + !acc_args.is_distinct + } + + fn create_groups_accumulator( + &self, + _args: AccumulatorArgs, + ) -> Result> { + Ok(Box::new(StddevGroupsAccumulator::new(StatsType::Sample))) + } } make_udaf_expr_and_func!( @@ -201,6 +216,19 @@ impl AggregateUDFImpl for StddevPop { Ok(DataType::Float64) } + + fn groups_accumulator_supported(&self, acc_args: AccumulatorArgs) -> bool { + !acc_args.is_distinct + } + + fn create_groups_accumulator( + &self, + _args: AccumulatorArgs, + ) -> Result> { + Ok(Box::new(StddevGroupsAccumulator::new( + StatsType::Population, + ))) + } } /// An accumulator to compute the average @@ -267,6 +295,57 @@ impl Accumulator for StddevAccumulator { } } +#[derive(Debug)] +pub struct StddevGroupsAccumulator { + variance: VarianceGroupsAccumulator, +} + +impl StddevGroupsAccumulator { + pub fn new(s_type: StatsType) -> Self { + Self { + variance: VarianceGroupsAccumulator::new(s_type), + } + } +} + +impl GroupsAccumulator for StddevGroupsAccumulator { + fn update_batch( + &mut self, + values: &[ArrayRef], + group_indices: &[usize], + opt_filter: Option<&arrow::array::BooleanArray>, + total_num_groups: usize, + ) -> Result<()> { + self.variance + .update_batch(values, group_indices, opt_filter, total_num_groups) + } + + fn merge_batch( + &mut self, + values: &[ArrayRef], + group_indices: &[usize], + opt_filter: Option<&arrow::array::BooleanArray>, + total_num_groups: usize, + ) -> Result<()> { + self.variance + .merge_batch(values, group_indices, opt_filter, total_num_groups) + } + + fn evaluate(&mut self, emit_to: datafusion_expr::EmitTo) -> Result { + let (mut variances, nulls) = self.variance.variance(emit_to); + variances.iter_mut().for_each(|v| *v = v.sqrt()); + Ok(Arc::new(Float64Array::new(variances.into(), Some(nulls)))) + } + + fn state(&mut self, emit_to: datafusion_expr::EmitTo) -> Result> { + self.variance.state(emit_to) + } + + fn size(&self) -> usize { + self.variance.size() + } +} + #[cfg(test)] mod tests { use super::*; diff --git a/datafusion/functions-aggregate/src/variance.rs b/datafusion/functions-aggregate/src/variance.rs index 4c78a42ea494..f5f2d06e3837 100644 --- a/datafusion/functions-aggregate/src/variance.rs +++ b/datafusion/functions-aggregate/src/variance.rs @@ -18,10 +18,11 @@ //! [`VarianceSample`]: variance sample aggregations. //! [`VariancePopulation`]: variance population aggregations. -use std::fmt::Debug; +use std::{fmt::Debug, sync::Arc}; use arrow::{ - array::{ArrayRef, Float64Array, UInt64Array}, + array::{Array, ArrayRef, BooleanArray, Float64Array, UInt64Array}, + buffer::NullBuffer, compute::kernels::cast, datatypes::{DataType, Field}, }; @@ -32,9 +33,11 @@ use datafusion_common::{ use datafusion_expr::{ function::{AccumulatorArgs, StateFieldsArgs}, utils::format_state_name, - Accumulator, AggregateUDFImpl, Signature, Volatility, + Accumulator, AggregateUDFImpl, GroupsAccumulator, Signature, Volatility, +}; +use datafusion_functions_aggregate_common::{ + aggregate::groups_accumulator::accumulate::accumulate, stats::StatsType, }; -use datafusion_functions_aggregate_common::stats::StatsType; make_udaf_expr_and_func!( VarianceSample, @@ -122,6 +125,17 @@ impl AggregateUDFImpl for VarianceSample { fn aliases(&self) -> &[String] { &self.aliases } + + fn groups_accumulator_supported(&self, acc_args: AccumulatorArgs) -> bool { + !acc_args.is_distinct + } + + fn create_groups_accumulator( + &self, + _args: AccumulatorArgs, + ) -> Result> { + Ok(Box::new(VarianceGroupsAccumulator::new(StatsType::Sample))) + } } pub struct VariancePopulation { @@ -196,6 +210,19 @@ impl AggregateUDFImpl for VariancePopulation { fn aliases(&self) -> &[String] { &self.aliases } + + fn groups_accumulator_supported(&self, acc_args: AccumulatorArgs) -> bool { + !acc_args.is_distinct + } + + fn create_groups_accumulator( + &self, + _args: AccumulatorArgs, + ) -> Result> { + Ok(Box::new(VarianceGroupsAccumulator::new( + StatsType::Population, + ))) + } } /// An accumulator to compute variance @@ -239,6 +266,36 @@ impl VarianceAccumulator { } } +#[inline] +fn merge( + count: u64, + mean: f64, + m2: f64, + count2: u64, + mean2: f64, + m22: f64, +) -> (u64, f64, f64) { + let new_count = count + count2; + let new_mean = + mean * count as f64 / new_count as f64 + mean2 * count2 as f64 / new_count as f64; + let delta = mean - mean2; + let new_m2 = + m2 + m22 + delta * delta * count as f64 * count2 as f64 / new_count as f64; + + (new_count, new_mean, new_m2) +} + +#[inline] +fn update(count: u64, mean: f64, m2: f64, value: f64) -> (u64, f64, f64) { + let new_count = count + 1; + let delta1 = value - mean; + let new_mean = delta1 / new_count as f64 + mean; + let delta2 = value - new_mean; + let new_m2 = m2 + delta1 * delta2; + + (new_count, new_mean, new_m2) +} + impl Accumulator for VarianceAccumulator { fn state(&mut self) -> Result> { Ok(vec![ @@ -253,15 +310,8 @@ impl Accumulator for VarianceAccumulator { let arr = downcast_value!(values, Float64Array).iter().flatten(); for value in arr { - let new_count = self.count + 1; - let delta1 = value - self.mean; - let new_mean = delta1 / new_count as f64 + self.mean; - let delta2 = value - new_mean; - let new_m2 = self.m2 + delta1 * delta2; - - self.count += 1; - self.mean = new_mean; - self.m2 = new_m2; + (self.count, self.mean, self.m2) = + update(self.count, self.mean, self.m2, value) } Ok(()) @@ -296,17 +346,14 @@ impl Accumulator for VarianceAccumulator { if c == 0_u64 { continue; } - let new_count = self.count + c; - let new_mean = self.mean * self.count as f64 / new_count as f64 - + means.value(i) * c as f64 / new_count as f64; - let delta = self.mean - means.value(i); - let new_m2 = self.m2 - + m2s.value(i) - + delta * delta * self.count as f64 * c as f64 / new_count as f64; - - self.count = new_count; - self.mean = new_mean; - self.m2 = new_m2; + (self.count, self.mean, self.m2) = merge( + self.count, + self.mean, + self.m2, + c, + means.value(i), + m2s.value(i), + ) } Ok(()) } @@ -344,3 +391,183 @@ impl Accumulator for VarianceAccumulator { true } } + +#[derive(Debug)] +pub struct VarianceGroupsAccumulator { + m2s: Vec, + means: Vec, + counts: Vec, + stats_type: StatsType, +} + +impl VarianceGroupsAccumulator { + pub fn new(s_type: StatsType) -> Self { + Self { + m2s: Vec::new(), + means: Vec::new(), + counts: Vec::new(), + stats_type: s_type, + } + } + + fn resize(&mut self, total_num_groups: usize) { + self.m2s.resize(total_num_groups, 0.0); + self.means.resize(total_num_groups, 0.0); + self.counts.resize(total_num_groups, 0); + } + + fn merge( + group_indices: &[usize], + counts: &UInt64Array, + means: &Float64Array, + m2s: &Float64Array, + opt_filter: Option<&BooleanArray>, + mut value_fn: F, + ) where + F: FnMut(usize, u64, f64, f64) + Send, + { + assert_eq!(counts.null_count(), 0); + assert_eq!(means.null_count(), 0); + assert_eq!(m2s.null_count(), 0); + + match opt_filter { + None => { + group_indices + .iter() + .zip(counts.values().iter()) + .zip(means.values().iter()) + .zip(m2s.values().iter()) + .for_each(|(((&group_index, &count), &mean), &m2)| { + value_fn(group_index, count, mean, m2); + }); + } + Some(filter) => { + group_indices + .iter() + .zip(counts.values().iter()) + .zip(means.values().iter()) + .zip(m2s.values().iter()) + .zip(filter.iter()) + .for_each( + |((((&group_index, &count), &mean), &m2), filter_value)| { + if let Some(true) = filter_value { + value_fn(group_index, count, mean, m2); + } + }, + ); + } + } + } + + pub fn variance( + &mut self, + emit_to: datafusion_expr::EmitTo, + ) -> (Vec, NullBuffer) { + let mut counts = emit_to.take_needed(&mut self.counts); + // means are only needed for updating m2s and are not needed for the final result. + // But we still need to take them to ensure the internal state is consistent. + let _ = emit_to.take_needed(&mut self.means); + let m2s = emit_to.take_needed(&mut self.m2s); + + if let StatsType::Sample = self.stats_type { + counts.iter_mut().for_each(|count| { + *count -= 1; + }); + } + let nulls = NullBuffer::from_iter(counts.iter().map(|&count| count != 0)); + let variance = m2s + .iter() + .zip(counts) + .map(|(m2, count)| m2 / count as f64) + .collect(); + (variance, nulls) + } +} + +impl GroupsAccumulator for VarianceGroupsAccumulator { + fn update_batch( + &mut self, + values: &[ArrayRef], + group_indices: &[usize], + opt_filter: Option<&arrow::array::BooleanArray>, + total_num_groups: usize, + ) -> Result<()> { + assert_eq!(values.len(), 1, "single argument to update_batch"); + let values = &cast(&values[0], &DataType::Float64)?; + let values = downcast_value!(values, Float64Array); + + self.resize(total_num_groups); + accumulate(group_indices, values, opt_filter, |group_index, value| { + let (new_count, new_mean, new_m2) = update( + self.counts[group_index], + self.means[group_index], + self.m2s[group_index], + value, + ); + self.counts[group_index] = new_count; + self.means[group_index] = new_mean; + self.m2s[group_index] = new_m2; + }); + Ok(()) + } + + fn merge_batch( + &mut self, + values: &[ArrayRef], + group_indices: &[usize], + opt_filter: Option<&arrow::array::BooleanArray>, + total_num_groups: usize, + ) -> Result<()> { + assert_eq!(values.len(), 3, "two arguments to merge_batch"); + // first batch is counts, second is partial means, third is partial m2s + let partial_counts = downcast_value!(values[0], UInt64Array); + let partial_means = downcast_value!(values[1], Float64Array); + let partial_m2s = downcast_value!(values[2], Float64Array); + + self.resize(total_num_groups); + Self::merge( + group_indices, + partial_counts, + partial_means, + partial_m2s, + opt_filter, + |group_index, partial_count, partial_mean, partial_m2| { + let (new_count, new_mean, new_m2) = merge( + self.counts[group_index], + self.means[group_index], + self.m2s[group_index], + partial_count, + partial_mean, + partial_m2, + ); + self.counts[group_index] = new_count; + self.means[group_index] = new_mean; + self.m2s[group_index] = new_m2; + }, + ); + Ok(()) + } + + fn evaluate(&mut self, emit_to: datafusion_expr::EmitTo) -> Result { + let (variances, nulls) = self.variance(emit_to); + Ok(Arc::new(Float64Array::new(variances.into(), Some(nulls)))) + } + + fn state(&mut self, emit_to: datafusion_expr::EmitTo) -> Result> { + let counts = emit_to.take_needed(&mut self.counts); + let means = emit_to.take_needed(&mut self.means); + let m2s = emit_to.take_needed(&mut self.m2s); + + Ok(vec![ + Arc::new(UInt64Array::new(counts.into(), None)), + Arc::new(Float64Array::new(means.into(), None)), + Arc::new(Float64Array::new(m2s.into(), None)), + ]) + } + + fn size(&self) -> usize { + self.m2s.capacity() * std::mem::size_of::() + + self.means.capacity() * std::mem::size_of::() + + self.counts.capacity() * std::mem::size_of::() + } +} diff --git a/datafusion/sqllogictest/test_files/aggregate.slt b/datafusion/sqllogictest/test_files/aggregate.slt index b826dd31f4d1..45cb4d4615d7 100644 --- a/datafusion/sqllogictest/test_files/aggregate.slt +++ b/datafusion/sqllogictest/test_files/aggregate.slt @@ -511,6 +511,85 @@ select stddev(sq.column1) from (values (1.1), (2.0), (3.0)) as sq ---- 0.950438495292 +# csv_query_stddev_7 +query IR +SELECT c2, stddev_samp(c12) FROM aggregate_test_100 GROUP BY c2 ORDER BY c2 +---- +1 0.303641032262 +2 0.284581967411 +3 0.296002660506 +4 0.284324609109 +5 0.331034486752 + +# csv_query_stddev_8 +query IR +SELECT c2, stddev_pop(c12) FROM aggregate_test_100 GROUP BY c2 ORDER BY c2 +---- +1 0.296659845456 +2 0.278038978602 +3 0.288107833475 +4 0.278074953424 +5 0.318992813225 + +# csv_query_stddev_9 +query IR +SELECT c2, var_pop(c12) FROM aggregate_test_100 GROUP BY c2 ORDER BY c2 +---- +1 0.088007063906 +2 0.077305673622 +3 0.083006123709 +4 0.077325679722 +5 0.101756414889 + +# csv_query_stddev_10 +query IR +SELECT c2, var_samp(c12) FROM aggregate_test_100 GROUP BY c2 ORDER BY c2 +---- +1 0.092197876473 +2 0.080986896176 +3 0.087617575027 +4 0.080840483345 +5 0.109583831419 + +# csv_query_stddev_11 +query IR +SELECT c2, var_samp(c12) FROM aggregate_test_100 WHERE c12 > 0.90 GROUP BY c2 ORDER BY c2 +---- +1 0.000889240174 +2 0.000785878272 +3 NULL +4 NULL +5 0.000269544643 + +# Use PostgresSQL dialect +statement ok +set datafusion.sql_parser.dialect = 'Postgres'; + +# csv_query_stddev_12 +query IR +SELECT c2, var_samp(c12) FILTER (WHERE c12 > 0.90) FROM aggregate_test_100 GROUP BY c2 ORDER BY c2 +---- +1 0.000889240174 +2 0.000785878272 +3 NULL +4 NULL +5 0.000269544643 + +# Restore the default dialect +statement ok +set datafusion.sql_parser.dialect = 'Generic'; + +# csv_query_stddev_13 +query IR +SELECT c2, var_samp(CASE WHEN c12 > 0.90 THEN c12 ELSE null END) FROM aggregate_test_100 GROUP BY c2 ORDER BY c2 +---- +1 0.000889240174 +2 0.000785878272 +3 NULL +4 NULL +5 0.000269544643 + + # csv_query_approx_median_1 query I SELECT approx_median(c2) FROM aggregate_test_100 From 7d49fb308d638bef2c1ed1fa96fc87e3f740c72e Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Mon, 26 Aug 2024 15:12:23 -0400 Subject: [PATCH 023/154] Minor: refine Partitioning documentation (#12145) --- datafusion/expr/src/logical_plan/plan.rs | 6 +++++- datafusion/physical-expr/src/partitioning.rs | 6 ++++-- 2 files changed, 9 insertions(+), 3 deletions(-) diff --git a/datafusion/expr/src/logical_plan/plan.rs b/datafusion/expr/src/logical_plan/plan.rs index 3ede7f25b753..d6574040c596 100644 --- a/datafusion/expr/src/logical_plan/plan.rs +++ b/datafusion/expr/src/logical_plan/plan.rs @@ -2932,7 +2932,11 @@ impl Debug for Subquery { } } -/// Logical partitioning schemes supported by the repartition operator. +/// Logical partitioning schemes supported by [`LogicalPlan::Repartition`] +/// +/// See [`Partitioning`] for more details on partitioning +/// +/// [`Partitioning`]: https://docs.rs/datafusion/latest/datafusion/physical_expr/enum.Partitioning.html# #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub enum Partitioning { /// Allocate batches using a round-robin algorithm and the specified number of partitions diff --git a/datafusion/physical-expr/src/partitioning.rs b/datafusion/physical-expr/src/partitioning.rs index 6472dd47489c..45beeb7b81af 100644 --- a/datafusion/physical-expr/src/partitioning.rs +++ b/datafusion/physical-expr/src/partitioning.rs @@ -24,8 +24,8 @@ use crate::{physical_exprs_equal, EquivalenceProperties, PhysicalExpr}; /// Output partitioning supported by [`ExecutionPlan`]s. /// -/// When `executed`, `ExecutionPlan`s produce one or more independent stream of -/// data batches in parallel, referred to as partitions. The streams are Rust +/// Calling [`ExecutionPlan::execute`] produce one or more independent streams of +/// [`RecordBatch`]es in parallel, referred to as partitions. The streams are Rust /// `async` [`Stream`]s (a special kind of future). The number of output /// partitions varies based on the input and the operation performed. /// @@ -102,6 +102,8 @@ use crate::{physical_exprs_equal, EquivalenceProperties, PhysicalExpr}; /// Plans such as `FilterExec` produce the same number of output streams /// (partitions) as input streams (partitions). /// +/// [`RecordBatch`]: arrow::record_batch::RecordBatch +/// [`ExecutionPlan::execute`]: https://docs.rs/datafusion/latest/datafusion/physical_plan/trait.ExecutionPlan.html#tymethod.execute /// [`ExecutionPlan`]: https://docs.rs/datafusion/latest/datafusion/physical_plan/trait.ExecutionPlan.html /// [`Stream`]: https://docs.rs/futures/latest/futures/stream/trait.Stream.html #[derive(Debug, Clone)] From ed12f113cb1d33a1eea80c41787a93bc52f196c7 Mon Sep 17 00:00:00 2001 From: theirix Date: Mon, 26 Aug 2024 20:13:34 +0100 Subject: [PATCH 024/154] Minor: allow to build RuntimeEnv from RuntimeConfig (#12151) * Allow to build RuntimeEnv from RuntimeConfig * Fix formatting --- datafusion/core/tests/fuzz_cases/sort_fuzz.rs | 9 +++++---- datafusion/execution/src/runtime_env.rs | 5 +++++ 2 files changed, 10 insertions(+), 4 deletions(-) diff --git a/datafusion/core/tests/fuzz_cases/sort_fuzz.rs b/datafusion/core/tests/fuzz_cases/sort_fuzz.rs index f4b4f16aa160..eda306dd3de5 100644 --- a/datafusion/core/tests/fuzz_cases/sort_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/sort_fuzz.rs @@ -22,7 +22,7 @@ use arrow::{ compute::SortOptions, record_batch::RecordBatch, }; -use datafusion::execution::runtime_env::{RuntimeConfig, RuntimeEnv}; +use datafusion::execution::runtime_env::RuntimeConfig; use datafusion::physical_plan::expressions::PhysicalSortExpr; use datafusion::physical_plan::memory::MemoryExec; use datafusion::physical_plan::sorts::sort::SortExec; @@ -136,9 +136,10 @@ impl SortTest { .sort_spill_reservation_bytes, ); - let runtime_config = RuntimeConfig::new() - .with_memory_pool(Arc::new(GreedyMemoryPool::new(pool_size))); - let runtime = Arc::new(RuntimeEnv::new(runtime_config).unwrap()); + let runtime_env = RuntimeConfig::new() + .with_memory_pool(Arc::new(GreedyMemoryPool::new(pool_size))) + .build(); + let runtime = Arc::new(runtime_env.unwrap()); SessionContext::new_with_config_rt(session_config, runtime) } else { SessionContext::new_with_config(session_config) diff --git a/datafusion/execution/src/runtime_env.rs b/datafusion/execution/src/runtime_env.rs index 420246595558..bddce81f537b 100644 --- a/datafusion/execution/src/runtime_env.rs +++ b/datafusion/execution/src/runtime_env.rs @@ -228,4 +228,9 @@ impl RuntimeConfig { pub fn with_temp_file_path(self, path: impl Into) -> Self { self.with_disk_manager(DiskManagerConfig::new_specified(vec![path.into()])) } + + /// Build a `RuntimeEnv` object from the configuration + pub fn build(self) -> Result { + RuntimeEnv::new(self) + } } From 55a1459b123c0bb16e2cbe41cc59cf3582115932 Mon Sep 17 00:00:00 2001 From: Trent Hauck Date: Mon, 26 Aug 2024 12:19:27 -0700 Subject: [PATCH 025/154] benches: add lower benches (#12152) --- datafusion/functions/benches/lower.rs | 92 ++++++++++++++++++++++++++- 1 file changed, 90 insertions(+), 2 deletions(-) diff --git a/datafusion/functions/benches/lower.rs b/datafusion/functions/benches/lower.rs index fa963f174e46..934c1c6bd189 100644 --- a/datafusion/functions/benches/lower.rs +++ b/datafusion/functions/benches/lower.rs @@ -17,8 +17,10 @@ extern crate criterion; -use arrow::array::{ArrayRef, StringArray}; -use arrow::util::bench_util::create_string_array_with_len; +use arrow::array::{ArrayRef, StringArray, StringViewBuilder}; +use arrow::util::bench_util::{ + create_string_array_with_len, create_string_view_array_with_len, +}; use criterion::{black_box, criterion_group, criterion_main, Criterion}; use datafusion_expr::ColumnarValue; use datafusion_functions::string; @@ -65,6 +67,58 @@ fn create_args3(size: usize) -> Vec { vec![ColumnarValue::Array(array)] } +/// Create an array of args containing StringViews, where all the values in the +/// StringViews are ASCII. +/// * `size` - the length of the StringViews, and +/// * `str_len` - the length of the strings within the array. +/// * `null_density` - the density of null values in the array. +/// * `mixed` - whether the array is mixed between inlined and referenced strings. +fn create_args4( + size: usize, + str_len: usize, + null_density: f32, + mixed: bool, +) -> Vec { + let array = Arc::new(create_string_view_array_with_len( + size, + null_density, + str_len, + mixed, + )); + + vec![ColumnarValue::Array(array)] +} + +/// Create an array of args containing a StringViewArray, where some of the values in the +/// array are non-ASCII. +/// * `size` - the length of the StringArray, and +/// * `non_ascii_density` - the density of non-ASCII values in the array. +/// * `null_density` - the density of null values in the array. +fn create_args5( + size: usize, + non_ascii_density: f32, + null_density: f32, +) -> Vec { + let mut string_view_builder = StringViewBuilder::with_capacity(size); + for _ in 0..size { + // sample null_density to determine if the value should be null + if rand::random::() < null_density { + string_view_builder.append_null(); + continue; + } + + // sample non_ascii_density to determine if the value should be non-ASCII + if rand::random::() < non_ascii_density { + string_view_builder.append_value("农历新年农历新年农历新年农历新年农历新年"); + } else { + string_view_builder.append_value("DATAFUSIONDATAFUSIONDATAFUSION"); + } + } + + let array = Arc::new(string_view_builder.finish()) as ArrayRef; + vec![ColumnarValue::Array(array)] +} + fn criterion_benchmark(c: &mut Criterion) { let lower = string::lower(); for size in [1024, 4096, 8192] { @@ -85,6 +139,40 @@ fn criterion_benchmark(c: &mut Criterion) { |b| b.iter(|| black_box(lower.invoke(&args))), ); } + + let sizes = [4096, 8192]; + let str_lens = [10, 64, 128]; + let mixes = [true, false]; + let null_densities = [0.0f32, 0.1f32]; + + for null_density in &null_densities { + for &mixed in &mixes { + for &str_len in &str_lens { + for &size in &sizes { + let args = create_args4(size, str_len, *null_density, mixed); + c.bench_function( + &format!("lower_all_values_are_ascii_string_views: size: {}, str_len: {}, null_density: {}, mixed: {}", + size, str_len, null_density, mixed), + |b| b.iter(|| black_box(lower.invoke(&args))), + ); + + let args = create_args4(size, str_len, *null_density, mixed); + c.bench_function( + &format!("lower_all_values_are_ascii_string_views: size: {}, str_len: {}, null_density: {}, mixed: {}", + size, str_len, null_density, mixed), + |b| b.iter(|| black_box(lower.invoke(&args))), + ); + + let args = create_args5(size, 0.1, *null_density); + c.bench_function( + &format!("lower_some_values_are_nonascii_string_views: size: {}, str_len: {}, non_ascii_density: {}, null_density: {}, mixed: {}", + size, str_len, 0.1, null_density, mixed), + |b| b.iter(|| black_box(lower.invoke(&args))), + ); + } + } + } + } } criterion_group!(benches, criterion_benchmark); From 0f96af5b500efff72314f840a59a736787cc3def Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Mon, 26 Aug 2024 21:24:51 +0200 Subject: [PATCH 026/154] Replace Arc::try_unwrap with Arc::unwrap_or_clone where cloning anyway (#12173) --- datafusion/core/src/execution/context/mod.rs | 2 +- datafusion/expr/src/expr_fn.rs | 8 ++++---- datafusion/expr/src/logical_plan/plan.rs | 5 +---- datafusion/sql/src/unparser/rewrite.rs | 5 +---- 4 files changed, 7 insertions(+), 13 deletions(-) diff --git a/datafusion/core/src/execution/context/mod.rs b/datafusion/core/src/execution/context/mod.rs index 49cbf9ebde9d..c2a072eef756 100644 --- a/datafusion/core/src/execution/context/mod.rs +++ b/datafusion/core/src/execution/context/mod.rs @@ -688,7 +688,7 @@ impl SessionContext { column_defaults, } = cmd; - let input = Arc::try_unwrap(input).unwrap_or_else(|e| e.as_ref().clone()); + let input = Arc::unwrap_or_clone(input); let input = self.state().optimize(&input)?; let table = self.table(name.clone()).await; match (if_not_exists, or_replace, table) { diff --git a/datafusion/expr/src/expr_fn.rs b/datafusion/expr/src/expr_fn.rs index 4e6022399653..1e0b601146dd 100644 --- a/datafusion/expr/src/expr_fn.rs +++ b/datafusion/expr/src/expr_fn.rs @@ -394,7 +394,7 @@ pub fn create_udf( volatility: Volatility, fun: ScalarFunctionImplementation, ) -> ScalarUDF { - let return_type = Arc::try_unwrap(return_type).unwrap_or_else(|t| t.as_ref().clone()); + let return_type = Arc::unwrap_or_clone(return_type); ScalarUDF::from(SimpleScalarUDF::new( name, input_types, @@ -476,8 +476,8 @@ pub fn create_udaf( accumulator: AccumulatorFactoryFunction, state_type: Arc>, ) -> AggregateUDF { - let return_type = Arc::try_unwrap(return_type).unwrap_or_else(|t| t.as_ref().clone()); - let state_type = Arc::try_unwrap(state_type).unwrap_or_else(|t| t.as_ref().clone()); + let return_type = Arc::unwrap_or_clone(return_type); + let state_type = Arc::unwrap_or_clone(state_type); let state_fields = state_type .into_iter() .enumerate() @@ -594,7 +594,7 @@ pub fn create_udwf( volatility: Volatility, partition_evaluator_factory: PartitionEvaluatorFactory, ) -> WindowUDF { - let return_type = Arc::try_unwrap(return_type).unwrap_or_else(|t| t.as_ref().clone()); + let return_type = Arc::unwrap_or_clone(return_type); WindowUDF::from(SimpleWindowUDF::new( name, input_type, diff --git a/datafusion/expr/src/logical_plan/plan.rs b/datafusion/expr/src/logical_plan/plan.rs index d6574040c596..096b65896b69 100644 --- a/datafusion/expr/src/logical_plan/plan.rs +++ b/datafusion/expr/src/logical_plan/plan.rs @@ -1160,10 +1160,7 @@ impl LogicalPlan { Ok(if let LogicalPlan::Prepare(prepare_lp) = plan_with_values { param_values.verify(&prepare_lp.data_types)?; // try and take ownership of the input if is not shared, clone otherwise - match Arc::try_unwrap(prepare_lp.input) { - Ok(input) => input, - Err(arc_input) => arc_input.as_ref().clone(), - } + Arc::unwrap_or_clone(prepare_lp.input) } else { plan_with_values }) diff --git a/datafusion/sql/src/unparser/rewrite.rs b/datafusion/sql/src/unparser/rewrite.rs index f6725485f920..fed3f2c39f9d 100644 --- a/datafusion/sql/src/unparser/rewrite.rs +++ b/datafusion/sql/src/unparser/rewrite.rs @@ -59,10 +59,7 @@ pub(super) fn normalize_union_schema(plan: &LogicalPlan) -> Result let transformed_plan = plan.transform_up(|plan| match plan { LogicalPlan::Union(mut union) => { - let schema = match Arc::try_unwrap(union.schema) { - Ok(inner) => inner, - Err(schema) => (*schema).clone(), - }; + let schema = Arc::unwrap_or_clone(union.schema); let schema = schema.strip_qualifiers(); union.schema = Arc::new(schema); From dff590bfd2bb9993b2c8ce6f76a3bdd973e520a8 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Mon, 26 Aug 2024 13:57:17 -0700 Subject: [PATCH 027/154] fix: Skip buffered rows which are not joined with streamed side when checking join filter results (#12159) --- datafusion/physical-plan/src/joins/sort_merge_join.rs | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/datafusion/physical-plan/src/joins/sort_merge_join.rs b/datafusion/physical-plan/src/joins/sort_merge_join.rs index 56cd699bf6e7..511cb4c55fcd 100644 --- a/datafusion/physical-plan/src/joins/sort_merge_join.rs +++ b/datafusion/physical-plan/src/joins/sort_merge_join.rs @@ -1474,6 +1474,12 @@ impl SMJStream { [chunk.buffered_batch_idx.unwrap()]; for i in 0..pre_mask.len() { + // If the buffered row is not joined with streamed side, + // skip it. + if buffered_indices.is_null(i) { + continue; + } + let buffered_index = buffered_indices.value(i); buffered_batch.join_filter_failed_map.insert( From 1e67cb6239dbaa896344335678356c24ef1df509 Mon Sep 17 00:00:00 2001 From: Jax Liu Date: Tue, 27 Aug 2024 08:22:42 +0800 Subject: [PATCH 028/154] enable the null map tests (#12176) --- datafusion/sqllogictest/test_files/map.slt | 9 ++++----- 1 file changed, 4 insertions(+), 5 deletions(-) diff --git a/datafusion/sqllogictest/test_files/map.slt b/datafusion/sqllogictest/test_files/map.slt index b7a0a74913b0..270e4beccc52 100644 --- a/datafusion/sqllogictest/test_files/map.slt +++ b/datafusion/sqllogictest/test_files/map.slt @@ -376,11 +376,10 @@ SELECT MAP {'a':1, 'b':2, 'c':3 }['a'] FROM t; 1 1 -# TODO(https://github.com/sqlparser-rs/sqlparser-rs/pull/1361): support parsing an empty map. Enable this after upgrading sqlparser-rs. -# query ? -# SELECT MAP {}; -# ---- -# {} +query ? +SELECT MAP {}; +---- +{} # values contain null query ? From a723b79eca998dfa61cd2aefa122779336b153c7 Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Tue, 27 Aug 2024 03:34:30 +0200 Subject: [PATCH 029/154] Remove unwrap_arc helper (#12172) It can now be replaced with single call `Arc::unwrap_or_clone`, with added bonus of slightly better name. --- datafusion/expr/src/logical_plan/builder.rs | 37 +++++++++++-------- datafusion/expr/src/logical_plan/plan.rs | 3 +- datafusion/expr/src/logical_plan/tree_node.rs | 11 +----- .../optimizer/src/analyzer/type_coercion.rs | 18 ++++++--- .../optimizer/src/common_subexpr_eliminate.rs | 11 +++--- .../src/decorrelate_predicate_subquery.rs | 9 +++-- .../optimizer/src/eliminate_cross_join.rs | 27 +++++++++++--- datafusion/optimizer/src/eliminate_filter.rs | 3 +- datafusion/optimizer/src/eliminate_limit.rs | 6 ++- .../optimizer/src/eliminate_nested_union.rs | 14 +++---- .../optimizer/src/eliminate_one_union.rs | 10 +++-- .../optimizer/src/eliminate_outer_join.rs | 3 +- .../optimizer/src/optimize_projections/mod.rs | 14 +++---- .../optimizer/src/propagate_empty_relation.rs | 3 +- datafusion/optimizer/src/push_down_filter.rs | 23 ++++++------ datafusion/optimizer/src/push_down_limit.rs | 3 +- datafusion/sql/src/select.rs | 10 ++--- 17 files changed, 110 insertions(+), 95 deletions(-) diff --git a/datafusion/expr/src/logical_plan/builder.rs b/datafusion/expr/src/logical_plan/builder.rs index fdd07da023e0..6abcfe4279bc 100644 --- a/datafusion/expr/src/logical_plan/builder.rs +++ b/datafusion/expr/src/logical_plan/builder.rs @@ -41,9 +41,8 @@ use crate::utils::{ find_valid_equijoin_key_pair, group_window_expr_by_sort_keys, }; use crate::{ - and, binary_expr, logical_plan::tree_node::unwrap_arc, DmlStatement, Expr, - ExprSchemable, Operator, RecursiveQuery, TableProviderFilterPushDown, TableSource, - WriteOp, + and, binary_expr, DmlStatement, Expr, ExprSchemable, Operator, RecursiveQuery, + TableProviderFilterPushDown, TableSource, WriteOp, }; use arrow::datatypes::{DataType, Field, Fields, Schema, SchemaRef}; @@ -376,7 +375,7 @@ impl LogicalPlanBuilder { self, expr: impl IntoIterator>, ) -> Result { - project(unwrap_arc(self.plan), expr).map(Self::new) + project(Arc::unwrap_or_clone(self.plan), expr).map(Self::new) } /// Select the given column indices @@ -429,7 +428,7 @@ impl LogicalPlanBuilder { /// Apply an alias pub fn alias(self, alias: impl Into) -> Result { - subquery_alias(unwrap_arc(self.plan), alias).map(Self::new) + subquery_alias(Arc::unwrap_or_clone(self.plan), alias).map(Self::new) } /// Add missing sort columns to all downstream projection @@ -484,7 +483,7 @@ impl LogicalPlanBuilder { Self::ambiguous_distinct_check(&missing_exprs, missing_cols, &expr)?; } expr.extend(missing_exprs); - project(unwrap_arc(input), expr) + project(Arc::unwrap_or_clone(input), expr) } _ => { let is_distinct = @@ -580,8 +579,11 @@ impl LogicalPlanBuilder { let new_expr = schema.columns().into_iter().map(Expr::Column).collect(); let is_distinct = false; - let plan = - Self::add_missing_columns(unwrap_arc(self.plan), &missing_cols, is_distinct)?; + let plan = Self::add_missing_columns( + Arc::unwrap_or_clone(self.plan), + &missing_cols, + is_distinct, + )?; let sort_plan = LogicalPlan::Sort(Sort { expr: normalize_cols(exprs, &plan)?, input: Arc::new(plan), @@ -595,12 +597,12 @@ impl LogicalPlanBuilder { /// Apply a union, preserving duplicate rows pub fn union(self, plan: LogicalPlan) -> Result { - union(unwrap_arc(self.plan), plan).map(Self::new) + union(Arc::unwrap_or_clone(self.plan), plan).map(Self::new) } /// Apply a union, removing duplicate rows pub fn union_distinct(self, plan: LogicalPlan) -> Result { - let left_plan: LogicalPlan = unwrap_arc(self.plan); + let left_plan: LogicalPlan = Arc::unwrap_or_clone(self.plan); let right_plan: LogicalPlan = plan; Ok(Self::new(LogicalPlan::Distinct(Distinct::All(Arc::new( @@ -1064,7 +1066,7 @@ impl LogicalPlanBuilder { /// Build the plan pub fn build(self) -> Result { - Ok(unwrap_arc(self.plan)) + Ok(Arc::unwrap_or_clone(self.plan)) } /// Apply a join with the expression on constraint. @@ -1138,7 +1140,7 @@ impl LogicalPlanBuilder { /// Unnest the given column. pub fn unnest_column(self, column: impl Into) -> Result { - unnest(unwrap_arc(self.plan), vec![column.into()]).map(Self::new) + unnest(Arc::unwrap_or_clone(self.plan), vec![column.into()]).map(Self::new) } /// Unnest the given column given [`UnnestOptions`] @@ -1147,8 +1149,12 @@ impl LogicalPlanBuilder { column: impl Into, options: UnnestOptions, ) -> Result { - unnest_with_options(unwrap_arc(self.plan), vec![column.into()], options) - .map(Self::new) + unnest_with_options( + Arc::unwrap_or_clone(self.plan), + vec![column.into()], + options, + ) + .map(Self::new) } /// Unnest the given columns with the given [`UnnestOptions`] @@ -1157,7 +1163,8 @@ impl LogicalPlanBuilder { columns: Vec, options: UnnestOptions, ) -> Result { - unnest_with_options(unwrap_arc(self.plan), columns, options).map(Self::new) + unnest_with_options(Arc::unwrap_or_clone(self.plan), columns, options) + .map(Self::new) } } diff --git a/datafusion/expr/src/logical_plan/plan.rs b/datafusion/expr/src/logical_plan/plan.rs index 096b65896b69..359de2d30a57 100644 --- a/datafusion/expr/src/logical_plan/plan.rs +++ b/datafusion/expr/src/logical_plan/plan.rs @@ -51,7 +51,6 @@ use datafusion_common::{ // backwards compatibility use crate::display::PgJsonVisitor; -use crate::logical_plan::tree_node::unwrap_arc; pub use datafusion_common::display::{PlanType, StringifiedPlan, ToStringifiedPlan}; pub use datafusion_common::{JoinConstraint, JoinType}; @@ -770,7 +769,7 @@ impl LogicalPlan { .. }) => { // Update schema with unnested column type. - unnest_with_options(unwrap_arc(input), exec_columns, options) + unnest_with_options(Arc::unwrap_or_clone(input), exec_columns, options) } } } diff --git a/datafusion/expr/src/logical_plan/tree_node.rs b/datafusion/expr/src/logical_plan/tree_node.rs index 539cb1cf5fb2..273404c8df31 100644 --- a/datafusion/expr/src/logical_plan/tree_node.rs +++ b/datafusion/expr/src/logical_plan/tree_node.rs @@ -379,21 +379,12 @@ impl TreeNode for LogicalPlan { } } -/// Converts a `Arc` without copying, if possible. Copies the plan -/// if there is a shared reference -pub fn unwrap_arc(plan: Arc) -> LogicalPlan { - Arc::try_unwrap(plan) - // if None is returned, there is another reference to this - // LogicalPlan, so we can not own it, and must clone instead - .unwrap_or_else(|node| node.as_ref().clone()) -} - /// Applies `f` to rewrite a `Arc` without copying, if possible fn rewrite_arc Result>>( plan: Arc, mut f: F, ) -> Result>> { - f(unwrap_arc(plan))?.map_data(|new_plan| Ok(Arc::new(new_plan))) + f(Arc::unwrap_or_clone(plan))?.map_data(|new_plan| Ok(Arc::new(new_plan))) } /// rewrite a `Vec` of `Arc` without copying, if possible diff --git a/datafusion/optimizer/src/analyzer/type_coercion.rs b/datafusion/optimizer/src/analyzer/type_coercion.rs index 315284c50839..a6b9bad6c5d9 100644 --- a/datafusion/optimizer/src/analyzer/type_coercion.rs +++ b/datafusion/optimizer/src/analyzer/type_coercion.rs @@ -37,7 +37,6 @@ use datafusion_expr::expr::{ }; use datafusion_expr::expr_rewriter::coerce_plan_expr_for_schema; use datafusion_expr::expr_schema::cast_subquery; -use datafusion_expr::logical_plan::tree_node::unwrap_arc; use datafusion_expr::logical_plan::Subquery; use datafusion_expr::type_coercion::binary::{ comparison_coercion, get_input_types, like_coercion, @@ -250,15 +249,19 @@ impl<'a> TreeNodeRewriter for TypeCoercionRewriter<'a> { subquery, outer_ref_columns, }) => { - let new_plan = analyze_internal(self.schema, unwrap_arc(subquery))?.data; + let new_plan = + analyze_internal(self.schema, Arc::unwrap_or_clone(subquery))?.data; Ok(Transformed::yes(Expr::ScalarSubquery(Subquery { subquery: Arc::new(new_plan), outer_ref_columns, }))) } Expr::Exists(Exists { subquery, negated }) => { - let new_plan = - analyze_internal(self.schema, unwrap_arc(subquery.subquery))?.data; + let new_plan = analyze_internal( + self.schema, + Arc::unwrap_or_clone(subquery.subquery), + )? + .data; Ok(Transformed::yes(Expr::Exists(Exists { subquery: Subquery { subquery: Arc::new(new_plan), @@ -272,8 +275,11 @@ impl<'a> TreeNodeRewriter for TypeCoercionRewriter<'a> { subquery, negated, }) => { - let new_plan = - analyze_internal(self.schema, unwrap_arc(subquery.subquery))?.data; + let new_plan = analyze_internal( + self.schema, + Arc::unwrap_or_clone(subquery.subquery), + )? + .data; let expr_type = expr.get_type(self.schema)?; let subquery_type = new_plan.schema().field(0).data_type(); let common_type = comparison_coercion(&expr_type, subquery_type).ok_or(plan_datafusion_err!( diff --git a/datafusion/optimizer/src/common_subexpr_eliminate.rs b/datafusion/optimizer/src/common_subexpr_eliminate.rs index feccf5679efb..3fcee4123b76 100644 --- a/datafusion/optimizer/src/common_subexpr_eliminate.rs +++ b/datafusion/optimizer/src/common_subexpr_eliminate.rs @@ -33,7 +33,6 @@ use datafusion_common::tree_node::{ }; use datafusion_common::{qualified_name, Column, DFSchema, DFSchemaRef, Result}; use datafusion_expr::expr::{Alias, ScalarFunction}; -use datafusion_expr::logical_plan::tree_node::unwrap_arc; use datafusion_expr::logical_plan::{ Aggregate, Filter, LogicalPlan, Projection, Sort, Window, }; @@ -314,7 +313,7 @@ impl CommonSubexprEliminate { schema, .. } = projection; - let input = unwrap_arc(input); + let input = Arc::unwrap_or_clone(input); self.try_unary_plan(expr, input, config)? .map_data(|(new_expr, new_input)| { Projection::try_new_with_schema(new_expr, Arc::new(new_input), schema) @@ -327,7 +326,7 @@ impl CommonSubexprEliminate { config: &dyn OptimizerConfig, ) -> Result> { let Sort { expr, input, fetch } = sort; - let input = unwrap_arc(input); + let input = Arc::unwrap_or_clone(input); let new_sort = self.try_unary_plan(expr, input, config)?.update_data( |(new_expr, new_input)| { LogicalPlan::Sort(Sort { @@ -348,7 +347,7 @@ impl CommonSubexprEliminate { let Filter { predicate, input, .. } = filter; - let input = unwrap_arc(input); + let input = Arc::unwrap_or_clone(input); let expr = vec![predicate]; self.try_unary_plan(expr, input, config)? .map_data(|(mut new_expr, new_input)| { @@ -458,7 +457,7 @@ impl CommonSubexprEliminate { schema, .. } = aggregate; - let input = unwrap_arc(input); + let input = Arc::unwrap_or_clone(input); // Extract common sub-expressions from the aggregate and grouping expressions. self.find_common_exprs(vec![group_expr, aggr_expr], config, ExprMask::Normal)? .map_data(|common| { @@ -729,7 +728,7 @@ fn get_consecutive_window_exprs( window_expr_list.push(window_expr); window_schemas.push(schema); - plan = unwrap_arc(input); + plan = Arc::unwrap_or_clone(input); } (window_expr_list, window_schemas, plan) } diff --git a/datafusion/optimizer/src/decorrelate_predicate_subquery.rs b/datafusion/optimizer/src/decorrelate_predicate_subquery.rs index b6d49490d437..f1cae1099a4d 100644 --- a/datafusion/optimizer/src/decorrelate_predicate_subquery.rs +++ b/datafusion/optimizer/src/decorrelate_predicate_subquery.rs @@ -37,7 +37,6 @@ use datafusion_expr::{ LogicalPlan, LogicalPlanBuilder, Operator, }; -use datafusion_expr::logical_plan::tree_node::unwrap_arc; use log::debug; /// Optimizer rule for rewriting predicate(IN/EXISTS) subquery to left semi/anti joins @@ -55,8 +54,10 @@ impl DecorrelatePredicateSubquery { mut subquery: Subquery, config: &dyn OptimizerConfig, ) -> Result { - subquery.subquery = - Arc::new(self.rewrite(unwrap_arc(subquery.subquery), config)?.data); + subquery.subquery = Arc::new( + self.rewrite(Arc::unwrap_or_clone(subquery.subquery), config)? + .data, + ); Ok(subquery) } @@ -164,7 +165,7 @@ impl OptimizerRule for DecorrelatePredicateSubquery { } // iterate through all exists clauses in predicate, turning each into a join - let mut cur_input = unwrap_arc(input); + let mut cur_input = Arc::unwrap_or_clone(input); for subquery in subqueries { if let Some(plan) = build_join(&subquery, &cur_input, config.alias_generator())? diff --git a/datafusion/optimizer/src/eliminate_cross_join.rs b/datafusion/optimizer/src/eliminate_cross_join.rs index fc4eaef80903..20e6641e4d62 100644 --- a/datafusion/optimizer/src/eliminate_cross_join.rs +++ b/datafusion/optimizer/src/eliminate_cross_join.rs @@ -24,7 +24,6 @@ use crate::join_key_set::JoinKeySet; use datafusion_common::tree_node::{Transformed, TreeNode}; use datafusion_common::{internal_err, Result}; use datafusion_expr::expr::{BinaryExpr, Expr}; -use datafusion_expr::logical_plan::tree_node::unwrap_arc; use datafusion_expr::logical_plan::{ CrossJoin, Filter, Join, JoinConstraint, JoinType, LogicalPlan, Projection, }; @@ -114,7 +113,7 @@ impl OptimizerRule for EliminateCrossJoin { input, predicate, .. } = filter; flatten_join_inputs( - unwrap_arc(input), + Arc::unwrap_or_clone(input), &mut possible_join_keys, &mut all_inputs, )?; @@ -217,12 +216,28 @@ fn flatten_join_inputs( ); } possible_join_keys.insert_all_owned(join.on); - flatten_join_inputs(unwrap_arc(join.left), possible_join_keys, all_inputs)?; - flatten_join_inputs(unwrap_arc(join.right), possible_join_keys, all_inputs)?; + flatten_join_inputs( + Arc::unwrap_or_clone(join.left), + possible_join_keys, + all_inputs, + )?; + flatten_join_inputs( + Arc::unwrap_or_clone(join.right), + possible_join_keys, + all_inputs, + )?; } LogicalPlan::CrossJoin(join) => { - flatten_join_inputs(unwrap_arc(join.left), possible_join_keys, all_inputs)?; - flatten_join_inputs(unwrap_arc(join.right), possible_join_keys, all_inputs)?; + flatten_join_inputs( + Arc::unwrap_or_clone(join.left), + possible_join_keys, + all_inputs, + )?; + flatten_join_inputs( + Arc::unwrap_or_clone(join.right), + possible_join_keys, + all_inputs, + )?; } _ => { all_inputs.push(plan); diff --git a/datafusion/optimizer/src/eliminate_filter.rs b/datafusion/optimizer/src/eliminate_filter.rs index 84bb8e782142..bb2b4547e9c2 100644 --- a/datafusion/optimizer/src/eliminate_filter.rs +++ b/datafusion/optimizer/src/eliminate_filter.rs @@ -19,7 +19,6 @@ use datafusion_common::tree_node::Transformed; use datafusion_common::{Result, ScalarValue}; -use datafusion_expr::logical_plan::tree_node::unwrap_arc; use datafusion_expr::{EmptyRelation, Expr, Filter, LogicalPlan}; use std::sync::Arc; @@ -65,7 +64,7 @@ impl OptimizerRule for EliminateFilter { input, .. }) => match v { - Some(true) => Ok(Transformed::yes(unwrap_arc(input))), + Some(true) => Ok(Transformed::yes(Arc::unwrap_or_clone(input))), Some(false) | None => Ok(Transformed::yes(LogicalPlan::EmptyRelation( EmptyRelation { produce_one_row: false, diff --git a/datafusion/optimizer/src/eliminate_limit.rs b/datafusion/optimizer/src/eliminate_limit.rs index a42fe6a6f95b..e48f37a77cd3 100644 --- a/datafusion/optimizer/src/eliminate_limit.rs +++ b/datafusion/optimizer/src/eliminate_limit.rs @@ -20,7 +20,7 @@ use crate::optimizer::ApplyOrder; use crate::{OptimizerConfig, OptimizerRule}; use datafusion_common::tree_node::Transformed; use datafusion_common::Result; -use datafusion_expr::logical_plan::{tree_node::unwrap_arc, EmptyRelation, LogicalPlan}; +use datafusion_expr::logical_plan::{EmptyRelation, LogicalPlan}; use std::sync::Arc; /// Optimizer rule to replace `LIMIT 0` or `LIMIT` whose ancestor LIMIT's skip is @@ -74,7 +74,9 @@ impl OptimizerRule for EliminateLimit { } } else if limit.skip == 0 { // input also can be Limit, so we should apply again. - return Ok(self.rewrite(unwrap_arc(limit.input), _config).unwrap()); + return Ok(self + .rewrite(Arc::unwrap_or_clone(limit.input), _config) + .unwrap()); } Ok(Transformed::no(LogicalPlan::Limit(limit))) } diff --git a/datafusion/optimizer/src/eliminate_nested_union.rs b/datafusion/optimizer/src/eliminate_nested_union.rs index 5d7895bba4d8..e9b38567a982 100644 --- a/datafusion/optimizer/src/eliminate_nested_union.rs +++ b/datafusion/optimizer/src/eliminate_nested_union.rs @@ -21,7 +21,6 @@ use crate::{OptimizerConfig, OptimizerRule}; use datafusion_common::tree_node::Transformed; use datafusion_common::Result; use datafusion_expr::expr_rewriter::coerce_plan_expr_for_schema; -use datafusion_expr::logical_plan::tree_node::unwrap_arc; use datafusion_expr::{Distinct, LogicalPlan, Union}; use itertools::Itertools; use std::sync::Arc; @@ -69,7 +68,7 @@ impl OptimizerRule for EliminateNestedUnion { }))) } LogicalPlan::Distinct(Distinct::All(nested_plan)) => { - match unwrap_arc(nested_plan) { + match Arc::unwrap_or_clone(nested_plan) { LogicalPlan::Union(Union { inputs, schema }) => { let inputs = inputs .into_iter() @@ -96,16 +95,17 @@ impl OptimizerRule for EliminateNestedUnion { } fn extract_plans_from_union(plan: Arc) -> Vec { - match unwrap_arc(plan) { - LogicalPlan::Union(Union { inputs, .. }) => { - inputs.into_iter().map(unwrap_arc).collect::>() - } + match Arc::unwrap_or_clone(plan) { + LogicalPlan::Union(Union { inputs, .. }) => inputs + .into_iter() + .map(Arc::unwrap_or_clone) + .collect::>(), plan => vec![plan], } } fn extract_plan_from_distinct(plan: Arc) -> Arc { - match unwrap_arc(plan) { + match Arc::unwrap_or_clone(plan) { LogicalPlan::Distinct(Distinct::All(plan)) => plan, plan => Arc::new(plan), } diff --git a/datafusion/optimizer/src/eliminate_one_union.rs b/datafusion/optimizer/src/eliminate_one_union.rs index 43024107c4f8..383b33637f6f 100644 --- a/datafusion/optimizer/src/eliminate_one_union.rs +++ b/datafusion/optimizer/src/eliminate_one_union.rs @@ -16,9 +16,11 @@ // under the License. //! [`EliminateOneUnion`] eliminates single element `Union` + use crate::{OptimizerConfig, OptimizerRule}; use datafusion_common::{tree_node::Transformed, Result}; -use datafusion_expr::logical_plan::{tree_node::unwrap_arc, LogicalPlan, Union}; +use datafusion_expr::logical_plan::{LogicalPlan, Union}; +use std::sync::Arc; use crate::optimizer::ApplyOrder; @@ -48,9 +50,9 @@ impl OptimizerRule for EliminateOneUnion { _config: &dyn OptimizerConfig, ) -> Result> { match plan { - LogicalPlan::Union(Union { mut inputs, .. }) if inputs.len() == 1 => { - Ok(Transformed::yes(unwrap_arc(inputs.pop().unwrap()))) - } + LogicalPlan::Union(Union { mut inputs, .. }) if inputs.len() == 1 => Ok( + Transformed::yes(Arc::unwrap_or_clone(inputs.pop().unwrap())), + ), _ => Ok(Transformed::no(plan)), } } diff --git a/datafusion/optimizer/src/eliminate_outer_join.rs b/datafusion/optimizer/src/eliminate_outer_join.rs index 12534e058152..e7c88df55122 100644 --- a/datafusion/optimizer/src/eliminate_outer_join.rs +++ b/datafusion/optimizer/src/eliminate_outer_join.rs @@ -18,7 +18,6 @@ //! [`EliminateOuterJoin`] converts `LEFT/RIGHT/FULL` joins to `INNER` joins use crate::{OptimizerConfig, OptimizerRule}; use datafusion_common::{Column, DFSchema, Result}; -use datafusion_expr::logical_plan::tree_node::unwrap_arc; use datafusion_expr::logical_plan::{Join, JoinType, LogicalPlan}; use datafusion_expr::{Expr, Filter, Operator}; @@ -79,7 +78,7 @@ impl OptimizerRule for EliminateOuterJoin { _config: &dyn OptimizerConfig, ) -> Result> { match plan { - LogicalPlan::Filter(mut filter) => match unwrap_arc(filter.input) { + LogicalPlan::Filter(mut filter) => match Arc::unwrap_or_clone(filter.input) { LogicalPlan::Join(join) => { let mut non_nullable_cols: Vec = vec![]; diff --git a/datafusion/optimizer/src/optimize_projections/mod.rs b/datafusion/optimizer/src/optimize_projections/mod.rs index ac4ed87a4a1a..5db5afd11062 100644 --- a/datafusion/optimizer/src/optimize_projections/mod.rs +++ b/datafusion/optimizer/src/optimize_projections/mod.rs @@ -41,7 +41,6 @@ use crate::utils::NamePreserver; use datafusion_common::tree_node::{ Transformed, TreeNode, TreeNodeIterator, TreeNodeRecursion, }; -use datafusion_expr::logical_plan::tree_node::unwrap_arc; /// Optimizer rule to prune unnecessary columns from intermediate schemas /// inside the [`LogicalPlan`]. This rule: @@ -181,7 +180,7 @@ fn optimize_projections( let necessary_exprs = necessary_indices.get_required_exprs(schema); return optimize_projections( - unwrap_arc(aggregate.input), + Arc::unwrap_or_clone(aggregate.input), config, necessary_indices, )? @@ -221,7 +220,7 @@ fn optimize_projections( child_reqs.with_exprs(&input_schema, &new_window_expr)?; return optimize_projections( - unwrap_arc(window.input), + Arc::unwrap_or_clone(window.input), config, required_indices.clone(), )? @@ -488,7 +487,7 @@ fn merge_consecutive_projections(proj: Projection) -> Result { let parents_predicates = split_conjunction_owned(filter.predicate); @@ -1139,19 +1138,19 @@ fn convert_to_cross_join_if_beneficial( match plan { // Can be converted back to cross join LogicalPlan::Join(join) if join.on.is_empty() && join.filter.is_none() => { - LogicalPlanBuilder::from(unwrap_arc(join.left)) - .cross_join(unwrap_arc(join.right))? + LogicalPlanBuilder::from(Arc::unwrap_or_clone(join.left)) + .cross_join(Arc::unwrap_or_clone(join.right))? .build() .map(Transformed::yes) } - LogicalPlan::Filter(filter) => convert_to_cross_join_if_beneficial(unwrap_arc( - filter.input, - ))? - .transform_data(|child_plan| { - Filter::try_new(filter.predicate, Arc::new(child_plan)) - .map(LogicalPlan::Filter) - .map(Transformed::yes) - }), + LogicalPlan::Filter(filter) => { + convert_to_cross_join_if_beneficial(Arc::unwrap_or_clone(filter.input))? + .transform_data(|child_plan| { + Filter::try_new(filter.predicate, Arc::new(child_plan)) + .map(LogicalPlan::Filter) + .map(Transformed::yes) + }) + } plan => Ok(Transformed::no(plan)), } } diff --git a/datafusion/optimizer/src/push_down_limit.rs b/datafusion/optimizer/src/push_down_limit.rs index 290b893577b8..dff0b61c6b22 100644 --- a/datafusion/optimizer/src/push_down_limit.rs +++ b/datafusion/optimizer/src/push_down_limit.rs @@ -26,7 +26,6 @@ use crate::{OptimizerConfig, OptimizerRule}; use datafusion_common::tree_node::Transformed; use datafusion_common::utils::combine_limit; use datafusion_common::Result; -use datafusion_expr::logical_plan::tree_node::unwrap_arc; use datafusion_expr::logical_plan::{Join, JoinType, Limit, LogicalPlan}; /// Optimization rule that tries to push down `LIMIT`. @@ -83,7 +82,7 @@ impl OptimizerRule for PushDownLimit { }))); }; - match unwrap_arc(input) { + match Arc::unwrap_or_clone(input) { LogicalPlan::TableScan(mut scan) => { let rows_needed = if fetch != 0 { fetch + skip } else { 0 }; let new_fetch = scan diff --git a/datafusion/sql/src/select.rs b/datafusion/sql/src/select.rs index 45fda094557b..384893bfa94c 100644 --- a/datafusion/sql/src/select.rs +++ b/datafusion/sql/src/select.rs @@ -33,7 +33,6 @@ use datafusion_expr::expr::{Alias, PlannedReplaceSelectItem, WildcardOptions}; use datafusion_expr::expr_rewriter::{ normalize_col, normalize_col_with_schemas_and_ambiguity_check, normalize_cols, }; -use datafusion_expr::logical_plan::tree_node::unwrap_arc; use datafusion_expr::utils::{ expr_as_column_expr, expr_to_columns, find_aggregate_exprs, find_window_exprs, }; @@ -361,9 +360,10 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { .build() } LogicalPlan::Filter(mut filter) => { - filter.input = Arc::new( - self.try_process_aggregate_unnest(unwrap_arc(filter.input))?, - ); + filter.input = + Arc::new(self.try_process_aggregate_unnest(Arc::unwrap_or_clone( + filter.input, + ))?); Ok(LogicalPlan::Filter(filter)) } _ => Ok(input), @@ -401,7 +401,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { // Projection: tab.array_col AS unnest(tab.array_col) // TableScan: tab // ``` - let mut intermediate_plan = unwrap_arc(input); + let mut intermediate_plan = Arc::unwrap_or_clone(input); let mut intermediate_select_exprs = group_expr; loop { From 9ef2f95241fc761929b71bdf9148ad221a6f34c2 Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Tue, 27 Aug 2024 03:37:09 +0200 Subject: [PATCH 030/154] Fix typo (#12169) --- datafusion/expr/src/utils.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/expr/src/utils.rs b/datafusion/expr/src/utils.rs index 11a244a944f8..8f04d3ec3067 100644 --- a/datafusion/expr/src/utils.rs +++ b/datafusion/expr/src/utils.rs @@ -463,7 +463,7 @@ pub fn expand_qualified_wildcard( /// if bool is true SortExpr comes from `PARTITION BY` column, if false comes from `ORDER BY` column type WindowSortKey = Vec<(Expr, bool)>; -/// Generate a sort key for a given window expr's partition_by and order_bu expr +/// Generate a sort key for a given window expr's partition_by and order_by expr pub fn generate_sort_key( partition_by: &[Expr], order_by: &[Expr], From 7e9ea3ad59071d56093c197c5ecd5c50021deb94 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Mon, 26 Aug 2024 21:38:43 -0400 Subject: [PATCH 031/154] Minor: remove vestigal github workflow (#12182) --- .github/workflows/pr_comment.yml | 53 -------------------------------- 1 file changed, 53 deletions(-) delete mode 100644 .github/workflows/pr_comment.yml diff --git a/.github/workflows/pr_comment.yml b/.github/workflows/pr_comment.yml deleted file mode 100644 index 8b6df1c75687..000000000000 --- a/.github/workflows/pr_comment.yml +++ /dev/null @@ -1,53 +0,0 @@ -# Downloads any `message` artifacts created by other jobs -# and posts them as comments to the PR -name: PR Comment - -on: - workflow_run: - workflows: ["Benchmarks"] - types: - - completed - -jobs: - comment: - name: PR Comment - runs-on: ubuntu-latest - if: github.event.workflow_run.conclusion == 'success' - steps: - - name: Dump GitHub context - env: - GITHUB_CONTEXT: ${{ toJSON(github) }} - run: echo "$GITHUB_CONTEXT" - - - name: Download comment message - uses: actions/download-artifact@v4 - with: - name: message - run-id: ${{ github.event.workflow_run.id }} - github-token: ${{ secrets.GITHUB_TOKEN }} - - - name: Download pr number - uses: actions/download-artifact@v4 - with: - name: pr - run-id: ${{ github.event.workflow_run.id }} - github-token: ${{ secrets.GITHUB_TOKEN }} - - - name: Print message and pr number - run: | - cat pr - echo "PR_NUMBER=$(cat pr)" >> "$GITHUB_ENV" - cat message.md - - - name: Post the comment - uses: actions/github-script@v7 - with: - script: | - const fs = require('fs'); - const content = fs.readFileSync('message.md', 'utf8'); - github.rest.issues.createComment({ - issue_number: process.env.PR_NUMBER, - owner: context.repo.owner, - repo: context.repo.repo, - body: content, - }) From 2ac0842954a1ee5605be54810ed668e63379dfdd Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=BC=A0=E6=9E=97=E4=BC=9F?= Date: Tue, 27 Aug 2024 21:09:58 +0800 Subject: [PATCH 032/154] Remove `AggregateExpr` trait (#12096) * save * Move AggregateFunctionExpr into physical-expr crate * Move AggregateExpr trait into physical-expr crate * Remove AggregateExpr trait * Fix doc and tests * Remove empty crate physical-expr-functions-aggregate * Use func name instead of expr name * Fix tests * Fix tests and clippy lints * Cargo update for datafusion-cli * Add docs * Add functions back * Fix doc --- Cargo.toml | 2 - datafusion-cli/Cargo.lock | 160 ++++++---- datafusion/core/Cargo.toml | 1 - datafusion/core/src/lib.rs | 6 - .../combine_partial_final_agg.rs | 13 +- .../physical_optimizer/update_aggr_exprs.rs | 9 +- datafusion/core/src/physical_planner.rs | 10 +- datafusion/core/src/test_util/mod.rs | 8 +- .../core/tests/fuzz_cases/aggregate_fuzz.rs | 2 +- .../src/aggregate.rs | 167 ---------- .../functions-aggregate-common/src/utils.rs | 21 +- .../Cargo.toml | 48 --- .../src/lib.rs | 20 -- .../src/aggregate.rs | 294 ++++++++++++------ datafusion/physical-expr/src/lib.rs | 25 +- .../physical-expr/src/window/aggregate.rs | 11 +- .../src/window/sliding_aggregate.rs | 13 +- .../src/aggregate_statistics.rs | 38 +-- datafusion/physical-plan/Cargo.toml | 1 - .../physical-plan/src/aggregates/mod.rs | 110 +++---- .../physical-plan/src/aggregates/row_hash.rs | 9 +- .../physical-plan/src/execution_plan.rs | 2 +- datafusion/physical-plan/src/lib.rs | 4 +- datafusion/physical-plan/src/windows/mod.rs | 7 +- datafusion/proto/src/physical_plan/mod.rs | 7 +- .../proto/src/physical_plan/to_proto.rs | 72 ++--- .../tests/cases/roundtrip_physical_plan.rs | 20 +- 27 files changed, 442 insertions(+), 638 deletions(-) delete mode 100644 datafusion/physical-expr-functions-aggregate/Cargo.toml delete mode 100644 datafusion/physical-expr-functions-aggregate/src/lib.rs rename datafusion/{physical-expr-functions-aggregate => physical-expr}/src/aggregate.rs (69%) diff --git a/Cargo.toml b/Cargo.toml index 124747999041..479d2cadc65b 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -33,7 +33,6 @@ members = [ "datafusion/optimizer", "datafusion/physical-expr", "datafusion/physical-expr-common", - "datafusion/physical-expr-functions-aggregate", "datafusion/physical-optimizer", "datafusion/physical-plan", "datafusion/proto", @@ -106,7 +105,6 @@ datafusion-functions-window = { path = "datafusion/functions-window", version = datafusion-optimizer = { path = "datafusion/optimizer", version = "41.0.0", default-features = false } datafusion-physical-expr = { path = "datafusion/physical-expr", version = "41.0.0", default-features = false } datafusion-physical-expr-common = { path = "datafusion/physical-expr-common", version = "41.0.0", default-features = false } -datafusion-physical-expr-functions-aggregate = { path = "datafusion/physical-expr-functions-aggregate", version = "41.0.0" } datafusion-physical-optimizer = { path = "datafusion/physical-optimizer", version = "41.0.0" } datafusion-physical-plan = { path = "datafusion/physical-plan", version = "41.0.0" } datafusion-proto = { path = "datafusion/proto", version = "41.0.0" } diff --git a/datafusion-cli/Cargo.lock b/datafusion-cli/Cargo.lock index 50333d17ca8d..1e89bb3af87e 100644 --- a/datafusion-cli/Cargo.lock +++ b/datafusion-cli/Cargo.lock @@ -17,6 +17,12 @@ version = "1.0.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "f26201604c87b1e01bd3d98f8d5d9a8fcbb815e8cedb41ffccbeb4bf593a35fe" +[[package]] +name = "adler2" +version = "2.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "512761e0bb2578dd7380c6baaa0f4ce03e84f95e960231d1dec8bf4d7d6e2627" + [[package]] name = "adler32" version = "1.2.0" @@ -167,9 +173,9 @@ checksum = "9d151e35f61089500b617991b791fc8bfd237ae50cd5950803758a179b41e67a" [[package]] name = "arrayvec" -version = "0.7.4" +version = "0.7.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "96d30a06541fbafbc7f82ed10c06164cfbd2c401138f6addd8404629c4b16711" +checksum = "7c02d123df017efcdfbd739ef81735b36c5ba83ec3c59c80a9d7ecc718f92e50" [[package]] name = "arrow" @@ -430,7 +436,7 @@ checksum = "6e0c28dcc82d7c8ead5cb13beb15405b57b8546e93215673ff8ca0349a028107" dependencies = [ "proc-macro2", "quote", - "syn 2.0.74", + "syn 2.0.75", ] [[package]] @@ -765,7 +771,7 @@ dependencies = [ "cc", "cfg-if", "libc", - "miniz_oxide", + "miniz_oxide 0.7.4", "object", "rustc-demangle", ] @@ -815,9 +821,9 @@ dependencies = [ [[package]] name = "blake3" -version = "1.5.3" +version = "1.5.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e9ec96fe9a81b5e365f9db71fe00edc4fe4ca2cc7dcb7861f0603012a7caa210" +checksum = "d82033247fd8e890df8f740e407ad4d038debb9eb1f40533fffb32e7d17dc6f7" dependencies = [ "arrayref", "arrayvec", @@ -999,7 +1005,7 @@ dependencies = [ "heck 0.5.0", "proc-macro2", "quote", - "syn 2.0.74", + "syn 2.0.75", ] [[package]] @@ -1155,7 +1161,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "edb49164822f3ee45b17acd4a208cfc1251410cf0cad9a833234c9890774dd9f" dependencies = [ "quote", - "syn 2.0.74", + "syn 2.0.75", ] [[package]] @@ -1206,7 +1212,6 @@ dependencies = [ "datafusion-optimizer", "datafusion-physical-expr", "datafusion-physical-expr-common", - "datafusion-physical-expr-functions-aggregate", "datafusion-physical-optimizer", "datafusion-physical-plan", "datafusion-sql", @@ -1501,20 +1506,6 @@ dependencies = [ "rand", ] -[[package]] -name = "datafusion-physical-expr-functions-aggregate" -version = "41.0.0" -dependencies = [ - "ahash", - "arrow", - "datafusion-common", - "datafusion-expr", - "datafusion-expr-common", - "datafusion-functions-aggregate-common", - "datafusion-physical-expr-common", - "rand", -] - [[package]] name = "datafusion-physical-optimizer" version = "41.0.0" @@ -1546,7 +1537,6 @@ dependencies = [ "datafusion-functions-aggregate-common", "datafusion-physical-expr", "datafusion-physical-expr-common", - "datafusion-physical-expr-functions-aggregate", "futures", "half", "hashbrown", @@ -1743,12 +1733,12 @@ dependencies = [ [[package]] name = "flate2" -version = "1.0.31" +version = "1.0.32" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7f211bbe8e69bbd0cfdea405084f128ae8b4aaa6b0b522fc8f2b009084797920" +checksum = "9c0596c1eac1f9e04ed902702e9878208b336edc9d6fddc8a48387349bab3666" dependencies = [ "crc32fast", - "miniz_oxide", + "miniz_oxide 0.8.0", ] [[package]] @@ -1831,7 +1821,7 @@ checksum = "87750cf4b7a4c0625b1529e4c543c2182106e4dedc60a2a6455e00d212c489ac" dependencies = [ "proc-macro2", "quote", - "syn 2.0.74", + "syn 2.0.75", ] [[package]] @@ -1924,9 +1914,9 @@ dependencies = [ [[package]] name = "h2" -version = "0.4.5" +version = "0.4.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fa82e28a107a8cc405f0839610bdc9b15f1e25ec7d696aa5cf173edbcb1486ab" +checksum = "524e8ac6999421f49a846c2d4411f337e53497d8ec55d67753beffa43c5d9205" dependencies = [ "atomic-waker", "bytes", @@ -2111,7 +2101,7 @@ dependencies = [ "bytes", "futures-channel", "futures-util", - "h2 0.4.5", + "h2 0.4.6", "http 1.1.0", "http-body 1.0.1", "httparse", @@ -2148,7 +2138,7 @@ dependencies = [ "hyper 1.4.1", "hyper-util", "rustls 0.23.12", - "rustls-native-certs 0.7.1", + "rustls-native-certs 0.7.2", "rustls-pki-types", "tokio", "tokio-rustls 0.26.0", @@ -2353,9 +2343,9 @@ dependencies = [ [[package]] name = "libc" -version = "0.2.156" +version = "0.2.158" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a5f43f184355eefb8d17fc948dbecf6c13be3c141f20d834ae842193a448c72a" +checksum = "d8adc4bb1803a324070e64a98ae98f38934d91957a99cfb3a43dcbc01bc56439" [[package]] name = "libflate" @@ -2489,6 +2479,15 @@ dependencies = [ "adler", ] +[[package]] +name = "miniz_oxide" +version = "0.8.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e2d80299ef12ff69b16a84bb182e3b9df68b5a91574d3d4fa6e41b65deec4df1" +dependencies = [ + "adler2", +] + [[package]] name = "mio" version = "1.0.2" @@ -2829,7 +2828,7 @@ checksum = "2f38a4412a78282e09a2cf38d195ea5420d15ba0602cb375210efbc877243965" dependencies = [ "proc-macro2", "quote", - "syn 2.0.74", + "syn 2.0.75", ] [[package]] @@ -3028,9 +3027,9 @@ dependencies = [ [[package]] name = "redox_users" -version = "0.4.5" +version = "0.4.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bd283d9651eeda4b2a83a43c1c91b266c40fd76ecd39a50a8c630ae69dc72891" +checksum = "ba009ff324d1fc1b900bd1fdb31564febe58a8ccc8a6fdbb93b543d33b13ca43" dependencies = [ "getrandom", "libredox", @@ -3074,15 +3073,15 @@ checksum = "7a66a03ae7c801facd77a29370b4faec201768915ac14a721ba36f20bc9c209b" [[package]] name = "reqwest" -version = "0.12.5" +version = "0.12.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c7d6d2a27d57148378eb5e111173f4276ad26340ecc5c49a4a2152167a2d6a37" +checksum = "f8f4955649ef5c38cc7f9e8aa41761d48fb9677197daea9984dc54f56aad5e63" dependencies = [ "base64 0.22.1", "bytes", "futures-core", "futures-util", - "h2 0.4.5", + "h2 0.4.6", "http 1.1.0", "http-body 1.0.1", "http-body-util", @@ -3098,7 +3097,7 @@ dependencies = [ "pin-project-lite", "quinn", "rustls 0.23.12", - "rustls-native-certs 0.7.1", + "rustls-native-certs 0.7.2", "rustls-pemfile 2.1.3", "rustls-pki-types", "serde", @@ -3114,7 +3113,7 @@ dependencies = [ "wasm-bindgen-futures", "wasm-streams", "web-sys", - "winreg", + "windows-registry", ] [[package]] @@ -3253,9 +3252,9 @@ dependencies = [ [[package]] name = "rustls-native-certs" -version = "0.7.1" +version = "0.7.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a88d6d420651b496bdd98684116959239430022a115c1240e6c3993be0b15fba" +checksum = "04182dffc9091a404e0fc069ea5cd60e5b866c3adf881eff99a32d048242dffa" dependencies = [ "openssl-probe", "rustls-pemfile 2.1.3", @@ -3421,7 +3420,7 @@ checksum = "24008e81ff7613ed8e5ba0cfaf24e2c2f1e5b8a0495711e44fcd4882fca62bcf" dependencies = [ "proc-macro2", "quote", - "syn 2.0.74", + "syn 2.0.75", ] [[package]] @@ -3563,7 +3562,7 @@ checksum = "01b2e185515564f15375f593fb966b5718bc624ba77fe49fa4616ad619690554" dependencies = [ "proc-macro2", "quote", - "syn 2.0.74", + "syn 2.0.75", ] [[package]] @@ -3609,7 +3608,7 @@ dependencies = [ "proc-macro2", "quote", "rustversion", - "syn 2.0.74", + "syn 2.0.75", ] [[package]] @@ -3622,7 +3621,7 @@ dependencies = [ "proc-macro2", "quote", "rustversion", - "syn 2.0.74", + "syn 2.0.75", ] [[package]] @@ -3644,9 +3643,9 @@ dependencies = [ [[package]] name = "syn" -version = "2.0.74" +version = "2.0.75" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1fceb41e3d546d0bd83421d3409b1460cc7444cd389341a4c880fe7a042cb3d7" +checksum = "f6af063034fc1935ede7be0122941bafa9bacb949334d090b77ca98b5817c7d9" dependencies = [ "proc-macro2", "quote", @@ -3658,6 +3657,9 @@ name = "sync_wrapper" version = "1.0.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "a7065abeca94b6a8a577f9bd45aa0867a2238b74e8eb67cf10d492bc39351394" +dependencies = [ + "futures-core", +] [[package]] name = "tempfile" @@ -3704,7 +3706,7 @@ checksum = "a4558b58466b9ad7ca0f102865eccc95938dca1a74a856f2b57b6629050da261" dependencies = [ "proc-macro2", "quote", - "syn 2.0.74", + "syn 2.0.75", ] [[package]] @@ -3774,9 +3776,9 @@ checksum = "1f3ccbac311fea05f86f61904b462b55fb3df8837a366dfc601a0161d0532f20" [[package]] name = "tokio" -version = "1.39.2" +version = "1.39.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "daa4fb1bc778bd6f04cbfc4bb2d06a7396a8f299dc33ea1900cedaa316f467b1" +checksum = "9babc99b9923bfa4804bd74722ff02c0381021eafa4db9949217e3be8e84fff5" dependencies = [ "backtrace", "bytes", @@ -3798,7 +3800,7 @@ checksum = "693d596312e88961bc67d7f1f97af8a70227d9f90c31bba5806eec004978d752" dependencies = [ "proc-macro2", "quote", - "syn 2.0.74", + "syn 2.0.75", ] [[package]] @@ -3895,7 +3897,7 @@ checksum = "34704c8d6ebcbc939824180af020566b01a7c01f80641264eba0999f6c2b6be7" dependencies = [ "proc-macro2", "quote", - "syn 2.0.74", + "syn 2.0.75", ] [[package]] @@ -3940,7 +3942,7 @@ checksum = "f03ca4cb38206e2bef0700092660bb74d696f808514dae47fa1467cbfe26e96e" dependencies = [ "proc-macro2", "quote", - "syn 2.0.74", + "syn 2.0.75", ] [[package]] @@ -4095,7 +4097,7 @@ dependencies = [ "once_cell", "proc-macro2", "quote", - "syn 2.0.74", + "syn 2.0.75", "wasm-bindgen-shared", ] @@ -4129,7 +4131,7 @@ checksum = "afc340c74d9005395cf9dd098506f7f44e38f2b4a21c6aaacf9a105ea5e1e836" dependencies = [ "proc-macro2", "quote", - "syn 2.0.74", + "syn 2.0.75", "wasm-bindgen-backend", "wasm-bindgen-shared", ] @@ -4213,6 +4215,36 @@ dependencies = [ "windows-targets 0.52.6", ] +[[package]] +name = "windows-registry" +version = "0.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e400001bb720a623c1c69032f8e3e4cf09984deec740f007dd2b03ec864804b0" +dependencies = [ + "windows-result", + "windows-strings", + "windows-targets 0.52.6", +] + +[[package]] +name = "windows-result" +version = "0.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1d1043d8214f791817bab27572aaa8af63732e11bf84aa21a45a78d6c317ae0e" +dependencies = [ + "windows-targets 0.52.6", +] + +[[package]] +name = "windows-strings" +version = "0.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4cd9b125c486025df0eabcb585e62173c6c9eddcec5d117d3b6e8c30e2ee4d10" +dependencies = [ + "windows-result", + "windows-targets 0.52.6", +] + [[package]] name = "windows-sys" version = "0.48.0" @@ -4361,16 +4393,6 @@ version = "0.52.6" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "589f6da84c646204747d1270a2a5661ea66ed1cced2631d546fdfb155959f9ec" -[[package]] -name = "winreg" -version = "0.52.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a277a57398d4bfa075df44f501a17cfdf8542d224f0d36095a2adc7aee4ef0a5" -dependencies = [ - "cfg-if", - "windows-sys 0.48.0", -] - [[package]] name = "xmlparser" version = "0.13.6" @@ -4404,7 +4426,7 @@ checksum = "fa4f8080344d4671fb4e831a13ad1e68092748387dfc4f55e356242fae12ce3e" dependencies = [ "proc-macro2", "quote", - "syn 2.0.74", + "syn 2.0.75", ] [[package]] diff --git a/datafusion/core/Cargo.toml b/datafusion/core/Cargo.toml index adbba3eb31d6..de228e058096 100644 --- a/datafusion/core/Cargo.toml +++ b/datafusion/core/Cargo.toml @@ -110,7 +110,6 @@ datafusion-functions-window = { workspace = true } datafusion-optimizer = { workspace = true } datafusion-physical-expr = { workspace = true } datafusion-physical-expr-common = { workspace = true } -datafusion-physical-expr-functions-aggregate = { workspace = true } datafusion-physical-optimizer = { workspace = true } datafusion-physical-plan = { workspace = true } datafusion-sql = { workspace = true } diff --git a/datafusion/core/src/lib.rs b/datafusion/core/src/lib.rs index 735a381586ad..67f3cb01c0a4 100644 --- a/datafusion/core/src/lib.rs +++ b/datafusion/core/src/lib.rs @@ -490,7 +490,6 @@ //! [`PhysicalOptimizerRule`]: datafusion::physical_optimizer::optimizer::PhysicalOptimizerRule //! [`Schema`]: arrow::datatypes::Schema //! [`PhysicalExpr`]: physical_plan::PhysicalExpr -//! [`AggregateExpr`]: physical_plan::AggregateExpr //! [`RecordBatch`]: arrow::record_batch::RecordBatch //! [`RecordBatchReader`]: arrow::record_batch::RecordBatchReader //! [`Array`]: arrow::array::Array @@ -556,11 +555,6 @@ pub mod physical_expr_common { pub use datafusion_physical_expr_common::*; } -/// re-export of [`datafusion_physical_expr_functions_aggregate`] crate -pub mod physical_expr_functions_aggregate { - pub use datafusion_physical_expr_functions_aggregate::*; -} - /// re-export of [`datafusion_physical_expr`] crate pub mod physical_expr { pub use datafusion_physical_expr::*; 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 8cbb187f7bd2..1a12fc7de888 100644 --- a/datafusion/core/src/physical_optimizer/combine_partial_final_agg.rs +++ b/datafusion/core/src/physical_optimizer/combine_partial_final_agg.rs @@ -26,7 +26,8 @@ use crate::physical_plan::ExecutionPlan; use datafusion_common::config::ConfigOptions; use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; -use datafusion_physical_expr::{physical_exprs_equal, AggregateExpr, PhysicalExpr}; +use datafusion_physical_expr::aggregate::AggregateFunctionExpr; +use datafusion_physical_expr::{physical_exprs_equal, PhysicalExpr}; use datafusion_physical_optimizer::PhysicalOptimizerRule; /// CombinePartialFinalAggregate optimizer rule combines the adjacent Partial and Final AggregateExecs @@ -122,7 +123,7 @@ impl PhysicalOptimizerRule for CombinePartialFinalAggregate { type GroupExprsRef<'a> = ( &'a PhysicalGroupBy, - &'a [Arc], + &'a [Arc], &'a [Option>], ); @@ -171,8 +172,8 @@ mod tests { use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; use datafusion_functions_aggregate::count::count_udaf; use datafusion_functions_aggregate::sum::sum_udaf; + use datafusion_physical_expr::aggregate::AggregateExprBuilder; use datafusion_physical_expr::expressions::col; - use datafusion_physical_expr_functions_aggregate::aggregate::AggregateExprBuilder; /// Runs the CombinePartialFinalAggregate optimizer and asserts the plan against the expected macro_rules! assert_optimized { @@ -224,7 +225,7 @@ mod tests { fn partial_aggregate_exec( input: Arc, group_by: PhysicalGroupBy, - aggr_expr: Vec>, + aggr_expr: Vec>, ) -> Arc { let schema = input.schema(); let n_aggr = aggr_expr.len(); @@ -244,7 +245,7 @@ mod tests { fn final_aggregate_exec( input: Arc, group_by: PhysicalGroupBy, - aggr_expr: Vec>, + aggr_expr: Vec>, ) -> Arc { let schema = input.schema(); let n_aggr = aggr_expr.len(); @@ -272,7 +273,7 @@ mod tests { expr: Arc, name: &str, schema: &Schema, - ) -> Arc { + ) -> Arc { AggregateExprBuilder::new(count_udaf(), vec![expr]) .schema(Arc::new(schema.clone())) .alias(name) diff --git a/datafusion/core/src/physical_optimizer/update_aggr_exprs.rs b/datafusion/core/src/physical_optimizer/update_aggr_exprs.rs index f8edf73e3d2a..a2726d62e9f6 100644 --- a/datafusion/core/src/physical_optimizer/update_aggr_exprs.rs +++ b/datafusion/core/src/physical_optimizer/update_aggr_exprs.rs @@ -23,8 +23,9 @@ use std::sync::Arc; use datafusion_common::config::ConfigOptions; use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; use datafusion_common::{plan_datafusion_err, Result}; +use datafusion_physical_expr::aggregate::AggregateFunctionExpr; use datafusion_physical_expr::{ - reverse_order_bys, AggregateExpr, EquivalenceProperties, PhysicalSortRequirement, + reverse_order_bys, EquivalenceProperties, PhysicalSortRequirement, }; use datafusion_physical_optimizer::PhysicalOptimizerRule; use datafusion_physical_plan::aggregates::concat_slices; @@ -117,7 +118,7 @@ impl PhysicalOptimizerRule for OptimizeAggregateOrder { /// /// # Parameters /// -/// * `aggr_exprs` - A vector of `Arc` representing the +/// * `aggr_exprs` - A vector of `Arc` representing the /// aggregate expressions to be optimized. /// * `prefix_requirement` - An array slice representing the ordering /// requirements preceding the aggregate expressions. @@ -130,10 +131,10 @@ impl PhysicalOptimizerRule for OptimizeAggregateOrder { /// successfully. Any errors occurring during the conversion process are /// passed through. fn try_convert_aggregate_if_better( - aggr_exprs: Vec>, + aggr_exprs: Vec>, prefix_requirement: &[PhysicalSortRequirement], eq_properties: &EquivalenceProperties, -) -> Result>> { +) -> Result>> { aggr_exprs .into_iter() .map(|aggr_expr| { diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index 8d6c5089fa34..9501d3c6bbbb 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -58,8 +58,8 @@ use crate::physical_plan::unnest::UnnestExec; use crate::physical_plan::values::ValuesExec; use crate::physical_plan::windows::{BoundedWindowAggExec, WindowAggExec}; use crate::physical_plan::{ - displayable, windows, AggregateExpr, ExecutionPlan, ExecutionPlanProperties, - InputOrderMode, Partitioning, PhysicalExpr, WindowExpr, + displayable, windows, ExecutionPlan, ExecutionPlanProperties, InputOrderMode, + Partitioning, PhysicalExpr, WindowExpr, }; use arrow::compute::SortOptions; @@ -81,9 +81,9 @@ use datafusion_expr::{ DescribeTable, DmlStatement, Extension, Filter, RecursiveQuery, StringifiedPlan, WindowFrame, WindowFrameBound, WriteOp, }; +use datafusion_physical_expr::aggregate::{AggregateExprBuilder, AggregateFunctionExpr}; use datafusion_physical_expr::expressions::Literal; use datafusion_physical_expr::LexOrdering; -use datafusion_physical_expr_functions_aggregate::aggregate::AggregateExprBuilder; use datafusion_physical_plan::placeholder_row::PlaceholderRowExec; use datafusion_sql::utils::window_expr_common_partition_keys; @@ -719,7 +719,7 @@ impl DefaultPhysicalPlanner { // optimization purposes. For example, a FIRST_VALUE may turn // into a LAST_VALUE with the reverse ordering requirement. // To reflect such changes to subsequent stages, use the updated - // `AggregateExpr`/`PhysicalSortExpr` objects. + // `AggregateFunctionExpr`/`PhysicalSortExpr` objects. let updated_aggregates = initial_aggr.aggr_expr().to_vec(); let next_partition_mode = if can_repartition { @@ -1541,7 +1541,7 @@ pub fn create_window_expr( } type AggregateExprWithOptionalArgs = ( - Arc, + Arc, // The filter clause, if any Option>, // Ordering requirements, if any diff --git a/datafusion/core/src/test_util/mod.rs b/datafusion/core/src/test_util/mod.rs index ca8376fdec0a..faa9378535fd 100644 --- a/datafusion/core/src/test_util/mod.rs +++ b/datafusion/core/src/test_util/mod.rs @@ -48,13 +48,11 @@ use datafusion_common::TableReference; use datafusion_expr::utils::COUNT_STAR_EXPANSION; use datafusion_expr::{CreateExternalTable, Expr, TableType}; use datafusion_functions_aggregate::count::count_udaf; -use datafusion_physical_expr::{ - expressions, AggregateExpr, EquivalenceProperties, PhysicalExpr, -}; +use datafusion_physical_expr::{expressions, EquivalenceProperties, PhysicalExpr}; use async_trait::async_trait; use datafusion_catalog::Session; -use datafusion_physical_expr_functions_aggregate::aggregate::AggregateExprBuilder; +use datafusion_physical_expr::aggregate::{AggregateExprBuilder, AggregateFunctionExpr}; use futures::Stream; use tempfile::TempDir; // backwards compatibility @@ -429,7 +427,7 @@ impl TestAggregate { } /// Return appropriate expr depending if COUNT is for col or table (*) - pub fn count_expr(&self, schema: &Schema) -> Arc { + pub fn count_expr(&self, schema: &Schema) -> Arc { AggregateExprBuilder::new(count_udaf(), vec![self.column()]) .schema(Arc::new(schema.clone())) .alias(self.column_name()) diff --git a/datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs b/datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs index 138e5bda7f39..62e9be63983c 100644 --- a/datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs @@ -25,7 +25,7 @@ use arrow::util::pretty::pretty_format_batches; use arrow_array::types::Int64Type; use datafusion::common::Result; use datafusion::datasource::MemTable; -use datafusion::physical_expr_functions_aggregate::aggregate::AggregateExprBuilder; +use datafusion::physical_expr::aggregate::AggregateExprBuilder; use datafusion::physical_plan::aggregates::{ AggregateExec, AggregateMode, PhysicalGroupBy, }; diff --git a/datafusion/functions-aggregate-common/src/aggregate.rs b/datafusion/functions-aggregate-common/src/aggregate.rs index 698d1350cb61..c9cbaa8396fc 100644 --- a/datafusion/functions-aggregate-common/src/aggregate.rs +++ b/datafusion/functions-aggregate-common/src/aggregate.rs @@ -15,172 +15,5 @@ // specific language governing permissions and limitations // under the License. -//! [`AggregateExpr`] which defines the interface all aggregate expressions -//! (built-in and custom) need to satisfy. - -use crate::order::AggregateOrderSensitivity; -use arrow::datatypes::{DataType, Field}; -use datafusion_common::{exec_err, not_impl_err, Result, ScalarValue}; -use datafusion_expr_common::accumulator::Accumulator; -use datafusion_expr_common::groups_accumulator::GroupsAccumulator; -use datafusion_physical_expr_common::physical_expr::PhysicalExpr; -use datafusion_physical_expr_common::sort_expr::PhysicalSortExpr; -use std::fmt::Debug; -use std::{any::Any, sync::Arc}; - pub mod count_distinct; pub mod groups_accumulator; - -/// An aggregate expression that: -/// * knows its resulting field -/// * knows how to create its accumulator -/// * knows its accumulator's state's field -/// * knows the expressions from whose its accumulator will receive values -/// -/// Any implementation of this trait also needs to implement the -/// `PartialEq` to allows comparing equality between the -/// trait objects. -pub trait AggregateExpr: Send + Sync + Debug + PartialEq { - /// Returns the aggregate expression as [`Any`] so that it can be - /// downcast to a specific implementation. - fn as_any(&self) -> &dyn Any; - - /// the field of the final result of this aggregation. - fn field(&self) -> Result; - - /// the accumulator used to accumulate values from the expressions. - /// the accumulator expects the same number of arguments as `expressions` and must - /// return states with the same description as `state_fields` - fn create_accumulator(&self) -> Result>; - - /// the fields that encapsulate the Accumulator's state - /// the number of fields here equals the number of states that the accumulator contains - fn state_fields(&self) -> Result>; - - /// expressions that are passed to the Accumulator. - /// Single-column aggregations such as `sum` return a single value, others (e.g. `cov`) return many. - fn expressions(&self) -> Vec>; - - /// Order by requirements for the aggregate function - /// By default it is `None` (there is no requirement) - /// Order-sensitive aggregators, such as `FIRST_VALUE(x ORDER BY y)` should implement this - fn order_bys(&self) -> Option<&[PhysicalSortExpr]> { - None - } - - /// Indicates whether aggregator can produce the correct result with any - /// arbitrary input ordering. By default, we assume that aggregate expressions - /// are order insensitive. - fn order_sensitivity(&self) -> AggregateOrderSensitivity { - AggregateOrderSensitivity::Insensitive - } - - /// Sets the indicator whether ordering requirements of the aggregator is - /// satisfied by its input. If this is not the case, aggregators with order - /// sensitivity `AggregateOrderSensitivity::Beneficial` can still produce - /// the correct result with possibly more work internally. - /// - /// # Returns - /// - /// Returns `Ok(Some(updated_expr))` if the process completes successfully. - /// If the expression can benefit from existing input ordering, but does - /// not implement the method, returns an error. Order insensitive and hard - /// requirement aggregators return `Ok(None)`. - fn with_beneficial_ordering( - self: Arc, - _requirement_satisfied: bool, - ) -> Result>> { - if self.order_bys().is_some() && self.order_sensitivity().is_beneficial() { - return exec_err!( - "Should implement with satisfied for aggregator :{:?}", - self.name() - ); - } - Ok(None) - } - - /// Human readable name such as `"MIN(c2)"`. The default - /// implementation returns placeholder text. - fn name(&self) -> &str { - "AggregateExpr: default name" - } - - /// If the aggregate expression has a specialized - /// [`GroupsAccumulator`] implementation. If this returns true, - /// `[Self::create_groups_accumulator`] will be called. - fn groups_accumulator_supported(&self) -> bool { - false - } - - /// Return a specialized [`GroupsAccumulator`] that manages state - /// for all groups. - /// - /// For maximum performance, a [`GroupsAccumulator`] should be - /// implemented in addition to [`Accumulator`]. - fn create_groups_accumulator(&self) -> Result> { - not_impl_err!("GroupsAccumulator hasn't been implemented for {self:?} yet") - } - - /// Construct an expression that calculates the aggregate in reverse. - /// Typically the "reverse" expression is itself (e.g. SUM, COUNT). - /// For aggregates that do not support calculation in reverse, - /// returns None (which is the default value). - fn reverse_expr(&self) -> Option> { - None - } - - /// Creates accumulator implementation that supports retract - fn create_sliding_accumulator(&self) -> Result> { - not_impl_err!("Retractable Accumulator hasn't been implemented for {self:?} yet") - } - - /// Returns all expressions used in the [`AggregateExpr`]. - /// These expressions are (1)function arguments, (2) order by expressions. - fn all_expressions(&self) -> AggregatePhysicalExpressions { - let args = self.expressions(); - let order_bys = self.order_bys().unwrap_or(&[]); - let order_by_exprs = order_bys - .iter() - .map(|sort_expr| Arc::clone(&sort_expr.expr)) - .collect::>(); - AggregatePhysicalExpressions { - args, - order_by_exprs, - } - } - - /// Rewrites [`AggregateExpr`], with new expressions given. The argument should be consistent - /// with the return value of the [`AggregateExpr::all_expressions`] method. - /// Returns `Some(Arc)` if re-write is supported, otherwise returns `None`. - fn with_new_expressions( - &self, - _args: Vec>, - _order_by_exprs: Vec>, - ) -> Option> { - None - } - - /// If this function is max, return (output_field, true) - /// if the function is min, return (output_field, false) - /// otherwise return None (the default) - /// - /// output_field is the name of the column produced by this aggregate - /// - /// Note: this is used to use special aggregate implementations in certain conditions - fn get_minmax_desc(&self) -> Option<(Field, bool)> { - None - } - - /// Returns default value of the function given the input is Null - /// Most of the aggregate function return Null if input is Null, - /// while `count` returns 0 if input is Null - fn default_value(&self, data_type: &DataType) -> Result; -} - -/// Stores the physical expressions used inside the `AggregateExpr`. -pub struct AggregatePhysicalExpressions { - /// Aggregate function arguments - pub args: Vec>, - /// Order by expressions - pub order_by_exprs: Vec>, -} diff --git a/datafusion/functions-aggregate-common/src/utils.rs b/datafusion/functions-aggregate-common/src/utils.rs index 7b8ce0397af8..4fba772d8ddc 100644 --- a/datafusion/functions-aggregate-common/src/utils.rs +++ b/datafusion/functions-aggregate-common/src/utils.rs @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -use std::{any::Any, sync::Arc}; +use std::sync::Arc; use arrow::array::{ArrayRef, AsArray}; use arrow::datatypes::ArrowNativeType; @@ -32,25 +32,6 @@ use datafusion_common::{exec_err, DataFusionError, Result}; use datafusion_expr_common::accumulator::Accumulator; use datafusion_physical_expr_common::sort_expr::PhysicalSortExpr; -use crate::aggregate::AggregateExpr; - -/// Downcast a `Box` or `Arc` -/// and return the inner trait object as [`Any`] so -/// that it can be downcast to a specific implementation. -/// -/// This method is used when implementing the `PartialEq` -/// for [`AggregateExpr`] aggregation expressions and allows comparing the equality -/// between the trait objects. -pub fn down_cast_any_ref(any: &dyn Any) -> &dyn Any { - if let Some(obj) = any.downcast_ref::>() { - obj.as_any() - } else if let Some(obj) = any.downcast_ref::>() { - obj.as_any() - } else { - any - } -} - /// Convert scalar values from an accumulator into arrays. pub fn get_accum_scalar_values_as_arrays( accum: &mut dyn Accumulator, diff --git a/datafusion/physical-expr-functions-aggregate/Cargo.toml b/datafusion/physical-expr-functions-aggregate/Cargo.toml deleted file mode 100644 index 6eed89614c53..000000000000 --- a/datafusion/physical-expr-functions-aggregate/Cargo.toml +++ /dev/null @@ -1,48 +0,0 @@ -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. - -[package] -name = "datafusion-physical-expr-functions-aggregate" -description = "Logical plan and expression representation for DataFusion query engine" -keywords = ["datafusion", "logical", "plan", "expressions"] -readme = "README.md" -version = { workspace = true } -edition = { workspace = true } -homepage = { workspace = true } -repository = { workspace = true } -license = { workspace = true } -authors = { workspace = true } -rust-version = { workspace = true } - -[lints] -workspace = true - -[lib] -name = "datafusion_physical_expr_functions_aggregate" -path = "src/lib.rs" - -[features] - -[dependencies] -ahash = { workspace = true } -arrow = { workspace = true } -datafusion-common = { workspace = true } -datafusion-expr = { workspace = true } -datafusion-expr-common = { workspace = true } -datafusion-functions-aggregate-common = { workspace = true } -datafusion-physical-expr-common = { workspace = true } -rand = { workspace = true } diff --git a/datafusion/physical-expr-functions-aggregate/src/lib.rs b/datafusion/physical-expr-functions-aggregate/src/lib.rs deleted file mode 100644 index 2ff7ff5777ec..000000000000 --- a/datafusion/physical-expr-functions-aggregate/src/lib.rs +++ /dev/null @@ -1,20 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -//! Technically, all aggregate functions that depend on `expr` crate should be included here. - -pub mod aggregate; diff --git a/datafusion/physical-expr-functions-aggregate/src/aggregate.rs b/datafusion/physical-expr/src/aggregate.rs similarity index 69% rename from datafusion/physical-expr-functions-aggregate/src/aggregate.rs rename to datafusion/physical-expr/src/aggregate.rs index fd986e00a7ef..5c1216f2a386 100644 --- a/datafusion/physical-expr-functions-aggregate/src/aggregate.rs +++ b/datafusion/physical-expr/src/aggregate.rs @@ -15,29 +15,46 @@ // specific language governing permissions and limitations // under the License. +pub(crate) mod groups_accumulator { + #[allow(unused_imports)] + pub(crate) mod accumulate { + pub use datafusion_functions_aggregate_common::aggregate::groups_accumulator::accumulate::NullState; + } + pub use datafusion_functions_aggregate_common::aggregate::groups_accumulator::{ + accumulate::NullState, GroupsAccumulatorAdapter, + }; +} +pub(crate) mod stats { + pub use datafusion_functions_aggregate_common::stats::StatsType; +} +pub mod utils { + pub use datafusion_functions_aggregate_common::utils::{ + adjust_output_array, get_accum_scalar_values_as_arrays, get_sort_options, + ordering_fields, DecimalAverager, Hashable, + }; +} + use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; use datafusion_common::ScalarValue; use datafusion_common::{internal_err, not_impl_err, Result}; use datafusion_expr::AggregateUDF; use datafusion_expr::ReversedUDAF; use datafusion_expr_common::accumulator::Accumulator; -use datafusion_expr_common::groups_accumulator::GroupsAccumulator; use datafusion_expr_common::type_coercion::aggregates::check_arg_count; use datafusion_functions_aggregate_common::accumulator::AccumulatorArgs; use datafusion_functions_aggregate_common::accumulator::StateFieldsArgs; -use datafusion_functions_aggregate_common::aggregate::AggregateExpr; use datafusion_functions_aggregate_common::order::AggregateOrderSensitivity; -use datafusion_functions_aggregate_common::utils::{self, down_cast_any_ref}; use datafusion_physical_expr_common::physical_expr::PhysicalExpr; use datafusion_physical_expr_common::sort_expr::{LexOrdering, PhysicalSortExpr}; use datafusion_physical_expr_common::utils::reverse_order_bys; +use datafusion_expr_common::groups_accumulator::GroupsAccumulator; use std::fmt::Debug; -use std::{any::Any, sync::Arc}; +use std::sync::Arc; -/// Builder for physical [`AggregateExpr`] +/// Builder for physical [`AggregateFunctionExpr`] /// -/// `AggregateExpr` contains the information necessary to call +/// `AggregateFunctionExpr` contains the information necessary to call /// an aggregate expression. #[derive(Debug, Clone)] pub struct AggregateExprBuilder { @@ -71,7 +88,7 @@ impl AggregateExprBuilder { } } - pub fn build(self) -> Result> { + pub fn build(self) -> Result> { let Self { fun, args, @@ -204,6 +221,17 @@ impl AggregateFunctionExpr { &self.fun } + /// expressions that are passed to the Accumulator. + /// Single-column aggregations such as `sum` return a single value, others (e.g. `cov`) return many. + pub fn expressions(&self) -> Vec> { + self.args.clone() + } + + /// Human readable name such as `"MIN(c2)"`. + pub fn name(&self) -> &str { + &self.name + } + /// Return if the aggregation is distinct pub fn is_distinct(&self) -> bool { self.is_distinct @@ -219,34 +247,13 @@ impl AggregateFunctionExpr { self.is_reversed } + /// Return if the aggregation is nullable pub fn is_nullable(&self) -> bool { self.is_nullable } -} -impl AggregateExpr for AggregateFunctionExpr { - /// Return a reference to Any that can be used for downcasting - fn as_any(&self) -> &dyn Any { - self - } - - fn expressions(&self) -> Vec> { - self.args.clone() - } - - fn state_fields(&self) -> Result> { - let args = StateFieldsArgs { - name: &self.name, - input_types: &self.input_types, - return_type: &self.data_type, - ordering_fields: &self.ordering_fields, - is_distinct: self.is_distinct, - }; - - self.fun.state_fields(args) - } - - fn field(&self) -> Result { + /// the field of the final result of this aggregation. + pub fn field(&self) -> Result { Ok(Field::new( &self.name, self.data_type.clone(), @@ -254,7 +261,10 @@ impl AggregateExpr for AggregateFunctionExpr { )) } - fn create_accumulator(&self) -> Result> { + /// the accumulator used to accumulate values from the expressions. + /// the accumulator expects the same number of arguments as `expressions` and must + /// return states with the same description as `state_fields` + pub fn create_accumulator(&self) -> Result> { let acc_args = AccumulatorArgs { return_type: &self.data_type, schema: &self.schema, @@ -269,7 +279,83 @@ impl AggregateExpr for AggregateFunctionExpr { self.fun.accumulator(acc_args) } - fn create_sliding_accumulator(&self) -> Result> { + /// the field of the final result of this aggregation. + pub fn state_fields(&self) -> Result> { + let args = StateFieldsArgs { + name: &self.name, + input_types: &self.input_types, + return_type: &self.data_type, + ordering_fields: &self.ordering_fields, + is_distinct: self.is_distinct, + }; + + self.fun.state_fields(args) + } + + /// Order by requirements for the aggregate function + /// By default it is `None` (there is no requirement) + /// Order-sensitive aggregators, such as `FIRST_VALUE(x ORDER BY y)` should implement this + pub fn order_bys(&self) -> Option<&[PhysicalSortExpr]> { + if self.ordering_req.is_empty() { + return None; + } + + if !self.order_sensitivity().is_insensitive() { + return Some(&self.ordering_req); + } + + None + } + + /// Indicates whether aggregator can produce the correct result with any + /// arbitrary input ordering. By default, we assume that aggregate expressions + /// are order insensitive. + pub fn order_sensitivity(&self) -> AggregateOrderSensitivity { + if !self.ordering_req.is_empty() { + // If there is requirement, use the sensitivity of the implementation + self.fun.order_sensitivity() + } else { + // If no requirement, aggregator is order insensitive + AggregateOrderSensitivity::Insensitive + } + } + + /// Sets the indicator whether ordering requirements of the aggregator is + /// satisfied by its input. If this is not the case, aggregators with order + /// sensitivity `AggregateOrderSensitivity::Beneficial` can still produce + /// the correct result with possibly more work internally. + /// + /// # Returns + /// + /// Returns `Ok(Some(updated_expr))` if the process completes successfully. + /// If the expression can benefit from existing input ordering, but does + /// not implement the method, returns an error. Order insensitive and hard + /// requirement aggregators return `Ok(None)`. + pub fn with_beneficial_ordering( + self: Arc, + beneficial_ordering: bool, + ) -> Result>> { + let Some(updated_fn) = self + .fun + .clone() + .with_beneficial_ordering(beneficial_ordering)? + else { + return Ok(None); + }; + + AggregateExprBuilder::new(Arc::new(updated_fn), self.args.to_vec()) + .order_by(self.ordering_req.to_vec()) + .schema(Arc::new(self.schema.clone())) + .alias(self.name().to_string()) + .with_ignore_nulls(self.ignore_nulls) + .with_distinct(self.is_distinct) + .with_reversed(self.is_reversed) + .build() + .map(Some) + } + + /// Creates accumulator implementation that supports retract + pub fn create_sliding_accumulator(&self) -> Result> { let args = AccumulatorArgs { return_type: &self.data_type, schema: &self.schema, @@ -335,11 +421,10 @@ impl AggregateExpr for AggregateFunctionExpr { Ok(accumulator) } - fn name(&self) -> &str { - &self.name - } - - fn groups_accumulator_supported(&self) -> bool { + /// If the aggregate expression has a specialized + /// [`GroupsAccumulator`] implementation. If this returns true, + /// `[Self::create_groups_accumulator`] will be called. + pub fn groups_accumulator_supported(&self) -> bool { let args = AccumulatorArgs { return_type: &self.data_type, schema: &self.schema, @@ -353,7 +438,12 @@ impl AggregateExpr for AggregateFunctionExpr { self.fun.groups_accumulator_supported(args) } - fn create_groups_accumulator(&self) -> Result> { + /// Return a specialized [`GroupsAccumulator`] that manages state + /// for all groups. + /// + /// For maximum performance, a [`GroupsAccumulator`] should be + /// implemented in addition to [`Accumulator`]. + pub fn create_groups_accumulator(&self) -> Result> { let args = AccumulatorArgs { return_type: &self.data_type, schema: &self.schema, @@ -367,52 +457,11 @@ impl AggregateExpr for AggregateFunctionExpr { self.fun.create_groups_accumulator(args) } - fn order_bys(&self) -> Option<&[PhysicalSortExpr]> { - if self.ordering_req.is_empty() { - return None; - } - - if !self.order_sensitivity().is_insensitive() { - return Some(&self.ordering_req); - } - - None - } - - fn order_sensitivity(&self) -> AggregateOrderSensitivity { - if !self.ordering_req.is_empty() { - // If there is requirement, use the sensitivity of the implementation - self.fun.order_sensitivity() - } else { - // If no requirement, aggregator is order insensitive - AggregateOrderSensitivity::Insensitive - } - } - - fn with_beneficial_ordering( - self: Arc, - beneficial_ordering: bool, - ) -> Result>> { - let Some(updated_fn) = self - .fun - .clone() - .with_beneficial_ordering(beneficial_ordering)? - else { - return Ok(None); - }; - - AggregateExprBuilder::new(Arc::new(updated_fn), self.args.to_vec()) - .order_by(self.ordering_req.to_vec()) - .schema(Arc::new(self.schema.clone())) - .alias(self.name().to_string()) - .with_ignore_nulls(self.ignore_nulls) - .with_distinct(self.is_distinct) - .with_reversed(self.is_reversed) - .build() - .map(Some) - } - - fn reverse_expr(&self) -> Option> { + /// Construct an expression that calculates the aggregate in reverse. + /// Typically the "reverse" expression is itself (e.g. SUM, COUNT). + /// For aggregates that do not support calculation in reverse, + /// returns None (which is the default value). + pub fn reverse_expr(&self) -> Option> { match self.fun.reverse_udf() { ReversedUDAF::NotSupported => None, ReversedUDAF::Identical => Some(Arc::new(self.clone())), @@ -440,33 +489,72 @@ impl AggregateExpr for AggregateFunctionExpr { } } - fn get_minmax_desc(&self) -> Option<(Field, bool)> { + /// Returns all expressions used in the [`AggregateFunctionExpr`]. + /// These expressions are (1)function arguments, (2) order by expressions. + pub fn all_expressions(&self) -> AggregatePhysicalExpressions { + let args = self.expressions(); + let order_bys = self.order_bys().unwrap_or(&[]); + let order_by_exprs = order_bys + .iter() + .map(|sort_expr| Arc::clone(&sort_expr.expr)) + .collect::>(); + AggregatePhysicalExpressions { + args, + order_by_exprs, + } + } + + /// Rewrites [`AggregateFunctionExpr`], with new expressions given. The argument should be consistent + /// with the return value of the [`AggregateFunctionExpr::all_expressions`] method. + /// Returns `Some(Arc)` if re-write is supported, otherwise returns `None`. + pub fn with_new_expressions( + &self, + _args: Vec>, + _order_by_exprs: Vec>, + ) -> Option> { + None + } + + /// If this function is max, return (output_field, true) + /// if the function is min, return (output_field, false) + /// otherwise return None (the default) + /// + /// output_field is the name of the column produced by this aggregate + /// + /// Note: this is used to use special aggregate implementations in certain conditions + pub fn get_minmax_desc(&self) -> Option<(Field, bool)> { self.fun .is_descending() .and_then(|flag| self.field().ok().map(|f| (f, flag))) } - fn default_value(&self, data_type: &DataType) -> Result { + /// Returns default value of the function given the input is Null + /// Most of the aggregate function return Null if input is Null, + /// while `count` returns 0 if input is Null + pub fn default_value(&self, data_type: &DataType) -> Result { self.fun.default_value(data_type) } } -impl PartialEq for AggregateFunctionExpr { - fn eq(&self, other: &dyn Any) -> bool { - down_cast_any_ref(other) - .downcast_ref::() - .map(|x| { - self.name == x.name - && self.data_type == x.data_type - && self.fun == x.fun - && self.args.len() == x.args.len() - && self - .args - .iter() - .zip(x.args.iter()) - .all(|(this_arg, other_arg)| this_arg.eq(other_arg)) - }) - .unwrap_or(false) +/// Stores the physical expressions used inside the `AggregateExpr`. +pub struct AggregatePhysicalExpressions { + /// Aggregate function arguments + pub args: Vec>, + /// Order by expressions + pub order_by_exprs: Vec>, +} + +impl PartialEq for AggregateFunctionExpr { + fn eq(&self, other: &Self) -> bool { + self.name == other.name + && self.data_type == other.data_type + && self.fun == other.fun + && self.args.len() == other.args.len() + && self + .args + .iter() + .zip(other.args.iter()) + .all(|(this_arg, other_arg)| this_arg.eq(other_arg)) } } diff --git a/datafusion/physical-expr/src/lib.rs b/datafusion/physical-expr/src/lib.rs index c4255172d680..7db7188b85d3 100644 --- a/datafusion/physical-expr/src/lib.rs +++ b/datafusion/physical-expr/src/lib.rs @@ -19,27 +19,7 @@ #![deny(clippy::clone_on_ref_ptr)] // Backward compatibility -pub mod aggregate { - pub(crate) mod groups_accumulator { - #[allow(unused_imports)] - pub(crate) mod accumulate { - pub use datafusion_functions_aggregate_common::aggregate::groups_accumulator::accumulate::NullState; - } - pub use datafusion_functions_aggregate_common::aggregate::groups_accumulator::{ - accumulate::NullState, GroupsAccumulatorAdapter, - }; - } - pub(crate) mod stats { - pub use datafusion_functions_aggregate_common::stats::StatsType; - } - pub mod utils { - pub use datafusion_functions_aggregate_common::utils::{ - adjust_output_array, down_cast_any_ref, get_accum_scalar_values_as_arrays, - get_sort_options, ordering_fields, DecimalAverager, Hashable, - }; - } - pub use datafusion_functions_aggregate_common::aggregate::AggregateExpr; -} +pub mod aggregate; pub mod analysis; pub mod binary_map { pub use datafusion_physical_expr_common::binary_map::{ArrowBytesSet, OutputType}; @@ -67,9 +47,6 @@ pub mod execution_props { pub use aggregate::groups_accumulator::{GroupsAccumulatorAdapter, NullState}; pub use analysis::{analyze, AnalysisContext, ExprBoundaries}; -pub use datafusion_functions_aggregate_common::aggregate::{ - AggregateExpr, AggregatePhysicalExpressions, -}; pub use equivalence::{calculate_union, ConstExpr, EquivalenceProperties}; pub use partitioning::{Distribution, Partitioning}; pub use physical_expr::{ diff --git a/datafusion/physical-expr/src/window/aggregate.rs b/datafusion/physical-expr/src/window/aggregate.rs index 52015f425217..5439e140502a 100644 --- a/datafusion/physical-expr/src/window/aggregate.rs +++ b/datafusion/physical-expr/src/window/aggregate.rs @@ -29,20 +29,19 @@ use datafusion_common::ScalarValue; use datafusion_common::{DataFusionError, Result}; use datafusion_expr::{Accumulator, WindowFrame}; +use crate::aggregate::AggregateFunctionExpr; use crate::window::window_expr::AggregateWindowExpr; use crate::window::{ PartitionBatches, PartitionWindowAggStates, SlidingAggregateWindowExpr, WindowExpr, }; -use crate::{ - expressions::PhysicalSortExpr, reverse_order_bys, AggregateExpr, PhysicalExpr, -}; +use crate::{expressions::PhysicalSortExpr, reverse_order_bys, PhysicalExpr}; /// A window expr that takes the form of an aggregate function. /// /// See comments on [`WindowExpr`] for more details. #[derive(Debug)] pub struct PlainAggregateWindowExpr { - aggregate: Arc, + aggregate: Arc, partition_by: Vec>, order_by: Vec, window_frame: Arc, @@ -51,7 +50,7 @@ pub struct PlainAggregateWindowExpr { impl PlainAggregateWindowExpr { /// Create a new aggregate window function expression pub fn new( - aggregate: Arc, + aggregate: Arc, partition_by: &[Arc], order_by: &[PhysicalSortExpr], window_frame: Arc, @@ -65,7 +64,7 @@ impl PlainAggregateWindowExpr { } /// Get aggregate expr of AggregateWindowExpr - pub fn get_aggregate_expr(&self) -> &Arc { + pub fn get_aggregate_expr(&self) -> &Arc { &self.aggregate } } diff --git a/datafusion/physical-expr/src/window/sliding_aggregate.rs b/datafusion/physical-expr/src/window/sliding_aggregate.rs index afa799e86953..ac3a4f4c09ec 100644 --- a/datafusion/physical-expr/src/window/sliding_aggregate.rs +++ b/datafusion/physical-expr/src/window/sliding_aggregate.rs @@ -28,13 +28,12 @@ use arrow::record_batch::RecordBatch; use datafusion_common::{Result, ScalarValue}; use datafusion_expr::{Accumulator, WindowFrame}; +use crate::aggregate::AggregateFunctionExpr; use crate::window::window_expr::AggregateWindowExpr; use crate::window::{ PartitionBatches, PartitionWindowAggStates, PlainAggregateWindowExpr, WindowExpr, }; -use crate::{ - expressions::PhysicalSortExpr, reverse_order_bys, AggregateExpr, PhysicalExpr, -}; +use crate::{expressions::PhysicalSortExpr, reverse_order_bys, PhysicalExpr}; /// A window expr that takes the form of an aggregate function that /// can be incrementally computed over sliding windows. @@ -42,7 +41,7 @@ use crate::{ /// See comments on [`WindowExpr`] for more details. #[derive(Debug)] pub struct SlidingAggregateWindowExpr { - aggregate: Arc, + aggregate: Arc, partition_by: Vec>, order_by: Vec, window_frame: Arc, @@ -51,7 +50,7 @@ pub struct SlidingAggregateWindowExpr { impl SlidingAggregateWindowExpr { /// Create a new (sliding) aggregate window function expression. pub fn new( - aggregate: Arc, + aggregate: Arc, partition_by: &[Arc], order_by: &[PhysicalSortExpr], window_frame: Arc, @@ -64,8 +63,8 @@ impl SlidingAggregateWindowExpr { } } - /// Get the [AggregateExpr] of this object. - pub fn get_aggregate_expr(&self) -> &Arc { + /// Get the [AggregateFunctionExpr] of this object. + pub fn get_aggregate_expr(&self) -> &Arc { &self.aggregate } } diff --git a/datafusion/physical-optimizer/src/aggregate_statistics.rs b/datafusion/physical-optimizer/src/aggregate_statistics.rs index 66b250c5063b..2b8725b5bac7 100644 --- a/datafusion/physical-optimizer/src/aggregate_statistics.rs +++ b/datafusion/physical-optimizer/src/aggregate_statistics.rs @@ -23,7 +23,7 @@ use datafusion_common::scalar::ScalarValue; use datafusion_common::Result; use datafusion_physical_plan::aggregates::AggregateExec; use datafusion_physical_plan::projection::ProjectionExec; -use datafusion_physical_plan::{expressions, AggregateExpr, ExecutionPlan, Statistics}; +use datafusion_physical_plan::{expressions, ExecutionPlan, Statistics}; use crate::PhysicalOptimizerRule; use datafusion_common::stats::Precision; @@ -58,12 +58,12 @@ impl PhysicalOptimizerRule for AggregateStatistics { let mut projections = vec![]; for expr in partial_agg_exec.aggr_expr() { if let Some((non_null_rows, name)) = - take_optimizable_column_and_table_count(&**expr, &stats) + take_optimizable_column_and_table_count(expr, &stats) { projections.push((expressions::lit(non_null_rows), name.to_owned())); - } else if let Some((min, name)) = take_optimizable_min(&**expr, &stats) { + } else if let Some((min, name)) = take_optimizable_min(expr, &stats) { projections.push((expressions::lit(min), name.to_owned())); - } else if let Some((max, name)) = take_optimizable_max(&**expr, &stats) { + } else if let Some((max, name)) = take_optimizable_max(expr, &stats) { projections.push((expressions::lit(max), name.to_owned())); } else { // TODO: we need all aggr_expr to be resolved (cf TODO fullres) @@ -137,7 +137,7 @@ fn take_optimizable(node: &dyn ExecutionPlan) -> Option> /// If this agg_expr is a count that can be exactly derived from the statistics, return it. fn take_optimizable_column_and_table_count( - agg_expr: &dyn AggregateExpr, + agg_expr: &AggregateFunctionExpr, stats: &Statistics, ) -> Option<(ScalarValue, String)> { let col_stats = &stats.column_statistics; @@ -174,7 +174,7 @@ fn take_optimizable_column_and_table_count( /// If this agg_expr is a min that is exactly defined in the statistics, return it. fn take_optimizable_min( - agg_expr: &dyn AggregateExpr, + agg_expr: &AggregateFunctionExpr, stats: &Statistics, ) -> Option<(ScalarValue, String)> { if let Precision::Exact(num_rows) = &stats.num_rows { @@ -220,7 +220,7 @@ fn take_optimizable_min( /// If this agg_expr is a max that is exactly defined in the statistics, return it. fn take_optimizable_max( - agg_expr: &dyn AggregateExpr, + agg_expr: &AggregateFunctionExpr, stats: &Statistics, ) -> Option<(ScalarValue, String)> { if let Precision::Exact(num_rows) = &stats.num_rows { @@ -266,33 +266,27 @@ fn take_optimizable_max( // TODO: Move this check into AggregateUDFImpl // https://github.com/apache/datafusion/issues/11153 -fn is_non_distinct_count(agg_expr: &dyn AggregateExpr) -> bool { - if let Some(agg_expr) = agg_expr.as_any().downcast_ref::() { - if agg_expr.fun().name() == "count" && !agg_expr.is_distinct() { - return true; - } +fn is_non_distinct_count(agg_expr: &AggregateFunctionExpr) -> bool { + if agg_expr.fun().name() == "count" && !agg_expr.is_distinct() { + return true; } false } // TODO: Move this check into AggregateUDFImpl // https://github.com/apache/datafusion/issues/11153 -fn is_min(agg_expr: &dyn AggregateExpr) -> bool { - if let Some(agg_expr) = agg_expr.as_any().downcast_ref::() { - if agg_expr.fun().name().to_lowercase() == "min" { - return true; - } +fn is_min(agg_expr: &AggregateFunctionExpr) -> bool { + if agg_expr.fun().name().to_lowercase() == "min" { + return true; } false } // TODO: Move this check into AggregateUDFImpl // https://github.com/apache/datafusion/issues/11153 -fn is_max(agg_expr: &dyn AggregateExpr) -> bool { - if let Some(agg_expr) = agg_expr.as_any().downcast_ref::() { - if agg_expr.fun().name().to_lowercase() == "max" { - return true; - } +fn is_max(agg_expr: &AggregateFunctionExpr) -> bool { + if agg_expr.fun().name().to_lowercase() == "max" { + return true; } false } diff --git a/datafusion/physical-plan/Cargo.toml b/datafusion/physical-plan/Cargo.toml index 78da4dc9c53f..24387c5f15ee 100644 --- a/datafusion/physical-plan/Cargo.toml +++ b/datafusion/physical-plan/Cargo.toml @@ -55,7 +55,6 @@ datafusion-functions-aggregate = { workspace = true } datafusion-functions-aggregate-common = { workspace = true } datafusion-physical-expr = { workspace = true, default-features = true } datafusion-physical-expr-common = { workspace = true } -datafusion-physical-expr-functions-aggregate = { workspace = true } futures = { workspace = true } half = { workspace = true } hashbrown = { workspace = true } diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 5aa255e7c341..0aeaa15b5f1d 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -42,10 +42,11 @@ use datafusion_expr::Accumulator; use datafusion_physical_expr::{ equivalence::{collapse_lex_req, ProjectionMapping}, expressions::{Column, UnKnownColumn}, - physical_exprs_contains, AggregateExpr, EquivalenceProperties, LexOrdering, - LexRequirement, PhysicalExpr, PhysicalSortRequirement, + physical_exprs_contains, EquivalenceProperties, LexOrdering, LexRequirement, + PhysicalExpr, PhysicalSortRequirement, }; +use datafusion_physical_expr::aggregate::AggregateFunctionExpr; use itertools::Itertools; pub mod group_values; @@ -253,7 +254,7 @@ pub struct AggregateExec { /// Group by expressions group_by: PhysicalGroupBy, /// Aggregate expressions - aggr_expr: Vec>, + aggr_expr: Vec>, /// FILTER (WHERE clause) expression for each aggregate expression filter_expr: Vec>>, /// Set if the output of this aggregation is truncated by a upstream sort/limit clause @@ -280,7 +281,10 @@ impl AggregateExec { /// Function used in `ConvertFirstLast` optimizer rule, /// where we need parts of the new value, others cloned from the old one /// Rewrites aggregate exec with new aggregate expressions. - pub fn with_new_aggr_exprs(&self, aggr_expr: Vec>) -> Self { + pub fn with_new_aggr_exprs( + &self, + aggr_expr: Vec>, + ) -> Self { Self { aggr_expr, // clone the rest of the fields @@ -306,7 +310,7 @@ impl AggregateExec { pub fn try_new( mode: AggregateMode, group_by: PhysicalGroupBy, - aggr_expr: Vec>, + aggr_expr: Vec>, filter_expr: Vec>>, input: Arc, input_schema: SchemaRef, @@ -343,7 +347,7 @@ impl AggregateExec { fn try_new_with_schema( mode: AggregateMode, group_by: PhysicalGroupBy, - mut aggr_expr: Vec>, + mut aggr_expr: Vec>, filter_expr: Vec>>, input: Arc, input_schema: SchemaRef, @@ -451,7 +455,7 @@ impl AggregateExec { } /// Aggregate expressions - pub fn aggr_expr(&self) -> &[Arc] { + pub fn aggr_expr(&self) -> &[Arc] { &self.aggr_expr } @@ -788,7 +792,7 @@ impl ExecutionPlan for AggregateExec { fn create_schema( input_schema: &Schema, group_expr: &[(Arc, String)], - aggr_expr: &[Arc], + aggr_expr: &[Arc], contains_null_expr: bool, mode: AggregateMode, ) -> Result { @@ -834,7 +838,7 @@ fn group_schema(schema: &Schema, group_count: usize) -> SchemaRef { /// /// # Parameters /// -/// - `aggr_expr`: A reference to an `Arc` representing the +/// - `aggr_expr`: A reference to an `Arc` representing the /// aggregate expression. /// - `group_by`: A reference to a `PhysicalGroupBy` instance representing the /// physical GROUP BY expression. @@ -846,7 +850,7 @@ fn group_schema(schema: &Schema, group_count: usize) -> SchemaRef { /// A `LexOrdering` instance indicating the lexical ordering requirement for /// the aggregate expression. fn get_aggregate_expr_req( - aggr_expr: &Arc, + aggr_expr: &Arc, group_by: &PhysicalGroupBy, agg_mode: &AggregateMode, ) -> LexOrdering { @@ -894,7 +898,7 @@ fn get_aggregate_expr_req( /// the aggregator requirement is incompatible. fn finer_ordering( existing_req: &LexOrdering, - aggr_expr: &Arc, + aggr_expr: &Arc, group_by: &PhysicalGroupBy, eq_properties: &EquivalenceProperties, agg_mode: &AggregateMode, @@ -912,7 +916,7 @@ pub fn concat_slices(lhs: &[T], rhs: &[T]) -> Vec { /// /// # Parameters /// -/// - `aggr_exprs`: A slice of `Arc` containing all the +/// - `aggr_exprs`: A slice of `Arc` containing all the /// aggregate expressions. /// - `group_by`: A reference to a `PhysicalGroupBy` instance representing the /// physical GROUP BY expression. @@ -926,7 +930,7 @@ pub fn concat_slices(lhs: &[T], rhs: &[T]) -> Vec { /// A `LexRequirement` instance, which is the requirement that satisfies all the /// aggregate requirements. Returns an error in case of conflicting requirements. pub fn get_finer_aggregate_exprs_requirement( - aggr_exprs: &mut [Arc], + aggr_exprs: &mut [Arc], group_by: &PhysicalGroupBy, eq_properties: &EquivalenceProperties, agg_mode: &AggregateMode, @@ -996,10 +1000,10 @@ pub fn get_finer_aggregate_exprs_requirement( /// returns physical expressions for arguments to evaluate against a batch /// The expressions are different depending on `mode`: -/// * Partial: AggregateExpr::expressions -/// * Final: columns of `AggregateExpr::state_fields()` +/// * Partial: AggregateFunctionExpr::expressions +/// * Final: columns of `AggregateFunctionExpr::state_fields()` pub fn aggregate_expressions( - aggr_expr: &[Arc], + aggr_expr: &[Arc], mode: &AggregateMode, col_idx_base: usize, ) -> Result>>> { @@ -1035,12 +1039,12 @@ pub fn aggregate_expressions( } /// uses `state_fields` to build a vec of physical column expressions required to merge the -/// AggregateExpr' accumulator's state. +/// AggregateFunctionExpr' accumulator's state. /// /// `index_base` is the starting physical column index for the next expanded state field. fn merge_expressions( index_base: usize, - expr: &Arc, + expr: &Arc, ) -> Result>> { expr.state_fields().map(|fields| { fields @@ -1054,7 +1058,7 @@ fn merge_expressions( pub type AccumulatorItem = Box; pub fn create_accumulators( - aggr_expr: &[Arc], + aggr_expr: &[Arc], ) -> Result> { aggr_expr .iter() @@ -1218,8 +1222,8 @@ mod tests { use datafusion_physical_expr::PhysicalSortExpr; use crate::common::collect; + use datafusion_physical_expr::aggregate::AggregateExprBuilder; use datafusion_physical_expr::expressions::Literal; - use datafusion_physical_expr_functions_aggregate::aggregate::AggregateExprBuilder; use futures::{FutureExt, Stream}; // Generate a schema which consists of 5 columns (a, b, c, d, e) @@ -1496,13 +1500,12 @@ mod tests { groups: vec![vec![false]], }; - let aggregates: Vec> = - vec![ - AggregateExprBuilder::new(avg_udaf(), vec![col("b", &input_schema)?]) - .schema(Arc::clone(&input_schema)) - .alias("AVG(b)") - .build()?, - ]; + let aggregates: Vec> = vec![ + AggregateExprBuilder::new(avg_udaf(), vec![col("b", &input_schema)?]) + .schema(Arc::clone(&input_schema)) + .alias("AVG(b)") + .build()?, + ]; let task_ctx = if spill { // set to an appropriate value to trigger spill @@ -1793,7 +1796,7 @@ mod tests { } // Median(a) - fn test_median_agg_expr(schema: SchemaRef) -> Result> { + fn test_median_agg_expr(schema: SchemaRef) -> Result> { AggregateExprBuilder::new(median_udaf(), vec![col("a", &schema)?]) .schema(schema) .alias("MEDIAN(a)") @@ -1819,17 +1822,16 @@ mod tests { }; // something that allocates within the aggregator - let aggregates_v0: Vec> = + let aggregates_v0: Vec> = vec![test_median_agg_expr(Arc::clone(&input_schema))?]; // use fast-path in `row_hash.rs`. - let aggregates_v2: Vec> = - vec![ - AggregateExprBuilder::new(avg_udaf(), vec![col("b", &input_schema)?]) - .schema(Arc::clone(&input_schema)) - .alias("AVG(b)") - .build()?, - ]; + let aggregates_v2: Vec> = vec![ + AggregateExprBuilder::new(avg_udaf(), vec![col("b", &input_schema)?]) + .schema(Arc::clone(&input_schema)) + .alias("AVG(b)") + .build()?, + ]; for (version, groups, aggregates) in [ (0, groups_none, aggregates_v0), @@ -1883,13 +1885,12 @@ mod tests { let groups = PhysicalGroupBy::default(); - let aggregates: Vec> = - vec![ - AggregateExprBuilder::new(avg_udaf(), vec![col("a", &schema)?]) - .schema(Arc::clone(&schema)) - .alias("AVG(a)") - .build()?, - ]; + let aggregates: Vec> = vec![ + AggregateExprBuilder::new(avg_udaf(), vec![col("a", &schema)?]) + .schema(Arc::clone(&schema)) + .alias("AVG(a)") + .build()?, + ]; let blocking_exec = Arc::new(BlockingExec::new(Arc::clone(&schema), 1)); let refs = blocking_exec.refs(); @@ -1923,13 +1924,12 @@ mod tests { let groups = PhysicalGroupBy::new_single(vec![(col("a", &schema)?, "a".to_string())]); - let aggregates: Vec> = - vec![ - AggregateExprBuilder::new(avg_udaf(), vec![col("b", &schema)?]) - .schema(Arc::clone(&schema)) - .alias("AVG(b)") - .build()?, - ]; + let aggregates: Vec> = vec![ + AggregateExprBuilder::new(avg_udaf(), vec![col("b", &schema)?]) + .schema(Arc::clone(&schema)) + .alias("AVG(b)") + .build()?, + ]; let blocking_exec = Arc::new(BlockingExec::new(Arc::clone(&schema), 1)); let refs = blocking_exec.refs(); @@ -1974,7 +1974,7 @@ mod tests { fn test_first_value_agg_expr( schema: &Schema, sort_options: SortOptions, - ) -> Result> { + ) -> Result> { let ordering_req = [PhysicalSortExpr { expr: col("b", schema)?, options: sort_options, @@ -1992,7 +1992,7 @@ mod tests { fn test_last_value_agg_expr( schema: &Schema, sort_options: SortOptions, - ) -> Result> { + ) -> Result> { let ordering_req = [PhysicalSortExpr { expr: col("b", schema)?, options: sort_options, @@ -2047,7 +2047,7 @@ mod tests { descending: false, nulls_first: false, }; - let aggregates: Vec> = if is_first_acc { + let aggregates: Vec> = if is_first_acc { vec![test_first_value_agg_expr(&schema, sort_options)?] } else { vec![test_last_value_agg_expr(&schema, sort_options)?] @@ -2212,7 +2212,7 @@ mod tests { }; let groups = PhysicalGroupBy::new_single(vec![(col_a, "a".to_string())]); - let aggregates: Vec> = vec![ + let aggregates: Vec> = vec![ test_first_value_agg_expr(&schema, option_desc)?, test_last_value_agg_expr(&schema, option_desc)?, ]; @@ -2270,7 +2270,7 @@ mod tests { ], ); - let aggregates: Vec> = + let aggregates: Vec> = vec![AggregateExprBuilder::new(count_udaf(), vec![lit(1)]) .schema(Arc::clone(&schema)) .alias("1") diff --git a/datafusion/physical-plan/src/aggregates/row_hash.rs b/datafusion/physical-plan/src/aggregates/row_hash.rs index 05f4ec621813..d022bb007d9b 100644 --- a/datafusion/physical-plan/src/aggregates/row_hash.rs +++ b/datafusion/physical-plan/src/aggregates/row_hash.rs @@ -47,10 +47,9 @@ use datafusion_execution::runtime_env::RuntimeEnv; use datafusion_execution::TaskContext; use datafusion_expr::{EmitTo, GroupsAccumulator}; use datafusion_physical_expr::expressions::Column; -use datafusion_physical_expr::{ - AggregateExpr, GroupsAccumulatorAdapter, PhysicalSortExpr, -}; +use datafusion_physical_expr::{GroupsAccumulatorAdapter, PhysicalSortExpr}; +use datafusion_physical_expr::aggregate::AggregateFunctionExpr; use futures::ready; use futures::stream::{Stream, StreamExt}; use log::debug; @@ -396,7 +395,7 @@ pub(crate) struct GroupedHashAggregateStream { /// processed. Reused across batches here to avoid reallocations current_group_indices: Vec, - /// Accumulators, one for each `AggregateExpr` in the query + /// Accumulators, one for each `AggregateFunctionExpr` in the query /// /// For example, if the query has aggregates, `SUM(x)`, /// `COUNT(y)`, there will be two accumulators, each one @@ -579,7 +578,7 @@ impl GroupedHashAggregateStream { /// that is supported by the aggregate, or a /// [`GroupsAccumulatorAdapter`] if not. pub(crate) fn create_group_accumulator( - agg_expr: &Arc, + agg_expr: &Arc, ) -> Result> { if agg_expr.groups_accumulator_supported() { agg_expr.create_groups_accumulator() diff --git a/datafusion/physical-plan/src/execution_plan.rs b/datafusion/physical-plan/src/execution_plan.rs index e1182719293d..c1c66f6d3923 100644 --- a/datafusion/physical-plan/src/execution_plan.rs +++ b/datafusion/physical-plan/src/execution_plan.rs @@ -34,7 +34,7 @@ pub use datafusion_execution::{RecordBatchStream, SendableRecordBatchStream}; pub use datafusion_expr::{Accumulator, ColumnarValue}; pub use datafusion_physical_expr::window::WindowExpr; pub use datafusion_physical_expr::{ - expressions, functions, udf, AggregateExpr, Distribution, Partitioning, PhysicalExpr, + expressions, functions, udf, Distribution, Partitioning, PhysicalExpr, }; use datafusion_physical_expr::{EquivalenceProperties, LexOrdering, PhysicalSortExpr}; use datafusion_physical_expr_common::sort_expr::LexRequirement; diff --git a/datafusion/physical-plan/src/lib.rs b/datafusion/physical-plan/src/lib.rs index fb86a008e2cd..026798c5798b 100644 --- a/datafusion/physical-plan/src/lib.rs +++ b/datafusion/physical-plan/src/lib.rs @@ -31,7 +31,7 @@ pub use datafusion_expr::{Accumulator, ColumnarValue}; pub use datafusion_physical_expr::window::WindowExpr; use datafusion_physical_expr::PhysicalSortExpr; pub use datafusion_physical_expr::{ - expressions, functions, udf, AggregateExpr, Distribution, Partitioning, PhysicalExpr, + expressions, functions, udf, Distribution, Partitioning, PhysicalExpr, }; pub use crate::display::{DefaultDisplay, DisplayAs, DisplayFormatType, VerboseDisplay}; @@ -82,7 +82,7 @@ pub mod windows; pub mod work_table; pub mod udaf { - pub use datafusion_physical_expr_functions_aggregate::aggregate::AggregateFunctionExpr; + pub use datafusion_physical_expr::aggregate::AggregateFunctionExpr; } pub mod coalesce; diff --git a/datafusion/physical-plan/src/windows/mod.rs b/datafusion/physical-plan/src/windows/mod.rs index d607bb79b44e..56823e6dec2d 100644 --- a/datafusion/physical-plan/src/windows/mod.rs +++ b/datafusion/physical-plan/src/windows/mod.rs @@ -37,14 +37,13 @@ use datafusion_expr::{ BuiltInWindowFunction, PartitionEvaluator, WindowFrame, WindowFunctionDefinition, WindowUDF, }; +use datafusion_physical_expr::aggregate::{AggregateExprBuilder, AggregateFunctionExpr}; use datafusion_physical_expr::equivalence::collapse_lex_req; use datafusion_physical_expr::{ reverse_order_bys, window::{BuiltInWindowFunctionExpr, SlidingAggregateWindowExpr}, - AggregateExpr, ConstExpr, EquivalenceProperties, LexOrdering, - PhysicalSortRequirement, + ConstExpr, EquivalenceProperties, LexOrdering, PhysicalSortRequirement, }; -use datafusion_physical_expr_functions_aggregate::aggregate::AggregateExprBuilder; use itertools::Itertools; mod bounded_window_agg_exec; @@ -142,7 +141,7 @@ fn window_expr_from_aggregate_expr( partition_by: &[Arc], order_by: &[PhysicalSortExpr], window_frame: Arc, - aggregate: Arc, + aggregate: Arc, ) -> Arc { // Is there a potentially unlimited sized window frame? let unbounded_window = window_frame.start_bound.is_unbounded(); diff --git a/datafusion/proto/src/physical_plan/mod.rs b/datafusion/proto/src/physical_plan/mod.rs index 96fb45eafe62..78f370c714cc 100644 --- a/datafusion/proto/src/physical_plan/mod.rs +++ b/datafusion/proto/src/physical_plan/mod.rs @@ -18,7 +18,7 @@ use std::fmt::Debug; use std::sync::Arc; -use datafusion::physical_expr_functions_aggregate::aggregate::AggregateExprBuilder; +use datafusion::physical_expr::aggregate::AggregateExprBuilder; use prost::bytes::BufMut; use prost::Message; @@ -34,6 +34,7 @@ use datafusion::datasource::physical_plan::ParquetExec; use datafusion::datasource::physical_plan::{AvroExec, CsvExec}; use datafusion::execution::runtime_env::RuntimeEnv; use datafusion::execution::FunctionRegistry; +use datafusion::physical_expr::aggregate::AggregateFunctionExpr; use datafusion::physical_expr::{PhysicalExprRef, PhysicalSortRequirement}; use datafusion::physical_plan::aggregates::AggregateMode; use datafusion::physical_plan::aggregates::{AggregateExec, PhysicalGroupBy}; @@ -59,7 +60,7 @@ use datafusion::physical_plan::sorts::sort_preserving_merge::SortPreservingMerge use datafusion::physical_plan::union::{InterleaveExec, UnionExec}; use datafusion::physical_plan::windows::{BoundedWindowAggExec, WindowAggExec}; use datafusion::physical_plan::{ - AggregateExpr, ExecutionPlan, InputOrderMode, PhysicalExpr, WindowExpr, + ExecutionPlan, InputOrderMode, PhysicalExpr, WindowExpr, }; use datafusion_common::{internal_err, not_impl_err, DataFusionError, Result}; use datafusion_expr::{AggregateUDF, ScalarUDF}; @@ -467,7 +468,7 @@ impl AsExecutionPlan for protobuf::PhysicalPlanNode { }) .collect::, _>>()?; - let physical_aggr_expr: Vec> = hash_agg + let physical_aggr_expr: Vec> = hash_agg .aggr_expr .iter() .zip(hash_agg.aggr_expr_name.iter()) diff --git a/datafusion/proto/src/physical_plan/to_proto.rs b/datafusion/proto/src/physical_plan/to_proto.rs index 7949a457f40f..555ad22a9bc1 100644 --- a/datafusion/proto/src/physical_plan/to_proto.rs +++ b/datafusion/proto/src/physical_plan/to_proto.rs @@ -29,7 +29,7 @@ use datafusion::physical_plan::expressions::{ }; use datafusion::physical_plan::udaf::AggregateFunctionExpr; use datafusion::physical_plan::windows::{BuiltInWindowExpr, PlainAggregateWindowExpr}; -use datafusion::physical_plan::{AggregateExpr, Partitioning, PhysicalExpr, WindowExpr}; +use datafusion::physical_plan::{Partitioning, PhysicalExpr, WindowExpr}; use datafusion::{ datasource::{ file_format::{csv::CsvSink, json::JsonSink}, @@ -49,58 +49,50 @@ use crate::protobuf::{ use super::PhysicalExtensionCodec; pub fn serialize_physical_aggr_expr( - aggr_expr: Arc, + aggr_expr: Arc, codec: &dyn PhysicalExtensionCodec, ) -> Result { let expressions = serialize_physical_exprs(aggr_expr.expressions(), codec)?; let ordering_req = aggr_expr.order_bys().unwrap_or(&[]).to_vec(); let ordering_req = serialize_physical_sort_exprs(ordering_req, codec)?; - if let Some(a) = aggr_expr.as_any().downcast_ref::() { - let name = a.fun().name().to_string(); - let mut buf = Vec::new(); - codec.try_encode_udaf(a.fun(), &mut buf)?; - Ok(protobuf::PhysicalExprNode { - expr_type: Some(protobuf::physical_expr_node::ExprType::AggregateExpr( - protobuf::PhysicalAggregateExprNode { - aggregate_function: Some(physical_aggregate_expr_node::AggregateFunction::UserDefinedAggrFunction(name)), - expr: expressions, - ordering_req, - distinct: a.is_distinct(), - ignore_nulls: a.ignore_nulls(), - fun_definition: (!buf.is_empty()).then_some(buf) - }, - )), - }) - } else { - unreachable!("No other types exists besides AggergationFunctionExpr"); - } + let name = aggr_expr.fun().name().to_string(); + let mut buf = Vec::new(); + codec.try_encode_udaf(aggr_expr.fun(), &mut buf)?; + Ok(protobuf::PhysicalExprNode { + expr_type: Some(protobuf::physical_expr_node::ExprType::AggregateExpr( + protobuf::PhysicalAggregateExprNode { + aggregate_function: Some(physical_aggregate_expr_node::AggregateFunction::UserDefinedAggrFunction(name)), + expr: expressions, + ordering_req, + distinct: aggr_expr.is_distinct(), + ignore_nulls: aggr_expr.ignore_nulls(), + fun_definition: (!buf.is_empty()).then_some(buf) + }, + )), + }) } fn serialize_physical_window_aggr_expr( - aggr_expr: &dyn AggregateExpr, + aggr_expr: &AggregateFunctionExpr, _window_frame: &WindowFrame, codec: &dyn PhysicalExtensionCodec, ) -> Result<(physical_window_expr_node::WindowFunction, Option>)> { - if let Some(a) = aggr_expr.as_any().downcast_ref::() { - if a.is_distinct() || a.ignore_nulls() { - // TODO - return not_impl_err!( - "Distinct aggregate functions not supported in window expressions" - ); - } - - let mut buf = Vec::new(); - codec.try_encode_udaf(a.fun(), &mut buf)?; - Ok(( - physical_window_expr_node::WindowFunction::UserDefinedAggrFunction( - a.fun().name().to_string(), - ), - (!buf.is_empty()).then_some(buf), - )) - } else { - unreachable!("No other types exists besides AggergationFunctionExpr"); + if aggr_expr.is_distinct() || aggr_expr.ignore_nulls() { + // TODO + return not_impl_err!( + "Distinct aggregate functions not supported in window expressions" + ); } + + let mut buf = Vec::new(); + codec.try_encode_udaf(aggr_expr.fun(), &mut buf)?; + Ok(( + physical_window_expr_node::WindowFunction::UserDefinedAggrFunction( + aggr_expr.fun().name().to_string(), + ), + (!buf.is_empty()).then_some(buf), + )) } pub fn serialize_physical_window_expr( diff --git a/datafusion/proto/tests/cases/roundtrip_physical_plan.rs b/datafusion/proto/tests/cases/roundtrip_physical_plan.rs index 0ffc494321fb..60f5565bdeee 100644 --- a/datafusion/proto/tests/cases/roundtrip_physical_plan.rs +++ b/datafusion/proto/tests/cases/roundtrip_physical_plan.rs @@ -24,7 +24,7 @@ use std::vec; use arrow::array::RecordBatch; use arrow::csv::WriterBuilder; -use datafusion::physical_expr_functions_aggregate::aggregate::AggregateExprBuilder; +use datafusion::physical_expr::aggregate::AggregateExprBuilder; use datafusion::physical_plan::coalesce_batches::CoalesceBatchesExec; use datafusion_functions_aggregate::approx_percentile_cont::approx_percentile_cont_udaf; use datafusion_functions_aggregate::array_agg::array_agg_udaf; @@ -47,7 +47,6 @@ use datafusion::datasource::physical_plan::{ use datafusion::execution::FunctionRegistry; use datafusion::functions_aggregate::sum::sum_udaf; use datafusion::logical_expr::{create_udf, JoinType, Operator, Volatility}; -use datafusion::physical_expr::aggregate::utils::down_cast_any_ref; use datafusion::physical_expr::expressions::Literal; use datafusion::physical_expr::window::SlidingAggregateWindowExpr; use datafusion::physical_expr::{PhysicalSortRequirement, ScalarFunctionExpr}; @@ -70,13 +69,12 @@ use datafusion::physical_plan::placeholder_row::PlaceholderRowExec; use datafusion::physical_plan::projection::ProjectionExec; use datafusion::physical_plan::repartition::RepartitionExec; use datafusion::physical_plan::sorts::sort::SortExec; +use datafusion::physical_plan::udaf::AggregateFunctionExpr; use datafusion::physical_plan::union::{InterleaveExec, UnionExec}; use datafusion::physical_plan::windows::{ BuiltInWindowExpr, PlainAggregateWindowExpr, WindowAggExec, }; -use datafusion::physical_plan::{ - AggregateExpr, ExecutionPlan, Partitioning, PhysicalExpr, Statistics, -}; +use datafusion::physical_plan::{ExecutionPlan, Partitioning, PhysicalExpr, Statistics}; use datafusion::prelude::SessionContext; use datafusion::scalar::ScalarValue; use datafusion_common::config::TableParquetOptions; @@ -362,7 +360,7 @@ fn rountrip_aggregate() -> Result<()> { .alias("NTH_VALUE(b, 1)") .build()?; - let test_cases: Vec>> = vec![ + let test_cases: Vec>> = vec![ // AVG vec![avg_expr], // NTH_VALUE @@ -395,7 +393,7 @@ fn rountrip_aggregate_with_limit() -> Result<()> { let groups: Vec<(Arc, String)> = vec![(col("a", &schema)?, "unused".to_string())]; - let aggregates: Vec> = + let aggregates: Vec> = vec![ AggregateExprBuilder::new(avg_udaf(), vec![col("b", &schema)?]) .schema(Arc::clone(&schema)) @@ -424,7 +422,7 @@ fn rountrip_aggregate_with_approx_pencentile_cont() -> Result<()> { let groups: Vec<(Arc, String)> = vec![(col("a", &schema)?, "unused".to_string())]; - let aggregates: Vec> = vec![AggregateExprBuilder::new( + let aggregates: Vec> = vec![AggregateExprBuilder::new( approx_percentile_cont_udaf(), vec![col("b", &schema)?, lit(0.5)], ) @@ -459,7 +457,7 @@ fn rountrip_aggregate_with_sort() -> Result<()> { }, }]; - let aggregates: Vec> = + let aggregates: Vec> = vec![ AggregateExprBuilder::new(array_agg_udaf(), vec![col("b", &schema)?]) .schema(Arc::clone(&schema)) @@ -526,7 +524,7 @@ fn roundtrip_aggregate_udaf() -> Result<()> { let groups: Vec<(Arc, String)> = vec![(col("a", &schema)?, "unused".to_string())]; - let aggregates: Vec> = + let aggregates: Vec> = vec![ AggregateExprBuilder::new(Arc::new(udaf), vec![col("b", &schema)?]) .schema(Arc::clone(&schema)) @@ -748,7 +746,7 @@ fn roundtrip_parquet_exec_with_custom_predicate_expr() -> Result<()> { } impl PartialEq for CustomPredicateExpr { fn eq(&self, other: &dyn Any) -> bool { - down_cast_any_ref(other) + other .downcast_ref::() .map(|x| self.inner.eq(&x.inner)) .unwrap_or(false) From 58ed702c8f7f2536a8f71cf8b7486812a3afa42d Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Tue, 27 Aug 2024 15:17:23 +0200 Subject: [PATCH 033/154] Deduplicate sort unparsing logic (#12175) * Deduplicate sort unparsing logic Reconstruction of `ast::OrderByExpr` was implemented twice, in `plan.rs` and `expr.rs` submodules of the unparser. * empty --- datafusion/sql/src/unparser/plan.rs | 22 ++++------------------ 1 file changed, 4 insertions(+), 18 deletions(-) diff --git a/datafusion/sql/src/unparser/plan.rs b/datafusion/sql/src/unparser/plan.rs index 8b5a5b0942b8..106705c322fc 100644 --- a/datafusion/sql/src/unparser/plan.rs +++ b/datafusion/sql/src/unparser/plan.rs @@ -528,24 +528,10 @@ impl Unparser<'_> { fn sort_to_sql(&self, sort_exprs: Vec) -> Result> { sort_exprs .iter() - .map(|expr: &Expr| match expr { - Expr::Sort(sort_expr) => { - let col = self.expr_to_sql(&sort_expr.expr)?; - - let nulls_first = if self.dialect.supports_nulls_first_in_sort() { - Some(sort_expr.nulls_first) - } else { - None - }; - - Ok(ast::OrderByExpr { - asc: Some(sort_expr.asc), - expr: col, - nulls_first, - with_fill: None, - }) - } - _ => plan_err!("Expecting Sort expr"), + .map(|expr: &Expr| { + self.expr_to_unparsed(expr)? + .into_order_by_expr() + .or(plan_err!("Expecting Sort expr")) }) .collect::>>() } From 4a943560e1a24108aeb915be260db7c4264c9b11 Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Tue, 27 Aug 2024 15:22:44 +0200 Subject: [PATCH 034/154] Require sort expressions to be of type Sort (#12171) * Require sort expressions to be of type Sort in compare_sort_expr * fixup! Require sort expressions to be of type Sort in compare_sort_expr * fixup! Require sort expressions to be of type Sort in compare_sort_expr --- datafusion/expr/src/utils.rs | 2 +- datafusion/sql/src/expr/function.rs | 4 +++- datafusion/sql/src/unparser/rewrite.rs | 2 ++ 3 files changed, 6 insertions(+), 2 deletions(-) diff --git a/datafusion/expr/src/utils.rs b/datafusion/expr/src/utils.rs index 8f04d3ec3067..a01d5ef8973a 100644 --- a/datafusion/expr/src/utils.rs +++ b/datafusion/expr/src/utils.rs @@ -569,7 +569,7 @@ pub fn compare_sort_expr( } Ordering::Equal } - _ => Ordering::Equal, + _ => panic!("Sort expressions must be of type Sort"), } } diff --git a/datafusion/sql/src/expr/function.rs b/datafusion/sql/src/expr/function.rs index b95414a8cafd..72e08e4b8fb5 100644 --- a/datafusion/sql/src/expr/function.rs +++ b/datafusion/sql/src/expr/function.rs @@ -295,8 +295,10 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { Some(false) }; } + Some(false) + } else { + panic!("order_by expression must be of type Sort"); } - Some(false) }); let window_frame = window diff --git a/datafusion/sql/src/unparser/rewrite.rs b/datafusion/sql/src/unparser/rewrite.rs index fed3f2c39f9d..9e1adcf4df31 100644 --- a/datafusion/sql/src/unparser/rewrite.rs +++ b/datafusion/sql/src/unparser/rewrite.rs @@ -161,6 +161,8 @@ pub(super) fn rewrite_plan_for_sort_on_non_projected_fields( for expr in &sort.expr { if let Expr::Sort(s) = expr { collects.push(s.expr.as_ref().clone()); + } else { + panic!("sort expression must be of type Sort"); } } From a616e8895bdde48153d70ba32281d5174d172cea Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Tue, 27 Aug 2024 15:24:11 +0200 Subject: [PATCH 035/154] Remove `parse_vec_expr` helper (#12178) * Remove `parse_vec_expr` helper Behavior of `parse_vec_expr` and `parse_exprs` is almost similar -- both take a collection of expressions to parse. The only difference is that `parse_vec_expr` returns `Option::None` when collections is empty, but this difference in behavior does not correspond to difference in function names. Since the function is used once only, remove it instead of coming up with a fancy name. * empty --- datafusion/proto/src/logical_plan/from_proto.rs | 15 ++++----------- 1 file changed, 4 insertions(+), 11 deletions(-) diff --git a/datafusion/proto/src/logical_plan/from_proto.rs b/datafusion/proto/src/logical_plan/from_proto.rs index b74237b5281b..acda1298dd80 100644 --- a/datafusion/proto/src/logical_plan/from_proto.rs +++ b/datafusion/proto/src/logical_plan/from_proto.rs @@ -586,7 +586,10 @@ pub fn parse_expr( parse_exprs(&pb.args, registry, codec)?, pb.distinct, parse_optional_expr(pb.filter.as_deref(), registry, codec)?.map(Box::new), - parse_vec_expr(&pb.order_by, registry, codec)?, + match pb.order_by.len() { + 0 => None, + _ => Some(parse_exprs(&pb.order_by, registry, codec)?), + }, None, ))) } @@ -676,16 +679,6 @@ pub fn from_proto_binary_op(op: &str) -> Result { } } -fn parse_vec_expr( - p: &[protobuf::LogicalExprNode], - registry: &dyn FunctionRegistry, - codec: &dyn LogicalExtensionCodec, -) -> Result>, Error> { - let res = parse_exprs(p, registry, codec)?; - // Convert empty vector to None. - Ok((!res.is_empty()).then_some(res)) -} - fn parse_optional_expr( p: Option<&protobuf::LogicalExprNode>, registry: &dyn FunctionRegistry, From bf46f33f0746598c69ad6651c11f36842c5e71a2 Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Tue, 27 Aug 2024 15:25:23 +0200 Subject: [PATCH 036/154] Reuse bulk serialization helpers for protobuf (#12179) Reuse `parse_exprs` and `serialize_exprs`. Reduce code duplication, increasing readability. --- datafusion/proto/src/logical_plan/mod.rs | 205 +++++++---------------- 1 file changed, 60 insertions(+), 145 deletions(-) diff --git a/datafusion/proto/src/logical_plan/mod.rs b/datafusion/proto/src/logical_plan/mod.rs index bc019725f36c..67977b1795a6 100644 --- a/datafusion/proto/src/logical_plan/mod.rs +++ b/datafusion/proto/src/logical_plan/mod.rs @@ -29,7 +29,7 @@ use crate::{ }, }; -use crate::protobuf::{proto_error, FromProtoError, ToProtoError}; +use crate::protobuf::{proto_error, ToProtoError}; use arrow::datatypes::{DataType, Schema, SchemaRef}; #[cfg(feature = "parquet")] use datafusion::datasource::file_format::parquet::ParquetFormat; @@ -66,11 +66,10 @@ use datafusion_expr::{ }; use datafusion_expr::{AggregateUDF, Unnest}; +use self::to_proto::{serialize_expr, serialize_exprs}; use prost::bytes::BufMut; use prost::Message; -use self::to_proto::serialize_expr; - pub mod file_formats; pub mod from_proto; pub mod to_proto; @@ -273,13 +272,7 @@ impl AsLogicalPlan for LogicalPlanNode { values .values_list .chunks_exact(n_cols) - .map(|r| { - r.iter() - .map(|expr| { - from_proto::parse_expr(expr, ctx, extension_codec) - }) - .collect::, FromProtoError>>() - }) + .map(|r| from_proto::parse_exprs(r, ctx, extension_codec)) .collect::, _>>() .map_err(|e| e.into()) }?; @@ -288,11 +281,8 @@ impl AsLogicalPlan for LogicalPlanNode { LogicalPlanType::Projection(projection) => { let input: LogicalPlan = into_logical_plan!(projection.input, ctx, extension_codec)?; - let expr: Vec = projection - .expr - .iter() - .map(|expr| from_proto::parse_expr(expr, ctx, extension_codec)) - .collect::, _>>()?; + let expr: Vec = + from_proto::parse_exprs(&projection.expr, ctx, extension_codec)?; let new_proj = project(input, expr)?; match projection.optional_alias.as_ref() { @@ -324,26 +314,17 @@ impl AsLogicalPlan for LogicalPlanNode { LogicalPlanType::Window(window) => { let input: LogicalPlan = into_logical_plan!(window.input, ctx, extension_codec)?; - let window_expr = window - .window_expr - .iter() - .map(|expr| from_proto::parse_expr(expr, ctx, extension_codec)) - .collect::, _>>()?; + let window_expr = + from_proto::parse_exprs(&window.window_expr, ctx, extension_codec)?; LogicalPlanBuilder::from(input).window(window_expr)?.build() } LogicalPlanType::Aggregate(aggregate) => { let input: LogicalPlan = into_logical_plan!(aggregate.input, ctx, extension_codec)?; - let group_expr = aggregate - .group_expr - .iter() - .map(|expr| from_proto::parse_expr(expr, ctx, extension_codec)) - .collect::, _>>()?; - let aggr_expr = aggregate - .aggr_expr - .iter() - .map(|expr| from_proto::parse_expr(expr, ctx, extension_codec)) - .collect::, _>>()?; + let group_expr = + from_proto::parse_exprs(&aggregate.group_expr, ctx, extension_codec)?; + let aggr_expr = + from_proto::parse_exprs(&aggregate.aggr_expr, ctx, extension_codec)?; LogicalPlanBuilder::from(input) .aggregate(group_expr, aggr_expr)? .build() @@ -361,20 +342,16 @@ impl AsLogicalPlan for LogicalPlanNode { projection = Some(column_indices); } - let filters = scan - .filters - .iter() - .map(|expr| from_proto::parse_expr(expr, ctx, extension_codec)) - .collect::, _>>()?; + let filters = + from_proto::parse_exprs(&scan.filters, ctx, extension_codec)?; let mut all_sort_orders = vec![]; for order in &scan.file_sort_order { - let file_sort_order = order - .logical_expr_nodes - .iter() - .map(|expr| from_proto::parse_expr(expr, ctx, extension_codec)) - .collect::, _>>()?; - all_sort_orders.push(file_sort_order) + all_sort_orders.push(from_proto::parse_exprs( + &order.logical_expr_nodes, + ctx, + extension_codec, + )?) } let file_format: Arc = @@ -475,11 +452,8 @@ impl AsLogicalPlan for LogicalPlanNode { projection = Some(column_indices); } - let filters = scan - .filters - .iter() - .map(|expr| from_proto::parse_expr(expr, ctx, extension_codec)) - .collect::, _>>()?; + let filters = + from_proto::parse_exprs(&scan.filters, ctx, extension_codec)?; let table_name = from_table_reference(scan.table_name.as_ref(), "CustomScan")?; @@ -502,11 +476,8 @@ impl AsLogicalPlan for LogicalPlanNode { LogicalPlanType::Sort(sort) => { let input: LogicalPlan = into_logical_plan!(sort.input, ctx, extension_codec)?; - let sort_expr: Vec = sort - .expr - .iter() - .map(|expr| from_proto::parse_expr(expr, ctx, extension_codec)) - .collect::, _>>()?; + let sort_expr: Vec = + from_proto::parse_exprs(&sort.expr, ctx, extension_codec)?; LogicalPlanBuilder::from(input).sort(sort_expr)?.build() } LogicalPlanType::Repartition(repartition) => { @@ -525,12 +496,7 @@ impl AsLogicalPlan for LogicalPlanNode { hash_expr: pb_hash_expr, partition_count, }) => Partitioning::Hash( - pb_hash_expr - .iter() - .map(|expr| { - from_proto::parse_expr(expr, ctx, extension_codec) - }) - .collect::, _>>()?, + from_proto::parse_exprs(pb_hash_expr, ctx, extension_codec)?, *partition_count as usize, ), PartitionMethod::RoundRobin(partition_count) => { @@ -570,12 +536,11 @@ impl AsLogicalPlan for LogicalPlanNode { let mut order_exprs = vec![]; for expr in &create_extern_table.order_exprs { - let order_expr = expr - .logical_expr_nodes - .iter() - .map(|expr| from_proto::parse_expr(expr, ctx, extension_codec)) - .collect::, _>>()?; - order_exprs.push(order_expr) + order_exprs.push(from_proto::parse_exprs( + &expr.logical_expr_nodes, + ctx, + extension_codec, + )?); } let mut column_defaults = @@ -693,16 +658,10 @@ impl AsLogicalPlan for LogicalPlanNode { LogicalPlanBuilder::from(input).limit(skip, fetch)?.build() } LogicalPlanType::Join(join) => { - let left_keys: Vec = join - .left_join_key - .iter() - .map(|expr| from_proto::parse_expr(expr, ctx, extension_codec)) - .collect::, _>>()?; - let right_keys: Vec = join - .right_join_key - .iter() - .map(|expr| from_proto::parse_expr(expr, ctx, extension_codec)) - .collect::, _>>()?; + let left_keys: Vec = + from_proto::parse_exprs(&join.left_join_key, ctx, extension_codec)?; + let right_keys: Vec = + from_proto::parse_exprs(&join.right_join_key, ctx, extension_codec)?; let join_type = protobuf::JoinType::try_from(join.join_type).map_err(|_| { proto_error(format!( @@ -804,27 +763,20 @@ impl AsLogicalPlan for LogicalPlanNode { LogicalPlanType::DistinctOn(distinct_on) => { let input: LogicalPlan = into_logical_plan!(distinct_on.input, ctx, extension_codec)?; - let on_expr = distinct_on - .on_expr - .iter() - .map(|expr| from_proto::parse_expr(expr, ctx, extension_codec)) - .collect::, _>>()?; - let select_expr = distinct_on - .select_expr - .iter() - .map(|expr| from_proto::parse_expr(expr, ctx, extension_codec)) - .collect::, _>>()?; + let on_expr = + from_proto::parse_exprs(&distinct_on.on_expr, ctx, extension_codec)?; + let select_expr = from_proto::parse_exprs( + &distinct_on.select_expr, + ctx, + extension_codec, + )?; let sort_expr = match distinct_on.sort_expr.len() { 0 => None, - _ => Some( - distinct_on - .sort_expr - .iter() - .map(|expr| { - from_proto::parse_expr(expr, ctx, extension_codec) - }) - .collect::, _>>()?, - ), + _ => Some(from_proto::parse_exprs( + &distinct_on.sort_expr, + ctx, + extension_codec, + )?), }; LogicalPlanBuilder::from(input) .distinct_on(on_expr, select_expr, sort_expr)? @@ -943,11 +895,8 @@ impl AsLogicalPlan for LogicalPlanNode { } else { values[0].len() } as u64; - let values_list = values - .iter() - .flatten() - .map(|v| serialize_expr(v, extension_codec)) - .collect::, _>>()?; + let values_list = + serialize_exprs(values.iter().flatten(), extension_codec)?; Ok(protobuf::LogicalPlanNode { logical_plan_type: Some(LogicalPlanType::Values( protobuf::ValuesNode { @@ -982,10 +931,8 @@ impl AsLogicalPlan for LogicalPlanNode { }; let schema: protobuf::Schema = schema.as_ref().try_into()?; - let filters: Vec = filters - .iter() - .map(|filter| serialize_expr(filter, extension_codec)) - .collect::, _>>()?; + let filters: Vec = + serialize_exprs(filters, extension_codec)?; if let Some(listing_table) = source.downcast_ref::() { let any = listing_table.options().format.as_any(); @@ -1037,10 +984,7 @@ impl AsLogicalPlan for LogicalPlanNode { let mut exprs_vec: Vec = vec![]; for order in &options.file_sort_order { let expr_vec = LogicalExprNodeCollection { - logical_expr_nodes: order - .iter() - .map(|expr| serialize_expr(expr, extension_codec)) - .collect::, ToProtoError>>()?, + logical_expr_nodes: serialize_exprs(order, extension_codec)?, }; exprs_vec.push(expr_vec); } @@ -1118,10 +1062,7 @@ impl AsLogicalPlan for LogicalPlanNode { extension_codec, )?, )), - expr: expr - .iter() - .map(|expr| serialize_expr(expr, extension_codec)) - .collect::, ToProtoError>>()?, + expr: serialize_exprs(expr, extension_codec)?, optional_alias: None, }, ))), @@ -1173,22 +1114,13 @@ impl AsLogicalPlan for LogicalPlanNode { )?; let sort_expr = match sort_expr { None => vec![], - Some(sort_expr) => sort_expr - .iter() - .map(|expr| serialize_expr(expr, extension_codec)) - .collect::, _>>()?, + Some(sort_expr) => serialize_exprs(sort_expr, extension_codec)?, }; Ok(protobuf::LogicalPlanNode { logical_plan_type: Some(LogicalPlanType::DistinctOn(Box::new( protobuf::DistinctOnNode { - on_expr: on_expr - .iter() - .map(|expr| serialize_expr(expr, extension_codec)) - .collect::, _>>()?, - select_expr: select_expr - .iter() - .map(|expr| serialize_expr(expr, extension_codec)) - .collect::, _>>()?, + on_expr: serialize_exprs(on_expr, extension_codec)?, + select_expr: serialize_exprs(select_expr, extension_codec)?, sort_expr, input: Some(Box::new(input)), }, @@ -1207,10 +1139,7 @@ impl AsLogicalPlan for LogicalPlanNode { logical_plan_type: Some(LogicalPlanType::Window(Box::new( protobuf::WindowNode { input: Some(Box::new(input)), - window_expr: window_expr - .iter() - .map(|expr| serialize_expr(expr, extension_codec)) - .collect::, _>>()?, + window_expr: serialize_exprs(window_expr, extension_codec)?, }, ))), }) @@ -1230,14 +1159,8 @@ impl AsLogicalPlan for LogicalPlanNode { logical_plan_type: Some(LogicalPlanType::Aggregate(Box::new( protobuf::AggregateNode { input: Some(Box::new(input)), - group_expr: group_expr - .iter() - .map(|expr| serialize_expr(expr, extension_codec)) - .collect::, _>>()?, - aggr_expr: aggr_expr - .iter() - .map(|expr| serialize_expr(expr, extension_codec)) - .collect::, _>>()?, + group_expr: serialize_exprs(group_expr, extension_codec)?, + aggr_expr: serialize_exprs(aggr_expr, extension_codec)?, }, ))), }) @@ -1335,10 +1258,8 @@ impl AsLogicalPlan for LogicalPlanNode { input.as_ref(), extension_codec, )?; - let selection_expr: Vec = expr - .iter() - .map(|expr| serialize_expr(expr, extension_codec)) - .collect::, ToProtoError>>()?; + let selection_expr: Vec = + serialize_exprs(expr, extension_codec)?; Ok(protobuf::LogicalPlanNode { logical_plan_type: Some(LogicalPlanType::Sort(Box::new( protobuf::SortNode { @@ -1367,10 +1288,7 @@ impl AsLogicalPlan for LogicalPlanNode { let pb_partition_method = match partitioning_scheme { Partitioning::Hash(exprs, partition_count) => { PartitionMethod::Hash(protobuf::HashRepartition { - hash_expr: exprs - .iter() - .map(|expr| serialize_expr(expr, extension_codec)) - .collect::, ToProtoError>>()?, + hash_expr: serialize_exprs(exprs, extension_codec)?, partition_count: *partition_count as u64, }) } @@ -1419,10 +1337,7 @@ impl AsLogicalPlan for LogicalPlanNode { let mut converted_order_exprs: Vec = vec![]; for order in order_exprs { let temp = LogicalExprNodeCollection { - logical_expr_nodes: order - .iter() - .map(|expr| serialize_expr(expr, extension_codec)) - .collect::, ToProtoError>>()?, + logical_expr_nodes: serialize_exprs(order, extension_codec)?, }; converted_order_exprs.push(temp); } From 66bc222c53e98e65f8b6811cffbaf5de8bdf1985 Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Tue, 27 Aug 2024 15:25:44 +0200 Subject: [PATCH 037/154] Make it easier to understand datafusion-cli exclusion (#12188) --- Cargo.toml | 1 + 1 file changed, 1 insertion(+) diff --git a/Cargo.toml b/Cargo.toml index 479d2cadc65b..50d234c39576 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -16,6 +16,7 @@ # under the License. [workspace] +# datafusion-cli is excluded because of its Cargo.lock. See datafusion-cli/README.md. exclude = ["datafusion-cli", "dev/depcheck"] members = [ "datafusion/common", From 6ffb1f64a7b3f48c7693d2ecb34f72a2dca24acc Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Tue, 27 Aug 2024 17:52:01 +0200 Subject: [PATCH 038/154] Remove unnecessary clones from `.../logical_plan/builder.rs` (#12196) --- datafusion/expr/src/logical_plan/builder.rs | 29 +++++++++------------ 1 file changed, 13 insertions(+), 16 deletions(-) diff --git a/datafusion/expr/src/logical_plan/builder.rs b/datafusion/expr/src/logical_plan/builder.rs index 6abcfe4279bc..9894fe887de0 100644 --- a/datafusion/expr/src/logical_plan/builder.rs +++ b/datafusion/expr/src/logical_plan/builder.rs @@ -211,7 +211,7 @@ impl LogicalPlanBuilder { }; common_type = Some(new_type); } else { - common_type = Some(data_type.clone()); + common_type = Some(data_type); } } field_types.push(common_type.unwrap_or(DataType::Utf8)); @@ -220,7 +220,7 @@ impl LogicalPlanBuilder { for row in &mut values { for (j, field_type) in field_types.iter().enumerate() { if let Expr::Literal(ScalarValue::Null) = row[j] { - row[j] = Expr::Literal(ScalarValue::try_from(field_type.clone())?); + row[j] = Expr::Literal(ScalarValue::try_from(field_type)?); } else { row[j] = std::mem::take(&mut row[j]).cast_to(field_type, &empty_schema)?; @@ -552,20 +552,17 @@ impl LogicalPlanBuilder { // Collect sort columns that are missing in the input plan's schema let mut missing_cols: Vec = vec![]; - exprs - .clone() - .into_iter() - .try_for_each::<_, Result<()>>(|expr| { - let columns = expr.column_refs(); + exprs.iter().try_for_each::<_, Result<()>>(|expr| { + let columns = expr.column_refs(); - columns.into_iter().for_each(|c| { - if !schema.has_column(c) { - missing_cols.push(c.clone()); - } - }); + columns.into_iter().for_each(|c| { + if !schema.has_column(c) { + missing_cols.push(c.clone()); + } + }); - Ok(()) - })?; + Ok(()) + })?; if missing_cols.is_empty() { return Ok(Self::new(LogicalPlan::Sort(Sort { @@ -710,7 +707,7 @@ impl LogicalPlanBuilder { pub(crate) fn normalize( plan: &LogicalPlan, - column: impl Into + Clone, + column: impl Into, ) -> Result { let schema = plan.schema(); let fallback_schemas = plan.fallback_normalize_schemas(); @@ -1536,7 +1533,7 @@ pub fn get_unnested_columns( | DataType::FixedSizeList(field, _) | DataType::LargeList(field) => { let new_field = Arc::new(Field::new( - col_name.clone(), + col_name, field.data_type().clone(), // Unnesting may produce NULLs even if the list is not null. // For example: unnset([1], []) -> 1, null From 8ba6732af5f4f32cbe0a23ef6bc2f393c640898b Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Wed, 28 Aug 2024 09:47:41 +0200 Subject: [PATCH 039/154] Remove unnecessary clones with clippy (#12197) * Remove unnecessary clones with clippy This is automated change done with ``` cargo clippy --fix -- -Aclippy::all -Wclippy::redundant_clone cargo fmt # manually fix few new clippy errors introduced by clippy itself ``` Note: it doesn't remove all unnecessary clones because the command reported error and backed out for the common crate. * Prevent clippy from removing intentional clone clippy can be run with `--fix` and then it won't obey the code comment instructing not to delete the clone. * Remove unnecessary clones pointed out by clippy Change code as instructed by ``` cargo clippy --fix -- -Aclippy::all -Wclippy::redundant_clone ``` where clippy didn't apply the suggested changes by itself. --- .../examples/advanced_parquet_index.rs | 2 +- .../examples/file_stream_provider.rs | 2 +- datafusion/common/src/dfschema.rs | 3 +- .../common/src/file_options/parquet_writer.rs | 2 +- datafusion/common/src/scalar/mod.rs | 8 ++-- datafusion/common/src/stats.rs | 1 + .../core/src/datasource/listing/helpers.rs | 2 +- .../core/src/datasource/listing/table.rs | 2 +- .../datasource/physical_plan/file_groups.rs | 7 +--- .../physical_plan/file_scan_config.rs | 4 +- .../core/src/datasource/schema_adapter.rs | 2 +- datafusion/core/src/execution/context/mod.rs | 2 +- .../enforce_distribution.rs | 23 +++++------ .../src/physical_optimizer/join_selection.rs | 21 +++++----- .../physical_optimizer/projection_pushdown.rs | 18 +++------ .../core/src/physical_optimizer/pruning.rs | 1 - datafusion/core/src/physical_planner.rs | 4 +- datafusion/core/tests/fuzz_cases/join_fuzz.rs | 10 ++--- .../physical_optimizer/limit_pushdown.rs | 16 ++++---- .../limited_distinct_aggregation.rs | 22 +++++----- .../user_defined/user_defined_aggregates.rs | 2 +- .../user_defined_scalar_functions.rs | 2 +- datafusion/execution/src/memory_pool/pool.rs | 4 +- .../expr-common/src/interval_arithmetic.rs | 10 ++--- .../expr-common/src/type_coercion/binary.rs | 2 +- datafusion/expr/src/expr_schema.rs | 4 +- datafusion/expr/src/logical_plan/builder.rs | 2 +- datafusion/expr/src/logical_plan/display.rs | 20 ++++------ .../aggregate/groups_accumulator/prim_op.rs | 2 +- .../src/approx_percentile_cont.rs | 4 +- .../functions-aggregate/src/array_agg.rs | 7 +--- .../functions-aggregate/src/nth_value.rs | 7 +--- datafusion/functions-nested/src/flatten.rs | 2 +- datafusion/functions-nested/src/utils.rs | 3 +- .../optimizer/src/common_subexpr_eliminate.rs | 12 +++--- .../optimizer/src/eliminate_nested_union.rs | 17 +++----- .../optimizer/src/eliminate_one_union.rs | 5 +-- .../optimizer/src/optimize_projections/mod.rs | 4 +- datafusion/optimizer/src/push_down_filter.rs | 2 +- .../simplify_expressions/expr_simplifier.rs | 26 ++++++------ .../src/simplify_expressions/guarantees.rs | 4 +- .../src/unwrap_cast_in_comparison.rs | 2 +- .../physical-expr/src/equivalence/ordering.rs | 2 +- .../physical-expr/src/expressions/binary.rs | 4 +- .../physical-expr/src/expressions/case.rs | 2 +- .../physical-plan/src/aggregates/mod.rs | 2 +- datafusion/physical-plan/src/joins/utils.rs | 2 +- .../physical-plan/src/recursive_query.rs | 2 +- .../src/windows/bounded_window_agg_exec.rs | 4 +- datafusion/proto/gen/src/main.rs | 2 +- .../tests/cases/roundtrip_logical_plan.rs | 2 +- .../tests/cases/roundtrip_physical_plan.rs | 40 +++++++++---------- datafusion/sql/src/expr/function.rs | 3 +- datafusion/sql/src/relation/mod.rs | 2 +- datafusion/sql/src/unparser/expr.rs | 2 +- datafusion/sql/tests/cases/plan_to_sql.rs | 2 +- 56 files changed, 162 insertions(+), 204 deletions(-) diff --git a/datafusion-examples/examples/advanced_parquet_index.rs b/datafusion-examples/examples/advanced_parquet_index.rs index 903defafe3ab..f6860bb5b87a 100644 --- a/datafusion-examples/examples/advanced_parquet_index.rs +++ b/datafusion-examples/examples/advanced_parquet_index.rs @@ -300,7 +300,7 @@ impl IndexTableProvider { // analyze the predicate. In a real system, using // `PruningPredicate::prune` would likely be easier to do. let pruning_predicate = - PruningPredicate::try_new(Arc::clone(predicate), self.schema().clone())?; + PruningPredicate::try_new(Arc::clone(predicate), self.schema())?; // The PruningPredicate's guarantees must all be satisfied in order for // the predicate to possibly evaluate to true. diff --git a/datafusion-examples/examples/file_stream_provider.rs b/datafusion-examples/examples/file_stream_provider.rs index b8549bd6b6e6..4db7e0200f53 100644 --- a/datafusion-examples/examples/file_stream_provider.rs +++ b/datafusion-examples/examples/file_stream_provider.rs @@ -100,7 +100,7 @@ mod non_windows { ) { // Timeout for a long period of BrokenPipe error let broken_pipe_timeout = Duration::from_secs(10); - let sa = file_path.clone(); + let sa = file_path; // Spawn a new thread to write to the FIFO file #[allow(clippy::disallowed_methods)] // spawn allowed only in tests tasks.spawn_blocking(move || { diff --git a/datafusion/common/src/dfschema.rs b/datafusion/common/src/dfschema.rs index f0eecd2ffeb1..095f4c510194 100644 --- a/datafusion/common/src/dfschema.rs +++ b/datafusion/common/src/dfschema.rs @@ -1242,10 +1242,9 @@ mod tests { #[test] fn into() { // Demonstrate how to convert back and forth between Schema, SchemaRef, DFSchema, and DFSchemaRef - let metadata = test_metadata(); let arrow_schema = Schema::new_with_metadata( vec![Field::new("c0", DataType::Int64, true)], - metadata.clone(), + test_metadata(), ); let arrow_schema_ref = Arc::new(arrow_schema.clone()); diff --git a/datafusion/common/src/file_options/parquet_writer.rs b/datafusion/common/src/file_options/parquet_writer.rs index 4a229fe01b54..e42fb96ed6a5 100644 --- a/datafusion/common/src/file_options/parquet_writer.rs +++ b/datafusion/common/src/file_options/parquet_writer.rs @@ -565,7 +565,7 @@ mod tests { column_options_with_non_defaults(&parquet_options), )] .into(), - key_value_metadata: [(key.clone(), value.clone())].into(), + key_value_metadata: [(key, value)].into(), }; let writer_props = WriterPropertiesBuilder::try_from(&table_parquet_opts) diff --git a/datafusion/common/src/scalar/mod.rs b/datafusion/common/src/scalar/mod.rs index 677685b2c65b..5acc2b6f188e 100644 --- a/datafusion/common/src/scalar/mod.rs +++ b/datafusion/common/src/scalar/mod.rs @@ -4905,7 +4905,7 @@ mod tests { 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!(non_null_list_scalar.data_type(), data_type); assert_eq!(null_list_scalar.data_type(), data_type); } @@ -5582,13 +5582,13 @@ mod tests { // Define list-of-structs scalars - let nl0_array = ScalarValue::iter_to_array(vec![s0.clone(), s1.clone()]).unwrap(); + let nl0_array = ScalarValue::iter_to_array(vec![s0, s1.clone()]).unwrap(); let nl0 = ScalarValue::List(Arc::new(array_into_list_array_nullable(nl0_array))); - let nl1_array = ScalarValue::iter_to_array(vec![s2.clone()]).unwrap(); + let nl1_array = ScalarValue::iter_to_array(vec![s2]).unwrap(); let nl1 = ScalarValue::List(Arc::new(array_into_list_array_nullable(nl1_array))); - let nl2_array = ScalarValue::iter_to_array(vec![s1.clone()]).unwrap(); + let nl2_array = ScalarValue::iter_to_array(vec![s1]).unwrap(); let nl2 = ScalarValue::List(Arc::new(array_into_list_array_nullable(nl2_array))); // iter_to_array for list-of-struct diff --git a/datafusion/common/src/stats.rs b/datafusion/common/src/stats.rs index 50ae4e3ca71f..d8e62b3045f9 100644 --- a/datafusion/common/src/stats.rs +++ b/datafusion/common/src/stats.rs @@ -557,6 +557,7 @@ mod tests { let precision: Precision = Precision::Exact(ScalarValue::Int64(Some(42))); // Clippy would complain about this if it were Copy + #[allow(clippy::redundant_clone)] let p2 = precision.clone(); assert_eq!(precision, p2); } diff --git a/datafusion/core/src/datasource/listing/helpers.rs b/datafusion/core/src/datasource/listing/helpers.rs index b5dd2dd12e10..f6e938b72dab 100644 --- a/datafusion/core/src/datasource/listing/helpers.rs +++ b/datafusion/core/src/datasource/listing/helpers.rs @@ -282,7 +282,7 @@ async fn prune_partitions( Default::default(), )?; - let batch = RecordBatch::try_new(schema.clone(), arrays)?; + let batch = RecordBatch::try_new(schema, arrays)?; // TODO: Plumb this down let props = ExecutionProps::new(); diff --git a/datafusion/core/src/datasource/listing/table.rs b/datafusion/core/src/datasource/listing/table.rs index 89066d8234ac..a0345a38e40c 100644 --- a/datafusion/core/src/datasource/listing/table.rs +++ b/datafusion/core/src/datasource/listing/table.rs @@ -1016,7 +1016,7 @@ impl ListingTable { .collected_statistics .get_with_extra(&part_file.object_meta.location, &part_file.object_meta) { - Some(statistics) => Ok(statistics.clone()), + Some(statistics) => Ok(statistics), None => { let statistics = self .options diff --git a/datafusion/core/src/datasource/physical_plan/file_groups.rs b/datafusion/core/src/datasource/physical_plan/file_groups.rs index fb2cd4ad06ec..28f975ae193d 100644 --- a/datafusion/core/src/datasource/physical_plan/file_groups.rs +++ b/datafusion/core/src/datasource/physical_plan/file_groups.rs @@ -394,7 +394,7 @@ mod test { #[test] fn repartition_empty_file_only() { let partitioned_file_empty = pfile("empty", 0); - let file_group = vec![vec![partitioned_file_empty.clone()]]; + let file_group = vec![vec![partitioned_file_empty]]; let partitioned_files = FileGroupPartitioner::new() .with_target_partitions(4) @@ -817,10 +817,7 @@ mod test { .with_preserve_order_within_groups(true) .repartition_file_groups(&file_groups); - assert_partitioned_files( - repartitioned.clone(), - repartitioned_preserving_sort.clone(), - ); + assert_partitioned_files(repartitioned.clone(), repartitioned_preserving_sort); repartitioned } } diff --git a/datafusion/core/src/datasource/physical_plan/file_scan_config.rs b/datafusion/core/src/datasource/physical_plan/file_scan_config.rs index 34fb6226c1a2..bfa5488e5b5e 100644 --- a/datafusion/core/src/datasource/physical_plan/file_scan_config.rs +++ b/datafusion/core/src/datasource/physical_plan/file_scan_config.rs @@ -908,7 +908,7 @@ mod tests { schema.clone(), Some(vec![0, 3, 5, schema.fields().len()]), Statistics::new_unknown(&schema), - to_partition_cols(partition_cols.clone()), + to_partition_cols(partition_cols), ) .projected_file_schema(); @@ -941,7 +941,7 @@ mod tests { schema.clone(), None, Statistics::new_unknown(&schema), - to_partition_cols(partition_cols.clone()), + to_partition_cols(partition_cols), ) .projected_file_schema(); diff --git a/datafusion/core/src/datasource/schema_adapter.rs b/datafusion/core/src/datasource/schema_adapter.rs index 40cb40a83af2..5d2d0ff91b15 100644 --- a/datafusion/core/src/datasource/schema_adapter.rs +++ b/datafusion/core/src/datasource/schema_adapter.rs @@ -369,7 +369,7 @@ mod tests { let f1 = Field::new("id", DataType::Int32, true); let f2 = Field::new("extra_column", DataType::Utf8, true); - let schema = Arc::new(Schema::new(vec![f1.clone(), f2.clone()])); + let schema = Arc::new(Schema::new(vec![f1, f2])); let extra_column = Arc::new(StringArray::from(vec!["foo"])); let mut new_columns = batch.columns().to_vec(); diff --git a/datafusion/core/src/execution/context/mod.rs b/datafusion/core/src/execution/context/mod.rs index c2a072eef756..18943599b136 100644 --- a/datafusion/core/src/execution/context/mod.rs +++ b/datafusion/core/src/execution/context/mod.rs @@ -1165,7 +1165,7 @@ impl SessionContext { // check schema uniqueness let mut batches = batches.into_iter().peekable(); let schema = if let Some(batch) = batches.peek() { - batch.schema().clone() + batch.schema() } else { Arc::new(Schema::empty()) }; diff --git a/datafusion/core/src/physical_optimizer/enforce_distribution.rs b/datafusion/core/src/physical_optimizer/enforce_distribution.rs index 77ddd098c7be..ba6f7d0439c2 100644 --- a/datafusion/core/src/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/src/physical_optimizer/enforce_distribution.rs @@ -3908,7 +3908,7 @@ pub(crate) mod tests { let alias = vec![("a".to_string(), "a".to_string())]; let plan_parquet = aggregate_exec_with_alias(parquet_exec_multiple(), alias.clone()); - let plan_csv = aggregate_exec_with_alias(csv_exec_multiple(), alias.clone()); + let plan_csv = aggregate_exec_with_alias(csv_exec_multiple(), alias); let expected_parquet = [ "AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[]", @@ -3934,7 +3934,7 @@ pub(crate) mod tests { let alias = vec![("a".to_string(), "a".to_string())]; let plan_parquet = aggregate_exec_with_alias(parquet_exec_multiple(), alias.clone()); - let plan_csv = aggregate_exec_with_alias(csv_exec_multiple(), alias.clone()); + let plan_csv = aggregate_exec_with_alias(csv_exec_multiple(), alias); let expected_parquet = [ "AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[]", @@ -3964,7 +3964,7 @@ pub(crate) mod tests { options: SortOptions::default(), }]; let plan_parquet = limit_exec(sort_exec(sort_key.clone(), parquet_exec(), false)); - let plan_csv = limit_exec(sort_exec(sort_key.clone(), csv_exec(), false)); + let plan_csv = limit_exec(sort_exec(sort_key, csv_exec(), false)); let expected_parquet = &[ "GlobalLimitExec: skip=0, fetch=100", @@ -4000,8 +4000,7 @@ pub(crate) mod tests { parquet_exec(), false, ))); - let plan_csv = - limit_exec(filter_exec(sort_exec(sort_key.clone(), csv_exec(), false))); + let plan_csv = limit_exec(filter_exec(sort_exec(sort_key, csv_exec(), false))); let expected_parquet = &[ "GlobalLimitExec: skip=0, fetch=100", @@ -4042,7 +4041,7 @@ pub(crate) mod tests { ); let plan_csv = aggregate_exec_with_alias( limit_exec(filter_exec(limit_exec(csv_exec()))), - alias.clone(), + alias, ); let expected_parquet = &[ @@ -4126,7 +4125,7 @@ pub(crate) mod tests { ); let plan_csv = sort_preserving_merge_exec( sort_key.clone(), - csv_exec_with_sort(vec![sort_key.clone()]), + csv_exec_with_sort(vec![sort_key]), ); // parallelization is not beneficial for SortPreservingMerge @@ -4154,7 +4153,7 @@ pub(crate) mod tests { union_exec(vec![parquet_exec_with_sort(vec![sort_key.clone()]); 2]); let input_csv = union_exec(vec![csv_exec_with_sort(vec![sort_key.clone()]); 2]); let plan_parquet = sort_preserving_merge_exec(sort_key.clone(), input_parquet); - let plan_csv = sort_preserving_merge_exec(sort_key.clone(), input_csv); + let plan_csv = sort_preserving_merge_exec(sort_key, input_csv); // should not repartition (union doesn't benefit from increased parallelism) // should not sort (as the data was already sorted) @@ -4224,8 +4223,8 @@ pub(crate) mod tests { ("c".to_string(), "c2".to_string()), ]; let proj_parquet = projection_exec_with_alias( - parquet_exec_with_sort(vec![sort_key.clone()]), - alias_pairs.clone(), + parquet_exec_with_sort(vec![sort_key]), + alias_pairs, ); let sort_key_after_projection = vec![PhysicalSortExpr { expr: col("c2", &proj_parquet.schema()).unwrap(), @@ -4560,7 +4559,7 @@ pub(crate) mod tests { }]; let alias = vec![("a".to_string(), "a".to_string())]; let input = parquet_exec_with_sort(vec![sort_key]); - let physical_plan = aggregate_exec_with_alias(input, alias.clone()); + let physical_plan = aggregate_exec_with_alias(input, alias); let expected = &[ "AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[]", @@ -4584,7 +4583,7 @@ pub(crate) mod tests { let alias = vec![("a".to_string(), "a".to_string())]; let input = parquet_exec_multiple_sorted(vec![sort_key]); let aggregate = aggregate_exec_with_alias(input, alias.clone()); - let physical_plan = aggregate_exec_with_alias(aggregate, alias.clone()); + let physical_plan = aggregate_exec_with_alias(aggregate, alias); let expected = &[ "AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[]", diff --git a/datafusion/core/src/physical_optimizer/join_selection.rs b/datafusion/core/src/physical_optimizer/join_selection.rs index b849df88e4aa..2643ade8f481 100644 --- a/datafusion/core/src/physical_optimizer/join_selection.rs +++ b/datafusion/core/src/physical_optimizer/join_selection.rs @@ -908,7 +908,7 @@ mod tests_statistical { ); let optimized_join = JoinSelection::new() - .optimize(join.clone(), &ConfigOptions::new()) + .optimize(join, &ConfigOptions::new()) .unwrap(); let swapping_projection = optimized_join @@ -964,7 +964,7 @@ mod tests_statistical { ); let optimized_join = JoinSelection::new() - .optimize(join.clone(), &ConfigOptions::new()) + .optimize(join, &ConfigOptions::new()) .unwrap(); let swapped_join = optimized_join @@ -1140,7 +1140,7 @@ mod tests_statistical { ); let optimized_join = JoinSelection::new() - .optimize(join.clone(), &ConfigOptions::new()) + .optimize(join, &ConfigOptions::new()) .unwrap(); let swapped_join = optimized_join @@ -1180,7 +1180,7 @@ mod tests_statistical { ); let optimized_join = JoinSelection::new() - .optimize(join.clone(), &ConfigOptions::new()) + .optimize(join, &ConfigOptions::new()) .unwrap(); let swapping_projection = optimized_join @@ -1356,7 +1356,7 @@ mod tests_statistical { Arc::new(Column::new_with_schema("small_col", &small.schema()).unwrap()) as _, )]; check_join_partition_mode( - big.clone(), + big, small.clone(), join_on, true, @@ -1380,8 +1380,8 @@ mod tests_statistical { Arc::new(Column::new_with_schema("small_col", &small.schema()).unwrap()) as _, )]; check_join_partition_mode( - empty.clone(), - small.clone(), + empty, + small, join_on, true, PartitionMode::CollectLeft, @@ -1424,7 +1424,7 @@ mod tests_statistical { Arc::new(Column::new_with_schema("big_col", &big.schema()).unwrap()) as _, )]; check_join_partition_mode( - bigger.clone(), + bigger, big.clone(), join_on, true, @@ -1472,7 +1472,7 @@ mod tests_statistical { ); let optimized_join = JoinSelection::new() - .optimize(join.clone(), &ConfigOptions::new()) + .optimize(join, &ConfigOptions::new()) .unwrap(); if !is_swapped { @@ -1913,8 +1913,7 @@ mod hash_join_tests { false, )?); - let optimized_join_plan = - hash_join_swap_subrule(join.clone(), &ConfigOptions::new())?; + let optimized_join_plan = hash_join_swap_subrule(join, &ConfigOptions::new())?; // If swap did happen let projection_added = optimized_join_plan.as_any().is::(); diff --git a/datafusion/core/src/physical_optimizer/projection_pushdown.rs b/datafusion/core/src/physical_optimizer/projection_pushdown.rs index 9c545c17da3c..b3f3f90154d0 100644 --- a/datafusion/core/src/physical_optimizer/projection_pushdown.rs +++ b/datafusion/core/src/physical_optimizer/projection_pushdown.rs @@ -1692,12 +1692,9 @@ mod tests { ])); Arc::new( CsvExec::builder( - FileScanConfig::new( - ObjectStoreUrl::parse("test:///").unwrap(), - schema.clone(), - ) - .with_file(PartitionedFile::new("x".to_string(), 100)) - .with_projection(Some(vec![0, 1, 2, 3, 4])), + FileScanConfig::new(ObjectStoreUrl::parse("test:///").unwrap(), schema) + .with_file(PartitionedFile::new("x".to_string(), 100)) + .with_projection(Some(vec![0, 1, 2, 3, 4])), ) .with_has_header(false) .with_delimeter(0) @@ -1719,12 +1716,9 @@ mod tests { ])); Arc::new( CsvExec::builder( - FileScanConfig::new( - ObjectStoreUrl::parse("test:///").unwrap(), - schema.clone(), - ) - .with_file(PartitionedFile::new("x".to_string(), 100)) - .with_projection(Some(vec![3, 2, 1])), + FileScanConfig::new(ObjectStoreUrl::parse("test:///").unwrap(), schema) + .with_file(PartitionedFile::new("x".to_string(), 100)) + .with_projection(Some(vec![3, 2, 1])), ) .with_has_header(false) .with_delimeter(0) diff --git a/datafusion/core/src/physical_optimizer/pruning.rs b/datafusion/core/src/physical_optimizer/pruning.rs index 0ef390fff45c..a16abc607ee6 100644 --- a/datafusion/core/src/physical_optimizer/pruning.rs +++ b/datafusion/core/src/physical_optimizer/pruning.rs @@ -1369,7 +1369,6 @@ fn build_predicate_expression( let change_expr = in_list .list() .iter() - .cloned() .map(|e| { Arc::new(phys_expr::BinaryExpr::new( in_list.expr().clone(), diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index 9501d3c6bbbb..fe8d79846630 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -701,7 +701,7 @@ impl DefaultPhysicalPlanner { let initial_aggr = Arc::new(AggregateExec::try_new( AggregateMode::Partial, groups.clone(), - aggregates.clone(), + aggregates, filters.clone(), input_exec, physical_input_schema.clone(), @@ -2569,7 +2569,7 @@ mod tests { impl NoOpExecutionPlan { fn new(schema: SchemaRef) -> Self { - let cache = Self::compute_properties(schema.clone()); + let cache = Self::compute_properties(schema); Self { cache } } diff --git a/datafusion/core/tests/fuzz_cases/join_fuzz.rs b/datafusion/core/tests/fuzz_cases/join_fuzz.rs index f1cca66712d7..1c2d8ece2f36 100644 --- a/datafusion/core/tests/fuzz_cases/join_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/join_fuzz.rs @@ -350,12 +350,10 @@ impl JoinFuzzTestCase { fn left_right(&self) -> (Arc, Arc) { let schema1 = self.input1[0].schema(); let schema2 = self.input2[0].schema(); - let left = Arc::new( - MemoryExec::try_new(&[self.input1.clone()], schema1.clone(), None).unwrap(), - ); - let right = Arc::new( - MemoryExec::try_new(&[self.input2.clone()], schema2.clone(), None).unwrap(), - ); + let left = + Arc::new(MemoryExec::try_new(&[self.input1.clone()], schema1, None).unwrap()); + let right = + Arc::new(MemoryExec::try_new(&[self.input2.clone()], schema2, None).unwrap()); (left, right) } diff --git a/datafusion/core/tests/physical_optimizer/limit_pushdown.rs b/datafusion/core/tests/physical_optimizer/limit_pushdown.rs index b051feb5750e..750544ecdec1 100644 --- a/datafusion/core/tests/physical_optimizer/limit_pushdown.rs +++ b/datafusion/core/tests/physical_optimizer/limit_pushdown.rs @@ -54,7 +54,7 @@ impl PartitionStream for DummyStreamPartition { fn transforms_streaming_table_exec_into_fetching_version_when_skip_is_zero( ) -> datafusion_common::Result<()> { let schema = create_schema(); - let streaming_table = streaming_table_exec(schema.clone())?; + let streaming_table = streaming_table_exec(schema)?; let global_limit = global_limit_exec(streaming_table, 0, Some(5)); let initial = get_plan_string(&global_limit); @@ -79,7 +79,7 @@ fn transforms_streaming_table_exec_into_fetching_version_when_skip_is_zero( fn transforms_streaming_table_exec_into_fetching_version_and_keeps_the_global_limit_when_skip_is_nonzero( ) -> datafusion_common::Result<()> { let schema = create_schema(); - let streaming_table = streaming_table_exec(schema.clone())?; + let streaming_table = streaming_table_exec(schema)?; let global_limit = global_limit_exec(streaming_table, 2, Some(5)); let initial = get_plan_string(&global_limit); @@ -107,7 +107,7 @@ fn transforms_coalesce_batches_exec_into_fetching_version_and_removes_local_limi let schema = create_schema(); let streaming_table = streaming_table_exec(schema.clone())?; let repartition = repartition_exec(streaming_table)?; - let filter = filter_exec(schema.clone(), repartition)?; + let filter = filter_exec(schema, repartition)?; let coalesce_batches = coalesce_batches_exec(filter); let local_limit = local_limit_exec(coalesce_batches, 5); let coalesce_partitions = coalesce_partitions_exec(local_limit); @@ -146,7 +146,7 @@ fn pushes_global_limit_exec_through_projection_exec() -> datafusion_common::Resu let schema = create_schema(); let streaming_table = streaming_table_exec(schema.clone())?; let filter = filter_exec(schema.clone(), streaming_table)?; - let projection = projection_exec(schema.clone(), filter)?; + let projection = projection_exec(schema, filter)?; let global_limit = global_limit_exec(projection, 0, Some(5)); let initial = get_plan_string(&global_limit); @@ -178,7 +178,7 @@ fn pushes_global_limit_exec_through_projection_exec_and_transforms_coalesce_batc let schema = create_schema(); let streaming_table = streaming_table_exec(schema.clone()).unwrap(); let coalesce_batches = coalesce_batches_exec(streaming_table); - let projection = projection_exec(schema.clone(), coalesce_batches)?; + let projection = projection_exec(schema, coalesce_batches)?; let global_limit = global_limit_exec(projection, 0, Some(5)); let initial = get_plan_string(&global_limit); @@ -256,7 +256,7 @@ fn keeps_pushed_local_limit_exec_when_there_are_multiple_input_partitions( let schema = create_schema(); let streaming_table = streaming_table_exec(schema.clone())?; let repartition = repartition_exec(streaming_table)?; - let filter = filter_exec(schema.clone(), repartition)?; + let filter = filter_exec(schema, repartition)?; let coalesce_partitions = coalesce_partitions_exec(filter); let global_limit = global_limit_exec(coalesce_partitions, 0, Some(5)); @@ -398,9 +398,7 @@ fn streaming_table_exec( ) -> datafusion_common::Result> { Ok(Arc::new(StreamingTableExec::try_new( schema.clone(), - vec![Arc::new(DummyStreamPartition { - schema: schema.clone(), - }) as _], + vec![Arc::new(DummyStreamPartition { schema }) as _], None, None, true, diff --git a/datafusion/core/tests/physical_optimizer/limited_distinct_aggregation.rs b/datafusion/core/tests/physical_optimizer/limited_distinct_aggregation.rs index 48389b0304f6..042f6d622565 100644 --- a/datafusion/core/tests/physical_optimizer/limited_distinct_aggregation.rs +++ b/datafusion/core/tests/physical_optimizer/limited_distinct_aggregation.rs @@ -316,11 +316,11 @@ fn test_no_group_by() -> Result<()> { // `SELECT FROM MemoryExec LIMIT 10;`, Single AggregateExec let single_agg = AggregateExec::try_new( AggregateMode::Single, - build_group_by(&schema.clone(), vec![]), - vec![], /* aggr_expr */ - vec![], /* filter_expr */ - source, /* input */ - schema.clone(), /* input_schema */ + build_group_by(&schema, vec![]), + vec![], /* aggr_expr */ + vec![], /* filter_expr */ + source, /* input */ + schema, /* input_schema */ )?; let limit_exec = LocalLimitExec::new( Arc::new(single_agg), @@ -346,7 +346,7 @@ fn test_has_aggregate_expression() -> Result<()> { // `SELECT FROM MemoryExec LIMIT 10;`, Single AggregateExec let single_agg = AggregateExec::try_new( AggregateMode::Single, - build_group_by(&schema.clone(), vec!["a".to_string()]), + build_group_by(&schema, vec!["a".to_string()]), vec![agg.count_expr(&schema)], /* aggr_expr */ vec![None], /* filter_expr */ source, /* input */ @@ -418,11 +418,11 @@ fn test_has_order_by() -> Result<()> { // the `a > 1` filter is applied in the AggregateExec let single_agg = AggregateExec::try_new( AggregateMode::Single, - build_group_by(&schema.clone(), vec!["a".to_string()]), - vec![], /* aggr_expr */ - vec![], /* filter_expr */ - source, /* input */ - schema.clone(), /* input_schema */ + build_group_by(&schema, vec!["a".to_string()]), + vec![], /* aggr_expr */ + vec![], /* filter_expr */ + source, /* input */ + schema, /* input_schema */ )?; let limit_exec = LocalLimitExec::new( Arc::new(single_agg), diff --git a/datafusion/core/tests/user_defined/user_defined_aggregates.rs b/datafusion/core/tests/user_defined/user_defined_aggregates.rs index 93550d38021a..1e0d3d9d514e 100644 --- a/datafusion/core/tests/user_defined/user_defined_aggregates.rs +++ b/datafusion/core/tests/user_defined/user_defined_aggregates.rs @@ -272,7 +272,7 @@ async fn deregister_udaf() -> Result<()> { Arc::new(vec![DataType::UInt64, DataType::Float64]), ); - ctx.register_udaf(my_avg.clone()); + ctx.register_udaf(my_avg); assert!(ctx.state().aggregate_functions().contains_key("my_avg")); diff --git a/datafusion/core/tests/user_defined/user_defined_scalar_functions.rs b/datafusion/core/tests/user_defined/user_defined_scalar_functions.rs index 259cce74f2e5..0f1c3b8e53c4 100644 --- a/datafusion/core/tests/user_defined/user_defined_scalar_functions.rs +++ b/datafusion/core/tests/user_defined/user_defined_scalar_functions.rs @@ -594,7 +594,7 @@ async fn deregister_udf() -> Result<()> { let cast2i64 = ScalarUDF::from(CastToI64UDF::new()); let ctx = SessionContext::new(); - ctx.register_udf(cast2i64.clone()); + ctx.register_udf(cast2i64); assert!(ctx.udfs().contains("cast_to_i64")); diff --git a/datafusion/execution/src/memory_pool/pool.rs b/datafusion/execution/src/memory_pool/pool.rs index d3cd93979baf..e169c1f319cc 100644 --- a/datafusion/execution/src/memory_pool/pool.rs +++ b/datafusion/execution/src/memory_pool/pool.rs @@ -366,7 +366,7 @@ impl MemoryPool for TrackConsumersPool { // wrap OOM message in top consumers DataFusionError::ResourcesExhausted( provide_top_memory_consumers_to_error_msg( - e.to_owned(), + e, self.report_top(self.top.into()), ), ) @@ -540,7 +540,7 @@ mod tests { // Test: will be the same per Top Consumers reported. r0.grow(10); // make r0=10, pool available=90 let new_consumer_same_name = MemoryConsumer::new(same_name); - let mut r1 = new_consumer_same_name.clone().register(&pool); + let mut r1 = new_consumer_same_name.register(&pool); // TODO: the insufficient_capacity_err() message is per reservation, not per consumer. // a followup PR will clarify this message "0 bytes already allocated for this reservation" let expected = "Additional allocation failed with top memory consumers (across reservations) as: foo consumed 10 bytes. Error: Failed to allocate additional 150 bytes for foo with 0 bytes already allocated for this reservation - 90 bytes remain available for the total pool"; diff --git a/datafusion/expr-common/src/interval_arithmetic.rs b/datafusion/expr-common/src/interval_arithmetic.rs index e3ff412e785b..6424888c896a 100644 --- a/datafusion/expr-common/src/interval_arithmetic.rs +++ b/datafusion/expr-common/src/interval_arithmetic.rs @@ -1877,11 +1877,7 @@ mod tests { .sub(value.clone()) .unwrap() .lt(&eps)); - assert!(value - .clone() - .sub(prev_value(value.clone())) - .unwrap() - .lt(&eps)); + assert!(value.sub(prev_value(value.clone())).unwrap().lt(&eps)); assert_ne!(next_value(value.clone()), value); assert_ne!(prev_value(value.clone()), value); }); @@ -1913,11 +1909,11 @@ mod tests { min_max.into_iter().zip(inf).for_each(|((min, max), inf)| { assert_eq!(next_value(max.clone()), inf); assert_ne!(prev_value(max.clone()), max); - assert_ne!(prev_value(max.clone()), inf); + assert_ne!(prev_value(max), inf); assert_eq!(prev_value(min.clone()), inf); assert_ne!(next_value(min.clone()), min); - assert_ne!(next_value(min.clone()), inf); + assert_ne!(next_value(min), inf); assert_eq!(next_value(inf.clone()), inf); assert_eq!(prev_value(inf.clone()), inf); diff --git a/datafusion/expr-common/src/type_coercion/binary.rs b/datafusion/expr-common/src/type_coercion/binary.rs index 552bc89b38f4..3617f56905a9 100644 --- a/datafusion/expr-common/src/type_coercion/binary.rs +++ b/datafusion/expr-common/src/type_coercion/binary.rs @@ -1881,7 +1881,7 @@ mod tests { ); test_coercion_binary_rule!( DataType::Timestamp(TimeUnit::Second, Some("Europe/Brussels".into())), - DataType::Timestamp(TimeUnit::Second, utc.clone()), + DataType::Timestamp(TimeUnit::Second, utc), Operator::Eq, DataType::Timestamp(TimeUnit::Second, Some("Europe/Brussels".into())) ); diff --git a/datafusion/expr/src/expr_schema.rs b/datafusion/expr/src/expr_schema.rs index 10ec10e61239..3920a1a3517c 100644 --- a/datafusion/expr/src/expr_schema.rs +++ b/datafusion/expr/src/expr_schema.rs @@ -184,7 +184,7 @@ impl ExprSchemable for Expr { err, utils::generate_signature_error_msg( fun.name(), - fun.signature().clone(), + fun.signature(), &data_types ) ) @@ -199,7 +199,7 @@ impl ExprSchemable for Expr { err, utils::generate_signature_error_msg( fun.name(), - fun.signature().clone(), + fun.signature(), &data_types ) ) diff --git a/datafusion/expr/src/logical_plan/builder.rs b/datafusion/expr/src/logical_plan/builder.rs index 9894fe887de0..2c2300b123c2 100644 --- a/datafusion/expr/src/logical_plan/builder.rs +++ b/datafusion/expr/src/logical_plan/builder.rs @@ -2072,7 +2072,7 @@ mod tests { let schema = Schema::new(vec![ Field::new("scalar", DataType::UInt32, false), Field::new_list("strings", string_field, false), - Field::new_list("structs", struct_field_in_list.clone(), false), + Field::new_list("structs", struct_field_in_list, false), Field::new( "struct_singular", DataType::Struct(Fields::from(vec![ diff --git a/datafusion/expr/src/logical_plan/display.rs b/datafusion/expr/src/logical_plan/display.rs index 343eda056ffe..5a881deb54e1 100644 --- a/datafusion/expr/src/logical_plan/display.rs +++ b/datafusion/expr/src/logical_plan/display.rs @@ -387,19 +387,16 @@ impl<'a, 'b> PgJsonVisitor<'a, 'b> { } if !full_filter.is_empty() { - object["Full Filters"] = serde_json::Value::String( - expr_vec_fmt!(full_filter).to_string(), - ); + object["Full Filters"] = + serde_json::Value::String(expr_vec_fmt!(full_filter)); }; if !partial_filter.is_empty() { - object["Partial Filters"] = serde_json::Value::String( - expr_vec_fmt!(partial_filter).to_string(), - ); + object["Partial Filters"] = + serde_json::Value::String(expr_vec_fmt!(partial_filter)); } if !unsupported_filters.is_empty() { - object["Unsupported Filters"] = serde_json::Value::String( - expr_vec_fmt!(unsupported_filters).to_string(), - ); + object["Unsupported Filters"] = + serde_json::Value::String(expr_vec_fmt!(unsupported_filters)); } } @@ -595,9 +592,8 @@ impl<'a, 'b> PgJsonVisitor<'a, 'b> { "Select": expr_vec_fmt!(select_expr), }); if let Some(sort_expr) = sort_expr { - object["Sort"] = serde_json::Value::String( - expr_vec_fmt!(sort_expr).to_string(), - ); + object["Sort"] = + serde_json::Value::String(expr_vec_fmt!(sort_expr)); } object diff --git a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/prim_op.rs b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/prim_op.rs index b5c6171af37c..8bbcf756c37c 100644 --- a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/prim_op.rs +++ b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/prim_op.rs @@ -167,7 +167,7 @@ where // Rebuilding input values with a new nulls mask, which is equal to // the union of original nulls and filter mask - let (dt, values_buf, original_nulls) = values.clone().into_parts(); + let (dt, values_buf, original_nulls) = values.into_parts(); let nulls_buf = NullBuffer::union(original_nulls.as_ref(), Some(&filter_nulls)); PrimitiveArray::::new(values_buf, nulls_buf).with_data_type(dt) diff --git a/datafusion/functions-aggregate/src/approx_percentile_cont.rs b/datafusion/functions-aggregate/src/approx_percentile_cont.rs index 867dfed485c7..5578aebbf403 100644 --- a/datafusion/functions-aggregate/src/approx_percentile_cont.rs +++ b/datafusion/functions-aggregate/src/approx_percentile_cont.rs @@ -126,9 +126,9 @@ impl ApproxPercentileCont { | DataType::Float32 | DataType::Float64) => { if let Some(max_size) = tdigest_max_size { - ApproxPercentileAccumulator::new_with_max_size(percentile, t.clone(), max_size) + ApproxPercentileAccumulator::new_with_max_size(percentile, t, max_size) }else{ - ApproxPercentileAccumulator::new(percentile, t.clone()) + ApproxPercentileAccumulator::new(percentile, t) } } diff --git a/datafusion/functions-aggregate/src/array_agg.rs b/datafusion/functions-aggregate/src/array_agg.rs index b641d388a7c5..15146fc4a2d8 100644 --- a/datafusion/functions-aggregate/src/array_agg.rs +++ b/datafusion/functions-aggregate/src/array_agg.rs @@ -501,11 +501,8 @@ impl OrderSensitiveArrayAggAccumulator { column_wise_ordering_values.push(array); } - let ordering_array = StructArray::try_new( - struct_field.clone(), - column_wise_ordering_values, - None, - )?; + let ordering_array = + StructArray::try_new(struct_field, column_wise_ordering_values, None)?; Ok(ScalarValue::List(Arc::new(array_into_list_array_nullable( Arc::new(ordering_array), )))) diff --git a/datafusion/functions-aggregate/src/nth_value.rs b/datafusion/functions-aggregate/src/nth_value.rs index cb1ddd4738c4..7425bdfa18e7 100644 --- a/datafusion/functions-aggregate/src/nth_value.rs +++ b/datafusion/functions-aggregate/src/nth_value.rs @@ -374,11 +374,8 @@ impl NthValueAccumulator { column_wise_ordering_values.push(array); } - let ordering_array = StructArray::try_new( - struct_field.clone(), - column_wise_ordering_values, - None, - )?; + let ordering_array = + StructArray::try_new(struct_field, column_wise_ordering_values, None)?; Ok(ScalarValue::List(Arc::new(array_into_list_array_nullable( Arc::new(ordering_array), diff --git a/datafusion/functions-nested/src/flatten.rs b/datafusion/functions-nested/src/flatten.rs index 2b383af3d456..b04c35667226 100644 --- a/datafusion/functions-nested/src/flatten.rs +++ b/datafusion/functions-nested/src/flatten.rs @@ -147,7 +147,7 @@ fn flatten_internal( let list_arr = GenericListArray::::new(field, offsets, values, None); Ok(list_arr) } else { - Ok(list_arr.clone()) + Ok(list_arr) } } } diff --git a/datafusion/functions-nested/src/utils.rs b/datafusion/functions-nested/src/utils.rs index 688e1633e5cf..3d5b261618d5 100644 --- a/datafusion/functions-nested/src/utils.rs +++ b/datafusion/functions-nested/src/utils.rs @@ -296,8 +296,7 @@ mod tests { let array3d_1 = Arc::new(array_into_list_array_nullable(array2d_1)) as ArrayRef; let array3d_2 = array_into_list_array_nullable(array2d_2.to_owned()); let res = - align_array_dimensions::(vec![array1d_1, Arc::new(array3d_2.clone())]) - .unwrap(); + align_array_dimensions::(vec![array1d_1, Arc::new(array3d_2)]).unwrap(); let expected = as_list_array(&array3d_1).unwrap(); let expected_dim = datafusion_common::utils::list_ndims(array3d_1.data_type()); diff --git a/datafusion/optimizer/src/common_subexpr_eliminate.rs b/datafusion/optimizer/src/common_subexpr_eliminate.rs index 3fcee4123b76..3a2b190359d4 100644 --- a/datafusion/optimizer/src/common_subexpr_eliminate.rs +++ b/datafusion/optimizer/src/common_subexpr_eliminate.rs @@ -1431,7 +1431,7 @@ mod test { fn nested_aliases() -> Result<()> { let table_scan = test_table_scan()?; - let plan = LogicalPlanBuilder::from(table_scan.clone()) + let plan = LogicalPlanBuilder::from(table_scan) .project(vec![ (col("a") + col("b") - col("c")).alias("alias1") * (col("a") + col("b")), col("a") + col("b"), @@ -1842,7 +1842,7 @@ mod test { let config = &OptimizerContext::new(); let _common_expr_1 = config.alias_generator().next(CSE_PREFIX); let common_expr_2 = config.alias_generator().next(CSE_PREFIX); - let plan = LogicalPlanBuilder::from(table_scan.clone()) + let plan = LogicalPlanBuilder::from(table_scan) .project(vec![ (col("a") + col("b")).alias(common_expr_2.clone()), col("c"), @@ -1886,7 +1886,7 @@ mod test { let extracted_short_circuit_leg_1 = (col("a") + col("b")).eq(lit(0)); let not_extracted_short_circuit_leg_2 = (col("a") - col("b")).eq(lit(0)); let extracted_short_circuit_leg_3 = (col("a") * col("b")).eq(lit(0)); - let plan = LogicalPlanBuilder::from(table_scan.clone()) + let plan = LogicalPlanBuilder::from(table_scan) .project(vec![ extracted_short_circuit.clone().alias("c1"), extracted_short_circuit.alias("c2"), @@ -1899,7 +1899,7 @@ mod test { .alias("c4"), extracted_short_circuit_leg_3 .clone() - .or(extracted_short_circuit_leg_3.clone()) + .or(extracted_short_circuit_leg_3) .alias("c5"), ])? .build()?; @@ -1920,7 +1920,7 @@ mod test { let extracted_child = col("a") + col("b"); let rand = rand_func().call(vec![]); let not_extracted_volatile = extracted_child + rand; - let plan = LogicalPlanBuilder::from(table_scan.clone()) + let plan = LogicalPlanBuilder::from(table_scan) .project(vec![ not_extracted_volatile.clone().alias("c1"), not_extracted_volatile.alias("c2"), @@ -1947,7 +1947,7 @@ mod test { let not_extracted_short_circuit_leg_2 = col("b").eq(lit(0)); let not_extracted_volatile_short_circuit_2 = rand.eq(lit(0)).or(not_extracted_short_circuit_leg_2); - let plan = LogicalPlanBuilder::from(table_scan.clone()) + let plan = LogicalPlanBuilder::from(table_scan) .project(vec![ not_extracted_volatile_short_circuit_1.clone().alias("c1"), not_extracted_volatile_short_circuit_1.alias("c2"), diff --git a/datafusion/optimizer/src/eliminate_nested_union.rs b/datafusion/optimizer/src/eliminate_nested_union.rs index e9b38567a982..965771326854 100644 --- a/datafusion/optimizer/src/eliminate_nested_union.rs +++ b/datafusion/optimizer/src/eliminate_nested_union.rs @@ -144,10 +144,7 @@ mod tests { fn eliminate_nothing() -> Result<()> { let plan_builder = table_scan(Some("table"), &schema(), None)?; - let plan = plan_builder - .clone() - .union(plan_builder.clone().build()?)? - .build()?; + let plan = plan_builder.clone().union(plan_builder.build()?)?.build()?; let expected = "\ Union\ @@ -162,7 +159,7 @@ mod tests { let plan = plan_builder .clone() - .union_distinct(plan_builder.clone().build()?)? + .union_distinct(plan_builder.build()?)? .build()?; let expected = "Distinct:\ @@ -180,7 +177,7 @@ mod tests { .clone() .union(plan_builder.clone().build()?)? .union(plan_builder.clone().build()?)? - .union(plan_builder.clone().build()?)? + .union(plan_builder.build()?)? .build()?; let expected = "\ @@ -200,7 +197,7 @@ mod tests { .clone() .union_distinct(plan_builder.clone().build()?)? .union(plan_builder.clone().build()?)? - .union(plan_builder.clone().build()?)? + .union(plan_builder.build()?)? .build()?; let expected = "Union\ @@ -222,7 +219,7 @@ mod tests { .union(plan_builder.clone().build()?)? .union_distinct(plan_builder.clone().build()?)? .union(plan_builder.clone().build()?)? - .union_distinct(plan_builder.clone().build()?)? + .union_distinct(plan_builder.build()?)? .build()?; let expected = "Distinct:\ @@ -243,7 +240,7 @@ mod tests { .clone() .union_distinct(plan_builder.clone().distinct()?.build()?)? .union(plan_builder.clone().distinct()?.build()?)? - .union_distinct(plan_builder.clone().build()?)? + .union_distinct(plan_builder.build()?)? .build()?; let expected = "Distinct:\ @@ -271,7 +268,6 @@ mod tests { )? .union( plan_builder - .clone() .project(vec![col("id").alias("_id"), col("key"), col("value")])? .build()?, )? @@ -300,7 +296,6 @@ mod tests { )? .union_distinct( plan_builder - .clone() .project(vec![col("id").alias("_id"), col("key"), col("value")])? .build()?, )? diff --git a/datafusion/optimizer/src/eliminate_one_union.rs b/datafusion/optimizer/src/eliminate_one_union.rs index 383b33637f6f..7a1c4e118e05 100644 --- a/datafusion/optimizer/src/eliminate_one_union.rs +++ b/datafusion/optimizer/src/eliminate_one_union.rs @@ -94,10 +94,7 @@ mod tests { fn eliminate_nothing() -> Result<()> { let plan_builder = table_scan(Some("table"), &schema(), None)?; - let plan = plan_builder - .clone() - .union(plan_builder.clone().build()?)? - .build()?; + let plan = plan_builder.clone().union(plan_builder.build()?)?.build()?; let expected = "\ Union\ diff --git a/datafusion/optimizer/src/optimize_projections/mod.rs b/datafusion/optimizer/src/optimize_projections/mod.rs index 5db5afd11062..35b0d07751ff 100644 --- a/datafusion/optimizer/src/optimize_projections/mod.rs +++ b/datafusion/optimizer/src/optimize_projections/mod.rs @@ -1336,8 +1336,8 @@ mod tests { let right_table = test_table_scan_with_name("r")?; let custom_plan = LogicalPlan::Extension(Extension { node: Arc::new(UserDefinedCrossJoin::new( - Arc::new(left_table.clone()), - Arc::new(right_table.clone()), + Arc::new(left_table), + Arc::new(right_table), )), }); let plan = LogicalPlanBuilder::from(custom_plan) diff --git a/datafusion/optimizer/src/push_down_filter.rs b/datafusion/optimizer/src/push_down_filter.rs index 6e75f46c2d0b..82149a087e63 100644 --- a/datafusion/optimizer/src/push_down_filter.rs +++ b/datafusion/optimizer/src/push_down_filter.rs @@ -819,7 +819,7 @@ impl OptimizerRule for PushDownFilter { .map(|e| Ok(Column::from_qualified_name(e.schema_name().to_string()))) .collect::>>()?; - let predicates = split_conjunction_owned(filter.predicate.clone()); + let predicates = split_conjunction_owned(filter.predicate); let mut keep_predicates = vec![]; let mut push_predicates = vec![]; diff --git a/datafusion/optimizer/src/simplify_expressions/expr_simplifier.rs b/datafusion/optimizer/src/simplify_expressions/expr_simplifier.rs index c45df74a564d..7129ceb0fea1 100644 --- a/datafusion/optimizer/src/simplify_expressions/expr_simplifier.rs +++ b/datafusion/optimizer/src/simplify_expressions/expr_simplifier.rs @@ -3407,32 +3407,32 @@ mod tests { let expr = in_list(col("c1"), vec![lit(1), lit(2), lit(3), lit(4)], false).and( in_list(col("c1"), vec![lit(5), lit(6), lit(7), lit(8)], false), ); - assert_eq!(simplify(expr.clone()), lit(false)); + assert_eq!(simplify(expr), lit(false)); // 2. c1 IN (1,2,3,4) AND c1 IN (4,5,6,7) -> c1 = 4 let expr = in_list(col("c1"), vec![lit(1), lit(2), lit(3), lit(4)], false).and( in_list(col("c1"), vec![lit(4), lit(5), lit(6), lit(7)], false), ); - assert_eq!(simplify(expr.clone()), col("c1").eq(lit(4))); + assert_eq!(simplify(expr), col("c1").eq(lit(4))); // 3. c1 NOT IN (1, 2, 3, 4) OR c1 NOT IN (5, 6, 7, 8) -> true let expr = in_list(col("c1"), vec![lit(1), lit(2), lit(3), lit(4)], true).or( in_list(col("c1"), vec![lit(5), lit(6), lit(7), lit(8)], true), ); - assert_eq!(simplify(expr.clone()), lit(true)); + assert_eq!(simplify(expr), lit(true)); // 3.5 c1 NOT IN (1, 2, 3, 4) OR c1 NOT IN (4, 5, 6, 7) -> c1 != 4 (4 overlaps) let expr = in_list(col("c1"), vec![lit(1), lit(2), lit(3), lit(4)], true).or( in_list(col("c1"), vec![lit(4), lit(5), lit(6), lit(7)], true), ); - assert_eq!(simplify(expr.clone()), col("c1").not_eq(lit(4))); + assert_eq!(simplify(expr), col("c1").not_eq(lit(4))); // 4. c1 NOT IN (1,2,3,4) AND c1 NOT IN (4,5,6,7) -> c1 NOT IN (1,2,3,4,5,6,7) let expr = in_list(col("c1"), vec![lit(1), lit(2), lit(3), lit(4)], true).and( in_list(col("c1"), vec![lit(4), lit(5), lit(6), lit(7)], true), ); assert_eq!( - simplify(expr.clone()), + simplify(expr), in_list( col("c1"), vec![lit(1), lit(2), lit(3), lit(4), lit(5), lit(6), lit(7)], @@ -3445,7 +3445,7 @@ mod tests { in_list(col("c1"), vec![lit(2), lit(3), lit(4), lit(5)], false), ); assert_eq!( - simplify(expr.clone()), + simplify(expr), in_list( col("c1"), vec![lit(1), lit(2), lit(3), lit(4), lit(5)], @@ -3459,7 +3459,7 @@ mod tests { vec![lit(1), lit(2), lit(3), lit(4), lit(5)], true, )); - assert_eq!(simplify(expr.clone()), lit(false)); + assert_eq!(simplify(expr), lit(false)); // 7. c1 NOT IN (1,2,3,4) AND c1 IN (1,2,3,4,5) -> c1 = 5 let expr = @@ -3468,14 +3468,14 @@ mod tests { vec![lit(1), lit(2), lit(3), lit(4), lit(5)], false, )); - assert_eq!(simplify(expr.clone()), col("c1").eq(lit(5))); + assert_eq!(simplify(expr), col("c1").eq(lit(5))); // 8. c1 IN (1,2,3,4) AND c1 NOT IN (5,6,7,8) -> c1 IN (1,2,3,4) let expr = in_list(col("c1"), vec![lit(1), lit(2), lit(3), lit(4)], false).and( in_list(col("c1"), vec![lit(5), lit(6), lit(7), lit(8)], true), ); assert_eq!( - simplify(expr.clone()), + simplify(expr), in_list(col("c1"), vec![lit(1), lit(2), lit(3), lit(4)], false) ); @@ -3493,7 +3493,7 @@ mod tests { )) .and(in_list(col("c1"), vec![lit(3), lit(6)], false)); assert_eq!( - simplify(expr.clone()), + simplify(expr), col("c1").eq(lit(3)).or(col("c1").eq(lit(6))) ); @@ -3507,7 +3507,7 @@ mod tests { )) .and(in_list(col("c1"), vec![lit(8), lit(9), lit(10)], false)), ); - assert_eq!(simplify(expr.clone()), col("c1").eq(lit(8))); + assert_eq!(simplify(expr), col("c1").eq(lit(8))); // Contains non-InList expression // c1 NOT IN (1,2,3,4) OR c1 != 5 OR c1 NOT IN (6,7,8,9) -> c1 NOT IN (1,2,3,4) OR c1 != 5 OR c1 NOT IN (6,7,8,9) @@ -3622,7 +3622,7 @@ mod tests { let expr_x = col("c3").gt(lit(3_i64)); let expr_y = (col("c4") + lit(2_u32)).lt(lit(10_u32)); let expr_z = col("c1").in_list(vec![lit("a"), lit("b")], true); - let expr = expr_x.clone().and(expr_y.clone().or(expr_z)); + let expr = expr_x.clone().and(expr_y.or(expr_z)); // All guaranteed null let guarantees = vec![ @@ -3698,7 +3698,7 @@ mod tests { col("c4"), NullableInterval::from(ScalarValue::UInt32(Some(3))), )]; - let output = simplify_with_guarantee(expr.clone(), guarantees); + let output = simplify_with_guarantee(expr, guarantees); assert_eq!(&output, &expr_x); } diff --git a/datafusion/optimizer/src/simplify_expressions/guarantees.rs b/datafusion/optimizer/src/simplify_expressions/guarantees.rs index 09fdd7685a9c..afcbe528083b 100644 --- a/datafusion/optimizer/src/simplify_expressions/guarantees.rs +++ b/datafusion/optimizer/src/simplify_expressions/guarantees.rs @@ -225,12 +225,12 @@ mod tests { // x IS NULL => guaranteed false let expr = col("x").is_null(); - let output = expr.clone().rewrite(&mut rewriter).data().unwrap(); + let output = expr.rewrite(&mut rewriter).data().unwrap(); assert_eq!(output, lit(false)); // x IS NOT NULL => guaranteed true let expr = col("x").is_not_null(); - let output = expr.clone().rewrite(&mut rewriter).data().unwrap(); + let output = expr.rewrite(&mut rewriter).data().unwrap(); assert_eq!(output, lit(true)); } diff --git a/datafusion/optimizer/src/unwrap_cast_in_comparison.rs b/datafusion/optimizer/src/unwrap_cast_in_comparison.rs index e0f50a470d43..b17d69437cbe 100644 --- a/datafusion/optimizer/src/unwrap_cast_in_comparison.rs +++ b/datafusion/optimizer/src/unwrap_cast_in_comparison.rs @@ -627,7 +627,7 @@ mod tests { Box::new(DataType::Int32), Box::new(ScalarValue::LargeUtf8(Some("value".to_owned()))), ); - let expr_input = cast(col("largestr"), dict.data_type()).eq(lit(dict.clone())); + let expr_input = cast(col("largestr"), dict.data_type()).eq(lit(dict)); let expected = col("largestr").eq(lit(ScalarValue::LargeUtf8(Some("value".to_owned())))); assert_eq!(optimize_test(expr_input, &schema), expected); diff --git a/datafusion/physical-expr/src/equivalence/ordering.rs b/datafusion/physical-expr/src/equivalence/ordering.rs index c4b8a5c46563..49a0de7252ab 100644 --- a/datafusion/physical-expr/src/equivalence/ordering.rs +++ b/datafusion/physical-expr/src/equivalence/ordering.rs @@ -272,7 +272,7 @@ mod tests { // Crude ordering doesn't satisfy finer ordering. should return false let mut eq_properties_crude = EquivalenceProperties::new(Arc::clone(&input_schema)); - eq_properties_crude.oeq_class.push(crude.clone()); + eq_properties_crude.oeq_class.push(crude); assert!(!eq_properties_crude.ordering_satisfy(&finer)); Ok(()) } diff --git a/datafusion/physical-expr/src/expressions/binary.rs b/datafusion/physical-expr/src/expressions/binary.rs index b663d8614275..2680a7930ff1 100644 --- a/datafusion/physical-expr/src/expressions/binary.rs +++ b/datafusion/physical-expr/src/expressions/binary.rs @@ -2582,7 +2582,7 @@ mod tests { &a, &b, Operator::RegexIMatch, - regex_expected.clone(), + regex_expected, )?; apply_logic_op( &Arc::new(schema.clone()), @@ -2596,7 +2596,7 @@ mod tests { &a, &b, Operator::RegexNotIMatch, - regex_not_expected.clone(), + regex_not_expected, )?; Ok(()) diff --git a/datafusion/physical-expr/src/expressions/case.rs b/datafusion/physical-expr/src/expressions/case.rs index c6afb5c05985..712175c9afbe 100644 --- a/datafusion/physical-expr/src/expressions/case.rs +++ b/datafusion/physical-expr/src/expressions/case.rs @@ -380,7 +380,7 @@ impl CaseExpr { // keep `else_expr`'s data type and return type consistent let e = self.else_expr.as_ref().unwrap(); - let expr = try_cast(Arc::clone(e), &batch.schema(), return_type.clone()) + let expr = try_cast(Arc::clone(e), &batch.schema(), return_type) .unwrap_or_else(|_| Arc::clone(e)); let else_ = Scalar::new(expr.evaluate(batch)?.into_array(1)?); diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 0aeaa15b5f1d..6ca01928c80f 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -2220,7 +2220,7 @@ mod tests { let aggregate_exec = Arc::new(AggregateExec::try_new( AggregateMode::Partial, groups, - aggregates.clone(), + aggregates, vec![None, None], Arc::clone(&blocking_exec) as Arc, schema, diff --git a/datafusion/physical-plan/src/joins/utils.rs b/datafusion/physical-plan/src/joins/utils.rs index 8fdbf7041e2f..89f3feaf07be 100644 --- a/datafusion/physical-plan/src/joins/utils.rs +++ b/datafusion/physical-plan/src/joins/utils.rs @@ -2443,7 +2443,7 @@ mod tests { Statistics { num_rows: Absent, total_byte_size: Absent, - column_statistics: dummy_column_stats.clone(), + column_statistics: dummy_column_stats, }, &join_on, ); diff --git a/datafusion/physical-plan/src/recursive_query.rs b/datafusion/physical-plan/src/recursive_query.rs index bd9303f97db0..e9ea9d4f5032 100644 --- a/datafusion/physical-plan/src/recursive_query.rs +++ b/datafusion/physical-plan/src/recursive_query.rs @@ -394,7 +394,7 @@ impl Stream for RecursiveQueryStream { self.recursive_stream = None; self.poll_next_iteration(cx) } - Some(Ok(batch)) => self.push_batch(batch.clone()), + Some(Ok(batch)) => self.push_batch(batch), _ => Poll::Ready(batch_result), } } else { diff --git a/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs b/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs index 084436ee376d..c1bcd83a6fd2 100644 --- a/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs +++ b/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs @@ -1312,7 +1312,7 @@ mod tests { &args, &partitionby_exprs, &orderby_exprs, - Arc::new(window_frame.clone()), + Arc::new(window_frame), &input.schema(), false, )?], @@ -1485,7 +1485,7 @@ mod tests { let partitions = vec![ Arc::new(TestStreamPartition { schema: Arc::clone(&schema), - batches: batches.clone(), + batches, idx: 0, state: PolingState::BatchReturn, sleep_duration: per_batch_wait_duration, diff --git a/datafusion/proto/gen/src/main.rs b/datafusion/proto/gen/src/main.rs index d3b3c92f6065..be61ff58fa8d 100644 --- a/datafusion/proto/gen/src/main.rs +++ b/datafusion/proto/gen/src/main.rs @@ -55,7 +55,7 @@ fn main() -> Result<(), String> { let common_path = proto_dir.join("src/datafusion_common.rs"); println!( "Copying {} to {}", - prost.clone().display(), + prost.display(), proto_dir.join("src/generated/prost.rs").display() ); std::fs::copy(prost, proto_dir.join("src/generated/prost.rs")).unwrap(); diff --git a/datafusion/proto/tests/cases/roundtrip_logical_plan.rs b/datafusion/proto/tests/cases/roundtrip_logical_plan.rs index 4f58185798f7..94ac913e1968 100644 --- a/datafusion/proto/tests/cases/roundtrip_logical_plan.rs +++ b/datafusion/proto/tests/cases/roundtrip_logical_plan.rs @@ -2434,7 +2434,7 @@ fn roundtrip_window() { WindowFunctionDefinition::AggregateUDF(avg_udaf()), vec![col("col1")], )) - .window_frame(row_number_frame.clone()) + .window_frame(row_number_frame) .build() .unwrap(); diff --git a/datafusion/proto/tests/cases/roundtrip_physical_plan.rs b/datafusion/proto/tests/cases/roundtrip_physical_plan.rs index 60f5565bdeee..3e49dc24fd5a 100644 --- a/datafusion/proto/tests/cases/roundtrip_physical_plan.rs +++ b/datafusion/proto/tests/cases/roundtrip_physical_plan.rs @@ -404,7 +404,7 @@ fn rountrip_aggregate_with_limit() -> Result<()> { let agg = AggregateExec::try_new( AggregateMode::Final, PhysicalGroupBy::new_single(groups.clone()), - aggregates.clone(), + aggregates, vec![None], Arc::new(EmptyExec::new(schema.clone())), schema, @@ -433,7 +433,7 @@ fn rountrip_aggregate_with_approx_pencentile_cont() -> Result<()> { let agg = AggregateExec::try_new( AggregateMode::Final, PhysicalGroupBy::new_single(groups.clone()), - aggregates.clone(), + aggregates, vec![None], Arc::new(EmptyExec::new(schema.clone())), schema, @@ -469,7 +469,7 @@ fn rountrip_aggregate_with_sort() -> Result<()> { let agg = AggregateExec::try_new( AggregateMode::Final, PhysicalGroupBy::new_single(groups.clone()), - aggregates.clone(), + aggregates, vec![None], Arc::new(EmptyExec::new(schema.clone())), schema, @@ -536,7 +536,7 @@ fn roundtrip_aggregate_udaf() -> Result<()> { Arc::new(AggregateExec::try_new( AggregateMode::Final, PhysicalGroupBy::new_single(groups.clone()), - aggregates.clone(), + aggregates, vec![None], Arc::new(EmptyExec::new(schema.clone())), schema, @@ -640,7 +640,7 @@ fn roundtrip_coalesce_with_fetch() -> Result<()> { )))?; roundtrip_test(Arc::new( - CoalesceBatchesExec::new(Arc::new(EmptyExec::new(schema.clone())), 8096) + CoalesceBatchesExec::new(Arc::new(EmptyExec::new(schema)), 8096) .with_fetch(Some(10)), )) } @@ -991,18 +991,16 @@ fn roundtrip_scalar_udf_extension_codec() -> Result<()> { )), input, )?); - let aggr_expr = AggregateExprBuilder::new( - max_udaf(), - vec![udf_expr.clone() as Arc], - ) - .schema(schema.clone()) - .alias("max") - .build()?; + let aggr_expr = + AggregateExprBuilder::new(max_udaf(), vec![udf_expr as Arc]) + .schema(schema.clone()) + .alias("max") + .build()?; let window = Arc::new(WindowAggExec::try_new( vec![Arc::new(PlainAggregateWindowExpr::new( aggr_expr.clone(), - &[col("author", &schema.clone())?], + &[col("author", &schema)?], &[], Arc::new(WindowFrame::new(None)), ))], @@ -1013,10 +1011,10 @@ fn roundtrip_scalar_udf_extension_codec() -> Result<()> { let aggregate = Arc::new(AggregateExec::try_new( AggregateMode::Final, PhysicalGroupBy::new(vec![], vec![], vec![]), - vec![aggr_expr.clone()], + vec![aggr_expr], vec![None], window, - schema.clone(), + schema, )?); let ctx = SessionContext::new(); @@ -1054,7 +1052,7 @@ fn roundtrip_aggregate_udf_extension_codec() -> Result<()> { Arc::new(BinaryExpr::new( col("published", &schema)?, Operator::And, - Arc::new(BinaryExpr::new(udf_expr.clone(), Operator::Gt, lit(0))), + Arc::new(BinaryExpr::new(udf_expr, Operator::Gt, lit(0))), )), input, )?); @@ -1083,7 +1081,7 @@ fn roundtrip_aggregate_udf_extension_codec() -> Result<()> { vec![aggr_expr], vec![None], window, - schema.clone(), + schema, )?); let ctx = SessionContext::new(); @@ -1158,7 +1156,7 @@ fn roundtrip_json_sink() -> Result<()> { roundtrip_test(Arc::new(DataSinkExec::new( input, data_sink, - schema.clone(), + schema, Some(sort_order), ))) } @@ -1197,7 +1195,7 @@ fn roundtrip_csv_sink() -> Result<()> { Arc::new(DataSinkExec::new( input, data_sink, - schema.clone(), + schema, Some(sort_order), )), &ctx, @@ -1253,7 +1251,7 @@ fn roundtrip_parquet_sink() -> Result<()> { roundtrip_test(Arc::new(DataSinkExec::new( input, data_sink, - schema.clone(), + schema, Some(sort_order), ))) } @@ -1342,7 +1340,7 @@ fn roundtrip_interleave() -> Result<()> { )?; let right = RepartitionExec::try_new( Arc::new(EmptyExec::new(Arc::new(schema_right))), - partition.clone(), + partition, )?; let inputs: Vec> = vec![Arc::new(left), Arc::new(right)]; let interleave = InterleaveExec::try_new(inputs)?; diff --git a/datafusion/sql/src/expr/function.rs b/datafusion/sql/src/expr/function.rs index 72e08e4b8fb5..71e40c20b80a 100644 --- a/datafusion/sql/src/expr/function.rs +++ b/datafusion/sql/src/expr/function.rs @@ -245,8 +245,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { // Build Unnest expression if name.eq("unnest") { - let mut exprs = - self.function_args_to_expr(args.clone(), schema, planner_context)?; + let mut exprs = self.function_args_to_expr(args, schema, planner_context)?; if exprs.len() != 1 { return plan_err!("unnest() requires exactly one argument"); } diff --git a/datafusion/sql/src/relation/mod.rs b/datafusion/sql/src/relation/mod.rs index 86e49780724b..f8ebb04f3810 100644 --- a/datafusion/sql/src/relation/mod.rs +++ b/datafusion/sql/src/relation/mod.rs @@ -214,7 +214,7 @@ fn optimize_subquery_sort(plan: LogicalPlan) -> Result> // 2. RANK / ROW_NUMBER ... => Handled by a `WindowAggr` and its requirements. // 3. LIMIT => Handled by a `Sort`, so we need to search for it. let mut has_limit = false; - let new_plan = plan.clone().transform_down(|c| { + let new_plan = plan.transform_down(|c| { if let LogicalPlan::Limit(_) = c { has_limit = true; return Ok(Transformed::no(c)); diff --git a/datafusion/sql/src/unparser/expr.rs b/datafusion/sql/src/unparser/expr.rs index 9ce627aecc76..0dbcba162bc0 100644 --- a/datafusion/sql/src/unparser/expr.rs +++ b/datafusion/sql/src/unparser/expr.rs @@ -1864,7 +1864,7 @@ mod tests { r#"EXISTS (SELECT * FROM t WHERE (t.a = 1))"#, ), ( - not_exists(Arc::new(dummy_logical_plan.clone())), + not_exists(Arc::new(dummy_logical_plan)), r#"NOT EXISTS (SELECT * FROM t WHERE (t.a = 1))"#, ), ( diff --git a/datafusion/sql/tests/cases/plan_to_sql.rs b/datafusion/sql/tests/cases/plan_to_sql.rs index ed23fada0cfb..cdc7bef06afd 100644 --- a/datafusion/sql/tests/cases/plan_to_sql.rs +++ b/datafusion/sql/tests/cases/plan_to_sql.rs @@ -203,7 +203,7 @@ fn roundtrip_crossjoin() -> Result<()> { println!("plan {}", plan.display_indent()); let plan_roundtrip = sql_to_rel - .sql_statement_to_plan(roundtrip_statement.clone()) + .sql_statement_to_plan(roundtrip_statement) .unwrap(); let expected = "Projection: j1.j1_id, j2.j2_string\ From 7d8bb0b346643381fbe79abc480cc57a5522a5d6 Mon Sep 17 00:00:00 2001 From: WeblWabl Date: Wed, 28 Aug 2024 07:26:41 -0500 Subject: [PATCH 040/154] Add documentation on `EXPLAIN` and `EXPLAIN ANALYZE` (#12122) * feat(wip): working on adding explain docs Signed-off-by: Devan * working on it Signed-off-by: Devan * working on it -- adding plan descriptions Signed-off-by: Devan * adds descriptions for phys plan and note on parallel Signed-off-by: Devan * adds information about the logical plan Signed-off-by: Devan * Add page to index * Update example, add information on how to get the datafile * Add example of reading explain analyze * Add section on partitioning * prettier * adding aggregate plan explain docs Signed-off-by: Devan * field -> column Signed-off-by: Devan * repartition update Signed-off-by: Devan * prettier Signed-off-by: Devan * clarify some points --------- Signed-off-by: Devan Co-authored-by: Andrew Lamb --- docs/source/index.rst | 1 + docs/source/user-guide/explain-usage.md | 365 ++++++++++++++++++++++++ docs/source/user-guide/sql/explain.md | 2 + 3 files changed, 368 insertions(+) create mode 100644 docs/source/user-guide/explain-usage.md diff --git a/docs/source/index.rst b/docs/source/index.rst index 9c8c886d2502..bb5ea430a321 100644 --- a/docs/source/index.rst +++ b/docs/source/index.rst @@ -89,6 +89,7 @@ To get started, see user-guide/expressions user-guide/sql/index user-guide/configs + user-guide/explain-usage user-guide/faq .. _toc.library-user-guide: diff --git a/docs/source/user-guide/explain-usage.md b/docs/source/user-guide/explain-usage.md new file mode 100644 index 000000000000..a65fad92d104 --- /dev/null +++ b/docs/source/user-guide/explain-usage.md @@ -0,0 +1,365 @@ + + +# Reading Explain Plans + +## Introduction + +This section describes of how to read a DataFusion query plan. While fully +comprehending all details of these plans requires significant expertise in the +DataFusion engine, this guide will help you get started with the basics. + +Datafusion executes queries using a `query plan`. To see the plan without +running the query, add the keyword `EXPLAIN` to your SQL query or call the +[DataFrame::explain] method + +[dataframe::explain]: https://docs.rs/datafusion/latest/datafusion/dataframe/struct.DataFrame.html#method.explain + +## Example: Select and filter + +In this section, we run example queries against the `hits.parquet` file. See +[below](#data-in-this-example)) for information on how to get this file. + +Let's see how DataFusion runs a query that selects the top 5 watch lists for the +site `http://domcheloveplanet.ru/`: + +```sql +EXPLAIN SELECT "WatchID" AS wid, "hits.parquet"."ClientIP" AS ip +FROM 'hits.parquet' +WHERE starts_with("URL", 'http://domcheloveplanet.ru/') +ORDER BY wid ASC, ip DESC +LIMIT 5; +``` + +The output will look like + +``` ++---------------+-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ +| plan_type | plan | ++---------------+-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ +| logical_plan | Sort: wid ASC NULLS LAST, ip DESC NULLS FIRST, fetch=5 | +| | Projection: hits.parquet.WatchID AS wid, hits.parquet.ClientIP AS ip | +| | Filter: starts_with(hits.parquet.URL, Utf8("http://domcheloveplanet.ru/")) | +| | TableScan: hits.parquet projection=[WatchID, ClientIP, URL], partial_filters=[starts_with(hits.parquet.URL, Utf8("http://domcheloveplanet.ru/"))] | +| physical_plan | SortPreservingMergeExec: [wid@0 ASC NULLS LAST,ip@1 DESC], fetch=5 | +| | SortExec: TopK(fetch=5), expr=[wid@0 ASC NULLS LAST,ip@1 DESC], preserve_partitioning=[true] | +| | ProjectionExec: expr=[WatchID@0 as wid, ClientIP@1 as ip] | +| | CoalesceBatchesExec: target_batch_size=8192 | +| | FilterExec: starts_with(URL@2, http://domcheloveplanet.ru/) | +| | ParquetExec: file_groups={16 groups: [[hits.parquet:0..923748528], ...]}, projection=[WatchID, ClientIP, URL], predicate=starts_with(URL@13, http://domcheloveplanet.ru/) | ++---------------+-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ +2 row(s) fetched. +Elapsed 0.060 seconds. +``` + +There are two sections: logical plan and physical plan + +- **Logical Plan:** is a plan generated for a specific SQL query, DataFrame, or other language without the + knowledge of the underlying data organization. +- **Physical Plan:** is a plan generated from a logical plan along with consideration of the hardware + configuration (e.g number of CPUs) and the underlying data organization (e.g number of files). + This physical plan is specific to your hardware configuration and your data. If you load the same + data to different hardware with different configurations, the same query may generate different query plans. + +Understanding a query plan can help to you understand its performance. For example, when the plan shows your query reads +many files, it signals you to either add more filter in the query to read less data or to modify your file +design to make fewer but larger files. This document focuses on how to read a query plan. How to make a +query run faster depends on the reason it is slow and beyond the scope of this document. + +## Query plans are trees + +A query plan is an upside down tree, and we always read from bottom up. The +physical plan in Figure 1 in tree format will look like + +``` + ▲ + │ + │ +┌─────────────────────────────────────────────────┐ +│ SortPreservingMergeExec │ +│ [wid@0 ASC NULLS LAST,ip@1 DESC] │ +│ fetch=5 │ +└─────────────────────────────────────────────────┘ + ▲ + │ +┌─────────────────────────────────────────────────┐ +│ SortExec TopK(fetch=5), │ +│ expr=[wid@0 ASC NULLS LAST,ip@1 DESC], │ +│ preserve_partitioning=[true] │ +└─────────────────────────────────────────────────┘ + ▲ + │ +┌─────────────────────────────────────────────────┐ +│ ProjectionExec │ +│ expr=[WatchID@0 as wid, ClientIP@1 as ip] │ +└─────────────────────────────────────────────────┘ + ▲ + │ +┌─────────────────────────────────────────────────┐ +│ CoalesceBatchesExec │ +└─────────────────────────────────────────────────┘ + ▲ + │ +┌─────────────────────────────────────────────────┐ +│ FilterExec │ +│ starts_with(URL@2, http://domcheloveplanet.ru/) │ +└─────────────────────────────────────────────────┘ + ▲ + │ +┌────────────────────────────────────────────────┐ +│ ParquetExec │ +│ hits.parquet (filter = ...) │ +└────────────────────────────────────────────────┘ +``` + +Each node in the tree/plan ends with `Exec` and is sometimes also called an `operator` or `ExecutionPlan` where data is +processed, transformed and sent up. + +1. First, data in parquet the `hits.parquet` file us read in parallel using 16 cores in 16 "partitions" (more on this later) from `ParquetExec`, which applies a first pass at filtering during the scan. +2. Next, the output is filtered using `FilterExec` to ensure only rows where `starts_with(URL, 'http://domcheloveplanet.ru/')` evaluates to true are passed on +3. The `CoalesceBatchesExec` then ensures that the data is grouped into larger batches for processing +4. The `ProjectionExec` then projects the data to rename the `WatchID` and `ClientIP` columns to `wid` and `ip` respectively. +5. The `SortExec` then sorts the data by `wid ASC, ip DESC`. The `Topk(fetch=5)` indicates that a special implementation is used that only tracks and emits the top 5 values in each partition. +6. Finally the `SortPreservingMergeExec` merges the sorted data from all partitions and returns the top 5 rows overall. + +## Understanding large query plans + +A large query plan may look intimidating, but you can quickly understand what it does by following these steps + +1. As always, read from bottom up, one operator at a time. +2. Understand the job of this operator by reading + the [Physical Plan documentation](https://docs.rs/datafusion/latest/datafusion/physical_plan/index.html). +3. Understand the input data of the operator and how large/small it may be. +4. Understand how much data that operator produces and what it would look like. + +If you can answer those questions, you will be able to estimate how much work +that plan has to do and thus how long it will take. However, the `EXPLAIN` just +shows you the plan without executing it. + +If you want to know more about how much work each operator in query plan does, +you can use the `EXPLAIN ANALYZE` to get the explain with runtime added (see +next section) + +## More Debugging Information: `EXPLAIN VERBOSE` + +If the plan has to read too many files, not all of them will be shown in the +`EXPLAIN`. To see them, use `EXPLAIN VEBOSE`. Like `EXPLAIN`, `EXPLAIN VERBOSE` +does not run the query. Instead it shows the full explain plan, with information +that is omitted from the default explain, as well as all intermediate physical +plans DataFusion generates before returning. This mode can be very helpful for +debugging to see why and when DataFusion added and removed operators from a plan. + +## Execution Counters: `EXPLAIN ANALYZE` + +During execution, DataFusion operators collect detailed metrics. You can access +them programmatically via [`ExecutionPlan::metrics`] as well as with the +`EXPLAIN ANALYZE` command. For example here is the same query query as +above but with `EXPLAIN ANALYZE` (note the output is edited for clarity) + +[`executionplan::metrics`]: https://docs.rs/datafusion/latest/datafusion/physical_plan/trait.ExecutionPlan.html#method.metrics + +``` +> EXPLAIN ANALYZE SELECT "WatchID" AS wid, "hits.parquet"."ClientIP" AS ip +FROM 'hits.parquet' +WHERE starts_with("URL", 'http://domcheloveplanet.ru/') +ORDER BY wid ASC, ip DESC +LIMIT 5; ++-------------------+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ +| plan_type | plan | ++-------------------+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ +| Plan with Metrics | SortPreservingMergeExec: [wid@0 ASC NULLS LAST,ip@1 DESC], fetch=5, metrics=[output_rows=5, elapsed_compute=2.375µs] | +| | SortExec: TopK(fetch=5), expr=[wid@0 ASC NULLS LAST,ip@1 DESC], preserve_partitioning=[true], metrics=[output_rows=75, elapsed_compute=7.243038ms, row_replacements=482] | +| | ProjectionExec: expr=[WatchID@0 as wid, ClientIP@1 as ip], metrics=[output_rows=811821, elapsed_compute=66.25µs] | +| | FilterExec: starts_with(URL@2, http://domcheloveplanet.ru/), metrics=[output_rows=811821, elapsed_compute=1.36923816s] | +| | ParquetExec: file_groups={16 groups: [[hits.parquet:0..923748528], ...]}, projection=[WatchID, ClientIP, URL], predicate=starts_with(URL@13, http://domcheloveplanet.ru/), metrics=[output_rows=99997497, elapsed_compute=16ns, ... bytes_scanned=3703192723, ... time_elapsed_opening=308.203002ms, time_elapsed_scanning_total=8.350342183s, ...] | ++-------------------+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ +1 row(s) fetched. +Elapsed 0.720 seconds. +``` + +In this case, DataFusion actually ran the query, but discarded any results, and +instead returned an annotated plan with a new field, `metrics=[...]` + +Most operators have the common metrics `output_rows` and `elapsed_compute` and +some have operator specific metrics such as `ParquetExec` which has +`bytes_scanned=3703192723`. Note that times and counters are reported across all +cores, so if you have 16 cores, the time reported is the sum of the time taken +by all 16 cores. + +Again, reading from bottom up: + +- `ParquetExec` + - `output_rows=99997497`: A total 99.9M rows were produced + - `bytes_scanned=3703192723`: Of the 14GB file, 3.7GB were actually read (due to projection pushdown) + - `time_elapsed_opening=308.203002ms`: It took 300ms to open the file and prepare to read it + - `time_elapsed_scanning_total=8.350342183s`: It took 8.3 seconds of CPU time (across 16 cores) to actually decode the parquet data +- `FilterExec` + - `output_rows=811821`: Of the 99.9M rows at its input, only 811K rows passed the filter and were produced at the output + - `elapsed_compute=1.36923816s`: In total, 1.36s of CPU time (across 16 cores) was spend evaluating the filter +- `CoalesceBatchesExec` + - `output_rows=811821`, `elapsed_compute=12.873379ms`: Produced 811K rows in 13ms +- `ProjectionExec` + - `output_rows=811821, elapsed_compute=66.25µs`: Produced 811K rows in 66µs (microseconds). This projection is almost instantaneous as it does not manipulate any data +- `SortExec` + - `output_rows=75`: Produced 75 rows in total. Each of 16 cores could produce up to 5 rows, but in this case not all cores did. + - `elapsed_compute=7.243038ms`: 7ms was used to determine the top 5 rows + - `row_replacements=482`: Internally, the TopK operator updated its top list 482 times +- `SortPreservingMergeExec` + - `output_rows=5`, `elapsed_compute=2.375µs`: Produced the final 5 rows in 2.375µs (microseconds) + +## Partitions and Execution + +DataFusion determines the optimal number of cores to use as part of query +planning. Roughly speaking, each "partition" in the plan is run independently using +a separate core. Data crosses between cores only within certain operators such as +`RepartitionExec`, `CoalescePartitions` and `SortPreservingMergeExec` + +You can read more about this in the [Partitoning Docs]. + +[partitoning docs]: https://docs.rs/datafusion/latest/datafusion/physical_expr/enum.Partitioning.html + +## Example of an Aggregate Query + +Let us delve into an example query that aggregates data from the `hits.parquet` +file. For example, this query from ClickBench finds the top 10 users by their +number of hits: + +```sql +SELECT "UserID", COUNT(*) +FROM 'hits.parquet' +GROUP BY "UserID" +ORDER BY COUNT(*) DESC +LIMIT 10; +``` + +We can again see the query plan by using `EXPLAIN`: + +``` +> EXPLAIN SELECT "UserID", COUNT(*) FROM 'hits.parquet' GROUP BY "UserID" ORDER BY COUNT(*) DESC LIMIT 10; ++---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ +| plan_type | plan | ++---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ +| logical_plan | Limit: skip=0, fetch=10 | +| | Sort: count(*) DESC NULLS FIRST, fetch=10 | +| | Aggregate: groupBy=[[hits.parquet.UserID]], aggr=[[count(Int64(1)) AS count(*)]] | +| | TableScan: hits.parquet projection=[UserID] | +| physical_plan | GlobalLimitExec: skip=0, fetch=10 | +| | SortPreservingMergeExec: [count(*)@1 DESC], fetch=10 | +| | SortExec: TopK(fetch=10), expr=[count(*)@1 DESC], preserve_partitioning=[true] | +| | AggregateExec: mode=FinalPartitioned, gby=[UserID@0 as UserID], aggr=[count(*)] | +| | CoalesceBatchesExec: target_batch_size=8192 | +| | RepartitionExec: partitioning=Hash([UserID@0], 10), input_partitions=10 | +| | AggregateExec: mode=Partial, gby=[UserID@0 as UserID], aggr=[count(*)] | +| | ParquetExec: file_groups={10 groups: [[hits.parquet:0..1477997645], [hits.parquet:1477997645..2955995290], [hits.parquet:2955995290..4433992935], [hits.parquet:4433992935..5911990580], [hits.parquet:5911990580..7389988225], ...]}, projection=[UserID] | +| | | ++---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ +``` + +For this query, let's again read the plan from the bottom to the top: + +**Logical plan operators** + +- `TableScan` + - `hits.parquet`: Scans data from the file `hits.parquet`. + - `projection=[UserID]`: Reads only the `UserID` column +- `Aggregate` + - `groupBy=[[hits.parquet.UserID]]`: Groups by `UserID` column. + - `aggr=[[count(Int64(1)) AS count(*)]]`: Applies the `COUNT` aggregate on each distinct group. +- `Sort` + - `count(*) DESC NULLS FIRST`: Sorts the data in descending count order. + - `fetch=10`: Returns only the first 10 rows. +- `Limit` + - `skip=0`: Does not skip any data for the results. + - `fetch=10`: Limits the results to 10 values. + +**Physical plan operators** + +- `ParquetExec` + - `file_groups={10 groups: [...]}`: Reads 10 groups in parallel from `hits.parquet`file. (The example above was run on a machine with 10 cores.) + - `projection=[UserID]`: Pushes down projection of the `UserID` column. The parquet format is columnar and the DataFusion reader only decodes the columns required. +- `AggregateExec` + - `mode=Partial` Runs a [partial aggregation] in parallel across each of the 10 partitions from the `ParquetExec` immediately after reading. + - `gby=[UserID@0 as UserID]`: Represents `GROUP BY` in the [physical plan] and groups together the same values of `UserID`. + - `aggr=[count(*)]`: Applies the `COUNT` aggregate on all rows for each group. +- `RepartitionExec` + - `partitioning=Hash([UserID@0], 10)`: Divides the input into into 10 (new) output partitions based on the value of `hash(UserID)`. You can read more about this in the [partitioning] documentation. + - `input_partitions=10`: Number of input partitions. +- `CoalesceBatchesExec` + - `target_batch_size=8192`: Combines smaller batches in to larger batches. In this case approximately 8192 rows in each batch. +- `AggregateExec` + - `mode=FinalPartitioned`: Performs the final aggregation on each group. See the [documentation on multi phase grouping] for more information. + - `gby=[UserID@0 as UserID]`: Groups by `UserID`. + - `aggr=[count(*)]`: Applies the `COUNT` aggregate on all rows for each group. +- `SortExec` + - `TopK(fetch=10)`: Use a special "TopK" sort that keeps only the largest 10 values in memory at a time. You can read more about this in the [TopK] documentation. + - `expr=[count(*)@1 DESC]`: Sorts all rows in descending order. Note this represents the `ORDER BY` in the physical plan. + - `preserve_partitioning=[true]`: The sort is done in parallel on each partition. In this case the top 10 values are found for each of the 10 partitions, in parallel. +- `SortPreservingMergeExec` + - `[count(*)@1 DESC]`: This operator merges the 10 distinct streams into a single stream using this expression. + - `fetch=10`: Returns only the first 10 rows +- `GlobalLimitExec` + - `skip=0`: Does not skip any rows + - `fetch=10`: Returns only the first 10 rows, denoted by `LIMIT 10` in the query. + +[partial aggregation]: https://docs.rs/datafusion/latest/datafusion/physical_plan/aggregates/enum.AggregateMode.html#variant.Partial +[physical plan]: https://docs.rs/datafusion/latest/datafusion/physical_plan/aggregates/struct.PhysicalGroupBy.html +[partitioning]: https://docs.rs/datafusion/latest/datafusion/physical_plan/repartition/struct.RepartitionExec.html +[topk]: https://docs.rs/datafusion/latest/datafusion/physical_plan/struct.TopK.html +[documentation on multi phase grouping]: https://docs.rs/datafusion/latest/datafusion/physical_plan/trait.Accumulator.html#tymethod.state + +### Data in this Example + +The examples in this section use data from [ClickBench], a benchmark for data +analytics. The examples are in terms of the 14GB [`hits.parquet`] file and can be +downloaded from the website or using the following commands: + +```shell +cd benchmarks +./bench.sh data clickbench_1 +*************************** +DataFusion Benchmark Runner and Data Generator +COMMAND: data +BENCHMARK: clickbench_1 +DATA_DIR: /Users/andrewlamb/Software/datafusion2/benchmarks/data +CARGO_COMMAND: cargo run --release +PREFER_HASH_JOIN: true +*************************** +Checking hits.parquet...... found 14779976446 bytes ... Done +``` + +Then you can run `datafusion-cli` to get plans: + +```shell +cd datafusion/benchmarks/data +datafusion-cli + +DataFusion CLI v41.0.0 +> select count(*) from 'hits.parquet'; ++----------+ +| count(*) | ++----------+ +| 99997497 | ++----------+ +1 row(s) fetched. +Elapsed 0.062 seconds. +> +``` + +[clickbench]: https://benchmark.clickhouse.com/ +[`hits.parquet`]: https://datasets.clickhouse.com/hits_compatible/hits.parquet diff --git a/docs/source/user-guide/sql/explain.md b/docs/source/user-guide/sql/explain.md index 22f73e3d76d7..45bb3a57aa7c 100644 --- a/docs/source/user-guide/sql/explain.md +++ b/docs/source/user-guide/sql/explain.md @@ -21,6 +21,8 @@ The `EXPLAIN` command shows the logical and physical execution plan for the specified SQL statement. +See the [Reading Explain Plans](../explain-usage.md) page for more information on how to interpret these plans. +
 EXPLAIN [ANALYZE] [VERBOSE] statement
 
From 5163e1559f9d84d04bdd967adf34ad07e63b249d Mon Sep 17 00:00:00 2001 From: WeblWabl Date: Wed, 28 Aug 2024 07:29:16 -0500 Subject: [PATCH 041/154] Make RuntimeEnvBuilder rather than RuntimeConfig (#12157) * feat/12156: Make RuntimeEnvBuilder rather than RuntimeConfig Signed-off-by: Devan * feat/12156: Make RuntimeEnvBuilder rather than RuntimeConfig Signed-off-by: Devan * doc link Signed-off-by: Devan * update to use builder for rt env Signed-off-by: Devan * update to use builder Signed-off-by: Devan * clippy Signed-off-by: Devan * touch Signed-off-by: Devan * fmt Signed-off-by: Devan * revert some formatting that occurred Signed-off-by: Devan * revert some formatting that occurred Signed-off-by: Devan * use builder Signed-off-by: Devan * fmt Signed-off-by: Devan * Update datafusion/execution/src/runtime_env.rs Co-authored-by: Andrew Lamb --------- Signed-off-by: Devan Co-authored-by: Devan Co-authored-by: Andrew Lamb --- .../core/src/datasource/file_format/csv.rs | 4 +- datafusion/core/src/execution/context/mod.rs | 13 +++--- datafusion/core/tests/fuzz_cases/sort_fuzz.rs | 12 +++--- datafusion/core/tests/memory_limit/mod.rs | 12 +++--- .../core/tests/parquet/file_statistics.rs | 7 ++- datafusion/execution/src/runtime_env.rs | 27 +++++++++--- datafusion/execution/src/task.rs | 5 ++- .../physical-plan/src/aggregates/mod.rs | 15 ++++--- .../physical-plan/src/joins/cross_join.rs | 9 ++-- .../physical-plan/src/joins/hash_join.rs | 16 ++++--- .../src/joins/nested_loop_join.rs | 9 ++-- .../src/joins/sort_merge_join.rs | 43 +++++++++++-------- .../physical-plan/src/repartition/mod.rs | 6 ++- datafusion/physical-plan/src/sorts/sort.rs | 21 +++++---- datafusion/wasmtest/src/lib.rs | 13 +++--- 15 files changed, 129 insertions(+), 83 deletions(-) diff --git a/datafusion/core/src/datasource/file_format/csv.rs b/datafusion/core/src/datasource/file_format/csv.rs index d1ce2afcccf3..e43f6ab29abc 100644 --- a/datafusion/core/src/datasource/file_format/csv.rs +++ b/datafusion/core/src/datasource/file_format/csv.rs @@ -680,7 +680,7 @@ mod tests { use datafusion_common::cast::as_string_array; use datafusion_common::internal_err; use datafusion_common::stats::Precision; - use datafusion_execution::runtime_env::{RuntimeConfig, RuntimeEnv}; + use datafusion_execution::runtime_env::RuntimeEnvBuilder; use datafusion_expr::{col, lit}; use crate::execution::session_state::SessionStateBuilder; @@ -863,7 +863,7 @@ mod tests { async fn query_compress_data( file_compression_type: FileCompressionType, ) -> Result<()> { - let runtime = Arc::new(RuntimeEnv::new(RuntimeConfig::new()).unwrap()); + let runtime = Arc::new(RuntimeEnvBuilder::new().build()?); let mut cfg = SessionConfig::new(); cfg.options_mut().catalog.has_header = true; let session_state = SessionStateBuilder::new() diff --git a/datafusion/core/src/execution/context/mod.rs b/datafusion/core/src/execution/context/mod.rs index 18943599b136..c67424c0fa53 100644 --- a/datafusion/core/src/execution/context/mod.rs +++ b/datafusion/core/src/execution/context/mod.rs @@ -212,15 +212,15 @@ where /// # use std::sync::Arc; /// # use datafusion::prelude::*; /// # use datafusion::execution::SessionStateBuilder; -/// # use datafusion_execution::runtime_env::{RuntimeConfig, RuntimeEnv}; +/// # use datafusion_execution::runtime_env::RuntimeEnvBuilder; /// // Configure a 4k batch size /// let config = SessionConfig::new() .with_batch_size(4 * 1024); /// /// // configure a memory limit of 1GB with 20% slop -/// let runtime_env = RuntimeEnv::new( -/// RuntimeConfig::new() +/// let runtime_env = RuntimeEnvBuilder::new() /// .with_memory_limit(1024 * 1024 * 1024, 0.80) -/// ).unwrap(); +/// .build() +/// .unwrap(); /// /// // Create a SessionState using the config and runtime_env /// let state = SessionStateBuilder::new() @@ -1623,7 +1623,7 @@ mod tests { use super::{super::options::CsvReadOptions, *}; use crate::assert_batches_eq; use crate::execution::memory_pool::MemoryConsumer; - use crate::execution::runtime_env::RuntimeConfig; + use crate::execution::runtime_env::RuntimeEnvBuilder; use crate::test; use crate::test_util::{plan_and_collect, populate_csv_partitions}; @@ -1758,8 +1758,7 @@ mod tests { let path = path.join("tests/tpch-csv"); let url = format!("file://{}", path.display()); - let rt_cfg = RuntimeConfig::new(); - let runtime = Arc::new(RuntimeEnv::new(rt_cfg).unwrap()); + let runtime = Arc::new(RuntimeEnvBuilder::new().build()?); let cfg = SessionConfig::new() .set_str("datafusion.catalog.location", url.as_str()) .set_str("datafusion.catalog.format", "CSV") diff --git a/datafusion/core/tests/fuzz_cases/sort_fuzz.rs b/datafusion/core/tests/fuzz_cases/sort_fuzz.rs index eda306dd3de5..1980589491a5 100644 --- a/datafusion/core/tests/fuzz_cases/sort_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/sort_fuzz.rs @@ -22,7 +22,7 @@ use arrow::{ compute::SortOptions, record_batch::RecordBatch, }; -use datafusion::execution::runtime_env::RuntimeConfig; +use datafusion::execution::runtime_env::RuntimeEnvBuilder; use datafusion::physical_plan::expressions::PhysicalSortExpr; use datafusion::physical_plan::memory::MemoryExec; use datafusion::physical_plan::sorts::sort::SortExec; @@ -136,10 +136,12 @@ impl SortTest { .sort_spill_reservation_bytes, ); - let runtime_env = RuntimeConfig::new() - .with_memory_pool(Arc::new(GreedyMemoryPool::new(pool_size))) - .build(); - let runtime = Arc::new(runtime_env.unwrap()); + let runtime = Arc::new( + RuntimeEnvBuilder::new() + .with_memory_pool(Arc::new(GreedyMemoryPool::new(pool_size))) + .build() + .unwrap(), + ); SessionContext::new_with_config_rt(session_config, runtime) } else { SessionContext::new_with_config(session_config) diff --git a/datafusion/core/tests/memory_limit/mod.rs b/datafusion/core/tests/memory_limit/mod.rs index dbd5592e8020..592c25dedc50 100644 --- a/datafusion/core/tests/memory_limit/mod.rs +++ b/datafusion/core/tests/memory_limit/mod.rs @@ -40,7 +40,7 @@ use tokio::fs::File; use datafusion::datasource::streaming::StreamingTable; use datafusion::datasource::{MemTable, TableProvider}; use datafusion::execution::disk_manager::DiskManagerConfig; -use datafusion::execution::runtime_env::{RuntimeConfig, RuntimeEnv}; +use datafusion::execution::runtime_env::RuntimeEnvBuilder; use datafusion::execution::session_state::SessionStateBuilder; use datafusion::physical_optimizer::join_selection::JoinSelection; use datafusion::physical_plan::stream::RecordBatchStreamAdapter; @@ -509,17 +509,17 @@ impl TestCase { let table = scenario.table(); - let mut rt_config = RuntimeConfig::new() + let rt_config = RuntimeEnvBuilder::new() // disk manager setting controls the spilling .with_disk_manager(disk_manager_config) .with_memory_limit(memory_limit, MEMORY_FRACTION); - if let Some(pool) = memory_pool { - rt_config = rt_config.with_memory_pool(pool); + let runtime = if let Some(pool) = memory_pool { + rt_config.with_memory_pool(pool).build().unwrap() + } else { + rt_config.build().unwrap() }; - let runtime = RuntimeEnv::new(rt_config).unwrap(); - // Configure execution let builder = SessionStateBuilder::new() .with_config(config) diff --git a/datafusion/core/tests/parquet/file_statistics.rs b/datafusion/core/tests/parquet/file_statistics.rs index bf25b36f48e8..bd251f1a6669 100644 --- a/datafusion/core/tests/parquet/file_statistics.rs +++ b/datafusion/core/tests/parquet/file_statistics.rs @@ -33,7 +33,7 @@ use datafusion_execution::cache::cache_unit::{ DefaultFileStatisticsCache, DefaultListFilesCache, }; use datafusion_execution::config::SessionConfig; -use datafusion_execution::runtime_env::{RuntimeConfig, RuntimeEnv}; +use datafusion_execution::runtime_env::RuntimeEnvBuilder; use datafusion::execution::session_state::SessionStateBuilder; use tempfile::tempdir; @@ -198,7 +198,10 @@ fn get_cache_runtime_state() -> ( .with_list_files_cache(Some(list_file_cache.clone())); let rt = Arc::new( - RuntimeEnv::new(RuntimeConfig::new().with_cache_manager(cache_config)).unwrap(), + RuntimeEnvBuilder::new() + .with_cache_manager(cache_config) + .build() + .expect("could not build runtime environment"), ); let state = SessionContext::new_with_config_rt(SessionConfig::default(), rt).state(); diff --git a/datafusion/execution/src/runtime_env.rs b/datafusion/execution/src/runtime_env.rs index bddce81f537b..e7b48be95cff 100644 --- a/datafusion/execution/src/runtime_env.rs +++ b/datafusion/execution/src/runtime_env.rs @@ -41,7 +41,7 @@ use url::Url; /// Execution runtime environment that manages system resources such /// as memory, disk, cache and storage. /// -/// A [`RuntimeEnv`] is created from a [`RuntimeConfig`] and has the +/// A [`RuntimeEnv`] is created from a [`RuntimeEnvBuilder`] and has the /// following resource management functionality: /// /// * [`MemoryPool`]: Manage memory @@ -147,13 +147,17 @@ impl RuntimeEnv { impl Default for RuntimeEnv { fn default() -> Self { - RuntimeEnv::new(RuntimeConfig::new()).unwrap() + RuntimeEnvBuilder::new().build().unwrap() } } +/// Please see: +/// This a type alias for backwards compatibility. +pub type RuntimeConfig = RuntimeEnvBuilder; + #[derive(Clone)] /// Execution runtime configuration -pub struct RuntimeConfig { +pub struct RuntimeEnvBuilder { /// DiskManager to manage temporary disk file usage pub disk_manager: DiskManagerConfig, /// [`MemoryPool`] from which to allocate memory @@ -166,13 +170,13 @@ pub struct RuntimeConfig { pub object_store_registry: Arc, } -impl Default for RuntimeConfig { +impl Default for RuntimeEnvBuilder { fn default() -> Self { Self::new() } } -impl RuntimeConfig { +impl RuntimeEnvBuilder { /// New with default values pub fn new() -> Self { Self { @@ -229,8 +233,17 @@ impl RuntimeConfig { self.with_disk_manager(DiskManagerConfig::new_specified(vec![path.into()])) } - /// Build a `RuntimeEnv` object from the configuration + /// Build a RuntimeEnv pub fn build(self) -> Result { - RuntimeEnv::new(self) + let memory_pool = self + .memory_pool + .unwrap_or_else(|| Arc::new(UnboundedMemoryPool::default())); + + Ok(RuntimeEnv { + memory_pool, + disk_manager: DiskManager::try_new(self.disk_manager)?, + cache_manager: CacheManager::try_new(&self.cache_manager)?, + object_store_registry: self.object_store_registry, + }) } } diff --git a/datafusion/execution/src/task.rs b/datafusion/execution/src/task.rs index 21a644284c42..35689b8e08df 100644 --- a/datafusion/execution/src/task.rs +++ b/datafusion/execution/src/task.rs @@ -24,7 +24,7 @@ use crate::{ config::SessionConfig, memory_pool::MemoryPool, registry::FunctionRegistry, - runtime_env::{RuntimeConfig, RuntimeEnv}, + runtime_env::{RuntimeEnv, RuntimeEnvBuilder}, }; use datafusion_common::{plan_datafusion_err, DataFusionError, Result}; use datafusion_expr::planner::ExprPlanner; @@ -57,7 +57,8 @@ pub struct TaskContext { impl Default for TaskContext { fn default() -> Self { - let runtime = RuntimeEnv::new(RuntimeConfig::new()) + let runtime = RuntimeEnvBuilder::new() + .build() .expect("default runtime created successfully"); // Create a default task context, mostly useful for testing diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 6ca01928c80f..b5447a2d7317 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -1212,7 +1212,7 @@ mod tests { }; use datafusion_execution::config::SessionConfig; use datafusion_execution::memory_pool::FairSpillPool; - use datafusion_execution::runtime_env::{RuntimeConfig, RuntimeEnv}; + use datafusion_execution::runtime_env::RuntimeEnvBuilder; use datafusion_functions_aggregate::array_agg::array_agg_udaf; use datafusion_functions_aggregate::average::avg_udaf; use datafusion_functions_aggregate::count::count_udaf; @@ -1324,11 +1324,10 @@ mod tests { fn new_spill_ctx(batch_size: usize, max_memory: usize) -> Arc { let session_config = SessionConfig::new().with_batch_size(batch_size); let runtime = Arc::new( - RuntimeEnv::new( - RuntimeConfig::default() - .with_memory_pool(Arc::new(FairSpillPool::new(max_memory))), - ) - .unwrap(), + RuntimeEnvBuilder::default() + .with_memory_pool(Arc::new(FairSpillPool::new(max_memory))) + .build() + .unwrap(), ); let task_ctx = TaskContext::default() .with_session_config(session_config) @@ -1809,7 +1808,9 @@ mod tests { let input_schema = input.schema(); let runtime = Arc::new( - RuntimeEnv::new(RuntimeConfig::default().with_memory_limit(1, 1.0)).unwrap(), + RuntimeEnvBuilder::default() + .with_memory_limit(1, 1.0) + .build()?, ); let task_ctx = TaskContext::default().with_runtime(runtime); let task_ctx = Arc::new(task_ctx); diff --git a/datafusion/physical-plan/src/joins/cross_join.rs b/datafusion/physical-plan/src/joins/cross_join.rs index 0868ee721665..b99d4f17c42a 100644 --- a/datafusion/physical-plan/src/joins/cross_join.rs +++ b/datafusion/physical-plan/src/joins/cross_join.rs @@ -488,7 +488,7 @@ mod tests { use crate::test::build_table_scan_i32; use datafusion_common::{assert_batches_sorted_eq, assert_contains}; - use datafusion_execution::runtime_env::{RuntimeConfig, RuntimeEnv}; + use datafusion_execution::runtime_env::RuntimeEnvBuilder; async fn join_collect( left: Arc, @@ -673,8 +673,11 @@ mod tests { #[tokio::test] async fn test_overallocation() -> Result<()> { - let runtime_config = RuntimeConfig::new().with_memory_limit(100, 1.0); - let runtime = Arc::new(RuntimeEnv::new(runtime_config)?); + let runtime = Arc::new( + RuntimeEnvBuilder::new() + .with_memory_limit(100, 1.0) + .build()?, + ); let task_ctx = TaskContext::default().with_runtime(runtime); let task_ctx = Arc::new(task_ctx); diff --git a/datafusion/physical-plan/src/joins/hash_join.rs b/datafusion/physical-plan/src/joins/hash_join.rs index 7fac23ad5557..f20d00e1a298 100644 --- a/datafusion/physical-plan/src/joins/hash_join.rs +++ b/datafusion/physical-plan/src/joins/hash_join.rs @@ -1572,7 +1572,7 @@ mod tests { ScalarValue, }; use datafusion_execution::config::SessionConfig; - use datafusion_execution::runtime_env::{RuntimeConfig, RuntimeEnv}; + use datafusion_execution::runtime_env::RuntimeEnvBuilder; use datafusion_expr::Operator; use datafusion_physical_expr::expressions::{BinaryExpr, Literal}; @@ -3798,8 +3798,11 @@ mod tests { ]; for join_type in join_types { - let runtime_config = RuntimeConfig::new().with_memory_limit(100, 1.0); - let runtime = Arc::new(RuntimeEnv::new(runtime_config)?); + let runtime = Arc::new( + RuntimeEnvBuilder::new() + .with_memory_limit(100, 1.0) + .build()?, + ); let task_ctx = TaskContext::default().with_runtime(runtime); let task_ctx = Arc::new(task_ctx); @@ -3871,8 +3874,11 @@ mod tests { ]; for join_type in join_types { - let runtime_config = RuntimeConfig::new().with_memory_limit(100, 1.0); - let runtime = Arc::new(RuntimeEnv::new(runtime_config)?); + let runtime = Arc::new( + RuntimeEnvBuilder::new() + .with_memory_limit(100, 1.0) + .build()?, + ); let session_config = SessionConfig::default().with_batch_size(50); let task_ctx = TaskContext::default() .with_session_config(session_config) diff --git a/datafusion/physical-plan/src/joins/nested_loop_join.rs b/datafusion/physical-plan/src/joins/nested_loop_join.rs index 18de2de03192..3cd373544157 100644 --- a/datafusion/physical-plan/src/joins/nested_loop_join.rs +++ b/datafusion/physical-plan/src/joins/nested_loop_join.rs @@ -644,7 +644,7 @@ mod tests { use arrow::datatypes::{DataType, Field}; use datafusion_common::{assert_batches_sorted_eq, assert_contains, ScalarValue}; - use datafusion_execution::runtime_env::{RuntimeConfig, RuntimeEnv}; + use datafusion_execution::runtime_env::RuntimeEnvBuilder; use datafusion_expr::Operator; use datafusion_physical_expr::expressions::{BinaryExpr, Literal}; use datafusion_physical_expr::{Partitioning, PhysicalExpr}; @@ -1019,8 +1019,11 @@ mod tests { ]; for join_type in join_types { - let runtime_config = RuntimeConfig::new().with_memory_limit(100, 1.0); - let runtime = Arc::new(RuntimeEnv::new(runtime_config)?); + let runtime = Arc::new( + RuntimeEnvBuilder::new() + .with_memory_limit(100, 1.0) + .build()?, + ); let task_ctx = TaskContext::default().with_runtime(runtime); let task_ctx = Arc::new(task_ctx); diff --git a/datafusion/physical-plan/src/joins/sort_merge_join.rs b/datafusion/physical-plan/src/joins/sort_merge_join.rs index 511cb4c55fcd..09fe5d9ebc54 100644 --- a/datafusion/physical-plan/src/joins/sort_merge_join.rs +++ b/datafusion/physical-plan/src/joins/sort_merge_join.rs @@ -1978,7 +1978,7 @@ mod tests { }; use datafusion_execution::config::SessionConfig; use datafusion_execution::disk_manager::DiskManagerConfig; - use datafusion_execution::runtime_env::{RuntimeConfig, RuntimeEnv}; + use datafusion_execution::runtime_env::RuntimeEnvBuilder; use datafusion_execution::TaskContext; use crate::expressions::Column; @@ -2900,10 +2900,12 @@ mod tests { ]; // Disable DiskManager to prevent spilling - let runtime_config = RuntimeConfig::new() - .with_memory_limit(100, 1.0) - .with_disk_manager(DiskManagerConfig::Disabled); - let runtime = Arc::new(RuntimeEnv::new(runtime_config)?); + let runtime = Arc::new( + RuntimeEnvBuilder::new() + .with_memory_limit(100, 1.0) + .with_disk_manager(DiskManagerConfig::Disabled) + .build()?, + ); let session_config = SessionConfig::default().with_batch_size(50); for join_type in join_types { @@ -2985,10 +2987,12 @@ mod tests { ]; // Disable DiskManager to prevent spilling - let runtime_config = RuntimeConfig::new() - .with_memory_limit(100, 1.0) - .with_disk_manager(DiskManagerConfig::Disabled); - let runtime = Arc::new(RuntimeEnv::new(runtime_config)?); + let runtime = Arc::new( + RuntimeEnvBuilder::new() + .with_memory_limit(100, 1.0) + .with_disk_manager(DiskManagerConfig::Disabled) + .build()?, + ); let session_config = SessionConfig::default().with_batch_size(50); for join_type in join_types { @@ -3048,10 +3052,12 @@ mod tests { ]; // Enable DiskManager to allow spilling - let runtime_config = RuntimeConfig::new() - .with_memory_limit(100, 1.0) - .with_disk_manager(DiskManagerConfig::NewOs); - let runtime = Arc::new(RuntimeEnv::new(runtime_config)?); + let runtime = Arc::new( + RuntimeEnvBuilder::new() + .with_memory_limit(100, 1.0) + .with_disk_manager(DiskManagerConfig::NewOs) + .build()?, + ); for batch_size in [1, 50] { let session_config = SessionConfig::default().with_batch_size(batch_size); @@ -3156,10 +3162,13 @@ mod tests { ]; // Enable DiskManager to allow spilling - let runtime_config = RuntimeConfig::new() - .with_memory_limit(500, 1.0) - .with_disk_manager(DiskManagerConfig::NewOs); - let runtime = Arc::new(RuntimeEnv::new(runtime_config)?); + let runtime = Arc::new( + RuntimeEnvBuilder::new() + .with_memory_limit(500, 1.0) + .with_disk_manager(DiskManagerConfig::NewOs) + .build()?, + ); + for batch_size in [1, 50] { let session_config = SessionConfig::default().with_batch_size(batch_size); diff --git a/datafusion/physical-plan/src/repartition/mod.rs b/datafusion/physical-plan/src/repartition/mod.rs index 5a3fcb5029e1..650006a9d02d 100644 --- a/datafusion/physical-plan/src/repartition/mod.rs +++ b/datafusion/physical-plan/src/repartition/mod.rs @@ -1025,7 +1025,7 @@ mod tests { use arrow::datatypes::{DataType, Field, Schema}; use datafusion_common::cast::as_string_array; use datafusion_common::{assert_batches_sorted_eq, exec_err}; - use datafusion_execution::runtime_env::{RuntimeConfig, RuntimeEnv}; + use datafusion_execution::runtime_env::RuntimeEnvBuilder; use tokio::task::JoinSet; @@ -1507,7 +1507,9 @@ mod tests { // setup up context let runtime = Arc::new( - RuntimeEnv::new(RuntimeConfig::default().with_memory_limit(1, 1.0)).unwrap(), + RuntimeEnvBuilder::default() + .with_memory_limit(1, 1.0) + .build()?, ); let task_ctx = TaskContext::default().with_runtime(runtime); diff --git a/datafusion/physical-plan/src/sorts/sort.rs b/datafusion/physical-plan/src/sorts/sort.rs index a81b09948cca..e92a57493141 100644 --- a/datafusion/physical-plan/src/sorts/sort.rs +++ b/datafusion/physical-plan/src/sorts/sort.rs @@ -966,7 +966,7 @@ mod tests { use arrow::datatypes::*; use datafusion_common::cast::as_primitive_array; use datafusion_execution::config::SessionConfig; - use datafusion_execution::runtime_env::RuntimeConfig; + use datafusion_execution::runtime_env::RuntimeEnvBuilder; use datafusion_common::ScalarValue; use datafusion_physical_expr::expressions::Literal; @@ -1009,9 +1009,11 @@ mod tests { .options() .execution .sort_spill_reservation_bytes; - let rt_config = RuntimeConfig::new() - .with_memory_limit(sort_spill_reservation_bytes + 12288, 1.0); - let runtime = Arc::new(RuntimeEnv::new(rt_config)?); + let runtime = Arc::new( + RuntimeEnvBuilder::new() + .with_memory_limit(sort_spill_reservation_bytes + 12288, 1.0) + .build()?, + ); let task_ctx = Arc::new( TaskContext::default() .with_session_config(session_config) @@ -1085,11 +1087,14 @@ mod tests { .execution .sort_spill_reservation_bytes; - let rt_config = RuntimeConfig::new().with_memory_limit( - sort_spill_reservation_bytes + avg_batch_size * (partitions - 1), - 1.0, + let runtime = Arc::new( + RuntimeEnvBuilder::new() + .with_memory_limit( + sort_spill_reservation_bytes + avg_batch_size * (partitions - 1), + 1.0, + ) + .build()?, ); - let runtime = Arc::new(RuntimeEnv::new(rt_config)?); let task_ctx = Arc::new( TaskContext::default() .with_runtime(runtime) diff --git a/datafusion/wasmtest/src/lib.rs b/datafusion/wasmtest/src/lib.rs index a74cce72ac64..50325d262d1d 100644 --- a/datafusion/wasmtest/src/lib.rs +++ b/datafusion/wasmtest/src/lib.rs @@ -78,9 +78,8 @@ mod test { use super::*; use datafusion::execution::context::SessionContext; use datafusion_execution::{ - config::SessionConfig, - disk_manager::DiskManagerConfig, - runtime_env::{RuntimeConfig, RuntimeEnv}, + config::SessionConfig, disk_manager::DiskManagerConfig, + runtime_env::RuntimeEnvBuilder, }; use datafusion_physical_plan::collect; use datafusion_sql::parser::DFParser; @@ -100,10 +99,10 @@ mod test { // Execute SQL (using datafusion) let rt = Arc::new( - RuntimeEnv::new( - RuntimeConfig::new().with_disk_manager(DiskManagerConfig::Disabled), - ) - .unwrap(), + RuntimeEnvBuilder::new() + .with_disk_manager(DiskManagerConfig::Disabled) + .build() + .unwrap(), ); let session_config = SessionConfig::new().with_target_partitions(1); let session_context = From 1fce2a98ef9c7f8dbd7f3dedcaf4aa069ab92154 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=BC=A0=E6=9E=97=E4=BC=9F?= Date: Wed, 28 Aug 2024 23:21:24 +0800 Subject: [PATCH 042/154] Minor: Fix grouping set typo (#12216) * Fix grouping set typo * More --- .../physical-optimizer/src/limited_distinct_aggregation.rs | 4 ++-- datafusion/physical-plan/src/aggregates/mod.rs | 6 +++--- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/datafusion/physical-optimizer/src/limited_distinct_aggregation.rs b/datafusion/physical-optimizer/src/limited_distinct_aggregation.rs index e18e530072db..8653ad19da77 100644 --- a/datafusion/physical-optimizer/src/limited_distinct_aggregation.rs +++ b/datafusion/physical-optimizer/src/limited_distinct_aggregation.rs @@ -33,8 +33,8 @@ use itertools::Itertools; /// An optimizer rule that passes a `limit` hint into grouped aggregations which don't require all /// rows in the group to be processed for correctness. Example queries fitting this description are: -/// `SELECT distinct l_orderkey FROM lineitem LIMIT 10;` -/// `SELECT l_orderkey FROM lineitem GROUP BY l_orderkey LIMIT 10;` +/// - `SELECT distinct l_orderkey FROM lineitem LIMIT 10;` +/// - `SELECT l_orderkey FROM lineitem GROUP BY l_orderkey LIMIT 10;` pub struct LimitedDistinctAggregation {} impl LimitedDistinctAggregation { diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index b5447a2d7317..0f33a9d7b992 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -111,9 +111,9 @@ impl AggregateMode { /// Represents `GROUP BY` clause in the plan (including the more general GROUPING SET) /// In the case of a simple `GROUP BY a, b` clause, this will contain the expression [a, b] /// and a single group [false, false]. -/// In the case of `GROUP BY GROUPING SET/CUBE/ROLLUP` the planner will expand the expression +/// In the case of `GROUP BY GROUPING SETS/CUBE/ROLLUP` the planner will expand the expression /// into multiple groups, using null expressions to align each group. -/// For example, with a group by clause `GROUP BY GROUPING SET ((a,b),(a),(b))` the planner should +/// For example, with a group by clause `GROUP BY GROUPING SETS ((a,b),(a),(b))` the planner should /// create a `PhysicalGroupBy` like /// ```text /// PhysicalGroupBy { @@ -134,7 +134,7 @@ pub struct PhysicalGroupBy { null_expr: Vec<(Arc, String)>, /// Null mask for each group in this grouping set. Each group is /// composed of either one of the group expressions in expr or a null - /// expression in null_expr. If `groups[i][j]` is true, then the the + /// expression in null_expr. If `groups[i][j]` is true, then the /// j-th expression in the i-th group is NULL, otherwise it is `expr[j]`. groups: Vec>, } From bd506980bd04c109d9fa979be5b627580e59d267 Mon Sep 17 00:00:00 2001 From: Austin Liu Date: Thu, 29 Aug 2024 12:49:04 +0800 Subject: [PATCH 043/154] Add `array_distance` function (#12211) * Add `distance` aggregation function Signed-off-by: Austin Liu Add `distance` aggregation function Signed-off-by: Austin Liu * Add sql logic test for `distance` Signed-off-by: Austin Liu * Simplify diff calculation Signed-off-by: Austin Liu * Add `array_distance`/`list_distance` as list function in functions-nested Signed-off-by: Austin Liu * Remove aggregate function `distance` Signed-off-by: Austin Liu * format Signed-off-by: Austin Liu * clean up error handling Signed-off-by: Austin Liu * Add `array_distance` in scalar array functions docs Signed-off-by: Austin Liu * Update bulletin Signed-off-by: Austin Liu * Prettify example Signed-off-by: Austin Liu --------- Signed-off-by: Austin Liu --- datafusion/functions-nested/src/distance.rs | 215 ++++++++++++++++++ datafusion/functions-nested/src/lib.rs | 3 + datafusion/sqllogictest/test_files/array.slt | 54 +++++ .../source/user-guide/sql/scalar_functions.md | 36 +++ 4 files changed, 308 insertions(+) create mode 100644 datafusion/functions-nested/src/distance.rs diff --git a/datafusion/functions-nested/src/distance.rs b/datafusion/functions-nested/src/distance.rs new file mode 100644 index 000000000000..4d779dc1fabf --- /dev/null +++ b/datafusion/functions-nested/src/distance.rs @@ -0,0 +1,215 @@ +// 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. + +//! [ScalarUDFImpl] definitions for array_distance function. + +use crate::utils::{downcast_arg, make_scalar_function}; +use arrow_array::{ + Array, ArrayRef, Float64Array, LargeListArray, ListArray, OffsetSizeTrait, +}; +use arrow_schema::DataType; +use arrow_schema::DataType::{FixedSizeList, Float64, LargeList, List}; +use core::any::type_name; +use datafusion_common::cast::{ + as_float32_array, as_float64_array, as_generic_list_array, as_int32_array, + as_int64_array, +}; +use datafusion_common::DataFusionError; +use datafusion_common::{exec_err, Result}; +use datafusion_expr::{ColumnarValue, ScalarUDFImpl, Signature, Volatility}; +use std::any::Any; +use std::sync::Arc; + +make_udf_expr_and_func!( + ArrayDistance, + array_distance, + array, + "returns the Euclidean distance between two numeric arrays.", + array_distance_udf +); + +#[derive(Debug)] +pub(super) struct ArrayDistance { + signature: Signature, + aliases: Vec, +} + +impl ArrayDistance { + pub fn new() -> Self { + Self { + signature: Signature::variadic_any(Volatility::Immutable), + aliases: vec!["list_distance".to_string()], + } + } +} + +impl ScalarUDFImpl for ArrayDistance { + fn as_any(&self) -> &dyn Any { + self + } + + fn name(&self) -> &str { + "array_distance" + } + + fn signature(&self) -> &Signature { + &self.signature + } + + fn return_type(&self, arg_types: &[DataType]) -> Result { + match arg_types[0] { + List(_) | LargeList(_) | FixedSizeList(_, _) => Ok(Float64), + _ => exec_err!("The array_distance function can only accept List/LargeList/FixedSizeList."), + } + } + + fn invoke(&self, args: &[ColumnarValue]) -> Result { + make_scalar_function(array_distance_inner)(args) + } + + fn aliases(&self) -> &[String] { + &self.aliases + } +} + +pub fn array_distance_inner(args: &[ArrayRef]) -> Result { + if args.len() != 2 { + return exec_err!("array_distance expects exactly two arguments"); + } + + match (&args[0].data_type(), &args[1].data_type()) { + (List(_), List(_)) => general_array_distance::(args), + (LargeList(_), LargeList(_)) => general_array_distance::(args), + (array_type1, array_type2) => { + exec_err!("array_distance does not support types '{array_type1:?}' and '{array_type2:?}'") + } + } +} + +fn general_array_distance(arrays: &[ArrayRef]) -> Result { + let list_array1 = as_generic_list_array::(&arrays[0])?; + let list_array2 = as_generic_list_array::(&arrays[1])?; + + let result = list_array1 + .iter() + .zip(list_array2.iter()) + .map(|(arr1, arr2)| compute_array_distance(arr1, arr2)) + .collect::>()?; + + Ok(Arc::new(result) as ArrayRef) +} + +/// Computes the Euclidean distance between two arrays +fn compute_array_distance( + arr1: Option, + arr2: Option, +) -> Result> { + let value1 = match arr1 { + Some(arr) => arr, + None => return Ok(None), + }; + let value2 = match arr2 { + Some(arr) => arr, + None => return Ok(None), + }; + + let mut value1 = value1; + let mut value2 = value2; + + loop { + match value1.data_type() { + List(_) => { + if downcast_arg!(value1, ListArray).null_count() > 0 { + return Ok(None); + } + value1 = downcast_arg!(value1, ListArray).value(0); + } + LargeList(_) => { + if downcast_arg!(value1, LargeListArray).null_count() > 0 { + return Ok(None); + } + value1 = downcast_arg!(value1, LargeListArray).value(0); + } + _ => break, + } + + match value2.data_type() { + List(_) => { + if downcast_arg!(value2, ListArray).null_count() > 0 { + return Ok(None); + } + value2 = downcast_arg!(value2, ListArray).value(0); + } + LargeList(_) => { + if downcast_arg!(value2, LargeListArray).null_count() > 0 { + return Ok(None); + } + value2 = downcast_arg!(value2, LargeListArray).value(0); + } + _ => break, + } + } + + // Check for NULL values inside the arrays + if value1.null_count() != 0 || value2.null_count() != 0 { + return Ok(None); + } + + let values1 = convert_to_f64_array(&value1)?; + let values2 = convert_to_f64_array(&value2)?; + + if values1.len() != values2.len() { + return exec_err!("Both arrays must have the same length"); + } + + let sum_squares: f64 = values1 + .iter() + .zip(values2.iter()) + .map(|(v1, v2)| { + let diff = v1.unwrap_or(0.0) - v2.unwrap_or(0.0); + diff * diff + }) + .sum(); + + Ok(Some(sum_squares.sqrt())) +} + +/// Converts an array of any numeric type to a Float64Array. +fn convert_to_f64_array(array: &ArrayRef) -> Result { + match array.data_type() { + DataType::Float64 => Ok(as_float64_array(array)?.clone()), + DataType::Float32 => { + let array = as_float32_array(array)?; + let converted: Float64Array = + array.iter().map(|v| v.map(|v| v as f64)).collect(); + Ok(converted) + } + DataType::Int64 => { + let array = as_int64_array(array)?; + let converted: Float64Array = + array.iter().map(|v| v.map(|v| v as f64)).collect(); + Ok(converted) + } + DataType::Int32 => { + let array = as_int32_array(array)?; + let converted: Float64Array = + array.iter().map(|v| v.map(|v| v as f64)).collect(); + Ok(converted) + } + _ => exec_err!("Unsupported array type for conversion to Float64Array"), + } +} diff --git a/datafusion/functions-nested/src/lib.rs b/datafusion/functions-nested/src/lib.rs index cc0a7b55cf86..863b5a876adc 100644 --- a/datafusion/functions-nested/src/lib.rs +++ b/datafusion/functions-nested/src/lib.rs @@ -34,6 +34,7 @@ pub mod array_has; pub mod cardinality; pub mod concat; pub mod dimension; +pub mod distance; pub mod empty; pub mod except; pub mod expr_ext; @@ -73,6 +74,7 @@ pub mod expr_fn { pub use super::concat::array_prepend; pub use super::dimension::array_dims; pub use super::dimension::array_ndims; + pub use super::distance::array_distance; pub use super::empty::array_empty; pub use super::except::array_except; pub use super::extract::array_element; @@ -128,6 +130,7 @@ pub fn all_default_nested_functions() -> Vec> { array_has::array_has_any_udf(), empty::array_empty_udf(), length::array_length_udf(), + distance::array_distance_udf(), flatten::flatten_udf(), sort::array_sort_udf(), repeat::array_repeat_udf(), diff --git a/datafusion/sqllogictest/test_files/array.slt b/datafusion/sqllogictest/test_files/array.slt index c80fd7e92417..edc0cd7577e1 100644 --- a/datafusion/sqllogictest/test_files/array.slt +++ b/datafusion/sqllogictest/test_files/array.slt @@ -4715,6 +4715,60 @@ NULL 10 NULL 10 NULL 10 +query RRR +select array_distance([2], [3]), list_distance([1], [2]), list_distance([1], [-2]); +---- +1 1 3 + +query error +select list_distance([1], [1, 2]); + +query R +select array_distance([[1, 1]], [1, 2]); +---- +1 + +query R +select array_distance([[1, 1]], [[1, 2]]); +---- +1 + +query R +select array_distance([[1, 1]], [[1, 2]]); +---- +1 + +query RR +select array_distance([1, 1, 0, 0], [2, 2, 1, 1]), list_distance([1, 2, 3], [1, 2, 3]); +---- +2 0 + +query RR +select array_distance([1.0, 1, 0, 0], [2, 2.0, 1, 1]), list_distance([1, 2.0, 3], [1, 2, 3]); +---- +2 0 + +query R +select list_distance([1, 1, NULL, 0], [2, 2, NULL, NULL]); +---- +NULL + +query R +select list_distance([NULL, NULL], [NULL, NULL]); +---- +NULL + +query R +select list_distance([1.0, 2.0, 3.0], [1.0, 2.0, 3.5]) AS distance; +---- +0.5 + +query R +select list_distance([1, 2, 3], [1, 2, 3]) AS distance; +---- +0 + + ## array_dims (aliases: `list_dims`) # array dims error diff --git a/docs/source/user-guide/sql/scalar_functions.md b/docs/source/user-guide/sql/scalar_functions.md index c7b3409ba7cd..9569f4e65ff3 100644 --- a/docs/source/user-guide/sql/scalar_functions.md +++ b/docs/source/user-guide/sql/scalar_functions.md @@ -2093,6 +2093,7 @@ to_unixtime(expression[, ..., format_n]) - [array_concat](#array_concat) - [array_contains](#array_contains) - [array_dims](#array_dims) +- [array_distance](#array_distance) - [array_distinct](#array_distinct) - [array_has](#array_has) - [array_has_all](#array_has_all) @@ -2135,6 +2136,7 @@ to_unixtime(expression[, ..., format_n]) - [list_cat](#list_cat) - [list_concat](#list_concat) - [list_dims](#list_dims) +- [list_distance](#list_distance) - [list_distinct](#list_distinct) - [list_element](#list_element) - [list_except](#list_except) @@ -2388,6 +2390,36 @@ array_dims(array) - list_dims +### `array_distance` + +Returns the Euclidean distance between two input arrays of equal length. + +``` +array_distance(array1, array2) +``` + +#### Arguments + +- **array1**: Array expression. + Can be a constant, column, or function, and any combination of array operators. +- **array2**: Array expression. + Can be a constant, column, or function, and any combination of array operators. + +#### Example + +``` +> select array_distance([1, 2], [1, 4]); ++------------------------------------+ +| array_distance(List([1,2], [1,4])) | ++------------------------------------+ +| 2.0 | ++------------------------------------+ +``` + +#### Aliases + +- list_distance + ### `array_distinct` Returns distinct values from the array after removing duplicates. @@ -3224,6 +3256,10 @@ _Alias of [array_concat](#array_concat)._ _Alias of [array_dims](#array_dims)._ +### `list_distance` + +_Alias of [array_distance](#array_distance)._ + ### `list_distinct` _Alias of [array_dims](#array_distinct)._ From f5dcdf00318e00f6e8e3200d81c1239c330a94c0 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Berkay=20=C5=9Eahin?= <124376117+berkaysynnada@users.noreply.github.com> Date: Thu, 29 Aug 2024 14:25:36 +0300 Subject: [PATCH 044/154] Unbounded SortExec (and Top-K) Implementation When Req's Are Satisfied (#12174) * Sort fetch updates execution mode * Update sort.rs * Update sort.rs * Update sort.rs * Update sort.rs * Update sort.rs * Apply suggestions from code review * Update sort.rs * Update datafusion/physical-plan/src/sorts/sort.rs * Update datafusion/physical-plan/src/sorts/sort.rs * Reuse LimitStream --------- Co-authored-by: Mehmet Ozan Kabak --- .../physical-plan/src/metrics/baseline.rs | 2 +- datafusion/physical-plan/src/sorts/sort.rs | 311 ++++++++++++++---- .../src/windows/bounded_window_agg_exec.rs | 41 ++- 3 files changed, 265 insertions(+), 89 deletions(-) diff --git a/datafusion/physical-plan/src/metrics/baseline.rs b/datafusion/physical-plan/src/metrics/baseline.rs index dc345cd8cdcd..b26a08dd0fad 100644 --- a/datafusion/physical-plan/src/metrics/baseline.rs +++ b/datafusion/physical-plan/src/metrics/baseline.rs @@ -56,7 +56,7 @@ pub struct BaselineMetrics { } impl BaselineMetrics { - /// Create a new BaselineMetric structure, and set `start_time` to now + /// Create a new BaselineMetric structure, and set `start_time` to now pub fn new(metrics: &ExecutionPlanMetricsSet, partition: usize) -> Self { let start_time = MetricBuilder::new(metrics).start_timestamp(partition); start_time.record(); diff --git a/datafusion/physical-plan/src/sorts/sort.rs b/datafusion/physical-plan/src/sorts/sort.rs index e92a57493141..e0041194016c 100644 --- a/datafusion/physical-plan/src/sorts/sort.rs +++ b/datafusion/physical-plan/src/sorts/sort.rs @@ -26,6 +26,7 @@ use std::sync::Arc; use crate::common::spawn_buffered; use crate::expressions::PhysicalSortExpr; +use crate::limit::LimitStream; use crate::metrics::{ BaselineMetrics, Count, ExecutionPlanMetricsSet, MetricBuilder, MetricsSet, }; @@ -51,6 +52,7 @@ use datafusion_execution::memory_pool::{MemoryConsumer, MemoryReservation}; use datafusion_execution::runtime_env::RuntimeEnv; use datafusion_execution::TaskContext; use datafusion_physical_expr::LexOrdering; +use datafusion_physical_expr_common::sort_expr::PhysicalSortRequirement; use futures::{StreamExt, TryStreamExt}; use log::{debug, trace}; @@ -737,9 +739,22 @@ impl SortExec { /// 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 + pub fn with_fetch(&self, fetch: Option) -> Self { + let mut cache = self.cache.clone(); + if fetch.is_some() && self.cache.execution_mode == ExecutionMode::Unbounded { + // When a theoretically unnecessary sort becomes a top-K (which + // sometimes arises as an intermediate state before full removal), + // its execution mode should become `Bounded`. + cache.execution_mode = ExecutionMode::Bounded; + } + SortExec { + input: Arc::clone(&self.input), + expr: self.expr.clone(), + metrics_set: self.metrics_set.clone(), + preserve_partitioning: self.preserve_partitioning, + fetch, + cache, + } } /// Input schema @@ -775,6 +790,16 @@ impl SortExec { sort_exprs: LexOrdering, preserve_partitioning: bool, ) -> PlanProperties { + // Determine execution mode: + let sort_satisfied = input.equivalence_properties().ordering_satisfy_requirement( + PhysicalSortRequirement::from_sort_exprs(sort_exprs.iter()).as_slice(), + ); + let mode = match input.execution_mode() { + ExecutionMode::Unbounded if sort_satisfied => ExecutionMode::Unbounded, + ExecutionMode::Bounded => ExecutionMode::Bounded, + _ => ExecutionMode::PipelineBreaking, + }; + // Calculate equivalence properties; i.e. reset the ordering equivalence // class with the new ordering: let eq_properties = input @@ -786,14 +811,6 @@ impl SortExec { let output_partitioning = Self::output_partitioning_helper(input, preserve_partitioning); - // Determine execution mode: - let mode = match input.execution_mode() { - ExecutionMode::Unbounded | ExecutionMode::PipelineBreaking => { - ExecutionMode::PipelineBreaking - } - ExecutionMode::Bounded => ExecutionMode::Bounded, - }; - PlanProperties::new(eq_properties, output_partitioning, mode) } } @@ -874,53 +891,68 @@ impl ExecutionPlan for SortExec { trace!("End SortExec's input.execute for partition: {}", partition); - if let Some(fetch) = self.fetch.as_ref() { - let mut topk = TopK::try_new( - partition, - input.schema(), - self.expr.clone(), - *fetch, - context.session_config().batch_size(), - context.runtime_env(), - &self.metrics_set, - partition, - )?; - - Ok(Box::pin(RecordBatchStreamAdapter::new( - self.schema(), - futures::stream::once(async move { - while let Some(batch) = input.next().await { - let batch = batch?; - topk.insert_batch(batch)?; - } - topk.emit() - }) - .try_flatten(), - ))) - } else { - let mut sorter = ExternalSorter::new( - partition, - input.schema(), - self.expr.clone(), - context.session_config().batch_size(), - self.fetch, - execution_options.sort_spill_reservation_bytes, - execution_options.sort_in_place_threshold_bytes, - &self.metrics_set, - context.runtime_env(), + let sort_satisfied = self + .input + .equivalence_properties() + .ordering_satisfy_requirement( + PhysicalSortRequirement::from_sort_exprs(self.expr.iter()).as_slice(), ); - Ok(Box::pin(RecordBatchStreamAdapter::new( - self.schema(), - futures::stream::once(async move { - while let Some(batch) = input.next().await { - let batch = batch?; - sorter.insert_batch(batch).await?; - } - sorter.sort() - }) - .try_flatten(), - ))) + match (sort_satisfied, self.fetch.as_ref()) { + (true, Some(fetch)) => Ok(Box::pin(LimitStream::new( + input, + 0, + Some(*fetch), + BaselineMetrics::new(&self.metrics_set, partition), + ))), + (true, None) => Ok(input), + (false, Some(fetch)) => { + let mut topk = TopK::try_new( + partition, + input.schema(), + self.expr.clone(), + *fetch, + context.session_config().batch_size(), + context.runtime_env(), + &self.metrics_set, + partition, + )?; + Ok(Box::pin(RecordBatchStreamAdapter::new( + self.schema(), + futures::stream::once(async move { + while let Some(batch) = input.next().await { + let batch = batch?; + topk.insert_batch(batch)?; + } + topk.emit() + }) + .try_flatten(), + ))) + } + (false, None) => { + let mut sorter = ExternalSorter::new( + partition, + input.schema(), + self.expr.clone(), + context.session_config().batch_size(), + self.fetch, + execution_options.sort_spill_reservation_bytes, + execution_options.sort_in_place_threshold_bytes, + &self.metrics_set, + context.runtime_env(), + ); + Ok(Box::pin(RecordBatchStreamAdapter::new( + self.schema(), + futures::stream::once(async move { + while let Some(batch) = input.next().await { + let batch = batch?; + sorter.insert_batch(batch).await?; + } + sorter.sort() + }) + .try_flatten(), + ))) + } } } @@ -933,14 +965,7 @@ impl ExecutionPlan for SortExec { } fn with_fetch(&self, limit: Option) -> Option> { - Some(Arc::new(SortExec { - input: Arc::clone(&self.input), - expr: self.expr.clone(), - metrics_set: self.metrics_set.clone(), - preserve_partitioning: self.preserve_partitioning, - fetch: limit, - cache: self.cache.clone(), - })) + Some(Arc::new(SortExec::with_fetch(self, limit))) } fn fetch(&self) -> Option { @@ -951,6 +976,8 @@ impl ExecutionPlan for SortExec { #[cfg(test)] mod tests { use std::collections::HashMap; + use std::pin::Pin; + use std::task::{Context, Poll}; use super::*; use crate::coalesce_partitions::CoalescePartitionsExec; @@ -965,12 +992,124 @@ mod tests { use arrow::compute::SortOptions; use arrow::datatypes::*; use datafusion_common::cast::as_primitive_array; + use datafusion_common::{assert_batches_eq, Result, ScalarValue}; use datafusion_execution::config::SessionConfig; use datafusion_execution::runtime_env::RuntimeEnvBuilder; + use datafusion_execution::RecordBatchStream; + use datafusion_physical_expr::expressions::{Column, Literal}; + use datafusion_physical_expr::EquivalenceProperties; + + use futures::{FutureExt, Stream}; + + #[derive(Debug, Clone)] + pub struct SortedUnboundedExec { + schema: Schema, + batch_size: u64, + cache: PlanProperties, + } + + impl DisplayAs for SortedUnboundedExec { + fn fmt_as(&self, t: DisplayFormatType, f: &mut Formatter) -> std::fmt::Result { + match t { + DisplayFormatType::Default | DisplayFormatType::Verbose => { + write!(f, "UnboundableExec",).unwrap() + } + } + Ok(()) + } + } + + impl SortedUnboundedExec { + fn compute_properties(schema: SchemaRef) -> PlanProperties { + let mut eq_properties = EquivalenceProperties::new(schema); + eq_properties.add_new_orderings(vec![vec![PhysicalSortExpr::new( + Arc::new(Column::new("c1", 0)), + SortOptions::default(), + )]]); + let mode = ExecutionMode::Unbounded; + PlanProperties::new(eq_properties, Partitioning::UnknownPartitioning(1), mode) + } + } + + impl ExecutionPlan for SortedUnboundedExec { + fn name(&self) -> &'static str { + Self::static_name() + } + + fn as_any(&self) -> &dyn Any { + self + } + + fn properties(&self) -> &PlanProperties { + &self.cache + } + + fn children(&self) -> Vec<&Arc> { + vec![] + } + + fn with_new_children( + self: Arc, + _: Vec>, + ) -> Result> { + Ok(self) + } + + fn execute( + &self, + _partition: usize, + _context: Arc, + ) -> Result { + Ok(Box::pin(SortedUnboundedStream { + schema: Arc::new(self.schema.clone()), + batch_size: self.batch_size, + offset: 0, + })) + } + } + + #[derive(Debug)] + pub struct SortedUnboundedStream { + schema: SchemaRef, + batch_size: u64, + offset: u64, + } - use datafusion_common::ScalarValue; - use datafusion_physical_expr::expressions::Literal; - use futures::FutureExt; + impl Stream for SortedUnboundedStream { + type Item = Result; + + fn poll_next( + mut self: Pin<&mut Self>, + _cx: &mut Context<'_>, + ) -> Poll> { + let batch = SortedUnboundedStream::create_record_batch( + Arc::clone(&self.schema), + self.offset, + self.batch_size, + ); + self.offset += self.batch_size; + Poll::Ready(Some(Ok(batch))) + } + } + + impl RecordBatchStream for SortedUnboundedStream { + fn schema(&self) -> SchemaRef { + Arc::clone(&self.schema) + } + } + + impl SortedUnboundedStream { + fn create_record_batch( + schema: SchemaRef, + offset: u64, + batch_size: u64, + ) -> RecordBatch { + let values = (0..batch_size).map(|i| offset + i).collect::>(); + let array = UInt64Array::from(values); + let array_ref: ArrayRef = Arc::new(array); + RecordBatch::try_new(schema, vec![array_ref]).unwrap() + } + } #[tokio::test] async fn test_in_mem_sort() -> Result<()> { @@ -1414,4 +1553,42 @@ mod tests { let result = sort_batch(&batch, &expressions, None).unwrap(); assert_eq!(result.num_rows(), 1); } + + #[tokio::test] + async fn topk_unbounded_source() -> Result<()> { + let task_ctx = Arc::new(TaskContext::default()); + let schema = Schema::new(vec![Field::new("c1", DataType::UInt64, false)]); + let source = SortedUnboundedExec { + schema: schema.clone(), + batch_size: 2, + cache: SortedUnboundedExec::compute_properties(Arc::new(schema.clone())), + }; + let mut plan = SortExec::new( + vec![PhysicalSortExpr::new( + Arc::new(Column::new("c1", 0)), + SortOptions::default(), + )], + Arc::new(source), + ); + plan = plan.with_fetch(Some(9)); + + let batches = collect(Arc::new(plan), task_ctx).await?; + #[rustfmt::skip] + let expected = [ + "+----+", + "| c1 |", + "+----+", + "| 0 |", + "| 1 |", + "| 2 |", + "| 3 |", + "| 4 |", + "| 5 |", + "| 6 |", + "| 7 |", + "| 8 |", + "+----+",]; + assert_batches_eq!(expected, &batches); + Ok(()) + } } diff --git a/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs b/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs index c1bcd83a6fd2..001e134581c0 100644 --- a/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs +++ b/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs @@ -551,7 +551,7 @@ impl PartitionSearcher for LinearSearch { window_expr: &[Arc], ) -> Result> { let partition_bys = - self.evaluate_partition_by_column_values(record_batch, window_expr)?; + evaluate_partition_by_column_values(record_batch, window_expr)?; // NOTE: In Linear or PartiallySorted modes, we are sure that // `partition_bys` are not empty. // Calculate indices for each partition and construct a new record @@ -618,25 +618,6 @@ impl LinearSearch { } } - /// Calculates partition by expression results for each window expression - /// on `record_batch`. - fn evaluate_partition_by_column_values( - &self, - record_batch: &RecordBatch, - window_expr: &[Arc], - ) -> Result> { - window_expr[0] - .partition_by() - .iter() - .map(|item| match item.evaluate(record_batch)? { - ColumnarValue::Array(array) => Ok(array), - ColumnarValue::Scalar(scalar) => { - scalar.to_array_of_size(record_batch.num_rows()) - } - }) - .collect() - } - /// Calculate indices of each partition (according to PARTITION BY expression) /// `columns` contain partition by expression results. fn get_per_partition_indices( @@ -683,7 +664,7 @@ impl LinearSearch { window_expr: &[Arc], ) -> Result)>> { let partition_by_columns = - self.evaluate_partition_by_column_values(input_buffer, window_expr)?; + evaluate_partition_by_column_values(input_buffer, window_expr)?; // Reset the row_map state: self.row_map_out.clear(); let mut partition_indices: Vec<(PartitionKey, Vec)> = vec![]; @@ -852,6 +833,24 @@ impl SortedSearch { } } +/// Calculates partition by expression results for each window expression +/// on `record_batch`. +fn evaluate_partition_by_column_values( + record_batch: &RecordBatch, + window_expr: &[Arc], +) -> Result> { + window_expr[0] + .partition_by() + .iter() + .map(|item| match item.evaluate(record_batch)? { + ColumnarValue::Array(array) => Ok(array), + ColumnarValue::Scalar(scalar) => { + scalar.to_array_of_size(record_batch.num_rows()) + } + }) + .collect() +} + /// Stream for the bounded window aggregation plan. pub struct BoundedWindowAggStream { schema: SchemaRef, From 85adb6c4e6c0b6009f9866118c318b078263e118 Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Thu, 29 Aug 2024 17:38:28 +0200 Subject: [PATCH 045/154] Remove Sort expression (`Expr::Sort`) (#12177) * Take Sort (SortExpr) in file options Part of effort to remove `Expr::Sort`. * Return Sort from Expr.Sort Part of effort to remove `Expr::Sort`. * Accept Sort (SortExpr) in `LogicalPlanBuilder.sort` Take `expr::Sort` in `LogicalPlanBuilder.sort`. Accept any `Expr` in new function, `LogicalPlanBuilder.sort_by` which apply default sort ordering. Part of effort to remove `Expr::Sort`. * Operate on `Sort` in to_substrait_sort_field / from_substrait_sorts Part of effort to remove `Expr::Sort`. * Take Sort (SortExpr) in tests' TopKPlanNode Part of effort to remove `Expr::Sort`. * Remove Sort expression (`Expr::Sort`) Remove sort as an expression, i.e. remove `Expr::Sort` from `Expr` enum. Use `expr::Sort` directly when sorting. The sort expression was used in context of ordering (sort, topk, create table, file sorting). Those places require their sort expression to be of type Sort anyway and no other expression was allowed, so this change improves static typing. Sort as an expression was illegal in other contexts. * use assert_eq just like in LogicalPlan.with_new_exprs * avoid clone in replace_sort_expressions * reduce cloning in EliminateDuplicatedExpr * restore SortExprWrapper this commit is longer than advised in the review comment, but after squashing the diff will be smaller * shorthand SortExprWrapper struct definition --- .../examples/file_stream_provider.rs | 4 +- datafusion/core/src/dataframe/mod.rs | 15 +- .../src/datasource/file_format/options.rs | 14 +- .../core/src/datasource/listing/helpers.rs | 3 +- .../core/src/datasource/listing/table.rs | 14 +- datafusion/core/src/datasource/memory.rs | 5 +- datafusion/core/src/datasource/mod.rs | 41 ++-- .../physical_plan/file_scan_config.rs | 2 +- datafusion/core/src/datasource/stream.rs | 6 +- datafusion/core/src/physical_planner.rs | 32 ++- datafusion/core/src/test_util/mod.rs | 4 +- datafusion/core/tests/dataframe/mod.rs | 20 +- datafusion/core/tests/expr_api/mod.rs | 17 +- datafusion/core/tests/fifo/mod.rs | 4 +- .../core/tests/fuzz_cases/limit_fuzz.rs | 2 +- .../tests/user_defined/user_defined_plan.rs | 9 +- datafusion/expr/src/expr.rs | 113 +++++------ datafusion/expr/src/expr_fn.rs | 28 +-- datafusion/expr/src/expr_rewriter/mod.rs | 34 ++-- datafusion/expr/src/expr_rewriter/order_by.rs | 38 ++-- datafusion/expr/src/expr_schema.rs | 15 +- datafusion/expr/src/logical_plan/builder.rs | 38 ++-- datafusion/expr/src/logical_plan/ddl.rs | 7 +- datafusion/expr/src/logical_plan/plan.rs | 41 ++-- datafusion/expr/src/logical_plan/tree_node.rs | 18 +- datafusion/expr/src/tree_node.rs | 53 ++++- datafusion/expr/src/utils.rs | 192 +++++++----------- datafusion/expr/src/window_frame.rs | 4 +- .../functions-aggregate/src/first_last.rs | 4 +- .../src/analyzer/count_wildcard_rule.rs | 2 +- .../optimizer/src/analyzer/type_coercion.rs | 7 +- .../optimizer/src/common_subexpr_eliminate.rs | 15 +- .../src/eliminate_duplicated_expr.rs | 30 +-- datafusion/optimizer/src/eliminate_limit.rs | 12 +- datafusion/optimizer/src/push_down_filter.rs | 3 +- datafusion/optimizer/src/push_down_limit.rs | 8 +- .../simplify_expressions/expr_simplifier.rs | 1 - .../src/single_distinct_to_groupby.rs | 4 +- datafusion/proto/proto/datafusion.proto | 17 +- datafusion/proto/src/generated/pbjson.rs | 105 ++++++++-- datafusion/proto/src/generated/prost.rs | 26 ++- .../proto/src/logical_plan/from_proto.rs | 49 +++-- datafusion/proto/src/logical_plan/mod.rs | 40 ++-- datafusion/proto/src/logical_plan/to_proto.rs | 48 +++-- .../tests/cases/roundtrip_logical_plan.rs | 10 +- datafusion/sql/src/expr/function.rs | 25 +-- datafusion/sql/src/expr/order_by.rs | 8 +- datafusion/sql/src/query.rs | 3 +- datafusion/sql/src/select.rs | 4 +- datafusion/sql/src/statement.rs | 13 +- datafusion/sql/src/unparser/expr.rs | 120 +++-------- datafusion/sql/src/unparser/mod.rs | 2 - datafusion/sql/src/unparser/plan.rs | 17 +- datafusion/sql/src/unparser/rewrite.rs | 37 ++-- .../substrait/src/logical_plan/consumer.rs | 12 +- .../substrait/src/logical_plan/producer.rs | 77 ++++--- .../using-the-dataframe-api.md | 4 +- 57 files changed, 704 insertions(+), 772 deletions(-) diff --git a/datafusion-examples/examples/file_stream_provider.rs b/datafusion-examples/examples/file_stream_provider.rs index 4db7e0200f53..e4fd937fd373 100644 --- a/datafusion-examples/examples/file_stream_provider.rs +++ b/datafusion-examples/examples/file_stream_provider.rs @@ -39,7 +39,7 @@ mod non_windows { use datafusion::datasource::TableProvider; use datafusion::prelude::{SessionConfig, SessionContext}; use datafusion_common::{exec_err, Result}; - use datafusion_expr::Expr; + use datafusion_expr::SortExpr; // Number of lines written to FIFO const TEST_BATCH_SIZE: usize = 5; @@ -49,7 +49,7 @@ mod non_windows { fn fifo_table( schema: SchemaRef, path: impl Into, - sort: Vec>, + sort: Vec>, ) -> Arc { let source = FileStreamProvider::new_file(schema, path.into()) .with_batch_size(TEST_BATCH_SIZE) diff --git a/datafusion/core/src/dataframe/mod.rs b/datafusion/core/src/dataframe/mod.rs index c516c7985d54..5dbeb535a546 100644 --- a/datafusion/core/src/dataframe/mod.rs +++ b/datafusion/core/src/dataframe/mod.rs @@ -52,7 +52,7 @@ use datafusion_common::config::{CsvOptions, JsonOptions}; use datafusion_common::{ plan_err, Column, DFSchema, DataFusionError, ParamValues, SchemaError, UnnestOptions, }; -use datafusion_expr::{case, is_null, lit}; +use datafusion_expr::{case, is_null, lit, SortExpr}; use datafusion_expr::{ utils::COUNT_STAR_EXPANSION, TableProviderFilterPushDown, UNNAMED_TABLE, }; @@ -577,7 +577,7 @@ impl DataFrame { self, on_expr: Vec, select_expr: Vec, - sort_expr: Option>, + sort_expr: Option>, ) -> Result { let plan = LogicalPlanBuilder::from(self.plan) .distinct_on(on_expr, select_expr, sort_expr)? @@ -776,6 +776,15 @@ impl DataFrame { }) } + /// Apply a sort by provided expressions with default direction + pub fn sort_by(self, expr: Vec) -> Result { + self.sort( + expr.into_iter() + .map(|e| e.sort(true, false)) + .collect::>(), + ) + } + /// Sort the DataFrame by the specified sorting expressions. /// /// Note that any expression can be turned into @@ -797,7 +806,7 @@ impl DataFrame { /// # Ok(()) /// # } /// ``` - pub fn sort(self, expr: Vec) -> Result { + pub fn sort(self, expr: Vec) -> Result { let plan = LogicalPlanBuilder::from(self.plan).sort(expr)?.build()?; Ok(DataFrame { session_state: self.session_state, diff --git a/datafusion/core/src/datasource/file_format/options.rs b/datafusion/core/src/datasource/file_format/options.rs index 552977baba17..db90262edbf8 100644 --- a/datafusion/core/src/datasource/file_format/options.rs +++ b/datafusion/core/src/datasource/file_format/options.rs @@ -31,7 +31,6 @@ use crate::datasource::{ }; use crate::error::Result; use crate::execution::context::{SessionConfig, SessionState}; -use crate::logical_expr::Expr; use arrow::datatypes::{DataType, Schema, SchemaRef}; use datafusion_common::config::TableOptions; @@ -41,6 +40,7 @@ use datafusion_common::{ }; use async_trait::async_trait; +use datafusion_expr::SortExpr; /// Options that control the reading of CSV files. /// @@ -84,7 +84,7 @@ pub struct CsvReadOptions<'a> { /// File compression type pub file_compression_type: FileCompressionType, /// Indicates how the file is sorted - pub file_sort_order: Vec>, + pub file_sort_order: Vec>, } impl<'a> Default for CsvReadOptions<'a> { @@ -199,7 +199,7 @@ impl<'a> CsvReadOptions<'a> { } /// Configure if file has known sort order - pub fn file_sort_order(mut self, file_sort_order: Vec>) -> Self { + pub fn file_sort_order(mut self, file_sort_order: Vec>) -> Self { self.file_sort_order = file_sort_order; self } @@ -231,7 +231,7 @@ pub struct ParquetReadOptions<'a> { /// based on data in file. pub schema: Option<&'a Schema>, /// Indicates how the file is sorted - pub file_sort_order: Vec>, + pub file_sort_order: Vec>, } impl<'a> Default for ParquetReadOptions<'a> { @@ -278,7 +278,7 @@ impl<'a> ParquetReadOptions<'a> { } /// Configure if file has known sort order - pub fn file_sort_order(mut self, file_sort_order: Vec>) -> Self { + pub fn file_sort_order(mut self, file_sort_order: Vec>) -> Self { self.file_sort_order = file_sort_order; self } @@ -397,7 +397,7 @@ pub struct NdJsonReadOptions<'a> { /// Flag indicating whether this file may be unbounded (as in a FIFO file). pub infinite: bool, /// Indicates how the file is sorted - pub file_sort_order: Vec>, + pub file_sort_order: Vec>, } impl<'a> Default for NdJsonReadOptions<'a> { @@ -452,7 +452,7 @@ impl<'a> NdJsonReadOptions<'a> { } /// Configure if file has known sort order - pub fn file_sort_order(mut self, file_sort_order: Vec>) -> Self { + pub fn file_sort_order(mut self, file_sort_order: Vec>) -> Self { self.file_sort_order = file_sort_order; self } diff --git a/datafusion/core/src/datasource/listing/helpers.rs b/datafusion/core/src/datasource/listing/helpers.rs index f6e938b72dab..dbeaf5dfcc36 100644 --- a/datafusion/core/src/datasource/listing/helpers.rs +++ b/datafusion/core/src/datasource/listing/helpers.rs @@ -102,11 +102,10 @@ pub fn expr_applicable_for_cols(col_names: &[String], expr: &Expr) -> bool { } // TODO other expressions are not handled yet: - // - AGGREGATE, WINDOW and SORT should not end up in filter conditions, except maybe in some edge cases + // - AGGREGATE and WINDOW should not end up in filter conditions, except maybe in some edge cases // - Can `Wildcard` be considered as a `Literal`? // - ScalarVariable could be `applicable`, but that would require access to the context Expr::AggregateFunction { .. } - | Expr::Sort { .. } | Expr::WindowFunction { .. } | Expr::Wildcard { .. } | Expr::Unnest { .. } diff --git a/datafusion/core/src/datasource/listing/table.rs b/datafusion/core/src/datasource/listing/table.rs index a0345a38e40c..1f5fa738b253 100644 --- a/datafusion/core/src/datasource/listing/table.rs +++ b/datafusion/core/src/datasource/listing/table.rs @@ -33,8 +33,8 @@ use crate::datasource::{ use crate::execution::context::SessionState; use datafusion_catalog::TableProvider; use datafusion_common::{DataFusionError, Result}; -use datafusion_expr::TableType; use datafusion_expr::{utils::conjunction, Expr, TableProviderFilterPushDown}; +use datafusion_expr::{SortExpr, TableType}; use datafusion_physical_plan::{empty::EmptyExec, ExecutionPlan, Statistics}; use arrow::datatypes::{DataType, Field, SchemaBuilder, SchemaRef}; @@ -222,7 +222,7 @@ pub struct ListingOptions { /// ordering (encapsulated by a `Vec`). If there aren't /// multiple equivalent orderings, the outer `Vec` will have a /// single element. - pub file_sort_order: Vec>, + pub file_sort_order: Vec>, } impl ListingOptions { @@ -385,7 +385,7 @@ impl ListingOptions { /// /// assert_eq!(listing_options.file_sort_order, file_sort_order); /// ``` - pub fn with_file_sort_order(mut self, file_sort_order: Vec>) -> Self { + pub fn with_file_sort_order(mut self, file_sort_order: Vec>) -> Self { self.file_sort_order = file_sort_order; self } @@ -909,8 +909,7 @@ impl TableProvider for ListingTable { keep_partition_by_columns, }; - let unsorted: Vec> = vec![]; - let order_requirements = if self.options().file_sort_order != unsorted { + let order_requirements = if !self.options().file_sort_order.is_empty() { // Multiple sort orders in outer vec are equivalent, so we pass only the first one let ordering = self .try_create_output_ordering()? @@ -1160,11 +1159,6 @@ mod tests { // (file_sort_order, expected_result) let cases = vec![ (vec![], Ok(vec![])), - // not a sort expr - ( - vec![vec![col("string_col")]], - Err("Expected Expr::Sort in output_ordering, but got string_col"), - ), // sort expr, but non column ( vec![vec![ diff --git a/datafusion/core/src/datasource/memory.rs b/datafusion/core/src/datasource/memory.rs index 44e01e71648a..cef7f210e118 100644 --- a/datafusion/core/src/datasource/memory.rs +++ b/datafusion/core/src/datasource/memory.rs @@ -43,6 +43,7 @@ use datafusion_physical_plan::metrics::MetricsSet; use async_trait::async_trait; use datafusion_catalog::Session; +use datafusion_expr::SortExpr; use futures::StreamExt; use log::debug; use parking_lot::Mutex; @@ -64,7 +65,7 @@ pub struct MemTable { column_defaults: HashMap, /// Optional pre-known sort order(s). Must be `SortExpr`s. /// inserting data into this table removes the order - pub sort_order: Arc>>>, + pub sort_order: Arc>>>, } impl MemTable { @@ -118,7 +119,7 @@ impl MemTable { /// /// Note that multiple sort orders are supported, if some are known to be /// equivalent, - pub fn with_sort_order(self, mut sort_order: Vec>) -> Self { + pub fn with_sort_order(self, mut sort_order: Vec>) -> Self { std::mem::swap(self.sort_order.lock().as_mut(), &mut sort_order); self } diff --git a/datafusion/core/src/datasource/mod.rs b/datafusion/core/src/datasource/mod.rs index 1c9924735735..55e88e572be1 100644 --- a/datafusion/core/src/datasource/mod.rs +++ b/datafusion/core/src/datasource/mod.rs @@ -50,38 +50,39 @@ pub use statistics::get_statistics_with_limit; use arrow_schema::{Schema, SortOptions}; use datafusion_common::{plan_err, Result}; -use datafusion_expr::Expr; +use datafusion_expr::{Expr, SortExpr}; use datafusion_physical_expr::{expressions, LexOrdering, PhysicalSortExpr}; fn create_ordering( schema: &Schema, - sort_order: &[Vec], + sort_order: &[Vec], ) -> Result> { let mut all_sort_orders = vec![]; for exprs in sort_order { // Construct PhysicalSortExpr objects from Expr objects: let mut sort_exprs = vec![]; - for expr in exprs { - match expr { - Expr::Sort(sort) => match sort.expr.as_ref() { - Expr::Column(col) => match expressions::col(&col.name, schema) { - Ok(expr) => { - sort_exprs.push(PhysicalSortExpr { - expr, - options: SortOptions { - descending: !sort.asc, - nulls_first: sort.nulls_first, - }, - }); - } - // Cannot find expression in the projected_schema, stop iterating - // since rest of the orderings are violated - Err(_) => break, + for sort in exprs { + match sort.expr.as_ref() { + Expr::Column(col) => match expressions::col(&col.name, schema) { + Ok(expr) => { + sort_exprs.push(PhysicalSortExpr { + expr, + options: SortOptions { + descending: !sort.asc, + nulls_first: sort.nulls_first, + }, + }); } - expr => return plan_err!("Expected single column references in output_ordering, got {expr}"), + // Cannot find expression in the projected_schema, stop iterating + // since rest of the orderings are violated + Err(_) => break, + }, + expr => { + return plan_err!( + "Expected single column references in output_ordering, got {expr}" + ) } - expr => return plan_err!("Expected Expr::Sort in output_ordering, but got {expr}"), } } if !sort_exprs.is_empty() { diff --git a/datafusion/core/src/datasource/physical_plan/file_scan_config.rs b/datafusion/core/src/datasource/physical_plan/file_scan_config.rs index bfa5488e5b5e..3ea467539adc 100644 --- a/datafusion/core/src/datasource/physical_plan/file_scan_config.rs +++ b/datafusion/core/src/datasource/physical_plan/file_scan_config.rs @@ -979,7 +979,7 @@ mod tests { name: &'static str, file_schema: Schema, files: Vec, - sort: Vec, + sort: Vec, expected_result: Result>, &'static str>, } diff --git a/datafusion/core/src/datasource/stream.rs b/datafusion/core/src/datasource/stream.rs index b53fe8663178..ef6d195cdaff 100644 --- a/datafusion/core/src/datasource/stream.rs +++ b/datafusion/core/src/datasource/stream.rs @@ -33,7 +33,7 @@ use arrow_schema::SchemaRef; use datafusion_common::{config_err, plan_err, Constraints, DataFusionError, Result}; use datafusion_common_runtime::SpawnedTask; use datafusion_execution::{SendableRecordBatchStream, TaskContext}; -use datafusion_expr::{CreateExternalTable, Expr, TableType}; +use datafusion_expr::{CreateExternalTable, Expr, SortExpr, TableType}; use datafusion_physical_plan::insert::{DataSink, DataSinkExec}; use datafusion_physical_plan::metrics::MetricsSet; use datafusion_physical_plan::stream::RecordBatchReceiverStreamBuilder; @@ -248,7 +248,7 @@ impl StreamProvider for FileStreamProvider { #[derive(Debug)] pub struct StreamConfig { source: Arc, - order: Vec>, + order: Vec>, constraints: Constraints, } @@ -263,7 +263,7 @@ impl StreamConfig { } /// Specify a sort order for the stream - pub fn with_order(mut self, order: Vec>) -> Self { + pub fn with_order(mut self, order: Vec>) -> Self { self.order = order; self } diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index fe8d79846630..82405dd98e30 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -73,13 +73,13 @@ use datafusion_common::{ }; use datafusion_expr::dml::CopyTo; use datafusion_expr::expr::{ - self, physical_name, AggregateFunction, Alias, GroupingSet, WindowFunction, + physical_name, AggregateFunction, Alias, GroupingSet, WindowFunction, }; use datafusion_expr::expr_rewriter::unnormalize_cols; use datafusion_expr::logical_plan::builder::wrap_projection_for_join_if_necessary; use datafusion_expr::{ - DescribeTable, DmlStatement, Extension, Filter, RecursiveQuery, StringifiedPlan, - WindowFrame, WindowFrameBound, WriteOp, + DescribeTable, DmlStatement, Extension, Filter, RecursiveQuery, SortExpr, + StringifiedPlan, WindowFrame, WindowFrameBound, WriteOp, }; use datafusion_physical_expr::aggregate::{AggregateExprBuilder, AggregateFunctionExpr}; use datafusion_physical_expr::expressions::Literal; @@ -1641,31 +1641,27 @@ pub fn create_aggregate_expr_and_maybe_filter( /// Create a physical sort expression from a logical expression pub fn create_physical_sort_expr( - e: &Expr, + e: &SortExpr, input_dfschema: &DFSchema, execution_props: &ExecutionProps, ) -> Result { - if let Expr::Sort(expr::Sort { + let SortExpr { expr, asc, nulls_first, - }) = e - { - Ok(PhysicalSortExpr { - expr: create_physical_expr(expr, input_dfschema, execution_props)?, - options: SortOptions { - descending: !asc, - nulls_first: *nulls_first, - }, - }) - } else { - internal_err!("Expects a sort expression") - } + } = e; + Ok(PhysicalSortExpr { + expr: create_physical_expr(expr, input_dfschema, execution_props)?, + options: SortOptions { + descending: !asc, + nulls_first: *nulls_first, + }, + }) } /// Create vector of physical sort expression from a vector of logical expression pub fn create_physical_sort_exprs( - exprs: &[Expr], + exprs: &[SortExpr], input_dfschema: &DFSchema, execution_props: &ExecutionProps, ) -> Result { diff --git a/datafusion/core/src/test_util/mod.rs b/datafusion/core/src/test_util/mod.rs index faa9378535fd..dd8b697666ee 100644 --- a/datafusion/core/src/test_util/mod.rs +++ b/datafusion/core/src/test_util/mod.rs @@ -46,7 +46,7 @@ use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; use arrow::record_batch::RecordBatch; use datafusion_common::TableReference; use datafusion_expr::utils::COUNT_STAR_EXPANSION; -use datafusion_expr::{CreateExternalTable, Expr, TableType}; +use datafusion_expr::{CreateExternalTable, Expr, SortExpr, TableType}; use datafusion_functions_aggregate::count::count_udaf; use datafusion_physical_expr::{expressions, EquivalenceProperties, PhysicalExpr}; @@ -360,7 +360,7 @@ pub fn register_unbounded_file_with_ordering( schema: SchemaRef, file_path: &Path, table_name: &str, - file_sort_order: Vec>, + file_sort_order: Vec>, ) -> Result<()> { let source = FileStreamProvider::new_file(schema, file_path.into()); let config = StreamConfig::new(Arc::new(source)).with_order(file_sort_order); diff --git a/datafusion/core/tests/dataframe/mod.rs b/datafusion/core/tests/dataframe/mod.rs index 86cacbaa06d8..c5b9db7588e9 100644 --- a/datafusion/core/tests/dataframe/mod.rs +++ b/datafusion/core/tests/dataframe/mod.rs @@ -184,7 +184,7 @@ async fn test_count_wildcard_on_window() -> Result<()> { WindowFunctionDefinition::AggregateUDF(count_udaf()), vec![wildcard()], )) - .order_by(vec![Expr::Sort(Sort::new(Box::new(col("a")), false, true))]) + .order_by(vec![Sort::new(Box::new(col("a")), false, true)]) .window_frame(WindowFrame::new_bounds( WindowFrameUnits::Range, WindowFrameBound::Preceding(ScalarValue::UInt32(Some(6))), @@ -352,7 +352,7 @@ async fn sort_on_unprojected_columns() -> Result<()> { .unwrap() .select(vec![col("a")]) .unwrap() - .sort(vec![Expr::Sort(Sort::new(Box::new(col("b")), false, true))]) + .sort(vec![Sort::new(Box::new(col("b")), false, true)]) .unwrap(); let results = df.collect().await.unwrap(); @@ -396,7 +396,7 @@ async fn sort_on_distinct_columns() -> Result<()> { .unwrap() .distinct() .unwrap() - .sort(vec![Expr::Sort(Sort::new(Box::new(col("a")), false, true))]) + .sort(vec![Sort::new(Box::new(col("a")), false, true)]) .unwrap(); let results = df.collect().await.unwrap(); @@ -435,7 +435,7 @@ async fn sort_on_distinct_unprojected_columns() -> Result<()> { .await? .select(vec![col("a")])? .distinct()? - .sort(vec![Expr::Sort(Sort::new(Box::new(col("b")), false, true))]) + .sort(vec![Sort::new(Box::new(col("b")), false, true)]) .unwrap_err(); assert_eq!(err.strip_backtrace(), "Error during planning: For SELECT DISTINCT, ORDER BY expressions b must appear in select list"); Ok(()) @@ -599,8 +599,8 @@ async fn test_grouping_sets() -> Result<()> { .await? .aggregate(vec![grouping_set_expr], vec![count(col("a"))])? .sort(vec![ - Expr::Sort(Sort::new(Box::new(col("a")), false, true)), - Expr::Sort(Sort::new(Box::new(col("b")), false, true)), + Sort::new(Box::new(col("a")), false, true), + Sort::new(Box::new(col("b")), false, true), ])?; let results = df.collect().await?; @@ -640,8 +640,8 @@ async fn test_grouping_sets_count() -> Result<()> { .await? .aggregate(vec![grouping_set_expr], vec![count(lit(1))])? .sort(vec![ - Expr::Sort(Sort::new(Box::new(col("c1")), false, true)), - Expr::Sort(Sort::new(Box::new(col("c2")), false, true)), + Sort::new(Box::new(col("c1")), false, true), + Sort::new(Box::new(col("c2")), false, true), ])?; let results = df.collect().await?; @@ -687,8 +687,8 @@ async fn test_grouping_set_array_agg_with_overflow() -> Result<()> { ], )? .sort(vec![ - Expr::Sort(Sort::new(Box::new(col("c1")), false, true)), - Expr::Sort(Sort::new(Box::new(col("c2")), false, true)), + Sort::new(Box::new(col("c1")), false, true), + Sort::new(Box::new(col("c2")), false, true), ])?; let results = df.collect().await?; diff --git a/datafusion/core/tests/expr_api/mod.rs b/datafusion/core/tests/expr_api/mod.rs index 051d65652633..cbd892672152 100644 --- a/datafusion/core/tests/expr_api/mod.rs +++ b/datafusion/core/tests/expr_api/mod.rs @@ -20,7 +20,7 @@ use arrow_array::builder::{ListBuilder, StringBuilder}; use arrow_array::{ArrayRef, Int64Array, RecordBatch, StringArray, StructArray}; use arrow_schema::{DataType, Field}; use datafusion::prelude::*; -use datafusion_common::{assert_contains, DFSchema, ScalarValue}; +use datafusion_common::{DFSchema, ScalarValue}; use datafusion_expr::ExprFunctionExt; use datafusion_functions::core::expr_ext::FieldAccessor; use datafusion_functions_aggregate::first_last::first_value_udaf; @@ -167,21 +167,6 @@ fn test_list_range() { ); } -#[tokio::test] -async fn test_aggregate_error() { - let err = first_value_udaf() - .call(vec![col("props")]) - // not a sort column - .order_by(vec![col("id")]) - .build() - .unwrap_err() - .to_string(); - assert_contains!( - err, - "Error during planning: ORDER BY expressions must be Expr::Sort" - ); -} - #[tokio::test] async fn test_aggregate_ext_order_by() { let agg = first_value_udaf().call(vec![col("props")]); diff --git a/datafusion/core/tests/fifo/mod.rs b/datafusion/core/tests/fifo/mod.rs index 6efbb9b029de..cb587e3510c2 100644 --- a/datafusion/core/tests/fifo/mod.rs +++ b/datafusion/core/tests/fifo/mod.rs @@ -38,7 +38,7 @@ mod unix_test { }; use datafusion_common::instant::Instant; use datafusion_common::{exec_err, Result}; - use datafusion_expr::Expr; + use datafusion_expr::SortExpr; use futures::StreamExt; use nix::sys::stat; @@ -51,7 +51,7 @@ mod unix_test { fn fifo_table( schema: SchemaRef, path: impl Into, - sort: Vec>, + sort: Vec>, ) -> Arc { let source = FileStreamProvider::new_file(schema, path.into()) .with_batch_size(TEST_BATCH_SIZE) diff --git a/datafusion/core/tests/fuzz_cases/limit_fuzz.rs b/datafusion/core/tests/fuzz_cases/limit_fuzz.rs index 9889ce2ae562..95d97709f319 100644 --- a/datafusion/core/tests/fuzz_cases/limit_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/limit_fuzz.rs @@ -226,7 +226,7 @@ impl SortedData { } /// Return the sort expression to use for this data, depending on the type - fn sort_expr(&self) -> Vec { + fn sort_expr(&self) -> Vec { match self { Self::I32 { .. } | Self::F64 { .. } | Self::Str { .. } => { vec![datafusion_expr::col("x").sort(true, true)] diff --git a/datafusion/core/tests/user_defined/user_defined_plan.rs b/datafusion/core/tests/user_defined/user_defined_plan.rs index 62ba113da0d3..da27cf8869d1 100644 --- a/datafusion/core/tests/user_defined/user_defined_plan.rs +++ b/datafusion/core/tests/user_defined/user_defined_plan.rs @@ -97,7 +97,8 @@ use datafusion::{ use datafusion_common::config::ConfigOptions; use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; use datafusion_common::ScalarValue; -use datafusion_expr::Projection; +use datafusion_expr::tree_node::replace_sort_expression; +use datafusion_expr::{Projection, SortExpr}; use datafusion_optimizer::optimizer::ApplyOrder; use datafusion_optimizer::AnalyzerRule; @@ -392,7 +393,7 @@ struct TopKPlanNode { input: LogicalPlan, /// The sort expression (this example only supports a single sort /// expr) - expr: Expr, + expr: SortExpr, } impl Debug for TopKPlanNode { @@ -418,7 +419,7 @@ impl UserDefinedLogicalNodeCore for TopKPlanNode { } fn expressions(&self) -> Vec { - vec![self.expr.clone()] + vec![self.expr.expr.as_ref().clone()] } /// For example: `TopK: k=10` @@ -436,7 +437,7 @@ impl UserDefinedLogicalNodeCore for TopKPlanNode { Ok(Self { k: self.k, input: inputs.swap_remove(0), - expr: exprs.swap_remove(0), + expr: replace_sort_expression(self.expr.clone(), exprs.swap_remove(0)), }) } } diff --git a/datafusion/expr/src/expr.rs b/datafusion/expr/src/expr.rs index 85ba80396c8e..b81c02ccd0b7 100644 --- a/datafusion/expr/src/expr.rs +++ b/datafusion/expr/src/expr.rs @@ -289,10 +289,6 @@ pub enum Expr { /// Casts the expression to a given type and will return a null value if the expression cannot be cast. /// This expression is guaranteed to have a fixed type. TryCast(TryCast), - /// A sort expression, that can be used to sort values. - /// - /// See [Expr::sort] for more details - Sort(Sort), /// Represents the call of a scalar function with a set of arguments. ScalarFunction(ScalarFunction), /// Calls an aggregate function with arguments, and optional @@ -633,6 +629,23 @@ impl Sort { } } +impl Display for Sort { + fn fmt(&self, f: &mut Formatter<'_>) -> fmt::Result { + write!(f, "{}", self.expr)?; + if self.asc { + write!(f, " ASC")?; + } else { + write!(f, " DESC")?; + } + if self.nulls_first { + write!(f, " NULLS FIRST")?; + } else { + write!(f, " NULLS LAST")?; + } + Ok(()) + } +} + /// Aggregate function /// /// See also [`ExprFunctionExt`] to set these fields on `Expr` @@ -649,7 +662,7 @@ pub struct AggregateFunction { /// Optional filter pub filter: Option>, /// Optional ordering - pub order_by: Option>, + pub order_by: Option>, pub null_treatment: Option, } @@ -660,7 +673,7 @@ impl AggregateFunction { args: Vec, distinct: bool, filter: Option>, - order_by: Option>, + order_by: Option>, null_treatment: Option, ) -> Self { Self { @@ -785,7 +798,7 @@ pub struct WindowFunction { /// List of partition by expressions pub partition_by: Vec, /// List of order by expressions - pub order_by: Vec, + pub order_by: Vec, /// Window frame pub window_frame: window_frame::WindowFrame, /// Specifies how NULL value is treated: ignore or respect @@ -1141,7 +1154,6 @@ impl Expr { Expr::ScalarFunction(..) => "ScalarFunction", Expr::ScalarSubquery { .. } => "ScalarSubquery", Expr::ScalarVariable(..) => "ScalarVariable", - Expr::Sort { .. } => "Sort", Expr::TryCast { .. } => "TryCast", Expr::WindowFunction { .. } => "WindowFunction", Expr::Wildcard { .. } => "Wildcard", @@ -1227,14 +1239,9 @@ impl Expr { Expr::Like(Like::new(true, Box::new(self), Box::new(other), None, true)) } - /// Return the name to use for the specific Expr, recursing into - /// `Expr::Sort` as appropriate + /// Return the name to use for the specific Expr pub fn name_for_alias(&self) -> Result { - match self { - // call Expr::display_name() on a Expr::Sort will throw an error - Expr::Sort(Sort { expr, .. }) => expr.name_for_alias(), - expr => Ok(expr.schema_name().to_string()), - } + Ok(self.schema_name().to_string()) } /// Ensure `expr` has the name as `original_name` by adding an @@ -1250,14 +1257,7 @@ impl Expr { /// Return `self AS name` alias expression pub fn alias(self, name: impl Into) -> Expr { - match self { - Expr::Sort(Sort { - expr, - asc, - nulls_first, - }) => Expr::Sort(Sort::new(Box::new(expr.alias(name)), asc, nulls_first)), - _ => Expr::Alias(Alias::new(self, None::<&str>, name.into())), - } + Expr::Alias(Alias::new(self, None::<&str>, name.into())) } /// Return `self AS name` alias expression with a specific qualifier @@ -1266,18 +1266,7 @@ impl Expr { relation: Option>, name: impl Into, ) -> Expr { - match self { - Expr::Sort(Sort { - expr, - asc, - nulls_first, - }) => Expr::Sort(Sort::new( - Box::new(expr.alias_qualified(relation, name)), - asc, - nulls_first, - )), - _ => Expr::Alias(Alias::new(self, relation, name.into())), - } + Expr::Alias(Alias::new(self, relation, name.into())) } /// Remove an alias from an expression if one exists. @@ -1372,14 +1361,14 @@ impl Expr { Expr::IsNotNull(Box::new(self)) } - /// Create a sort expression from an existing expression. + /// Create a sort configuration from an existing expression. /// /// ``` /// # use datafusion_expr::col; /// let sort_expr = col("foo").sort(true, true); // SORT ASC NULLS_FIRST /// ``` - pub fn sort(self, asc: bool, nulls_first: bool) -> Expr { - Expr::Sort(Sort::new(Box::new(self), asc, nulls_first)) + pub fn sort(self, asc: bool, nulls_first: bool) -> Sort { + Sort::new(Box::new(self), asc, nulls_first) } /// Return `IsTrue(Box(self))` @@ -1655,7 +1644,6 @@ impl Expr { | Expr::Wildcard { .. } | Expr::WindowFunction(..) | Expr::Literal(..) - | Expr::Sort(..) | Expr::Placeholder(..) => false, } } @@ -1752,14 +1740,6 @@ impl Expr { }) => { data_type.hash(hasher); } - Expr::Sort(Sort { - expr: _expr, - asc, - nulls_first, - }) => { - asc.hash(hasher); - nulls_first.hash(hasher); - } Expr::ScalarFunction(ScalarFunction { func, args: _args }) => { func.hash(hasher); } @@ -1871,7 +1851,6 @@ impl<'a> Display for SchemaDisplay<'a> { Expr::Column(_) | Expr::Literal(_) | Expr::ScalarVariable(..) - | Expr::Sort(_) | Expr::OuterReferenceColumn(..) | Expr::Placeholder(_) | Expr::Wildcard { .. } => write!(f, "{}", self.0), @@ -1901,7 +1880,7 @@ impl<'a> Display for SchemaDisplay<'a> { }; if let Some(order_by) = order_by { - write!(f, " ORDER BY [{}]", schema_name_from_exprs(order_by)?)?; + write!(f, " ORDER BY [{}]", schema_name_from_sorts(order_by)?)?; }; Ok(()) @@ -2107,7 +2086,7 @@ impl<'a> Display for SchemaDisplay<'a> { } if !order_by.is_empty() { - write!(f, " ORDER BY [{}]", schema_name_from_exprs(order_by)?)?; + write!(f, " ORDER BY [{}]", schema_name_from_sorts(order_by)?)?; }; write!(f, " {window_frame}") @@ -2144,6 +2123,24 @@ fn schema_name_from_exprs_inner(exprs: &[Expr], sep: &str) -> Result Result { + let mut s = String::new(); + for (i, e) in sorts.iter().enumerate() { + if i > 0 { + write!(&mut s, ", ")?; + } + let ordering = if e.asc { "ASC" } else { "DESC" }; + let nulls_ordering = if e.nulls_first { + "NULLS FIRST" + } else { + "NULLS LAST" + }; + write!(&mut s, "{} {} {}", e.expr, ordering, nulls_ordering)?; + } + + Ok(s) +} + /// Format expressions for display as part of a logical plan. In many cases, this will produce /// similar output to `Expr.name()` except that column names will be prefixed with '#'. impl fmt::Display for Expr { @@ -2203,22 +2200,6 @@ impl fmt::Display for Expr { }) => write!(f, "{expr} IN ({subquery:?})"), Expr::ScalarSubquery(subquery) => write!(f, "({subquery:?})"), Expr::BinaryExpr(expr) => write!(f, "{expr}"), - Expr::Sort(Sort { - expr, - asc, - nulls_first, - }) => { - if *asc { - write!(f, "{expr} ASC")?; - } else { - write!(f, "{expr} DESC")?; - } - if *nulls_first { - write!(f, " NULLS FIRST") - } else { - write!(f, " NULLS LAST") - } - } Expr::ScalarFunction(fun) => { fmt_function(f, fun.name(), false, &fun.args, true) } diff --git a/datafusion/expr/src/expr_fn.rs b/datafusion/expr/src/expr_fn.rs index 1e0b601146dd..8d01712b95ad 100644 --- a/datafusion/expr/src/expr_fn.rs +++ b/datafusion/expr/src/expr_fn.rs @@ -26,9 +26,9 @@ use crate::function::{ StateFieldsArgs, }; use crate::{ - conditional_expressions::CaseBuilder, logical_plan::Subquery, AggregateUDF, Expr, - LogicalPlan, Operator, ScalarFunctionImplementation, ScalarUDF, Signature, - Volatility, + conditional_expressions::CaseBuilder, expr::Sort, logical_plan::Subquery, + AggregateUDF, Expr, LogicalPlan, Operator, ScalarFunctionImplementation, ScalarUDF, + Signature, Volatility, }; use crate::{ AggregateUDFImpl, ColumnarValue, ScalarUDFImpl, WindowFrame, WindowUDF, WindowUDFImpl, @@ -723,9 +723,7 @@ pub fn interval_month_day_nano_lit(value: &str) -> Expr { /// ``` pub trait ExprFunctionExt { /// Add `ORDER BY ` - /// - /// Note: `order_by` must be [`Expr::Sort`] - fn order_by(self, order_by: Vec) -> ExprFuncBuilder; + fn order_by(self, order_by: Vec) -> ExprFuncBuilder; /// Add `FILTER ` fn filter(self, filter: Expr) -> ExprFuncBuilder; /// Add `DISTINCT` @@ -753,7 +751,7 @@ pub enum ExprFuncKind { #[derive(Debug, Clone)] pub struct ExprFuncBuilder { fun: Option, - order_by: Option>, + order_by: Option>, filter: Option, distinct: bool, null_treatment: Option, @@ -798,16 +796,6 @@ impl ExprFuncBuilder { ); }; - if let Some(order_by) = &order_by { - for expr in order_by.iter() { - if !matches!(expr, Expr::Sort(_)) { - return plan_err!( - "ORDER BY expressions must be Expr::Sort, found {expr:?}" - ); - } - } - } - let fun_expr = match fun { ExprFuncKind::Aggregate(mut udaf) => { udaf.order_by = order_by; @@ -833,9 +821,7 @@ impl ExprFuncBuilder { impl ExprFunctionExt for ExprFuncBuilder { /// Add `ORDER BY ` - /// - /// Note: `order_by` must be [`Expr::Sort`] - fn order_by(mut self, order_by: Vec) -> ExprFuncBuilder { + fn order_by(mut self, order_by: Vec) -> ExprFuncBuilder { self.order_by = Some(order_by); self } @@ -873,7 +859,7 @@ impl ExprFunctionExt for ExprFuncBuilder { } impl ExprFunctionExt for Expr { - fn order_by(self, order_by: Vec) -> ExprFuncBuilder { + fn order_by(self, order_by: Vec) -> ExprFuncBuilder { let mut builder = match self { Expr::AggregateFunction(udaf) => { ExprFuncBuilder::new(Some(ExprFuncKind::Aggregate(udaf))) diff --git a/datafusion/expr/src/expr_rewriter/mod.rs b/datafusion/expr/src/expr_rewriter/mod.rs index 768c4aabc840..b809b015d929 100644 --- a/datafusion/expr/src/expr_rewriter/mod.rs +++ b/datafusion/expr/src/expr_rewriter/mod.rs @@ -21,7 +21,7 @@ use std::collections::HashMap; use std::collections::HashSet; use std::sync::Arc; -use crate::expr::{Alias, Unnest}; +use crate::expr::{Alias, Sort, Unnest}; use crate::logical_plan::Projection; use crate::{Expr, ExprSchemable, LogicalPlan, LogicalPlanBuilder}; @@ -117,6 +117,20 @@ pub fn normalize_cols( .collect() } +pub fn normalize_sorts( + sorts: impl IntoIterator>, + plan: &LogicalPlan, +) -> Result> { + sorts + .into_iter() + .map(|e| { + let sort = e.into(); + normalize_col(*sort.expr, plan) + .map(|expr| Sort::new(Box::new(expr), sort.asc, sort.nulls_first)) + }) + .collect() +} + /// Recursively replace all [`Column`] expressions in a given expression tree with /// `Column` expressions provided by the hash map argument. pub fn replace_col(expr: Expr, replace_map: &HashMap<&Column, &Column>) -> Result { @@ -335,7 +349,6 @@ mod test { use std::ops::Add; use super::*; - use crate::expr::Sort; use crate::{col, lit, Cast}; use arrow::datatypes::{DataType, Field, Schema}; use datafusion_common::ScalarValue; @@ -496,12 +509,6 @@ mod test { // change literal type from i32 to i64 test_rewrite(col("a").add(lit(1i32)), col("a").add(lit(1i64))); - - // SortExpr a+1 ==> b + 2 - test_rewrite( - Expr::Sort(Sort::new(Box::new(col("a").add(lit(1i32))), true, false)), - Expr::Sort(Sort::new(Box::new(col("b").add(lit(2i64))), true, false)), - ); } /// rewrites `expr_from` to `rewrite_to` using @@ -524,15 +531,8 @@ mod test { }; let expr = rewrite_preserving_name(expr_from.clone(), &mut rewriter).unwrap(); - let original_name = match &expr_from { - Expr::Sort(Sort { expr, .. }) => expr.schema_name().to_string(), - expr => expr.schema_name().to_string(), - }; - - let new_name = match &expr { - Expr::Sort(Sort { expr, .. }) => expr.schema_name().to_string(), - expr => expr.schema_name().to_string(), - }; + let original_name = expr_from.schema_name().to_string(); + let new_name = expr.schema_name().to_string(); assert_eq!( original_name, new_name, diff --git a/datafusion/expr/src/expr_rewriter/order_by.rs b/datafusion/expr/src/expr_rewriter/order_by.rs index bbb855801c3e..af5b8c4f9177 100644 --- a/datafusion/expr/src/expr_rewriter/order_by.rs +++ b/datafusion/expr/src/expr_rewriter/order_by.rs @@ -17,9 +17,9 @@ //! Rewrite for order by expressions -use crate::expr::{Alias, Sort}; +use crate::expr::Alias; use crate::expr_rewriter::normalize_col; -use crate::{Cast, Expr, ExprSchemable, LogicalPlan, TryCast}; +use crate::{expr::Sort, Cast, Expr, ExprSchemable, LogicalPlan, TryCast}; use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; use datafusion_common::{Column, Result}; @@ -27,28 +27,18 @@ use datafusion_common::{Column, Result}; /// Rewrite sort on aggregate expressions to sort on the column of aggregate output /// For example, `max(x)` is written to `col("max(x)")` pub fn rewrite_sort_cols_by_aggs( - exprs: impl IntoIterator>, + sorts: impl IntoIterator>, plan: &LogicalPlan, -) -> Result> { - exprs +) -> Result> { + sorts .into_iter() .map(|e| { - let expr = e.into(); - match expr { - Expr::Sort(Sort { - expr, - asc, - nulls_first, - }) => { - let sort = Expr::Sort(Sort::new( - Box::new(rewrite_sort_col_by_aggs(*expr, plan)?), - asc, - nulls_first, - )); - Ok(sort) - } - expr => Ok(expr), - } + let sort = e.into(); + Ok(Sort::new( + Box::new(rewrite_sort_col_by_aggs(*sort.expr, plan)?), + sort.asc, + sort.nulls_first, + )) }) .collect() } @@ -289,8 +279,8 @@ mod test { struct TestCase { desc: &'static str, - input: Expr, - expected: Expr, + input: Sort, + expected: Sort, } impl TestCase { @@ -332,7 +322,7 @@ mod test { .unwrap() } - fn sort(expr: Expr) -> Expr { + fn sort(expr: Expr) -> Sort { let asc = true; let nulls_first = true; expr.sort(asc, nulls_first) diff --git a/datafusion/expr/src/expr_schema.rs b/datafusion/expr/src/expr_schema.rs index 3920a1a3517c..894b7e58d954 100644 --- a/datafusion/expr/src/expr_schema.rs +++ b/datafusion/expr/src/expr_schema.rs @@ -18,7 +18,7 @@ use super::{Between, Expr, Like}; use crate::expr::{ AggregateFunction, Alias, BinaryExpr, Cast, InList, InSubquery, Placeholder, - ScalarFunction, Sort, TryCast, Unnest, WindowFunction, + ScalarFunction, TryCast, Unnest, WindowFunction, }; use crate::type_coercion::binary::get_result_type; use crate::type_coercion::functions::{ @@ -107,7 +107,7 @@ impl ExprSchemable for Expr { }, _ => expr.get_type(schema), }, - Expr::Sort(Sort { expr, .. }) | Expr::Negative(expr) => expr.get_type(schema), + Expr::Negative(expr) => expr.get_type(schema), Expr::Column(c) => Ok(schema.data_type(c)?.clone()), Expr::OuterReferenceColumn(ty, _) => Ok(ty.clone()), Expr::ScalarVariable(ty, _) => Ok(ty.clone()), @@ -280,10 +280,9 @@ impl ExprSchemable for Expr { /// column that does not exist in the schema. fn nullable(&self, input_schema: &dyn ExprSchema) -> Result { match self { - Expr::Alias(Alias { expr, .. }) - | Expr::Not(expr) - | Expr::Negative(expr) - | Expr::Sort(Sort { expr, .. }) => expr.nullable(input_schema), + Expr::Alias(Alias { expr, .. }) | Expr::Not(expr) | Expr::Negative(expr) => { + expr.nullable(input_schema) + } Expr::InList(InList { expr, list, .. }) => { // Avoid inspecting too many expressions. @@ -422,9 +421,7 @@ impl ExprSchemable for Expr { }, _ => expr.data_type_and_nullable(schema), }, - Expr::Sort(Sort { expr, .. }) | Expr::Negative(expr) => { - expr.data_type_and_nullable(schema) - } + Expr::Negative(expr) => expr.data_type_and_nullable(schema), Expr::Column(c) => schema .data_type_and_nullable(c) .map(|(d, n)| (d.clone(), n)), diff --git a/datafusion/expr/src/logical_plan/builder.rs b/datafusion/expr/src/logical_plan/builder.rs index 2c2300b123c2..f5770167861b 100644 --- a/datafusion/expr/src/logical_plan/builder.rs +++ b/datafusion/expr/src/logical_plan/builder.rs @@ -23,10 +23,10 @@ use std::collections::{HashMap, HashSet}; use std::sync::Arc; use crate::dml::CopyTo; -use crate::expr::Alias; +use crate::expr::{Alias, Sort as SortExpr}; use crate::expr_rewriter::{ coerce_plan_expr_for_schema, normalize_col, - normalize_col_with_schemas_and_ambiguity_check, normalize_cols, + normalize_col_with_schemas_and_ambiguity_check, normalize_cols, normalize_sorts, rewrite_sort_cols_by_aggs, }; use crate::logical_plan::{ @@ -541,19 +541,31 @@ impl LogicalPlanBuilder { plan_err!("For SELECT DISTINCT, ORDER BY expressions {missing_col_names} must appear in select list") } + /// Apply a sort by provided expressions with default direction + pub fn sort_by( + self, + expr: impl IntoIterator> + Clone, + ) -> Result { + self.sort( + expr.into_iter() + .map(|e| e.into().sort(true, false)) + .collect::>(), + ) + } + /// Apply a sort pub fn sort( self, - exprs: impl IntoIterator> + Clone, + sorts: impl IntoIterator> + Clone, ) -> Result { - let exprs = rewrite_sort_cols_by_aggs(exprs, &self.plan)?; + let sorts = rewrite_sort_cols_by_aggs(sorts, &self.plan)?; let schema = self.plan.schema(); // Collect sort columns that are missing in the input plan's schema let mut missing_cols: Vec = vec![]; - exprs.iter().try_for_each::<_, Result<()>>(|expr| { - let columns = expr.column_refs(); + sorts.iter().try_for_each::<_, Result<()>>(|sort| { + let columns = sort.expr.column_refs(); columns.into_iter().for_each(|c| { if !schema.has_column(c) { @@ -566,7 +578,7 @@ impl LogicalPlanBuilder { if missing_cols.is_empty() { return Ok(Self::new(LogicalPlan::Sort(Sort { - expr: normalize_cols(exprs, &self.plan)?, + expr: normalize_sorts(sorts, &self.plan)?, input: self.plan, fetch: None, }))); @@ -582,7 +594,7 @@ impl LogicalPlanBuilder { is_distinct, )?; let sort_plan = LogicalPlan::Sort(Sort { - expr: normalize_cols(exprs, &plan)?, + expr: normalize_sorts(sorts, &plan)?, input: Arc::new(plan), fetch: None, }); @@ -618,7 +630,7 @@ impl LogicalPlanBuilder { self, on_expr: Vec, select_expr: Vec, - sort_expr: Option>, + sort_expr: Option>, ) -> Result { Ok(Self::new(LogicalPlan::Distinct(Distinct::On( DistinctOn::try_new(on_expr, select_expr, sort_expr, self.plan)?, @@ -1708,8 +1720,8 @@ mod tests { let plan = table_scan(Some("employee_csv"), &employee_schema(), Some(vec![3, 4]))? .sort(vec![ - Expr::Sort(expr::Sort::new(Box::new(col("state")), true, true)), - Expr::Sort(expr::Sort::new(Box::new(col("salary")), false, false)), + expr::Sort::new(Box::new(col("state")), true, true), + expr::Sort::new(Box::new(col("salary")), false, false), ])? .build()?; @@ -2135,8 +2147,8 @@ mod tests { let plan = table_scan(Some("employee_csv"), &employee_schema(), Some(vec![3, 4]))? .sort(vec![ - Expr::Sort(expr::Sort::new(Box::new(col("state")), true, true)), - Expr::Sort(expr::Sort::new(Box::new(col("salary")), false, false)), + expr::Sort::new(Box::new(col("state")), true, true), + expr::Sort::new(Box::new(col("salary")), false, false), ])? .build()?; diff --git a/datafusion/expr/src/logical_plan/ddl.rs b/datafusion/expr/src/logical_plan/ddl.rs index ad0fcd2d4771..3fc43200efe6 100644 --- a/datafusion/expr/src/logical_plan/ddl.rs +++ b/datafusion/expr/src/logical_plan/ddl.rs @@ -22,8 +22,9 @@ use std::{ hash::{Hash, Hasher}, }; -use crate::{Expr, LogicalPlan, Volatility}; +use crate::{Expr, LogicalPlan, SortExpr, Volatility}; +use crate::expr::Sort; use arrow::datatypes::DataType; use datafusion_common::{Constraints, DFSchemaRef, SchemaReference, TableReference}; use sqlparser::ast::Ident; @@ -204,7 +205,7 @@ pub struct CreateExternalTable { /// SQL used to create the table, if available pub definition: Option, /// Order expressions supplied by user - pub order_exprs: Vec>, + pub order_exprs: Vec>, /// Whether the table is an infinite streams pub unbounded: bool, /// Table(provider) specific options @@ -365,7 +366,7 @@ pub struct CreateIndex { pub name: Option, pub table: TableReference, pub using: Option, - pub columns: Vec, + pub columns: Vec, pub unique: bool, pub if_not_exists: bool, pub schema: DFSchemaRef, diff --git a/datafusion/expr/src/logical_plan/plan.rs b/datafusion/expr/src/logical_plan/plan.rs index 359de2d30a57..8e6ec762f549 100644 --- a/datafusion/expr/src/logical_plan/plan.rs +++ b/datafusion/expr/src/logical_plan/plan.rs @@ -26,7 +26,9 @@ use super::dml::CopyTo; use super::DdlStatement; use crate::builder::{change_redundant_column, unnest_with_options}; use crate::expr::{Placeholder, Sort as SortExpr, WindowFunction}; -use crate::expr_rewriter::{create_col_from_scalar_expr, normalize_cols, NamePreserver}; +use crate::expr_rewriter::{ + create_col_from_scalar_expr, normalize_cols, normalize_sorts, NamePreserver, +}; use crate::logical_plan::display::{GraphvizVisitor, IndentVisitor}; use crate::logical_plan::extension::UserDefinedLogicalNode; use crate::logical_plan::{DmlStatement, Statement}; @@ -51,6 +53,7 @@ use datafusion_common::{ // backwards compatibility use crate::display::PgJsonVisitor; +use crate::tree_node::replace_sort_expressions; pub use datafusion_common::display::{PlanType, StringifiedPlan, ToStringifiedPlan}; pub use datafusion_common::{JoinConstraint, JoinType}; @@ -884,8 +887,12 @@ impl LogicalPlan { Aggregate::try_new(Arc::new(inputs.swap_remove(0)), expr, agg_expr) .map(LogicalPlan::Aggregate) } - LogicalPlan::Sort(Sort { fetch, .. }) => Ok(LogicalPlan::Sort(Sort { - expr, + LogicalPlan::Sort(Sort { + expr: sort_expr, + fetch, + .. + }) => Ok(LogicalPlan::Sort(Sort { + expr: replace_sort_expressions(sort_expr.clone(), expr), input: Arc::new(inputs.swap_remove(0)), fetch: *fetch, })), @@ -1014,14 +1021,11 @@ impl LogicalPlan { }) => { let sort_expr = expr.split_off(on_expr.len() + select_expr.len()); let select_expr = expr.split_off(on_expr.len()); + assert!(sort_expr.is_empty(), "with_new_exprs for Distinct does not support sort expressions"); Distinct::On(DistinctOn::try_new( expr, select_expr, - if !sort_expr.is_empty() { - Some(sort_expr) - } else { - None - }, + None, // no sort expressions accepted Arc::new(inputs.swap_remove(0)), )?) } @@ -2559,7 +2563,7 @@ pub struct DistinctOn { /// The `ORDER BY` clause, whose initial expressions must match those of the `ON` clause when /// present. Note that those matching expressions actually wrap the `ON` expressions with /// additional info pertaining to the sorting procedure (i.e. ASC/DESC, and NULLS FIRST/LAST). - pub sort_expr: Option>, + pub sort_expr: Option>, /// The logical plan that is being DISTINCT'd pub input: Arc, /// The schema description of the DISTINCT ON output @@ -2571,7 +2575,7 @@ impl DistinctOn { pub fn try_new( on_expr: Vec, select_expr: Vec, - sort_expr: Option>, + sort_expr: Option>, input: Arc, ) -> Result { if on_expr.is_empty() { @@ -2606,20 +2610,15 @@ impl DistinctOn { /// Try to update `self` with a new sort expressions. /// /// Validates that the sort expressions are a super-set of the `ON` expressions. - pub fn with_sort_expr(mut self, sort_expr: Vec) -> Result { - let sort_expr = normalize_cols(sort_expr, self.input.as_ref())?; + pub fn with_sort_expr(mut self, sort_expr: Vec) -> Result { + let sort_expr = normalize_sorts(sort_expr, self.input.as_ref())?; // Check that the left-most sort expressions are the same as the `ON` expressions. let mut matched = true; for (on, sort) in self.on_expr.iter().zip(sort_expr.iter()) { - match sort { - Expr::Sort(SortExpr { expr, .. }) => { - if on != &**expr { - matched = false; - break; - } - } - _ => return plan_err!("Not a sort expression: {sort}"), + if on != &*sort.expr { + matched = false; + break; } } @@ -2833,7 +2832,7 @@ fn calc_func_dependencies_for_project( #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct Sort { /// The sort expressions - pub expr: Vec, + pub expr: Vec, /// The incoming logical plan pub input: Arc, /// Optional fetch limit diff --git a/datafusion/expr/src/logical_plan/tree_node.rs b/datafusion/expr/src/logical_plan/tree_node.rs index 273404c8df31..29a99a8e8886 100644 --- a/datafusion/expr/src/logical_plan/tree_node.rs +++ b/datafusion/expr/src/logical_plan/tree_node.rs @@ -46,7 +46,7 @@ use crate::{ use std::sync::Arc; use crate::expr::{Exists, InSubquery}; -use crate::tree_node::transform_option_vec; +use crate::tree_node::{transform_sort_option_vec, transform_sort_vec}; use datafusion_common::tree_node::{ Transformed, TreeNode, TreeNodeIterator, TreeNodeRecursion, TreeNodeRewriter, TreeNodeVisitor, @@ -481,7 +481,9 @@ impl LogicalPlan { .apply_until_stop(|e| f(&e))? .visit_sibling(|| filter.iter().apply_until_stop(f)) } - LogicalPlan::Sort(Sort { expr, .. }) => expr.iter().apply_until_stop(f), + LogicalPlan::Sort(Sort { expr, .. }) => { + expr.iter().apply_until_stop(|sort| f(&sort.expr)) + } LogicalPlan::Extension(extension) => { // would be nice to avoid this copy -- maybe can // update extension to just observer Exprs @@ -507,7 +509,7 @@ impl LogicalPlan { })) => on_expr .iter() .chain(select_expr.iter()) - .chain(sort_expr.iter().flatten()) + .chain(sort_expr.iter().flatten().map(|sort| &*sort.expr)) .apply_until_stop(f), // plans without expressions LogicalPlan::EmptyRelation(_) @@ -658,10 +660,10 @@ impl LogicalPlan { null_equals_null, }) }), - LogicalPlan::Sort(Sort { expr, input, fetch }) => expr - .into_iter() - .map_until_stop_and_collect(f)? - .update_data(|expr| LogicalPlan::Sort(Sort { expr, input, fetch })), + LogicalPlan::Sort(Sort { expr, input, fetch }) => { + transform_sort_vec(expr, &mut f)? + .update_data(|expr| LogicalPlan::Sort(Sort { expr, input, fetch })) + } LogicalPlan::Extension(Extension { node }) => { // would be nice to avoid this copy -- maybe can // update extension to just observer Exprs @@ -709,7 +711,7 @@ impl LogicalPlan { select_expr, select_expr.into_iter().map_until_stop_and_collect(&mut f), sort_expr, - transform_option_vec(sort_expr, &mut f) + transform_sort_option_vec(sort_expr, &mut f) )? .update_data(|(on_expr, select_expr, sort_expr)| { LogicalPlan::Distinct(Distinct::On(DistinctOn { diff --git a/datafusion/expr/src/tree_node.rs b/datafusion/expr/src/tree_node.rs index 450ebb6c2275..90d61bf63763 100644 --- a/datafusion/expr/src/tree_node.rs +++ b/datafusion/expr/src/tree_node.rs @@ -48,7 +48,6 @@ impl TreeNode for Expr { | Expr::Negative(expr) | Expr::Cast(Cast { expr, .. }) | Expr::TryCast(TryCast { expr, .. }) - | Expr::Sort(Sort { expr, .. }) | Expr::InSubquery(InSubquery{ expr, .. }) => vec![expr.as_ref()], Expr::GroupingSet(GroupingSet::Rollup(exprs)) | Expr::GroupingSet(GroupingSet::Cube(exprs)) => exprs.iter().collect(), @@ -98,7 +97,7 @@ impl TreeNode for Expr { expr_vec.push(f.as_ref()); } if let Some(order_by) = order_by { - expr_vec.extend(order_by); + expr_vec.extend(order_by.iter().map(|sort| sort.expr.as_ref())); } expr_vec } @@ -110,7 +109,7 @@ impl TreeNode for Expr { }) => { let mut expr_vec = args.iter().collect::>(); expr_vec.extend(partition_by); - expr_vec.extend(order_by); + expr_vec.extend(order_by.iter().map(|sort| sort.expr.as_ref())); expr_vec } Expr::InList(InList { expr, list, .. }) => { @@ -265,12 +264,6 @@ impl TreeNode for Expr { .update_data(|be| Expr::Cast(Cast::new(be, data_type))), Expr::TryCast(TryCast { expr, data_type }) => transform_box(expr, &mut f)? .update_data(|be| Expr::TryCast(TryCast::new(be, data_type))), - Expr::Sort(Sort { - expr, - asc, - nulls_first, - }) => transform_box(expr, &mut f)? - .update_data(|be| Expr::Sort(Sort::new(be, asc, nulls_first))), Expr::ScalarFunction(ScalarFunction { func, args }) => { transform_vec(args, &mut f)?.map_data(|new_args| { Ok(Expr::ScalarFunction(ScalarFunction::new_udf( @@ -290,7 +283,7 @@ impl TreeNode for Expr { partition_by, transform_vec(partition_by, &mut f), order_by, - transform_vec(order_by, &mut f) + transform_sort_vec(order_by, &mut f) )? .update_data(|(new_args, new_partition_by, new_order_by)| { Expr::WindowFunction(WindowFunction::new(fun, new_args)) @@ -313,7 +306,7 @@ impl TreeNode for Expr { filter, transform_option_box(filter, &mut f), order_by, - transform_option_vec(order_by, &mut f) + transform_sort_option_vec(order_by, &mut f) )? .map_data(|(new_args, new_filter, new_order_by)| { Ok(Expr::AggregateFunction(AggregateFunction::new_udf( @@ -386,3 +379,41 @@ fn transform_vec Result>>( ) -> Result>> { ve.into_iter().map_until_stop_and_collect(f) } + +pub fn transform_sort_option_vec Result>>( + sorts_option: Option>, + f: &mut F, +) -> Result>>> { + sorts_option.map_or(Ok(Transformed::no(None)), |sorts| { + Ok(transform_sort_vec(sorts, f)?.update_data(Some)) + }) +} + +pub fn transform_sort_vec Result>>( + sorts: Vec, + mut f: &mut F, +) -> Result>> { + Ok(sorts + .iter() + .map(|sort| (*sort.expr).clone()) + .map_until_stop_and_collect(&mut f)? + .update_data(|transformed_exprs| { + replace_sort_expressions(sorts, transformed_exprs) + })) +} + +pub fn replace_sort_expressions(sorts: Vec, new_expr: Vec) -> Vec { + assert_eq!(sorts.len(), new_expr.len()); + sorts + .into_iter() + .zip(new_expr) + .map(|(sort, expr)| replace_sort_expression(sort, expr)) + .collect() +} + +pub fn replace_sort_expression(sort: Sort, new_expr: Expr) -> Sort { + Sort { + expr: Box::new(new_expr), + ..sort + } +} diff --git a/datafusion/expr/src/utils.rs b/datafusion/expr/src/utils.rs index a01d5ef8973a..b6b1b5660a81 100644 --- a/datafusion/expr/src/utils.rs +++ b/datafusion/expr/src/utils.rs @@ -296,7 +296,6 @@ pub fn expr_to_columns(expr: &Expr, accum: &mut HashSet) -> Result<()> { | Expr::Case { .. } | Expr::Cast { .. } | Expr::TryCast { .. } - | Expr::Sort { .. } | Expr::ScalarFunction(..) | Expr::WindowFunction { .. } | Expr::AggregateFunction { .. } @@ -461,22 +460,20 @@ pub fn expand_qualified_wildcard( /// (expr, "is the SortExpr for window (either comes from PARTITION BY or ORDER BY columns)") /// if bool is true SortExpr comes from `PARTITION BY` column, if false comes from `ORDER BY` column -type WindowSortKey = Vec<(Expr, bool)>; +type WindowSortKey = Vec<(Sort, bool)>; /// Generate a sort key for a given window expr's partition_by and order_by expr pub fn generate_sort_key( partition_by: &[Expr], - order_by: &[Expr], + order_by: &[Sort], ) -> Result { let normalized_order_by_keys = order_by .iter() - .map(|e| match e { - Expr::Sort(Sort { expr, .. }) => { - Ok(Expr::Sort(Sort::new(expr.clone(), true, false))) - } - _ => plan_err!("Order by only accepts sort expressions"), + .map(|e| { + let Sort { expr, .. } = e; + Sort::new(expr.clone(), true, false) }) - .collect::>>()?; + .collect::>(); let mut final_sort_keys = vec![]; let mut is_partition_flag = vec![]; @@ -512,65 +509,61 @@ pub fn generate_sort_key( /// Compare the sort expr as PostgreSQL's common_prefix_cmp(): /// pub fn compare_sort_expr( - sort_expr_a: &Expr, - sort_expr_b: &Expr, + sort_expr_a: &Sort, + sort_expr_b: &Sort, schema: &DFSchemaRef, ) -> Ordering { - match (sort_expr_a, sort_expr_b) { - ( - Expr::Sort(Sort { - expr: expr_a, - asc: asc_a, - nulls_first: nulls_first_a, - }), - Expr::Sort(Sort { - expr: expr_b, - asc: asc_b, - nulls_first: nulls_first_b, - }), - ) => { - let ref_indexes_a = find_column_indexes_referenced_by_expr(expr_a, schema); - let ref_indexes_b = find_column_indexes_referenced_by_expr(expr_b, schema); - for (idx_a, idx_b) in ref_indexes_a.iter().zip(ref_indexes_b.iter()) { - match idx_a.cmp(idx_b) { - Ordering::Less => { - return Ordering::Less; - } - Ordering::Greater => { - return Ordering::Greater; - } - Ordering::Equal => {} - } + let Sort { + expr: expr_a, + asc: asc_a, + nulls_first: nulls_first_a, + } = sort_expr_a; + + let Sort { + expr: expr_b, + asc: asc_b, + nulls_first: nulls_first_b, + } = sort_expr_b; + + let ref_indexes_a = find_column_indexes_referenced_by_expr(expr_a, schema); + let ref_indexes_b = find_column_indexes_referenced_by_expr(expr_b, schema); + for (idx_a, idx_b) in ref_indexes_a.iter().zip(ref_indexes_b.iter()) { + match idx_a.cmp(idx_b) { + Ordering::Less => { + return Ordering::Less; } - match ref_indexes_a.len().cmp(&ref_indexes_b.len()) { - Ordering::Less => return Ordering::Greater, - Ordering::Greater => { - return Ordering::Less; - } - Ordering::Equal => {} + Ordering::Greater => { + return Ordering::Greater; } - match (asc_a, asc_b) { - (true, false) => { - return Ordering::Greater; - } - (false, true) => { - return Ordering::Less; - } - _ => {} - } - match (nulls_first_a, nulls_first_b) { - (true, false) => { - return Ordering::Less; - } - (false, true) => { - return Ordering::Greater; - } - _ => {} - } - Ordering::Equal + Ordering::Equal => {} } - _ => panic!("Sort expressions must be of type Sort"), } + match ref_indexes_a.len().cmp(&ref_indexes_b.len()) { + Ordering::Less => return Ordering::Greater, + Ordering::Greater => { + return Ordering::Less; + } + Ordering::Equal => {} + } + match (asc_a, asc_b) { + (true, false) => { + return Ordering::Greater; + } + (false, true) => { + return Ordering::Less; + } + _ => {} + } + match (nulls_first_a, nulls_first_b) { + (true, false) => { + return Ordering::Less; + } + (false, true) => { + return Ordering::Greater; + } + _ => {} + } + Ordering::Equal } /// group a slice of window expression expr by their order by expressions @@ -606,14 +599,6 @@ pub fn find_aggregate_exprs(exprs: &[Expr]) -> Vec { }) } -/// Collect all deeply nested `Expr::Sort`. They are returned in order of occurrence -/// (depth first), with duplicates omitted. -pub fn find_sort_exprs(exprs: &[Expr]) -> Vec { - find_exprs_in_exprs(exprs, &|nested_expr| { - matches!(nested_expr, Expr::Sort { .. }) - }) -} - /// Collect all deeply nested `Expr::WindowFunction`. They are returned in order of occurrence /// (depth first), with duplicates omitted. pub fn find_window_exprs(exprs: &[Expr]) -> Vec { @@ -1376,8 +1361,7 @@ mod tests { use crate::{ col, cube, expr, expr_vec_fmt, grouping_set, lit, rollup, test::function_stub::max_udaf, test::function_stub::min_udaf, - test::function_stub::sum_udaf, Cast, ExprFunctionExt, WindowFrame, - WindowFunctionDefinition, + test::function_stub::sum_udaf, Cast, ExprFunctionExt, WindowFunctionDefinition, }; #[test] @@ -1417,10 +1401,9 @@ mod tests { #[test] fn test_group_window_expr_by_sort_keys() -> Result<()> { - let age_asc = Expr::Sort(expr::Sort::new(Box::new(col("age")), true, true)); - let name_desc = Expr::Sort(expr::Sort::new(Box::new(col("name")), false, true)); - let created_at_desc = - Expr::Sort(expr::Sort::new(Box::new(col("created_at")), false, true)); + let age_asc = expr::Sort::new(Box::new(col("age")), true, true); + let name_desc = expr::Sort::new(Box::new(col("name")), false, true); + let created_at_desc = expr::Sort::new(Box::new(col("created_at")), false, true); let max1 = Expr::WindowFunction(expr::WindowFunction::new( WindowFunctionDefinition::AggregateUDF(max_udaf()), vec![col("name")], @@ -1471,43 +1454,6 @@ mod tests { Ok(()) } - #[test] - fn test_find_sort_exprs() -> Result<()> { - let exprs = &[ - Expr::WindowFunction(expr::WindowFunction::new( - WindowFunctionDefinition::AggregateUDF(max_udaf()), - vec![col("name")], - )) - .order_by(vec![ - Expr::Sort(expr::Sort::new(Box::new(col("age")), true, true)), - Expr::Sort(expr::Sort::new(Box::new(col("name")), false, true)), - ]) - .window_frame(WindowFrame::new(Some(false))) - .build() - .unwrap(), - Expr::WindowFunction(expr::WindowFunction::new( - WindowFunctionDefinition::AggregateUDF(sum_udaf()), - vec![col("age")], - )) - .order_by(vec![ - Expr::Sort(expr::Sort::new(Box::new(col("name")), false, true)), - Expr::Sort(expr::Sort::new(Box::new(col("age")), true, true)), - Expr::Sort(expr::Sort::new(Box::new(col("created_at")), false, true)), - ]) - .window_frame(WindowFrame::new(Some(false))) - .build() - .unwrap(), - ]; - let expected = vec![ - Expr::Sort(expr::Sort::new(Box::new(col("age")), true, true)), - Expr::Sort(expr::Sort::new(Box::new(col("name")), false, true)), - Expr::Sort(expr::Sort::new(Box::new(col("created_at")), false, true)), - ]; - let result = find_sort_exprs(exprs); - assert_eq!(expected, result); - Ok(()) - } - #[test] fn avoid_generate_duplicate_sort_keys() -> Result<()> { let asc_or_desc = [true, false]; @@ -1516,41 +1462,41 @@ mod tests { for asc_ in asc_or_desc { for nulls_first_ in nulls_first_or_last { let order_by = &[ - Expr::Sort(Sort { + Sort { expr: Box::new(col("age")), asc: asc_, nulls_first: nulls_first_, - }), - Expr::Sort(Sort { + }, + Sort { expr: Box::new(col("name")), asc: asc_, nulls_first: nulls_first_, - }), + }, ]; let expected = vec![ ( - Expr::Sort(Sort { + Sort { expr: Box::new(col("age")), asc: asc_, nulls_first: nulls_first_, - }), + }, true, ), ( - Expr::Sort(Sort { + Sort { expr: Box::new(col("name")), asc: asc_, nulls_first: nulls_first_, - }), + }, true, ), ( - Expr::Sort(Sort { + Sort { expr: Box::new(col("created_at")), asc: true, nulls_first: false, - }), + }, true, ), ]; diff --git a/datafusion/expr/src/window_frame.rs b/datafusion/expr/src/window_frame.rs index 0e1d917419f8..6c935cdcd121 100644 --- a/datafusion/expr/src/window_frame.rs +++ b/datafusion/expr/src/window_frame.rs @@ -26,7 +26,7 @@ use std::fmt::{self, Formatter}; use std::hash::Hash; -use crate::{lit, Expr}; +use crate::{expr::Sort, lit}; use datafusion_common::{plan_err, sql_err, DataFusionError, Result, ScalarValue}; use sqlparser::ast; @@ -247,7 +247,7 @@ impl WindowFrame { } /// Regularizes the ORDER BY clause of the window frame. - pub fn regularize_order_bys(&self, order_by: &mut Vec) -> Result<()> { + pub fn regularize_order_bys(&self, order_by: &mut Vec) -> Result<()> { match self.units { // Normally, RANGE frames require an ORDER BY clause with exactly // one column. However, an ORDER BY clause may be absent or have diff --git a/datafusion/functions-aggregate/src/first_last.rs b/datafusion/functions-aggregate/src/first_last.rs index 2162442f054e..30f5d5b07561 100644 --- a/datafusion/functions-aggregate/src/first_last.rs +++ b/datafusion/functions-aggregate/src/first_last.rs @@ -32,7 +32,7 @@ use datafusion_expr::function::{AccumulatorArgs, StateFieldsArgs}; use datafusion_expr::utils::{format_state_name, AggregateOrderSensitivity}; use datafusion_expr::{ Accumulator, AggregateUDFImpl, ArrayFunctionSignature, Expr, ExprFunctionExt, - Signature, TypeSignature, Volatility, + Signature, SortExpr, TypeSignature, Volatility, }; use datafusion_functions_aggregate_common::utils::get_sort_options; use datafusion_physical_expr_common::sort_expr::{LexOrdering, PhysicalSortExpr}; @@ -40,7 +40,7 @@ use datafusion_physical_expr_common::sort_expr::{LexOrdering, PhysicalSortExpr}; create_func!(FirstValue, first_value_udaf); /// Returns the first value in a group of values. -pub fn first_value(expression: Expr, order_by: Option>) -> Expr { +pub fn first_value(expression: Expr, order_by: Option>) -> Expr { if let Some(order_by) = order_by { first_value_udaf() .call(vec![expression]) diff --git a/datafusion/optimizer/src/analyzer/count_wildcard_rule.rs b/datafusion/optimizer/src/analyzer/count_wildcard_rule.rs index e114efb99960..35d4f91e3b6f 100644 --- a/datafusion/optimizer/src/analyzer/count_wildcard_rule.rs +++ b/datafusion/optimizer/src/analyzer/count_wildcard_rule.rs @@ -229,7 +229,7 @@ mod tests { WindowFunctionDefinition::AggregateUDF(count_udaf()), vec![wildcard()], )) - .order_by(vec![Expr::Sort(Sort::new(Box::new(col("a")), false, true))]) + .order_by(vec![Sort::new(Box::new(col("a")), false, true)]) .window_frame(WindowFrame::new_bounds( WindowFrameUnits::Range, WindowFrameBound::Preceding(ScalarValue::UInt32(Some(6))), diff --git a/datafusion/optimizer/src/analyzer/type_coercion.rs b/datafusion/optimizer/src/analyzer/type_coercion.rs index a6b9bad6c5d9..61ff4b4fd5a8 100644 --- a/datafusion/optimizer/src/analyzer/type_coercion.rs +++ b/datafusion/optimizer/src/analyzer/type_coercion.rs @@ -33,7 +33,7 @@ use datafusion_common::{ }; use datafusion_expr::expr::{ self, Alias, Between, BinaryExpr, Case, Exists, InList, InSubquery, Like, - ScalarFunction, WindowFunction, + ScalarFunction, Sort, WindowFunction, }; use datafusion_expr::expr_rewriter::coerce_plan_expr_for_schema; use datafusion_expr::expr_schema::cast_subquery; @@ -506,7 +506,6 @@ impl<'a> TreeNodeRewriter for TypeCoercionRewriter<'a> { | Expr::Negative(_) | Expr::Cast(_) | Expr::TryCast(_) - | Expr::Sort(_) | Expr::Wildcard { .. } | Expr::GroupingSet(_) | Expr::Placeholder(_) @@ -593,12 +592,12 @@ fn coerce_frame_bound( fn coerce_window_frame( window_frame: WindowFrame, schema: &DFSchema, - expressions: &[Expr], + expressions: &[Sort], ) -> Result { let mut window_frame = window_frame; let current_types = expressions .iter() - .map(|e| e.get_type(schema)) + .map(|s| s.expr.get_type(schema)) .collect::>>()?; let target_type = match window_frame.units { WindowFrameUnits::Range => { diff --git a/datafusion/optimizer/src/common_subexpr_eliminate.rs b/datafusion/optimizer/src/common_subexpr_eliminate.rs index 3a2b190359d4..25bef7e2d0e4 100644 --- a/datafusion/optimizer/src/common_subexpr_eliminate.rs +++ b/datafusion/optimizer/src/common_subexpr_eliminate.rs @@ -36,6 +36,7 @@ use datafusion_expr::expr::{Alias, ScalarFunction}; use datafusion_expr::logical_plan::{ Aggregate, Filter, LogicalPlan, Projection, Sort, Window, }; +use datafusion_expr::tree_node::replace_sort_expressions; use datafusion_expr::{col, BinaryExpr, Case, Expr, ExprSchemable, Operator}; use indexmap::IndexMap; @@ -327,15 +328,17 @@ impl CommonSubexprEliminate { ) -> Result> { let Sort { expr, input, fetch } = sort; let input = Arc::unwrap_or_clone(input); - let new_sort = self.try_unary_plan(expr, input, config)?.update_data( - |(new_expr, new_input)| { + let sort_expressions = + expr.iter().map(|sort| sort.expr.as_ref().clone()).collect(); + let new_sort = self + .try_unary_plan(sort_expressions, input, config)? + .update_data(|(new_expr, new_input)| { LogicalPlan::Sort(Sort { - expr: new_expr, + expr: replace_sort_expressions(expr, new_expr), input: Arc::new(new_input), fetch, }) - }, - ); + }); Ok(new_sort) } @@ -882,7 +885,6 @@ enum ExprMask { /// - [`Columns`](Expr::Column) /// - [`ScalarVariable`](Expr::ScalarVariable) /// - [`Alias`](Expr::Alias) - /// - [`Sort`](Expr::Sort) /// - [`Wildcard`](Expr::Wildcard) /// - [`AggregateFunction`](Expr::AggregateFunction) Normal, @@ -899,7 +901,6 @@ impl ExprMask { | Expr::Column(..) | Expr::ScalarVariable(..) | Expr::Alias(..) - | Expr::Sort { .. } | Expr::Wildcard { .. } ); diff --git a/datafusion/optimizer/src/eliminate_duplicated_expr.rs b/datafusion/optimizer/src/eliminate_duplicated_expr.rs index e9d091d52b00..c460d7a93d26 100644 --- a/datafusion/optimizer/src/eliminate_duplicated_expr.rs +++ b/datafusion/optimizer/src/eliminate_duplicated_expr.rs @@ -22,7 +22,7 @@ use crate::{OptimizerConfig, OptimizerRule}; use datafusion_common::tree_node::Transformed; use datafusion_common::Result; use datafusion_expr::logical_plan::LogicalPlan; -use datafusion_expr::{Aggregate, Expr, Sort}; +use datafusion_expr::{Aggregate, Expr, Sort, SortExpr}; use indexmap::IndexSet; use std::hash::{Hash, Hasher}; /// Optimization rule that eliminate duplicated expr. @@ -37,29 +37,15 @@ impl EliminateDuplicatedExpr { } // use this structure to avoid initial clone #[derive(Eq, Clone, Debug)] -struct SortExprWrapper { - expr: Expr, -} +struct SortExprWrapper(SortExpr); impl PartialEq for SortExprWrapper { fn eq(&self, other: &Self) -> bool { - match (&self.expr, &other.expr) { - (Expr::Sort(own_sort), Expr::Sort(other_sort)) => { - own_sort.expr == other_sort.expr - } - _ => self.expr == other.expr, - } + self.0.expr == other.0.expr } } impl Hash for SortExprWrapper { fn hash(&self, state: &mut H) { - match &self.expr { - Expr::Sort(sort) => { - sort.expr.hash(state); - } - _ => { - self.expr.hash(state); - } - } + self.0.expr.hash(state); } } impl OptimizerRule for EliminateDuplicatedExpr { @@ -82,10 +68,10 @@ impl OptimizerRule for EliminateDuplicatedExpr { let unique_exprs: Vec<_> = sort .expr .into_iter() - .map(|e| SortExprWrapper { expr: e }) + .map(SortExprWrapper) .collect::>() .into_iter() - .map(|wrapper| wrapper.expr) + .map(|wrapper| wrapper.0) .collect(); let transformed = if len != unique_exprs.len() { @@ -146,11 +132,11 @@ mod tests { fn eliminate_sort_expr() -> Result<()> { let table_scan = test_table_scan().unwrap(); let plan = LogicalPlanBuilder::from(table_scan) - .sort(vec![col("a"), col("a"), col("b"), col("c")])? + .sort_by(vec![col("a"), col("a"), col("b"), col("c")])? .limit(5, Some(10))? .build()?; let expected = "Limit: skip=5, fetch=10\ - \n Sort: test.a, test.b, test.c\ + \n Sort: test.a ASC NULLS LAST, test.b ASC NULLS LAST, test.c ASC NULLS LAST\ \n TableScan: test"; assert_optimized_plan_eq(plan, expected) } diff --git a/datafusion/optimizer/src/eliminate_limit.rs b/datafusion/optimizer/src/eliminate_limit.rs index e48f37a77cd3..2503475bd8df 100644 --- a/datafusion/optimizer/src/eliminate_limit.rs +++ b/datafusion/optimizer/src/eliminate_limit.rs @@ -182,14 +182,14 @@ mod tests { let plan = LogicalPlanBuilder::from(table_scan) .aggregate(vec![col("a")], vec![sum(col("b"))])? .limit(0, Some(2))? - .sort(vec![col("a")])? + .sort_by(vec![col("a")])? .limit(2, Some(1))? .build()?; // After remove global-state, we don't record the parent // So, bottom don't know parent info, so can't eliminate. let expected = "Limit: skip=2, fetch=1\ - \n Sort: test.a, fetch=3\ + \n Sort: test.a ASC NULLS LAST, fetch=3\ \n Limit: skip=0, fetch=2\ \n Aggregate: groupBy=[[test.a]], aggr=[[sum(test.b)]]\ \n TableScan: test"; @@ -202,12 +202,12 @@ mod tests { let plan = LogicalPlanBuilder::from(table_scan) .aggregate(vec![col("a")], vec![sum(col("b"))])? .limit(0, Some(2))? - .sort(vec![col("a")])? + .sort_by(vec![col("a")])? .limit(0, Some(1))? .build()?; let expected = "Limit: skip=0, fetch=1\ - \n Sort: test.a\ + \n Sort: test.a ASC NULLS LAST\ \n Limit: skip=0, fetch=2\ \n Aggregate: groupBy=[[test.a]], aggr=[[sum(test.b)]]\ \n TableScan: test"; @@ -220,12 +220,12 @@ mod tests { let plan = LogicalPlanBuilder::from(table_scan) .aggregate(vec![col("a")], vec![sum(col("b"))])? .limit(2, Some(1))? - .sort(vec![col("a")])? + .sort_by(vec![col("a")])? .limit(3, Some(1))? .build()?; let expected = "Limit: skip=3, fetch=1\ - \n Sort: test.a\ + \n Sort: test.a ASC NULLS LAST\ \n Limit: skip=2, fetch=1\ \n Aggregate: groupBy=[[test.a]], aggr=[[sum(test.b)]]\ \n TableScan: test"; diff --git a/datafusion/optimizer/src/push_down_filter.rs b/datafusion/optimizer/src/push_down_filter.rs index 82149a087e63..33a58a810b08 100644 --- a/datafusion/optimizer/src/push_down_filter.rs +++ b/datafusion/optimizer/src/push_down_filter.rs @@ -284,8 +284,7 @@ fn can_evaluate_as_join_condition(predicate: &Expr) -> Result { | Expr::TryCast(_) | Expr::InList { .. } | Expr::ScalarFunction(_) => Ok(TreeNodeRecursion::Continue), - Expr::Sort(_) - | Expr::AggregateFunction(_) + Expr::AggregateFunction(_) | Expr::WindowFunction(_) | Expr::Wildcard { .. } | Expr::GroupingSet(_) => internal_err!("Unsupported predicate type"), diff --git a/datafusion/optimizer/src/push_down_limit.rs b/datafusion/optimizer/src/push_down_limit.rs index dff0b61c6b22..ab7880213692 100644 --- a/datafusion/optimizer/src/push_down_limit.rs +++ b/datafusion/optimizer/src/push_down_limit.rs @@ -347,13 +347,13 @@ mod test { let table_scan = test_table_scan()?; let plan = LogicalPlanBuilder::from(table_scan) - .sort(vec![col("a")])? + .sort_by(vec![col("a")])? .limit(0, Some(10))? .build()?; // Should push down limit to sort let expected = "Limit: skip=0, fetch=10\ - \n Sort: test.a, fetch=10\ + \n Sort: test.a ASC NULLS LAST, fetch=10\ \n TableScan: test"; assert_optimized_plan_equal(plan, expected) @@ -364,13 +364,13 @@ mod test { let table_scan = test_table_scan()?; let plan = LogicalPlanBuilder::from(table_scan) - .sort(vec![col("a")])? + .sort_by(vec![col("a")])? .limit(5, Some(10))? .build()?; // Should push down limit to sort let expected = "Limit: skip=5, fetch=10\ - \n Sort: test.a, fetch=15\ + \n Sort: test.a ASC NULLS LAST, fetch=15\ \n TableScan: test"; assert_optimized_plan_equal(plan, expected) diff --git a/datafusion/optimizer/src/simplify_expressions/expr_simplifier.rs b/datafusion/optimizer/src/simplify_expressions/expr_simplifier.rs index 7129ceb0fea1..f299d4542c36 100644 --- a/datafusion/optimizer/src/simplify_expressions/expr_simplifier.rs +++ b/datafusion/optimizer/src/simplify_expressions/expr_simplifier.rs @@ -591,7 +591,6 @@ impl<'a> ConstEvaluator<'a> { | Expr::InSubquery(_) | Expr::ScalarSubquery(_) | Expr::WindowFunction { .. } - | Expr::Sort { .. } | Expr::GroupingSet(_) | Expr::Wildcard { .. } | Expr::Placeholder(_) => false, diff --git a/datafusion/optimizer/src/single_distinct_to_groupby.rs b/datafusion/optimizer/src/single_distinct_to_groupby.rs index 30cae17eaf9f..dd82b056d0a6 100644 --- a/datafusion/optimizer/src/single_distinct_to_groupby.rs +++ b/datafusion/optimizer/src/single_distinct_to_groupby.rs @@ -624,14 +624,14 @@ mod tests { vec![col("a")], false, None, - Some(vec![col("a")]), + Some(vec![col("a").sort(true, false)]), None, )); let plan = LogicalPlanBuilder::from(table_scan) .aggregate(vec![col("c")], vec![expr, count_distinct(col("b"))])? .build()?; // Do nothing - let expected = "Aggregate: groupBy=[[test.c]], aggr=[[sum(test.a) ORDER BY [test.a], count(DISTINCT test.b)]] [c:UInt32, sum(test.a) ORDER BY [test.a]:UInt64;N, count(DISTINCT test.b):Int64]\ + let expected = "Aggregate: groupBy=[[test.c]], aggr=[[sum(test.a) ORDER BY [test.a ASC NULLS LAST], count(DISTINCT test.b)]] [c:UInt32, sum(test.a) ORDER BY [test.a ASC NULLS LAST]:UInt64;N, count(DISTINCT test.b):Int64]\ \n TableScan: test [a:UInt32, b:UInt32, c:UInt32]"; assert_optimized_plan_equal(plan, expected) diff --git a/datafusion/proto/proto/datafusion.proto b/datafusion/proto/proto/datafusion.proto index 826992e132ba..19759a897068 100644 --- a/datafusion/proto/proto/datafusion.proto +++ b/datafusion/proto/proto/datafusion.proto @@ -75,6 +75,10 @@ message LogicalExprNodeCollection { repeated LogicalExprNode logical_expr_nodes = 1; } +message SortExprNodeCollection { + repeated SortExprNode sort_expr_nodes = 1; +} + message ListingTableScanNode { reserved 1; // was string table_name TableReference table_name = 14; @@ -92,7 +96,7 @@ message ListingTableScanNode { datafusion_common.AvroFormat avro = 12; datafusion_common.NdJsonFormat json = 15; } - repeated LogicalExprNodeCollection file_sort_order = 13; + repeated SortExprNodeCollection file_sort_order = 13; } message ViewTableScanNode { @@ -129,7 +133,7 @@ message SelectionNode { message SortNode { LogicalPlanNode input = 1; - repeated LogicalExprNode expr = 2; + repeated SortExprNode expr = 2; // Maximum number of highest/lowest rows to fetch; negative means no limit int64 fetch = 3; } @@ -160,7 +164,7 @@ message CreateExternalTableNode { repeated string table_partition_cols = 5; bool if_not_exists = 6; string definition = 7; - repeated LogicalExprNodeCollection order_exprs = 10; + repeated SortExprNodeCollection order_exprs = 10; bool unbounded = 11; map options = 8; datafusion_common.Constraints constraints = 12; @@ -245,7 +249,7 @@ message DistinctNode { message DistinctOnNode { repeated LogicalExprNode on_expr = 1; repeated LogicalExprNode select_expr = 2; - repeated LogicalExprNode sort_expr = 3; + repeated SortExprNode sort_expr = 3; LogicalPlanNode input = 4; } @@ -320,7 +324,6 @@ message LogicalExprNode { BetweenNode between = 9; CaseNode case_ = 10; CastNode cast = 11; - SortExprNode sort = 12; NegativeNode negative = 13; InListNode in_list = 14; Wildcard wildcard = 15; @@ -470,7 +473,7 @@ message AggregateUDFExprNode { repeated LogicalExprNode args = 2; bool distinct = 5; LogicalExprNode filter = 3; - repeated LogicalExprNode order_by = 4; + repeated SortExprNode order_by = 4; optional bytes fun_definition = 6; } @@ -503,7 +506,7 @@ message WindowExprNode { } LogicalExprNode expr = 4; repeated LogicalExprNode partition_by = 5; - repeated LogicalExprNode order_by = 6; + repeated SortExprNode order_by = 6; // repeated LogicalExprNode filter = 7; WindowFrame window_frame = 8; optional bytes fun_definition = 10; diff --git a/datafusion/proto/src/generated/pbjson.rs b/datafusion/proto/src/generated/pbjson.rs index b4d63798f080..cff58d3ddc4a 100644 --- a/datafusion/proto/src/generated/pbjson.rs +++ b/datafusion/proto/src/generated/pbjson.rs @@ -9291,9 +9291,6 @@ impl serde::Serialize for LogicalExprNode { logical_expr_node::ExprType::Cast(v) => { struct_ser.serialize_field("cast", v)?; } - logical_expr_node::ExprType::Sort(v) => { - struct_ser.serialize_field("sort", v)?; - } logical_expr_node::ExprType::Negative(v) => { struct_ser.serialize_field("negative", v)?; } @@ -9384,7 +9381,6 @@ impl<'de> serde::Deserialize<'de> for LogicalExprNode { "case_", "case", "cast", - "sort", "negative", "in_list", "inList", @@ -9433,7 +9429,6 @@ impl<'de> serde::Deserialize<'de> for LogicalExprNode { Between, Case, Cast, - Sort, Negative, InList, Wildcard, @@ -9486,7 +9481,6 @@ impl<'de> serde::Deserialize<'de> for LogicalExprNode { "between" => Ok(GeneratedField::Between), "case" | "case_" => Ok(GeneratedField::Case), "cast" => Ok(GeneratedField::Cast), - "sort" => Ok(GeneratedField::Sort), "negative" => Ok(GeneratedField::Negative), "inList" | "in_list" => Ok(GeneratedField::InList), "wildcard" => Ok(GeneratedField::Wildcard), @@ -9598,13 +9592,6 @@ impl<'de> serde::Deserialize<'de> for LogicalExprNode { return Err(serde::de::Error::duplicate_field("cast")); } expr_type__ = map_.next_value::<::std::option::Option<_>>()?.map(logical_expr_node::ExprType::Cast) -; - } - GeneratedField::Sort => { - if expr_type__.is_some() { - return Err(serde::de::Error::duplicate_field("sort")); - } - expr_type__ = map_.next_value::<::std::option::Option<_>>()?.map(logical_expr_node::ExprType::Sort) ; } GeneratedField::Negative => { @@ -17947,6 +17934,98 @@ impl<'de> serde::Deserialize<'de> for SortExprNode { deserializer.deserialize_struct("datafusion.SortExprNode", FIELDS, GeneratedVisitor) } } +impl serde::Serialize for SortExprNodeCollection { + #[allow(deprecated)] + fn serialize(&self, serializer: S) -> std::result::Result + where + S: serde::Serializer, + { + use serde::ser::SerializeStruct; + let mut len = 0; + if !self.sort_expr_nodes.is_empty() { + len += 1; + } + let mut struct_ser = serializer.serialize_struct("datafusion.SortExprNodeCollection", len)?; + if !self.sort_expr_nodes.is_empty() { + struct_ser.serialize_field("sortExprNodes", &self.sort_expr_nodes)?; + } + struct_ser.end() + } +} +impl<'de> serde::Deserialize<'de> for SortExprNodeCollection { + #[allow(deprecated)] + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + const FIELDS: &[&str] = &[ + "sort_expr_nodes", + "sortExprNodes", + ]; + + #[allow(clippy::enum_variant_names)] + enum GeneratedField { + SortExprNodes, + } + impl<'de> serde::Deserialize<'de> for GeneratedField { + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + struct GeneratedVisitor; + + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = GeneratedField; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(formatter, "expected one of: {:?}", &FIELDS) + } + + #[allow(unused_variables)] + fn visit_str(self, value: &str) -> std::result::Result + where + E: serde::de::Error, + { + match value { + "sortExprNodes" | "sort_expr_nodes" => Ok(GeneratedField::SortExprNodes), + _ => Err(serde::de::Error::unknown_field(value, FIELDS)), + } + } + } + deserializer.deserialize_identifier(GeneratedVisitor) + } + } + struct GeneratedVisitor; + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = SortExprNodeCollection; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + formatter.write_str("struct datafusion.SortExprNodeCollection") + } + + fn visit_map(self, mut map_: V) -> std::result::Result + where + V: serde::de::MapAccess<'de>, + { + let mut sort_expr_nodes__ = None; + while let Some(k) = map_.next_key()? { + match k { + GeneratedField::SortExprNodes => { + if sort_expr_nodes__.is_some() { + return Err(serde::de::Error::duplicate_field("sortExprNodes")); + } + sort_expr_nodes__ = Some(map_.next_value()?); + } + } + } + Ok(SortExprNodeCollection { + sort_expr_nodes: sort_expr_nodes__.unwrap_or_default(), + }) + } + } + deserializer.deserialize_struct("datafusion.SortExprNodeCollection", FIELDS, GeneratedVisitor) + } +} impl serde::Serialize for SortNode { #[allow(deprecated)] fn serialize(&self, serializer: S) -> std::result::Result diff --git a/datafusion/proto/src/generated/prost.rs b/datafusion/proto/src/generated/prost.rs index 875d2af75dd7..2ce8004e3248 100644 --- a/datafusion/proto/src/generated/prost.rs +++ b/datafusion/proto/src/generated/prost.rs @@ -97,6 +97,12 @@ pub struct LogicalExprNodeCollection { } #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] +pub struct SortExprNodeCollection { + #[prost(message, repeated, tag = "1")] + pub sort_expr_nodes: ::prost::alloc::vec::Vec, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] pub struct ListingTableScanNode { #[prost(message, optional, tag = "14")] pub table_name: ::core::option::Option, @@ -117,7 +123,7 @@ pub struct ListingTableScanNode { #[prost(uint32, tag = "9")] pub target_partitions: u32, #[prost(message, repeated, tag = "13")] - pub file_sort_order: ::prost::alloc::vec::Vec, + pub file_sort_order: ::prost::alloc::vec::Vec, #[prost(oneof = "listing_table_scan_node::FileFormatType", tags = "10, 11, 12, 15")] pub file_format_type: ::core::option::Option< listing_table_scan_node::FileFormatType, @@ -200,7 +206,7 @@ pub struct SortNode { #[prost(message, optional, boxed, tag = "1")] pub input: ::core::option::Option<::prost::alloc::boxed::Box>, #[prost(message, repeated, tag = "2")] - pub expr: ::prost::alloc::vec::Vec, + pub expr: ::prost::alloc::vec::Vec, /// Maximum number of highest/lowest rows to fetch; negative means no limit #[prost(int64, tag = "3")] pub fetch: i64, @@ -256,7 +262,7 @@ pub struct CreateExternalTableNode { #[prost(string, tag = "7")] pub definition: ::prost::alloc::string::String, #[prost(message, repeated, tag = "10")] - pub order_exprs: ::prost::alloc::vec::Vec, + pub order_exprs: ::prost::alloc::vec::Vec, #[prost(bool, tag = "11")] pub unbounded: bool, #[prost(map = "string, string", tag = "8")] @@ -402,7 +408,7 @@ pub struct DistinctOnNode { #[prost(message, repeated, tag = "2")] pub select_expr: ::prost::alloc::vec::Vec, #[prost(message, repeated, tag = "3")] - pub sort_expr: ::prost::alloc::vec::Vec, + pub sort_expr: ::prost::alloc::vec::Vec, #[prost(message, optional, boxed, tag = "4")] pub input: ::core::option::Option<::prost::alloc::boxed::Box>, } @@ -488,7 +494,7 @@ pub struct SubqueryAliasNode { pub struct LogicalExprNode { #[prost( oneof = "logical_expr_node::ExprType", - tags = "1, 2, 3, 4, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 17, 18, 19, 20, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35" + tags = "1, 2, 3, 4, 6, 7, 8, 9, 10, 11, 13, 14, 15, 17, 18, 19, 20, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35" )] pub expr_type: ::core::option::Option, } @@ -521,8 +527,6 @@ pub mod logical_expr_node { Case(::prost::alloc::boxed::Box), #[prost(message, tag = "11")] Cast(::prost::alloc::boxed::Box), - #[prost(message, tag = "12")] - Sort(::prost::alloc::boxed::Box), #[prost(message, tag = "13")] Negative(::prost::alloc::boxed::Box), #[prost(message, tag = "14")] @@ -740,7 +744,7 @@ pub struct AggregateUdfExprNode { #[prost(message, optional, boxed, tag = "3")] pub filter: ::core::option::Option<::prost::alloc::boxed::Box>, #[prost(message, repeated, tag = "4")] - pub order_by: ::prost::alloc::vec::Vec, + pub order_by: ::prost::alloc::vec::Vec, #[prost(bytes = "vec", optional, tag = "6")] pub fun_definition: ::core::option::Option<::prost::alloc::vec::Vec>, } @@ -762,7 +766,7 @@ pub struct WindowExprNode { #[prost(message, repeated, tag = "5")] pub partition_by: ::prost::alloc::vec::Vec, #[prost(message, repeated, tag = "6")] - pub order_by: ::prost::alloc::vec::Vec, + pub order_by: ::prost::alloc::vec::Vec, /// repeated LogicalExprNode filter = 7; #[prost(message, optional, tag = "8")] pub window_frame: ::core::option::Option, @@ -869,8 +873,8 @@ pub struct TryCastNode { #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct SortExprNode { - #[prost(message, optional, boxed, tag = "1")] - pub expr: ::core::option::Option<::prost::alloc::boxed::Box>, + #[prost(message, optional, tag = "1")] + pub expr: ::core::option::Option, #[prost(bool, tag = "2")] pub asc: bool, #[prost(bool, tag = "3")] diff --git a/datafusion/proto/src/logical_plan/from_proto.rs b/datafusion/proto/src/logical_plan/from_proto.rs index acda1298dd80..3ba1cb945e9c 100644 --- a/datafusion/proto/src/logical_plan/from_proto.rs +++ b/datafusion/proto/src/logical_plan/from_proto.rs @@ -22,11 +22,11 @@ use datafusion_common::{ exec_datafusion_err, internal_err, plan_datafusion_err, Result, ScalarValue, TableReference, UnnestOptions, }; -use datafusion_expr::expr::{Alias, Placeholder}; +use datafusion_expr::expr::{Alias, Placeholder, Sort}; use datafusion_expr::expr::{Unnest, WildcardOptions}; use datafusion_expr::ExprFunctionExt; use datafusion_expr::{ - expr::{self, InList, Sort, WindowFunction}, + expr::{self, InList, WindowFunction}, logical_plan::{PlanType, StringifiedPlan}, Between, BinaryExpr, BuiltInWindowFunction, Case, Cast, Expr, GroupingSet, GroupingSet::GroupingSets, @@ -267,7 +267,7 @@ pub fn parse_expr( .as_ref() .ok_or_else(|| Error::required("window_function"))?; let partition_by = parse_exprs(&expr.partition_by, registry, codec)?; - let mut order_by = parse_exprs(&expr.order_by, registry, codec)?; + let mut order_by = parse_sorts(&expr.order_by, registry, codec)?; let window_frame = expr .window_frame .as_ref() @@ -524,16 +524,6 @@ pub fn parse_expr( let data_type = cast.arrow_type.as_ref().required("arrow_type")?; Ok(Expr::TryCast(TryCast::new(expr, data_type))) } - ExprType::Sort(sort) => Ok(Expr::Sort(Sort::new( - Box::new(parse_required_expr( - sort.expr.as_deref(), - registry, - "expr", - codec, - )?), - sort.asc, - sort.nulls_first, - ))), ExprType::Negative(negative) => Ok(Expr::Negative(Box::new( parse_required_expr(negative.expr.as_deref(), registry, "expr", codec)?, ))), @@ -588,7 +578,7 @@ pub fn parse_expr( parse_optional_expr(pb.filter.as_deref(), registry, codec)?.map(Box::new), match pb.order_by.len() { 0 => None, - _ => Some(parse_exprs(&pb.order_by, registry, codec)?), + _ => Some(parse_sorts(&pb.order_by, registry, codec)?), }, None, ))) @@ -635,6 +625,37 @@ where Ok(res) } +pub fn parse_sorts<'a, I>( + protos: I, + registry: &dyn FunctionRegistry, + codec: &dyn LogicalExtensionCodec, +) -> Result, Error> +where + I: IntoIterator, +{ + protos + .into_iter() + .map(|sort| parse_sort(sort, registry, codec)) + .collect::, Error>>() +} + +pub fn parse_sort( + sort: &protobuf::SortExprNode, + registry: &dyn FunctionRegistry, + codec: &dyn LogicalExtensionCodec, +) -> Result { + Ok(Sort::new( + Box::new(parse_required_expr( + sort.expr.as_ref(), + registry, + "expr", + codec, + )?), + sort.asc, + sort.nulls_first, + )) +} + /// Parse an optional escape_char for Like, ILike, SimilarTo fn parse_escape_char(s: &str) -> Result> { match s.len() { diff --git a/datafusion/proto/src/logical_plan/mod.rs b/datafusion/proto/src/logical_plan/mod.rs index 67977b1795a6..bf5394ec01de 100644 --- a/datafusion/proto/src/logical_plan/mod.rs +++ b/datafusion/proto/src/logical_plan/mod.rs @@ -20,7 +20,7 @@ use std::fmt::Debug; use std::sync::Arc; use crate::protobuf::logical_plan_node::LogicalPlanType::CustomScan; -use crate::protobuf::{CustomTableScanNode, LogicalExprNodeCollection}; +use crate::protobuf::{CustomTableScanNode, SortExprNodeCollection}; use crate::{ convert_required, into_required, protobuf::{ @@ -62,11 +62,13 @@ use datafusion_expr::{ EmptyRelation, Extension, Join, JoinConstraint, Limit, Prepare, Projection, Repartition, Sort, SubqueryAlias, TableScan, Values, Window, }, - DistinctOn, DropView, Expr, LogicalPlan, LogicalPlanBuilder, ScalarUDF, WindowUDF, + DistinctOn, DropView, Expr, LogicalPlan, LogicalPlanBuilder, ScalarUDF, SortExpr, + WindowUDF, }; use datafusion_expr::{AggregateUDF, Unnest}; use self::to_proto::{serialize_expr, serialize_exprs}; +use crate::logical_plan::to_proto::serialize_sorts; use prost::bytes::BufMut; use prost::Message; @@ -347,8 +349,8 @@ impl AsLogicalPlan for LogicalPlanNode { let mut all_sort_orders = vec![]; for order in &scan.file_sort_order { - all_sort_orders.push(from_proto::parse_exprs( - &order.logical_expr_nodes, + all_sort_orders.push(from_proto::parse_sorts( + &order.sort_expr_nodes, ctx, extension_codec, )?) @@ -476,8 +478,8 @@ impl AsLogicalPlan for LogicalPlanNode { LogicalPlanType::Sort(sort) => { let input: LogicalPlan = into_logical_plan!(sort.input, ctx, extension_codec)?; - let sort_expr: Vec = - from_proto::parse_exprs(&sort.expr, ctx, extension_codec)?; + let sort_expr: Vec = + from_proto::parse_sorts(&sort.expr, ctx, extension_codec)?; LogicalPlanBuilder::from(input).sort(sort_expr)?.build() } LogicalPlanType::Repartition(repartition) => { @@ -536,8 +538,8 @@ impl AsLogicalPlan for LogicalPlanNode { let mut order_exprs = vec![]; for expr in &create_extern_table.order_exprs { - order_exprs.push(from_proto::parse_exprs( - &expr.logical_expr_nodes, + order_exprs.push(from_proto::parse_sorts( + &expr.sort_expr_nodes, ctx, extension_codec, )?); @@ -772,7 +774,7 @@ impl AsLogicalPlan for LogicalPlanNode { )?; let sort_expr = match distinct_on.sort_expr.len() { 0 => None, - _ => Some(from_proto::parse_exprs( + _ => Some(from_proto::parse_sorts( &distinct_on.sort_expr, ctx, extension_codec, @@ -981,10 +983,10 @@ impl AsLogicalPlan for LogicalPlanNode { let options = listing_table.options(); - let mut exprs_vec: Vec = vec![]; + let mut exprs_vec: Vec = vec![]; for order in &options.file_sort_order { - let expr_vec = LogicalExprNodeCollection { - logical_expr_nodes: serialize_exprs(order, extension_codec)?, + let expr_vec = SortExprNodeCollection { + sort_expr_nodes: serialize_sorts(order, extension_codec)?, }; exprs_vec.push(expr_vec); } @@ -1114,7 +1116,7 @@ impl AsLogicalPlan for LogicalPlanNode { )?; let sort_expr = match sort_expr { None => vec![], - Some(sort_expr) => serialize_exprs(sort_expr, extension_codec)?, + Some(sort_expr) => serialize_sorts(sort_expr, extension_codec)?, }; Ok(protobuf::LogicalPlanNode { logical_plan_type: Some(LogicalPlanType::DistinctOn(Box::new( @@ -1258,13 +1260,13 @@ impl AsLogicalPlan for LogicalPlanNode { input.as_ref(), extension_codec, )?; - let selection_expr: Vec = - serialize_exprs(expr, extension_codec)?; + let sort_expr: Vec = + serialize_sorts(expr, extension_codec)?; Ok(protobuf::LogicalPlanNode { logical_plan_type: Some(LogicalPlanType::Sort(Box::new( protobuf::SortNode { input: Some(Box::new(input)), - expr: selection_expr, + expr: sort_expr, fetch: fetch.map(|f| f as i64).unwrap_or(-1i64), }, ))), @@ -1334,10 +1336,10 @@ impl AsLogicalPlan for LogicalPlanNode { column_defaults, }, )) => { - let mut converted_order_exprs: Vec = vec![]; + let mut converted_order_exprs: Vec = vec![]; for order in order_exprs { - let temp = LogicalExprNodeCollection { - logical_expr_nodes: serialize_exprs(order, extension_codec)?, + let temp = SortExprNodeCollection { + sort_expr_nodes: serialize_sorts(order, extension_codec)?, }; converted_order_exprs.push(temp); } diff --git a/datafusion/proto/src/logical_plan/to_proto.rs b/datafusion/proto/src/logical_plan/to_proto.rs index bb7bf84a3387..b937c03f79d9 100644 --- a/datafusion/proto/src/logical_plan/to_proto.rs +++ b/datafusion/proto/src/logical_plan/to_proto.rs @@ -22,12 +22,12 @@ use datafusion_common::{TableReference, UnnestOptions}; use datafusion_expr::expr::{ self, Alias, Between, BinaryExpr, Cast, GroupingSet, InList, Like, Placeholder, - ScalarFunction, Sort, Unnest, + ScalarFunction, Unnest, }; use datafusion_expr::{ logical_plan::PlanType, logical_plan::StringifiedPlan, BuiltInWindowFunction, Expr, - JoinConstraint, JoinType, TryCast, WindowFrame, WindowFrameBound, WindowFrameUnits, - WindowFunctionDefinition, + JoinConstraint, JoinType, SortExpr, TryCast, WindowFrame, WindowFrameBound, + WindowFrameUnits, WindowFunctionDefinition, }; use crate::protobuf::{ @@ -343,7 +343,7 @@ pub fn serialize_expr( None }; let partition_by = serialize_exprs(partition_by, codec)?; - let order_by = serialize_exprs(order_by, codec)?; + let order_by = serialize_sorts(order_by, codec)?; let window_frame: Option = Some(window_frame.try_into()?); @@ -380,7 +380,7 @@ pub fn serialize_expr( None => None, }, order_by: match order_by { - Some(e) => serialize_exprs(e, codec)?, + Some(e) => serialize_sorts(e, codec)?, None => vec![], }, fun_definition: (!buf.is_empty()).then_some(buf), @@ -537,20 +537,6 @@ pub fn serialize_expr( expr_type: Some(ExprType::TryCast(expr)), } } - Expr::Sort(Sort { - expr, - asc, - nulls_first, - }) => { - let expr = Box::new(protobuf::SortExprNode { - expr: Some(Box::new(serialize_expr(expr.as_ref(), codec)?)), - asc: *asc, - nulls_first: *nulls_first, - }); - protobuf::LogicalExprNode { - expr_type: Some(ExprType::Sort(expr)), - } - } Expr::Negative(expr) => { let expr = Box::new(protobuf::NegativeNode { expr: Some(Box::new(serialize_expr(expr.as_ref(), codec)?)), @@ -635,6 +621,30 @@ pub fn serialize_expr( Ok(expr_node) } +pub fn serialize_sorts<'a, I>( + sorts: I, + codec: &dyn LogicalExtensionCodec, +) -> Result, Error> +where + I: IntoIterator, +{ + sorts + .into_iter() + .map(|sort| { + let SortExpr { + expr, + asc, + nulls_first, + } = sort; + Ok(protobuf::SortExprNode { + expr: Some(serialize_expr(expr.as_ref(), codec)?), + asc: *asc, + nulls_first: *nulls_first, + }) + }) + .collect::, Error>>() +} + impl From for protobuf::TableReference { fn from(t: TableReference) -> Self { use protobuf::table_reference::TableReferenceEnum; diff --git a/datafusion/proto/tests/cases/roundtrip_logical_plan.rs b/datafusion/proto/tests/cases/roundtrip_logical_plan.rs index 94ac913e1968..e174d1b50713 100644 --- a/datafusion/proto/tests/cases/roundtrip_logical_plan.rs +++ b/datafusion/proto/tests/cases/roundtrip_logical_plan.rs @@ -59,7 +59,7 @@ use datafusion_common::{ use datafusion_expr::dml::CopyTo; use datafusion_expr::expr::{ self, Between, BinaryExpr, Case, Cast, GroupingSet, InList, Like, ScalarFunction, - Sort, Unnest, WildcardOptions, + Unnest, WildcardOptions, }; use datafusion_expr::logical_plan::{Extension, UserDefinedLogicalNodeCore}; use datafusion_expr::{ @@ -1937,14 +1937,6 @@ fn roundtrip_try_cast() { roundtrip_expr_test(test_expr, ctx); } -#[test] -fn roundtrip_sort_expr() { - let test_expr = Expr::Sort(Sort::new(Box::new(lit(1.0_f32)), true, true)); - - let ctx = SessionContext::new(); - roundtrip_expr_test(test_expr, ctx); -} - #[test] fn roundtrip_negative() { let test_expr = Expr::Negative(Box::new(lit(1.0_f32))); diff --git a/datafusion/sql/src/expr/function.rs b/datafusion/sql/src/expr/function.rs index 71e40c20b80a..9c768eb73c2e 100644 --- a/datafusion/sql/src/expr/function.rs +++ b/datafusion/sql/src/expr/function.rs @@ -282,22 +282,17 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { let func_deps = schema.functional_dependencies(); // Find whether ties are possible in the given ordering let is_ordering_strict = order_by.iter().find_map(|orderby_expr| { - if let Expr::Sort(sort_expr) = orderby_expr { - if let Expr::Column(col) = sort_expr.expr.as_ref() { - let idx = schema.index_of_column(col).ok()?; - return if func_deps.iter().any(|dep| { - dep.source_indices == vec![idx] - && dep.mode == Dependency::Single - }) { - Some(true) - } else { - Some(false) - }; - } - Some(false) - } else { - panic!("order_by expression must be of type Sort"); + if let Expr::Column(col) = orderby_expr.expr.as_ref() { + let idx = schema.index_of_column(col).ok()?; + return if func_deps.iter().any(|dep| { + dep.source_indices == vec![idx] && dep.mode == Dependency::Single + }) { + Some(true) + } else { + Some(false) + }; } + Some(false) }); let window_frame = window diff --git a/datafusion/sql/src/expr/order_by.rs b/datafusion/sql/src/expr/order_by.rs index 7fb32f714cfa..cdaa787cedd0 100644 --- a/datafusion/sql/src/expr/order_by.rs +++ b/datafusion/sql/src/expr/order_by.rs @@ -20,7 +20,7 @@ use datafusion_common::{ not_impl_err, plan_datafusion_err, plan_err, Column, DFSchema, Result, }; use datafusion_expr::expr::Sort; -use datafusion_expr::Expr; +use datafusion_expr::{Expr, SortExpr}; use sqlparser::ast::{Expr as SQLExpr, OrderByExpr, Value}; impl<'a, S: ContextProvider> SqlToRel<'a, S> { @@ -44,7 +44,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { planner_context: &mut PlannerContext, literal_to_column: bool, additional_schema: Option<&DFSchema>, - ) -> Result> { + ) -> Result> { if exprs.is_empty() { return Ok(vec![]); } @@ -99,13 +99,13 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { } }; let asc = asc.unwrap_or(true); - expr_vec.push(Expr::Sort(Sort::new( + expr_vec.push(Sort::new( Box::new(expr), asc, // when asc is true, by default nulls last to be consistent with postgres // postgres rule: https://www.postgresql.org/docs/current/queries-order.html nulls_first.unwrap_or(!asc), - ))) + )) } Ok(expr_vec) } diff --git a/datafusion/sql/src/query.rs b/datafusion/sql/src/query.rs index ba2b41bb6ecf..71328cfd018c 100644 --- a/datafusion/sql/src/query.rs +++ b/datafusion/sql/src/query.rs @@ -20,6 +20,7 @@ use std::sync::Arc; use crate::planner::{ContextProvider, PlannerContext, SqlToRel}; use datafusion_common::{not_impl_err, plan_err, Constraints, Result, ScalarValue}; +use datafusion_expr::expr::Sort; use datafusion_expr::{ CreateMemoryTable, DdlStatement, Distinct, Expr, LogicalPlan, LogicalPlanBuilder, Operator, @@ -119,7 +120,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { pub(super) fn order_by( &self, plan: LogicalPlan, - order_by: Vec, + order_by: Vec, ) -> Result { if order_by.is_empty() { return Ok(plan); diff --git a/datafusion/sql/src/select.rs b/datafusion/sql/src/select.rs index 384893bfa94c..8a26671fcb6c 100644 --- a/datafusion/sql/src/select.rs +++ b/datafusion/sql/src/select.rs @@ -31,7 +31,7 @@ use datafusion_common::UnnestOptions; use datafusion_common::{not_impl_err, plan_err, DataFusionError, Result}; use datafusion_expr::expr::{Alias, PlannedReplaceSelectItem, WildcardOptions}; use datafusion_expr::expr_rewriter::{ - normalize_col, normalize_col_with_schemas_and_ambiguity_check, normalize_cols, + normalize_col, normalize_col_with_schemas_and_ambiguity_check, normalize_sorts, }; use datafusion_expr::utils::{ expr_as_column_expr, expr_to_columns, find_aggregate_exprs, find_window_exprs, @@ -107,7 +107,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { true, Some(base_plan.schema().as_ref()), )?; - let order_by_rex = normalize_cols(order_by_rex, &projected_plan)?; + let order_by_rex = normalize_sorts(order_by_rex, &projected_plan)?; // this alias map is resolved and looked up in both having exprs and group by exprs let alias_map = extract_aliases(&select_exprs); diff --git a/datafusion/sql/src/statement.rs b/datafusion/sql/src/statement.rs index e75a96e78d48..3dfc379b039a 100644 --- a/datafusion/sql/src/statement.rs +++ b/datafusion/sql/src/statement.rs @@ -48,9 +48,10 @@ use datafusion_expr::{ CreateIndex as PlanCreateIndex, CreateMemoryTable, CreateView, DescribeTable, DmlStatement, DropCatalogSchema, DropFunction, DropTable, DropView, EmptyRelation, Explain, Expr, ExprSchemable, Filter, LogicalPlan, LogicalPlanBuilder, - OperateFunctionArg, PlanType, Prepare, SetVariable, Statement as PlanStatement, - ToStringifiedPlan, TransactionAccessMode, TransactionConclusion, TransactionEnd, - TransactionIsolationLevel, TransactionStart, Volatility, WriteOp, + OperateFunctionArg, PlanType, Prepare, SetVariable, SortExpr, + Statement as PlanStatement, ToStringifiedPlan, TransactionAccessMode, + TransactionConclusion, TransactionEnd, TransactionIsolationLevel, TransactionStart, + Volatility, WriteOp, }; use sqlparser::ast; use sqlparser::ast::{ @@ -952,7 +953,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { order_exprs: Vec, schema: &DFSchemaRef, planner_context: &mut PlannerContext, - ) -> Result>> { + ) -> Result>> { // Ask user to provide a schema if schema is empty. if !order_exprs.is_empty() && schema.fields().is_empty() { return plan_err!( @@ -966,8 +967,8 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { let expr_vec = self.order_by_to_sort_expr(expr, schema, planner_context, true, None)?; // Verify that columns of all SortExprs exist in the schema: - for expr in expr_vec.iter() { - for column in expr.column_refs().iter() { + for sort in expr_vec.iter() { + for column in sort.expr.column_refs().iter() { if !schema.has_column(column) { // Return an error if any column is not in the schema: return plan_err!("Column {column} is not in schema"); diff --git a/datafusion/sql/src/unparser/expr.rs b/datafusion/sql/src/unparser/expr.rs index 0dbcba162bc0..9a3f139fdee8 100644 --- a/datafusion/sql/src/unparser/expr.rs +++ b/datafusion/sql/src/unparser/expr.rs @@ -15,8 +15,6 @@ // specific language governing permissions and limitations // under the License. -use core::fmt; - use datafusion_expr::ScalarUDF; use sqlparser::ast::Value::SingleQuotedString; use sqlparser::ast::{ @@ -24,7 +22,7 @@ use sqlparser::ast::{ ObjectName, TimezoneInfo, UnaryOperator, }; use std::sync::Arc; -use std::{fmt::Display, vec}; +use std::vec; use super::dialect::{DateFieldExtractStyle, IntervalStyle}; use super::Unparser; @@ -46,33 +44,6 @@ use datafusion_expr::{ Between, BinaryExpr, Case, Cast, Expr, GroupingSet, Like, Operator, TryCast, }; -/// DataFusion's Exprs can represent either an `Expr` or an `OrderByExpr` -pub enum Unparsed { - // SQL Expression - Expr(ast::Expr), - // SQL ORDER BY expression (e.g. `col ASC NULLS FIRST`) - OrderByExpr(ast::OrderByExpr), -} - -impl Unparsed { - pub fn into_order_by_expr(self) -> Result { - if let Unparsed::OrderByExpr(order_by_expr) = self { - Ok(order_by_expr) - } else { - internal_err!("Expected Sort expression to be converted an OrderByExpr") - } - } -} - -impl Display for Unparsed { - fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { - match self { - Unparsed::Expr(expr) => write!(f, "{}", expr), - Unparsed::OrderByExpr(order_by_expr) => write!(f, "{}", order_by_expr), - } - } -} - /// Convert a DataFusion [`Expr`] to [`ast::Expr`] /// /// This function is the opposite of [`SqlToRel::sql_to_expr`] and can be used @@ -106,13 +77,9 @@ pub fn expr_to_sql(expr: &Expr) -> Result { unparser.expr_to_sql(expr) } -/// Convert a DataFusion [`Expr`] to [`Unparsed`] -/// -/// This function is similar to expr_to_sql, but it supports converting more [`Expr`] types like -/// `Sort` expressions to `OrderByExpr` expressions. -pub fn expr_to_unparsed(expr: &Expr) -> Result { +pub fn sort_to_sql(sort: &Sort) -> Result { let unparser = Unparser::default(); - unparser.expr_to_unparsed(expr) + unparser.sort_to_sql(sort) } const LOWEST: &BinaryOperator = &BinaryOperator::Or; @@ -286,7 +253,7 @@ impl Unparser<'_> { }; let order_by: Vec = order_by .iter() - .map(|expr| expr_to_unparsed(expr)?.into_order_by_expr()) + .map(sort_to_sql) .collect::>>()?; let start_bound = self.convert_bound(&window_frame.start_bound)?; @@ -413,11 +380,6 @@ impl Unparser<'_> { negated: *negated, }) } - Expr::Sort(Sort { - expr: _, - asc: _, - nulls_first: _, - }) => plan_err!("Sort expression should be handled by expr_to_unparsed"), Expr::IsNull(expr) => { Ok(ast::Expr::IsNull(Box::new(self.expr_to_sql_inner(expr)?))) } @@ -534,36 +496,26 @@ impl Unparser<'_> { } } - /// This function can convert more [`Expr`] types than `expr_to_sql`, - /// returning an [`Unparsed`] like `Sort` expressions to `OrderByExpr` - /// expressions. - pub fn expr_to_unparsed(&self, expr: &Expr) -> Result { - match expr { - Expr::Sort(Sort { - expr, - asc, - nulls_first, - }) => { - let sql_parser_expr = self.expr_to_sql(expr)?; + pub fn sort_to_sql(&self, sort: &Sort) -> Result { + let Sort { + expr, + asc, + nulls_first, + } = sort; + let sql_parser_expr = self.expr_to_sql(expr)?; - let nulls_first = if self.dialect.supports_nulls_first_in_sort() { - Some(*nulls_first) - } else { - None - }; + let nulls_first = if self.dialect.supports_nulls_first_in_sort() { + Some(*nulls_first) + } else { + None + }; - Ok(Unparsed::OrderByExpr(ast::OrderByExpr { - expr: sql_parser_expr, - asc: Some(*asc), - nulls_first, - with_fill: None, - })) - } - _ => { - let sql_parser_expr = self.expr_to_sql(expr)?; - Ok(Unparsed::Expr(sql_parser_expr)) - } - } + Ok(ast::OrderByExpr { + expr: sql_parser_expr, + asc: Some(*asc), + nulls_first, + with_fill: None, + }) } fn scalar_function_to_sql_overrides( @@ -1809,11 +1761,7 @@ mod tests { fun: WindowFunctionDefinition::AggregateUDF(count_udaf()), args: vec![wildcard()], partition_by: vec![], - order_by: vec![Expr::Sort(Sort::new( - Box::new(col("a")), - false, - true, - ))], + order_by: vec![Sort::new(Box::new(col("a")), false, true)], window_frame: WindowFrame::new_bounds( datafusion_expr::WindowFrameUnits::Range, datafusion_expr::WindowFrameBound::Preceding( @@ -1941,24 +1889,6 @@ mod tests { Ok(()) } - #[test] - fn expr_to_unparsed_ok() -> Result<()> { - let tests: Vec<(Expr, &str)> = vec![ - ((col("a") + col("b")).gt(lit(4)), r#"((a + b) > 4)"#), - (col("a").sort(true, true), r#"a ASC NULLS FIRST"#), - ]; - - for (expr, expected) in tests { - let ast = expr_to_unparsed(&expr)?; - - let actual = format!("{}", ast); - - assert_eq!(actual, expected); - } - - Ok(()) - } - #[test] fn custom_dialect_with_identifier_quote_style() -> Result<()> { let dialect = CustomDialectBuilder::new() @@ -2047,7 +1977,7 @@ mod tests { #[test] fn customer_dialect_support_nulls_first_in_ort() -> Result<()> { - let tests: Vec<(Expr, &str, bool)> = vec![ + let tests: Vec<(Sort, &str, bool)> = vec![ (col("a").sort(true, true), r#"a ASC NULLS FIRST"#, true), (col("a").sort(true, true), r#"a ASC"#, false), ]; @@ -2057,7 +1987,7 @@ mod tests { .with_supports_nulls_first_in_sort(supports_nulls_first_in_sort) .build(); let unparser = Unparser::new(&dialect); - let ast = unparser.expr_to_unparsed(&expr)?; + let ast = unparser.sort_to_sql(&expr)?; let actual = format!("{}", ast); diff --git a/datafusion/sql/src/unparser/mod.rs b/datafusion/sql/src/unparser/mod.rs index b2fd32566aa8..83ae64ba238b 100644 --- a/datafusion/sql/src/unparser/mod.rs +++ b/datafusion/sql/src/unparser/mod.rs @@ -29,8 +29,6 @@ pub use plan::plan_to_sql; use self::dialect::{DefaultDialect, Dialect}; pub mod dialect; -pub use expr::Unparsed; - /// Convert a DataFusion [`Expr`] to [`sqlparser::ast::Expr`] /// /// See [`expr_to_sql`] for background. `Unparser` allows greater control of diff --git a/datafusion/sql/src/unparser/plan.rs b/datafusion/sql/src/unparser/plan.rs index 106705c322fc..509c5dd52cd4 100644 --- a/datafusion/sql/src/unparser/plan.rs +++ b/datafusion/sql/src/unparser/plan.rs @@ -15,11 +15,10 @@ // specific language governing permissions and limitations // under the License. -use datafusion_common::{ - internal_err, not_impl_err, plan_err, Column, DataFusionError, Result, -}; +use datafusion_common::{internal_err, not_impl_err, Column, DataFusionError, Result}; use datafusion_expr::{ expr::Alias, Distinct, Expr, JoinConstraint, JoinType, LogicalPlan, Projection, + SortExpr, }; use sqlparser::ast::{self, Ident, SetExpr}; @@ -318,7 +317,7 @@ impl Unparser<'_> { return self.derive(plan, relation); } if let Some(query_ref) = query { - query_ref.order_by(self.sort_to_sql(sort.expr.clone())?); + query_ref.order_by(self.sorts_to_sql(sort.expr.clone())?); } else { return internal_err!( "Sort operator only valid in a statement context." @@ -361,7 +360,7 @@ impl Unparser<'_> { .collect::>>()?; if let Some(sort_expr) = &on.sort_expr { if let Some(query_ref) = query { - query_ref.order_by(self.sort_to_sql(sort_expr.clone())?); + query_ref.order_by(self.sorts_to_sql(sort_expr.clone())?); } else { return internal_err!( "Sort operator only valid in a statement context." @@ -525,14 +524,10 @@ impl Unparser<'_> { } } - fn sort_to_sql(&self, sort_exprs: Vec) -> Result> { + fn sorts_to_sql(&self, sort_exprs: Vec) -> Result> { sort_exprs .iter() - .map(|expr: &Expr| { - self.expr_to_unparsed(expr)? - .into_order_by_expr() - .or(plan_err!("Expecting Sort expr")) - }) + .map(|sort_expr| self.sort_to_sql(sort_expr)) .collect::>>() } diff --git a/datafusion/sql/src/unparser/rewrite.rs b/datafusion/sql/src/unparser/rewrite.rs index 9e1adcf4df31..522a08af8546 100644 --- a/datafusion/sql/src/unparser/rewrite.rs +++ b/datafusion/sql/src/unparser/rewrite.rs @@ -21,10 +21,11 @@ use std::{ }; use datafusion_common::{ - tree_node::{Transformed, TransformedResult, TreeNode, TreeNodeIterator}, + tree_node::{Transformed, TransformedResult, TreeNode}, Result, }; -use datafusion_expr::{Expr, LogicalPlan, Projection, Sort}; +use datafusion_expr::tree_node::transform_sort_vec; +use datafusion_expr::{Expr, LogicalPlan, Projection, Sort, SortExpr}; use sqlparser::ast::Ident; /// Normalize the schema of a union plan to remove qualifiers from the schema fields and sort expressions. @@ -83,20 +84,18 @@ pub(super) fn normalize_union_schema(plan: &LogicalPlan) -> Result } /// Rewrite sort expressions that have a UNION plan as their input to remove the table reference. -fn rewrite_sort_expr_for_union(exprs: Vec) -> Result> { - let sort_exprs: Vec = exprs - .into_iter() - .map_until_stop_and_collect(|expr| { - expr.transform_up(|expr| { - if let Expr::Column(mut col) = expr { - col.relation = None; - Ok(Transformed::yes(Expr::Column(col))) - } else { - Ok(Transformed::no(expr)) - } - }) +fn rewrite_sort_expr_for_union(exprs: Vec) -> Result> { + let sort_exprs = transform_sort_vec(exprs, &mut |expr| { + expr.transform_up(|expr| { + if let Expr::Column(mut col) = expr { + col.relation = None; + Ok(Transformed::yes(Expr::Column(col))) + } else { + Ok(Transformed::no(expr)) + } }) - .data()?; + }) + .data()?; Ok(sort_exprs) } @@ -158,12 +157,8 @@ pub(super) fn rewrite_plan_for_sort_on_non_projected_fields( .collect::>(); let mut collects = p.expr.clone(); - for expr in &sort.expr { - if let Expr::Sort(s) = expr { - collects.push(s.expr.as_ref().clone()); - } else { - panic!("sort expression must be of type Sort"); - } + for sort in &sort.expr { + collects.push(sort.expr.as_ref().clone()); } // Compare outer collects Expr::to_string with inner collected transformed values diff --git a/datafusion/substrait/src/logical_plan/consumer.rs b/datafusion/substrait/src/logical_plan/consumer.rs index b1b510f1792d..05903bb56cfe 100644 --- a/datafusion/substrait/src/logical_plan/consumer.rs +++ b/datafusion/substrait/src/logical_plan/consumer.rs @@ -31,7 +31,7 @@ use datafusion::logical_expr::expr::{Exists, InSubquery, Sort}; use datafusion::logical_expr::{ expr::find_df_window_func, Aggregate, BinaryExpr, Case, EmptyRelation, Expr, - ExprSchemable, LogicalPlan, Operator, Projection, Values, + ExprSchemable, LogicalPlan, Operator, Projection, SortExpr, Values, }; use substrait::proto::expression::subquery::set_predicate::PredicateOp; use url::Url; @@ -900,8 +900,8 @@ pub async fn from_substrait_sorts( substrait_sorts: &Vec, input_schema: &DFSchema, extensions: &Extensions, -) -> Result> { - let mut sorts: Vec = vec![]; +) -> Result> { + let mut sorts: Vec = vec![]; for s in substrait_sorts { let expr = from_substrait_rex(ctx, s.expr.as_ref().unwrap(), input_schema, extensions) @@ -935,11 +935,11 @@ pub async fn from_substrait_sorts( None => not_impl_err!("Sort without sort kind is invalid"), }; let (asc, nulls_first) = asc_nullfirst.unwrap(); - sorts.push(Expr::Sort(Sort { + sorts.push(Sort { expr: Box::new(expr), asc, nulls_first, - })); + }); } Ok(sorts) } @@ -986,7 +986,7 @@ pub async fn from_substrait_agg_func( input_schema: &DFSchema, extensions: &Extensions, filter: Option>, - order_by: Option>, + order_by: Option>, distinct: bool, ) -> Result> { let args = diff --git a/datafusion/substrait/src/logical_plan/producer.rs b/datafusion/substrait/src/logical_plan/producer.rs index 72b6760be29c..592390a285ba 100644 --- a/datafusion/substrait/src/logical_plan/producer.rs +++ b/datafusion/substrait/src/logical_plan/producer.rs @@ -808,31 +808,26 @@ pub fn to_substrait_agg_measure( /// Converts sort expression to corresponding substrait `SortField` fn to_substrait_sort_field( ctx: &SessionContext, - expr: &Expr, + sort: &Sort, schema: &DFSchemaRef, extensions: &mut Extensions, ) -> Result { - match expr { - Expr::Sort(sort) => { - let sort_kind = match (sort.asc, sort.nulls_first) { - (true, true) => SortDirection::AscNullsFirst, - (true, false) => SortDirection::AscNullsLast, - (false, true) => SortDirection::DescNullsFirst, - (false, false) => SortDirection::DescNullsLast, - }; - Ok(SortField { - expr: Some(to_substrait_rex( - ctx, - sort.expr.deref(), - schema, - 0, - extensions, - )?), - sort_kind: Some(SortKind::Direction(sort_kind.into())), - }) - } - _ => exec_err!("expects to receive sort expression"), - } + let sort_kind = match (sort.asc, sort.nulls_first) { + (true, true) => SortDirection::AscNullsFirst, + (true, false) => SortDirection::AscNullsLast, + (false, true) => SortDirection::DescNullsFirst, + (false, false) => SortDirection::DescNullsLast, + }; + Ok(SortField { + expr: Some(to_substrait_rex( + ctx, + sort.expr.deref(), + schema, + 0, + extensions, + )?), + sort_kind: Some(SortKind::Direction(sort_kind.into())), + }) } /// Return Substrait scalar function with two arguments @@ -2107,30 +2102,26 @@ fn try_to_substrait_field_reference( fn substrait_sort_field( ctx: &SessionContext, - expr: &Expr, + sort: &Sort, schema: &DFSchemaRef, extensions: &mut Extensions, ) -> Result { - match expr { - Expr::Sort(Sort { - expr, - asc, - nulls_first, - }) => { - let e = to_substrait_rex(ctx, expr, schema, 0, extensions)?; - let d = match (asc, nulls_first) { - (true, true) => SortDirection::AscNullsFirst, - (true, false) => SortDirection::AscNullsLast, - (false, true) => SortDirection::DescNullsFirst, - (false, false) => SortDirection::DescNullsLast, - }; - Ok(SortField { - expr: Some(e), - sort_kind: Some(SortKind::Direction(d as i32)), - }) - } - _ => not_impl_err!("Expecting sort expression but got {expr:?}"), - } + let Sort { + expr, + asc, + nulls_first, + } = sort; + let e = to_substrait_rex(ctx, expr, schema, 0, extensions)?; + let d = match (asc, nulls_first) { + (true, true) => SortDirection::AscNullsFirst, + (true, false) => SortDirection::AscNullsLast, + (false, true) => SortDirection::DescNullsFirst, + (false, false) => SortDirection::DescNullsLast, + }; + Ok(SortField { + expr: Some(e), + sort_kind: Some(SortKind::Direction(d as i32)), + }) } fn substrait_field_ref(index: usize) -> Result { diff --git a/docs/source/library-user-guide/using-the-dataframe-api.md b/docs/source/library-user-guide/using-the-dataframe-api.md index 3bd47ef50e51..7f3e28c255c6 100644 --- a/docs/source/library-user-guide/using-the-dataframe-api.md +++ b/docs/source/library-user-guide/using-the-dataframe-api.md @@ -263,14 +263,14 @@ async fn main() -> Result<()>{ let df = ctx.read_csv("tests/data/example.csv", CsvReadOptions::new()).await?; // Create a new DataFrame sorted by `id`, `bank_account` let new_df = df.select(vec![col("a"), col("b")])? - .sort(vec![col("a")])?; + .sort_by(vec![col("a")])?; // Build the same plan using the LogicalPlanBuilder // Similar to `SELECT a, b FROM example.csv ORDER BY a` let df = ctx.read_csv("tests/data/example.csv", CsvReadOptions::new()).await?; let (_state, plan) = df.into_parts(); // get the DataFrame's LogicalPlan let plan = LogicalPlanBuilder::from(plan) .project(vec![col("a"), col("b")])? - .sort(vec![col("a")])? + .sort_by(vec![col("a")])? .build()?; // prove they are the same assert_eq!(new_df.logical_plan(), &plan); From 095feb9cfc03bcfa0c34ad74e77b570174fc0244 Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Thu, 29 Aug 2024 17:39:35 +0200 Subject: [PATCH 046/154] Remove normalize_with_schemas function (#12233) It was deprecated since 20.0.0. --- datafusion/common/src/column.rs | 75 +----------------------- datafusion/expr/src/expr_rewriter/mod.rs | 2 +- 2 files changed, 2 insertions(+), 75 deletions(-) diff --git a/datafusion/common/src/column.rs b/datafusion/common/src/column.rs index 2e2bfff40340..d855198fa7c6 100644 --- a/datafusion/common/src/column.rs +++ b/datafusion/common/src/column.rs @@ -26,7 +26,6 @@ use std::collections::HashSet; use std::convert::Infallible; use std::fmt; use std::str::FromStr; -use std::sync::Arc; /// A named reference to a qualified field in a schema. #[derive(Debug, Clone, PartialEq, Eq, Hash, PartialOrd, Ord)] @@ -156,79 +155,6 @@ impl Column { } } - /// Qualify column if not done yet. - /// - /// If this column already has a [relation](Self::relation), it will be returned as is and the given parameters are - /// ignored. Otherwise this will search through the given schemas to find the column. This will use the first schema - /// that matches. - /// - /// A schema matches if there is a single column that -- when unqualified -- matches this column. There is an - /// exception for `USING` statements, see below. - /// - /// # Using columns - /// Take the following SQL statement: - /// - /// ```sql - /// SELECT id FROM t1 JOIN t2 USING(id) - /// ``` - /// - /// In this case, both `t1.id` and `t2.id` will match unqualified column `id`. To express this possibility, use - /// `using_columns`. Each entry in this array is a set of columns that are bound together via a `USING` clause. So - /// in this example this would be `[{t1.id, t2.id}]`. - #[deprecated( - since = "20.0.0", - note = "use normalize_with_schemas_and_ambiguity_check instead" - )] - pub fn normalize_with_schemas( - self, - schemas: &[&Arc], - using_columns: &[HashSet], - ) -> Result { - if self.relation.is_some() { - return Ok(self); - } - - for schema in schemas { - let qualified_fields = - schema.qualified_fields_with_unqualified_name(&self.name); - match qualified_fields.len() { - 0 => continue, - 1 => { - return Ok(Column::from(qualified_fields[0])); - } - _ => { - // More than 1 fields in this schema have their names set to self.name. - // - // This should only happen when a JOIN query with USING constraint references - // join columns using unqualified column name. For example: - // - // ```sql - // SELECT id FROM t1 JOIN t2 USING(id) - // ``` - // - // In this case, both `t1.id` and `t2.id` will match unqualified column `id`. - // We will use the relation from the first matched field to normalize self. - - // Compare matched fields with one USING JOIN clause at a time - let columns = schema.columns_with_unqualified_name(&self.name); - for using_col in using_columns { - let all_matched = columns.iter().all(|f| using_col.contains(f)); - // All matched fields belong to the same using column set, in orther words - // the same join clause. We simply pick the qualifier from the first match. - if all_matched { - return Ok(columns[0].clone()); - } - } - } - } - } - - _schema_err!(SchemaError::FieldNotFound { - field: Box::new(Column::new(self.relation.clone(), self.name)), - valid_fields: schemas.iter().flat_map(|s| s.columns()).collect(), - }) - } - /// Qualify column if not done yet. /// /// If this column already has a [relation](Self::relation), it will be returned as is and the given parameters are @@ -381,6 +307,7 @@ mod tests { use super::*; use arrow::datatypes::DataType; use arrow_schema::SchemaBuilder; + use std::sync::Arc; fn create_qualified_schema(qualifier: &str, names: Vec<&str>) -> Result { let mut schema_builder = SchemaBuilder::new(); diff --git a/datafusion/expr/src/expr_rewriter/mod.rs b/datafusion/expr/src/expr_rewriter/mod.rs index b809b015d929..5e7fedb4cbd8 100644 --- a/datafusion/expr/src/expr_rewriter/mod.rs +++ b/datafusion/expr/src/expr_rewriter/mod.rs @@ -60,7 +60,7 @@ pub trait FunctionRewrite { ) -> Result>; } -/// Recursively call [`Column::normalize_with_schemas`] on all [`Column`] expressions +/// Recursively call `LogicalPlanBuilder::normalize` on all [`Column`] expressions /// in the `expr` expression tree. pub fn normalize_col(expr: Expr, plan: &LogicalPlan) -> Result { expr.transform(|expr| { From 827d7e125b8876e0c706d1f7edaaabf6ce9e37b0 Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Thu, 29 Aug 2024 17:40:47 +0200 Subject: [PATCH 047/154] Update AWS dependencies in CLI (#12229) --- datafusion-cli/Cargo.lock | 355 ++++++++++++--------------- datafusion-cli/Cargo.toml | 4 +- datafusion-cli/src/object_storage.rs | 3 +- 3 files changed, 160 insertions(+), 202 deletions(-) diff --git a/datafusion-cli/Cargo.lock b/datafusion-cli/Cargo.lock index 1e89bb3af87e..ddc6242977d3 100644 --- a/datafusion-cli/Cargo.lock +++ b/datafusion-cli/Cargo.lock @@ -473,160 +473,155 @@ checksum = "0c4b4d0bd25bd0b74681c0ad21497610ce1b7c91b1022cd21c80c6fbdd9476b0" [[package]] name = "aws-config" -version = "0.55.3" +version = "1.5.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bcdcf0d683fe9c23d32cf5b53c9918ea0a500375a9fb20109802552658e576c9" +checksum = "4e95816a168520d72c0e7680c405a5a8c1fb6a035b4bc4b9d7b0de8e1a941697" dependencies = [ "aws-credential-types", - "aws-http", + "aws-runtime", "aws-sdk-sso", + "aws-sdk-ssooidc", "aws-sdk-sts", "aws-smithy-async", - "aws-smithy-client", "aws-smithy-http", - "aws-smithy-http-tower", "aws-smithy-json", + "aws-smithy-runtime", + "aws-smithy-runtime-api", "aws-smithy-types", "aws-types", "bytes", - "fastrand 1.9.0", + "fastrand", "hex", "http 0.2.12", - "hyper 0.14.30", - "ring 0.16.20", + "ring", "time", "tokio", - "tower", "tracing", + "url", "zeroize", ] [[package]] name = "aws-credential-types" -version = "0.55.3" +version = "1.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1fcdb2f7acbc076ff5ad05e7864bdb191ca70a6fd07668dc3a1a8bcd051de5ae" +checksum = "e16838e6c9e12125face1c1eff1343c75e3ff540de98ff7ebd61874a89bcfeb9" dependencies = [ "aws-smithy-async", + "aws-smithy-runtime-api", "aws-smithy-types", - "fastrand 1.9.0", - "tokio", - "tracing", "zeroize", ] [[package]] -name = "aws-endpoint" -version = "0.55.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8cce1c41a6cfaa726adee9ebb9a56fcd2bbfd8be49fd8a04c5e20fd968330b04" -dependencies = [ - "aws-smithy-http", - "aws-smithy-types", - "aws-types", - "http 0.2.12", - "regex", - "tracing", -] - -[[package]] -name = "aws-http" -version = "0.55.3" +name = "aws-runtime" +version = "1.4.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "aadbc44e7a8f3e71c8b374e03ecd972869eb91dd2bc89ed018954a52ba84bc44" +checksum = "f42c2d4218de4dcd890a109461e2f799a1a2ba3bcd2cde9af88360f5df9266c6" dependencies = [ "aws-credential-types", + "aws-sigv4", + "aws-smithy-async", "aws-smithy-http", + "aws-smithy-runtime-api", "aws-smithy-types", "aws-types", "bytes", + "fastrand", "http 0.2.12", "http-body 0.4.6", - "lazy_static", + "once_cell", "percent-encoding", "pin-project-lite", "tracing", + "uuid", ] [[package]] name = "aws-sdk-sso" -version = "0.28.0" +version = "1.39.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c8b812340d86d4a766b2ca73f740dfd47a97c2dff0c06c8517a16d88241957e4" +checksum = "11822090cf501c316c6f75711d77b96fba30658e3867a7762e5e2f5d32d31e81" dependencies = [ "aws-credential-types", - "aws-endpoint", - "aws-http", - "aws-sig-auth", + "aws-runtime", "aws-smithy-async", - "aws-smithy-client", "aws-smithy-http", - "aws-smithy-http-tower", "aws-smithy-json", + "aws-smithy-runtime", + "aws-smithy-runtime-api", "aws-smithy-types", "aws-types", "bytes", "http 0.2.12", - "regex", - "tokio-stream", - "tower", + "once_cell", + "regex-lite", "tracing", ] [[package]] -name = "aws-sdk-sts" -version = "0.28.0" +name = "aws-sdk-ssooidc" +version = "1.40.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "265fac131fbfc188e5c3d96652ea90ecc676a934e3174eaaee523c6cec040b3b" +checksum = "78a2a06ff89176123945d1bbe865603c4d7101bea216a550bb4d2e4e9ba74d74" dependencies = [ "aws-credential-types", - "aws-endpoint", - "aws-http", - "aws-sig-auth", + "aws-runtime", "aws-smithy-async", - "aws-smithy-client", "aws-smithy-http", - "aws-smithy-http-tower", "aws-smithy-json", - "aws-smithy-query", + "aws-smithy-runtime", + "aws-smithy-runtime-api", "aws-smithy-types", - "aws-smithy-xml", "aws-types", "bytes", "http 0.2.12", - "regex", - "tower", + "once_cell", + "regex-lite", "tracing", ] [[package]] -name = "aws-sig-auth" -version = "0.55.3" +name = "aws-sdk-sts" +version = "1.39.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3b94acb10af0c879ecd5c7bdf51cda6679a0a4f4643ce630905a77673bfa3c61" +checksum = "a20a91795850826a6f456f4a48eff1dfa59a0e69bdbf5b8c50518fd372106574" dependencies = [ "aws-credential-types", - "aws-sigv4", + "aws-runtime", + "aws-smithy-async", "aws-smithy-http", + "aws-smithy-json", + "aws-smithy-query", + "aws-smithy-runtime", + "aws-smithy-runtime-api", + "aws-smithy-types", + "aws-smithy-xml", "aws-types", "http 0.2.12", + "once_cell", + "regex-lite", "tracing", ] [[package]] name = "aws-sigv4" -version = "0.55.3" +version = "1.2.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9d2ce6f507be68e968a33485ced670111d1cbad161ddbbab1e313c03d37d8f4c" +checksum = "5df1b0fa6be58efe9d4ccc257df0a53b89cd8909e86591a13ca54817c87517be" dependencies = [ + "aws-credential-types", "aws-smithy-http", + "aws-smithy-runtime-api", + "aws-smithy-types", + "bytes", "form_urlencoded", "hex", "hmac", "http 0.2.12", + "http 1.1.0", "once_cell", "percent-encoding", - "regex", "sha2", "time", "tracing", @@ -634,53 +629,28 @@ dependencies = [ [[package]] name = "aws-smithy-async" -version = "0.55.3" +version = "1.2.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "13bda3996044c202d75b91afeb11a9afae9db9a721c6a7a427410018e286b880" +checksum = "62220bc6e97f946ddd51b5f1361f78996e704677afc518a4ff66b7a72ea1378c" dependencies = [ "futures-util", "pin-project-lite", "tokio", - "tokio-stream", -] - -[[package]] -name = "aws-smithy-client" -version = "0.55.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0a86aa6e21e86c4252ad6a0e3e74da9617295d8d6e374d552be7d3059c41cedd" -dependencies = [ - "aws-smithy-async", - "aws-smithy-http", - "aws-smithy-http-tower", - "aws-smithy-types", - "bytes", - "fastrand 1.9.0", - "http 0.2.12", - "http-body 0.4.6", - "hyper 0.14.30", - "hyper-rustls 0.23.2", - "lazy_static", - "pin-project-lite", - "rustls 0.20.9", - "tokio", - "tower", - "tracing", ] [[package]] name = "aws-smithy-http" -version = "0.55.3" +version = "0.60.10" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2b3b693869133551f135e1f2c77cb0b8277d9e3e17feaf2213f735857c4f0d28" +checksum = "01dbcb6e2588fd64cfb6d7529661b06466419e4c54ed1c62d6510d2d0350a728" dependencies = [ + "aws-smithy-runtime-api", "aws-smithy-types", "bytes", "bytes-utils", "futures-core", "http 0.2.12", "http-body 0.4.6", - "hyper 0.14.30", "once_cell", "percent-encoding", "pin-project-lite", @@ -689,74 +659,110 @@ dependencies = [ ] [[package]] -name = "aws-smithy-http-tower" -version = "0.55.3" +name = "aws-smithy-json" +version = "0.60.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3ae4f6c5798a247fac98a867698197d9ac22643596dc3777f0c76b91917616b9" +checksum = "4683df9469ef09468dad3473d129960119a0d3593617542b7d52086c8486f2d6" dependencies = [ - "aws-smithy-http", "aws-smithy-types", - "bytes", - "http 0.2.12", - "http-body 0.4.6", - "pin-project-lite", - "tower", - "tracing", ] [[package]] -name = "aws-smithy-json" -version = "0.55.3" +name = "aws-smithy-query" +version = "0.60.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "23f9f42fbfa96d095194a632fbac19f60077748eba536eb0b9fecc28659807f8" +checksum = "f2fbd61ceb3fe8a1cb7352e42689cec5335833cd9f94103a61e98f9bb61c64bb" dependencies = [ "aws-smithy-types", + "urlencoding", ] [[package]] -name = "aws-smithy-query" -version = "0.55.3" +name = "aws-smithy-runtime" +version = "1.7.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "98819eb0b04020a1c791903533b638534ae6c12e2aceda3e6e6fba015608d51d" +checksum = "d1ce695746394772e7000b39fe073095db6d45a862d0767dd5ad0ac0d7f8eb87" dependencies = [ + "aws-smithy-async", + "aws-smithy-http", + "aws-smithy-runtime-api", "aws-smithy-types", - "urlencoding", + "bytes", + "fastrand", + "h2 0.3.26", + "http 0.2.12", + "http-body 0.4.6", + "http-body 1.0.1", + "httparse", + "hyper 0.14.30", + "hyper-rustls 0.24.2", + "once_cell", + "pin-project-lite", + "pin-utils", + "rustls 0.21.12", + "tokio", + "tracing", +] + +[[package]] +name = "aws-smithy-runtime-api" +version = "1.7.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e086682a53d3aa241192aa110fa8dfce98f2f5ac2ead0de84d41582c7e8fdb96" +dependencies = [ + "aws-smithy-async", + "aws-smithy-types", + "bytes", + "http 0.2.12", + "http 1.1.0", + "pin-project-lite", + "tokio", + "tracing", + "zeroize", ] [[package]] name = "aws-smithy-types" -version = "0.55.3" +version = "1.2.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "16a3d0bf4f324f4ef9793b86a1701d9700fbcdbd12a846da45eed104c634c6e8" +checksum = "273dcdfd762fae3e1650b8024624e7cd50e484e37abdab73a7a706188ad34543" dependencies = [ "base64-simd", + "bytes", + "bytes-utils", + "http 0.2.12", + "http 1.1.0", + "http-body 0.4.6", + "http-body 1.0.1", + "http-body-util", "itoa", "num-integer", + "pin-project-lite", + "pin-utils", "ryu", + "serde", "time", ] [[package]] name = "aws-smithy-xml" -version = "0.55.3" +version = "0.60.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b1b9d12875731bd07e767be7baad95700c3137b56730ec9ddeedb52a5e5ca63b" +checksum = "d123fbc2a4adc3c301652ba8e149bf4bc1d1725affb9784eb20c953ace06bf55" dependencies = [ "xmlparser", ] [[package]] name = "aws-types" -version = "0.55.3" +version = "1.3.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6dd209616cc8d7bfb82f87811a5c655dc97537f592689b18743bddf5dc5c4829" +checksum = "5221b91b3e441e6675310829fd8984801b772cb1546ef6c0e54dec9f1ac13fef" dependencies = [ "aws-credential-types", "aws-smithy-async", - "aws-smithy-client", - "aws-smithy-http", + "aws-smithy-runtime-api", "aws-smithy-types", - "http 0.2.12", "rustc_version", "tracing", ] @@ -1689,15 +1695,6 @@ dependencies = [ "str-buf", ] -[[package]] -name = "fastrand" -version = "1.9.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e51093e27b0797c359783294ca4f0a911c270184cb10f85783b118614a1501be" -dependencies = [ - "instant", -] - [[package]] name = "fastrand" version = "2.1.0" @@ -2114,17 +2111,18 @@ dependencies = [ [[package]] name = "hyper-rustls" -version = "0.23.2" +version = "0.24.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1788965e61b367cd03a62950836d5cd41560c3577d90e40e0819373194d1661c" +checksum = "ec3efd23720e2049821a693cbc7e65ea87c72f1c58ff2f9522ff332b1491e590" dependencies = [ + "futures-util", "http 0.2.12", "hyper 0.14.30", "log", - "rustls 0.20.9", + "rustls 0.21.12", "rustls-native-certs 0.6.3", "tokio", - "tokio-rustls 0.23.4", + "tokio-rustls 0.24.1", ] [[package]] @@ -2645,7 +2643,7 @@ dependencies = [ "quick-xml", "rand", "reqwest", - "ring 0.17.8", + "ring", "rustls-pemfile 2.1.3", "serde", "serde_json", @@ -2945,7 +2943,7 @@ checksum = "ba92fb39ec7ad06ca2582c0ca834dfeadcaf06ddfc8e635c80aa7e1c05315fdd" dependencies = [ "bytes", "rand", - "ring 0.17.8", + "ring", "rustc-hash", "rustls 0.23.12", "slab", @@ -3116,21 +3114,6 @@ dependencies = [ "windows-registry", ] -[[package]] -name = "ring" -version = "0.16.20" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3053cf52e236a3ed746dfc745aa9cacf1b791d846bdaf412f60a8d7d6e17c8fc" -dependencies = [ - "cc", - "libc", - "once_cell", - "spin 0.5.2", - "untrusted 0.7.1", - "web-sys", - "winapi", -] - [[package]] name = "ring" version = "0.17.8" @@ -3141,8 +3124,8 @@ dependencies = [ "cfg-if", "getrandom", "libc", - "spin 0.9.8", - "untrusted 0.9.0", + "spin", + "untrusted", "windows-sys 0.52.0", ] @@ -3214,14 +3197,14 @@ dependencies = [ [[package]] name = "rustls" -version = "0.20.9" +version = "0.21.12" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1b80e3dec595989ea8510028f30c408a4630db12c9cbb8de34203b89d6577e99" +checksum = "3f56a14d1f48b391359b22f731fd4bd7e43c97f3c50eee276f3aa09c94784d3e" dependencies = [ "log", - "ring 0.16.20", + "ring", + "rustls-webpki 0.101.7", "sct", - "webpki", ] [[package]] @@ -3231,9 +3214,9 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "c58f8c84392efc0a126acce10fa59ff7b3d2ac06ab451a33f2741989b806b044" dependencies = [ "once_cell", - "ring 0.17.8", + "ring", "rustls-pki-types", - "rustls-webpki", + "rustls-webpki 0.102.6", "subtle", "zeroize", ] @@ -3288,15 +3271,25 @@ version = "1.8.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "fc0a2ce646f8655401bb81e7927b812614bd5d91dbc968696be50603510fcaf0" +[[package]] +name = "rustls-webpki" +version = "0.101.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8b6275d1ee7a1cd780b64aca7726599a1dbc893b1e64144529e55c3c2f745765" +dependencies = [ + "ring", + "untrusted", +] + [[package]] name = "rustls-webpki" version = "0.102.6" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "8e6b52d4fda176fd835fdc55a835d4a89b8499cad995885a21149d5ad62f852e" dependencies = [ - "ring 0.17.8", + "ring", "rustls-pki-types", - "untrusted 0.9.0", + "untrusted", ] [[package]] @@ -3364,8 +3357,8 @@ version = "0.7.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "da046153aa2352493d6cb7da4b6e5c0c057d8a1d0a9aa8560baffdd945acd414" dependencies = [ - "ring 0.17.8", - "untrusted 0.9.0", + "ring", + "untrusted", ] [[package]] @@ -3532,12 +3525,6 @@ dependencies = [ "windows-sys 0.52.0", ] -[[package]] -name = "spin" -version = "0.5.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6e63cff320ae2c57904679ba7cb63280a3dc4613885beafb148ee7bf9aa9042d" - [[package]] name = "spin" version = "0.9.8" @@ -3668,7 +3655,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "04cbcdd0c794ebb0d4cf35e88edd2f7d2c4c3e9a5a6dab322839b321c6a87a64" dependencies = [ "cfg-if", - "fastrand 2.1.0", + "fastrand", "once_cell", "rustix", "windows-sys 0.59.0", @@ -3805,13 +3792,12 @@ dependencies = [ [[package]] name = "tokio-rustls" -version = "0.23.4" +version = "0.24.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c43ee83903113e03984cb9e5cebe6c04a5116269e900e3ddba8f068a62adda59" +checksum = "c28327cf380ac148141087fbfb9de9d7bd4e84ab5d2c28fbc911d753de8a7081" dependencies = [ - "rustls 0.20.9", + "rustls 0.21.12", "tokio", - "webpki", ] [[package]] @@ -3825,17 +3811,6 @@ dependencies = [ "tokio", ] -[[package]] -name = "tokio-stream" -version = "0.1.15" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "267ac89e0bec6e691e5813911606935d77c476ff49024f98abcea3e7b15e37af" -dependencies = [ - "futures-core", - "pin-project-lite", - "tokio", -] - [[package]] name = "tokio-util" version = "0.7.11" @@ -3862,7 +3837,6 @@ dependencies = [ "tokio", "tower-layer", "tower-service", - "tracing", ] [[package]] @@ -3883,7 +3857,6 @@ version = "0.1.40" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "c3523ab5a71916ccf420eebdf5521fcef02141234bbc0b8a49f2fdc4544364ef" dependencies = [ - "log", "pin-project-lite", "tracing-attributes", "tracing-core", @@ -3984,12 +3957,6 @@ version = "0.1.13" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "0336d538f7abc86d282a4189614dfaa90810dfc2c6f6427eaf88e16311dd225d" -[[package]] -name = "untrusted" -version = "0.7.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a156c684c91ea7d62626509bce3cb4e1d9ed5c4d978f7b4352658f96a4c26b4a" - [[package]] name = "untrusted" version = "0.9.0" @@ -4165,16 +4132,6 @@ dependencies = [ "wasm-bindgen", ] -[[package]] -name = "webpki" -version = "0.22.4" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ed63aea5ce73d0ff405984102c42de94fc55a6b75765d621c65262469b3c9b53" -dependencies = [ - "ring 0.17.8", - "untrusted 0.9.0", -] - [[package]] name = "winapi" version = "0.3.9" diff --git a/datafusion-cli/Cargo.toml b/datafusion-cli/Cargo.toml index 0a4523a1c04e..f477bad69a2c 100644 --- a/datafusion-cli/Cargo.toml +++ b/datafusion-cli/Cargo.toml @@ -32,8 +32,8 @@ readme = "README.md" [dependencies] arrow = { version = "52.2.0" } async-trait = "0.1.73" -aws-config = "0.55" -aws-credential-types = "0.55" +aws-config = "1.5.5" +aws-credential-types = "1.2.0" clap = { version = "4.5.16", features = ["derive", "cargo"] } datafusion = { path = "../datafusion/core", version = "41.0.0", features = [ "avro", diff --git a/datafusion-cli/src/object_storage.rs b/datafusion-cli/src/object_storage.rs index 87eb04d113de..e8d60e4f0926 100644 --- a/datafusion-cli/src/object_storage.rs +++ b/datafusion-cli/src/object_storage.rs @@ -27,6 +27,7 @@ use datafusion::error::{DataFusionError, Result}; use datafusion::execution::context::SessionState; use async_trait::async_trait; +use aws_config::BehaviorVersion; use aws_credential_types::provider::ProvideCredentials; use object_store::aws::{AmazonS3Builder, AwsCredential}; use object_store::gcp::GoogleCloudStorageBuilder; @@ -61,7 +62,7 @@ pub async fn get_s3_object_store_builder( builder = builder.with_token(session_token); } } else { - let config = aws_config::from_env().load().await; + let config = aws_config::defaults(BehaviorVersion::latest()).load().await; if let Some(region) = config.region() { builder = builder.with_region(region.to_string()); } From 0f16849ea2025cfecb25d23eee68f8ec25edadd0 Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Thu, 29 Aug 2024 17:47:35 +0200 Subject: [PATCH 048/154] Avoid Arc::clone when serializing physical expressions (#12235) `Arc::clone` is indispensable when passing shared references between threads. For synchronous code`&` and `&Arc` can be (and often are) used, with the latter being future-compatible, should the code start to need `Arc::clone` (e.g. due to parallelization). --- datafusion/proto/src/physical_plan/mod.rs | 56 +++++------ .../proto/src/physical_plan/to_proto.rs | 93 ++++++------------- .../tests/cases/roundtrip_physical_plan.rs | 3 +- 3 files changed, 55 insertions(+), 97 deletions(-) diff --git a/datafusion/proto/src/physical_plan/mod.rs b/datafusion/proto/src/physical_plan/mod.rs index 78f370c714cc..04cbf8b537b3 100644 --- a/datafusion/proto/src/physical_plan/mod.rs +++ b/datafusion/proto/src/physical_plan/mod.rs @@ -1122,7 +1122,7 @@ impl AsExecutionPlan for protobuf::PhysicalPlanNode { let expr = exec .expr() .iter() - .map(|expr| serialize_physical_expr(Arc::clone(&expr.0), extension_codec)) + .map(|expr| serialize_physical_expr(&expr.0, extension_codec)) .collect::>>()?; let expr_name = exec.expr().iter().map(|expr| expr.1.clone()).collect(); return Ok(protobuf::PhysicalPlanNode { @@ -1163,7 +1163,7 @@ impl AsExecutionPlan for protobuf::PhysicalPlanNode { protobuf::FilterExecNode { input: Some(Box::new(input)), expr: Some(serialize_physical_expr( - Arc::clone(exec.predicate()), + exec.predicate(), extension_codec, )?), default_filter_selectivity: exec.default_selectivity() as u32, @@ -1220,8 +1220,8 @@ impl AsExecutionPlan for protobuf::PhysicalPlanNode { .on() .iter() .map(|tuple| { - let l = serialize_physical_expr(tuple.0.to_owned(), extension_codec)?; - let r = serialize_physical_expr(tuple.1.to_owned(), extension_codec)?; + let l = serialize_physical_expr(&tuple.0, extension_codec)?; + let r = serialize_physical_expr(&tuple.1, extension_codec)?; Ok::<_, DataFusionError>(protobuf::JoinOn { left: Some(l), right: Some(r), @@ -1233,10 +1233,8 @@ impl AsExecutionPlan for protobuf::PhysicalPlanNode { .filter() .as_ref() .map(|f| { - let expression = serialize_physical_expr( - f.expression().to_owned(), - extension_codec, - )?; + let expression = + serialize_physical_expr(f.expression(), extension_codec)?; let column_indices = f .column_indices() .iter() @@ -1294,8 +1292,8 @@ impl AsExecutionPlan for protobuf::PhysicalPlanNode { .on() .iter() .map(|tuple| { - let l = serialize_physical_expr(tuple.0.to_owned(), extension_codec)?; - let r = serialize_physical_expr(tuple.1.to_owned(), extension_codec)?; + let l = serialize_physical_expr(&tuple.0, extension_codec)?; + let r = serialize_physical_expr(&tuple.1, extension_codec)?; Ok::<_, DataFusionError>(protobuf::JoinOn { left: Some(l), right: Some(r), @@ -1307,10 +1305,8 @@ impl AsExecutionPlan for protobuf::PhysicalPlanNode { .filter() .as_ref() .map(|f| { - let expression = serialize_physical_expr( - f.expression().to_owned(), - extension_codec, - )?; + let expression = + serialize_physical_expr(f.expression(), extension_codec)?; let column_indices = f .column_indices() .iter() @@ -1348,7 +1344,7 @@ impl AsExecutionPlan for protobuf::PhysicalPlanNode { .map(|expr| { Ok(protobuf::PhysicalSortExprNode { expr: Some(Box::new(serialize_physical_expr( - expr.expr.to_owned(), + &expr.expr, extension_codec, )?)), asc: !expr.options.descending, @@ -1368,7 +1364,7 @@ impl AsExecutionPlan for protobuf::PhysicalPlanNode { .map(|expr| { Ok(protobuf::PhysicalSortExprNode { expr: Some(Box::new(serialize_physical_expr( - expr.expr.to_owned(), + &expr.expr, extension_codec, )?)), asc: !expr.options.descending, @@ -1475,14 +1471,14 @@ impl AsExecutionPlan for protobuf::PhysicalPlanNode { .group_expr() .null_expr() .iter() - .map(|expr| serialize_physical_expr(expr.0.to_owned(), extension_codec)) + .map(|expr| serialize_physical_expr(&expr.0, extension_codec)) .collect::>>()?; let group_expr = exec .group_expr() .expr() .iter() - .map(|expr| serialize_physical_expr(expr.0.to_owned(), extension_codec)) + .map(|expr| serialize_physical_expr(&expr.0, extension_codec)) .collect::>>()?; let limit = exec.limit().map(|value| protobuf::AggLimit { @@ -1581,7 +1577,7 @@ impl AsExecutionPlan for protobuf::PhysicalPlanNode { if let Some(exec) = plan.downcast_ref::() { let predicate = exec .predicate() - .map(|pred| serialize_physical_expr(Arc::clone(pred), extension_codec)) + .map(|pred| serialize_physical_expr(pred, extension_codec)) .transpose()?; return Ok(protobuf::PhysicalPlanNode { physical_plan_type: Some(PhysicalPlanType::ParquetScan( @@ -1653,7 +1649,7 @@ impl AsExecutionPlan for protobuf::PhysicalPlanNode { .map(|expr| { let sort_expr = Box::new(protobuf::PhysicalSortExprNode { expr: Some(Box::new(serialize_physical_expr( - expr.expr.to_owned(), + &expr.expr, extension_codec, )?)), asc: !expr.options.descending, @@ -1722,7 +1718,7 @@ impl AsExecutionPlan for protobuf::PhysicalPlanNode { .map(|expr| { let sort_expr = Box::new(protobuf::PhysicalSortExprNode { expr: Some(Box::new(serialize_physical_expr( - expr.expr.to_owned(), + &expr.expr, extension_codec, )?)), asc: !expr.options.descending, @@ -1761,10 +1757,8 @@ impl AsExecutionPlan for protobuf::PhysicalPlanNode { .filter() .as_ref() .map(|f| { - let expression = serialize_physical_expr( - f.expression().to_owned(), - extension_codec, - )?; + let expression = + serialize_physical_expr(f.expression(), extension_codec)?; let column_indices = f .column_indices() .iter() @@ -1806,13 +1800,13 @@ impl AsExecutionPlan for protobuf::PhysicalPlanNode { let window_expr = exec .window_expr() .iter() - .map(|e| serialize_physical_window_expr(Arc::clone(e), extension_codec)) + .map(|e| serialize_physical_window_expr(e, extension_codec)) .collect::>>()?; let partition_keys = exec .partition_keys .iter() - .map(|e| serialize_physical_expr(Arc::clone(e), extension_codec)) + .map(|e| serialize_physical_expr(e, extension_codec)) .collect::>>()?; return Ok(protobuf::PhysicalPlanNode { @@ -1836,13 +1830,13 @@ impl AsExecutionPlan for protobuf::PhysicalPlanNode { let window_expr = exec .window_expr() .iter() - .map(|e| serialize_physical_window_expr(Arc::clone(e), extension_codec)) + .map(|e| serialize_physical_window_expr(e, extension_codec)) .collect::>>()?; let partition_keys = exec .partition_keys .iter() - .map(|e| serialize_physical_expr(Arc::clone(e), extension_codec)) + .map(|e| serialize_physical_expr(e, extension_codec)) .collect::>>()?; let input_order_mode = match &exec.input_order_mode { @@ -1886,7 +1880,7 @@ impl AsExecutionPlan for protobuf::PhysicalPlanNode { let expr: PhysicalSortExpr = requirement.to_owned().into(); let sort_expr = protobuf::PhysicalSortExprNode { expr: Some(Box::new(serialize_physical_expr( - expr.expr.to_owned(), + &expr.expr, extension_codec, )?)), asc: !expr.options.descending, @@ -2025,7 +2019,7 @@ pub trait PhysicalExtensionCodec: Debug + Send + Sync { fn try_encode_expr( &self, - _node: Arc, + _node: &Arc, _buf: &mut Vec, ) -> Result<()> { not_impl_err!("PhysicalExtensionCodec is not provided") diff --git a/datafusion/proto/src/physical_plan/to_proto.rs b/datafusion/proto/src/physical_plan/to_proto.rs index 555ad22a9bc1..25be7de61cc3 100644 --- a/datafusion/proto/src/physical_plan/to_proto.rs +++ b/datafusion/proto/src/physical_plan/to_proto.rs @@ -52,7 +52,7 @@ pub fn serialize_physical_aggr_expr( aggr_expr: Arc, codec: &dyn PhysicalExtensionCodec, ) -> Result { - let expressions = serialize_physical_exprs(aggr_expr.expressions(), codec)?; + let expressions = serialize_physical_exprs(&aggr_expr.expressions(), codec)?; let ordering_req = aggr_expr.order_bys().unwrap_or(&[]).to_vec(); let ordering_req = serialize_physical_sort_exprs(ordering_req, codec)?; @@ -96,7 +96,7 @@ fn serialize_physical_window_aggr_expr( } pub fn serialize_physical_window_expr( - window_expr: Arc, + window_expr: &Arc, codec: &dyn PhysicalExtensionCodec, ) -> Result { let expr = window_expr.as_any(); @@ -187,9 +187,8 @@ pub fn serialize_physical_window_expr( return not_impl_err!("WindowExpr not supported: {window_expr:?}"); }; - let args = serialize_physical_exprs(args, codec)?; - let partition_by = - serialize_physical_exprs(window_expr.partition_by().to_vec(), codec)?; + let args = serialize_physical_exprs(&args, codec)?; + let partition_by = serialize_physical_exprs(window_expr.partition_by(), codec)?; let order_by = serialize_physical_sort_exprs(window_expr.order_by().to_vec(), codec)?; let window_frame: protobuf::WindowFrame = window_frame .as_ref() @@ -225,7 +224,7 @@ pub fn serialize_physical_sort_expr( codec: &dyn PhysicalExtensionCodec, ) -> Result { let PhysicalSortExpr { expr, options } = sort_expr; - let expr = serialize_physical_expr(expr, codec)?; + let expr = serialize_physical_expr(&expr, codec)?; Ok(PhysicalSortExprNode { expr: Some(Box::new(expr)), asc: !options.descending, @@ -233,12 +232,12 @@ pub fn serialize_physical_sort_expr( }) } -pub fn serialize_physical_exprs( +pub fn serialize_physical_exprs<'a, I>( values: I, codec: &dyn PhysicalExtensionCodec, ) -> Result> where - I: IntoIterator>, + I: IntoIterator>, { values .into_iter() @@ -251,7 +250,7 @@ where /// If required, a [`PhysicalExtensionCodec`] can be provided which can handle /// serialization of udfs requiring specialized serialization (see [`PhysicalExtensionCodec::try_encode_udf`]) pub fn serialize_physical_expr( - value: Arc, + value: &Arc, codec: &dyn PhysicalExtensionCodec, ) -> Result { let expr = value.as_any(); @@ -267,14 +266,8 @@ pub fn serialize_physical_expr( }) } else if let Some(expr) = expr.downcast_ref::() { let binary_expr = Box::new(protobuf::PhysicalBinaryExprNode { - l: Some(Box::new(serialize_physical_expr( - Arc::clone(expr.left()), - codec, - )?)), - r: Some(Box::new(serialize_physical_expr( - Arc::clone(expr.right()), - codec, - )?)), + l: Some(Box::new(serialize_physical_expr(expr.left(), codec)?)), + r: Some(Box::new(serialize_physical_expr(expr.right(), codec)?)), op: format!("{:?}", expr.op()), }); @@ -292,8 +285,7 @@ pub fn serialize_physical_expr( expr: expr .expr() .map(|exp| { - serialize_physical_expr(Arc::clone(exp), codec) - .map(Box::new) + serialize_physical_expr(exp, codec).map(Box::new) }) .transpose()?, when_then_expr: expr @@ -308,10 +300,7 @@ pub fn serialize_physical_expr( >>()?, else_expr: expr .else_expr() - .map(|a| { - serialize_physical_expr(Arc::clone(a), codec) - .map(Box::new) - }) + .map(|a| serialize_physical_expr(a, codec).map(Box::new)) .transpose()?, }, ), @@ -322,10 +311,7 @@ pub fn serialize_physical_expr( Ok(protobuf::PhysicalExprNode { expr_type: Some(protobuf::physical_expr_node::ExprType::NotExpr(Box::new( protobuf::PhysicalNot { - expr: Some(Box::new(serialize_physical_expr( - expr.arg().to_owned(), - codec, - )?)), + expr: Some(Box::new(serialize_physical_expr(expr.arg(), codec)?)), }, ))), }) @@ -333,10 +319,7 @@ pub fn serialize_physical_expr( Ok(protobuf::PhysicalExprNode { expr_type: Some(protobuf::physical_expr_node::ExprType::IsNullExpr( Box::new(protobuf::PhysicalIsNull { - expr: Some(Box::new(serialize_physical_expr( - expr.arg().to_owned(), - codec, - )?)), + expr: Some(Box::new(serialize_physical_expr(expr.arg(), codec)?)), }), )), }) @@ -344,10 +327,7 @@ pub fn serialize_physical_expr( Ok(protobuf::PhysicalExprNode { expr_type: Some(protobuf::physical_expr_node::ExprType::IsNotNullExpr( Box::new(protobuf::PhysicalIsNotNull { - expr: Some(Box::new(serialize_physical_expr( - expr.arg().to_owned(), - codec, - )?)), + expr: Some(Box::new(serialize_physical_expr(expr.arg(), codec)?)), }), )), }) @@ -355,11 +335,8 @@ pub fn serialize_physical_expr( Ok(protobuf::PhysicalExprNode { expr_type: Some(protobuf::physical_expr_node::ExprType::InList(Box::new( protobuf::PhysicalInListNode { - expr: Some(Box::new(serialize_physical_expr( - expr.expr().to_owned(), - codec, - )?)), - list: serialize_physical_exprs(expr.list().to_vec(), codec)?, + expr: Some(Box::new(serialize_physical_expr(expr.expr(), codec)?)), + list: serialize_physical_exprs(expr.list(), codec)?, negated: expr.negated(), }, ))), @@ -368,10 +345,7 @@ pub fn serialize_physical_expr( Ok(protobuf::PhysicalExprNode { expr_type: Some(protobuf::physical_expr_node::ExprType::Negative(Box::new( protobuf::PhysicalNegativeNode { - expr: Some(Box::new(serialize_physical_expr( - expr.arg().to_owned(), - codec, - )?)), + expr: Some(Box::new(serialize_physical_expr(expr.arg(), codec)?)), }, ))), }) @@ -385,10 +359,7 @@ pub fn serialize_physical_expr( Ok(protobuf::PhysicalExprNode { expr_type: Some(protobuf::physical_expr_node::ExprType::Cast(Box::new( protobuf::PhysicalCastNode { - expr: Some(Box::new(serialize_physical_expr( - cast.expr().to_owned(), - codec, - )?)), + expr: Some(Box::new(serialize_physical_expr(cast.expr(), codec)?)), arrow_type: Some(cast.cast_type().try_into()?), }, ))), @@ -397,10 +368,7 @@ pub fn serialize_physical_expr( Ok(protobuf::PhysicalExprNode { expr_type: Some(protobuf::physical_expr_node::ExprType::TryCast(Box::new( protobuf::PhysicalTryCastNode { - expr: Some(Box::new(serialize_physical_expr( - cast.expr().to_owned(), - codec, - )?)), + expr: Some(Box::new(serialize_physical_expr(cast.expr(), codec)?)), arrow_type: Some(cast.cast_type().try_into()?), }, ))), @@ -412,7 +380,7 @@ pub fn serialize_physical_expr( expr_type: Some(protobuf::physical_expr_node::ExprType::ScalarUdf( protobuf::PhysicalScalarUdfNode { name: expr.name().to_string(), - args: serialize_physical_exprs(expr.args().to_vec(), codec)?, + args: serialize_physical_exprs(expr.args(), codec)?, fun_definition: (!buf.is_empty()).then_some(buf), return_type: Some(expr.return_type().try_into()?), }, @@ -424,12 +392,9 @@ pub fn serialize_physical_expr( protobuf::PhysicalLikeExprNode { negated: expr.negated(), case_insensitive: expr.case_insensitive(), - expr: Some(Box::new(serialize_physical_expr( - expr.expr().to_owned(), - codec, - )?)), + expr: Some(Box::new(serialize_physical_expr(expr.expr(), codec)?)), pattern: Some(Box::new(serialize_physical_expr( - expr.pattern().to_owned(), + expr.pattern(), codec, )?)), }, @@ -437,12 +402,12 @@ pub fn serialize_physical_expr( }) } else { let mut buf: Vec = vec![]; - match codec.try_encode_expr(Arc::clone(&value), &mut buf) { + match codec.try_encode_expr(value, &mut buf) { Ok(_) => { let inputs: Vec = value .children() .into_iter() - .map(|e| serialize_physical_expr(Arc::clone(e), codec)) + .map(|e| serialize_physical_expr(e, codec)) .collect::>()?; Ok(protobuf::PhysicalExprNode { expr_type: Some(protobuf::physical_expr_node::ExprType::Extension( @@ -468,7 +433,7 @@ pub fn serialize_partitioning( )), }, Partitioning::Hash(exprs, partition_count) => { - let serialized_exprs = serialize_physical_exprs(exprs.clone(), codec)?; + let serialized_exprs = serialize_physical_exprs(exprs, codec)?; protobuf::Partitioning { partition_method: Some(protobuf::partitioning::PartitionMethod::Hash( protobuf::PhysicalHashRepartition { @@ -493,8 +458,8 @@ fn serialize_when_then_expr( codec: &dyn PhysicalExtensionCodec, ) -> Result { Ok(protobuf::PhysicalWhenThen { - when_expr: Some(serialize_physical_expr(Arc::clone(when_expr), codec)?), - then_expr: Some(serialize_physical_expr(Arc::clone(then_expr), codec)?), + when_expr: Some(serialize_physical_expr(when_expr, codec)?), + then_expr: Some(serialize_physical_expr(then_expr, codec)?), }) } @@ -608,7 +573,7 @@ pub fn serialize_maybe_filter( match expr { None => Ok(protobuf::MaybeFilter { expr: None }), Some(expr) => Ok(protobuf::MaybeFilter { - expr: Some(serialize_physical_expr(expr, codec)?), + expr: Some(serialize_physical_expr(&expr, codec)?), }), } } diff --git a/datafusion/proto/tests/cases/roundtrip_physical_plan.rs b/datafusion/proto/tests/cases/roundtrip_physical_plan.rs index 3e49dc24fd5a..b2ded88dfaf4 100644 --- a/datafusion/proto/tests/cases/roundtrip_physical_plan.rs +++ b/datafusion/proto/tests/cases/roundtrip_physical_plan.rs @@ -821,11 +821,10 @@ fn roundtrip_parquet_exec_with_custom_predicate_expr() -> Result<()> { fn try_encode_expr( &self, - node: Arc, + node: &Arc, buf: &mut Vec, ) -> Result<()> { if node - .as_ref() .as_any() .downcast_ref::() .is_some() From 650dfdc96a6e4e641befd520580cfdac22ff9902 Mon Sep 17 00:00:00 2001 From: Edmondo Porcu Date: Thu, 29 Aug 2024 12:13:36 -0400 Subject: [PATCH 049/154] Adding alias to custom udf (#12219) --- datafusion/proto/tests/cases/mod.rs | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/datafusion/proto/tests/cases/mod.rs b/datafusion/proto/tests/cases/mod.rs index 1f837b7f42e8..fbb2cd8f1e83 100644 --- a/datafusion/proto/tests/cases/mod.rs +++ b/datafusion/proto/tests/cases/mod.rs @@ -34,12 +34,17 @@ struct MyRegexUdf { signature: Signature, // regex as original string pattern: String, + aliases: Vec, } impl MyRegexUdf { fn new(pattern: String) -> Self { let signature = Signature::exact(vec![DataType::Utf8], Volatility::Immutable); - Self { signature, pattern } + Self { + signature, + pattern, + aliases: vec!["aggregate_udf_alias".to_string()], + } } } @@ -67,6 +72,9 @@ impl ScalarUDFImpl for MyRegexUdf { ) -> datafusion_common::Result { unimplemented!() } + fn aliases(&self) -> &[String] { + &self.aliases + } } #[derive(Clone, PartialEq, ::prost::Message)] From b691b35f6904942c3544ae2af80aef927dfc8d11 Mon Sep 17 00:00:00 2001 From: Austin Liu Date: Fri, 30 Aug 2024 09:25:07 +0800 Subject: [PATCH 050/154] Minor: fix link title (#12246) Signed-off-by: Austin Liu --- docs/source/user-guide/sql/scalar_functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/source/user-guide/sql/scalar_functions.md b/docs/source/user-guide/sql/scalar_functions.md index 9569f4e65ff3..47e35d2e72e3 100644 --- a/docs/source/user-guide/sql/scalar_functions.md +++ b/docs/source/user-guide/sql/scalar_functions.md @@ -3262,7 +3262,7 @@ _Alias of [array_distance](#array_distance)._ ### `list_distinct` -_Alias of [array_dims](#array_distinct)._ +_Alias of [array_distinct](#array_distinct)._ ### `list_element` From f9b14693228cfefdbf7bbc7f8a41179c2c0bcd64 Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Fri, 30 Aug 2024 04:02:29 +0200 Subject: [PATCH 051/154] Remove TableSource::supports_filter_pushdown function (#12239) It was deprecated since 20.0.0. --- datafusion/expr/src/table_source.rs | 18 +++------------ .../src/analyzer/inline_table_scan.rs | 22 +++++++++++-------- datafusion/optimizer/src/push_down_filter.rs | 10 +++++---- 3 files changed, 22 insertions(+), 28 deletions(-) diff --git a/datafusion/expr/src/table_source.rs b/datafusion/expr/src/table_source.rs index 8b8d2dfcf2df..bdb602d48dee 100644 --- a/datafusion/expr/src/table_source.rs +++ b/datafusion/expr/src/table_source.rs @@ -98,27 +98,15 @@ pub trait TableSource: Sync + Send { TableType::Base } - /// Tests whether the table provider can make use of a filter expression - /// to optimise data retrieval. - #[deprecated(since = "20.0.0", note = "use supports_filters_pushdown instead")] - fn supports_filter_pushdown( - &self, - _filter: &Expr, - ) -> Result { - Ok(TableProviderFilterPushDown::Unsupported) - } - /// Tests whether the table provider can make use of any or all filter expressions /// to optimise data retrieval. - #[allow(deprecated)] fn supports_filters_pushdown( &self, filters: &[&Expr], ) -> Result> { - filters - .iter() - .map(|f| self.supports_filter_pushdown(f)) - .collect() + Ok((0..filters.len()) + .map(|_| TableProviderFilterPushDown::Unsupported) + .collect()) } /// Get the Logical plan of this table provider, if available. diff --git a/datafusion/optimizer/src/analyzer/inline_table_scan.rs b/datafusion/optimizer/src/analyzer/inline_table_scan.rs index d5b3648725b9..2fc9b91da6cb 100644 --- a/datafusion/optimizer/src/analyzer/inline_table_scan.rs +++ b/datafusion/optimizer/src/analyzer/inline_table_scan.rs @@ -109,7 +109,7 @@ mod tests { use crate::test::assert_analyzed_plan_eq; use arrow::datatypes::{DataType, Field, Schema}; - use datafusion_expr::{col, lit, LogicalPlan, LogicalPlanBuilder, TableSource}; + use datafusion_expr::{col, lit, Expr, LogicalPlan, LogicalPlanBuilder, TableSource}; pub struct RawTableSource {} @@ -125,12 +125,14 @@ mod tests { ])) } - fn supports_filter_pushdown( + fn supports_filters_pushdown( &self, - _filter: &datafusion_expr::Expr, - ) -> datafusion_common::Result + filters: &[&Expr], + ) -> datafusion_common::Result> { - Ok(datafusion_expr::TableProviderFilterPushDown::Inexact) + Ok((0..filters.len()) + .map(|_| datafusion_expr::TableProviderFilterPushDown::Inexact) + .collect()) } } @@ -154,12 +156,14 @@ mod tests { self } - fn supports_filter_pushdown( + fn supports_filters_pushdown( &self, - _filter: &datafusion_expr::Expr, - ) -> datafusion_common::Result + filters: &[&Expr], + ) -> datafusion_common::Result> { - Ok(datafusion_expr::TableProviderFilterPushDown::Exact) + Ok((0..filters.len()) + .map(|_| datafusion_expr::TableProviderFilterPushDown::Exact) + .collect()) } fn schema(&self) -> arrow::datatypes::SchemaRef { diff --git a/datafusion/optimizer/src/push_down_filter.rs b/datafusion/optimizer/src/push_down_filter.rs index 33a58a810b08..0625652a6be1 100644 --- a/datafusion/optimizer/src/push_down_filter.rs +++ b/datafusion/optimizer/src/push_down_filter.rs @@ -2416,11 +2416,13 @@ mod tests { TableType::Base } - fn supports_filter_pushdown( + fn supports_filters_pushdown( &self, - _e: &Expr, - ) -> Result { - Ok(self.filter_support.clone()) + filters: &[&Expr], + ) -> Result> { + Ok((0..filters.len()) + .map(|_| self.filter_support.clone()) + .collect()) } fn as_any(&self) -> &dyn std::any::Any { From 94d178ebe9674669b32ecd7896b5597f49e90791 Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Fri, 30 Aug 2024 04:08:12 +0200 Subject: [PATCH 052/154] Remove Box from Sort (#12207) `expr::Sort` had `Box` because Sort was also an expression (via `expr::Expr::Sort`). This has been removed, obsoleting need to use a `Box`. --- datafusion/core/src/datasource/mod.rs | 2 +- datafusion/core/tests/dataframe/mod.rs | 20 +++++++++---------- .../tests/user_defined/user_defined_plan.rs | 2 +- datafusion/expr/src/expr.rs | 6 +++--- datafusion/expr/src/expr_rewriter/mod.rs | 4 ++-- datafusion/expr/src/expr_rewriter/order_by.rs | 2 +- datafusion/expr/src/logical_plan/builder.rs | 8 ++++---- datafusion/expr/src/logical_plan/plan.rs | 2 +- datafusion/expr/src/logical_plan/tree_node.rs | 2 +- datafusion/expr/src/tree_node.rs | 8 ++++---- datafusion/expr/src/utils.rs | 16 +++++++-------- .../src/analyzer/count_wildcard_rule.rs | 2 +- .../optimizer/src/common_subexpr_eliminate.rs | 3 +-- .../proto/src/logical_plan/from_proto.rs | 7 +------ datafusion/proto/src/logical_plan/to_proto.rs | 2 +- datafusion/sql/src/expr/function.rs | 2 +- datafusion/sql/src/expr/order_by.rs | 2 +- datafusion/sql/src/unparser/expr.rs | 2 +- datafusion/sql/src/unparser/rewrite.rs | 2 +- .../substrait/src/logical_plan/consumer.rs | 2 +- .../substrait/src/logical_plan/producer.rs | 9 +-------- 21 files changed, 46 insertions(+), 59 deletions(-) diff --git a/datafusion/core/src/datasource/mod.rs b/datafusion/core/src/datasource/mod.rs index 55e88e572be1..529bb799e23d 100644 --- a/datafusion/core/src/datasource/mod.rs +++ b/datafusion/core/src/datasource/mod.rs @@ -63,7 +63,7 @@ fn create_ordering( // Construct PhysicalSortExpr objects from Expr objects: let mut sort_exprs = vec![]; for sort in exprs { - match sort.expr.as_ref() { + match &sort.expr { Expr::Column(col) => match expressions::col(&col.name, schema) { Ok(expr) => { sort_exprs.push(PhysicalSortExpr { diff --git a/datafusion/core/tests/dataframe/mod.rs b/datafusion/core/tests/dataframe/mod.rs index c5b9db7588e9..19ce9294cfad 100644 --- a/datafusion/core/tests/dataframe/mod.rs +++ b/datafusion/core/tests/dataframe/mod.rs @@ -184,7 +184,7 @@ async fn test_count_wildcard_on_window() -> Result<()> { WindowFunctionDefinition::AggregateUDF(count_udaf()), vec![wildcard()], )) - .order_by(vec![Sort::new(Box::new(col("a")), false, true)]) + .order_by(vec![Sort::new(col("a"), false, true)]) .window_frame(WindowFrame::new_bounds( WindowFrameUnits::Range, WindowFrameBound::Preceding(ScalarValue::UInt32(Some(6))), @@ -352,7 +352,7 @@ async fn sort_on_unprojected_columns() -> Result<()> { .unwrap() .select(vec![col("a")]) .unwrap() - .sort(vec![Sort::new(Box::new(col("b")), false, true)]) + .sort(vec![Sort::new(col("b"), false, true)]) .unwrap(); let results = df.collect().await.unwrap(); @@ -396,7 +396,7 @@ async fn sort_on_distinct_columns() -> Result<()> { .unwrap() .distinct() .unwrap() - .sort(vec![Sort::new(Box::new(col("a")), false, true)]) + .sort(vec![Sort::new(col("a"), false, true)]) .unwrap(); let results = df.collect().await.unwrap(); @@ -435,7 +435,7 @@ async fn sort_on_distinct_unprojected_columns() -> Result<()> { .await? .select(vec![col("a")])? .distinct()? - .sort(vec![Sort::new(Box::new(col("b")), false, true)]) + .sort(vec![Sort::new(col("b"), false, true)]) .unwrap_err(); assert_eq!(err.strip_backtrace(), "Error during planning: For SELECT DISTINCT, ORDER BY expressions b must appear in select list"); Ok(()) @@ -599,8 +599,8 @@ async fn test_grouping_sets() -> Result<()> { .await? .aggregate(vec![grouping_set_expr], vec![count(col("a"))])? .sort(vec![ - Sort::new(Box::new(col("a")), false, true), - Sort::new(Box::new(col("b")), false, true), + Sort::new(col("a"), false, true), + Sort::new(col("b"), false, true), ])?; let results = df.collect().await?; @@ -640,8 +640,8 @@ async fn test_grouping_sets_count() -> Result<()> { .await? .aggregate(vec![grouping_set_expr], vec![count(lit(1))])? .sort(vec![ - Sort::new(Box::new(col("c1")), false, true), - Sort::new(Box::new(col("c2")), false, true), + Sort::new(col("c1"), false, true), + Sort::new(col("c2"), false, true), ])?; let results = df.collect().await?; @@ -687,8 +687,8 @@ async fn test_grouping_set_array_agg_with_overflow() -> Result<()> { ], )? .sort(vec![ - Sort::new(Box::new(col("c1")), false, true), - Sort::new(Box::new(col("c2")), false, true), + Sort::new(col("c1"), false, true), + Sort::new(col("c2"), false, true), ])?; let results = df.collect().await?; diff --git a/datafusion/core/tests/user_defined/user_defined_plan.rs b/datafusion/core/tests/user_defined/user_defined_plan.rs index da27cf8869d1..56edeab443c7 100644 --- a/datafusion/core/tests/user_defined/user_defined_plan.rs +++ b/datafusion/core/tests/user_defined/user_defined_plan.rs @@ -419,7 +419,7 @@ impl UserDefinedLogicalNodeCore for TopKPlanNode { } fn expressions(&self) -> Vec { - vec![self.expr.expr.as_ref().clone()] + vec![self.expr.expr.clone()] } /// For example: `TopK: k=10` diff --git a/datafusion/expr/src/expr.rs b/datafusion/expr/src/expr.rs index b81c02ccd0b7..8914214d084f 100644 --- a/datafusion/expr/src/expr.rs +++ b/datafusion/expr/src/expr.rs @@ -602,7 +602,7 @@ impl TryCast { #[derive(Clone, PartialEq, Eq, Hash, Debug)] pub struct Sort { /// The expression to sort on - pub expr: Box, + pub expr: Expr, /// The direction of the sort pub asc: bool, /// Whether to put Nulls before all other data values @@ -611,7 +611,7 @@ pub struct Sort { impl Sort { /// Create a new Sort expression - pub fn new(expr: Box, asc: bool, nulls_first: bool) -> Self { + pub fn new(expr: Expr, asc: bool, nulls_first: bool) -> Self { Self { expr, asc, @@ -1368,7 +1368,7 @@ impl Expr { /// let sort_expr = col("foo").sort(true, true); // SORT ASC NULLS_FIRST /// ``` pub fn sort(self, asc: bool, nulls_first: bool) -> Sort { - Sort::new(Box::new(self), asc, nulls_first) + Sort::new(self, asc, nulls_first) } /// Return `IsTrue(Box(self))` diff --git a/datafusion/expr/src/expr_rewriter/mod.rs b/datafusion/expr/src/expr_rewriter/mod.rs index 5e7fedb4cbd8..61b0f6d9bb2b 100644 --- a/datafusion/expr/src/expr_rewriter/mod.rs +++ b/datafusion/expr/src/expr_rewriter/mod.rs @@ -125,8 +125,8 @@ pub fn normalize_sorts( .into_iter() .map(|e| { let sort = e.into(); - normalize_col(*sort.expr, plan) - .map(|expr| Sort::new(Box::new(expr), sort.asc, sort.nulls_first)) + normalize_col(sort.expr, plan) + .map(|expr| Sort::new(expr, sort.asc, sort.nulls_first)) }) .collect() } diff --git a/datafusion/expr/src/expr_rewriter/order_by.rs b/datafusion/expr/src/expr_rewriter/order_by.rs index af5b8c4f9177..48d380cd59e2 100644 --- a/datafusion/expr/src/expr_rewriter/order_by.rs +++ b/datafusion/expr/src/expr_rewriter/order_by.rs @@ -35,7 +35,7 @@ pub fn rewrite_sort_cols_by_aggs( .map(|e| { let sort = e.into(); Ok(Sort::new( - Box::new(rewrite_sort_col_by_aggs(*sort.expr, plan)?), + rewrite_sort_col_by_aggs(sort.expr, plan)?, sort.asc, sort.nulls_first, )) diff --git a/datafusion/expr/src/logical_plan/builder.rs b/datafusion/expr/src/logical_plan/builder.rs index f5770167861b..fc961b83f7b5 100644 --- a/datafusion/expr/src/logical_plan/builder.rs +++ b/datafusion/expr/src/logical_plan/builder.rs @@ -1720,8 +1720,8 @@ mod tests { let plan = table_scan(Some("employee_csv"), &employee_schema(), Some(vec![3, 4]))? .sort(vec![ - expr::Sort::new(Box::new(col("state")), true, true), - expr::Sort::new(Box::new(col("salary")), false, false), + expr::Sort::new(col("state"), true, true), + expr::Sort::new(col("salary"), false, false), ])? .build()?; @@ -2147,8 +2147,8 @@ mod tests { let plan = table_scan(Some("employee_csv"), &employee_schema(), Some(vec![3, 4]))? .sort(vec![ - expr::Sort::new(Box::new(col("state")), true, true), - expr::Sort::new(Box::new(col("salary")), false, false), + expr::Sort::new(col("state"), true, true), + expr::Sort::new(col("salary"), false, false), ])? .build()?; diff --git a/datafusion/expr/src/logical_plan/plan.rs b/datafusion/expr/src/logical_plan/plan.rs index 8e6ec762f549..5bd6ab10331a 100644 --- a/datafusion/expr/src/logical_plan/plan.rs +++ b/datafusion/expr/src/logical_plan/plan.rs @@ -2616,7 +2616,7 @@ impl DistinctOn { // Check that the left-most sort expressions are the same as the `ON` expressions. let mut matched = true; for (on, sort) in self.on_expr.iter().zip(sort_expr.iter()) { - if on != &*sort.expr { + if on != &sort.expr { matched = false; break; } diff --git a/datafusion/expr/src/logical_plan/tree_node.rs b/datafusion/expr/src/logical_plan/tree_node.rs index 29a99a8e8886..0964fb601879 100644 --- a/datafusion/expr/src/logical_plan/tree_node.rs +++ b/datafusion/expr/src/logical_plan/tree_node.rs @@ -509,7 +509,7 @@ impl LogicalPlan { })) => on_expr .iter() .chain(select_expr.iter()) - .chain(sort_expr.iter().flatten().map(|sort| &*sort.expr)) + .chain(sort_expr.iter().flatten().map(|sort| &sort.expr)) .apply_until_stop(f), // plans without expressions LogicalPlan::EmptyRelation(_) diff --git a/datafusion/expr/src/tree_node.rs b/datafusion/expr/src/tree_node.rs index 90d61bf63763..c7c498dd3f01 100644 --- a/datafusion/expr/src/tree_node.rs +++ b/datafusion/expr/src/tree_node.rs @@ -97,7 +97,7 @@ impl TreeNode for Expr { expr_vec.push(f.as_ref()); } if let Some(order_by) = order_by { - expr_vec.extend(order_by.iter().map(|sort| sort.expr.as_ref())); + expr_vec.extend(order_by.iter().map(|sort| &sort.expr)); } expr_vec } @@ -109,7 +109,7 @@ impl TreeNode for Expr { }) => { let mut expr_vec = args.iter().collect::>(); expr_vec.extend(partition_by); - expr_vec.extend(order_by.iter().map(|sort| sort.expr.as_ref())); + expr_vec.extend(order_by.iter().map(|sort| &sort.expr)); expr_vec } Expr::InList(InList { expr, list, .. }) => { @@ -395,7 +395,7 @@ pub fn transform_sort_vec Result>>( ) -> Result>> { Ok(sorts .iter() - .map(|sort| (*sort.expr).clone()) + .map(|sort| sort.expr.clone()) .map_until_stop_and_collect(&mut f)? .update_data(|transformed_exprs| { replace_sort_expressions(sorts, transformed_exprs) @@ -413,7 +413,7 @@ pub fn replace_sort_expressions(sorts: Vec, new_expr: Vec) -> Vec Sort { Sort { - expr: Box::new(new_expr), + expr: new_expr, ..sort } } diff --git a/datafusion/expr/src/utils.rs b/datafusion/expr/src/utils.rs index b6b1b5660a81..c4c6b076e5ba 100644 --- a/datafusion/expr/src/utils.rs +++ b/datafusion/expr/src/utils.rs @@ -1401,9 +1401,9 @@ mod tests { #[test] fn test_group_window_expr_by_sort_keys() -> Result<()> { - let age_asc = expr::Sort::new(Box::new(col("age")), true, true); - let name_desc = expr::Sort::new(Box::new(col("name")), false, true); - let created_at_desc = expr::Sort::new(Box::new(col("created_at")), false, true); + let age_asc = expr::Sort::new(col("age"), true, true); + let name_desc = expr::Sort::new(col("name"), false, true); + let created_at_desc = expr::Sort::new(col("created_at"), false, true); let max1 = Expr::WindowFunction(expr::WindowFunction::new( WindowFunctionDefinition::AggregateUDF(max_udaf()), vec![col("name")], @@ -1463,12 +1463,12 @@ mod tests { for nulls_first_ in nulls_first_or_last { let order_by = &[ Sort { - expr: Box::new(col("age")), + expr: col("age"), asc: asc_, nulls_first: nulls_first_, }, Sort { - expr: Box::new(col("name")), + expr: col("name"), asc: asc_, nulls_first: nulls_first_, }, @@ -1477,7 +1477,7 @@ mod tests { let expected = vec![ ( Sort { - expr: Box::new(col("age")), + expr: col("age"), asc: asc_, nulls_first: nulls_first_, }, @@ -1485,7 +1485,7 @@ mod tests { ), ( Sort { - expr: Box::new(col("name")), + expr: col("name"), asc: asc_, nulls_first: nulls_first_, }, @@ -1493,7 +1493,7 @@ mod tests { ), ( Sort { - expr: Box::new(col("created_at")), + expr: col("created_at"), asc: true, nulls_first: false, }, diff --git a/datafusion/optimizer/src/analyzer/count_wildcard_rule.rs b/datafusion/optimizer/src/analyzer/count_wildcard_rule.rs index 35d4f91e3b6f..0036f6df43f6 100644 --- a/datafusion/optimizer/src/analyzer/count_wildcard_rule.rs +++ b/datafusion/optimizer/src/analyzer/count_wildcard_rule.rs @@ -229,7 +229,7 @@ mod tests { WindowFunctionDefinition::AggregateUDF(count_udaf()), vec![wildcard()], )) - .order_by(vec![Sort::new(Box::new(col("a")), false, true)]) + .order_by(vec![Sort::new(col("a"), false, true)]) .window_frame(WindowFrame::new_bounds( WindowFrameUnits::Range, WindowFrameBound::Preceding(ScalarValue::UInt32(Some(6))), diff --git a/datafusion/optimizer/src/common_subexpr_eliminate.rs b/datafusion/optimizer/src/common_subexpr_eliminate.rs index 25bef7e2d0e4..22e9d220d324 100644 --- a/datafusion/optimizer/src/common_subexpr_eliminate.rs +++ b/datafusion/optimizer/src/common_subexpr_eliminate.rs @@ -328,8 +328,7 @@ impl CommonSubexprEliminate { ) -> Result> { let Sort { expr, input, fetch } = sort; let input = Arc::unwrap_or_clone(input); - let sort_expressions = - expr.iter().map(|sort| sort.expr.as_ref().clone()).collect(); + let sort_expressions = expr.iter().map(|sort| sort.expr.clone()).collect(); let new_sort = self .try_unary_plan(sort_expressions, input, config)? .update_data(|(new_expr, new_input)| { diff --git a/datafusion/proto/src/logical_plan/from_proto.rs b/datafusion/proto/src/logical_plan/from_proto.rs index 3ba1cb945e9c..893255ccc8ce 100644 --- a/datafusion/proto/src/logical_plan/from_proto.rs +++ b/datafusion/proto/src/logical_plan/from_proto.rs @@ -645,12 +645,7 @@ pub fn parse_sort( codec: &dyn LogicalExtensionCodec, ) -> Result { Ok(Sort::new( - Box::new(parse_required_expr( - sort.expr.as_ref(), - registry, - "expr", - codec, - )?), + parse_required_expr(sort.expr.as_ref(), registry, "expr", codec)?, sort.asc, sort.nulls_first, )) diff --git a/datafusion/proto/src/logical_plan/to_proto.rs b/datafusion/proto/src/logical_plan/to_proto.rs index b937c03f79d9..63d1a007c1e5 100644 --- a/datafusion/proto/src/logical_plan/to_proto.rs +++ b/datafusion/proto/src/logical_plan/to_proto.rs @@ -637,7 +637,7 @@ where nulls_first, } = sort; Ok(protobuf::SortExprNode { - expr: Some(serialize_expr(expr.as_ref(), codec)?), + expr: Some(serialize_expr(expr, codec)?), asc: *asc, nulls_first: *nulls_first, }) diff --git a/datafusion/sql/src/expr/function.rs b/datafusion/sql/src/expr/function.rs index 9c768eb73c2e..190a7e918928 100644 --- a/datafusion/sql/src/expr/function.rs +++ b/datafusion/sql/src/expr/function.rs @@ -282,7 +282,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { let func_deps = schema.functional_dependencies(); // Find whether ties are possible in the given ordering let is_ordering_strict = order_by.iter().find_map(|orderby_expr| { - if let Expr::Column(col) = orderby_expr.expr.as_ref() { + if let Expr::Column(col) = &orderby_expr.expr { let idx = schema.index_of_column(col).ok()?; return if func_deps.iter().any(|dep| { dep.source_indices == vec![idx] && dep.mode == Dependency::Single diff --git a/datafusion/sql/src/expr/order_by.rs b/datafusion/sql/src/expr/order_by.rs index cdaa787cedd0..6a3a4d6ccbb7 100644 --- a/datafusion/sql/src/expr/order_by.rs +++ b/datafusion/sql/src/expr/order_by.rs @@ -100,7 +100,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { }; let asc = asc.unwrap_or(true); expr_vec.push(Sort::new( - Box::new(expr), + expr, asc, // when asc is true, by default nulls last to be consistent with postgres // postgres rule: https://www.postgresql.org/docs/current/queries-order.html diff --git a/datafusion/sql/src/unparser/expr.rs b/datafusion/sql/src/unparser/expr.rs index 9a3f139fdee8..549635a31aef 100644 --- a/datafusion/sql/src/unparser/expr.rs +++ b/datafusion/sql/src/unparser/expr.rs @@ -1761,7 +1761,7 @@ mod tests { fun: WindowFunctionDefinition::AggregateUDF(count_udaf()), args: vec![wildcard()], partition_by: vec![], - order_by: vec![Sort::new(Box::new(col("a")), false, true)], + order_by: vec![Sort::new(col("a"), false, true)], window_frame: WindowFrame::new_bounds( datafusion_expr::WindowFrameUnits::Range, datafusion_expr::WindowFrameBound::Preceding( diff --git a/datafusion/sql/src/unparser/rewrite.rs b/datafusion/sql/src/unparser/rewrite.rs index 522a08af8546..2529385849e0 100644 --- a/datafusion/sql/src/unparser/rewrite.rs +++ b/datafusion/sql/src/unparser/rewrite.rs @@ -158,7 +158,7 @@ pub(super) fn rewrite_plan_for_sort_on_non_projected_fields( let mut collects = p.expr.clone(); for sort in &sort.expr { - collects.push(sort.expr.as_ref().clone()); + collects.push(sort.expr.clone()); } // Compare outer collects Expr::to_string with inner collected transformed values diff --git a/datafusion/substrait/src/logical_plan/consumer.rs b/datafusion/substrait/src/logical_plan/consumer.rs index 05903bb56cfe..21bef3c2c98e 100644 --- a/datafusion/substrait/src/logical_plan/consumer.rs +++ b/datafusion/substrait/src/logical_plan/consumer.rs @@ -936,7 +936,7 @@ pub async fn from_substrait_sorts( }; let (asc, nulls_first) = asc_nullfirst.unwrap(); sorts.push(Sort { - expr: Box::new(expr), + expr, asc, nulls_first, }); diff --git a/datafusion/substrait/src/logical_plan/producer.rs b/datafusion/substrait/src/logical_plan/producer.rs index 592390a285ba..e71cf04cd341 100644 --- a/datafusion/substrait/src/logical_plan/producer.rs +++ b/datafusion/substrait/src/logical_plan/producer.rs @@ -16,7 +16,6 @@ // under the License. use itertools::Itertools; -use std::ops::Deref; use std::sync::Arc; use arrow_buffer::ToByteSlice; @@ -819,13 +818,7 @@ fn to_substrait_sort_field( (false, false) => SortDirection::DescNullsLast, }; Ok(SortField { - expr: Some(to_substrait_rex( - ctx, - sort.expr.deref(), - schema, - 0, - extensions, - )?), + expr: Some(to_substrait_rex(ctx, &sort.expr, schema, 0, extensions)?), sort_kind: Some(SortKind::Direction(sort_kind.into())), }) } From e60318553d6ac36b2d07466acebef861fde2936e Mon Sep 17 00:00:00 2001 From: Jonah Gao Date: Fri, 30 Aug 2024 14:08:29 +0800 Subject: [PATCH 053/154] Reuse `NamePreserver` in `SimplifyExpressions` (#12238) --- datafusion/expr/src/expr_rewriter/mod.rs | 2 ++ .../src/simplify_expressions/simplify_exprs.rs | 18 +++++++----------- 2 files changed, 9 insertions(+), 11 deletions(-) diff --git a/datafusion/expr/src/expr_rewriter/mod.rs b/datafusion/expr/src/expr_rewriter/mod.rs index 61b0f6d9bb2b..35c573836419 100644 --- a/datafusion/expr/src/expr_rewriter/mod.rs +++ b/datafusion/expr/src/expr_rewriter/mod.rs @@ -311,6 +311,8 @@ impl NamePreserver { /// Create a new NamePreserver for rewriting the `expr` that is part of the specified plan pub fn new(plan: &LogicalPlan) -> Self { Self { + // The schema of Filter and Join nodes comes from their inputs rather than their output expressions, + // so there is no need to use aliases to preserve expression names. use_alias: !matches!(plan, LogicalPlan::Filter(_) | LogicalPlan::Join(_)), } } diff --git a/datafusion/optimizer/src/simplify_expressions/simplify_exprs.rs b/datafusion/optimizer/src/simplify_expressions/simplify_exprs.rs index e13ce382a3e0..1c39313ea132 100644 --- a/datafusion/optimizer/src/simplify_expressions/simplify_exprs.rs +++ b/datafusion/optimizer/src/simplify_expressions/simplify_exprs.rs @@ -27,6 +27,7 @@ use datafusion_expr::simplify::SimplifyContext; use datafusion_expr::utils::merge_schema; use crate::optimizer::ApplyOrder; +use crate::utils::NamePreserver; use crate::{OptimizerConfig, OptimizerRule}; use super::ExprSimplifier; @@ -119,18 +120,13 @@ impl SimplifyExpressions { simplifier }; - // the output schema of a filter or join is the input schema. Thus they - // can't handle aliased expressions - let use_alias = !matches!(plan, LogicalPlan::Filter(_) | LogicalPlan::Join(_)); + // Preserve expression names to avoid changing the schema of the plan. + let name_preserver = NamePreserver::new(&plan); plan.map_expressions(|e| { - let new_e = if use_alias { - // TODO: unify with `rewrite_preserving_name` - let original_name = e.name_for_alias()?; - simplifier.simplify(e)?.alias_if_changed(original_name) - } else { - simplifier.simplify(e) - }?; - + let original_name = name_preserver.save(&e)?; + let new_e = simplifier + .simplify(e) + .and_then(|expr| original_name.restore(expr))?; // TODO it would be nice to have a way to know if the expression was simplified // or not. For now conservatively return Transformed::yes Ok(Transformed::yes(new_e)) From 88dd30504cbe9b9e4059f592d8688029c6d35b22 Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Fri, 30 Aug 2024 23:19:55 +0200 Subject: [PATCH 054/154] Remove redundant argument and improve error message (#12217) * Remove redundant argument and improve error message * Update datafusion/functions/src/string/common.rs Co-authored-by: Oleks V * Update datafusion/functions/src/string/common.rs Co-authored-by: Oleks V --------- Co-authored-by: Oleks V --- datafusion/functions/src/string/common.rs | 10 ++++------ 1 file changed, 4 insertions(+), 6 deletions(-) diff --git a/datafusion/functions/src/string/common.rs b/datafusion/functions/src/string/common.rs index a5dc22b4d9e4..9738cb812f90 100644 --- a/datafusion/functions/src/string/common.rs +++ b/datafusion/functions/src/string/common.rs @@ -73,15 +73,14 @@ pub(crate) fn general_trim( }; if use_string_view { - string_view_trim::(trim_type, func, args) + string_view_trim::(func, args) } else { - string_trim::(trim_type, func, args) + string_trim::(func, args) } } // removing 'a will cause compiler complaining lifetime of `func` fn string_view_trim<'a, T: OffsetSizeTrait>( - trim_type: TrimType, func: fn(&'a str, &'a str) -> &'a str, args: &'a [ArrayRef], ) -> Result { @@ -129,14 +128,13 @@ fn string_view_trim<'a, T: OffsetSizeTrait>( } other => { exec_err!( - "{trim_type} was called with {other} arguments. It requires at least 1 and at most 2." + "Function TRIM was called with {other} arguments. It requires at least 1 and at most 2." ) } } } fn string_trim<'a, T: OffsetSizeTrait>( - trim_type: TrimType, func: fn(&'a str, &'a str) -> &'a str, args: &'a [ArrayRef], ) -> Result { @@ -183,7 +181,7 @@ fn string_trim<'a, T: OffsetSizeTrait>( } other => { exec_err!( - "{trim_type} was called with {other} arguments. It requires at least 1 and at most 2." + "Function TRIM was called with {other} arguments. It requires at least 1 and at most 2." ) } } From 442d9bfcc5f8706a15e92f16a6fbc5c108c97a78 Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Sat, 31 Aug 2024 23:18:54 +0200 Subject: [PATCH 055/154] Remove deprecated from_plan function (#12265) * Remove deprecated from_plan function Deprecated since 31.0.0. * empty --- datafusion/expr/src/utils.rs | 31 ------------------------------- 1 file changed, 31 deletions(-) diff --git a/datafusion/expr/src/utils.rs b/datafusion/expr/src/utils.rs index c4c6b076e5ba..9b64bca8121e 100644 --- a/datafusion/expr/src/utils.rs +++ b/datafusion/expr/src/utils.rs @@ -679,37 +679,6 @@ where err } -/// Returns a new logical plan based on the original one with inputs -/// and expressions replaced. -/// -/// The exprs correspond to the same order of expressions returned by -/// `LogicalPlan::expressions`. This function is used in optimizers in -/// the following way: -/// -/// ```text -/// let new_inputs = optimize_children(..., plan, props); -/// -/// // get the plans expressions to optimize -/// let exprs = plan.expressions(); -/// -/// // potentially rewrite plan expressions -/// let rewritten_exprs = rewrite_exprs(exprs); -/// -/// // create new plan using rewritten_exprs in same position -/// let new_plan = from_plan(&plan, rewritten_exprs, new_inputs); -/// ``` -/// -/// Notice: sometimes [from_plan] will use schema of original plan, it don't change schema! -/// Such as `Projection/Aggregate/Window` -#[deprecated(since = "31.0.0", note = "use LogicalPlan::with_new_exprs instead")] -pub fn from_plan( - plan: &LogicalPlan, - expr: &[Expr], - inputs: &[LogicalPlan], -) -> Result { - plan.with_new_exprs(expr.to_vec(), inputs.to_vec()) -} - /// Create field meta-data from an expression, for use in a result set schema pub fn exprlist_to_fields<'a>( exprs: impl IntoIterator, From 016ed03cf18c3f60594cf6f8fedd202561c27a91 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=BC=A0=E6=9E=97=E4=BC=9F?= Date: Sun, 1 Sep 2024 15:58:53 +0800 Subject: [PATCH 056/154] Remove redundant result of AggregateFunctionExpr::field (#12258) --- datafusion/physical-expr/src/aggregate.rs | 12 +++--------- datafusion/physical-expr/src/window/aggregate.rs | 4 ++-- .../physical-expr/src/window/sliding_aggregate.rs | 4 ++-- .../physical-optimizer/src/aggregate_statistics.rs | 4 ++-- datafusion/physical-plan/src/aggregates/mod.rs | 2 +- datafusion/proto/src/physical_plan/mod.rs | 7 ++----- 6 files changed, 12 insertions(+), 21 deletions(-) diff --git a/datafusion/physical-expr/src/aggregate.rs b/datafusion/physical-expr/src/aggregate.rs index 5c1216f2a386..d62dc27ece86 100644 --- a/datafusion/physical-expr/src/aggregate.rs +++ b/datafusion/physical-expr/src/aggregate.rs @@ -253,12 +253,8 @@ impl AggregateFunctionExpr { } /// the field of the final result of this aggregation. - pub fn field(&self) -> Result { - Ok(Field::new( - &self.name, - self.data_type.clone(), - self.is_nullable, - )) + pub fn field(&self) -> Field { + Field::new(&self.name, self.data_type.clone(), self.is_nullable) } /// the accumulator used to accumulate values from the expressions. @@ -523,9 +519,7 @@ impl AggregateFunctionExpr { /// /// Note: this is used to use special aggregate implementations in certain conditions pub fn get_minmax_desc(&self) -> Option<(Field, bool)> { - self.fun - .is_descending() - .and_then(|flag| self.field().ok().map(|f| (f, flag))) + self.fun.is_descending().map(|flag| (self.field(), flag)) } /// Returns default value of the function given the input is Null diff --git a/datafusion/physical-expr/src/window/aggregate.rs b/datafusion/physical-expr/src/window/aggregate.rs index 5439e140502a..1cc08a4e99aa 100644 --- a/datafusion/physical-expr/src/window/aggregate.rs +++ b/datafusion/physical-expr/src/window/aggregate.rs @@ -79,7 +79,7 @@ impl WindowExpr for PlainAggregateWindowExpr { } fn field(&self) -> Result { - self.aggregate.field() + Ok(self.aggregate.field()) } fn name(&self) -> &str { @@ -177,7 +177,7 @@ impl AggregateWindowExpr for PlainAggregateWindowExpr { ) -> Result { if cur_range.start == cur_range.end { self.aggregate - .default_value(self.aggregate.field()?.data_type()) + .default_value(self.aggregate.field().data_type()) } else { // Accumulate any new rows that have entered the window: let update_bound = cur_range.end - last_range.end; diff --git a/datafusion/physical-expr/src/window/sliding_aggregate.rs b/datafusion/physical-expr/src/window/sliding_aggregate.rs index ac3a4f4c09ec..b3848e15ee42 100644 --- a/datafusion/physical-expr/src/window/sliding_aggregate.rs +++ b/datafusion/physical-expr/src/window/sliding_aggregate.rs @@ -81,7 +81,7 @@ impl WindowExpr for SlidingAggregateWindowExpr { } fn field(&self) -> Result { - self.aggregate.field() + Ok(self.aggregate.field()) } fn name(&self) -> &str { @@ -183,7 +183,7 @@ impl AggregateWindowExpr for SlidingAggregateWindowExpr { ) -> Result { if cur_range.start == cur_range.end { self.aggregate - .default_value(self.aggregate.field()?.data_type()) + .default_value(self.aggregate.field().data_type()) } else { // Accumulate any new rows that have entered the window: let update_bound = cur_range.end - last_range.end; diff --git a/datafusion/physical-optimizer/src/aggregate_statistics.rs b/datafusion/physical-optimizer/src/aggregate_statistics.rs index 2b8725b5bac7..863c5ab2d288 100644 --- a/datafusion/physical-optimizer/src/aggregate_statistics.rs +++ b/datafusion/physical-optimizer/src/aggregate_statistics.rs @@ -183,7 +183,7 @@ fn take_optimizable_min( // MIN/MAX with 0 rows is always null if is_min(agg_expr) { if let Ok(min_data_type) = - ScalarValue::try_from(agg_expr.field().unwrap().data_type()) + ScalarValue::try_from(agg_expr.field().data_type()) { return Some((min_data_type, agg_expr.name().to_string())); } @@ -229,7 +229,7 @@ fn take_optimizable_max( // MIN/MAX with 0 rows is always null if is_max(agg_expr) { if let Ok(max_data_type) = - ScalarValue::try_from(agg_expr.field().unwrap().data_type()) + ScalarValue::try_from(agg_expr.field().data_type()) { return Some((max_data_type, agg_expr.name().to_string())); } diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 0f33a9d7b992..2f974f1ef4e0 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -821,7 +821,7 @@ fn create_schema( | AggregateMode::SinglePartitioned => { // in final mode, the field with the final result of the accumulator for expr in aggr_expr { - fields.push(expr.field()?) + fields.push(expr.field()) } } } diff --git a/datafusion/proto/src/physical_plan/mod.rs b/datafusion/proto/src/physical_plan/mod.rs index 04cbf8b537b3..e622af745062 100644 --- a/datafusion/proto/src/physical_plan/mod.rs +++ b/datafusion/proto/src/physical_plan/mod.rs @@ -1444,11 +1444,8 @@ impl AsExecutionPlan for protobuf::PhysicalPlanNode { let agg_names = exec .aggr_expr() .iter() - .map(|expr| match expr.field() { - Ok(field) => Ok(field.name().clone()), - Err(e) => Err(e), - }) - .collect::>()?; + .map(|expr| expr.name().to_string()) + .collect::>(); let agg_mode = match exec.mode() { AggregateMode::Partial => protobuf::AggregateMode::Partial, From 8746e073b7a7ffcf86fac19d5ea1984ee6970d20 Mon Sep 17 00:00:00 2001 From: Dharan Aditya Date: Sun, 1 Sep 2024 14:10:10 +0530 Subject: [PATCH 057/154] Support `map_keys` & `map_values` for MAP type (#12194) * impl map_keys * rename field name * add logic tests * one more * owned to clone * more tests * typo * impl * add logic tests * chore * add docs * trying to make prettier happy * Update scalar_functions.md Co-authored-by: Alex Huang * reface signature * format docs * Update map_values.rs Co-authored-by: Alex Huang --------- Co-authored-by: Alex Huang --- datafusion/common/src/utils/mod.rs | 17 +-- datafusion/functions-nested/src/lib.rs | 6 ++ .../functions-nested/src/map_extract.rs | 3 +- datafusion/functions-nested/src/map_keys.rs | 102 ++++++++++++++++++ datafusion/functions-nested/src/map_values.rs | 102 ++++++++++++++++++ datafusion/functions-nested/src/utils.rs | 19 +++- datafusion/sqllogictest/test_files/map.slt | 97 ++++++++++++++++- .../source/user-guide/sql/scalar_functions.md | 52 +++++++++ 8 files changed, 377 insertions(+), 21 deletions(-) create mode 100644 datafusion/functions-nested/src/map_keys.rs create mode 100644 datafusion/functions-nested/src/map_values.rs diff --git a/datafusion/common/src/utils/mod.rs b/datafusion/common/src/utils/mod.rs index 839f890bf077..418ea380bc2c 100644 --- a/datafusion/common/src/utils/mod.rs +++ b/datafusion/common/src/utils/mod.rs @@ -35,7 +35,7 @@ use arrow_array::{ Array, FixedSizeListArray, LargeListArray, ListArray, OffsetSizeTrait, RecordBatchOptions, }; -use arrow_schema::{DataType, Fields}; +use arrow_schema::DataType; use sqlparser::ast::Ident; use sqlparser::dialect::GenericDialect; use sqlparser::parser::Parser; @@ -754,21 +754,6 @@ pub fn combine_limit( (combined_skip, combined_fetch) } -pub fn get_map_entry_field(data_type: &DataType) -> Result<&Fields> { - match data_type { - DataType::Map(field, _) => { - let field_data_type = field.data_type(); - match field_data_type { - DataType::Struct(fields) => Ok(fields), - _ => { - _internal_err!("Expected a Struct type, got {:?}", field_data_type) - } - } - } - _ => _internal_err!("Expected a Map type, got {:?}", data_type), - } -} - #[cfg(test)] mod tests { use crate::ScalarValue::Null; diff --git a/datafusion/functions-nested/src/lib.rs b/datafusion/functions-nested/src/lib.rs index 863b5a876adc..b548cf6db8b1 100644 --- a/datafusion/functions-nested/src/lib.rs +++ b/datafusion/functions-nested/src/lib.rs @@ -44,6 +44,8 @@ pub mod length; pub mod make_array; pub mod map; pub mod map_extract; +pub mod map_keys; +pub mod map_values; pub mod planner; pub mod position; pub mod range; @@ -85,6 +87,8 @@ pub mod expr_fn { pub use super::length::array_length; pub use super::make_array::make_array; pub use super::map_extract::map_extract; + pub use super::map_keys::map_keys; + pub use super::map_values::map_values; pub use super::position::array_position; pub use super::position::array_positions; pub use super::range::gen_series; @@ -149,6 +153,8 @@ pub fn all_default_nested_functions() -> Vec> { replace::array_replace_udf(), map::map_udf(), map_extract::map_extract_udf(), + map_keys::map_keys_udf(), + map_values::map_values_udf(), ] } diff --git a/datafusion/functions-nested/src/map_extract.rs b/datafusion/functions-nested/src/map_extract.rs index 82f0d8d6c15e..9f0c4ad29c60 100644 --- a/datafusion/functions-nested/src/map_extract.rs +++ b/datafusion/functions-nested/src/map_extract.rs @@ -24,7 +24,6 @@ use arrow::datatypes::DataType; use arrow_array::{Array, MapArray}; use arrow_buffer::OffsetBuffer; use arrow_schema::Field; -use datafusion_common::utils::get_map_entry_field; use datafusion_common::{cast::as_map_array, exec_err, Result}; use datafusion_expr::{ColumnarValue, ScalarUDFImpl, Signature, Volatility}; @@ -32,7 +31,7 @@ use std::any::Any; use std::sync::Arc; use std::vec; -use crate::utils::make_scalar_function; +use crate::utils::{get_map_entry_field, make_scalar_function}; // Create static instances of ScalarUDFs for each function make_udf_expr_and_func!( diff --git a/datafusion/functions-nested/src/map_keys.rs b/datafusion/functions-nested/src/map_keys.rs new file mode 100644 index 000000000000..0b1cebb27c86 --- /dev/null +++ b/datafusion/functions-nested/src/map_keys.rs @@ -0,0 +1,102 @@ +// 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. + +//! [`ScalarUDFImpl`] definitions for map_keys function. + +use crate::utils::{get_map_entry_field, make_scalar_function}; +use arrow_array::{Array, ArrayRef, ListArray}; +use arrow_schema::{DataType, Field}; +use datafusion_common::{cast::as_map_array, exec_err, Result}; +use datafusion_expr::{ + ArrayFunctionSignature, ColumnarValue, ScalarUDFImpl, Signature, TypeSignature, + Volatility, +}; +use std::any::Any; +use std::sync::Arc; + +make_udf_expr_and_func!( + MapKeysFunc, + map_keys, + map, + "Return a list of all keys in the map.", + map_keys_udf +); + +#[derive(Debug)] +pub(crate) struct MapKeysFunc { + signature: Signature, +} + +impl MapKeysFunc { + pub fn new() -> Self { + Self { + signature: Signature::new( + TypeSignature::ArraySignature(ArrayFunctionSignature::MapArray), + Volatility::Immutable, + ), + } + } +} + +impl ScalarUDFImpl for MapKeysFunc { + fn as_any(&self) -> &dyn Any { + self + } + + fn name(&self) -> &str { + "map_keys" + } + + fn signature(&self) -> &Signature { + &self.signature + } + + fn return_type(&self, arg_types: &[DataType]) -> datafusion_common::Result { + if arg_types.len() != 1 { + return exec_err!("map_keys expects single argument"); + } + let map_type = &arg_types[0]; + let map_fields = get_map_entry_field(map_type)?; + Ok(DataType::List(Arc::new(Field::new( + "item", + map_fields.first().unwrap().data_type().clone(), + false, + )))) + } + + fn invoke(&self, args: &[ColumnarValue]) -> datafusion_common::Result { + make_scalar_function(map_keys_inner)(args) + } +} + +fn map_keys_inner(args: &[ArrayRef]) -> Result { + if args.len() != 1 { + return exec_err!("map_keys expects single argument"); + } + + let map_array = match args[0].data_type() { + DataType::Map(_, _) => as_map_array(&args[0])?, + _ => return exec_err!("Argument for map_keys should be a map"), + }; + + Ok(Arc::new(ListArray::new( + Arc::new(Field::new("item", map_array.key_type().clone(), false)), + map_array.offsets().clone(), + Arc::clone(map_array.keys()), + None, + ))) +} diff --git a/datafusion/functions-nested/src/map_values.rs b/datafusion/functions-nested/src/map_values.rs new file mode 100644 index 000000000000..58c0d74eed5f --- /dev/null +++ b/datafusion/functions-nested/src/map_values.rs @@ -0,0 +1,102 @@ +// 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. + +//! [`ScalarUDFImpl`] definitions for map_values function. + +use crate::utils::{get_map_entry_field, make_scalar_function}; +use arrow_array::{Array, ArrayRef, ListArray}; +use arrow_schema::{DataType, Field}; +use datafusion_common::{cast::as_map_array, exec_err, Result}; +use datafusion_expr::{ + ArrayFunctionSignature, ColumnarValue, ScalarUDFImpl, Signature, TypeSignature, + Volatility, +}; +use std::any::Any; +use std::sync::Arc; + +make_udf_expr_and_func!( + MapValuesFunc, + map_values, + map, + "Return a list of all values in the map.", + map_values_udf +); + +#[derive(Debug)] +pub(crate) struct MapValuesFunc { + signature: Signature, +} + +impl MapValuesFunc { + pub fn new() -> Self { + Self { + signature: Signature::new( + TypeSignature::ArraySignature(ArrayFunctionSignature::MapArray), + Volatility::Immutable, + ), + } + } +} + +impl ScalarUDFImpl for MapValuesFunc { + fn as_any(&self) -> &dyn Any { + self + } + + fn name(&self) -> &str { + "map_values" + } + + fn signature(&self) -> &Signature { + &self.signature + } + + fn return_type(&self, arg_types: &[DataType]) -> datafusion_common::Result { + if arg_types.len() != 1 { + return exec_err!("map_values expects single argument"); + } + let map_type = &arg_types[0]; + let map_fields = get_map_entry_field(map_type)?; + Ok(DataType::List(Arc::new(Field::new( + "item", + map_fields.last().unwrap().data_type().clone(), + true, + )))) + } + + fn invoke(&self, args: &[ColumnarValue]) -> datafusion_common::Result { + make_scalar_function(map_values_inner)(args) + } +} + +fn map_values_inner(args: &[ArrayRef]) -> Result { + if args.len() != 1 { + return exec_err!("map_values expects single argument"); + } + + let map_array = match args[0].data_type() { + DataType::Map(_, _) => as_map_array(&args[0])?, + _ => return exec_err!("Argument for map_values should be a map"), + }; + + Ok(Arc::new(ListArray::new( + Arc::new(Field::new("item", map_array.value_type().clone(), true)), + map_array.offsets().clone(), + Arc::clone(map_array.values()), + None, + ))) +} diff --git a/datafusion/functions-nested/src/utils.rs b/datafusion/functions-nested/src/utils.rs index 3d5b261618d5..0765f6cd237d 100644 --- a/datafusion/functions-nested/src/utils.rs +++ b/datafusion/functions-nested/src/utils.rs @@ -26,9 +26,9 @@ use arrow_array::{ UInt32Array, }; use arrow_buffer::OffsetBuffer; -use arrow_schema::Field; +use arrow_schema::{Field, Fields}; use datafusion_common::cast::{as_large_list_array, as_list_array}; -use datafusion_common::{exec_err, plan_err, Result, ScalarValue}; +use datafusion_common::{exec_err, internal_err, plan_err, Result, ScalarValue}; use core::any::type_name; use datafusion_common::DataFusionError; @@ -253,6 +253,21 @@ pub(crate) fn compute_array_dims( } } +pub(crate) fn get_map_entry_field(data_type: &DataType) -> Result<&Fields> { + match data_type { + DataType::Map(field, _) => { + let field_data_type = field.data_type(); + match field_data_type { + DataType::Struct(fields) => Ok(fields), + _ => { + internal_err!("Expected a Struct type, got {:?}", field_data_type) + } + } + } + _ => internal_err!("Expected a Map type, got {:?}", data_type), + } +} + #[cfg(test)] mod tests { use super::*; diff --git a/datafusion/sqllogictest/test_files/map.slt b/datafusion/sqllogictest/test_files/map.slt index 270e4beccc52..c66334c4de2a 100644 --- a/datafusion/sqllogictest/test_files/map.slt +++ b/datafusion/sqllogictest/test_files/map.slt @@ -568,8 +568,103 @@ select map_extract(column1, 1), map_extract(column1, 5), map_extract(column1, 7) [] [[4, , 6]] [] [] [] [[1, , 3]] +# Tests for map_keys + +query ? +SELECT map_keys(MAP { 'a': 1, 2: 3 }); +---- +[a, 2] + +query ? +SELECT map_keys(MAP {'a':1, 'b':2, 'c':3 }) FROM t; +---- +[a, b, c] +[a, b, c] +[a, b, c] + +query ? +SELECT map_keys(Map{column1: column2, column3: column4}) FROM t; +---- +[a, k1] +[b, k3] +[d, k5] + +query ? +SELECT map_keys(map(column5, column6)) FROM t; +---- +[k1, k2] +[k3] +[k5] + +query ? +SELECT map_keys(map(column8, column9)) FROM t; +---- +[[1, 2, 3]] +[[4]] +[[1, 2]] + +query ? +SELECT map_keys(Map{}); +---- +[] + +query ? +SELECT map_keys(column1) from map_array_table_1; +---- +[1, 2, 3] +[4, 5, 6] +[7, 8, 9] + + +# Tests for map_values + +query ? +SELECT map_values(MAP { 'a': 1, 2: 3 }); +---- +[1, 3] + +query ? +SELECT map_values(MAP {'a':1, 'b':2, 'c':3 }) FROM t; +---- +[1, 2, 3] +[1, 2, 3] +[1, 2, 3] + +query ? +SELECT map_values(Map{column1: column2, column3: column4}) FROM t; +---- +[1, 10] +[2, 30] +[4, 50] + +query ? +SELECT map_values(map(column5, column6)) FROM t; +---- +[1, 2] +[3] +[5] + +query ? +SELECT map_values(map(column8, column9)) FROM t; +---- +[a] +[b] +[c] + +query ? +SELECT map_values(Map{}); +---- +[] + +query ? +SELECT map_values(column1) from map_array_table_1; +---- +[[1, , 3], [4, , 6], [7, 8, 9]] +[[1, , 3], [4, , 6], [7, 8, 9]] +[[1, , 3], [9, , 6], [7, 8, 9]] + statement ok drop table map_array_table_1; statement ok -drop table map_array_table_2; \ No newline at end of file +drop table map_array_table_2; diff --git a/docs/source/user-guide/sql/scalar_functions.md b/docs/source/user-guide/sql/scalar_functions.md index 47e35d2e72e3..80b61f8242ef 100644 --- a/docs/source/user-guide/sql/scalar_functions.md +++ b/docs/source/user-guide/sql/scalar_functions.md @@ -3677,6 +3677,8 @@ Unwraps struct fields into columns. - [map](#map) - [make_map](#make_map) - [map_extract](#map_extract) +- [map_keys](#map_keys) +- [map_values](#map_values) ### `map` @@ -3765,6 +3767,56 @@ SELECT map_extract(MAP {'a': 1, 'b': NULL, 'c': 3}, 'a'); - element_at +### `map_keys` + +Return a list of all keys in the map. + +``` +map_keys(map) +``` + +#### Arguments + +- `map`: Map expression. + Can be a constant, column, or function, and any combination of map operators. + +#### Example + +``` +SELECT map_keys(MAP {'a': 1, 'b': NULL, 'c': 3}); +---- +[a, b, c] + +select map_keys(map([100, 5], [42,43])); +---- +[100, 5] +``` + +### `map_values` + +Return a list of all values in the map. + +``` +map_values(map) +``` + +#### Arguments + +- `map`: Map expression. + Can be a constant, column, or function, and any combination of map operators. + +#### Example + +``` +SELECT map_values(MAP {'a': 1, 'b': NULL, 'c': 3}); +---- +[1, , 3] + +select map_values(map([100, 5], [42,43])); +---- +[42, 43] +``` + ## Hashing Functions - [digest](#digest) From cefccc7be922dbfa1a0308ef1ca2952c008a3fcf Mon Sep 17 00:00:00 2001 From: wiedld Date: Sun, 1 Sep 2024 05:43:33 -0700 Subject: [PATCH 058/154] test: define current arrow_cast behavior for BinaryView (#12200) --- .../sqllogictest/test_files/arrow_typeof.slt | 18 +++++++++++------- 1 file changed, 11 insertions(+), 7 deletions(-) diff --git a/datafusion/sqllogictest/test_files/arrow_typeof.slt b/datafusion/sqllogictest/test_files/arrow_typeof.slt index b9ceb5bf05f1..bae6dc33c8cf 100644 --- a/datafusion/sqllogictest/test_files/arrow_typeof.slt +++ b/datafusion/sqllogictest/test_files/arrow_typeof.slt @@ -102,7 +102,7 @@ query error Error unrecognized word: unknown SELECT arrow_cast('1', 'unknown') # Round Trip tests: -query TTTTTTTTTTTTTTTTTTTTTTTT +query TTTTTTTTTTTTTTTTTTTTTTTTT SELECT arrow_typeof(arrow_cast(1, 'Int8')) as col_i8, arrow_typeof(arrow_cast(1, 'Int16')) as col_i16, @@ -117,6 +117,7 @@ SELECT arrow_typeof(arrow_cast(1, 'Float64')) as col_f64, arrow_typeof(arrow_cast('foo', 'Utf8')) as col_utf8, arrow_typeof(arrow_cast('foo', 'LargeUtf8')) as col_large_utf8, + arrow_typeof(arrow_cast('foo', 'Utf8View')) as col_utf8_view, arrow_typeof(arrow_cast('foo', 'Binary')) as col_binary, arrow_typeof(arrow_cast('foo', 'LargeBinary')) as col_large_binary, arrow_typeof(arrow_cast(to_timestamp('2020-01-02 01:01:11.1234567890Z'), 'Timestamp(Second, None)')) as col_ts_s, @@ -129,7 +130,7 @@ SELECT arrow_typeof(arrow_cast(to_timestamp('2020-01-02 01:01:11.1234567890Z'), 'Timestamp(Nanosecond, Some("+08:00"))')) as col_tstz_ns, arrow_typeof(arrow_cast('foo', 'Dictionary(Int32, Utf8)')) as col_dict ---- -Int8 Int16 Int32 Int64 UInt8 UInt16 UInt32 UInt64 Float16 Float32 Float64 Utf8 LargeUtf8 Binary LargeBinary Timestamp(Second, None) Timestamp(Millisecond, None) Timestamp(Microsecond, None) Timestamp(Nanosecond, None) Timestamp(Second, Some("+08:00")) Timestamp(Millisecond, Some("+08:00")) Timestamp(Microsecond, Some("+08:00")) Timestamp(Nanosecond, Some("+08:00")) Dictionary(Int32, Utf8) +Int8 Int16 Int32 Int64 UInt8 UInt16 UInt32 UInt64 Float16 Float32 Float64 Utf8 LargeUtf8 Utf8View Binary LargeBinary Timestamp(Second, None) Timestamp(Millisecond, None) Timestamp(Microsecond, None) Timestamp(Nanosecond, None) Timestamp(Second, Some("+08:00")) Timestamp(Millisecond, Some("+08:00")) Timestamp(Microsecond, Some("+08:00")) Timestamp(Nanosecond, Some("+08:00")) Dictionary(Int32, Utf8) @@ -211,21 +212,23 @@ statement ok create table foo as select arrow_cast('foo', 'Utf8') as col_utf8, arrow_cast('foo', 'LargeUtf8') as col_large_utf8, + arrow_cast('foo', 'Utf8View') as col_utf8_view, arrow_cast('foo', 'Binary') as col_binary, arrow_cast('foo', 'LargeBinary') as col_large_binary ; ## Ensure each column in the table has the expected type -query TTTT +query TTTTT SELECT arrow_typeof(col_utf8), arrow_typeof(col_large_utf8), + arrow_typeof(col_utf8_view), arrow_typeof(col_binary), arrow_typeof(col_large_binary) FROM foo; ---- -Utf8 LargeUtf8 Binary LargeBinary +Utf8 LargeUtf8 Utf8View Binary LargeBinary statement ok @@ -427,6 +430,7 @@ select arrow_cast('MyAwesomeString', 'Utf8View'), arrow_typeof(arrow_cast('MyAwe ---- MyAwesomeString Utf8View -# Fails until we update arrow-rs with support for https://github.com/apache/arrow-rs/pull/5894 -query error DataFusion error: SQL error: ParserError\("Expected: an SQL statement, found: arrow_cast"\) -arrow_cast('MyAwesomeString', 'BinaryView'), arrow_typeof(arrow_cast('MyAwesomeString', 'BinaryView')) +# Fails until we update to use the arrow-cast release with support for casting utf8 types to BinaryView +# refer to merge commit https://github.com/apache/arrow-rs/commit/4bd737dab2aa17aca200259347909d48ed793ba1 +query error DataFusion error: This feature is not implemented: Unsupported CAST from Utf8 to BinaryView +select arrow_cast('MyAwesomeString', 'BinaryView'), arrow_typeof(arrow_cast('MyAwesomeString', 'BinaryView')) From 5e9a5be84789f77fbf3757c721f60d7e38cb4bbf Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Sun, 1 Sep 2024 14:48:33 +0200 Subject: [PATCH 059/154] Update prost dependency (#12237) --- Cargo.toml | 2 ++ datafusion-examples/Cargo.toml | 2 +- datafusion/proto-common/Cargo.toml | 2 +- datafusion/proto/Cargo.toml | 2 +- datafusion/substrait/Cargo.toml | 1 + 5 files changed, 6 insertions(+), 3 deletions(-) diff --git a/Cargo.toml b/Cargo.toml index 50d234c39576..877cead93673 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -129,6 +129,8 @@ parquet = { version = "52.2.0", default-features = false, features = [ "async", "object_store", ] } +# Should match arrow-flight's version of prost. +prost = "0.12.3" rand = "0.8" regex = "1.8" rstest = "0.22.0" diff --git a/datafusion-examples/Cargo.toml b/datafusion-examples/Cargo.toml index 626c365af21c..57b9930177d4 100644 --- a/datafusion-examples/Cargo.toml +++ b/datafusion-examples/Cargo.toml @@ -72,7 +72,7 @@ log = { workspace = true } mimalloc = { version = "0.1", default-features = false } num_cpus = { workspace = true } object_store = { workspace = true, features = ["aws", "http"] } -prost = { version = "0.12", default-features = false } +prost = { workspace = true } prost-derive = { version = "0.13", default-features = false } serde = { version = "1.0.136", features = ["derive"] } serde_json = { workspace = true } diff --git a/datafusion/proto-common/Cargo.toml b/datafusion/proto-common/Cargo.toml index e5d65827cdec..33a7ecd2daab 100644 --- a/datafusion/proto-common/Cargo.toml +++ b/datafusion/proto-common/Cargo.toml @@ -45,7 +45,7 @@ chrono = { workspace = true } datafusion-common = { workspace = true } object_store = { workspace = true } pbjson = { version = "0.6.0", optional = true } -prost = "0.12.0" +prost = { workspace = true } serde = { version = "1.0", optional = true } serde_json = { workspace = true, optional = true } diff --git a/datafusion/proto/Cargo.toml b/datafusion/proto/Cargo.toml index 95d9e6700a50..2804ed019b61 100644 --- a/datafusion/proto/Cargo.toml +++ b/datafusion/proto/Cargo.toml @@ -53,7 +53,7 @@ datafusion-expr = { workspace = true } datafusion-proto-common = { workspace = true } object_store = { workspace = true } pbjson = { version = "0.6.0", optional = true } -prost = "0.12.0" +prost = { workspace = true } serde = { version = "1.0", optional = true } serde_json = { workspace = true, optional = true } diff --git a/datafusion/substrait/Cargo.toml b/datafusion/substrait/Cargo.toml index ff02ef8c7ef6..f379ac6042dd 100644 --- a/datafusion/substrait/Cargo.toml +++ b/datafusion/substrait/Cargo.toml @@ -39,6 +39,7 @@ datafusion = { workspace = true, default-features = true } itertools = { workspace = true } object_store = { workspace = true } pbjson-types = "0.7" +# TODO use workspace version prost = "0.13" substrait = { version = "0.41", features = ["serde"] } url = { workspace = true } From a8bca7548cb35352123d1f03bc48b04d75ddd772 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sun, 1 Sep 2024 08:49:36 -0400 Subject: [PATCH 060/154] Bump webpack in /datafusion/wasmtest/datafusion-wasm-app (#12236) Bumps [webpack](https://github.com/webpack/webpack) from 5.88.2 to 5.94.0. - [Release notes](https://github.com/webpack/webpack/releases) - [Commits](https://github.com/webpack/webpack/compare/v5.88.2...v5.94.0) --- updated-dependencies: - dependency-name: webpack dependency-type: direct:development ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- .../datafusion-wasm-app/package-lock.json | 456 ++++++++---------- .../wasmtest/datafusion-wasm-app/package.json | 2 +- 2 files changed, 208 insertions(+), 250 deletions(-) diff --git a/datafusion/wasmtest/datafusion-wasm-app/package-lock.json b/datafusion/wasmtest/datafusion-wasm-app/package-lock.json index 995984a13ebc..ebb03e417e5d 100644 --- a/datafusion/wasmtest/datafusion-wasm-app/package-lock.json +++ b/datafusion/wasmtest/datafusion-wasm-app/package-lock.json @@ -13,7 +13,7 @@ }, "devDependencies": { "copy-webpack-plugin": "6.4.1", - "webpack": "5.88.2", + "webpack": "5.94.0", "webpack-cli": "5.1.4", "webpack-dev-server": "4.15.1" } @@ -38,57 +38,57 @@ "dev": true }, "node_modules/@jridgewell/gen-mapping": { - "version": "0.3.3", - "resolved": "https://registry.npmjs.org/@jridgewell/gen-mapping/-/gen-mapping-0.3.3.tgz", - "integrity": "sha512-HLhSWOLRi875zjjMG/r+Nv0oCW8umGb0BgEhyX3dDX3egwZtB8PqLnjz3yedt8R5StBrzcg4aBpnh8UA9D1BoQ==", + "version": "0.3.5", + "resolved": "https://registry.npmjs.org/@jridgewell/gen-mapping/-/gen-mapping-0.3.5.tgz", + "integrity": "sha512-IzL8ZoEDIBRWEzlCcRhOaCupYyN5gdIK+Q6fbFdPDg6HqX6jpkItn7DFIpW9LQzXG6Df9sA7+OKnq0qlz/GaQg==", "dev": true, "dependencies": { - "@jridgewell/set-array": "^1.0.1", + "@jridgewell/set-array": "^1.2.1", "@jridgewell/sourcemap-codec": "^1.4.10", - "@jridgewell/trace-mapping": "^0.3.9" + "@jridgewell/trace-mapping": "^0.3.24" }, "engines": { "node": ">=6.0.0" } }, "node_modules/@jridgewell/resolve-uri": { - "version": "3.1.1", - "resolved": "https://registry.npmjs.org/@jridgewell/resolve-uri/-/resolve-uri-3.1.1.tgz", - "integrity": "sha512-dSYZh7HhCDtCKm4QakX0xFpsRDqjjtZf/kjI/v3T3Nwt5r8/qz/M19F9ySyOqU94SXBmeG9ttTul+YnR4LOxFA==", + "version": "3.1.2", + "resolved": "https://registry.npmjs.org/@jridgewell/resolve-uri/-/resolve-uri-3.1.2.tgz", + "integrity": "sha512-bRISgCIjP20/tbWSPWMEi54QVPRZExkuD9lJL+UIxUKtwVJA8wW1Trb1jMs1RFXo1CBTNZ/5hpC9QvmKWdopKw==", "dev": true, "engines": { "node": ">=6.0.0" } }, "node_modules/@jridgewell/set-array": { - "version": "1.1.2", - "resolved": "https://registry.npmjs.org/@jridgewell/set-array/-/set-array-1.1.2.tgz", - "integrity": "sha512-xnkseuNADM0gt2bs+BvhO0p78Mk762YnZdsuzFV018NoG1Sj1SCQvpSqa7XUaTam5vAGasABV9qXASMKnFMwMw==", + "version": "1.2.1", + "resolved": "https://registry.npmjs.org/@jridgewell/set-array/-/set-array-1.2.1.tgz", + "integrity": "sha512-R8gLRTZeyp03ymzP/6Lil/28tGeGEzhx1q2k703KGWRAI1VdvPIXdG70VJc2pAMw3NA6JKL5hhFu1sJX0Mnn/A==", "dev": true, "engines": { "node": ">=6.0.0" } }, "node_modules/@jridgewell/source-map": { - "version": "0.3.5", - "resolved": "https://registry.npmjs.org/@jridgewell/source-map/-/source-map-0.3.5.tgz", - "integrity": "sha512-UTYAUj/wviwdsMfzoSJspJxbkH5o1snzwX0//0ENX1u/55kkZZkcTZP6u9bwKGkv+dkk9at4m1Cpt0uY80kcpQ==", + "version": "0.3.6", + "resolved": "https://registry.npmjs.org/@jridgewell/source-map/-/source-map-0.3.6.tgz", + "integrity": "sha512-1ZJTZebgqllO79ue2bm3rIGud/bOe0pP5BjSRCRxxYkEZS8STV7zN84UBbiYu7jy+eCKSnVIUgoWWE/tt+shMQ==", "dev": true, "dependencies": { - "@jridgewell/gen-mapping": "^0.3.0", - "@jridgewell/trace-mapping": "^0.3.9" + "@jridgewell/gen-mapping": "^0.3.5", + "@jridgewell/trace-mapping": "^0.3.25" } }, "node_modules/@jridgewell/sourcemap-codec": { - "version": "1.4.15", - "resolved": "https://registry.npmjs.org/@jridgewell/sourcemap-codec/-/sourcemap-codec-1.4.15.tgz", - "integrity": "sha512-eF2rxCRulEKXHTRiDrDy6erMYWqNw4LPdQ8UQA4huuxaQsVeRPFl2oM8oDGxMFhJUWZf9McpLtJasDDZb/Bpeg==", + "version": "1.5.0", + "resolved": "https://registry.npmjs.org/@jridgewell/sourcemap-codec/-/sourcemap-codec-1.5.0.tgz", + "integrity": "sha512-gv3ZRaISU3fjPAgNsriBRqGWQL6quFx04YMPW/zD8XMLsU32mhCCbfbO6KZFLjvYpCZ8zyDEgqsgf+PwPaM7GQ==", "dev": true }, "node_modules/@jridgewell/trace-mapping": { - "version": "0.3.19", - "resolved": "https://registry.npmjs.org/@jridgewell/trace-mapping/-/trace-mapping-0.3.19.tgz", - "integrity": "sha512-kf37QtfW+Hwx/buWGMPcR60iF9ziHa6r/CZJIHbmcm4+0qrXiVdxegAH0F6yddEVQ7zdkjcGCgCzUu+BcbhQxw==", + "version": "0.3.25", + "resolved": "https://registry.npmjs.org/@jridgewell/trace-mapping/-/trace-mapping-0.3.25.tgz", + "integrity": "sha512-vNk6aEwybGtawWmy/PzwnGDOjCkLWSD2wqvjGGAgOAwCGWySYXfYoxt00IJkTF+8Lb57DwOb3Aa0o9CApepiYQ==", "dev": true, "dependencies": { "@jridgewell/resolve-uri": "^3.1.0", @@ -198,30 +198,10 @@ "@types/node": "*" } }, - "node_modules/@types/eslint": { - "version": "8.44.2", - "resolved": "https://registry.npmjs.org/@types/eslint/-/eslint-8.44.2.tgz", - "integrity": "sha512-sdPRb9K6iL5XZOmBubg8yiFp5yS/JdUDQsq5e6h95km91MCYMuvp7mh1fjPEYUhvHepKpZOjnEaMBR4PxjWDzg==", - "dev": true, - "dependencies": { - "@types/estree": "*", - "@types/json-schema": "*" - } - }, - "node_modules/@types/eslint-scope": { - "version": "3.7.4", - "resolved": "https://registry.npmjs.org/@types/eslint-scope/-/eslint-scope-3.7.4.tgz", - "integrity": "sha512-9K4zoImiZc3HlIp6AVUDE4CWYx22a+lhSZMYNpbjW04+YF0KWj4pJXnEMjdnFTiQibFFmElcsasJXDbdI/EPhA==", - "dev": true, - "dependencies": { - "@types/eslint": "*", - "@types/estree": "*" - } - }, "node_modules/@types/estree": { - "version": "1.0.1", - "resolved": "https://registry.npmjs.org/@types/estree/-/estree-1.0.1.tgz", - "integrity": "sha512-LG4opVs2ANWZ1TJoKc937iMmNstM/d0ae1vNbnBvBhqCSezgVUOzcLCqbI5elV8Vy6WKwKjaqR+zO9VKirBBCA==", + "version": "1.0.5", + "resolved": "https://registry.npmjs.org/@types/estree/-/estree-1.0.5.tgz", + "integrity": "sha512-/kYRxGDLWzHOB7q+wtSUQlFrtcdUccpfy+X+9iMBpHK8QLLhx2wIPYuS5DYtR9Wa/YlZAbIovy7qVdB1Aq6Lyw==", "dev": true }, "node_modules/@types/express": { @@ -348,9 +328,9 @@ } }, "node_modules/@webassemblyjs/ast": { - "version": "1.11.6", - "resolved": "https://registry.npmjs.org/@webassemblyjs/ast/-/ast-1.11.6.tgz", - "integrity": "sha512-IN1xI7PwOvLPgjcf180gC1bqn3q/QaOCwYUahIOhbYUu8KA/3tw2RT/T0Gidi1l7Hhj5D/INhJxiICObqpMu4Q==", + "version": "1.12.1", + "resolved": "https://registry.npmjs.org/@webassemblyjs/ast/-/ast-1.12.1.tgz", + "integrity": "sha512-EKfMUOPRRUTy5UII4qJDGPpqfwjOmZ5jeGFwid9mnoqIFK+e0vqoi1qH56JpmZSzEL53jKnNzScdmftJyG5xWg==", "dev": true, "dependencies": { "@webassemblyjs/helper-numbers": "1.11.6", @@ -370,9 +350,9 @@ "dev": true }, "node_modules/@webassemblyjs/helper-buffer": { - "version": "1.11.6", - "resolved": "https://registry.npmjs.org/@webassemblyjs/helper-buffer/-/helper-buffer-1.11.6.tgz", - "integrity": "sha512-z3nFzdcp1mb8nEOFFk8DrYLpHvhKC3grJD2ardfKOzmbmJvEf/tPIqCY+sNcwZIY8ZD7IkB2l7/pqhUhqm7hLA==", + "version": "1.12.1", + "resolved": "https://registry.npmjs.org/@webassemblyjs/helper-buffer/-/helper-buffer-1.12.1.tgz", + "integrity": "sha512-nzJwQw99DNDKr9BVCOZcLuJJUlqkJh+kVzVl6Fmq/tI5ZtEyWT1KZMyOXltXLZJmDtvLCDgwsyrkohEtopTXCw==", "dev": true }, "node_modules/@webassemblyjs/helper-numbers": { @@ -393,15 +373,15 @@ "dev": true }, "node_modules/@webassemblyjs/helper-wasm-section": { - "version": "1.11.6", - "resolved": "https://registry.npmjs.org/@webassemblyjs/helper-wasm-section/-/helper-wasm-section-1.11.6.tgz", - "integrity": "sha512-LPpZbSOwTpEC2cgn4hTydySy1Ke+XEu+ETXuoyvuyezHO3Kjdu90KK95Sh9xTbmjrCsUwvWwCOQQNta37VrS9g==", + "version": "1.12.1", + "resolved": "https://registry.npmjs.org/@webassemblyjs/helper-wasm-section/-/helper-wasm-section-1.12.1.tgz", + "integrity": "sha512-Jif4vfB6FJlUlSbgEMHUyk1j234GTNG9dBJ4XJdOySoj518Xj0oGsNi59cUQF4RRMS9ouBUxDDdyBVfPTypa5g==", "dev": true, "dependencies": { - "@webassemblyjs/ast": "1.11.6", - "@webassemblyjs/helper-buffer": "1.11.6", + "@webassemblyjs/ast": "1.12.1", + "@webassemblyjs/helper-buffer": "1.12.1", "@webassemblyjs/helper-wasm-bytecode": "1.11.6", - "@webassemblyjs/wasm-gen": "1.11.6" + "@webassemblyjs/wasm-gen": "1.12.1" } }, "node_modules/@webassemblyjs/ieee754": { @@ -429,28 +409,28 @@ "dev": true }, "node_modules/@webassemblyjs/wasm-edit": { - "version": "1.11.6", - "resolved": "https://registry.npmjs.org/@webassemblyjs/wasm-edit/-/wasm-edit-1.11.6.tgz", - "integrity": "sha512-Ybn2I6fnfIGuCR+Faaz7YcvtBKxvoLV3Lebn1tM4o/IAJzmi9AWYIPWpyBfU8cC+JxAO57bk4+zdsTjJR+VTOw==", + "version": "1.12.1", + "resolved": "https://registry.npmjs.org/@webassemblyjs/wasm-edit/-/wasm-edit-1.12.1.tgz", + "integrity": "sha512-1DuwbVvADvS5mGnXbE+c9NfA8QRcZ6iKquqjjmR10k6o+zzsRVesil54DKexiowcFCPdr/Q0qaMgB01+SQ1u6g==", "dev": true, "dependencies": { - "@webassemblyjs/ast": "1.11.6", - "@webassemblyjs/helper-buffer": "1.11.6", + "@webassemblyjs/ast": "1.12.1", + "@webassemblyjs/helper-buffer": "1.12.1", "@webassemblyjs/helper-wasm-bytecode": "1.11.6", - "@webassemblyjs/helper-wasm-section": "1.11.6", - "@webassemblyjs/wasm-gen": "1.11.6", - "@webassemblyjs/wasm-opt": "1.11.6", - "@webassemblyjs/wasm-parser": "1.11.6", - "@webassemblyjs/wast-printer": "1.11.6" + "@webassemblyjs/helper-wasm-section": "1.12.1", + "@webassemblyjs/wasm-gen": "1.12.1", + "@webassemblyjs/wasm-opt": "1.12.1", + "@webassemblyjs/wasm-parser": "1.12.1", + "@webassemblyjs/wast-printer": "1.12.1" } }, "node_modules/@webassemblyjs/wasm-gen": { - "version": "1.11.6", - "resolved": "https://registry.npmjs.org/@webassemblyjs/wasm-gen/-/wasm-gen-1.11.6.tgz", - "integrity": "sha512-3XOqkZP/y6B4F0PBAXvI1/bky7GryoogUtfwExeP/v7Nzwo1QLcq5oQmpKlftZLbT+ERUOAZVQjuNVak6UXjPA==", + "version": "1.12.1", + "resolved": "https://registry.npmjs.org/@webassemblyjs/wasm-gen/-/wasm-gen-1.12.1.tgz", + "integrity": "sha512-TDq4Ojh9fcohAw6OIMXqiIcTq5KUXTGRkVxbSo1hQnSy6lAM5GSdfwWeSxpAo0YzgsgF182E/U0mDNhuA0tW7w==", "dev": true, "dependencies": { - "@webassemblyjs/ast": "1.11.6", + "@webassemblyjs/ast": "1.12.1", "@webassemblyjs/helper-wasm-bytecode": "1.11.6", "@webassemblyjs/ieee754": "1.11.6", "@webassemblyjs/leb128": "1.11.6", @@ -458,24 +438,24 @@ } }, "node_modules/@webassemblyjs/wasm-opt": { - "version": "1.11.6", - "resolved": "https://registry.npmjs.org/@webassemblyjs/wasm-opt/-/wasm-opt-1.11.6.tgz", - "integrity": "sha512-cOrKuLRE7PCe6AsOVl7WasYf3wbSo4CeOk6PkrjS7g57MFfVUF9u6ysQBBODX0LdgSvQqRiGz3CXvIDKcPNy4g==", + "version": "1.12.1", + "resolved": "https://registry.npmjs.org/@webassemblyjs/wasm-opt/-/wasm-opt-1.12.1.tgz", + "integrity": "sha512-Jg99j/2gG2iaz3hijw857AVYekZe2SAskcqlWIZXjji5WStnOpVoat3gQfT/Q5tb2djnCjBtMocY/Su1GfxPBg==", "dev": true, "dependencies": { - "@webassemblyjs/ast": "1.11.6", - "@webassemblyjs/helper-buffer": "1.11.6", - "@webassemblyjs/wasm-gen": "1.11.6", - "@webassemblyjs/wasm-parser": "1.11.6" + "@webassemblyjs/ast": "1.12.1", + "@webassemblyjs/helper-buffer": "1.12.1", + "@webassemblyjs/wasm-gen": "1.12.1", + "@webassemblyjs/wasm-parser": "1.12.1" } }, "node_modules/@webassemblyjs/wasm-parser": { - "version": "1.11.6", - "resolved": "https://registry.npmjs.org/@webassemblyjs/wasm-parser/-/wasm-parser-1.11.6.tgz", - "integrity": "sha512-6ZwPeGzMJM3Dqp3hCsLgESxBGtT/OeCvCZ4TA1JUPYgmhAx38tTPR9JaKy0S5H3evQpO/h2uWs2j6Yc/fjkpTQ==", + "version": "1.12.1", + "resolved": "https://registry.npmjs.org/@webassemblyjs/wasm-parser/-/wasm-parser-1.12.1.tgz", + "integrity": "sha512-xikIi7c2FHXysxXe3COrVUPSheuBtpcfhbpFj4gmu7KRLYOzANztwUU0IbsqvMqzuNK2+glRGWCEqZo1WCLyAQ==", "dev": true, "dependencies": { - "@webassemblyjs/ast": "1.11.6", + "@webassemblyjs/ast": "1.12.1", "@webassemblyjs/helper-api-error": "1.11.6", "@webassemblyjs/helper-wasm-bytecode": "1.11.6", "@webassemblyjs/ieee754": "1.11.6", @@ -484,12 +464,12 @@ } }, "node_modules/@webassemblyjs/wast-printer": { - "version": "1.11.6", - "resolved": "https://registry.npmjs.org/@webassemblyjs/wast-printer/-/wast-printer-1.11.6.tgz", - "integrity": "sha512-JM7AhRcE+yW2GWYaKeHL5vt4xqee5N2WcezptmgyhNS+ScggqcT1OtXykhAb13Sn5Yas0j2uv9tHgrjwvzAP4A==", + "version": "1.12.1", + "resolved": "https://registry.npmjs.org/@webassemblyjs/wast-printer/-/wast-printer-1.12.1.tgz", + "integrity": "sha512-+X4WAlOisVWQMikjbcvY2e0rwPsKQ9F688lksZhBcPycBBuii3O7m8FACbDMWDojpAqvjIncrG8J0XHKyQfVeA==", "dev": true, "dependencies": { - "@webassemblyjs/ast": "1.11.6", + "@webassemblyjs/ast": "1.12.1", "@xtuc/long": "4.2.2" } }, @@ -563,9 +543,9 @@ } }, "node_modules/acorn": { - "version": "8.10.0", - "resolved": "https://registry.npmjs.org/acorn/-/acorn-8.10.0.tgz", - "integrity": "sha512-F0SAmZ8iUtS//m8DmCTA0jlh6TDKkHQyK6xc6V4KDTyZKA9dnvX9/3sRTVQrWm79glUAZbnmmNcdYwUIHWVybw==", + "version": "8.12.1", + "resolved": "https://registry.npmjs.org/acorn/-/acorn-8.12.1.tgz", + "integrity": "sha512-tcpGyI9zbizT9JbV6oYE477V6mTlXvvi0T0G3SNIYE2apm/G5huBa1+K89VGeovbg+jycCrfhl3ADxErOuO6Jg==", "dev": true, "bin": { "acorn": "bin/acorn" @@ -574,10 +554,10 @@ "node": ">=0.4.0" } }, - "node_modules/acorn-import-assertions": { - "version": "1.9.0", - "resolved": "https://registry.npmjs.org/acorn-import-assertions/-/acorn-import-assertions-1.9.0.tgz", - "integrity": "sha512-cmMwop9x+8KFhxvKrKfPYmN6/pKTYYHBqLa0DfvVZcKMJWNyWLnaqND7dx/qn66R7ewM1UX5XMaDVP5wlVTaVA==", + "node_modules/acorn-import-attributes": { + "version": "1.9.5", + "resolved": "https://registry.npmjs.org/acorn-import-attributes/-/acorn-import-attributes-1.9.5.tgz", + "integrity": "sha512-n02Vykv5uA3eHGM/Z2dQrcD56kL8TyDb2p1+0P83PClMnC/nc+anbQRhIOWnSq4Ke/KvDPrY3C9hDtC/A3eHnQ==", "dev": true, "peerDependencies": { "acorn": "^8" @@ -1322,9 +1302,9 @@ } }, "node_modules/enhanced-resolve": { - "version": "5.15.0", - "resolved": "https://registry.npmjs.org/enhanced-resolve/-/enhanced-resolve-5.15.0.tgz", - "integrity": "sha512-LXYT42KJ7lpIKECr2mAXIaMldcNCh/7E0KBKOu4KSfkHmP+mZmSs+8V5gBAqisWBy0OO4W5Oyys0GO1Y8KtdKg==", + "version": "5.17.1", + "resolved": "https://registry.npmjs.org/enhanced-resolve/-/enhanced-resolve-5.17.1.tgz", + "integrity": "sha512-LMHl3dXhTcfv8gM4kEzIUeTQ+7fpdA0l2tUf34BddXPkz2A5xJ5L/Pchd5BL6rdccM9QGvu0sWZzK1Z1t4wwyg==", "dev": true, "dependencies": { "graceful-fs": "^4.2.4", @@ -3724,9 +3704,9 @@ } }, "node_modules/terser": { - "version": "5.20.0", - "resolved": "https://registry.npmjs.org/terser/-/terser-5.20.0.tgz", - "integrity": "sha512-e56ETryaQDyebBwJIWYB2TT6f2EZ0fL0sW/JRXNMN26zZdKi2u/E/5my5lG6jNxym6qsrVXfFRmOdV42zlAgLQ==", + "version": "5.31.6", + "resolved": "https://registry.npmjs.org/terser/-/terser-5.31.6.tgz", + "integrity": "sha512-PQ4DAriWzKj+qgehQ7LK5bQqCFNMmlhjR2PFFLuqGCpuCAauxemVBWwWOxo3UIwWQx8+Pr61Df++r76wDmkQBg==", "dev": true, "dependencies": { "@jridgewell/source-map": "^0.3.3", @@ -3742,16 +3722,16 @@ } }, "node_modules/terser-webpack-plugin": { - "version": "5.3.9", - "resolved": "https://registry.npmjs.org/terser-webpack-plugin/-/terser-webpack-plugin-5.3.9.tgz", - "integrity": "sha512-ZuXsqE07EcggTWQjXUj+Aot/OMcD0bMKGgF63f7UxYcu5/AJF53aIpK1YoP5xR9l6s/Hy2b+t1AM0bLNPRuhwA==", + "version": "5.3.10", + "resolved": "https://registry.npmjs.org/terser-webpack-plugin/-/terser-webpack-plugin-5.3.10.tgz", + "integrity": "sha512-BKFPWlPDndPs+NGGCr1U59t0XScL5317Y0UReNrHaw9/FwhPENlq6bfgs+4yPfyP51vqC1bQ4rp1EfXW5ZSH9w==", "dev": true, "dependencies": { - "@jridgewell/trace-mapping": "^0.3.17", + "@jridgewell/trace-mapping": "^0.3.20", "jest-worker": "^27.4.5", "schema-utils": "^3.1.1", "serialize-javascript": "^6.0.1", - "terser": "^5.16.8" + "terser": "^5.26.0" }, "engines": { "node": ">= 10.13.0" @@ -3776,9 +3756,9 @@ } }, "node_modules/terser-webpack-plugin/node_modules/serialize-javascript": { - "version": "6.0.1", - "resolved": "https://registry.npmjs.org/serialize-javascript/-/serialize-javascript-6.0.1.tgz", - "integrity": "sha512-owoXEFjWRllis8/M1Q+Cw5k8ZH40e3zhp/ovX+Xr/vi1qj6QesbyXXViFbpNvWvPNAD62SutwEXavefrLJWj7w==", + "version": "6.0.2", + "resolved": "https://registry.npmjs.org/serialize-javascript/-/serialize-javascript-6.0.2.tgz", + "integrity": "sha512-Saa1xPByTTq2gdeFZYLLo+RFE35NHZkAbqZeWNd3BpzppeVisAqpDjcp8dyf6uIvEqJRd46jemmyA4iFIeVk8g==", "dev": true, "dependencies": { "randombytes": "^2.1.0" @@ -3930,9 +3910,9 @@ } }, "node_modules/watchpack": { - "version": "2.4.0", - "resolved": "https://registry.npmjs.org/watchpack/-/watchpack-2.4.0.tgz", - "integrity": "sha512-Lcvm7MGST/4fup+ifyKi2hjyIAwcdI4HRgtvTpIUxBRhB+RFtUh8XtDOxUfctVCnhVi+QQj49i91OyvzkJl6cg==", + "version": "2.4.2", + "resolved": "https://registry.npmjs.org/watchpack/-/watchpack-2.4.2.tgz", + "integrity": "sha512-TnbFSbcOCcDgjZ4piURLCbJ3nJhznVh9kw6F6iokjiFPl8ONxe9A6nMDVXDiNbrSfLILs6vB07F7wLBrwPYzJw==", "dev": true, "dependencies": { "glob-to-regexp": "^0.4.1", @@ -3952,34 +3932,33 @@ } }, "node_modules/webpack": { - "version": "5.88.2", - "resolved": "https://registry.npmjs.org/webpack/-/webpack-5.88.2.tgz", - "integrity": "sha512-JmcgNZ1iKj+aiR0OvTYtWQqJwq37Pf683dY9bVORwVbUrDhLhdn/PlO2sHsFHPkj7sHNQF3JwaAkp49V+Sq1tQ==", + "version": "5.94.0", + "resolved": "https://registry.npmjs.org/webpack/-/webpack-5.94.0.tgz", + "integrity": "sha512-KcsGn50VT+06JH/iunZJedYGUJS5FGjow8wb9c0v5n1Om8O1g4L6LjtfxwlXIATopoQu+vOXXa7gYisWxCoPyg==", "dev": true, "dependencies": { - "@types/eslint-scope": "^3.7.3", - "@types/estree": "^1.0.0", - "@webassemblyjs/ast": "^1.11.5", - "@webassemblyjs/wasm-edit": "^1.11.5", - "@webassemblyjs/wasm-parser": "^1.11.5", + "@types/estree": "^1.0.5", + "@webassemblyjs/ast": "^1.12.1", + "@webassemblyjs/wasm-edit": "^1.12.1", + "@webassemblyjs/wasm-parser": "^1.12.1", "acorn": "^8.7.1", - "acorn-import-assertions": "^1.9.0", - "browserslist": "^4.14.5", + "acorn-import-attributes": "^1.9.5", + "browserslist": "^4.21.10", "chrome-trace-event": "^1.0.2", - "enhanced-resolve": "^5.15.0", + "enhanced-resolve": "^5.17.1", "es-module-lexer": "^1.2.1", "eslint-scope": "5.1.1", "events": "^3.2.0", "glob-to-regexp": "^0.4.1", - "graceful-fs": "^4.2.9", + "graceful-fs": "^4.2.11", "json-parse-even-better-errors": "^2.3.1", "loader-runner": "^4.2.0", "mime-types": "^2.1.27", "neo-async": "^2.6.2", "schema-utils": "^3.2.0", "tapable": "^2.1.1", - "terser-webpack-plugin": "^5.3.7", - "watchpack": "^2.4.0", + "terser-webpack-plugin": "^5.3.10", + "watchpack": "^2.4.1", "webpack-sources": "^3.2.3" }, "bin": { @@ -4376,48 +4355,48 @@ "dev": true }, "@jridgewell/gen-mapping": { - "version": "0.3.3", - "resolved": "https://registry.npmjs.org/@jridgewell/gen-mapping/-/gen-mapping-0.3.3.tgz", - "integrity": "sha512-HLhSWOLRi875zjjMG/r+Nv0oCW8umGb0BgEhyX3dDX3egwZtB8PqLnjz3yedt8R5StBrzcg4aBpnh8UA9D1BoQ==", + "version": "0.3.5", + "resolved": "https://registry.npmjs.org/@jridgewell/gen-mapping/-/gen-mapping-0.3.5.tgz", + "integrity": "sha512-IzL8ZoEDIBRWEzlCcRhOaCupYyN5gdIK+Q6fbFdPDg6HqX6jpkItn7DFIpW9LQzXG6Df9sA7+OKnq0qlz/GaQg==", "dev": true, "requires": { - "@jridgewell/set-array": "^1.0.1", + "@jridgewell/set-array": "^1.2.1", "@jridgewell/sourcemap-codec": "^1.4.10", - "@jridgewell/trace-mapping": "^0.3.9" + "@jridgewell/trace-mapping": "^0.3.24" } }, "@jridgewell/resolve-uri": { - "version": "3.1.1", - "resolved": "https://registry.npmjs.org/@jridgewell/resolve-uri/-/resolve-uri-3.1.1.tgz", - "integrity": "sha512-dSYZh7HhCDtCKm4QakX0xFpsRDqjjtZf/kjI/v3T3Nwt5r8/qz/M19F9ySyOqU94SXBmeG9ttTul+YnR4LOxFA==", + "version": "3.1.2", + "resolved": "https://registry.npmjs.org/@jridgewell/resolve-uri/-/resolve-uri-3.1.2.tgz", + "integrity": "sha512-bRISgCIjP20/tbWSPWMEi54QVPRZExkuD9lJL+UIxUKtwVJA8wW1Trb1jMs1RFXo1CBTNZ/5hpC9QvmKWdopKw==", "dev": true }, "@jridgewell/set-array": { - "version": "1.1.2", - "resolved": "https://registry.npmjs.org/@jridgewell/set-array/-/set-array-1.1.2.tgz", - "integrity": "sha512-xnkseuNADM0gt2bs+BvhO0p78Mk762YnZdsuzFV018NoG1Sj1SCQvpSqa7XUaTam5vAGasABV9qXASMKnFMwMw==", + "version": "1.2.1", + "resolved": "https://registry.npmjs.org/@jridgewell/set-array/-/set-array-1.2.1.tgz", + "integrity": "sha512-R8gLRTZeyp03ymzP/6Lil/28tGeGEzhx1q2k703KGWRAI1VdvPIXdG70VJc2pAMw3NA6JKL5hhFu1sJX0Mnn/A==", "dev": true }, "@jridgewell/source-map": { - "version": "0.3.5", - "resolved": "https://registry.npmjs.org/@jridgewell/source-map/-/source-map-0.3.5.tgz", - "integrity": "sha512-UTYAUj/wviwdsMfzoSJspJxbkH5o1snzwX0//0ENX1u/55kkZZkcTZP6u9bwKGkv+dkk9at4m1Cpt0uY80kcpQ==", + "version": "0.3.6", + "resolved": "https://registry.npmjs.org/@jridgewell/source-map/-/source-map-0.3.6.tgz", + "integrity": "sha512-1ZJTZebgqllO79ue2bm3rIGud/bOe0pP5BjSRCRxxYkEZS8STV7zN84UBbiYu7jy+eCKSnVIUgoWWE/tt+shMQ==", "dev": true, "requires": { - "@jridgewell/gen-mapping": "^0.3.0", - "@jridgewell/trace-mapping": "^0.3.9" + "@jridgewell/gen-mapping": "^0.3.5", + "@jridgewell/trace-mapping": "^0.3.25" } }, "@jridgewell/sourcemap-codec": { - "version": "1.4.15", - "resolved": "https://registry.npmjs.org/@jridgewell/sourcemap-codec/-/sourcemap-codec-1.4.15.tgz", - "integrity": "sha512-eF2rxCRulEKXHTRiDrDy6erMYWqNw4LPdQ8UQA4huuxaQsVeRPFl2oM8oDGxMFhJUWZf9McpLtJasDDZb/Bpeg==", + "version": "1.5.0", + "resolved": "https://registry.npmjs.org/@jridgewell/sourcemap-codec/-/sourcemap-codec-1.5.0.tgz", + "integrity": "sha512-gv3ZRaISU3fjPAgNsriBRqGWQL6quFx04YMPW/zD8XMLsU32mhCCbfbO6KZFLjvYpCZ8zyDEgqsgf+PwPaM7GQ==", "dev": true }, "@jridgewell/trace-mapping": { - "version": "0.3.19", - "resolved": "https://registry.npmjs.org/@jridgewell/trace-mapping/-/trace-mapping-0.3.19.tgz", - "integrity": "sha512-kf37QtfW+Hwx/buWGMPcR60iF9ziHa6r/CZJIHbmcm4+0qrXiVdxegAH0F6yddEVQ7zdkjcGCgCzUu+BcbhQxw==", + "version": "0.3.25", + "resolved": "https://registry.npmjs.org/@jridgewell/trace-mapping/-/trace-mapping-0.3.25.tgz", + "integrity": "sha512-vNk6aEwybGtawWmy/PzwnGDOjCkLWSD2wqvjGGAgOAwCGWySYXfYoxt00IJkTF+8Lb57DwOb3Aa0o9CApepiYQ==", "dev": true, "requires": { "@jridgewell/resolve-uri": "^3.1.0", @@ -4514,30 +4493,10 @@ "@types/node": "*" } }, - "@types/eslint": { - "version": "8.44.2", - "resolved": "https://registry.npmjs.org/@types/eslint/-/eslint-8.44.2.tgz", - "integrity": "sha512-sdPRb9K6iL5XZOmBubg8yiFp5yS/JdUDQsq5e6h95km91MCYMuvp7mh1fjPEYUhvHepKpZOjnEaMBR4PxjWDzg==", - "dev": true, - "requires": { - "@types/estree": "*", - "@types/json-schema": "*" - } - }, - "@types/eslint-scope": { - "version": "3.7.4", - "resolved": "https://registry.npmjs.org/@types/eslint-scope/-/eslint-scope-3.7.4.tgz", - "integrity": "sha512-9K4zoImiZc3HlIp6AVUDE4CWYx22a+lhSZMYNpbjW04+YF0KWj4pJXnEMjdnFTiQibFFmElcsasJXDbdI/EPhA==", - "dev": true, - "requires": { - "@types/eslint": "*", - "@types/estree": "*" - } - }, "@types/estree": { - "version": "1.0.1", - "resolved": "https://registry.npmjs.org/@types/estree/-/estree-1.0.1.tgz", - "integrity": "sha512-LG4opVs2ANWZ1TJoKc937iMmNstM/d0ae1vNbnBvBhqCSezgVUOzcLCqbI5elV8Vy6WKwKjaqR+zO9VKirBBCA==", + "version": "1.0.5", + "resolved": "https://registry.npmjs.org/@types/estree/-/estree-1.0.5.tgz", + "integrity": "sha512-/kYRxGDLWzHOB7q+wtSUQlFrtcdUccpfy+X+9iMBpHK8QLLhx2wIPYuS5DYtR9Wa/YlZAbIovy7qVdB1Aq6Lyw==", "dev": true }, "@types/express": { @@ -4664,9 +4623,9 @@ } }, "@webassemblyjs/ast": { - "version": "1.11.6", - "resolved": "https://registry.npmjs.org/@webassemblyjs/ast/-/ast-1.11.6.tgz", - "integrity": "sha512-IN1xI7PwOvLPgjcf180gC1bqn3q/QaOCwYUahIOhbYUu8KA/3tw2RT/T0Gidi1l7Hhj5D/INhJxiICObqpMu4Q==", + "version": "1.12.1", + "resolved": "https://registry.npmjs.org/@webassemblyjs/ast/-/ast-1.12.1.tgz", + "integrity": "sha512-EKfMUOPRRUTy5UII4qJDGPpqfwjOmZ5jeGFwid9mnoqIFK+e0vqoi1qH56JpmZSzEL53jKnNzScdmftJyG5xWg==", "dev": true, "requires": { "@webassemblyjs/helper-numbers": "1.11.6", @@ -4686,9 +4645,9 @@ "dev": true }, "@webassemblyjs/helper-buffer": { - "version": "1.11.6", - "resolved": "https://registry.npmjs.org/@webassemblyjs/helper-buffer/-/helper-buffer-1.11.6.tgz", - "integrity": "sha512-z3nFzdcp1mb8nEOFFk8DrYLpHvhKC3grJD2ardfKOzmbmJvEf/tPIqCY+sNcwZIY8ZD7IkB2l7/pqhUhqm7hLA==", + "version": "1.12.1", + "resolved": "https://registry.npmjs.org/@webassemblyjs/helper-buffer/-/helper-buffer-1.12.1.tgz", + "integrity": "sha512-nzJwQw99DNDKr9BVCOZcLuJJUlqkJh+kVzVl6Fmq/tI5ZtEyWT1KZMyOXltXLZJmDtvLCDgwsyrkohEtopTXCw==", "dev": true }, "@webassemblyjs/helper-numbers": { @@ -4709,15 +4668,15 @@ "dev": true }, "@webassemblyjs/helper-wasm-section": { - "version": "1.11.6", - "resolved": "https://registry.npmjs.org/@webassemblyjs/helper-wasm-section/-/helper-wasm-section-1.11.6.tgz", - "integrity": "sha512-LPpZbSOwTpEC2cgn4hTydySy1Ke+XEu+ETXuoyvuyezHO3Kjdu90KK95Sh9xTbmjrCsUwvWwCOQQNta37VrS9g==", + "version": "1.12.1", + "resolved": "https://registry.npmjs.org/@webassemblyjs/helper-wasm-section/-/helper-wasm-section-1.12.1.tgz", + "integrity": "sha512-Jif4vfB6FJlUlSbgEMHUyk1j234GTNG9dBJ4XJdOySoj518Xj0oGsNi59cUQF4RRMS9ouBUxDDdyBVfPTypa5g==", "dev": true, "requires": { - "@webassemblyjs/ast": "1.11.6", - "@webassemblyjs/helper-buffer": "1.11.6", + "@webassemblyjs/ast": "1.12.1", + "@webassemblyjs/helper-buffer": "1.12.1", "@webassemblyjs/helper-wasm-bytecode": "1.11.6", - "@webassemblyjs/wasm-gen": "1.11.6" + "@webassemblyjs/wasm-gen": "1.12.1" } }, "@webassemblyjs/ieee754": { @@ -4745,28 +4704,28 @@ "dev": true }, "@webassemblyjs/wasm-edit": { - "version": "1.11.6", - "resolved": "https://registry.npmjs.org/@webassemblyjs/wasm-edit/-/wasm-edit-1.11.6.tgz", - "integrity": "sha512-Ybn2I6fnfIGuCR+Faaz7YcvtBKxvoLV3Lebn1tM4o/IAJzmi9AWYIPWpyBfU8cC+JxAO57bk4+zdsTjJR+VTOw==", + "version": "1.12.1", + "resolved": "https://registry.npmjs.org/@webassemblyjs/wasm-edit/-/wasm-edit-1.12.1.tgz", + "integrity": "sha512-1DuwbVvADvS5mGnXbE+c9NfA8QRcZ6iKquqjjmR10k6o+zzsRVesil54DKexiowcFCPdr/Q0qaMgB01+SQ1u6g==", "dev": true, "requires": { - "@webassemblyjs/ast": "1.11.6", - "@webassemblyjs/helper-buffer": "1.11.6", + "@webassemblyjs/ast": "1.12.1", + "@webassemblyjs/helper-buffer": "1.12.1", "@webassemblyjs/helper-wasm-bytecode": "1.11.6", - "@webassemblyjs/helper-wasm-section": "1.11.6", - "@webassemblyjs/wasm-gen": "1.11.6", - "@webassemblyjs/wasm-opt": "1.11.6", - "@webassemblyjs/wasm-parser": "1.11.6", - "@webassemblyjs/wast-printer": "1.11.6" + "@webassemblyjs/helper-wasm-section": "1.12.1", + "@webassemblyjs/wasm-gen": "1.12.1", + "@webassemblyjs/wasm-opt": "1.12.1", + "@webassemblyjs/wasm-parser": "1.12.1", + "@webassemblyjs/wast-printer": "1.12.1" } }, "@webassemblyjs/wasm-gen": { - "version": "1.11.6", - "resolved": "https://registry.npmjs.org/@webassemblyjs/wasm-gen/-/wasm-gen-1.11.6.tgz", - "integrity": "sha512-3XOqkZP/y6B4F0PBAXvI1/bky7GryoogUtfwExeP/v7Nzwo1QLcq5oQmpKlftZLbT+ERUOAZVQjuNVak6UXjPA==", + "version": "1.12.1", + "resolved": "https://registry.npmjs.org/@webassemblyjs/wasm-gen/-/wasm-gen-1.12.1.tgz", + "integrity": "sha512-TDq4Ojh9fcohAw6OIMXqiIcTq5KUXTGRkVxbSo1hQnSy6lAM5GSdfwWeSxpAo0YzgsgF182E/U0mDNhuA0tW7w==", "dev": true, "requires": { - "@webassemblyjs/ast": "1.11.6", + "@webassemblyjs/ast": "1.12.1", "@webassemblyjs/helper-wasm-bytecode": "1.11.6", "@webassemblyjs/ieee754": "1.11.6", "@webassemblyjs/leb128": "1.11.6", @@ -4774,24 +4733,24 @@ } }, "@webassemblyjs/wasm-opt": { - "version": "1.11.6", - "resolved": "https://registry.npmjs.org/@webassemblyjs/wasm-opt/-/wasm-opt-1.11.6.tgz", - "integrity": "sha512-cOrKuLRE7PCe6AsOVl7WasYf3wbSo4CeOk6PkrjS7g57MFfVUF9u6ysQBBODX0LdgSvQqRiGz3CXvIDKcPNy4g==", + "version": "1.12.1", + "resolved": "https://registry.npmjs.org/@webassemblyjs/wasm-opt/-/wasm-opt-1.12.1.tgz", + "integrity": "sha512-Jg99j/2gG2iaz3hijw857AVYekZe2SAskcqlWIZXjji5WStnOpVoat3gQfT/Q5tb2djnCjBtMocY/Su1GfxPBg==", "dev": true, "requires": { - "@webassemblyjs/ast": "1.11.6", - "@webassemblyjs/helper-buffer": "1.11.6", - "@webassemblyjs/wasm-gen": "1.11.6", - "@webassemblyjs/wasm-parser": "1.11.6" + "@webassemblyjs/ast": "1.12.1", + "@webassemblyjs/helper-buffer": "1.12.1", + "@webassemblyjs/wasm-gen": "1.12.1", + "@webassemblyjs/wasm-parser": "1.12.1" } }, "@webassemblyjs/wasm-parser": { - "version": "1.11.6", - "resolved": "https://registry.npmjs.org/@webassemblyjs/wasm-parser/-/wasm-parser-1.11.6.tgz", - "integrity": "sha512-6ZwPeGzMJM3Dqp3hCsLgESxBGtT/OeCvCZ4TA1JUPYgmhAx38tTPR9JaKy0S5H3evQpO/h2uWs2j6Yc/fjkpTQ==", + "version": "1.12.1", + "resolved": "https://registry.npmjs.org/@webassemblyjs/wasm-parser/-/wasm-parser-1.12.1.tgz", + "integrity": "sha512-xikIi7c2FHXysxXe3COrVUPSheuBtpcfhbpFj4gmu7KRLYOzANztwUU0IbsqvMqzuNK2+glRGWCEqZo1WCLyAQ==", "dev": true, "requires": { - "@webassemblyjs/ast": "1.11.6", + "@webassemblyjs/ast": "1.12.1", "@webassemblyjs/helper-api-error": "1.11.6", "@webassemblyjs/helper-wasm-bytecode": "1.11.6", "@webassemblyjs/ieee754": "1.11.6", @@ -4800,12 +4759,12 @@ } }, "@webassemblyjs/wast-printer": { - "version": "1.11.6", - "resolved": "https://registry.npmjs.org/@webassemblyjs/wast-printer/-/wast-printer-1.11.6.tgz", - "integrity": "sha512-JM7AhRcE+yW2GWYaKeHL5vt4xqee5N2WcezptmgyhNS+ScggqcT1OtXykhAb13Sn5Yas0j2uv9tHgrjwvzAP4A==", + "version": "1.12.1", + "resolved": "https://registry.npmjs.org/@webassemblyjs/wast-printer/-/wast-printer-1.12.1.tgz", + "integrity": "sha512-+X4WAlOisVWQMikjbcvY2e0rwPsKQ9F688lksZhBcPycBBuii3O7m8FACbDMWDojpAqvjIncrG8J0XHKyQfVeA==", "dev": true, "requires": { - "@webassemblyjs/ast": "1.11.6", + "@webassemblyjs/ast": "1.12.1", "@xtuc/long": "4.2.2" } }, @@ -4853,15 +4812,15 @@ } }, "acorn": { - "version": "8.10.0", - "resolved": "https://registry.npmjs.org/acorn/-/acorn-8.10.0.tgz", - "integrity": "sha512-F0SAmZ8iUtS//m8DmCTA0jlh6TDKkHQyK6xc6V4KDTyZKA9dnvX9/3sRTVQrWm79glUAZbnmmNcdYwUIHWVybw==", + "version": "8.12.1", + "resolved": "https://registry.npmjs.org/acorn/-/acorn-8.12.1.tgz", + "integrity": "sha512-tcpGyI9zbizT9JbV6oYE477V6mTlXvvi0T0G3SNIYE2apm/G5huBa1+K89VGeovbg+jycCrfhl3ADxErOuO6Jg==", "dev": true }, - "acorn-import-assertions": { - "version": "1.9.0", - "resolved": "https://registry.npmjs.org/acorn-import-assertions/-/acorn-import-assertions-1.9.0.tgz", - "integrity": "sha512-cmMwop9x+8KFhxvKrKfPYmN6/pKTYYHBqLa0DfvVZcKMJWNyWLnaqND7dx/qn66R7ewM1UX5XMaDVP5wlVTaVA==", + "acorn-import-attributes": { + "version": "1.9.5", + "resolved": "https://registry.npmjs.org/acorn-import-attributes/-/acorn-import-attributes-1.9.5.tgz", + "integrity": "sha512-n02Vykv5uA3eHGM/Z2dQrcD56kL8TyDb2p1+0P83PClMnC/nc+anbQRhIOWnSq4Ke/KvDPrY3C9hDtC/A3eHnQ==", "dev": true, "requires": {} }, @@ -5415,9 +5374,9 @@ "dev": true }, "enhanced-resolve": { - "version": "5.15.0", - "resolved": "https://registry.npmjs.org/enhanced-resolve/-/enhanced-resolve-5.15.0.tgz", - "integrity": "sha512-LXYT42KJ7lpIKECr2mAXIaMldcNCh/7E0KBKOu4KSfkHmP+mZmSs+8V5gBAqisWBy0OO4W5Oyys0GO1Y8KtdKg==", + "version": "5.17.1", + "resolved": "https://registry.npmjs.org/enhanced-resolve/-/enhanced-resolve-5.17.1.tgz", + "integrity": "sha512-LMHl3dXhTcfv8gM4kEzIUeTQ+7fpdA0l2tUf34BddXPkz2A5xJ5L/Pchd5BL6rdccM9QGvu0sWZzK1Z1t4wwyg==", "dev": true, "requires": { "graceful-fs": "^4.2.4", @@ -7199,9 +7158,9 @@ } }, "terser": { - "version": "5.20.0", - "resolved": "https://registry.npmjs.org/terser/-/terser-5.20.0.tgz", - "integrity": "sha512-e56ETryaQDyebBwJIWYB2TT6f2EZ0fL0sW/JRXNMN26zZdKi2u/E/5my5lG6jNxym6qsrVXfFRmOdV42zlAgLQ==", + "version": "5.31.6", + "resolved": "https://registry.npmjs.org/terser/-/terser-5.31.6.tgz", + "integrity": "sha512-PQ4DAriWzKj+qgehQ7LK5bQqCFNMmlhjR2PFFLuqGCpuCAauxemVBWwWOxo3UIwWQx8+Pr61Df++r76wDmkQBg==", "dev": true, "requires": { "@jridgewell/source-map": "^0.3.3", @@ -7211,22 +7170,22 @@ } }, "terser-webpack-plugin": { - "version": "5.3.9", - "resolved": "https://registry.npmjs.org/terser-webpack-plugin/-/terser-webpack-plugin-5.3.9.tgz", - "integrity": "sha512-ZuXsqE07EcggTWQjXUj+Aot/OMcD0bMKGgF63f7UxYcu5/AJF53aIpK1YoP5xR9l6s/Hy2b+t1AM0bLNPRuhwA==", + "version": "5.3.10", + "resolved": "https://registry.npmjs.org/terser-webpack-plugin/-/terser-webpack-plugin-5.3.10.tgz", + "integrity": "sha512-BKFPWlPDndPs+NGGCr1U59t0XScL5317Y0UReNrHaw9/FwhPENlq6bfgs+4yPfyP51vqC1bQ4rp1EfXW5ZSH9w==", "dev": true, "requires": { - "@jridgewell/trace-mapping": "^0.3.17", + "@jridgewell/trace-mapping": "^0.3.20", "jest-worker": "^27.4.5", "schema-utils": "^3.1.1", "serialize-javascript": "^6.0.1", - "terser": "^5.16.8" + "terser": "^5.26.0" }, "dependencies": { "serialize-javascript": { - "version": "6.0.1", - "resolved": "https://registry.npmjs.org/serialize-javascript/-/serialize-javascript-6.0.1.tgz", - "integrity": "sha512-owoXEFjWRllis8/M1Q+Cw5k8ZH40e3zhp/ovX+Xr/vi1qj6QesbyXXViFbpNvWvPNAD62SutwEXavefrLJWj7w==", + "version": "6.0.2", + "resolved": "https://registry.npmjs.org/serialize-javascript/-/serialize-javascript-6.0.2.tgz", + "integrity": "sha512-Saa1xPByTTq2gdeFZYLLo+RFE35NHZkAbqZeWNd3BpzppeVisAqpDjcp8dyf6uIvEqJRd46jemmyA4iFIeVk8g==", "dev": true, "requires": { "randombytes": "^2.1.0" @@ -7339,9 +7298,9 @@ "dev": true }, "watchpack": { - "version": "2.4.0", - "resolved": "https://registry.npmjs.org/watchpack/-/watchpack-2.4.0.tgz", - "integrity": "sha512-Lcvm7MGST/4fup+ifyKi2hjyIAwcdI4HRgtvTpIUxBRhB+RFtUh8XtDOxUfctVCnhVi+QQj49i91OyvzkJl6cg==", + "version": "2.4.2", + "resolved": "https://registry.npmjs.org/watchpack/-/watchpack-2.4.2.tgz", + "integrity": "sha512-TnbFSbcOCcDgjZ4piURLCbJ3nJhznVh9kw6F6iokjiFPl8ONxe9A6nMDVXDiNbrSfLILs6vB07F7wLBrwPYzJw==", "dev": true, "requires": { "glob-to-regexp": "^0.4.1", @@ -7358,34 +7317,33 @@ } }, "webpack": { - "version": "5.88.2", - "resolved": "https://registry.npmjs.org/webpack/-/webpack-5.88.2.tgz", - "integrity": "sha512-JmcgNZ1iKj+aiR0OvTYtWQqJwq37Pf683dY9bVORwVbUrDhLhdn/PlO2sHsFHPkj7sHNQF3JwaAkp49V+Sq1tQ==", + "version": "5.94.0", + "resolved": "https://registry.npmjs.org/webpack/-/webpack-5.94.0.tgz", + "integrity": "sha512-KcsGn50VT+06JH/iunZJedYGUJS5FGjow8wb9c0v5n1Om8O1g4L6LjtfxwlXIATopoQu+vOXXa7gYisWxCoPyg==", "dev": true, "requires": { - "@types/eslint-scope": "^3.7.3", - "@types/estree": "^1.0.0", - "@webassemblyjs/ast": "^1.11.5", - "@webassemblyjs/wasm-edit": "^1.11.5", - "@webassemblyjs/wasm-parser": "^1.11.5", + "@types/estree": "^1.0.5", + "@webassemblyjs/ast": "^1.12.1", + "@webassemblyjs/wasm-edit": "^1.12.1", + "@webassemblyjs/wasm-parser": "^1.12.1", "acorn": "^8.7.1", - "acorn-import-assertions": "^1.9.0", - "browserslist": "^4.14.5", + "acorn-import-attributes": "^1.9.5", + "browserslist": "^4.21.10", "chrome-trace-event": "^1.0.2", - "enhanced-resolve": "^5.15.0", + "enhanced-resolve": "^5.17.1", "es-module-lexer": "^1.2.1", "eslint-scope": "5.1.1", "events": "^3.2.0", "glob-to-regexp": "^0.4.1", - "graceful-fs": "^4.2.9", + "graceful-fs": "^4.2.11", "json-parse-even-better-errors": "^2.3.1", "loader-runner": "^4.2.0", "mime-types": "^2.1.27", "neo-async": "^2.6.2", "schema-utils": "^3.2.0", "tapable": "^2.1.1", - "terser-webpack-plugin": "^5.3.7", - "watchpack": "^2.4.0", + "terser-webpack-plugin": "^5.3.10", + "watchpack": "^2.4.1", "webpack-sources": "^3.2.3" }, "dependencies": { diff --git a/datafusion/wasmtest/datafusion-wasm-app/package.json b/datafusion/wasmtest/datafusion-wasm-app/package.json index cd32070fa0bc..0860473276ea 100644 --- a/datafusion/wasmtest/datafusion-wasm-app/package.json +++ b/datafusion/wasmtest/datafusion-wasm-app/package.json @@ -27,7 +27,7 @@ "datafusion-wasmtest": "../pkg" }, "devDependencies": { - "webpack": "5.88.2", + "webpack": "5.94.0", "webpack-cli": "5.1.4", "webpack-dev-server": "4.15.1", "copy-webpack-plugin": "6.4.1" From 5272007f364b8d43deab19f530e6d7a6ffe7b678 Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Sun, 1 Sep 2024 14:59:32 +0200 Subject: [PATCH 061/154] Avoid redundant pass-by-value in physical optimizer (#12261) --- .../enforce_distribution.rs | 6 ++--- .../physical_optimizer/projection_pushdown.rs | 24 ++++++++--------- .../src/physical_optimizer/sort_pushdown.rs | 26 ++++++++++--------- .../physical_optimizer/topk_aggregation.rs | 14 +++++----- 4 files changed, 35 insertions(+), 35 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/enforce_distribution.rs b/datafusion/core/src/physical_optimizer/enforce_distribution.rs index ba6f7d0439c2..095590fe03f6 100644 --- a/datafusion/core/src/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/src/physical_optimizer/enforce_distribution.rs @@ -309,7 +309,7 @@ fn adjust_input_keys_ordering( return reorder_partitioned_join_keys( requirements, on, - vec![], + &[], &join_constructor, ) .map(Transformed::yes); @@ -373,7 +373,7 @@ fn adjust_input_keys_ordering( return reorder_partitioned_join_keys( requirements, on, - sort_options.clone(), + sort_options, &join_constructor, ) .map(Transformed::yes); @@ -421,7 +421,7 @@ fn adjust_input_keys_ordering( fn reorder_partitioned_join_keys( mut join_plan: PlanWithKeyRequirements, on: &[(PhysicalExprRef, PhysicalExprRef)], - sort_options: Vec, + sort_options: &[SortOptions], join_constructor: &F, ) -> Result where diff --git a/datafusion/core/src/physical_optimizer/projection_pushdown.rs b/datafusion/core/src/physical_optimizer/projection_pushdown.rs index b3f3f90154d0..e09332e8c36a 100644 --- a/datafusion/core/src/physical_optimizer/projection_pushdown.rs +++ b/datafusion/core/src/physical_optimizer/projection_pushdown.rs @@ -631,7 +631,7 @@ fn try_pushdown_through_hash_join( if !join_allows_pushdown( &projection_as_columns, - hash_join.schema(), + &hash_join.schema(), far_right_left_col_ind, far_left_right_col_ind, ) { @@ -662,7 +662,7 @@ fn try_pushdown_through_hash_join( }; let (new_left, new_right) = new_join_children( - projection_as_columns, + &projection_as_columns, far_right_left_col_ind, far_left_right_col_ind, hash_join.left(), @@ -700,7 +700,7 @@ fn try_swapping_with_cross_join( if !join_allows_pushdown( &projection_as_columns, - cross_join.schema(), + &cross_join.schema(), far_right_left_col_ind, far_left_right_col_ind, ) { @@ -708,7 +708,7 @@ fn try_swapping_with_cross_join( } let (new_left, new_right) = new_join_children( - projection_as_columns, + &projection_as_columns, far_right_left_col_ind, far_left_right_col_ind, cross_join.left(), @@ -740,7 +740,7 @@ fn try_swapping_with_nested_loop_join( if !join_allows_pushdown( &projection_as_columns, - nl_join.schema(), + &nl_join.schema(), far_right_left_col_ind, far_left_right_col_ind, ) { @@ -762,7 +762,7 @@ fn try_swapping_with_nested_loop_join( }; let (new_left, new_right) = new_join_children( - projection_as_columns, + &projection_as_columns, far_right_left_col_ind, far_left_right_col_ind, nl_join.left(), @@ -796,7 +796,7 @@ fn try_swapping_with_sort_merge_join( if !join_allows_pushdown( &projection_as_columns, - sm_join.schema(), + &sm_join.schema(), far_right_left_col_ind, far_left_right_col_ind, ) { @@ -813,7 +813,7 @@ fn try_swapping_with_sort_merge_join( }; let (new_left, new_right) = new_join_children( - projection_as_columns, + &projection_as_columns, far_right_left_col_ind, far_left_right_col_ind, sm_join.children()[0], @@ -850,7 +850,7 @@ fn try_swapping_with_sym_hash_join( if !join_allows_pushdown( &projection_as_columns, - sym_join.schema(), + &sym_join.schema(), far_right_left_col_ind, far_left_right_col_ind, ) { @@ -881,7 +881,7 @@ fn try_swapping_with_sym_hash_join( }; let (new_left, new_right) = new_join_children( - projection_as_columns, + &projection_as_columns, far_right_left_col_ind, far_left_right_col_ind, sym_join.left(), @@ -1243,7 +1243,7 @@ fn new_indices_for_join_filter( /// - Left or right table is not lost after the projection. fn join_allows_pushdown( projection_as_columns: &[(Column, String)], - join_schema: SchemaRef, + join_schema: &SchemaRef, far_right_left_col_ind: i32, far_left_right_col_ind: i32, ) -> bool { @@ -1260,7 +1260,7 @@ fn join_allows_pushdown( /// this function constructs the new [`ProjectionExec`]s that will come on top /// of the original children of the join. fn new_join_children( - projection_as_columns: Vec<(Column, String)>, + projection_as_columns: &[(Column, String)], far_right_left_col_ind: i32, far_left_right_col_ind: i32, left_child: &Arc, diff --git a/datafusion/core/src/physical_optimizer/sort_pushdown.rs b/datafusion/core/src/physical_optimizer/sort_pushdown.rs index 9ab6802d18f1..41059ef7b1ef 100644 --- a/datafusion/core/src/physical_optimizer/sort_pushdown.rs +++ b/datafusion/core/src/physical_optimizer/sort_pushdown.rs @@ -165,7 +165,7 @@ fn pushdown_requirement_to_children( if is_window(plan) { let required_input_ordering = plan.required_input_ordering(); let request_child = required_input_ordering[0].as_deref().unwrap_or(&[]); - let child_plan = plan.children().swap_remove(0).clone(); + let child_plan = plan.children().swap_remove(0); match determine_children_requirement(parent_required, request_child, child_plan) { RequirementsCompatibility::Satisfy => { let req = (!request_child.is_empty()).then(|| request_child.to_vec()); @@ -225,7 +225,7 @@ fn pushdown_requirement_to_children( Some(JoinSide::Left) => try_pushdown_requirements_to_join( smj, parent_required, - parent_required_expr, + &parent_required_expr, JoinSide::Left, ), Some(JoinSide::Right) => { @@ -238,7 +238,7 @@ fn pushdown_requirement_to_children( try_pushdown_requirements_to_join( smj, parent_required, - new_right_required_expr, + &new_right_required_expr, JoinSide::Right, ) } @@ -321,7 +321,7 @@ fn pushdown_would_violate_requirements( fn determine_children_requirement( parent_required: LexRequirementRef, request_child: LexRequirementRef, - child_plan: Arc, + child_plan: &Arc, ) -> RequirementsCompatibility { if child_plan .equivalence_properties() @@ -344,7 +344,7 @@ fn determine_children_requirement( fn try_pushdown_requirements_to_join( smj: &SortMergeJoinExec, parent_required: LexRequirementRef, - sort_expr: Vec, + sort_expr: &[PhysicalSortExpr], push_side: JoinSide, ) -> Result>>> { let left_eq_properties = smj.left().equivalence_properties(); @@ -356,25 +356,27 @@ fn try_pushdown_requirements_to_join( let right_ordering = smj.right().output_ordering().unwrap_or(&[]); let (new_left_ordering, new_right_ordering) = match push_side { JoinSide::Left => { - let left_eq_properties = - left_eq_properties.clone().with_reorder(sort_expr.clone()); + let left_eq_properties = left_eq_properties + .clone() + .with_reorder(Vec::from(sort_expr)); if left_eq_properties .ordering_satisfy_requirement(&left_requirement.unwrap_or_default()) { // After re-ordering requirement is still satisfied - (sort_expr.as_slice(), right_ordering) + (sort_expr, right_ordering) } else { return Ok(None); } } JoinSide::Right => { - let right_eq_properties = - right_eq_properties.clone().with_reorder(sort_expr.clone()); + let right_eq_properties = right_eq_properties + .clone() + .with_reorder(Vec::from(sort_expr)); if right_eq_properties .ordering_satisfy_requirement(&right_requirement.unwrap_or_default()) { // After re-ordering requirement is still satisfied - (left_ordering, sort_expr.as_slice()) + (left_ordering, sort_expr) } else { return Ok(None); } @@ -397,7 +399,7 @@ fn try_pushdown_requirements_to_join( let should_pushdown = smj_eqs.ordering_satisfy_requirement(parent_required); Ok(should_pushdown.then(|| { let mut required_input_ordering = smj.required_input_ordering(); - let new_req = Some(PhysicalSortRequirement::from_sort_exprs(&sort_expr)); + let new_req = Some(PhysicalSortRequirement::from_sort_exprs(sort_expr)); match push_side { JoinSide::Left => { required_input_ordering[0] = new_req; diff --git a/datafusion/core/src/physical_optimizer/topk_aggregation.rs b/datafusion/core/src/physical_optimizer/topk_aggregation.rs index 82cf44ad7796..b2c8f640fda0 100644 --- a/datafusion/core/src/physical_optimizer/topk_aggregation.rs +++ b/datafusion/core/src/physical_optimizer/topk_aggregation.rs @@ -84,7 +84,7 @@ impl TopKAggregation { Some(Arc::new(new_aggr)) } - fn transform_sort(plan: Arc) -> Option> { + fn transform_sort(plan: &Arc) -> Option> { let sort = plan.as_any().downcast_ref::()?; let children = sort.children(); @@ -142,13 +142,11 @@ impl PhysicalOptimizerRule for TopKAggregation { ) -> Result> { if config.optimizer.enable_topk_aggregation { plan.transform_down(|plan| { - Ok( - if let Some(plan) = TopKAggregation::transform_sort(plan.clone()) { - Transformed::yes(plan) - } else { - Transformed::no(plan) - }, - ) + Ok(if let Some(plan) = TopKAggregation::transform_sort(&plan) { + Transformed::yes(plan) + } else { + Transformed::no(plan) + }) }) .data() } else { From ff7d76d8fe5527101557327cf0047f1ef72ee893 Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Sun, 1 Sep 2024 15:00:27 +0200 Subject: [PATCH 062/154] Remove FileScanConfig::repartition_file_groups function (#12242) Deprecated since 33.0.0. --- .../physical_plan/file_scan_config.rs | 17 +---------------- 1 file changed, 1 insertion(+), 16 deletions(-) diff --git a/datafusion/core/src/datasource/physical_plan/file_scan_config.rs b/datafusion/core/src/datasource/physical_plan/file_scan_config.rs index 3ea467539adc..9f674185694d 100644 --- a/datafusion/core/src/datasource/physical_plan/file_scan_config.rs +++ b/datafusion/core/src/datasource/physical_plan/file_scan_config.rs @@ -22,9 +22,7 @@ use std::{ borrow::Cow, collections::HashMap, fmt::Debug, marker::PhantomData, sync::Arc, vec, }; -use super::{ - get_projected_output_ordering, statistics::MinMaxStatistics, FileGroupPartitioner, -}; +use super::{get_projected_output_ordering, statistics::MinMaxStatistics}; use crate::datasource::{listing::PartitionedFile, object_store::ObjectStoreUrl}; use crate::{error::Result, scalar::ScalarValue}; @@ -296,19 +294,6 @@ impl FileScanConfig { }) } - #[allow(missing_docs)] - #[deprecated(since = "33.0.0", note = "Use SessionContext::new_with_config")] - pub fn repartition_file_groups( - file_groups: Vec>, - target_partitions: usize, - repartition_file_min_size: usize, - ) -> Option>> { - FileGroupPartitioner::new() - .with_target_partitions(target_partitions) - .with_repartition_file_min_size(repartition_file_min_size) - .repartition_file_groups(&file_groups) - } - /// Attempts to do a bin-packing on files into file groups, such that any two files /// in a file group are ordered and non-overlapping with respect to their statistics. /// It will produce the smallest number of file groups possible. From 780cccb5201f9c37eba647419c63a2a55625adfc Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Sun, 1 Sep 2024 09:16:14 -0400 Subject: [PATCH 063/154] Minor: Improve ExecutionMode documentation (#12214) * Minor: Improve ExecutionMode documentation * Update datafusion/physical-plan/src/execution_plan.rs Co-authored-by: Mehmet Ozan Kabak * Update datafusion/physical-plan/src/execution_plan.rs Co-authored-by: Mehmet Ozan Kabak --------- Co-authored-by: Mehmet Ozan Kabak --- .../physical-plan/src/execution_plan.rs | 45 +++++++++++++------ 1 file changed, 32 insertions(+), 13 deletions(-) diff --git a/datafusion/physical-plan/src/execution_plan.rs b/datafusion/physical-plan/src/execution_plan.rs index c1c66f6d3923..53ae59f7072f 100644 --- a/datafusion/physical-plan/src/execution_plan.rs +++ b/datafusion/physical-plan/src/execution_plan.rs @@ -482,24 +482,43 @@ impl ExecutionPlanProperties for &dyn ExecutionPlan { } } -/// Describes the execution mode of an operator's resulting stream with respect -/// to its size and behavior. There are three possible execution modes: `Bounded`, -/// `Unbounded` and `PipelineBreaking`. +/// Describes the execution mode of the result of calling +/// [`ExecutionPlan::execute`] with respect to its size and behavior. +/// +/// The mode of the execution plan is determined by the mode of its input +/// execution plans and the details of the operator itself. For example, a +/// `FilterExec` operator will have the same execution mode as its input, but a +/// `SortExec` operator may have a different execution mode than its input, +/// depending on how the input stream is sorted. +/// +/// There are three possible execution modes: `Bounded`, `Unbounded` and +/// `PipelineBreaking`. #[derive(Clone, Copy, PartialEq, Debug)] pub enum ExecutionMode { - /// Represents the mode where generated stream is bounded, e.g. finite. + /// The stream is bounded / finite. + /// + /// In this case the stream will eventually return `None` to indicate that + /// there are no more records to process. Bounded, - /// Represents the mode where generated stream is unbounded, e.g. infinite. - /// Even though the operator generates an unbounded stream of results, it - /// works with bounded memory and execution can still continue successfully. + /// The stream is unbounded / infinite. /// - /// The stream that results from calling `execute` on an `ExecutionPlan` that is `Unbounded` - /// will never be done (return `None`), except in case of error. + /// In this case, the stream will never be done (never return `None`), + /// except in case of error. + /// + /// This mode is often used in "Steaming" use cases where data is + /// incrementally processed as it arrives. + /// + /// Note that even though the operator generates an unbounded stream of + /// results, it can execute with bounded memory and incrementally produces + /// output. Unbounded, - /// Represents the mode where some of the operator's input stream(s) are - /// unbounded; however, the operator cannot generate streaming results from - /// these streaming inputs. In this case, the execution mode will be pipeline - /// breaking, e.g. the operator requires unbounded memory to generate results. + /// Some of the operator's input stream(s) are unbounded, but the operator + /// cannot generate streaming results from these streaming inputs. + /// + /// In this case, the execution mode will be pipeline breaking, e.g. the + /// operator requires unbounded memory to generate results. This + /// information is used by the planner when performing sanity checks + /// on plans processings unbounded data sources. PipelineBreaking, } From d86b7f968d55a25326d56d80f0cdd17d6a81c748 Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Mon, 2 Sep 2024 03:31:19 +0200 Subject: [PATCH 064/154] Avoid unnecessary move when setting SessionConfig (#12260) --- datafusion/common/src/config.rs | 2 +- datafusion/core/src/dataframe/mod.rs | 8 +++++--- datafusion/core/src/dataframe/parquet.rs | 2 +- datafusion/core/src/datasource/listing/table.rs | 4 ++-- datafusion/core/src/execution/context/parquet.rs | 2 +- datafusion/execution/src/config.rs | 6 +++--- datafusion/physical-plan/src/aggregates/mod.rs | 8 ++++---- 7 files changed, 17 insertions(+), 15 deletions(-) diff --git a/datafusion/common/src/config.rs b/datafusion/common/src/config.rs index 37d26c6f00c4..4a0c61910ba4 100644 --- a/datafusion/common/src/config.rs +++ b/datafusion/common/src/config.rs @@ -781,7 +781,7 @@ impl ConfigOptions { /// /// Only the built-in configurations will be extracted from the hash map /// and other key value pairs will be ignored. - pub fn from_string_hash_map(settings: HashMap) -> Result { + pub fn from_string_hash_map(settings: &HashMap) -> Result { struct Visitor(Vec); impl Visit for Visitor { diff --git a/datafusion/core/src/dataframe/mod.rs b/datafusion/core/src/dataframe/mod.rs index 5dbeb535a546..b8c0bd9d744c 100644 --- a/datafusion/core/src/dataframe/mod.rs +++ b/datafusion/core/src/dataframe/mod.rs @@ -3277,7 +3277,7 @@ mod tests { #[tokio::test] async fn with_column_renamed_case_sensitive() -> Result<()> { let config = - SessionConfig::from_string_hash_map(std::collections::HashMap::from([( + SessionConfig::from_string_hash_map(&std::collections::HashMap::from([( "datafusion.sql_parser.enable_ident_normalization".to_owned(), "false".to_owned(), )]))?; @@ -3713,8 +3713,10 @@ mod tests { // Test issue: https://github.com/apache/datafusion/issues/12065 #[tokio::test] async fn filtered_aggr_with_param_values() -> Result<()> { - let cfg = SessionConfig::new() - .set("datafusion.sql_parser.dialect", "PostgreSQL".into()); + let cfg = SessionConfig::new().set( + "datafusion.sql_parser.dialect", + &ScalarValue::from("PostgreSQL"), + ); let ctx = SessionContext::new_with_config(cfg); register_aggregate_csv(&ctx, "table1").await?; diff --git a/datafusion/core/src/dataframe/parquet.rs b/datafusion/core/src/dataframe/parquet.rs index 2a23f045f3b2..66974e37f453 100644 --- a/datafusion/core/src/dataframe/parquet.rs +++ b/datafusion/core/src/dataframe/parquet.rs @@ -190,7 +190,7 @@ mod tests { // This test verifies writing a parquet file with small rg size // relative to datafusion.execution.batch_size does not panic let ctx = SessionContext::new_with_config(SessionConfig::from_string_hash_map( - HashMap::from_iter( + &HashMap::from_iter( [("datafusion.execution.batch_size", "10")] .iter() .map(|(s1, s2)| (s1.to_string(), s2.to_string())), diff --git a/datafusion/core/src/datasource/listing/table.rs b/datafusion/core/src/datasource/listing/table.rs index 1f5fa738b253..35286612a887 100644 --- a/datafusion/core/src/datasource/listing/table.rs +++ b/datafusion/core/src/datasource/listing/table.rs @@ -1781,7 +1781,7 @@ mod tests { // Create the initial context, schema, and batch. let session_ctx = match session_config_map { Some(cfg) => { - let config = SessionConfig::from_string_hash_map(cfg)?; + let config = SessionConfig::from_string_hash_map(&cfg)?; SessionContext::new_with_config(config) } None => SessionContext::new(), @@ -1979,7 +1979,7 @@ mod tests { // Create the initial context let session_ctx = match session_config_map { Some(cfg) => { - let config = SessionConfig::from_string_hash_map(cfg)?; + let config = SessionConfig::from_string_hash_map(&cfg)?; SessionContext::new_with_config(config) } None => SessionContext::new(), diff --git a/datafusion/core/src/execution/context/parquet.rs b/datafusion/core/src/execution/context/parquet.rs index fef20df6e69d..1d83c968c1a8 100644 --- a/datafusion/core/src/execution/context/parquet.rs +++ b/datafusion/core/src/execution/context/parquet.rs @@ -106,7 +106,7 @@ mod tests { #[tokio::test] async fn read_with_glob_path_issue_2465() -> Result<()> { let config = - SessionConfig::from_string_hash_map(std::collections::HashMap::from([( + SessionConfig::from_string_hash_map(&std::collections::HashMap::from([( "datafusion.execution.listing_table_ignore_subdirectory".to_owned(), "false".to_owned(), )]))?; diff --git a/datafusion/execution/src/config.rs b/datafusion/execution/src/config.rs index 7f872db049a5..c2b1f9e74128 100644 --- a/datafusion/execution/src/config.rs +++ b/datafusion/execution/src/config.rs @@ -48,7 +48,7 @@ use datafusion_common::{ /// use datafusion_common::ScalarValue; /// /// let config = SessionConfig::new() -/// .set("datafusion.execution.batch_size", ScalarValue::UInt64(Some(1234))) +/// .set("datafusion.execution.batch_size", &ScalarValue::UInt64(Some(1234))) /// .set_bool("datafusion.execution.parquet.pushdown_filters", true); /// /// assert_eq!(config.batch_size(), 1234); @@ -123,7 +123,7 @@ impl SessionConfig { } /// Create new ConfigOptions struct, taking values from a string hash map. - pub fn from_string_hash_map(settings: HashMap) -> Result { + pub fn from_string_hash_map(settings: &HashMap) -> Result { Ok(ConfigOptions::from_string_hash_map(settings)?.into()) } @@ -157,7 +157,7 @@ impl SessionConfig { } /// Set a configuration option - pub fn set(self, key: &str, value: ScalarValue) -> Self { + pub fn set(self, key: &str, value: &ScalarValue) -> Self { self.set_str(key, &value.to_string()) } diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 2f974f1ef4e0..161ffb6c6541 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -2373,11 +2373,11 @@ mod tests { let mut session_config = SessionConfig::default(); session_config = session_config.set( "datafusion.execution.skip_partial_aggregation_probe_rows_threshold", - ScalarValue::Int64(Some(2)), + &ScalarValue::Int64(Some(2)), ); session_config = session_config.set( "datafusion.execution.skip_partial_aggregation_probe_ratio_threshold", - ScalarValue::Float64(Some(0.1)), + &ScalarValue::Float64(Some(0.1)), ); let ctx = TaskContext::default().with_session_config(session_config); @@ -2462,11 +2462,11 @@ mod tests { let mut session_config = SessionConfig::default(); session_config = session_config.set( "datafusion.execution.skip_partial_aggregation_probe_rows_threshold", - ScalarValue::Int64(Some(5)), + &ScalarValue::Int64(Some(5)), ); session_config = session_config.set( "datafusion.execution.skip_partial_aggregation_probe_ratio_threshold", - ScalarValue::Float64(Some(0.1)), + &ScalarValue::Float64(Some(0.1)), ); let ctx = TaskContext::default().with_session_config(session_config); From dd3208943d728d845497d6a12ce4c0eacc061dcd Mon Sep 17 00:00:00 2001 From: JC <1950050+jc4x4@users.noreply.github.com> Date: Mon, 2 Sep 2024 09:41:21 +0800 Subject: [PATCH 065/154] Remove unused `AggregateOptions` struct and `scalar_update_factor` config setting (#12241) * Remove scalar_update_factor from AggregateOptions Keeping the AggregateOptions struct in case we need it later. https://github.com/apache/datafusion/issues/11998 * Remove AggregateOption; update configs.md * Run prettier * Fix cargo doc --- datafusion/common/src/config.rs | 24 --- datafusion/execution/src/config.rs | 14 -- .../test_files/information_schema.slt | 2 - docs/source/user-guide/configs.md | 173 +++++++++--------- 4 files changed, 86 insertions(+), 127 deletions(-) diff --git a/datafusion/common/src/config.rs b/datafusion/common/src/config.rs index 4a0c61910ba4..7c247103f6e7 100644 --- a/datafusion/common/src/config.rs +++ b/datafusion/common/src/config.rs @@ -261,9 +261,6 @@ config_namespace! { /// Parquet options pub parquet: ParquetOptions, default = Default::default() - /// Aggregate options - pub aggregate: AggregateOptions, default = Default::default() - /// Fan-out during initial physical planning. /// /// This is mostly use to plan `UNION` children in parallel. @@ -493,27 +490,6 @@ config_namespace! { } } -config_namespace! { - /// Options related to aggregate execution - /// - /// See also: [`SessionConfig`] - /// - /// [`SessionConfig`]: https://docs.rs/datafusion/latest/datafusion/prelude/struct.SessionConfig.html - pub struct AggregateOptions { - /// Specifies the threshold for using `ScalarValue`s to update - /// accumulators during high-cardinality aggregations for each input batch. - /// - /// The aggregation is considered high-cardinality if the number of affected groups - /// is greater than or equal to `batch_size / scalar_update_factor`. In such cases, - /// `ScalarValue`s are utilized for updating accumulators, rather than the default - /// batch-slice approach. This can lead to performance improvements. - /// - /// By adjusting the `scalar_update_factor`, you can balance the trade-off between - /// more efficient accumulator updates and the number of groups affected. - pub scalar_update_factor: usize, default = 10 - } -} - config_namespace! { /// Options related to query optimization /// diff --git a/datafusion/execution/src/config.rs b/datafusion/execution/src/config.rs index c2b1f9e74128..cede75d21ca4 100644 --- a/datafusion/execution/src/config.rs +++ b/datafusion/execution/src/config.rs @@ -78,7 +78,6 @@ use datafusion_common::{ /// | --------- | ------------- | /// | `datafusion.catalog` | [CatalogOptions][datafusion_common::config::CatalogOptions] | /// | `datafusion.execution` | [ExecutionOptions][datafusion_common::config::ExecutionOptions] | -/// | `datafusion.execution.aggregate` | [AggregateOptions][datafusion_common::config::AggregateOptions] | /// | `datafusion.execution.parquet` | [ParquetOptions][datafusion_common::config::ParquetOptions] | /// | `datafusion.optimizer` | [OptimizerOptions][datafusion_common::config::OptimizerOptions] | /// | `datafusion.sql_parser` | [SqlParserOptions][datafusion_common::config::SqlParserOptions] | @@ -383,19 +382,6 @@ impl SessionConfig { self.options.execution.batch_size } - /// Get the currently configured scalar_update_factor for aggregate - pub fn agg_scalar_update_factor(&self) -> usize { - self.options.execution.aggregate.scalar_update_factor - } - - /// Customize scalar_update_factor for aggregate - pub fn with_agg_scalar_update_factor(mut self, n: usize) -> Self { - // scalar update factor must be greater than zero - assert!(n > 0); - self.options.execution.aggregate.scalar_update_factor = n; - self - } - /// Enables or disables the coalescence of small batches into larger batches pub fn with_coalesce_batches(mut self, enabled: bool) -> Self { self.options.execution.coalesce_batches = enabled; diff --git a/datafusion/sqllogictest/test_files/information_schema.slt b/datafusion/sqllogictest/test_files/information_schema.slt index efd017a90bc4..f797a7a6539d 100644 --- a/datafusion/sqllogictest/test_files/information_schema.slt +++ b/datafusion/sqllogictest/test_files/information_schema.slt @@ -169,7 +169,6 @@ datafusion.catalog.has_header true datafusion.catalog.information_schema true datafusion.catalog.location NULL datafusion.catalog.newlines_in_values false -datafusion.execution.aggregate.scalar_update_factor 10 datafusion.execution.batch_size 8192 datafusion.execution.coalesce_batches true datafusion.execution.collect_statistics false @@ -259,7 +258,6 @@ datafusion.catalog.has_header true Default value for `format.has_header` for `CR datafusion.catalog.information_schema true Should DataFusion provide access to `information_schema` virtual tables for displaying schema information datafusion.catalog.location NULL Location scanned to load tables for `default` schema datafusion.catalog.newlines_in_values false Specifies whether newlines in (quoted) CSV values are supported. This is the default value for `format.newlines_in_values` for `CREATE EXTERNAL TABLE` if not specified explicitly in the statement. Parsing newlines in quoted values may be affected by execution behaviour such as parallel file scanning. Setting this to `true` ensures that newlines in values are parsed successfully, which may reduce performance. -datafusion.execution.aggregate.scalar_update_factor 10 Specifies the threshold for using `ScalarValue`s to update accumulators during high-cardinality aggregations for each input batch. The aggregation is considered high-cardinality if the number of affected groups is greater than or equal to `batch_size / scalar_update_factor`. In such cases, `ScalarValue`s are utilized for updating accumulators, rather than the default batch-slice approach. This can lead to performance improvements. By adjusting the `scalar_update_factor`, you can balance the trade-off between more efficient accumulator updates and the number of groups affected. datafusion.execution.batch_size 8192 Default batch size while creating new batches, it's especially useful for buffer-in-memory batches since creating tiny batches would result in too much metadata memory consumption datafusion.execution.coalesce_batches true When set to true, record batches will be examined between each operator and small batches will be coalesced into larger batches. This is helpful when there are highly selective filters or joins that could produce tiny output batches. The target batch size is determined by the configuration setting datafusion.execution.collect_statistics false Should DataFusion collect statistics after listing files diff --git a/docs/source/user-guide/configs.md b/docs/source/user-guide/configs.md index 4255307781b6..8514fb1fbd93 100644 --- a/docs/source/user-guide/configs.md +++ b/docs/source/user-guide/configs.md @@ -35,90 +35,89 @@ Values are parsed according to the [same rules used in casts from Utf8](https:// If the value in the environment variable cannot be cast to the type of the configuration option, the default value will be used instead and a warning emitted. Environment variables are read during `SessionConfig` initialisation so they must be set beforehand and will not affect running sessions. -| key | default | description | -| ----------------------------------------------------------------------- | ------------------------- | ------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- | -| datafusion.catalog.create_default_catalog_and_schema | true | Whether the default catalog and schema should be created automatically. | -| datafusion.catalog.default_catalog | datafusion | The default catalog name - this impacts what SQL queries use if not specified | -| datafusion.catalog.default_schema | public | The default schema name - this impacts what SQL queries use if not specified | -| datafusion.catalog.information_schema | false | Should DataFusion provide access to `information_schema` virtual tables for displaying schema information | -| datafusion.catalog.location | NULL | Location scanned to load tables for `default` schema | -| datafusion.catalog.format | NULL | Type of `TableProvider` to use when loading `default` schema | -| datafusion.catalog.has_header | true | Default value for `format.has_header` for `CREATE EXTERNAL TABLE` if not specified explicitly in the statement. | -| datafusion.catalog.newlines_in_values | false | Specifies whether newlines in (quoted) CSV values are supported. This is the default value for `format.newlines_in_values` for `CREATE EXTERNAL TABLE` if not specified explicitly in the statement. Parsing newlines in quoted values may be affected by execution behaviour such as parallel file scanning. Setting this to `true` ensures that newlines in values are parsed successfully, which may reduce performance. | -| datafusion.execution.batch_size | 8192 | Default batch size while creating new batches, it's especially useful for buffer-in-memory batches since creating tiny batches would result in too much metadata memory consumption | -| datafusion.execution.coalesce_batches | true | When set to true, record batches will be examined between each operator and small batches will be coalesced into larger batches. This is helpful when there are highly selective filters or joins that could produce tiny output batches. The target batch size is determined by the configuration setting | -| datafusion.execution.collect_statistics | false | Should DataFusion collect statistics after listing files | -| datafusion.execution.target_partitions | 0 | Number of partitions for query execution. Increasing partitions can increase concurrency. Defaults to the number of CPU cores on the system | -| datafusion.execution.time_zone | +00:00 | The default time zone Some functions, e.g. `EXTRACT(HOUR from SOME_TIME)`, shift the underlying datetime according to this time zone, and then extract the hour | -| datafusion.execution.parquet.enable_page_index | true | (reading) If true, reads the Parquet data page level metadata (the Page Index), if present, to reduce the I/O and number of rows decoded. | -| datafusion.execution.parquet.pruning | true | (reading) If true, the parquet reader attempts to skip entire row groups based on the predicate in the query and the metadata (min/max values) stored in the parquet file | -| datafusion.execution.parquet.skip_metadata | true | (reading) If true, the parquet reader skip the optional embedded metadata that may be in the file Schema. This setting can help avoid schema conflicts when querying multiple parquet files with schemas containing compatible types but different metadata | -| datafusion.execution.parquet.metadata_size_hint | NULL | (reading) If specified, the parquet reader will try and fetch the last `size_hint` bytes of the parquet file optimistically. If not specified, two reads are required: One read to fetch the 8-byte parquet footer and another to fetch the metadata length encoded in the footer | -| datafusion.execution.parquet.pushdown_filters | false | (reading) If true, filter expressions are be applied during the parquet decoding operation to reduce the number of rows decoded. This optimization is sometimes called "late materialization". | -| datafusion.execution.parquet.reorder_filters | false | (reading) If true, filter expressions evaluated during the parquet decoding operation will be reordered heuristically to minimize the cost of evaluation. If false, the filters are applied in the same order as written in the query | -| datafusion.execution.parquet.data_pagesize_limit | 1048576 | (writing) Sets best effort maximum size of data page in bytes | -| datafusion.execution.parquet.write_batch_size | 1024 | (writing) Sets write_batch_size in bytes | -| datafusion.execution.parquet.writer_version | 1.0 | (writing) Sets parquet writer version valid values are "1.0" and "2.0" | -| datafusion.execution.parquet.compression | zstd(3) | (writing) Sets default parquet compression codec. Valid values are: uncompressed, snappy, gzip(level), lzo, brotli(level), lz4, zstd(level), and lz4_raw. These values are not case sensitive. If NULL, uses default parquet writer setting Note that this default setting is not the same as the default parquet writer setting. | -| datafusion.execution.parquet.dictionary_enabled | true | (writing) Sets if dictionary encoding is enabled. If NULL, uses default parquet writer setting | -| datafusion.execution.parquet.dictionary_page_size_limit | 1048576 | (writing) Sets best effort maximum dictionary page size, in bytes | -| datafusion.execution.parquet.statistics_enabled | page | (writing) Sets if statistics are enabled for any column Valid values are: "none", "chunk", and "page" These values are not case sensitive. If NULL, uses default parquet writer setting | -| datafusion.execution.parquet.max_statistics_size | 4096 | (writing) Sets max statistics size for any column. If NULL, uses default parquet writer setting | -| datafusion.execution.parquet.max_row_group_size | 1048576 | (writing) Target maximum number of rows in each row group (defaults to 1M rows). Writing larger row groups requires more memory to write, but can get better compression and be faster to read. | -| datafusion.execution.parquet.created_by | datafusion version 41.0.0 | (writing) Sets "created by" property | -| datafusion.execution.parquet.column_index_truncate_length | 64 | (writing) Sets column index truncate length | -| datafusion.execution.parquet.data_page_row_count_limit | 20000 | (writing) Sets best effort maximum number of rows in data page | -| datafusion.execution.parquet.encoding | NULL | (writing) Sets default encoding for any column. Valid values are: plain, plain_dictionary, rle, bit_packed, delta_binary_packed, delta_length_byte_array, delta_byte_array, rle_dictionary, and byte_stream_split. These values are not case sensitive. If NULL, uses default parquet writer setting | -| datafusion.execution.parquet.bloom_filter_on_read | true | (writing) Use any available bloom filters when reading parquet files | -| datafusion.execution.parquet.bloom_filter_on_write | false | (writing) Write bloom filters for all columns when creating parquet files | -| datafusion.execution.parquet.bloom_filter_fpp | NULL | (writing) Sets bloom filter false positive probability. If NULL, uses default parquet writer setting | -| datafusion.execution.parquet.bloom_filter_ndv | NULL | (writing) Sets bloom filter number of distinct values. If NULL, uses default parquet writer setting | -| datafusion.execution.parquet.allow_single_file_parallelism | true | (writing) Controls whether DataFusion will attempt to speed up writing parquet files by serializing them in parallel. Each column in each row group in each output file are serialized in parallel leveraging a maximum possible core count of n_files*n_row_groups*n_columns. | -| datafusion.execution.parquet.maximum_parallel_row_group_writers | 1 | (writing) By default parallel parquet writer is tuned for minimum memory usage in a streaming execution plan. You may see a performance benefit when writing large parquet files by increasing maximum_parallel_row_group_writers and maximum_buffered_record_batches_per_stream if your system has idle cores and can tolerate additional memory usage. Boosting these values is likely worthwhile when writing out already in-memory data, such as from a cached data frame. | -| datafusion.execution.parquet.maximum_buffered_record_batches_per_stream | 2 | (writing) By default parallel parquet writer is tuned for minimum memory usage in a streaming execution plan. You may see a performance benefit when writing large parquet files by increasing maximum_parallel_row_group_writers and maximum_buffered_record_batches_per_stream if your system has idle cores and can tolerate additional memory usage. Boosting these values is likely worthwhile when writing out already in-memory data, such as from a cached data frame. | -| datafusion.execution.parquet.schema_force_string_view | false | (reading) If true, parquet reader will read columns of `Utf8/Utf8Large` with `Utf8View`, and `Binary/BinaryLarge` with `BinaryView`. | -| datafusion.execution.aggregate.scalar_update_factor | 10 | Specifies the threshold for using `ScalarValue`s to update accumulators during high-cardinality aggregations for each input batch. The aggregation is considered high-cardinality if the number of affected groups is greater than or equal to `batch_size / scalar_update_factor`. In such cases, `ScalarValue`s are utilized for updating accumulators, rather than the default batch-slice approach. This can lead to performance improvements. By adjusting the `scalar_update_factor`, you can balance the trade-off between more efficient accumulator updates and the number of groups affected. | -| datafusion.execution.planning_concurrency | 0 | Fan-out during initial physical planning. This is mostly use to plan `UNION` children in parallel. Defaults to the number of CPU cores on the system | -| datafusion.execution.sort_spill_reservation_bytes | 10485760 | Specifies the reserved memory for each spillable sort operation to facilitate an in-memory merge. When a sort operation spills to disk, the in-memory data must be sorted and merged before being written to a file. This setting reserves a specific amount of memory for that in-memory sort/merge process. Note: This setting is irrelevant if the sort operation cannot spill (i.e., if there's no `DiskManager` configured). | -| datafusion.execution.sort_in_place_threshold_bytes | 1048576 | When sorting, below what size should data be concatenated and sorted in a single RecordBatch rather than sorted in batches and merged. | -| datafusion.execution.meta_fetch_concurrency | 32 | Number of files to read in parallel when inferring schema and statistics | -| datafusion.execution.minimum_parallel_output_files | 4 | Guarantees a minimum level of output files running in parallel. RecordBatches will be distributed in round robin fashion to each parallel writer. Each writer is closed and a new file opened once soft_max_rows_per_output_file is reached. | -| datafusion.execution.soft_max_rows_per_output_file | 50000000 | Target number of rows in output files when writing multiple. This is a soft max, so it can be exceeded slightly. There also will be one file smaller than the limit if the total number of rows written is not roughly divisible by the soft max | -| datafusion.execution.max_buffered_batches_per_output_file | 2 | This is the maximum number of RecordBatches buffered for each output file being worked. Higher values can potentially give faster write performance at the cost of higher peak memory consumption | -| datafusion.execution.listing_table_ignore_subdirectory | true | Should sub directories be ignored when scanning directories for data files. Defaults to true (ignores subdirectories), consistent with Hive. Note that this setting does not affect reading partitioned tables (e.g. `/table/year=2021/month=01/data.parquet`). | -| datafusion.execution.enable_recursive_ctes | true | Should DataFusion support recursive CTEs | -| datafusion.execution.split_file_groups_by_statistics | false | Attempt to eliminate sorts by packing & sorting files with non-overlapping statistics into the same file groups. Currently experimental | -| datafusion.execution.keep_partition_by_columns | false | Should DataFusion keep the columns used for partition_by in the output RecordBatches | -| datafusion.execution.skip_partial_aggregation_probe_ratio_threshold | 0.8 | Aggregation ratio (number of distinct groups / number of input rows) threshold for skipping partial aggregation. If the value is greater then partial aggregation will skip aggregation for further input | -| datafusion.execution.skip_partial_aggregation_probe_rows_threshold | 100000 | Number of input rows partial aggregation partition should process, before aggregation ratio check and trying to switch to skipping aggregation mode | -| datafusion.execution.use_row_number_estimates_to_optimize_partitioning | false | Should DataFusion use row number estimates at the input to decide whether increasing parallelism is beneficial or not. By default, only exact row numbers (not estimates) are used for this decision. Setting this flag to `true` will likely produce better plans. if the source of statistics is accurate. We plan to make this the default in the future. | -| datafusion.optimizer.enable_distinct_aggregation_soft_limit | true | When set to true, the optimizer will push a limit operation into grouped aggregations which have no aggregate expressions, as a soft limit, emitting groups once the limit is reached, before all rows in the group are read. | -| datafusion.optimizer.enable_round_robin_repartition | true | When set to true, the physical plan optimizer will try to add round robin repartitioning to increase parallelism to leverage more CPU cores | -| datafusion.optimizer.enable_topk_aggregation | true | When set to true, the optimizer will attempt to perform limit operations during aggregations, if possible | -| datafusion.optimizer.filter_null_join_keys | false | When set to true, the optimizer will insert filters before a join between a nullable and non-nullable column to filter out nulls on the nullable side. This filter can add additional overhead when the file format does not fully support predicate push down. | -| datafusion.optimizer.repartition_aggregations | true | Should DataFusion repartition data using the aggregate keys to execute aggregates in parallel using the provided `target_partitions` level | -| datafusion.optimizer.repartition_file_min_size | 10485760 | Minimum total files size in bytes to perform file scan repartitioning. | -| datafusion.optimizer.repartition_joins | true | Should DataFusion repartition data using the join keys to execute joins in parallel using the provided `target_partitions` level | -| datafusion.optimizer.allow_symmetric_joins_without_pruning | true | Should DataFusion allow symmetric hash joins for unbounded data sources even when its inputs do not have any ordering or filtering If the flag is not enabled, the SymmetricHashJoin operator will be unable to prune its internal buffers, resulting in certain join types - such as Full, Left, LeftAnti, LeftSemi, Right, RightAnti, and RightSemi - being produced only at the end of the execution. This is not typical in stream processing. Additionally, without proper design for long runner execution, all types of joins may encounter out-of-memory errors. | -| datafusion.optimizer.repartition_file_scans | true | When set to `true`, file groups will be repartitioned to achieve maximum parallelism. Currently Parquet and CSV formats are supported. If set to `true`, all files will be repartitioned evenly (i.e., a single large file might be partitioned into smaller chunks) for parallel scanning. If set to `false`, different files will be read in parallel, but repartitioning won't happen within a single file. | -| datafusion.optimizer.repartition_windows | true | Should DataFusion repartition data using the partitions keys to execute window functions in parallel using the provided `target_partitions` level | -| datafusion.optimizer.repartition_sorts | true | Should DataFusion execute sorts in a per-partition fashion and merge afterwards instead of coalescing first and sorting globally. With this flag is enabled, plans in the form below `text "SortExec: [a@0 ASC]", " CoalescePartitionsExec", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", ` would turn into the plan below which performs better in multithreaded environments `text "SortPreservingMergeExec: [a@0 ASC]", " SortExec: [a@0 ASC]", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", ` | -| datafusion.optimizer.prefer_existing_sort | false | When true, DataFusion will opportunistically remove sorts when the data is already sorted, (i.e. setting `preserve_order` to true on `RepartitionExec` and using `SortPreservingMergeExec`) When false, DataFusion will maximize plan parallelism using `RepartitionExec` even if this requires subsequently resorting data using a `SortExec`. | -| datafusion.optimizer.skip_failed_rules | false | When set to true, the logical plan optimizer will produce warning messages if any optimization rules produce errors and then proceed to the next rule. When set to false, any rules that produce errors will cause the query to fail | -| datafusion.optimizer.max_passes | 3 | Number of times that the optimizer will attempt to optimize the plan | -| datafusion.optimizer.top_down_join_key_reordering | true | When set to true, the physical plan optimizer will run a top down process to reorder the join keys | -| datafusion.optimizer.prefer_hash_join | true | When set to true, the physical plan optimizer will prefer HashJoin over SortMergeJoin. HashJoin can work more efficiently than SortMergeJoin but consumes more memory | -| datafusion.optimizer.hash_join_single_partition_threshold | 1048576 | The maximum estimated size in bytes for one input side of a HashJoin will be collected into a single partition | -| datafusion.optimizer.hash_join_single_partition_threshold_rows | 131072 | The maximum estimated size in rows for one input side of a HashJoin will be collected into a single partition | -| datafusion.optimizer.default_filter_selectivity | 20 | The default filter selectivity used by Filter Statistics when an exact selectivity cannot be determined. Valid values are between 0 (no selectivity) and 100 (all rows are selected). | -| datafusion.optimizer.prefer_existing_union | false | When set to true, the optimizer will not attempt to convert Union to Interleave | -| datafusion.explain.logical_plan_only | false | When set to true, the explain statement will only print logical plans | -| datafusion.explain.physical_plan_only | false | When set to true, the explain statement will only print physical plans | -| datafusion.explain.show_statistics | false | When set to true, the explain statement will print operator statistics for physical plans | -| datafusion.explain.show_sizes | true | When set to true, the explain statement will print the partition sizes | -| datafusion.explain.show_schema | false | When set to true, the explain statement will print schema information | -| datafusion.sql_parser.parse_float_as_decimal | false | When set to true, SQL parser will parse float as decimal type | -| datafusion.sql_parser.enable_ident_normalization | true | When set to true, SQL parser will normalize ident (convert ident to lowercase when not quoted) | -| datafusion.sql_parser.enable_options_value_normalization | true | When set to true, SQL parser will normalize options value (convert value to lowercase) | -| datafusion.sql_parser.dialect | generic | Configure the SQL dialect used by DataFusion's parser; supported values include: Generic, MySQL, PostgreSQL, Hive, SQLite, Snowflake, Redshift, MsSQL, ClickHouse, BigQuery, and Ansi. | -| datafusion.sql_parser.support_varchar_with_length | true | If true, permit lengths for `VARCHAR` such as `VARCHAR(20)`, but ignore the length. If false, error if a `VARCHAR` with a length is specified. The Arrow type system does not have a notion of maximum string length and thus DataFusion can not enforce such limits. | +| key | default | description | +| ----------------------------------------------------------------------- | ------------------------- | ------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------ | +| datafusion.catalog.create_default_catalog_and_schema | true | Whether the default catalog and schema should be created automatically. | +| datafusion.catalog.default_catalog | datafusion | The default catalog name - this impacts what SQL queries use if not specified | +| datafusion.catalog.default_schema | public | The default schema name - this impacts what SQL queries use if not specified | +| datafusion.catalog.information_schema | false | Should DataFusion provide access to `information_schema` virtual tables for displaying schema information | +| datafusion.catalog.location | NULL | Location scanned to load tables for `default` schema | +| datafusion.catalog.format | NULL | Type of `TableProvider` to use when loading `default` schema | +| datafusion.catalog.has_header | true | Default value for `format.has_header` for `CREATE EXTERNAL TABLE` if not specified explicitly in the statement. | +| datafusion.catalog.newlines_in_values | false | Specifies whether newlines in (quoted) CSV values are supported. This is the default value for `format.newlines_in_values` for `CREATE EXTERNAL TABLE` if not specified explicitly in the statement. Parsing newlines in quoted values may be affected by execution behaviour such as parallel file scanning. Setting this to `true` ensures that newlines in values are parsed successfully, which may reduce performance. | +| datafusion.execution.batch_size | 8192 | Default batch size while creating new batches, it's especially useful for buffer-in-memory batches since creating tiny batches would result in too much metadata memory consumption | +| datafusion.execution.coalesce_batches | true | When set to true, record batches will be examined between each operator and small batches will be coalesced into larger batches. This is helpful when there are highly selective filters or joins that could produce tiny output batches. The target batch size is determined by the configuration setting | +| datafusion.execution.collect_statistics | false | Should DataFusion collect statistics after listing files | +| datafusion.execution.target_partitions | 0 | Number of partitions for query execution. Increasing partitions can increase concurrency. Defaults to the number of CPU cores on the system | +| datafusion.execution.time_zone | +00:00 | The default time zone Some functions, e.g. `EXTRACT(HOUR from SOME_TIME)`, shift the underlying datetime according to this time zone, and then extract the hour | +| datafusion.execution.parquet.enable_page_index | true | (reading) If true, reads the Parquet data page level metadata (the Page Index), if present, to reduce the I/O and number of rows decoded. | +| datafusion.execution.parquet.pruning | true | (reading) If true, the parquet reader attempts to skip entire row groups based on the predicate in the query and the metadata (min/max values) stored in the parquet file | +| datafusion.execution.parquet.skip_metadata | true | (reading) If true, the parquet reader skip the optional embedded metadata that may be in the file Schema. This setting can help avoid schema conflicts when querying multiple parquet files with schemas containing compatible types but different metadata | +| datafusion.execution.parquet.metadata_size_hint | NULL | (reading) If specified, the parquet reader will try and fetch the last `size_hint` bytes of the parquet file optimistically. If not specified, two reads are required: One read to fetch the 8-byte parquet footer and another to fetch the metadata length encoded in the footer | +| datafusion.execution.parquet.pushdown_filters | false | (reading) If true, filter expressions are be applied during the parquet decoding operation to reduce the number of rows decoded. This optimization is sometimes called "late materialization". | +| datafusion.execution.parquet.reorder_filters | false | (reading) If true, filter expressions evaluated during the parquet decoding operation will be reordered heuristically to minimize the cost of evaluation. If false, the filters are applied in the same order as written in the query | +| datafusion.execution.parquet.data_pagesize_limit | 1048576 | (writing) Sets best effort maximum size of data page in bytes | +| datafusion.execution.parquet.write_batch_size | 1024 | (writing) Sets write_batch_size in bytes | +| datafusion.execution.parquet.writer_version | 1.0 | (writing) Sets parquet writer version valid values are "1.0" and "2.0" | +| datafusion.execution.parquet.compression | zstd(3) | (writing) Sets default parquet compression codec. Valid values are: uncompressed, snappy, gzip(level), lzo, brotli(level), lz4, zstd(level), and lz4_raw. These values are not case sensitive. If NULL, uses default parquet writer setting Note that this default setting is not the same as the default parquet writer setting. | +| datafusion.execution.parquet.dictionary_enabled | true | (writing) Sets if dictionary encoding is enabled. If NULL, uses default parquet writer setting | +| datafusion.execution.parquet.dictionary_page_size_limit | 1048576 | (writing) Sets best effort maximum dictionary page size, in bytes | +| datafusion.execution.parquet.statistics_enabled | page | (writing) Sets if statistics are enabled for any column Valid values are: "none", "chunk", and "page" These values are not case sensitive. If NULL, uses default parquet writer setting | +| datafusion.execution.parquet.max_statistics_size | 4096 | (writing) Sets max statistics size for any column. If NULL, uses default parquet writer setting | +| datafusion.execution.parquet.max_row_group_size | 1048576 | (writing) Target maximum number of rows in each row group (defaults to 1M rows). Writing larger row groups requires more memory to write, but can get better compression and be faster to read. | +| datafusion.execution.parquet.created_by | datafusion version 41.0.0 | (writing) Sets "created by" property | +| datafusion.execution.parquet.column_index_truncate_length | 64 | (writing) Sets column index truncate length | +| datafusion.execution.parquet.data_page_row_count_limit | 20000 | (writing) Sets best effort maximum number of rows in data page | +| datafusion.execution.parquet.encoding | NULL | (writing) Sets default encoding for any column. Valid values are: plain, plain_dictionary, rle, bit_packed, delta_binary_packed, delta_length_byte_array, delta_byte_array, rle_dictionary, and byte_stream_split. These values are not case sensitive. If NULL, uses default parquet writer setting | +| datafusion.execution.parquet.bloom_filter_on_read | true | (writing) Use any available bloom filters when reading parquet files | +| datafusion.execution.parquet.bloom_filter_on_write | false | (writing) Write bloom filters for all columns when creating parquet files | +| datafusion.execution.parquet.bloom_filter_fpp | NULL | (writing) Sets bloom filter false positive probability. If NULL, uses default parquet writer setting | +| datafusion.execution.parquet.bloom_filter_ndv | NULL | (writing) Sets bloom filter number of distinct values. If NULL, uses default parquet writer setting | +| datafusion.execution.parquet.allow_single_file_parallelism | true | (writing) Controls whether DataFusion will attempt to speed up writing parquet files by serializing them in parallel. Each column in each row group in each output file are serialized in parallel leveraging a maximum possible core count of n_files*n_row_groups*n_columns. | +| datafusion.execution.parquet.maximum_parallel_row_group_writers | 1 | (writing) By default parallel parquet writer is tuned for minimum memory usage in a streaming execution plan. You may see a performance benefit when writing large parquet files by increasing maximum_parallel_row_group_writers and maximum_buffered_record_batches_per_stream if your system has idle cores and can tolerate additional memory usage. Boosting these values is likely worthwhile when writing out already in-memory data, such as from a cached data frame. | +| datafusion.execution.parquet.maximum_buffered_record_batches_per_stream | 2 | (writing) By default parallel parquet writer is tuned for minimum memory usage in a streaming execution plan. You may see a performance benefit when writing large parquet files by increasing maximum_parallel_row_group_writers and maximum_buffered_record_batches_per_stream if your system has idle cores and can tolerate additional memory usage. Boosting these values is likely worthwhile when writing out already in-memory data, such as from a cached data frame. | +| datafusion.execution.parquet.schema_force_string_view | false | (reading) If true, parquet reader will read columns of `Utf8/Utf8Large` with `Utf8View`, and `Binary/BinaryLarge` with `BinaryView`. | +| datafusion.execution.planning_concurrency | 0 | Fan-out during initial physical planning. This is mostly use to plan `UNION` children in parallel. Defaults to the number of CPU cores on the system | +| datafusion.execution.sort_spill_reservation_bytes | 10485760 | Specifies the reserved memory for each spillable sort operation to facilitate an in-memory merge. When a sort operation spills to disk, the in-memory data must be sorted and merged before being written to a file. This setting reserves a specific amount of memory for that in-memory sort/merge process. Note: This setting is irrelevant if the sort operation cannot spill (i.e., if there's no `DiskManager` configured). | +| datafusion.execution.sort_in_place_threshold_bytes | 1048576 | When sorting, below what size should data be concatenated and sorted in a single RecordBatch rather than sorted in batches and merged. | +| datafusion.execution.meta_fetch_concurrency | 32 | Number of files to read in parallel when inferring schema and statistics | +| datafusion.execution.minimum_parallel_output_files | 4 | Guarantees a minimum level of output files running in parallel. RecordBatches will be distributed in round robin fashion to each parallel writer. Each writer is closed and a new file opened once soft_max_rows_per_output_file is reached. | +| datafusion.execution.soft_max_rows_per_output_file | 50000000 | Target number of rows in output files when writing multiple. This is a soft max, so it can be exceeded slightly. There also will be one file smaller than the limit if the total number of rows written is not roughly divisible by the soft max | +| datafusion.execution.max_buffered_batches_per_output_file | 2 | This is the maximum number of RecordBatches buffered for each output file being worked. Higher values can potentially give faster write performance at the cost of higher peak memory consumption | +| datafusion.execution.listing_table_ignore_subdirectory | true | Should sub directories be ignored when scanning directories for data files. Defaults to true (ignores subdirectories), consistent with Hive. Note that this setting does not affect reading partitioned tables (e.g. `/table/year=2021/month=01/data.parquet`). | +| datafusion.execution.enable_recursive_ctes | true | Should DataFusion support recursive CTEs | +| datafusion.execution.split_file_groups_by_statistics | false | Attempt to eliminate sorts by packing & sorting files with non-overlapping statistics into the same file groups. Currently experimental | +| datafusion.execution.keep_partition_by_columns | false | Should DataFusion keep the columns used for partition_by in the output RecordBatches | +| datafusion.execution.skip_partial_aggregation_probe_ratio_threshold | 0.8 | Aggregation ratio (number of distinct groups / number of input rows) threshold for skipping partial aggregation. If the value is greater then partial aggregation will skip aggregation for further input | +| datafusion.execution.skip_partial_aggregation_probe_rows_threshold | 100000 | Number of input rows partial aggregation partition should process, before aggregation ratio check and trying to switch to skipping aggregation mode | +| datafusion.execution.use_row_number_estimates_to_optimize_partitioning | false | Should DataFusion use row number estimates at the input to decide whether increasing parallelism is beneficial or not. By default, only exact row numbers (not estimates) are used for this decision. Setting this flag to `true` will likely produce better plans. if the source of statistics is accurate. We plan to make this the default in the future. | +| datafusion.optimizer.enable_distinct_aggregation_soft_limit | true | When set to true, the optimizer will push a limit operation into grouped aggregations which have no aggregate expressions, as a soft limit, emitting groups once the limit is reached, before all rows in the group are read. | +| datafusion.optimizer.enable_round_robin_repartition | true | When set to true, the physical plan optimizer will try to add round robin repartitioning to increase parallelism to leverage more CPU cores | +| datafusion.optimizer.enable_topk_aggregation | true | When set to true, the optimizer will attempt to perform limit operations during aggregations, if possible | +| datafusion.optimizer.filter_null_join_keys | false | When set to true, the optimizer will insert filters before a join between a nullable and non-nullable column to filter out nulls on the nullable side. This filter can add additional overhead when the file format does not fully support predicate push down. | +| datafusion.optimizer.repartition_aggregations | true | Should DataFusion repartition data using the aggregate keys to execute aggregates in parallel using the provided `target_partitions` level | +| datafusion.optimizer.repartition_file_min_size | 10485760 | Minimum total files size in bytes to perform file scan repartitioning. | +| datafusion.optimizer.repartition_joins | true | Should DataFusion repartition data using the join keys to execute joins in parallel using the provided `target_partitions` level | +| datafusion.optimizer.allow_symmetric_joins_without_pruning | true | Should DataFusion allow symmetric hash joins for unbounded data sources even when its inputs do not have any ordering or filtering If the flag is not enabled, the SymmetricHashJoin operator will be unable to prune its internal buffers, resulting in certain join types - such as Full, Left, LeftAnti, LeftSemi, Right, RightAnti, and RightSemi - being produced only at the end of the execution. This is not typical in stream processing. Additionally, without proper design for long runner execution, all types of joins may encounter out-of-memory errors. | +| datafusion.optimizer.repartition_file_scans | true | When set to `true`, file groups will be repartitioned to achieve maximum parallelism. Currently Parquet and CSV formats are supported. If set to `true`, all files will be repartitioned evenly (i.e., a single large file might be partitioned into smaller chunks) for parallel scanning. If set to `false`, different files will be read in parallel, but repartitioning won't happen within a single file. | +| datafusion.optimizer.repartition_windows | true | Should DataFusion repartition data using the partitions keys to execute window functions in parallel using the provided `target_partitions` level | +| datafusion.optimizer.repartition_sorts | true | Should DataFusion execute sorts in a per-partition fashion and merge afterwards instead of coalescing first and sorting globally. With this flag is enabled, plans in the form below `text "SortExec: [a@0 ASC]", " CoalescePartitionsExec", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", ` would turn into the plan below which performs better in multithreaded environments `text "SortPreservingMergeExec: [a@0 ASC]", " SortExec: [a@0 ASC]", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", ` | +| datafusion.optimizer.prefer_existing_sort | false | When true, DataFusion will opportunistically remove sorts when the data is already sorted, (i.e. setting `preserve_order` to true on `RepartitionExec` and using `SortPreservingMergeExec`) When false, DataFusion will maximize plan parallelism using `RepartitionExec` even if this requires subsequently resorting data using a `SortExec`. | +| datafusion.optimizer.skip_failed_rules | false | When set to true, the logical plan optimizer will produce warning messages if any optimization rules produce errors and then proceed to the next rule. When set to false, any rules that produce errors will cause the query to fail | +| datafusion.optimizer.max_passes | 3 | Number of times that the optimizer will attempt to optimize the plan | +| datafusion.optimizer.top_down_join_key_reordering | true | When set to true, the physical plan optimizer will run a top down process to reorder the join keys | +| datafusion.optimizer.prefer_hash_join | true | When set to true, the physical plan optimizer will prefer HashJoin over SortMergeJoin. HashJoin can work more efficiently than SortMergeJoin but consumes more memory | +| datafusion.optimizer.hash_join_single_partition_threshold | 1048576 | The maximum estimated size in bytes for one input side of a HashJoin will be collected into a single partition | +| datafusion.optimizer.hash_join_single_partition_threshold_rows | 131072 | The maximum estimated size in rows for one input side of a HashJoin will be collected into a single partition | +| datafusion.optimizer.default_filter_selectivity | 20 | The default filter selectivity used by Filter Statistics when an exact selectivity cannot be determined. Valid values are between 0 (no selectivity) and 100 (all rows are selected). | +| datafusion.optimizer.prefer_existing_union | false | When set to true, the optimizer will not attempt to convert Union to Interleave | +| datafusion.explain.logical_plan_only | false | When set to true, the explain statement will only print logical plans | +| datafusion.explain.physical_plan_only | false | When set to true, the explain statement will only print physical plans | +| datafusion.explain.show_statistics | false | When set to true, the explain statement will print operator statistics for physical plans | +| datafusion.explain.show_sizes | true | When set to true, the explain statement will print the partition sizes | +| datafusion.explain.show_schema | false | When set to true, the explain statement will print schema information | +| datafusion.sql_parser.parse_float_as_decimal | false | When set to true, SQL parser will parse float as decimal type | +| datafusion.sql_parser.enable_ident_normalization | true | When set to true, SQL parser will normalize ident (convert ident to lowercase when not quoted) | +| datafusion.sql_parser.enable_options_value_normalization | true | When set to true, SQL parser will normalize options value (convert value to lowercase) | +| datafusion.sql_parser.dialect | generic | Configure the SQL dialect used by DataFusion's parser; supported values include: Generic, MySQL, PostgreSQL, Hive, SQLite, Snowflake, Redshift, MsSQL, ClickHouse, BigQuery, and Ansi. | +| datafusion.sql_parser.support_varchar_with_length | true | If true, permit lengths for `VARCHAR` such as `VARCHAR(20)`, but ignore the length. If false, error if a `VARCHAR` with a length is specified. The Arrow type system does not have a notion of maximum string length and thus DataFusion can not enforce such limits. | From 53de592381b8585f7d9de82f2ac1eb58b1524e44 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=BC=A0=E6=9E=97=E4=BC=9F?= Date: Mon, 2 Sep 2024 16:48:24 +0800 Subject: [PATCH 066/154] Make group expressions nullable more accurate (#12256) * Make group expressions nullable more accurate * Add test --- .../physical-plan/src/aggregates/mod.rs | 69 ++++++++++++++++--- 1 file changed, 61 insertions(+), 8 deletions(-) diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 161ffb6c6541..e66a57fd2ee5 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -164,9 +164,17 @@ impl PhysicalGroupBy { } } - /// Returns true if this GROUP BY contains NULL expressions - pub fn contains_null(&self) -> bool { - self.groups.iter().flatten().any(|is_null| *is_null) + /// Calculate GROUP BY expressions nullable + pub fn exprs_nullable(&self) -> Vec { + let mut exprs_nullable = vec![false; self.expr.len()]; + for group in self.groups.iter() { + group.iter().enumerate().for_each(|(index, is_null)| { + if *is_null { + exprs_nullable[index] = true; + } + }) + } + exprs_nullable } /// Returns the group expressions @@ -278,7 +286,7 @@ pub struct AggregateExec { } impl AggregateExec { - /// Function used in `ConvertFirstLast` optimizer rule, + /// Function used in `OptimizeAggregateOrder` optimizer rule, /// where we need parts of the new value, others cloned from the old one /// Rewrites aggregate exec with new aggregate expressions. pub fn with_new_aggr_exprs( @@ -319,7 +327,7 @@ impl AggregateExec { &input.schema(), &group_by.expr, &aggr_expr, - group_by.contains_null(), + group_by.exprs_nullable(), mode, )?; @@ -793,18 +801,18 @@ fn create_schema( input_schema: &Schema, group_expr: &[(Arc, String)], aggr_expr: &[Arc], - contains_null_expr: bool, + group_expr_nullable: Vec, mode: AggregateMode, ) -> Result { let mut fields = Vec::with_capacity(group_expr.len() + aggr_expr.len()); - for (expr, name) in group_expr { + for (index, (expr, name)) in group_expr.iter().enumerate() { fields.push(Field::new( name, expr.data_type(input_schema)?, // In cases where we have multiple grouping sets, we will use NULL expressions in // order to align the grouping sets. So the field must be nullable even if the underlying // schema field is not. - contains_null_expr || expr.nullable(input_schema)?, + group_expr_nullable[index] || expr.nullable(input_schema)?, )) } @@ -2489,4 +2497,49 @@ mod tests { Ok(()) } + + #[test] + fn group_exprs_nullable() -> Result<()> { + let input_schema = Arc::new(Schema::new(vec![ + Field::new("a", DataType::Float32, false), + Field::new("b", DataType::Float32, false), + ])); + + let aggr_expr = + vec![ + AggregateExprBuilder::new(count_udaf(), vec![col("a", &input_schema)?]) + .schema(Arc::clone(&input_schema)) + .alias("COUNT(a)") + .build()?, + ]; + + let grouping_set = PhysicalGroupBy { + expr: vec![ + (col("a", &input_schema)?, "a".to_string()), + (col("b", &input_schema)?, "b".to_string()), + ], + null_expr: vec![ + (lit(ScalarValue::Float32(None)), "a".to_string()), + (lit(ScalarValue::Float32(None)), "b".to_string()), + ], + groups: vec![ + vec![false, true], // (a, NULL) + vec![false, false], // (a,b) + ], + }; + let aggr_schema = create_schema( + &input_schema, + &grouping_set.expr, + &aggr_expr, + grouping_set.exprs_nullable(), + AggregateMode::Final, + )?; + let expected_schema = Schema::new(vec![ + Field::new("a", DataType::Float32, false), + Field::new("b", DataType::Float32, true), + Field::new("COUNT(a)", DataType::Int64, false), + ]); + assert_eq!(aggr_schema, expected_schema); + Ok(()) + } } From 447cb02a3a0a26cf696e7dffddb91b85822c2e02 Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Mon, 2 Sep 2024 11:17:15 +0200 Subject: [PATCH 067/154] Avoid redundant pass-by-value in optimizer (#12262) --- .../core/src/execution/session_state.rs | 4 +-- datafusion/expr/src/utils.rs | 2 +- .../src/analyzer/expand_wildcard_rule.rs | 6 ++-- .../src/analyzer/function_rewrite.rs | 2 +- datafusion/optimizer/src/analyzer/subquery.rs | 3 +- .../optimizer/src/analyzer/type_coercion.rs | 32 +++++++++---------- .../optimizer/src/common_subexpr_eliminate.rs | 6 ++-- datafusion/optimizer/src/decorrelate.rs | 14 ++++---- .../src/decorrelate_predicate_subquery.rs | 2 +- .../optimizer/src/eliminate_cross_join.rs | 2 +- datafusion/optimizer/src/join_key_set.rs | 4 +-- .../optimizer/src/optimize_projections/mod.rs | 4 +-- datafusion/optimizer/src/optimizer.rs | 6 ++-- .../optimizer/src/scalar_subquery_to_join.rs | 8 ++--- .../simplify_expressions/expr_simplifier.rs | 20 ++++++------ .../simplify_expressions/simplify_exprs.rs | 2 +- .../src/simplify_expressions/utils.rs | 4 +-- .../src/unwrap_cast_in_comparison.rs | 20 ++++++------ datafusion/optimizer/src/utils.rs | 6 ++-- 19 files changed, 73 insertions(+), 74 deletions(-) diff --git a/datafusion/core/src/execution/session_state.rs b/datafusion/core/src/execution/session_state.rs index 88a90e1e1d09..90f175b59385 100644 --- a/datafusion/core/src/execution/session_state.rs +++ b/datafusion/core/src/execution/session_state.rs @@ -1740,8 +1740,8 @@ impl OptimizerConfig for SessionState { self.execution_props.query_execution_start_time } - fn alias_generator(&self) -> Arc { - self.execution_props.alias_generator.clone() + fn alias_generator(&self) -> &Arc { + &self.execution_props.alias_generator } fn options(&self) -> &ConfigOptions { diff --git a/datafusion/expr/src/utils.rs b/datafusion/expr/src/utils.rs index 9b64bca8121e..1d8eb9445eda 100644 --- a/datafusion/expr/src/utils.rs +++ b/datafusion/expr/src/utils.rs @@ -1305,7 +1305,7 @@ pub fn only_or_err(slice: &[T]) -> Result<&T> { } /// merge inputs schema into a single schema. -pub fn merge_schema(inputs: Vec<&LogicalPlan>) -> DFSchema { +pub fn merge_schema(inputs: &[&LogicalPlan]) -> DFSchema { if inputs.len() == 1 { inputs[0].schema().as_ref().clone() } else { diff --git a/datafusion/optimizer/src/analyzer/expand_wildcard_rule.rs b/datafusion/optimizer/src/analyzer/expand_wildcard_rule.rs index dd422f7aab95..b2de541a34c7 100644 --- a/datafusion/optimizer/src/analyzer/expand_wildcard_rule.rs +++ b/datafusion/optimizer/src/analyzer/expand_wildcard_rule.rs @@ -84,7 +84,7 @@ fn expand_exprlist(input: &LogicalPlan, expr: Vec) -> Result> { // If there is a REPLACE statement, replace that column with the given // replace expression. Column name remains the same. let replaced = if let Some(replace) = options.replace { - replace_columns(expanded, replace)? + replace_columns(expanded, &replace)? } else { expanded }; @@ -95,7 +95,7 @@ fn expand_exprlist(input: &LogicalPlan, expr: Vec) -> Result> { // If there is a REPLACE statement, replace that column with the given // replace expression. Column name remains the same. let replaced = if let Some(replace) = options.replace { - replace_columns(expanded, replace)? + replace_columns(expanded, &replace)? } else { expanded }; @@ -139,7 +139,7 @@ fn expand_exprlist(input: &LogicalPlan, expr: Vec) -> Result> { /// Multiple REPLACEs are also possible with comma separations. fn replace_columns( mut exprs: Vec, - replace: PlannedReplaceSelectItem, + replace: &PlannedReplaceSelectItem, ) -> Result> { for expr in exprs.iter_mut() { if let Expr::Column(Column { name, .. }) = expr { diff --git a/datafusion/optimizer/src/analyzer/function_rewrite.rs b/datafusion/optimizer/src/analyzer/function_rewrite.rs index 098c934bf7e1..94f5657b899b 100644 --- a/datafusion/optimizer/src/analyzer/function_rewrite.rs +++ b/datafusion/optimizer/src/analyzer/function_rewrite.rs @@ -48,7 +48,7 @@ impl ApplyFunctionRewrites { ) -> Result> { // get schema representing all available input fields. This is used for data type // resolution only, so order does not matter here - let mut schema = merge_schema(plan.inputs()); + let mut schema = merge_schema(&plan.inputs()); if let LogicalPlan::TableScan(ts) = &plan { let source_schema = DFSchema::try_from_qualified_schema( diff --git a/datafusion/optimizer/src/analyzer/subquery.rs b/datafusion/optimizer/src/analyzer/subquery.rs index 78be48a189f7..996dc238853e 100644 --- a/datafusion/optimizer/src/analyzer/subquery.rs +++ b/datafusion/optimizer/src/analyzer/subquery.rs @@ -16,7 +16,6 @@ // under the License. use std::ops::Deref; -use std::sync::Arc; use crate::analyzer::check_plan; use crate::utils::collect_subquery_cols; @@ -246,7 +245,7 @@ fn check_aggregation_in_scalar_subquery( if !agg.group_expr.is_empty() { let correlated_exprs = get_correlated_expressions(inner_plan)?; let inner_subquery_cols = - collect_subquery_cols(&correlated_exprs, Arc::clone(agg.input.schema()))?; + collect_subquery_cols(&correlated_exprs, agg.input.schema())?; let mut group_columns = agg .group_expr .iter() diff --git a/datafusion/optimizer/src/analyzer/type_coercion.rs b/datafusion/optimizer/src/analyzer/type_coercion.rs index 61ff4b4fd5a8..284650c3d64e 100644 --- a/datafusion/optimizer/src/analyzer/type_coercion.rs +++ b/datafusion/optimizer/src/analyzer/type_coercion.rs @@ -91,7 +91,7 @@ fn analyze_internal( ) -> Result> { // get schema representing all available input fields. This is used for data type // resolution only, so order does not matter here - let mut schema = merge_schema(plan.inputs()); + let mut schema = merge_schema(&plan.inputs()); if let LogicalPlan::TableScan(ts) = &plan { let source_schema = DFSchema::try_from_qualified_schema( @@ -544,12 +544,12 @@ fn coerce_scalar(target_type: &DataType, value: &ScalarValue) -> Result Result { - coerce_scalar(target_type, &value).or_else(|err| { + coerce_scalar(target_type, value).or_else(|err| { // If type coercion fails, check if the largest type in family works: if let Some(largest_type) = get_widest_type_in_family(target_type) { - coerce_scalar(largest_type, &value).map_or_else( + coerce_scalar(largest_type, value).map_or_else( |_| exec_err!("Cannot cast {value:?} to {target_type:?}"), |_| ScalarValue::try_from(target_type), ) @@ -578,11 +578,11 @@ fn coerce_frame_bound( ) -> Result { match bound { WindowFrameBound::Preceding(v) => { - coerce_scalar_range_aware(target_type, v).map(WindowFrameBound::Preceding) + coerce_scalar_range_aware(target_type, &v).map(WindowFrameBound::Preceding) } WindowFrameBound::CurrentRow => Ok(WindowFrameBound::CurrentRow), WindowFrameBound::Following(v) => { - coerce_scalar_range_aware(target_type, v).map(WindowFrameBound::Following) + coerce_scalar_range_aware(target_type, &v).map(WindowFrameBound::Following) } } } @@ -1459,26 +1459,26 @@ mod test { fn cast_helper( case: Case, - case_when_type: DataType, - then_else_type: DataType, + case_when_type: &DataType, + then_else_type: &DataType, schema: &DFSchemaRef, ) -> Case { let expr = case .expr - .map(|e| cast_if_not_same_type(e, &case_when_type, schema)); + .map(|e| cast_if_not_same_type(e, case_when_type, schema)); let when_then_expr = case .when_then_expr .into_iter() .map(|(when, then)| { ( - cast_if_not_same_type(when, &case_when_type, schema), - cast_if_not_same_type(then, &then_else_type, schema), + cast_if_not_same_type(when, case_when_type, schema), + cast_if_not_same_type(then, then_else_type, schema), ) }) .collect::>(); let else_expr = case .else_expr - .map(|e| cast_if_not_same_type(e, &then_else_type, schema)); + .map(|e| cast_if_not_same_type(e, then_else_type, schema)); Case { expr, @@ -1526,8 +1526,8 @@ mod test { let then_else_common_type = DataType::Utf8; let expected = cast_helper( case.clone(), - case_when_common_type, - then_else_common_type, + &case_when_common_type, + &then_else_common_type, &schema, ); let actual = coerce_case_expression(case, &schema)?; @@ -1546,8 +1546,8 @@ mod test { let then_else_common_type = DataType::Utf8; let expected = cast_helper( case.clone(), - case_when_common_type, - then_else_common_type, + &case_when_common_type, + &then_else_common_type, &schema, ); let actual = coerce_case_expression(case, &schema)?; diff --git a/datafusion/optimizer/src/common_subexpr_eliminate.rs b/datafusion/optimizer/src/common_subexpr_eliminate.rs index 22e9d220d324..583c6cf50de3 100644 --- a/datafusion/optimizer/src/common_subexpr_eliminate.rs +++ b/datafusion/optimizer/src/common_subexpr_eliminate.rs @@ -229,7 +229,7 @@ impl CommonSubexprEliminate { fn rewrite_exprs_list<'n>( &self, exprs_list: Vec>, - arrays_list: Vec>>, + arrays_list: &[Vec>], expr_stats: &ExprStats<'n>, common_exprs: &mut CommonExprs<'n>, alias_generator: &AliasGenerator, @@ -284,10 +284,10 @@ impl CommonSubexprEliminate { // Must clone as Identifiers use references to original expressions so we have // to keep the original expressions intact. exprs_list.clone(), - id_arrays_list, + &id_arrays_list, &expr_stats, &mut common_exprs, - &config.alias_generator(), + config.alias_generator().as_ref(), )?; assert!(!common_exprs.is_empty()); diff --git a/datafusion/optimizer/src/decorrelate.rs b/datafusion/optimizer/src/decorrelate.rs index 4d0770ccbbfb..7f918c03e3ac 100644 --- a/datafusion/optimizer/src/decorrelate.rs +++ b/datafusion/optimizer/src/decorrelate.rs @@ -148,7 +148,7 @@ impl TreeNodeRewriter for PullUpCorrelatedExpr { } fn f_up(&mut self, plan: LogicalPlan) -> Result> { - let subquery_schema = Arc::clone(plan.schema()); + let subquery_schema = plan.schema(); match &plan { LogicalPlan::Filter(plan_filter) => { let subquery_filter_exprs = split_conjunction(&plan_filter.predicate); @@ -231,7 +231,7 @@ impl TreeNodeRewriter for PullUpCorrelatedExpr { { proj_exprs_evaluation_result_on_empty_batch( &projection.expr, - Arc::clone(projection.input.schema()), + projection.input.schema(), expr_result_map, &mut expr_result_map_for_count_bug, )?; @@ -277,7 +277,7 @@ impl TreeNodeRewriter for PullUpCorrelatedExpr { { agg_exprs_evaluation_result_on_empty_batch( &aggregate.aggr_expr, - Arc::clone(aggregate.input.schema()), + aggregate.input.schema(), &mut expr_result_map_for_count_bug, )?; if !expr_result_map_for_count_bug.is_empty() { @@ -423,7 +423,7 @@ fn remove_duplicated_filter(filters: Vec, in_predicate: &Expr) -> Vec Result<()> { for e in agg_expr.iter() { @@ -446,7 +446,7 @@ fn agg_exprs_evaluation_result_on_empty_batch( let result_expr = result_expr.unalias(); let props = ExecutionProps::new(); - let info = SimplifyContext::new(&props).with_schema(Arc::clone(&schema)); + let info = SimplifyContext::new(&props).with_schema(Arc::clone(schema)); let simplifier = ExprSimplifier::new(info); let result_expr = simplifier.simplify(result_expr)?; if matches!(result_expr, Expr::Literal(ScalarValue::Int64(_))) { @@ -459,7 +459,7 @@ fn agg_exprs_evaluation_result_on_empty_batch( fn proj_exprs_evaluation_result_on_empty_batch( proj_expr: &[Expr], - schema: DFSchemaRef, + schema: &DFSchemaRef, input_expr_result_map_for_count_bug: &ExprResultMap, expr_result_map_for_count_bug: &mut ExprResultMap, ) -> Result<()> { @@ -483,7 +483,7 @@ fn proj_exprs_evaluation_result_on_empty_batch( if result_expr.ne(expr) { let props = ExecutionProps::new(); - let info = SimplifyContext::new(&props).with_schema(Arc::clone(&schema)); + let info = SimplifyContext::new(&props).with_schema(Arc::clone(schema)); let simplifier = ExprSimplifier::new(info); let result_expr = simplifier.simplify(result_expr)?; let expr_name = match expr { diff --git a/datafusion/optimizer/src/decorrelate_predicate_subquery.rs b/datafusion/optimizer/src/decorrelate_predicate_subquery.rs index f1cae1099a4d..9b1ada382b41 100644 --- a/datafusion/optimizer/src/decorrelate_predicate_subquery.rs +++ b/datafusion/optimizer/src/decorrelate_predicate_subquery.rs @@ -249,7 +249,7 @@ impl OptimizerRule for DecorrelatePredicateSubquery { fn build_join( query_info: &SubqueryInfo, left: &LogicalPlan, - alias: Arc, + alias: &Arc, ) -> Result> { let where_in_expr_opt = &query_info.where_in_expr; let in_predicate_opt = where_in_expr_opt diff --git a/datafusion/optimizer/src/eliminate_cross_join.rs b/datafusion/optimizer/src/eliminate_cross_join.rs index 20e6641e4d62..93df0dcfd500 100644 --- a/datafusion/optimizer/src/eliminate_cross_join.rs +++ b/datafusion/optimizer/src/eliminate_cross_join.rs @@ -378,7 +378,7 @@ fn extract_possible_join_keys(expr: &Expr, join_keys: &mut JoinKeySet) { extract_possible_join_keys(left, &mut left_join_keys); extract_possible_join_keys(right, &mut right_join_keys); - join_keys.insert_intersection(left_join_keys, right_join_keys) + join_keys.insert_intersection(&left_join_keys, &right_join_keys) } _ => (), }; diff --git a/datafusion/optimizer/src/join_key_set.rs b/datafusion/optimizer/src/join_key_set.rs index cd8ed382f069..c0eec78b183d 100644 --- a/datafusion/optimizer/src/join_key_set.rs +++ b/datafusion/optimizer/src/join_key_set.rs @@ -107,7 +107,7 @@ impl JoinKeySet { } /// Inserts any join keys that are common to both `s1` and `s2` into self - pub fn insert_intersection(&mut self, s1: JoinKeySet, s2: JoinKeySet) { + pub fn insert_intersection(&mut self, s1: &JoinKeySet, s2: &JoinKeySet) { // note can't use inner.intersection as we need to consider both (l, r) // and (r, l) in equality for (left, right) in s1.inner.iter() { @@ -234,7 +234,7 @@ mod test { let mut set = JoinKeySet::new(); // put something in there already set.insert(&col("x"), &col("y")); - set.insert_intersection(set1, set2); + set.insert_intersection(&set1, &set2); assert_contents( &set, diff --git a/datafusion/optimizer/src/optimize_projections/mod.rs b/datafusion/optimizer/src/optimize_projections/mod.rs index 35b0d07751ff..96772d3f2864 100644 --- a/datafusion/optimizer/src/optimize_projections/mod.rs +++ b/datafusion/optimizer/src/optimize_projections/mod.rs @@ -120,7 +120,7 @@ fn optimize_projections( match plan { LogicalPlan::Projection(proj) => { return merge_consecutive_projections(proj)?.transform_data(|proj| { - rewrite_projection_given_requirements(proj, config, indices) + rewrite_projection_given_requirements(proj, config, &indices) }) } LogicalPlan::Aggregate(aggregate) => { @@ -754,7 +754,7 @@ fn add_projection_on_top_if_helpful( fn rewrite_projection_given_requirements( proj: Projection, config: &dyn OptimizerConfig, - indices: RequiredIndicies, + indices: &RequiredIndicies, ) -> Result> { let Projection { expr, input, .. } = proj; diff --git a/datafusion/optimizer/src/optimizer.rs b/datafusion/optimizer/src/optimizer.rs index 93923a4e1e74..2fc560cceeb4 100644 --- a/datafusion/optimizer/src/optimizer.rs +++ b/datafusion/optimizer/src/optimizer.rs @@ -126,7 +126,7 @@ pub trait OptimizerConfig { fn query_execution_start_time(&self) -> DateTime; /// Return alias generator used to generate unique aliases for subqueries - fn alias_generator(&self) -> Arc; + fn alias_generator(&self) -> &Arc; fn options(&self) -> &ConfigOptions; @@ -204,8 +204,8 @@ impl OptimizerConfig for OptimizerContext { self.query_execution_start_time } - fn alias_generator(&self) -> Arc { - Arc::clone(&self.alias_generator) + fn alias_generator(&self) -> &Arc { + &self.alias_generator } fn options(&self) -> &ConfigOptions { diff --git a/datafusion/optimizer/src/scalar_subquery_to_join.rs b/datafusion/optimizer/src/scalar_subquery_to_join.rs index c79180b79256..e5c8757405cf 100644 --- a/datafusion/optimizer/src/scalar_subquery_to_join.rs +++ b/datafusion/optimizer/src/scalar_subquery_to_join.rs @@ -54,7 +54,7 @@ impl ScalarSubqueryToJoin { fn extract_subquery_exprs( &self, predicate: &Expr, - alias_gen: Arc, + alias_gen: &Arc, ) -> Result<(Vec<(Subquery, String)>, Expr)> { let mut extract = ExtractScalarSubQuery { sub_query_info: vec![], @@ -223,12 +223,12 @@ fn contains_scalar_subquery(expr: &Expr) -> bool { .expect("Inner is always Ok") } -struct ExtractScalarSubQuery { +struct ExtractScalarSubQuery<'a> { sub_query_info: Vec<(Subquery, String)>, - alias_gen: Arc, + alias_gen: &'a Arc, } -impl TreeNodeRewriter for ExtractScalarSubQuery { +impl TreeNodeRewriter for ExtractScalarSubQuery<'_> { type Node = Expr; fn f_down(&mut self, expr: Expr) -> Result> { diff --git a/datafusion/optimizer/src/simplify_expressions/expr_simplifier.rs b/datafusion/optimizer/src/simplify_expressions/expr_simplifier.rs index f299d4542c36..fc3921d29615 100644 --- a/datafusion/optimizer/src/simplify_expressions/expr_simplifier.rs +++ b/datafusion/optimizer/src/simplify_expressions/expr_simplifier.rs @@ -730,7 +730,7 @@ impl<'a, S: SimplifyInfo> TreeNodeRewriter for Simplifier<'a, S> { op: Eq, right, }) if is_bool_lit(&left) && info.is_boolean_type(&right)? => { - Transformed::yes(match as_bool_lit(*left)? { + Transformed::yes(match as_bool_lit(&left)? { Some(true) => *right, Some(false) => Expr::Not(right), None => lit_bool_null(), @@ -744,7 +744,7 @@ impl<'a, S: SimplifyInfo> TreeNodeRewriter for Simplifier<'a, S> { op: Eq, right, }) if is_bool_lit(&right) && info.is_boolean_type(&left)? => { - Transformed::yes(match as_bool_lit(*right)? { + Transformed::yes(match as_bool_lit(&right)? { Some(true) => *left, Some(false) => Expr::Not(left), None => lit_bool_null(), @@ -761,7 +761,7 @@ impl<'a, S: SimplifyInfo> TreeNodeRewriter for Simplifier<'a, S> { op: NotEq, right, }) if is_bool_lit(&left) && info.is_boolean_type(&right)? => { - Transformed::yes(match as_bool_lit(*left)? { + Transformed::yes(match as_bool_lit(&left)? { Some(true) => Expr::Not(right), Some(false) => *right, None => lit_bool_null(), @@ -775,7 +775,7 @@ impl<'a, S: SimplifyInfo> TreeNodeRewriter for Simplifier<'a, S> { op: NotEq, right, }) if is_bool_lit(&right) && info.is_boolean_type(&left)? => { - Transformed::yes(match as_bool_lit(*right)? { + Transformed::yes(match as_bool_lit(&right)? { Some(true) => Expr::Not(left), Some(false) => *left, None => lit_bool_null(), @@ -1569,7 +1569,7 @@ impl<'a, S: SimplifyInfo> TreeNodeRewriter for Simplifier<'a, S> { { match (*left, *right) { (Expr::InList(l1), Expr::InList(l2)) => { - return inlist_intersection(l1, l2, false).map(Transformed::yes); + return inlist_intersection(l1, &l2, false).map(Transformed::yes); } // Matched previously once _ => unreachable!(), @@ -1609,7 +1609,7 @@ impl<'a, S: SimplifyInfo> TreeNodeRewriter for Simplifier<'a, S> { { match (*left, *right) { (Expr::InList(l1), Expr::InList(l2)) => { - return inlist_except(l1, l2).map(Transformed::yes); + return inlist_except(l1, &l2).map(Transformed::yes); } // Matched previously once _ => unreachable!(), @@ -1629,7 +1629,7 @@ impl<'a, S: SimplifyInfo> TreeNodeRewriter for Simplifier<'a, S> { { match (*left, *right) { (Expr::InList(l1), Expr::InList(l2)) => { - return inlist_except(l2, l1).map(Transformed::yes); + return inlist_except(l2, &l1).map(Transformed::yes); } // Matched previously once _ => unreachable!(), @@ -1649,7 +1649,7 @@ impl<'a, S: SimplifyInfo> TreeNodeRewriter for Simplifier<'a, S> { { match (*left, *right) { (Expr::InList(l1), Expr::InList(l2)) => { - return inlist_intersection(l1, l2, true).map(Transformed::yes); + return inlist_intersection(l1, &l2, true).map(Transformed::yes); } // Matched previously once _ => unreachable!(), @@ -1759,7 +1759,7 @@ fn inlist_union(mut l1: InList, l2: InList, negated: bool) -> Result { /// Return the intersection of two inlist expressions /// maintaining the order of the elements in the two lists -fn inlist_intersection(mut l1: InList, l2: InList, negated: bool) -> Result { +fn inlist_intersection(mut l1: InList, l2: &InList, negated: bool) -> Result { let l2_items = l2.list.iter().collect::>(); // remove all items from l1 that are not in l2 @@ -1775,7 +1775,7 @@ fn inlist_intersection(mut l1: InList, l2: InList, negated: bool) -> Result Result { +fn inlist_except(mut l1: InList, l2: &InList) -> Result { let l2_items = l2.list.iter().collect::>(); // keep only items from l1 that are not in l2 diff --git a/datafusion/optimizer/src/simplify_expressions/simplify_exprs.rs b/datafusion/optimizer/src/simplify_expressions/simplify_exprs.rs index 1c39313ea132..877989c02d74 100644 --- a/datafusion/optimizer/src/simplify_expressions/simplify_exprs.rs +++ b/datafusion/optimizer/src/simplify_expressions/simplify_exprs.rs @@ -80,7 +80,7 @@ impl SimplifyExpressions { execution_props: &ExecutionProps, ) -> Result> { let schema = if !plan.inputs().is_empty() { - DFSchemaRef::new(merge_schema(plan.inputs())) + DFSchemaRef::new(merge_schema(&plan.inputs())) } else if let LogicalPlan::TableScan(scan) = &plan { // When predicates are pushed into a table scan, there is no input // schema to resolve predicates against, so it must be handled specially diff --git a/datafusion/optimizer/src/simplify_expressions/utils.rs b/datafusion/optimizer/src/simplify_expressions/utils.rs index 5da727cb5990..38bfc1a93403 100644 --- a/datafusion/optimizer/src/simplify_expressions/utils.rs +++ b/datafusion/optimizer/src/simplify_expressions/utils.rs @@ -221,9 +221,9 @@ pub fn is_negative_of(not_expr: &Expr, expr: &Expr) -> bool { /// returns the contained boolean value in `expr` as /// `Expr::Literal(ScalarValue::Boolean(v))`. -pub fn as_bool_lit(expr: Expr) -> Result> { +pub fn as_bool_lit(expr: &Expr) -> Result> { match expr { - Expr::Literal(ScalarValue::Boolean(v)) => Ok(v), + Expr::Literal(ScalarValue::Boolean(v)) => Ok(*v), _ => internal_err!("Expected boolean literal, got {expr:?}"), } } diff --git a/datafusion/optimizer/src/unwrap_cast_in_comparison.rs b/datafusion/optimizer/src/unwrap_cast_in_comparison.rs index b17d69437cbe..6043f0d7c8b5 100644 --- a/datafusion/optimizer/src/unwrap_cast_in_comparison.rs +++ b/datafusion/optimizer/src/unwrap_cast_in_comparison.rs @@ -99,7 +99,7 @@ impl OptimizerRule for UnwrapCastInComparison { plan: LogicalPlan, _config: &dyn OptimizerConfig, ) -> Result> { - let mut schema = merge_schema(plan.inputs()); + let mut schema = merge_schema(&plan.inputs()); if let LogicalPlan::TableScan(ts) = &plan { let source_schema = DFSchema::try_from_qualified_schema( @@ -414,32 +414,32 @@ fn try_cast_numeric_literal( DataType::UInt64 => ScalarValue::UInt64(Some(value as u64)), DataType::Timestamp(TimeUnit::Second, tz) => { let value = cast_between_timestamp( - lit_data_type, - DataType::Timestamp(TimeUnit::Second, tz.clone()), + &lit_data_type, + &DataType::Timestamp(TimeUnit::Second, tz.clone()), value, ); ScalarValue::TimestampSecond(value, tz.clone()) } DataType::Timestamp(TimeUnit::Millisecond, tz) => { let value = cast_between_timestamp( - lit_data_type, - DataType::Timestamp(TimeUnit::Millisecond, tz.clone()), + &lit_data_type, + &DataType::Timestamp(TimeUnit::Millisecond, tz.clone()), value, ); ScalarValue::TimestampMillisecond(value, tz.clone()) } DataType::Timestamp(TimeUnit::Microsecond, tz) => { let value = cast_between_timestamp( - lit_data_type, - DataType::Timestamp(TimeUnit::Microsecond, tz.clone()), + &lit_data_type, + &DataType::Timestamp(TimeUnit::Microsecond, tz.clone()), value, ); ScalarValue::TimestampMicrosecond(value, tz.clone()) } DataType::Timestamp(TimeUnit::Nanosecond, tz) => { let value = cast_between_timestamp( - lit_data_type, - DataType::Timestamp(TimeUnit::Nanosecond, tz.clone()), + &lit_data_type, + &DataType::Timestamp(TimeUnit::Nanosecond, tz.clone()), value, ); ScalarValue::TimestampNanosecond(value, tz.clone()) @@ -505,7 +505,7 @@ fn try_cast_dictionary( } /// Cast a timestamp value from one unit to another -fn cast_between_timestamp(from: DataType, to: DataType, value: i128) -> Option { +fn cast_between_timestamp(from: &DataType, to: &DataType, value: i128) -> Option { let value = value as i64; let from_scale = match from { DataType::Timestamp(TimeUnit::Second, _) => 1, diff --git a/datafusion/optimizer/src/utils.rs b/datafusion/optimizer/src/utils.rs index 45cef55bf272..9e602ad49715 100644 --- a/datafusion/optimizer/src/utils.rs +++ b/datafusion/optimizer/src/utils.rs @@ -21,7 +21,7 @@ use std::collections::{BTreeSet, HashMap, HashSet}; use crate::{OptimizerConfig, OptimizerRule}; -use datafusion_common::{Column, DFSchema, DFSchemaRef, Result}; +use datafusion_common::{Column, DFSchema, Result}; use datafusion_expr::expr_rewriter::replace_col; use datafusion_expr::utils as expr_utils; use datafusion_expr::{logical_plan::LogicalPlan, Expr, Operator}; @@ -83,7 +83,7 @@ pub(crate) fn has_all_column_refs(expr: &Expr, schema_cols: &HashSet) -> pub(crate) fn collect_subquery_cols( exprs: &[Expr], - subquery_schema: DFSchemaRef, + subquery_schema: &DFSchema, ) -> Result> { exprs.iter().try_fold(BTreeSet::new(), |mut cols, expr| { let mut using_cols: Vec = vec![]; @@ -296,5 +296,5 @@ pub fn only_or_err(slice: &[T]) -> Result<&T> { note = "use `datafusion_expr::utils::merge_schema` instead" )] pub fn merge_schema(inputs: Vec<&LogicalPlan>) -> DFSchema { - expr_utils::merge_schema(inputs) + expr_utils::merge_schema(&inputs) } From 4838cfbf453f3c21d9c5a84f9577329dd78aa763 Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Mon, 2 Sep 2024 11:17:44 +0200 Subject: [PATCH 068/154] Remove deprecated LogicalPlan::with_new_inputs function (#12285) Deprecated since 35.0.0. --- datafusion/expr/src/logical_plan/plan.rs | 6 ------ 1 file changed, 6 deletions(-) diff --git a/datafusion/expr/src/logical_plan/plan.rs b/datafusion/expr/src/logical_plan/plan.rs index 5bd6ab10331a..19be9143fd23 100644 --- a/datafusion/expr/src/logical_plan/plan.rs +++ b/datafusion/expr/src/logical_plan/plan.rs @@ -602,12 +602,6 @@ impl LogicalPlan { } } - /// Returns a copy of this `LogicalPlan` with the new inputs - #[deprecated(since = "35.0.0", note = "please use `with_new_exprs` instead")] - pub fn with_new_inputs(&self, inputs: &[LogicalPlan]) -> Result { - self.with_new_exprs(self.expressions(), inputs.to_vec()) - } - /// Recomputes schema and type information for this LogicalPlan if needed. /// /// Some `LogicalPlan`s may need to recompute their schema if the number or From b583591f9067cfa3a03962383b34196fbd5acc04 Mon Sep 17 00:00:00 2001 From: Sergei Grebnov Date: Mon, 2 Sep 2024 15:05:29 +0300 Subject: [PATCH 069/154] Support alternate format for Date32 unparsing (TEXT/SQLite) (#12282) --- datafusion/sql/src/unparser/dialect.rs | 24 ++++++++++++++++++++ datafusion/sql/src/unparser/expr.rs | 31 ++++++++++++++++++++++++-- 2 files changed, 53 insertions(+), 2 deletions(-) diff --git a/datafusion/sql/src/unparser/dialect.rs b/datafusion/sql/src/unparser/dialect.rs index 74f154d7870f..706b7633cb90 100644 --- a/datafusion/sql/src/unparser/dialect.rs +++ b/datafusion/sql/src/unparser/dialect.rs @@ -96,6 +96,12 @@ pub trait Dialect: Send + Sync { ast::DataType::Timestamp(None, tz_info) } + + /// The SQL type to use for Arrow Date32 unparsing + /// Most dialects use Date, but some, like SQLite require TEXT + fn date32_cast_dtype(&self) -> sqlparser::ast::DataType { + sqlparser::ast::DataType::Date + } } /// `IntervalStyle` to use for unparsing @@ -206,6 +212,10 @@ impl Dialect for SqliteDialect { fn identifier_quote_style(&self, _: &str) -> Option { Some('`') } + + fn date32_cast_dtype(&self) -> sqlparser::ast::DataType { + sqlparser::ast::DataType::Text + } } pub struct CustomDialect { @@ -220,6 +230,7 @@ pub struct CustomDialect { int64_cast_dtype: ast::DataType, timestamp_cast_dtype: ast::DataType, timestamp_tz_cast_dtype: ast::DataType, + date32_cast_dtype: sqlparser::ast::DataType, } impl Default for CustomDialect { @@ -239,6 +250,7 @@ impl Default for CustomDialect { None, TimezoneInfo::WithTimeZone, ), + date32_cast_dtype: sqlparser::ast::DataType::Date, } } } @@ -302,6 +314,10 @@ impl Dialect for CustomDialect { self.timestamp_cast_dtype.clone() } } + + fn date32_cast_dtype(&self) -> sqlparser::ast::DataType { + self.date32_cast_dtype.clone() + } } /// `CustomDialectBuilder` to build `CustomDialect` using builder pattern @@ -330,6 +346,7 @@ pub struct CustomDialectBuilder { int64_cast_dtype: ast::DataType, timestamp_cast_dtype: ast::DataType, timestamp_tz_cast_dtype: ast::DataType, + date32_cast_dtype: ast::DataType, } impl Default for CustomDialectBuilder { @@ -355,6 +372,7 @@ impl CustomDialectBuilder { None, TimezoneInfo::WithTimeZone, ), + date32_cast_dtype: sqlparser::ast::DataType::Date, } } @@ -371,6 +389,7 @@ impl CustomDialectBuilder { int64_cast_dtype: self.int64_cast_dtype, timestamp_cast_dtype: self.timestamp_cast_dtype, timestamp_tz_cast_dtype: self.timestamp_tz_cast_dtype, + date32_cast_dtype: self.date32_cast_dtype, } } @@ -453,4 +472,9 @@ impl CustomDialectBuilder { self.timestamp_tz_cast_dtype = timestamp_tz_cast_dtype; self } + + pub fn with_date32_cast_dtype(mut self, date32_cast_dtype: ast::DataType) -> Self { + self.date32_cast_dtype = date32_cast_dtype; + self + } } diff --git a/datafusion/sql/src/unparser/expr.rs b/datafusion/sql/src/unparser/expr.rs index 549635a31aef..8ff4773e8e94 100644 --- a/datafusion/sql/src/unparser/expr.rs +++ b/datafusion/sql/src/unparser/expr.rs @@ -1386,7 +1386,7 @@ impl Unparser<'_> { DataType::Timestamp(time_unit, tz) => { Ok(self.dialect.timestamp_cast_dtype(time_unit, tz)) } - DataType::Date32 => Ok(ast::DataType::Date), + DataType::Date32 => Ok(self.dialect.date32_cast_dtype()), DataType::Date64 => Ok(self.ast_type_for_date64_in_cast()), DataType::Time32(_) => { not_impl_err!("Unsupported DataType: conversion: {data_type:?}") @@ -2242,7 +2242,7 @@ mod tests { } #[test] - fn custom_dialect_with_teimstamp_cast_dtype() -> Result<()> { + fn custom_dialect_with_timestamp_cast_dtype() -> Result<()> { let default_dialect = CustomDialectBuilder::new().build(); let mysql_dialect = CustomDialectBuilder::new() .with_timestamp_cast_dtype( @@ -2279,4 +2279,31 @@ mod tests { } Ok(()) } + + #[test] + fn custom_dialect_date32_ast_dtype() -> Result<()> { + let default_dialect = CustomDialectBuilder::default().build(); + let sqlite_custom_dialect = CustomDialectBuilder::new() + .with_date32_cast_dtype(ast::DataType::Text) + .build(); + + for (dialect, data_type, identifier) in [ + (&default_dialect, DataType::Date32, "DATE"), + (&sqlite_custom_dialect, DataType::Date32, "TEXT"), + ] { + let unparser = Unparser::new(dialect); + + let expr = Expr::Cast(Cast { + expr: Box::new(col("a")), + data_type, + }); + let ast = unparser.expr_to_sql(&expr)?; + + let actual = format!("{}", ast); + let expected = format!(r#"CAST(a AS {identifier})"#); + + assert_eq!(actual, expected); + } + Ok(()) + } } From 93fb7157b454f00319d45ce01e1ba2bdebb52b48 Mon Sep 17 00:00:00 2001 From: June <61218022+itsjunetime@users.noreply.github.com> Date: Mon, 2 Sep 2024 06:10:17 -0600 Subject: [PATCH 070/154] Extract drive-by fixes from PR 12135 for easier reviewing (#12240) * Extract drive-by fixes from PR 12135 for easier reviewing * Add a few more cfgs to silence warnings with different feature sets * fmt --- datafusion/common/src/hash_utils.rs | 2 + .../core/src/datasource/listing/helpers.rs | 16 ++-- .../core/src/datasource/listing/table.rs | 7 +- .../datasource/physical_plan/parquet/mod.rs | 13 +-- .../physical_plan/parquet/row_filter.rs | 95 +++++++------------ datafusion/core/src/datasource/statistics.rs | 26 +++-- .../src/execution/session_state_defaults.rs | 3 + .../core/src/physical_optimizer/pruning.rs | 4 + .../physical-expr/src/expressions/binary.rs | 20 ++-- .../physical-plan/src/execution_plan.rs | 2 +- 10 files changed, 93 insertions(+), 95 deletions(-) diff --git a/datafusion/common/src/hash_utils.rs b/datafusion/common/src/hash_utils.rs index f3d2a0a4f9ab..72cfeafd0bfe 100644 --- a/datafusion/common/src/hash_utils.rs +++ b/datafusion/common/src/hash_utils.rs @@ -245,6 +245,8 @@ fn hash_struct_array( Ok(()) } +// only adding this `cfg` b/c this function is only used with this `cfg` +#[cfg(not(feature = "force_hash_collisions"))] fn hash_map_array( array: &MapArray, random_state: &RandomState, diff --git a/datafusion/core/src/datasource/listing/helpers.rs b/datafusion/core/src/datasource/listing/helpers.rs index dbeaf5dfcc36..33a16237e162 100644 --- a/datafusion/core/src/datasource/listing/helpers.rs +++ b/datafusion/core/src/datasource/listing/helpers.rs @@ -51,12 +51,12 @@ use object_store::{ObjectMeta, ObjectStore}; /// - the table provider can filter the table partition values with this expression /// - the expression can be marked as `TableProviderFilterPushDown::Exact` once this filtering /// was performed -pub fn expr_applicable_for_cols(col_names: &[String], expr: &Expr) -> bool { +pub fn expr_applicable_for_cols(col_names: &[&str], expr: &Expr) -> bool { let mut is_applicable = true; expr.apply(|expr| { match expr { Expr::Column(Column { ref name, .. }) => { - is_applicable &= col_names.contains(name); + is_applicable &= col_names.contains(&name.as_str()); if is_applicable { Ok(TreeNodeRecursion::Jump) } else { @@ -745,27 +745,27 @@ mod tests { #[test] fn test_expr_applicable_for_cols() { assert!(expr_applicable_for_cols( - &[String::from("c1")], + &["c1"], &Expr::eq(col("c1"), lit("value")) )); assert!(!expr_applicable_for_cols( - &[String::from("c1")], + &["c1"], &Expr::eq(col("c2"), lit("value")) )); assert!(!expr_applicable_for_cols( - &[String::from("c1")], + &["c1"], &Expr::eq(col("c1"), col("c2")) )); assert!(expr_applicable_for_cols( - &[String::from("c1"), String::from("c2")], + &["c1", "c2"], &Expr::eq(col("c1"), col("c2")) )); assert!(expr_applicable_for_cols( - &[String::from("c1"), String::from("c2")], + &["c1", "c2"], &(Expr::eq(col("c1"), col("c2").alias("c2_alias"))).not() )); assert!(expr_applicable_for_cols( - &[String::from("c1"), String::from("c2")], + &["c1", "c2"], &(case(col("c1")) .when(lit("v1"), lit(true)) .otherwise(lit(false)) diff --git a/datafusion/core/src/datasource/listing/table.rs b/datafusion/core/src/datasource/listing/table.rs index 35286612a887..9246226d431e 100644 --- a/datafusion/core/src/datasource/listing/table.rs +++ b/datafusion/core/src/datasource/listing/table.rs @@ -826,7 +826,7 @@ impl TableProvider for ListingTable { &self, filters: &[&Expr], ) -> Result> { - let support: Vec<_> = filters + Ok(filters .iter() .map(|filter| { if expr_applicable_for_cols( @@ -834,7 +834,7 @@ impl TableProvider for ListingTable { .options .table_partition_cols .iter() - .map(|x| x.0.clone()) + .map(|x| x.0.as_str()) .collect::>(), filter, ) { @@ -846,8 +846,7 @@ impl TableProvider for ListingTable { TableProviderFilterPushDown::Inexact } }) - .collect(); - Ok(support) + .collect()) } fn get_table_definition(&self) -> Option<&str> { diff --git a/datafusion/core/src/datasource/physical_plan/parquet/mod.rs b/datafusion/core/src/datasource/physical_plan/parquet/mod.rs index 85d6f8db2373..b2f86db742f2 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet/mod.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet/mod.rs @@ -685,10 +685,12 @@ impl ExecutionPlan for ParquetExec { partition_index: usize, ctx: Arc, ) -> Result { - let projection = match self.base_config.file_column_projection_indices() { - Some(proj) => proj, - None => (0..self.base_config.file_schema.fields().len()).collect(), - }; + let projection = self + .base_config + .file_column_projection_indices() + .unwrap_or_else(|| { + (0..self.base_config.file_schema.fields().len()).collect() + }); let parquet_file_reader_factory = self .parquet_file_reader_factory @@ -698,8 +700,7 @@ impl ExecutionPlan for ParquetExec { ctx.runtime_env() .object_store(&self.base_config.object_store_url) .map(|store| { - Arc::new(DefaultParquetFileReaderFactory::new(store)) - as Arc + Arc::new(DefaultParquetFileReaderFactory::new(store)) as _ }) })?; diff --git a/datafusion/core/src/datasource/physical_plan/parquet/row_filter.rs b/datafusion/core/src/datasource/physical_plan/parquet/row_filter.rs index 23fdadc2cdee..59d23fd68c31 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet/row_filter.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet/row_filter.rs @@ -59,6 +59,7 @@ //! the unsorted predicates. Within each partition, predicates are //! still be sorted by size. +use std::cmp::Ordering; use std::collections::BTreeSet; use std::sync::Arc; @@ -129,7 +130,7 @@ impl DatafusionArrowPredicate { // on the order they appear in the file let projection = match candidate.projection.len() { 0 | 1 => vec![], - _ => remap_projection(&candidate.projection), + 2.. => remap_projection(&candidate.projection), }; Ok(Self { @@ -151,32 +152,31 @@ impl ArrowPredicate for DatafusionArrowPredicate { &self.projection_mask } - fn evaluate(&mut self, batch: RecordBatch) -> ArrowResult { - let batch = match self.projection.is_empty() { - true => batch, - false => batch.project(&self.projection)?, + fn evaluate(&mut self, mut batch: RecordBatch) -> ArrowResult { + if !self.projection.is_empty() { + batch = batch.project(&self.projection)?; }; let batch = self.schema_mapping.map_partial_batch(batch)?; // scoped timer updates on drop let mut timer = self.time.timer(); - match self - .physical_expr + + self.physical_expr .evaluate(&batch) .and_then(|v| v.into_array(batch.num_rows())) - { - Ok(array) => { + .and_then(|array| { let bool_arr = as_boolean_array(&array)?.clone(); let num_filtered = bool_arr.len() - bool_arr.true_count(); self.rows_filtered.add(num_filtered); timer.stop(); Ok(bool_arr) - } - Err(e) => Err(ArrowError::ComputeError(format!( - "Error evaluating filter predicate: {e:?}" - ))), - } + }) + .map_err(|e| { + ArrowError::ComputeError(format!( + "Error evaluating filter predicate: {e:?}" + )) + }) } } @@ -453,62 +453,33 @@ pub fn build_row_filter( // no candidates if candidates.is_empty() { - Ok(None) - } else if reorder_predicates { - // attempt to reorder the predicates by size and whether they are sorted - candidates.sort_by_key(|c| c.required_bytes); - - let (indexed_candidates, other_candidates): (Vec<_>, Vec<_>) = - candidates.into_iter().partition(|c| c.can_use_index); - - let mut filters: Vec> = vec![]; - - for candidate in indexed_candidates { - let filter = DatafusionArrowPredicate::try_new( - candidate, - file_schema, - metadata, - rows_filtered.clone(), - time.clone(), - Arc::clone(&schema_mapping), - )?; - - filters.push(Box::new(filter)); - } - - for candidate in other_candidates { - let filter = DatafusionArrowPredicate::try_new( - candidate, - file_schema, - metadata, - rows_filtered.clone(), - time.clone(), - Arc::clone(&schema_mapping), - )?; + return Ok(None); + } - filters.push(Box::new(filter)); - } + if reorder_predicates { + candidates.sort_unstable_by(|c1, c2| { + match c1.can_use_index.cmp(&c2.can_use_index) { + Ordering::Equal => c1.required_bytes.cmp(&c2.required_bytes), + ord => ord, + } + }); + } - Ok(Some(RowFilter::new(filters))) - } else { - // otherwise evaluate the predicates in the order the appeared in the - // original expressions - let mut filters: Vec> = vec![]; - for candidate in candidates { - let filter = DatafusionArrowPredicate::try_new( + candidates + .into_iter() + .map(|candidate| { + DatafusionArrowPredicate::try_new( candidate, file_schema, metadata, rows_filtered.clone(), time.clone(), Arc::clone(&schema_mapping), - )?; - - filters.push(Box::new(filter)); - } - - Ok(Some(RowFilter::new(filters))) - } + ) + .map(|pred| Box::new(pred) as _) + }) + .collect::, _>>() + .map(|filters| Some(RowFilter::new(filters))) } #[cfg(test)] diff --git a/datafusion/core/src/datasource/statistics.rs b/datafusion/core/src/datasource/statistics.rs index 6f89657defd3..201bbfd5c007 100644 --- a/datafusion/core/src/datasource/statistics.rs +++ b/datafusion/core/src/datasource/statistics.rs @@ -18,16 +18,21 @@ use std::mem; use std::sync::Arc; -use arrow_schema::DataType; use futures::{Stream, StreamExt}; use datafusion_common::stats::Precision; use datafusion_common::ScalarValue; -use crate::arrow::datatypes::{Schema, SchemaRef}; +use crate::arrow::datatypes::SchemaRef; use crate::error::Result; -use crate::functions_aggregate::min_max::{MaxAccumulator, MinAccumulator}; -use crate::physical_plan::{Accumulator, ColumnStatistics, Statistics}; +use crate::physical_plan::{ColumnStatistics, Statistics}; + +#[cfg(feature = "parquet")] +use crate::{ + arrow::datatypes::Schema, + functions_aggregate::min_max::{MaxAccumulator, MinAccumulator}, + physical_plan::Accumulator, +}; use super::listing::PartitionedFile; @@ -144,6 +149,8 @@ pub async fn get_statistics_with_limit( Ok((result_files, statistics)) } +// only adding this cfg b/c this is the only feature it's used with currently +#[cfg(feature = "parquet")] pub(crate) fn create_max_min_accs( schema: &Schema, ) -> (Vec>, Vec>) { @@ -175,6 +182,8 @@ fn add_row_stats( } } +// only adding this cfg b/c this is the only feature it's used with currently +#[cfg(feature = "parquet")] pub(crate) fn get_col_stats( schema: &Schema, null_counts: Vec>, @@ -205,8 +214,13 @@ pub(crate) fn get_col_stats( // (aka non Dictionary) output. We need to adjust the output data type to reflect this. // The reason min/max aggregate produces unpacked output because there is only one // min/max value per group; there is no needs to keep them Dictionary encode -fn min_max_aggregate_data_type(input_type: &DataType) -> &DataType { - if let DataType::Dictionary(_, value_type) = input_type { +// +// only adding this cfg b/c this is the only feature it's used with currently +#[cfg(feature = "parquet")] +fn min_max_aggregate_data_type( + input_type: &arrow_schema::DataType, +) -> &arrow_schema::DataType { + if let arrow_schema::DataType::Dictionary(_, value_type) = input_type { value_type.as_ref() } else { input_type diff --git a/datafusion/core/src/execution/session_state_defaults.rs b/datafusion/core/src/execution/session_state_defaults.rs index bc7e194caeae..b5370efa0a97 100644 --- a/datafusion/core/src/execution/session_state_defaults.rs +++ b/datafusion/core/src/execution/session_state_defaults.rs @@ -100,7 +100,9 @@ impl SessionStateDefaults { /// returns the list of default [`ScalarUDF']'s pub fn default_scalar_functions() -> Vec> { + #[cfg_attr(not(feature = "nested_expressions"), allow(unused_mut))] let mut functions: Vec> = functions::all_default_functions(); + #[cfg(feature = "nested_expressions")] functions.append(&mut functions_nested::all_default_nested_functions()); @@ -144,6 +146,7 @@ impl SessionStateDefaults { } /// registers all the builtin array functions + #[cfg_attr(not(feature = "nested_expressions"), allow(unused_variables))] pub fn register_array_functions(state: &mut SessionState) { // register crate of array expressions (if enabled) #[cfg(feature = "nested_expressions")] diff --git a/datafusion/core/src/physical_optimizer/pruning.rs b/datafusion/core/src/physical_optimizer/pruning.rs index a16abc607ee6..9bc2bb1d1db9 100644 --- a/datafusion/core/src/physical_optimizer/pruning.rs +++ b/datafusion/core/src/physical_optimizer/pruning.rs @@ -615,6 +615,8 @@ impl PruningPredicate { is_always_true(&self.predicate_expr) && self.literal_guarantees.is_empty() } + // this is only used by `parquet` feature right now + #[allow(dead_code)] pub(crate) fn required_columns(&self) -> &RequiredColumns { &self.required_columns } @@ -746,6 +748,8 @@ impl RequiredColumns { /// * `a > 5 OR a < 10` returns `Some(a)` /// * `a > 5 OR b < 10` returns `None` /// * `true` returns None + #[allow(dead_code)] + // this fn is only used by `parquet` feature right now, thus the `allow(dead_code)` pub(crate) fn single_column(&self) -> Option<&phys_expr::Column> { if self.columns.windows(2).all(|w| { // check if all columns are the same (ignoring statistics and field) diff --git a/datafusion/physical-expr/src/expressions/binary.rs b/datafusion/physical-expr/src/expressions/binary.rs index 2680a7930ff1..08c133d7193a 100644 --- a/datafusion/physical-expr/src/expressions/binary.rs +++ b/datafusion/physical-expr/src/expressions/binary.rs @@ -33,6 +33,7 @@ use arrow::compute::kernels::comparison::{ use arrow::compute::kernels::concat_elements::concat_elements_utf8; use arrow::compute::{cast, ilike, like, nilike, nlike}; use arrow::datatypes::*; +use arrow_schema::ArrowError; use datafusion_common::cast::as_boolean_array; use datafusion_common::{internal_err, Result, ScalarValue}; use datafusion_expr::interval_arithmetic::{apply_operator, Interval}; @@ -133,12 +134,15 @@ impl std::fmt::Display for BinaryExpr { } /// Invoke a boolean kernel on a pair of arrays -macro_rules! boolean_op { - ($LEFT:expr, $RIGHT:expr, $OP:ident) => {{ - let ll = as_boolean_array($LEFT).expect("boolean_op failed to downcast array"); - let rr = as_boolean_array($RIGHT).expect("boolean_op failed to downcast array"); - Ok(Arc::new($OP(&ll, &rr)?)) - }}; +#[inline] +fn boolean_op( + left: &dyn Array, + right: &dyn Array, + op: impl FnOnce(&BooleanArray, &BooleanArray) -> Result, +) -> Result, ArrowError> { + let ll = as_boolean_array(left).expect("boolean_op failed to downcast left array"); + let rr = as_boolean_array(right).expect("boolean_op failed to downcast right array"); + op(ll, rr).map(|t| Arc::new(t) as _) } macro_rules! binary_string_array_flag_op { @@ -596,7 +600,7 @@ impl BinaryExpr { | NotLikeMatch | NotILikeMatch => unreachable!(), And => { if left_data_type == &DataType::Boolean { - boolean_op!(&left, &right, and_kleene) + Ok(boolean_op(&left, &right, and_kleene)?) } else { internal_err!( "Cannot evaluate binary expression {:?} with types {:?} and {:?}", @@ -608,7 +612,7 @@ impl BinaryExpr { } Or => { if left_data_type == &DataType::Boolean { - boolean_op!(&left, &right, or_kleene) + Ok(boolean_op(&left, &right, or_kleene)?) } else { internal_err!( "Cannot evaluate binary expression {:?} with types {:?} and {:?}", diff --git a/datafusion/physical-plan/src/execution_plan.rs b/datafusion/physical-plan/src/execution_plan.rs index 53ae59f7072f..f584542fafcf 100644 --- a/datafusion/physical-plan/src/execution_plan.rs +++ b/datafusion/physical-plan/src/execution_plan.rs @@ -718,7 +718,7 @@ pub fn execute_stream( match plan.output_partitioning().partition_count() { 0 => Ok(Box::pin(EmptyRecordBatchStream::new(plan.schema()))), 1 => plan.execute(0, context), - _ => { + 2.. => { // merge into a single partition let plan = CoalescePartitionsExec::new(Arc::clone(&plan)); // CoalescePartitionsExec must produce a single partition From 244d8a1bc2213163d110b40cb959649511086df6 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=BC=A0=E6=9E=97=E4=BC=9F?= Date: Mon, 2 Sep 2024 20:48:49 +0800 Subject: [PATCH 071/154] Move `CombinePartialFinalAggregate` rule into physical-optimizer crate (#12167) --- .../combine_partial_final_agg.rs | 435 ------------------ datafusion/core/src/physical_optimizer/mod.rs | 1 - .../combine_partial_final_agg.rs | 290 ++++++++++++ .../core/tests/physical_optimizer/mod.rs | 1 + .../src/combine_partial_final_agg.rs | 164 +++++++ datafusion/physical-optimizer/src/lib.rs | 1 + 6 files changed, 456 insertions(+), 436 deletions(-) delete mode 100644 datafusion/core/src/physical_optimizer/combine_partial_final_agg.rs create mode 100644 datafusion/core/tests/physical_optimizer/combine_partial_final_agg.rs create mode 100644 datafusion/physical-optimizer/src/combine_partial_final_agg.rs diff --git a/datafusion/core/src/physical_optimizer/combine_partial_final_agg.rs b/datafusion/core/src/physical_optimizer/combine_partial_final_agg.rs deleted file mode 100644 index 1a12fc7de888..000000000000 --- a/datafusion/core/src/physical_optimizer/combine_partial_final_agg.rs +++ /dev/null @@ -1,435 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -//! CombinePartialFinalAggregate optimizer rule checks the adjacent Partial and Final AggregateExecs -//! and try to combine them if necessary - -use std::sync::Arc; - -use crate::error::Result; -use crate::physical_plan::aggregates::{AggregateExec, AggregateMode, PhysicalGroupBy}; -use crate::physical_plan::ExecutionPlan; - -use datafusion_common::config::ConfigOptions; -use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; -use datafusion_physical_expr::aggregate::AggregateFunctionExpr; -use datafusion_physical_expr::{physical_exprs_equal, PhysicalExpr}; -use datafusion_physical_optimizer::PhysicalOptimizerRule; - -/// CombinePartialFinalAggregate optimizer rule combines the adjacent Partial and Final AggregateExecs -/// into a Single AggregateExec if their grouping exprs and aggregate exprs equal. -/// -/// This rule should be applied after the EnforceDistribution and EnforceSorting rules -/// -#[derive(Default)] -pub struct CombinePartialFinalAggregate {} - -impl CombinePartialFinalAggregate { - #[allow(missing_docs)] - pub fn new() -> Self { - Self {} - } -} - -impl PhysicalOptimizerRule for CombinePartialFinalAggregate { - fn optimize( - &self, - plan: Arc, - _config: &ConfigOptions, - ) -> Result> { - plan.transform_down(|plan| { - // Check if the plan is AggregateExec - let Some(agg_exec) = plan.as_any().downcast_ref::() else { - return Ok(Transformed::no(plan)); - }; - - if !matches!( - agg_exec.mode(), - AggregateMode::Final | AggregateMode::FinalPartitioned - ) { - return Ok(Transformed::no(plan)); - } - - // Check if the input is AggregateExec - let Some(input_agg_exec) = - agg_exec.input().as_any().downcast_ref::() - else { - return Ok(Transformed::no(plan)); - }; - - let transformed = if matches!(input_agg_exec.mode(), AggregateMode::Partial) - && can_combine( - ( - agg_exec.group_expr(), - agg_exec.aggr_expr(), - agg_exec.filter_expr(), - ), - ( - input_agg_exec.group_expr(), - input_agg_exec.aggr_expr(), - input_agg_exec.filter_expr(), - ), - ) { - let mode = if agg_exec.mode() == &AggregateMode::Final { - AggregateMode::Single - } else { - AggregateMode::SinglePartitioned - }; - AggregateExec::try_new( - mode, - input_agg_exec.group_expr().clone(), - input_agg_exec.aggr_expr().to_vec(), - input_agg_exec.filter_expr().to_vec(), - input_agg_exec.input().clone(), - input_agg_exec.input_schema(), - ) - .map(|combined_agg| combined_agg.with_limit(agg_exec.limit())) - .ok() - .map(Arc::new) - } else { - None - }; - Ok(if let Some(transformed) = transformed { - Transformed::yes(transformed) - } else { - Transformed::no(plan) - }) - }) - .data() - } - - fn name(&self) -> &str { - "CombinePartialFinalAggregate" - } - - fn schema_check(&self) -> bool { - true - } -} - -type GroupExprsRef<'a> = ( - &'a PhysicalGroupBy, - &'a [Arc], - &'a [Option>], -); - -fn can_combine(final_agg: GroupExprsRef, partial_agg: GroupExprsRef) -> bool { - let (final_group_by, final_aggr_expr, final_filter_expr) = final_agg; - let (input_group_by, input_aggr_expr, input_filter_expr) = partial_agg; - - // Compare output expressions of the partial, and input expressions of the final operator. - physical_exprs_equal( - &input_group_by.output_exprs(), - &final_group_by.input_exprs(), - ) && input_group_by.groups() == final_group_by.groups() - && input_group_by.null_expr().len() == final_group_by.null_expr().len() - && input_group_by - .null_expr() - .iter() - .zip(final_group_by.null_expr().iter()) - .all(|((lhs_expr, lhs_str), (rhs_expr, rhs_str))| { - lhs_expr.eq(rhs_expr) && lhs_str == rhs_str - }) - && final_aggr_expr.len() == input_aggr_expr.len() - && final_aggr_expr - .iter() - .zip(input_aggr_expr.iter()) - .all(|(final_expr, partial_expr)| final_expr.eq(partial_expr)) - && final_filter_expr.len() == input_filter_expr.len() - && final_filter_expr.iter().zip(input_filter_expr.iter()).all( - |(final_expr, partial_expr)| match (final_expr, partial_expr) { - (Some(l), Some(r)) => l.eq(r), - (None, None) => true, - _ => false, - }, - ) -} - -#[cfg(test)] -mod tests { - use super::*; - use crate::datasource::listing::PartitionedFile; - use crate::datasource::object_store::ObjectStoreUrl; - use crate::datasource::physical_plan::{FileScanConfig, ParquetExec}; - use crate::physical_plan::expressions::lit; - use crate::physical_plan::repartition::RepartitionExec; - use crate::physical_plan::{displayable, Partitioning}; - - use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; - use datafusion_functions_aggregate::count::count_udaf; - use datafusion_functions_aggregate::sum::sum_udaf; - use datafusion_physical_expr::aggregate::AggregateExprBuilder; - use datafusion_physical_expr::expressions::col; - - /// Runs the CombinePartialFinalAggregate optimizer and asserts the plan against the expected - macro_rules! assert_optimized { - ($EXPECTED_LINES: expr, $PLAN: expr) => { - let expected_lines: Vec<&str> = $EXPECTED_LINES.iter().map(|s| *s).collect(); - - // run optimizer - let optimizer = CombinePartialFinalAggregate {}; - let config = ConfigOptions::new(); - let optimized = optimizer.optimize($PLAN, &config)?; - // Now format correctly - let plan = displayable(optimized.as_ref()).indent(true).to_string(); - let actual_lines = trim_plan_display(&plan); - - assert_eq!( - &expected_lines, &actual_lines, - "\n\nexpected:\n\n{:#?}\nactual:\n\n{:#?}\n\n", - expected_lines, actual_lines - ); - }; - } - - fn trim_plan_display(plan: &str) -> Vec<&str> { - plan.split('\n') - .map(|s| s.trim()) - .filter(|s| !s.is_empty()) - .collect() - } - - fn schema() -> SchemaRef { - Arc::new(Schema::new(vec![ - Field::new("a", DataType::Int64, true), - Field::new("b", DataType::Int64, true), - Field::new("c", DataType::Int64, true), - ])) - } - - fn parquet_exec(schema: &SchemaRef) -> Arc { - ParquetExec::builder( - FileScanConfig::new( - ObjectStoreUrl::parse("test:///").unwrap(), - schema.clone(), - ) - .with_file(PartitionedFile::new("x".to_string(), 100)), - ) - .build_arc() - } - - fn partial_aggregate_exec( - input: Arc, - group_by: PhysicalGroupBy, - aggr_expr: Vec>, - ) -> Arc { - let schema = input.schema(); - let n_aggr = aggr_expr.len(); - Arc::new( - AggregateExec::try_new( - AggregateMode::Partial, - group_by, - aggr_expr, - vec![None; n_aggr], - input, - schema, - ) - .unwrap(), - ) - } - - fn final_aggregate_exec( - input: Arc, - group_by: PhysicalGroupBy, - aggr_expr: Vec>, - ) -> Arc { - let schema = input.schema(); - let n_aggr = aggr_expr.len(); - Arc::new( - AggregateExec::try_new( - AggregateMode::Final, - group_by, - aggr_expr, - vec![None; n_aggr], - input, - schema, - ) - .unwrap(), - ) - } - - fn repartition_exec(input: Arc) -> Arc { - Arc::new( - RepartitionExec::try_new(input, Partitioning::RoundRobinBatch(10)).unwrap(), - ) - } - - // Return appropriate expr depending if COUNT is for col or table (*) - fn count_expr( - expr: Arc, - name: &str, - schema: &Schema, - ) -> Arc { - AggregateExprBuilder::new(count_udaf(), vec![expr]) - .schema(Arc::new(schema.clone())) - .alias(name) - .build() - .unwrap() - } - - #[test] - fn aggregations_not_combined() -> Result<()> { - let schema = schema(); - - let aggr_expr = vec![count_expr(lit(1i8), "COUNT(1)", &schema)]; - - let plan = final_aggregate_exec( - repartition_exec(partial_aggregate_exec( - parquet_exec(&schema), - PhysicalGroupBy::default(), - aggr_expr.clone(), - )), - PhysicalGroupBy::default(), - aggr_expr, - ); - // should not combine the Partial/Final AggregateExecs - let expected = &[ - "AggregateExec: mode=Final, gby=[], aggr=[COUNT(1)]", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "AggregateExec: mode=Partial, gby=[], aggr=[COUNT(1)]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c]", - ]; - assert_optimized!(expected, plan); - - let aggr_expr1 = vec![count_expr(lit(1i8), "COUNT(1)", &schema)]; - let aggr_expr2 = vec![count_expr(lit(1i8), "COUNT(2)", &schema)]; - - let plan = final_aggregate_exec( - partial_aggregate_exec( - parquet_exec(&schema), - PhysicalGroupBy::default(), - aggr_expr1, - ), - PhysicalGroupBy::default(), - aggr_expr2, - ); - // should not combine the Partial/Final AggregateExecs - let expected = &[ - "AggregateExec: mode=Final, gby=[], aggr=[COUNT(2)]", - "AggregateExec: mode=Partial, gby=[], aggr=[COUNT(1)]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c]", - ]; - - assert_optimized!(expected, plan); - - Ok(()) - } - - #[test] - fn aggregations_combined() -> Result<()> { - let schema = schema(); - let aggr_expr = vec![count_expr(lit(1i8), "COUNT(1)", &schema)]; - - let plan = final_aggregate_exec( - partial_aggregate_exec( - parquet_exec(&schema), - PhysicalGroupBy::default(), - aggr_expr.clone(), - ), - PhysicalGroupBy::default(), - aggr_expr, - ); - // should combine the Partial/Final AggregateExecs to the Single AggregateExec - let expected = &[ - "AggregateExec: mode=Single, gby=[], aggr=[COUNT(1)]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c]", - ]; - - assert_optimized!(expected, plan); - Ok(()) - } - - #[test] - fn aggregations_with_group_combined() -> Result<()> { - let schema = schema(); - let aggr_expr = - vec![ - AggregateExprBuilder::new(sum_udaf(), vec![col("b", &schema)?]) - .schema(Arc::clone(&schema)) - .alias("Sum(b)") - .build() - .unwrap(), - ]; - let groups: Vec<(Arc, String)> = - vec![(col("c", &schema)?, "c".to_string())]; - - let partial_group_by = PhysicalGroupBy::new_single(groups); - let partial_agg = partial_aggregate_exec( - parquet_exec(&schema), - partial_group_by, - aggr_expr.clone(), - ); - - let groups: Vec<(Arc, String)> = - vec![(col("c", &partial_agg.schema())?, "c".to_string())]; - let final_group_by = PhysicalGroupBy::new_single(groups); - - let plan = final_aggregate_exec(partial_agg, final_group_by, aggr_expr); - // should combine the Partial/Final AggregateExecs to the Single AggregateExec - let expected = &[ - "AggregateExec: mode=Single, gby=[c@2 as c], aggr=[Sum(b)]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c]", - ]; - - assert_optimized!(expected, plan); - Ok(()) - } - - #[test] - fn aggregations_with_limit_combined() -> Result<()> { - let schema = schema(); - let aggr_expr = vec![]; - - let groups: Vec<(Arc, String)> = - vec![(col("c", &schema)?, "c".to_string())]; - - let partial_group_by = PhysicalGroupBy::new_single(groups); - let partial_agg = partial_aggregate_exec( - parquet_exec(&schema), - partial_group_by, - aggr_expr.clone(), - ); - - let groups: Vec<(Arc, String)> = - vec![(col("c", &partial_agg.schema())?, "c".to_string())]; - let final_group_by = PhysicalGroupBy::new_single(groups); - - let schema = partial_agg.schema(); - let final_agg = Arc::new( - AggregateExec::try_new( - AggregateMode::Final, - final_group_by, - aggr_expr, - vec![], - partial_agg, - schema, - ) - .unwrap() - .with_limit(Some(5)), - ); - let plan: Arc = final_agg; - // should combine the Partial/Final AggregateExecs to a Single AggregateExec - // with the final limit preserved - let expected = &[ - "AggregateExec: mode=Single, gby=[c@2 as c], aggr=[], lim=[5]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c]", - ]; - - assert_optimized!(expected, plan); - Ok(()) - } -} diff --git a/datafusion/core/src/physical_optimizer/mod.rs b/datafusion/core/src/physical_optimizer/mod.rs index c32c77043f15..46d86ead183b 100644 --- a/datafusion/core/src/physical_optimizer/mod.rs +++ b/datafusion/core/src/physical_optimizer/mod.rs @@ -22,7 +22,6 @@ //! //! [`ExecutionPlan`]: crate::physical_plan::ExecutionPlan pub mod coalesce_batches; -pub mod combine_partial_final_agg; pub mod enforce_distribution; pub mod enforce_sorting; pub mod join_selection; diff --git a/datafusion/core/tests/physical_optimizer/combine_partial_final_agg.rs b/datafusion/core/tests/physical_optimizer/combine_partial_final_agg.rs new file mode 100644 index 000000000000..5152afa6c050 --- /dev/null +++ b/datafusion/core/tests/physical_optimizer/combine_partial_final_agg.rs @@ -0,0 +1,290 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +use std::sync::Arc; + +use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; +use datafusion::datasource::listing::PartitionedFile; +use datafusion::datasource::physical_plan::{FileScanConfig, ParquetExec}; +use datafusion::physical_optimizer::combine_partial_final_agg::CombinePartialFinalAggregate; +use datafusion_common::config::ConfigOptions; +use datafusion_execution::object_store::ObjectStoreUrl; +use datafusion_functions_aggregate::count::count_udaf; +use datafusion_functions_aggregate::sum::sum_udaf; +use datafusion_physical_expr::aggregate::{AggregateExprBuilder, AggregateFunctionExpr}; +use datafusion_physical_expr::expressions::{col, lit}; +use datafusion_physical_expr::Partitioning; +use datafusion_physical_expr_common::physical_expr::PhysicalExpr; +use datafusion_physical_optimizer::PhysicalOptimizerRule; +use datafusion_physical_plan::aggregates::{ + AggregateExec, AggregateMode, PhysicalGroupBy, +}; +use datafusion_physical_plan::displayable; +use datafusion_physical_plan::repartition::RepartitionExec; +use datafusion_physical_plan::ExecutionPlan; + +/// Runs the CombinePartialFinalAggregate optimizer and asserts the plan against the expected +macro_rules! assert_optimized { + ($EXPECTED_LINES: expr, $PLAN: expr) => { + let expected_lines: Vec<&str> = $EXPECTED_LINES.iter().map(|s| *s).collect(); + + // run optimizer + let optimizer = CombinePartialFinalAggregate {}; + let config = ConfigOptions::new(); + let optimized = optimizer.optimize($PLAN, &config)?; + // Now format correctly + let plan = displayable(optimized.as_ref()).indent(true).to_string(); + let actual_lines = trim_plan_display(&plan); + + assert_eq!( + &expected_lines, &actual_lines, + "\n\nexpected:\n\n{:#?}\nactual:\n\n{:#?}\n\n", + expected_lines, actual_lines + ); + }; +} + +fn trim_plan_display(plan: &str) -> Vec<&str> { + plan.split('\n') + .map(|s| s.trim()) + .filter(|s| !s.is_empty()) + .collect() +} + +fn schema() -> SchemaRef { + Arc::new(Schema::new(vec![ + Field::new("a", DataType::Int64, true), + Field::new("b", DataType::Int64, true), + Field::new("c", DataType::Int64, true), + ])) +} + +fn parquet_exec(schema: &SchemaRef) -> Arc { + ParquetExec::builder( + FileScanConfig::new(ObjectStoreUrl::parse("test:///").unwrap(), schema.clone()) + .with_file(PartitionedFile::new("x".to_string(), 100)), + ) + .build_arc() +} + +fn partial_aggregate_exec( + input: Arc, + group_by: PhysicalGroupBy, + aggr_expr: Vec>, +) -> Arc { + let schema = input.schema(); + let n_aggr = aggr_expr.len(); + Arc::new( + AggregateExec::try_new( + AggregateMode::Partial, + group_by, + aggr_expr, + vec![None; n_aggr], + input, + schema, + ) + .unwrap(), + ) +} + +fn final_aggregate_exec( + input: Arc, + group_by: PhysicalGroupBy, + aggr_expr: Vec>, +) -> Arc { + let schema = input.schema(); + let n_aggr = aggr_expr.len(); + Arc::new( + AggregateExec::try_new( + AggregateMode::Final, + group_by, + aggr_expr, + vec![None; n_aggr], + input, + schema, + ) + .unwrap(), + ) +} + +fn repartition_exec(input: Arc) -> Arc { + Arc::new(RepartitionExec::try_new(input, Partitioning::RoundRobinBatch(10)).unwrap()) +} + +// Return appropriate expr depending if COUNT is for col or table (*) +fn count_expr( + expr: Arc, + name: &str, + schema: &Schema, +) -> Arc { + AggregateExprBuilder::new(count_udaf(), vec![expr]) + .schema(Arc::new(schema.clone())) + .alias(name) + .build() + .unwrap() +} + +#[test] +fn aggregations_not_combined() -> datafusion_common::Result<()> { + let schema = schema(); + + let aggr_expr = vec![count_expr(lit(1i8), "COUNT(1)", &schema)]; + + let plan = final_aggregate_exec( + repartition_exec(partial_aggregate_exec( + parquet_exec(&schema), + PhysicalGroupBy::default(), + aggr_expr.clone(), + )), + PhysicalGroupBy::default(), + aggr_expr, + ); + // should not combine the Partial/Final AggregateExecs + let expected = &[ + "AggregateExec: mode=Final, gby=[], aggr=[COUNT(1)]", + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "AggregateExec: mode=Partial, gby=[], aggr=[COUNT(1)]", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c]", + ]; + assert_optimized!(expected, plan); + + let aggr_expr1 = vec![count_expr(lit(1i8), "COUNT(1)", &schema)]; + let aggr_expr2 = vec![count_expr(lit(1i8), "COUNT(2)", &schema)]; + + let plan = final_aggregate_exec( + partial_aggregate_exec( + parquet_exec(&schema), + PhysicalGroupBy::default(), + aggr_expr1, + ), + PhysicalGroupBy::default(), + aggr_expr2, + ); + // should not combine the Partial/Final AggregateExecs + let expected = &[ + "AggregateExec: mode=Final, gby=[], aggr=[COUNT(2)]", + "AggregateExec: mode=Partial, gby=[], aggr=[COUNT(1)]", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c]", + ]; + + assert_optimized!(expected, plan); + + Ok(()) +} + +#[test] +fn aggregations_combined() -> datafusion_common::Result<()> { + let schema = schema(); + let aggr_expr = vec![count_expr(lit(1i8), "COUNT(1)", &schema)]; + + let plan = final_aggregate_exec( + partial_aggregate_exec( + parquet_exec(&schema), + PhysicalGroupBy::default(), + aggr_expr.clone(), + ), + PhysicalGroupBy::default(), + aggr_expr, + ); + // should combine the Partial/Final AggregateExecs to the Single AggregateExec + let expected = &[ + "AggregateExec: mode=Single, gby=[], aggr=[COUNT(1)]", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c]", + ]; + + assert_optimized!(expected, plan); + Ok(()) +} + +#[test] +fn aggregations_with_group_combined() -> datafusion_common::Result<()> { + let schema = schema(); + let aggr_expr = vec![ + AggregateExprBuilder::new(sum_udaf(), vec![col("b", &schema)?]) + .schema(Arc::clone(&schema)) + .alias("Sum(b)") + .build() + .unwrap(), + ]; + let groups: Vec<(Arc, String)> = + vec![(col("c", &schema)?, "c".to_string())]; + + let partial_group_by = PhysicalGroupBy::new_single(groups); + let partial_agg = partial_aggregate_exec( + parquet_exec(&schema), + partial_group_by, + aggr_expr.clone(), + ); + + let groups: Vec<(Arc, String)> = + vec![(col("c", &partial_agg.schema())?, "c".to_string())]; + let final_group_by = PhysicalGroupBy::new_single(groups); + + let plan = final_aggregate_exec(partial_agg, final_group_by, aggr_expr); + // should combine the Partial/Final AggregateExecs to the Single AggregateExec + let expected = &[ + "AggregateExec: mode=Single, gby=[c@2 as c], aggr=[Sum(b)]", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c]", + ]; + + assert_optimized!(expected, plan); + Ok(()) +} + +#[test] +fn aggregations_with_limit_combined() -> datafusion_common::Result<()> { + let schema = schema(); + let aggr_expr = vec![]; + + let groups: Vec<(Arc, String)> = + vec![(col("c", &schema)?, "c".to_string())]; + + let partial_group_by = PhysicalGroupBy::new_single(groups); + let partial_agg = partial_aggregate_exec( + parquet_exec(&schema), + partial_group_by, + aggr_expr.clone(), + ); + + let groups: Vec<(Arc, String)> = + vec![(col("c", &partial_agg.schema())?, "c".to_string())]; + let final_group_by = PhysicalGroupBy::new_single(groups); + + let schema = partial_agg.schema(); + let final_agg = Arc::new( + AggregateExec::try_new( + AggregateMode::Final, + final_group_by, + aggr_expr, + vec![], + partial_agg, + schema, + ) + .unwrap() + .with_limit(Some(5)), + ); + let plan: Arc = final_agg; + // should combine the Partial/Final AggregateExecs to a Single AggregateExec + // with the final limit preserved + let expected = &[ + "AggregateExec: mode=Single, gby=[c@2 as c], aggr=[], lim=[5]", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c]", + ]; + + assert_optimized!(expected, plan); + Ok(()) +} diff --git a/datafusion/core/tests/physical_optimizer/mod.rs b/datafusion/core/tests/physical_optimizer/mod.rs index 149103cf3482..4ec981bf2a74 100644 --- a/datafusion/core/tests/physical_optimizer/mod.rs +++ b/datafusion/core/tests/physical_optimizer/mod.rs @@ -16,6 +16,7 @@ // under the License. mod aggregate_statistics; +mod combine_partial_final_agg; mod limit_pushdown; mod limited_distinct_aggregation; mod test_util; diff --git a/datafusion/physical-optimizer/src/combine_partial_final_agg.rs b/datafusion/physical-optimizer/src/combine_partial_final_agg.rs new file mode 100644 index 000000000000..12ff13f8f6ae --- /dev/null +++ b/datafusion/physical-optimizer/src/combine_partial_final_agg.rs @@ -0,0 +1,164 @@ +// 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. + +//! CombinePartialFinalAggregate optimizer rule checks the adjacent Partial and Final AggregateExecs +//! and try to combine them if necessary + +use std::sync::Arc; + +use datafusion_common::error::Result; +use datafusion_physical_plan::aggregates::{ + AggregateExec, AggregateMode, PhysicalGroupBy, +}; +use datafusion_physical_plan::ExecutionPlan; + +use crate::PhysicalOptimizerRule; +use datafusion_common::config::ConfigOptions; +use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; +use datafusion_physical_expr::aggregate::AggregateFunctionExpr; +use datafusion_physical_expr::{physical_exprs_equal, PhysicalExpr}; + +/// CombinePartialFinalAggregate optimizer rule combines the adjacent Partial and Final AggregateExecs +/// into a Single AggregateExec if their grouping exprs and aggregate exprs equal. +/// +/// This rule should be applied after the EnforceDistribution and EnforceSorting rules +/// +#[derive(Default)] +pub struct CombinePartialFinalAggregate {} + +impl CombinePartialFinalAggregate { + #[allow(missing_docs)] + pub fn new() -> Self { + Self {} + } +} + +impl PhysicalOptimizerRule for CombinePartialFinalAggregate { + fn optimize( + &self, + plan: Arc, + _config: &ConfigOptions, + ) -> Result> { + plan.transform_down(|plan| { + // Check if the plan is AggregateExec + let Some(agg_exec) = plan.as_any().downcast_ref::() else { + return Ok(Transformed::no(plan)); + }; + + if !matches!( + agg_exec.mode(), + AggregateMode::Final | AggregateMode::FinalPartitioned + ) { + return Ok(Transformed::no(plan)); + } + + // Check if the input is AggregateExec + let Some(input_agg_exec) = + agg_exec.input().as_any().downcast_ref::() + else { + return Ok(Transformed::no(plan)); + }; + + let transformed = if matches!(input_agg_exec.mode(), AggregateMode::Partial) + && can_combine( + ( + agg_exec.group_expr(), + agg_exec.aggr_expr(), + agg_exec.filter_expr(), + ), + ( + input_agg_exec.group_expr(), + input_agg_exec.aggr_expr(), + input_agg_exec.filter_expr(), + ), + ) { + let mode = if agg_exec.mode() == &AggregateMode::Final { + AggregateMode::Single + } else { + AggregateMode::SinglePartitioned + }; + AggregateExec::try_new( + mode, + input_agg_exec.group_expr().clone(), + input_agg_exec.aggr_expr().to_vec(), + input_agg_exec.filter_expr().to_vec(), + Arc::clone(input_agg_exec.input()), + input_agg_exec.input_schema(), + ) + .map(|combined_agg| combined_agg.with_limit(agg_exec.limit())) + .ok() + .map(Arc::new) + } else { + None + }; + Ok(if let Some(transformed) = transformed { + Transformed::yes(transformed) + } else { + Transformed::no(plan) + }) + }) + .data() + } + + fn name(&self) -> &str { + "CombinePartialFinalAggregate" + } + + fn schema_check(&self) -> bool { + true + } +} + +type GroupExprsRef<'a> = ( + &'a PhysicalGroupBy, + &'a [Arc], + &'a [Option>], +); + +fn can_combine(final_agg: GroupExprsRef, partial_agg: GroupExprsRef) -> bool { + let (final_group_by, final_aggr_expr, final_filter_expr) = final_agg; + let (input_group_by, input_aggr_expr, input_filter_expr) = partial_agg; + + // Compare output expressions of the partial, and input expressions of the final operator. + physical_exprs_equal( + &input_group_by.output_exprs(), + &final_group_by.input_exprs(), + ) && input_group_by.groups() == final_group_by.groups() + && input_group_by.null_expr().len() == final_group_by.null_expr().len() + && input_group_by + .null_expr() + .iter() + .zip(final_group_by.null_expr().iter()) + .all(|((lhs_expr, lhs_str), (rhs_expr, rhs_str))| { + lhs_expr.eq(rhs_expr) && lhs_str == rhs_str + }) + && final_aggr_expr.len() == input_aggr_expr.len() + && final_aggr_expr + .iter() + .zip(input_aggr_expr.iter()) + .all(|(final_expr, partial_expr)| final_expr.eq(partial_expr)) + && final_filter_expr.len() == input_filter_expr.len() + && final_filter_expr.iter().zip(input_filter_expr.iter()).all( + |(final_expr, partial_expr)| match (final_expr, partial_expr) { + (Some(l), Some(r)) => l.eq(r), + (None, None) => true, + _ => false, + }, + ) +} + +// See tests in datafusion/core/tests/physical_optimizer diff --git a/datafusion/physical-optimizer/src/lib.rs b/datafusion/physical-optimizer/src/lib.rs index caebdcc927ae..41dfdb84a9f4 100644 --- a/datafusion/physical-optimizer/src/lib.rs +++ b/datafusion/physical-optimizer/src/lib.rs @@ -18,6 +18,7 @@ #![deny(clippy::clone_on_ref_ptr)] pub mod aggregate_statistics; +pub mod combine_partial_final_agg; pub mod limit_pushdown; pub mod limited_distinct_aggregation; mod optimizer; From 77e0e3b8b4df21eede9f1e3d1f8ee7709681a2d4 Mon Sep 17 00:00:00 2001 From: jcsherin Date: Mon, 2 Sep 2024 18:24:14 +0530 Subject: [PATCH 072/154] Fixes missing `nth_value` UDAF expr function (#12279) * Makes `nth_value` expression API public * Updates type of `order_by` parameter --- datafusion/functions-aggregate/src/lib.rs | 1 + .../functions-aggregate/src/nth_value.rs | 28 ++++++++++++++----- .../tests/cases/roundtrip_logical_plan.rs | 13 +++++++++ 3 files changed, 35 insertions(+), 7 deletions(-) diff --git a/datafusion/functions-aggregate/src/lib.rs b/datafusion/functions-aggregate/src/lib.rs index b54cd181a0cb..ca0276d326a4 100644 --- a/datafusion/functions-aggregate/src/lib.rs +++ b/datafusion/functions-aggregate/src/lib.rs @@ -113,6 +113,7 @@ pub mod expr_fn { pub use super::median::median; pub use super::min_max::max; pub use super::min_max::min; + pub use super::nth_value::nth_value; pub use super::regr::regr_avgx; pub use super::regr::regr_avgy; pub use super::regr::regr_count; diff --git a/datafusion/functions-aggregate/src/nth_value.rs b/datafusion/functions-aggregate/src/nth_value.rs index 7425bdfa18e7..bbfe56914c91 100644 --- a/datafusion/functions-aggregate/src/nth_value.rs +++ b/datafusion/functions-aggregate/src/nth_value.rs @@ -30,19 +30,33 @@ use datafusion_common::{exec_err, internal_err, not_impl_err, Result, ScalarValu use datafusion_expr::function::{AccumulatorArgs, StateFieldsArgs}; use datafusion_expr::utils::format_state_name; use datafusion_expr::{ - Accumulator, AggregateUDFImpl, ReversedUDAF, Signature, Volatility, + lit, Accumulator, AggregateUDFImpl, ExprFunctionExt, ReversedUDAF, Signature, + SortExpr, Volatility, }; use datafusion_functions_aggregate_common::merge_arrays::merge_ordered_arrays; use datafusion_functions_aggregate_common::utils::ordering_fields; use datafusion_physical_expr::expressions::Literal; use datafusion_physical_expr_common::sort_expr::{LexOrdering, PhysicalSortExpr}; -make_udaf_expr_and_func!( - NthValueAgg, - nth_value, - "Returns the nth value in a group of values.", - nth_value_udaf -); +create_func!(NthValueAgg, nth_value_udaf); + +/// Returns the nth value in a group of values. +pub fn nth_value( + expr: datafusion_expr::Expr, + n: i64, + order_by: Vec, +) -> datafusion_expr::Expr { + let args = vec![expr, lit(n)]; + if !order_by.is_empty() { + nth_value_udaf() + .call(args) + .order_by(order_by) + .build() + .unwrap() + } else { + nth_value_udaf().call(args) + } +} /// Expression for a `NTH_VALUE(... ORDER BY ..., ...)` aggregation. In a multi /// partition setting, partial aggregations are computed for every partition, diff --git a/datafusion/proto/tests/cases/roundtrip_logical_plan.rs b/datafusion/proto/tests/cases/roundtrip_logical_plan.rs index e174d1b50713..994ed8ad2352 100644 --- a/datafusion/proto/tests/cases/roundtrip_logical_plan.rs +++ b/datafusion/proto/tests/cases/roundtrip_logical_plan.rs @@ -71,6 +71,7 @@ use datafusion_expr::{ use datafusion_functions_aggregate::average::avg_udaf; use datafusion_functions_aggregate::expr_fn::{ approx_distinct, array_agg, avg, bit_and, bit_or, bit_xor, bool_and, bool_or, corr, + nth_value, }; use datafusion_functions_aggregate::string_agg::string_agg; use datafusion_proto::bytes::{ @@ -903,6 +904,18 @@ async fn roundtrip_expr_api() -> Result<()> { vec![lit(10), lit(20), lit(30)], ), row_number(), + nth_value(col("b"), 1, vec![]), + nth_value( + col("b"), + 1, + vec![col("a").sort(false, false), col("b").sort(true, false)], + ), + nth_value(col("b"), -1, vec![]), + nth_value( + col("b"), + -1, + vec![col("a").sort(false, false), col("b").sort(true, false)], + ), ]; // ensure expressions created with the expr api can be round tripped From ac74cd3163e43563807a8c6e8e72bb058cb6f459 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Mon, 2 Sep 2024 09:02:42 -0400 Subject: [PATCH 073/154] Minor: Add `RuntimeEnvBuilder::build_arc() (#12213) --- datafusion/core/src/execution/context/mod.rs | 6 +-- datafusion/core/tests/fuzz_cases/sort_fuzz.rs | 10 ++--- datafusion/core/tests/memory_limit/mod.rs | 11 +++-- .../core/tests/parquet/file_statistics.rs | 11 +++-- datafusion/execution/src/runtime_env.rs | 5 +++ datafusion/execution/src/task.rs | 4 +- .../physical-plan/src/aggregates/mod.rs | 18 ++++----- .../physical-plan/src/joins/cross_join.rs | 8 ++-- .../physical-plan/src/joins/hash_join.rs | 16 +++----- .../src/joins/nested_loop_join.rs | 8 ++-- .../src/joins/sort_merge_join.rs | 40 ++++++++----------- .../physical-plan/src/repartition/mod.rs | 8 ++-- datafusion/physical-plan/src/sorts/sort.rs | 22 +++++----- datafusion/wasmtest/src/lib.rs | 10 ++--- 14 files changed, 75 insertions(+), 102 deletions(-) diff --git a/datafusion/core/src/execution/context/mod.rs b/datafusion/core/src/execution/context/mod.rs index c67424c0fa53..06dc797ae27a 100644 --- a/datafusion/core/src/execution/context/mod.rs +++ b/datafusion/core/src/execution/context/mod.rs @@ -219,13 +219,13 @@ where /// // configure a memory limit of 1GB with 20% slop /// let runtime_env = RuntimeEnvBuilder::new() /// .with_memory_limit(1024 * 1024 * 1024, 0.80) -/// .build() +/// .build_arc() /// .unwrap(); /// /// // Create a SessionState using the config and runtime_env /// let state = SessionStateBuilder::new() /// .with_config(config) -/// .with_runtime_env(Arc::new(runtime_env)) +/// .with_runtime_env(runtime_env) /// // include support for built in functions and configurations /// .with_default_features() /// .build(); @@ -1758,7 +1758,7 @@ mod tests { let path = path.join("tests/tpch-csv"); let url = format!("file://{}", path.display()); - let runtime = Arc::new(RuntimeEnvBuilder::new().build()?); + let runtime = RuntimeEnvBuilder::new().build_arc()?; let cfg = SessionConfig::new() .set_str("datafusion.catalog.location", url.as_str()) .set_str("datafusion.catalog.format", "CSV") diff --git a/datafusion/core/tests/fuzz_cases/sort_fuzz.rs b/datafusion/core/tests/fuzz_cases/sort_fuzz.rs index 1980589491a5..fae4731569b6 100644 --- a/datafusion/core/tests/fuzz_cases/sort_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/sort_fuzz.rs @@ -136,12 +136,10 @@ impl SortTest { .sort_spill_reservation_bytes, ); - let runtime = Arc::new( - RuntimeEnvBuilder::new() - .with_memory_pool(Arc::new(GreedyMemoryPool::new(pool_size))) - .build() - .unwrap(), - ); + let runtime = RuntimeEnvBuilder::new() + .with_memory_pool(Arc::new(GreedyMemoryPool::new(pool_size))) + .build_arc() + .unwrap(); SessionContext::new_with_config_rt(session_config, runtime) } else { SessionContext::new_with_config(session_config) diff --git a/datafusion/core/tests/memory_limit/mod.rs b/datafusion/core/tests/memory_limit/mod.rs index 592c25dedc50..69ef6058a2f6 100644 --- a/datafusion/core/tests/memory_limit/mod.rs +++ b/datafusion/core/tests/memory_limit/mod.rs @@ -509,21 +509,20 @@ impl TestCase { let table = scenario.table(); - let rt_config = RuntimeEnvBuilder::new() + let mut builder = RuntimeEnvBuilder::new() // disk manager setting controls the spilling .with_disk_manager(disk_manager_config) .with_memory_limit(memory_limit, MEMORY_FRACTION); - let runtime = if let Some(pool) = memory_pool { - rt_config.with_memory_pool(pool).build().unwrap() - } else { - rt_config.build().unwrap() + if let Some(pool) = memory_pool { + builder = builder.with_memory_pool(pool); }; + let runtime = builder.build_arc().unwrap(); // Configure execution let builder = SessionStateBuilder::new() .with_config(config) - .with_runtime_env(Arc::new(runtime)) + .with_runtime_env(runtime) .with_default_features(); let builder = match scenario.rules() { Some(rules) => builder.with_physical_optimizer_rules(rules), diff --git a/datafusion/core/tests/parquet/file_statistics.rs b/datafusion/core/tests/parquet/file_statistics.rs index bd251f1a6669..cd62c3bf426f 100644 --- a/datafusion/core/tests/parquet/file_statistics.rs +++ b/datafusion/core/tests/parquet/file_statistics.rs @@ -197,12 +197,11 @@ fn get_cache_runtime_state() -> ( .with_files_statistics_cache(Some(file_static_cache.clone())) .with_list_files_cache(Some(list_file_cache.clone())); - let rt = Arc::new( - RuntimeEnvBuilder::new() - .with_cache_manager(cache_config) - .build() - .expect("could not build runtime environment"), - ); + let rt = RuntimeEnvBuilder::new() + .with_cache_manager(cache_config) + .build_arc() + .expect("could not build runtime environment"); + let state = SessionContext::new_with_config_rt(SessionConfig::default(), rt).state(); (file_static_cache, list_file_cache, state) diff --git a/datafusion/execution/src/runtime_env.rs b/datafusion/execution/src/runtime_env.rs index e7b48be95cff..574d387ae697 100644 --- a/datafusion/execution/src/runtime_env.rs +++ b/datafusion/execution/src/runtime_env.rs @@ -246,4 +246,9 @@ impl RuntimeEnvBuilder { object_store_registry: self.object_store_registry, }) } + + /// Convenience method to create a new `Arc` + pub fn build_arc(self) -> Result> { + self.build().map(Arc::new) + } } diff --git a/datafusion/execution/src/task.rs b/datafusion/execution/src/task.rs index 35689b8e08df..57fcac0ee5ab 100644 --- a/datafusion/execution/src/task.rs +++ b/datafusion/execution/src/task.rs @@ -58,7 +58,7 @@ pub struct TaskContext { impl Default for TaskContext { fn default() -> Self { let runtime = RuntimeEnvBuilder::new() - .build() + .build_arc() .expect("default runtime created successfully"); // Create a default task context, mostly useful for testing @@ -69,7 +69,7 @@ impl Default for TaskContext { scalar_functions: HashMap::new(), aggregate_functions: HashMap::new(), window_functions: HashMap::new(), - runtime: Arc::new(runtime), + runtime, } } } diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index e66a57fd2ee5..764227e5e717 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -1331,12 +1331,10 @@ mod tests { fn new_spill_ctx(batch_size: usize, max_memory: usize) -> Arc { let session_config = SessionConfig::new().with_batch_size(batch_size); - let runtime = Arc::new( - RuntimeEnvBuilder::default() - .with_memory_pool(Arc::new(FairSpillPool::new(max_memory))) - .build() - .unwrap(), - ); + let runtime = RuntimeEnvBuilder::default() + .with_memory_pool(Arc::new(FairSpillPool::new(max_memory))) + .build_arc() + .unwrap(); let task_ctx = TaskContext::default() .with_session_config(session_config) .with_runtime(runtime); @@ -1815,11 +1813,9 @@ mod tests { let input: Arc = Arc::new(TestYieldingExec::new(true)); let input_schema = input.schema(); - let runtime = Arc::new( - RuntimeEnvBuilder::default() - .with_memory_limit(1, 1.0) - .build()?, - ); + let runtime = RuntimeEnvBuilder::default() + .with_memory_limit(1, 1.0) + .build_arc()?; let task_ctx = TaskContext::default().with_runtime(runtime); let task_ctx = Arc::new(task_ctx); diff --git a/datafusion/physical-plan/src/joins/cross_join.rs b/datafusion/physical-plan/src/joins/cross_join.rs index b99d4f17c42a..11153556f253 100644 --- a/datafusion/physical-plan/src/joins/cross_join.rs +++ b/datafusion/physical-plan/src/joins/cross_join.rs @@ -673,11 +673,9 @@ mod tests { #[tokio::test] async fn test_overallocation() -> Result<()> { - let runtime = Arc::new( - RuntimeEnvBuilder::new() - .with_memory_limit(100, 1.0) - .build()?, - ); + let runtime = RuntimeEnvBuilder::new() + .with_memory_limit(100, 1.0) + .build_arc()?; let task_ctx = TaskContext::default().with_runtime(runtime); let task_ctx = Arc::new(task_ctx); diff --git a/datafusion/physical-plan/src/joins/hash_join.rs b/datafusion/physical-plan/src/joins/hash_join.rs index f20d00e1a298..38827108e815 100644 --- a/datafusion/physical-plan/src/joins/hash_join.rs +++ b/datafusion/physical-plan/src/joins/hash_join.rs @@ -3798,11 +3798,9 @@ mod tests { ]; for join_type in join_types { - let runtime = Arc::new( - RuntimeEnvBuilder::new() - .with_memory_limit(100, 1.0) - .build()?, - ); + let runtime = RuntimeEnvBuilder::new() + .with_memory_limit(100, 1.0) + .build_arc()?; let task_ctx = TaskContext::default().with_runtime(runtime); let task_ctx = Arc::new(task_ctx); @@ -3874,11 +3872,9 @@ mod tests { ]; for join_type in join_types { - let runtime = Arc::new( - RuntimeEnvBuilder::new() - .with_memory_limit(100, 1.0) - .build()?, - ); + let runtime = RuntimeEnvBuilder::new() + .with_memory_limit(100, 1.0) + .build_arc()?; let session_config = SessionConfig::default().with_batch_size(50); let task_ctx = TaskContext::default() .with_session_config(session_config) diff --git a/datafusion/physical-plan/src/joins/nested_loop_join.rs b/datafusion/physical-plan/src/joins/nested_loop_join.rs index 3cd373544157..dadd20714ead 100644 --- a/datafusion/physical-plan/src/joins/nested_loop_join.rs +++ b/datafusion/physical-plan/src/joins/nested_loop_join.rs @@ -1019,11 +1019,9 @@ mod tests { ]; for join_type in join_types { - let runtime = Arc::new( - RuntimeEnvBuilder::new() - .with_memory_limit(100, 1.0) - .build()?, - ); + let runtime = RuntimeEnvBuilder::new() + .with_memory_limit(100, 1.0) + .build_arc()?; let task_ctx = TaskContext::default().with_runtime(runtime); let task_ctx = Arc::new(task_ctx); diff --git a/datafusion/physical-plan/src/joins/sort_merge_join.rs b/datafusion/physical-plan/src/joins/sort_merge_join.rs index 09fe5d9ebc54..2118c1a5266f 100644 --- a/datafusion/physical-plan/src/joins/sort_merge_join.rs +++ b/datafusion/physical-plan/src/joins/sort_merge_join.rs @@ -2900,12 +2900,10 @@ mod tests { ]; // Disable DiskManager to prevent spilling - let runtime = Arc::new( - RuntimeEnvBuilder::new() - .with_memory_limit(100, 1.0) - .with_disk_manager(DiskManagerConfig::Disabled) - .build()?, - ); + let runtime = RuntimeEnvBuilder::new() + .with_memory_limit(100, 1.0) + .with_disk_manager(DiskManagerConfig::Disabled) + .build_arc()?; let session_config = SessionConfig::default().with_batch_size(50); for join_type in join_types { @@ -2987,12 +2985,10 @@ mod tests { ]; // Disable DiskManager to prevent spilling - let runtime = Arc::new( - RuntimeEnvBuilder::new() - .with_memory_limit(100, 1.0) - .with_disk_manager(DiskManagerConfig::Disabled) - .build()?, - ); + let runtime = RuntimeEnvBuilder::new() + .with_memory_limit(100, 1.0) + .with_disk_manager(DiskManagerConfig::Disabled) + .build_arc()?; let session_config = SessionConfig::default().with_batch_size(50); for join_type in join_types { @@ -3052,12 +3048,10 @@ mod tests { ]; // Enable DiskManager to allow spilling - let runtime = Arc::new( - RuntimeEnvBuilder::new() - .with_memory_limit(100, 1.0) - .with_disk_manager(DiskManagerConfig::NewOs) - .build()?, - ); + let runtime = RuntimeEnvBuilder::new() + .with_memory_limit(100, 1.0) + .with_disk_manager(DiskManagerConfig::NewOs) + .build_arc()?; for batch_size in [1, 50] { let session_config = SessionConfig::default().with_batch_size(batch_size); @@ -3162,12 +3156,10 @@ mod tests { ]; // Enable DiskManager to allow spilling - let runtime = Arc::new( - RuntimeEnvBuilder::new() - .with_memory_limit(500, 1.0) - .with_disk_manager(DiskManagerConfig::NewOs) - .build()?, - ); + let runtime = RuntimeEnvBuilder::new() + .with_memory_limit(500, 1.0) + .with_disk_manager(DiskManagerConfig::NewOs) + .build_arc()?; for batch_size in [1, 50] { let session_config = SessionConfig::default().with_batch_size(batch_size); diff --git a/datafusion/physical-plan/src/repartition/mod.rs b/datafusion/physical-plan/src/repartition/mod.rs index 650006a9d02d..47e5192c237e 100644 --- a/datafusion/physical-plan/src/repartition/mod.rs +++ b/datafusion/physical-plan/src/repartition/mod.rs @@ -1506,11 +1506,9 @@ mod tests { let partitioning = Partitioning::RoundRobinBatch(4); // setup up context - let runtime = Arc::new( - RuntimeEnvBuilder::default() - .with_memory_limit(1, 1.0) - .build()?, - ); + let runtime = RuntimeEnvBuilder::default() + .with_memory_limit(1, 1.0) + .build_arc()?; let task_ctx = TaskContext::default().with_runtime(runtime); let task_ctx = Arc::new(task_ctx); diff --git a/datafusion/physical-plan/src/sorts/sort.rs b/datafusion/physical-plan/src/sorts/sort.rs index e0041194016c..fa9628abdfbb 100644 --- a/datafusion/physical-plan/src/sorts/sort.rs +++ b/datafusion/physical-plan/src/sorts/sort.rs @@ -1148,11 +1148,9 @@ mod tests { .options() .execution .sort_spill_reservation_bytes; - let runtime = Arc::new( - RuntimeEnvBuilder::new() - .with_memory_limit(sort_spill_reservation_bytes + 12288, 1.0) - .build()?, - ); + let runtime = RuntimeEnvBuilder::new() + .with_memory_limit(sort_spill_reservation_bytes + 12288, 1.0) + .build_arc()?; let task_ctx = Arc::new( TaskContext::default() .with_session_config(session_config) @@ -1226,14 +1224,12 @@ mod tests { .execution .sort_spill_reservation_bytes; - let runtime = Arc::new( - RuntimeEnvBuilder::new() - .with_memory_limit( - sort_spill_reservation_bytes + avg_batch_size * (partitions - 1), - 1.0, - ) - .build()?, - ); + let runtime = RuntimeEnvBuilder::new() + .with_memory_limit( + sort_spill_reservation_bytes + avg_batch_size * (partitions - 1), + 1.0, + ) + .build_arc()?; let task_ctx = Arc::new( TaskContext::default() .with_runtime(runtime) diff --git a/datafusion/wasmtest/src/lib.rs b/datafusion/wasmtest/src/lib.rs index 50325d262d1d..0f24449cbed3 100644 --- a/datafusion/wasmtest/src/lib.rs +++ b/datafusion/wasmtest/src/lib.rs @@ -98,12 +98,10 @@ mod test { let sql = "SELECT 2 + 2;"; // Execute SQL (using datafusion) - let rt = Arc::new( - RuntimeEnvBuilder::new() - .with_disk_manager(DiskManagerConfig::Disabled) - .build() - .unwrap(), - ); + let rt = RuntimeEnvBuilder::new() + .with_disk_manager(DiskManagerConfig::Disabled) + .build_arc() + .unwrap(); let session_config = SessionConfig::new().with_target_partitions(1); let session_context = Arc::new(SessionContext::new_with_config_rt(session_config, rt)); From 8db30e25d6fe65a9779d237cf48aea9aee297502 Mon Sep 17 00:00:00 2001 From: Jay Zhan Date: Tue, 3 Sep 2024 07:57:40 +0800 Subject: [PATCH 074/154] Introduce `Signature::Coercible` (#12275) * introduce signature float Signed-off-by: jayzhan211 * fix test Signed-off-by: jayzhan211 * change float to coercible Signed-off-by: jayzhan211 * fmt Signed-off-by: jayzhan211 * rm test Signed-off-by: jayzhan211 * add comment Signed-off-by: jayzhan211 * typo Signed-off-by: jayzhan211 --------- Signed-off-by: jayzhan211 --- datafusion/core/src/dataframe/mod.rs | 6 ++-- datafusion/expr-common/src/signature.rs | 15 ++++++++- .../src/type_coercion/aggregates.rs | 6 ++-- .../expr/src/type_coercion/functions.rs | 33 ++++++++++++++++++- datafusion/functions-aggregate/src/stddev.rs | 11 +++---- .../functions-aggregate/src/variance.rs | 11 +++---- 6 files changed, 64 insertions(+), 18 deletions(-) diff --git a/datafusion/core/src/dataframe/mod.rs b/datafusion/core/src/dataframe/mod.rs index b8c0bd9d744c..2138bd1294b4 100644 --- a/datafusion/core/src/dataframe/mod.rs +++ b/datafusion/core/src/dataframe/mod.rs @@ -2428,7 +2428,8 @@ mod tests { let df: Vec = df.select(aggr_expr)?.collect().await?; assert_batches_sorted_eq!( - ["+-------------+----------+-----------------+---------------+--------+-----+------+----+------+", + [ + "+-------------+----------+-----------------+---------------+--------+-----+------+----+------+", "| first_value | last_val | approx_distinct | approx_median | median | max | min | c2 | c3 |", "+-------------+----------+-----------------+---------------+--------+-----+------+----+------+", "| | | | | | | | 1 | -85 |", @@ -2452,7 +2453,8 @@ mod tests { "| -85 | 45 | 8 | -34 | 45 | 83 | -85 | 3 | -72 |", "| -85 | 65 | 17 | -17 | 65 | 83 | -101 | 5 | -101 |", "| -85 | 83 | 5 | -25 | 83 | 83 | -85 | 2 | -48 |", - "+-------------+----------+-----------------+---------------+--------+-----+------+----+------+"], + "+-------------+----------+-----------------+---------------+--------+-----+------+----+------+", + ], &df ); diff --git a/datafusion/expr-common/src/signature.rs b/datafusion/expr-common/src/signature.rs index 4dcfa423e371..2043757a49fb 100644 --- a/datafusion/expr-common/src/signature.rs +++ b/datafusion/expr-common/src/signature.rs @@ -105,6 +105,11 @@ pub enum TypeSignature { Uniform(usize, Vec), /// Exact number of arguments of an exact type Exact(Vec), + /// The number of arguments that can be coerced to in order + /// For example, `Coercible(vec![DataType::Float64])` accepts + /// arguments like `vec![DataType::Int32]` or `vec![DataType::Float32]` + /// since i32 and f32 can be casted to f64 + Coercible(Vec), /// Fixed number of arguments of arbitrary types /// If a function takes 0 argument, its `TypeSignature` should be `Any(0)` Any(usize), @@ -188,7 +193,7 @@ impl TypeSignature { TypeSignature::Numeric(num) => { vec![format!("Numeric({})", num)] } - TypeSignature::Exact(types) => { + TypeSignature::Exact(types) | TypeSignature::Coercible(types) => { vec![Self::join_types(types, ", ")] } TypeSignature::Any(arg_count) => { @@ -300,6 +305,14 @@ impl Signature { volatility, } } + /// Target coerce types in order + pub fn coercible(target_types: Vec, volatility: Volatility) -> Self { + Self { + type_signature: TypeSignature::Coercible(target_types), + volatility, + } + } + /// A specified number of arguments of any type pub fn any(arg_count: usize, volatility: Volatility) -> Self { Signature { diff --git a/datafusion/expr-common/src/type_coercion/aggregates.rs b/datafusion/expr-common/src/type_coercion/aggregates.rs index 40ee596eee05..2add9e7c1867 100644 --- a/datafusion/expr-common/src/type_coercion/aggregates.rs +++ b/datafusion/expr-common/src/type_coercion/aggregates.rs @@ -128,9 +128,11 @@ pub fn check_arg_count( ); } } - TypeSignature::UserDefined | TypeSignature::Numeric(_) => { + TypeSignature::UserDefined + | TypeSignature::Numeric(_) + | TypeSignature::Coercible(_) => { // User-defined signature is validated in `coerce_types` - // Numreic signature is validated in `get_valid_types` + // Numeric and Coercible signature is validated in `get_valid_types` } _ => { return internal_err!( diff --git a/datafusion/expr/src/type_coercion/functions.rs b/datafusion/expr/src/type_coercion/functions.rs index b0b14a1a4e6e..d30d202df050 100644 --- a/datafusion/expr/src/type_coercion/functions.rs +++ b/datafusion/expr/src/type_coercion/functions.rs @@ -175,7 +175,14 @@ fn try_coerce_types( let mut valid_types = valid_types; // Well-supported signature that returns exact valid types. - if !valid_types.is_empty() && matches!(type_signature, TypeSignature::UserDefined) { + if !valid_types.is_empty() + && matches!( + type_signature, + TypeSignature::UserDefined + | TypeSignature::Numeric(_) + | TypeSignature::Coercible(_) + ) + { // exact valid types assert_eq!(valid_types.len(), 1); let valid_types = valid_types.swap_remove(0); @@ -397,6 +404,30 @@ fn get_valid_types( vec![vec![valid_type; *number]] } + TypeSignature::Coercible(target_types) => { + if target_types.is_empty() { + return plan_err!( + "The signature expected at least one argument but received {}", + current_types.len() + ); + } + if target_types.len() != current_types.len() { + return plan_err!( + "The signature expected {} arguments but received {}", + target_types.len(), + current_types.len() + ); + } + + for (data_type, target_type) in current_types.iter().zip(target_types.iter()) + { + if !can_cast_types(data_type, target_type) { + return plan_err!("{data_type} is not coercible to {target_type}"); + } + } + + vec![target_types.to_owned()] + } TypeSignature::Uniform(number, valid_types) => valid_types .iter() .map(|valid_type| (0..*number).map(|_| valid_type.clone()).collect()) diff --git a/datafusion/functions-aggregate/src/stddev.rs b/datafusion/functions-aggregate/src/stddev.rs index 3534fb5b4d26..a25ab5e31991 100644 --- a/datafusion/functions-aggregate/src/stddev.rs +++ b/datafusion/functions-aggregate/src/stddev.rs @@ -68,7 +68,10 @@ impl Stddev { /// Create a new STDDEV aggregate function pub fn new() -> Self { Self { - signature: Signature::numeric(1, Volatility::Immutable), + signature: Signature::coercible( + vec![DataType::Float64], + Volatility::Immutable, + ), alias: vec!["stddev_samp".to_string()], } } @@ -88,11 +91,7 @@ impl AggregateUDFImpl for Stddev { &self.signature } - fn return_type(&self, arg_types: &[DataType]) -> Result { - if !arg_types[0].is_numeric() { - return plan_err!("Stddev requires numeric input types"); - } - + fn return_type(&self, _arg_types: &[DataType]) -> Result { Ok(DataType::Float64) } diff --git a/datafusion/functions-aggregate/src/variance.rs b/datafusion/functions-aggregate/src/variance.rs index f5f2d06e3837..367a8669ab7d 100644 --- a/datafusion/functions-aggregate/src/variance.rs +++ b/datafusion/functions-aggregate/src/variance.rs @@ -79,7 +79,10 @@ impl VarianceSample { pub fn new() -> Self { Self { aliases: vec![String::from("var_sample"), String::from("var_samp")], - signature: Signature::numeric(1, Volatility::Immutable), + signature: Signature::coercible( + vec![DataType::Float64], + Volatility::Immutable, + ), } } } @@ -97,11 +100,7 @@ impl AggregateUDFImpl for VarianceSample { &self.signature } - fn return_type(&self, arg_types: &[DataType]) -> Result { - if !arg_types[0].is_numeric() { - return plan_err!("Variance requires numeric input types"); - } - + fn return_type(&self, _arg_types: &[DataType]) -> Result { Ok(DataType::Float64) } From 4e1b6de5549cc7ed399766dd22cafd450701db16 Mon Sep 17 00:00:00 2001 From: iamthinh Date: Tue, 3 Sep 2024 00:35:56 -0700 Subject: [PATCH 075/154] fix partition-by panic (#12297) --- datafusion/physical-plan/src/coalesce/mod.rs | 15 ++++++++++++-- .../physical-plan/src/repartition/mod.rs | 12 ++++++++--- .../sqllogictest/test_files/repartition.slt | 20 +++++++++++++++++++ 3 files changed, 42 insertions(+), 5 deletions(-) diff --git a/datafusion/physical-plan/src/coalesce/mod.rs b/datafusion/physical-plan/src/coalesce/mod.rs index 5befa5ecda99..ce5a1e53abfd 100644 --- a/datafusion/physical-plan/src/coalesce/mod.rs +++ b/datafusion/physical-plan/src/coalesce/mod.rs @@ -18,7 +18,7 @@ use arrow::compute::concat_batches; use arrow_array::builder::StringViewBuilder; use arrow_array::cast::AsArray; -use arrow_array::{Array, ArrayRef, RecordBatch}; +use arrow_array::{Array, ArrayRef, RecordBatch, RecordBatchOptions}; use arrow_schema::SchemaRef; use std::sync::Arc; @@ -265,7 +265,9 @@ fn gc_string_view_batch(batch: &RecordBatch) -> RecordBatch { } }) .collect(); - RecordBatch::try_new(batch.schema(), new_columns) + let mut options = RecordBatchOptions::new(); + options = options.with_row_count(Some(batch.num_rows())); + RecordBatch::try_new_with_options(batch.schema(), new_columns, &options) .expect("Failed to re-create the gc'ed record batch") } @@ -501,6 +503,15 @@ mod tests { assert_eq!(array.data_buffers().len(), gc_array.data_buffers().len()); // no compaction } + #[test] + fn test_gc_string_view_test_batch_empty() { + let schema = Schema::empty(); + let batch = RecordBatch::new_empty(schema.into()); + let output_batch = gc_string_view_batch(&batch); + assert_eq!(batch.num_columns(), output_batch.num_columns()); + assert_eq!(batch.num_rows(), output_batch.num_rows()); + } + #[test] fn test_gc_string_view_batch_large_no_compact() { // view with large strings (has buffers) but full --> no need to compact diff --git a/datafusion/physical-plan/src/repartition/mod.rs b/datafusion/physical-plan/src/repartition/mod.rs index 47e5192c237e..093803e3c8b3 100644 --- a/datafusion/physical-plan/src/repartition/mod.rs +++ b/datafusion/physical-plan/src/repartition/mod.rs @@ -41,7 +41,7 @@ use crate::{DisplayFormatType, ExecutionPlan, Partitioning, PlanProperties, Stat use arrow::array::ArrayRef; use arrow::datatypes::{SchemaRef, UInt64Type}; use arrow::record_batch::RecordBatch; -use arrow_array::PrimitiveArray; +use arrow_array::{PrimitiveArray, RecordBatchOptions}; use datafusion_common::utils::transpose; use datafusion_common::{arrow_datafusion_err, not_impl_err, DataFusionError, Result}; use datafusion_common_runtime::SpawnedTask; @@ -309,8 +309,14 @@ impl BatchPartitioner { }) .collect::>>()?; - let batch = - RecordBatch::try_new(batch.schema(), columns).unwrap(); + let mut options = RecordBatchOptions::new(); + options = options.with_row_count(Some(indices.len())); + let batch = RecordBatch::try_new_with_options( + batch.schema(), + columns, + &options, + ) + .unwrap(); Ok((partition, batch)) }); diff --git a/datafusion/sqllogictest/test_files/repartition.slt b/datafusion/sqllogictest/test_files/repartition.slt index e3c204a4f4ce..2d59ad2b5b0e 100644 --- a/datafusion/sqllogictest/test_files/repartition.slt +++ b/datafusion/sqllogictest/test_files/repartition.slt @@ -127,3 +127,23 @@ physical_plan 04)------FilterExec: c3@2 > 0 05)--------RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1 06)----------StreamingTableExec: partition_sizes=1, projection=[c1, c2, c3], infinite_source=true + +# Start repratition on empty column test. +# See https://github.com/apache/datafusion/issues/12057 + +statement ok +CREATE TABLE t1(v1 int); + +statement ok +INSERT INTO t1 values(42); + +query I +SELECT sum(1) OVER (PARTITION BY false=false) +FROM t1 WHERE ((false > (v1 = v1)) IS DISTINCT FROM true); +---- +1 + +statement ok +DROP TABLE t1; + +# End repartition on empty columns test \ No newline at end of file From 4a227c524f9130eb9b911af17a25c23b25aa41fa Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Tue, 3 Sep 2024 09:51:19 +0200 Subject: [PATCH 076/154] Remove unsafe Send impl from PriorityMap (#12289) It's not necessary to use unsafe Send impl. It's enough to require the referenced trait objects as Send. --- .../physical-plan/src/aggregates/topk/hash_table.rs | 5 ++++- datafusion/physical-plan/src/aggregates/topk/heap.rs | 6 +++++- .../physical-plan/src/aggregates/topk/priority_map.rs | 9 ++------- 3 files changed, 11 insertions(+), 9 deletions(-) diff --git a/datafusion/physical-plan/src/aggregates/topk/hash_table.rs b/datafusion/physical-plan/src/aggregates/topk/hash_table.rs index 2b02fff1f573..232b87de3231 100644 --- a/datafusion/physical-plan/src/aggregates/topk/hash_table.rs +++ b/datafusion/physical-plan/src/aggregates/topk/hash_table.rs @@ -367,7 +367,10 @@ has_integer!(u8, u16, u32, u64); has_integer!(IntervalDayTime, IntervalMonthDayNano); hash_float!(f16, f32, f64); -pub fn new_hash_table(limit: usize, kt: DataType) -> Result> { +pub fn new_hash_table( + limit: usize, + kt: DataType, +) -> Result> { macro_rules! downcast_helper { ($kt:ty, $d:ident) => { return Ok(Box::new(PrimitiveHashTable::<$kt>::new(limit))) diff --git a/datafusion/physical-plan/src/aggregates/topk/heap.rs b/datafusion/physical-plan/src/aggregates/topk/heap.rs index 81eadbc018b3..e694422e443d 100644 --- a/datafusion/physical-plan/src/aggregates/topk/heap.rs +++ b/datafusion/physical-plan/src/aggregates/topk/heap.rs @@ -459,7 +459,11 @@ compare_integer!(u8, u16, u32, u64); compare_integer!(IntervalDayTime, IntervalMonthDayNano); compare_float!(f16, f32, f64); -pub fn new_heap(limit: usize, desc: bool, vt: DataType) -> Result> { +pub fn new_heap( + limit: usize, + desc: bool, + vt: DataType, +) -> Result> { macro_rules! downcast_helper { ($vt:ty, $d:ident) => { return Ok(Box::new(PrimitiveHeap::<$vt>::new(limit, desc, vt))) diff --git a/datafusion/physical-plan/src/aggregates/topk/priority_map.rs b/datafusion/physical-plan/src/aggregates/topk/priority_map.rs index 668018b9c24c..ed41d22e935b 100644 --- a/datafusion/physical-plan/src/aggregates/topk/priority_map.rs +++ b/datafusion/physical-plan/src/aggregates/topk/priority_map.rs @@ -25,17 +25,12 @@ use datafusion_common::Result; /// A `Map` / `PriorityQueue` combo that evicts the worst values after reaching `capacity` pub struct PriorityMap { - map: Box, - heap: Box, + map: Box, + heap: Box, capacity: usize, mapper: Vec<(usize, usize)>, } -// JUSTIFICATION -// Benefit: ~15% speedup + required to index into RawTable from binary heap -// Soundness: it is only accessed by one thread at a time, and indexes are kept up to date -unsafe impl Send for PriorityMap {} - impl PriorityMap { pub fn new( key_type: DataType, From 6bceeae6c622485c8cfaae34eb60152f5c7cbe01 Mon Sep 17 00:00:00 2001 From: Eduard Karacharov Date: Tue, 3 Sep 2024 20:03:24 +0300 Subject: [PATCH 077/154] test: check record count and types in parquet window test (#12277) * test: check record count and types in paqruet window test * Review suggestions Co-authored-by: Andrew Lamb --------- Co-authored-by: Andrew Lamb --- .../sqllogictest/test_files/parquet.slt | 32 ++++++++----------- 1 file changed, 14 insertions(+), 18 deletions(-) diff --git a/datafusion/sqllogictest/test_files/parquet.slt b/datafusion/sqllogictest/test_files/parquet.slt index 34d4ed6ff284..9a7b085312bb 100644 --- a/datafusion/sqllogictest/test_files/parquet.slt +++ b/datafusion/sqllogictest/test_files/parquet.slt @@ -251,25 +251,21 @@ SELECT COUNT(*) FROM timestamp_with_tz; ---- 131072 -# Perform the query: -query IPT -SELECT - count, - LAG(timestamp, 1) OVER (ORDER BY timestamp), - arrow_typeof(LAG(timestamp, 1) OVER (ORDER BY timestamp)) -FROM timestamp_with_tz -LIMIT 10; +# Ensure that output timestamp columns preserve the timezone from the input +# and output record count will match input file record count +query TPI +SELECT arrow_typeof(lag_timestamp), + MIN(lag_timestamp), + COUNT(1) +FROM ( + SELECT + count, + LAG(timestamp, 1) OVER (ORDER BY timestamp) AS lag_timestamp + FROM timestamp_with_tz +) t +GROUP BY 1 ---- -0 NULL Timestamp(Millisecond, Some("UTC")) -0 2014-08-27T14:00:00Z Timestamp(Millisecond, Some("UTC")) -0 2014-08-27T14:00:00Z Timestamp(Millisecond, Some("UTC")) -4 2014-08-27T14:00:00Z Timestamp(Millisecond, Some("UTC")) -0 2014-08-27T14:00:00Z Timestamp(Millisecond, Some("UTC")) -0 2014-08-27T14:00:00Z Timestamp(Millisecond, Some("UTC")) -0 2014-08-27T14:00:00Z Timestamp(Millisecond, Some("UTC")) -14 2014-08-27T14:00:00Z Timestamp(Millisecond, Some("UTC")) -0 2014-08-27T14:00:00Z Timestamp(Millisecond, Some("UTC")) -0 2014-08-27T14:00:00Z Timestamp(Millisecond, Some("UTC")) +Timestamp(Millisecond, Some("UTC")) 2014-08-27T14:00:00Z 131072 # Test config listing_table_ignore_subdirectory: From bf6c82fc18a8243380b8302f9b266344b6955656 Mon Sep 17 00:00:00 2001 From: Alexander Rafferty Date: Wed, 4 Sep 2024 06:12:04 +1000 Subject: [PATCH 078/154] Optimize `struct` and `named_struct` functions (#11688) * Remove unnecessary heap allocations in implementation of `named_struct_expr` caused by zipping then unzipping fields and values. * Change implementation of `array_struct` to reduce number of allocations * Remove tests already covered by `struct.slt` --- datafusion/functions/src/core/named_struct.rs | 21 +++--- datafusion/functions/src/core/struct.rs | 68 ++++--------------- 2 files changed, 21 insertions(+), 68 deletions(-) diff --git a/datafusion/functions/src/core/named_struct.rs b/datafusion/functions/src/core/named_struct.rs index f71b1b00f0fe..85c332745355 100644 --- a/datafusion/functions/src/core/named_struct.rs +++ b/datafusion/functions/src/core/named_struct.rs @@ -70,20 +70,17 @@ fn named_struct_expr(args: &[ColumnarValue]) -> Result { } } - let arrays = ColumnarValue::values_to_arrays(&values)?; - - let fields = names + let fields: Fields = names .into_iter() - .zip(arrays) - .map(|(name, value)| { - ( - Arc::new(Field::new(name, value.data_type().clone(), true)), - value, - ) - }) - .collect::>(); + .zip(&values) + .map(|(name, value)| Arc::new(Field::new(name, value.data_type().clone(), true))) + .collect::>() + .into(); + + let arrays = ColumnarValue::values_to_arrays(&values)?; - Ok(ColumnarValue::Array(Arc::new(StructArray::from(fields)))) + let struct_array = StructArray::new(fields, arrays, None); + Ok(ColumnarValue::Array(Arc::new(struct_array))) } #[derive(Debug)] diff --git a/datafusion/functions/src/core/struct.rs b/datafusion/functions/src/core/struct.rs index c3dee8b1ccb4..bdddbb81beab 100644 --- a/datafusion/functions/src/core/struct.rs +++ b/datafusion/functions/src/core/struct.rs @@ -29,23 +29,23 @@ fn array_struct(args: &[ArrayRef]) -> Result { return exec_err!("struct requires at least one argument"); } - let vec: Vec<_> = args + let fields = args .iter() .enumerate() .map(|(i, arg)| { let field_name = format!("c{i}"); - Ok(( - Arc::new(Field::new( - field_name.as_str(), - arg.data_type().clone(), - true, - )), - Arc::clone(arg), - )) + Ok(Arc::new(Field::new( + field_name.as_str(), + arg.data_type().clone(), + true, + ))) }) - .collect::>>()?; + .collect::>>()? + .into(); - Ok(Arc::new(StructArray::from(vec))) + let arrays = args.to_vec(); + + Ok(Arc::new(StructArray::new(fields, arrays, None))) } /// put values in a struct array. @@ -53,6 +53,7 @@ fn struct_expr(args: &[ColumnarValue]) -> Result { let arrays = ColumnarValue::values_to_arrays(args)?; Ok(ColumnarValue::Array(array_struct(arrays.as_slice())?)) } + #[derive(Debug)] pub struct StructFunc { signature: Signature, @@ -97,48 +98,3 @@ impl ScalarUDFImpl for StructFunc { struct_expr(args) } } - -#[cfg(test)] -mod tests { - use super::*; - use arrow::array::Int64Array; - use datafusion_common::cast::as_struct_array; - use datafusion_common::ScalarValue; - - #[test] - fn test_struct() { - // struct(1, 2, 3) = {"c0": 1, "c1": 2, "c2": 3} - let args = [ - ColumnarValue::Scalar(ScalarValue::Int64(Some(1))), - ColumnarValue::Scalar(ScalarValue::Int64(Some(2))), - ColumnarValue::Scalar(ScalarValue::Int64(Some(3))), - ]; - let struc = struct_expr(&args) - .expect("failed to initialize function struct") - .into_array(1) - .expect("Failed to convert to array"); - let result = - as_struct_array(&struc).expect("failed to initialize function struct"); - assert_eq!( - &Int64Array::from(vec![1]), - Arc::clone(result.column_by_name("c0").unwrap()) - .as_any() - .downcast_ref::() - .unwrap() - ); - assert_eq!( - &Int64Array::from(vec![2]), - Arc::clone(result.column_by_name("c1").unwrap()) - .as_any() - .downcast_ref::() - .unwrap() - ); - assert_eq!( - &Int64Array::from(vec![3]), - Arc::clone(result.column_by_name("c2").unwrap()) - .as_any() - .downcast_ref::() - .unwrap() - ); - } -} From e4a94243b502da2ad07a358b4401052651952eea Mon Sep 17 00:00:00 2001 From: WeblWabl Date: Tue, 3 Sep 2024 15:12:47 -0500 Subject: [PATCH 079/154] Update the CONCAT scalar function to support Utf8View (#12224) * wip * feat: Update the CONCAT scalar function to support Utf8View * fmt * fmt and add default return type for concat * fix clippy lint Signed-off-by: Devan * fmt Signed-off-by: Devan * add more tests for sqllogic Signed-off-by: Devan * make sure no casting with LargeUtf8 * fixing utf8large * fix large utf8 Signed-off-by: Devan * fix large utf8 Signed-off-by: Devan * add test Signed-off-by: Devan * fmt Signed-off-by: Devan * make it so Utf8View just returns Utf8 Signed-off-by: Devan * wip -- trying to build a stringview with columnar refs Signed-off-by: Devan * built stringview builder but it does allocate a new String each iter :( Signed-off-by: Devan * add some testing Signed-off-by: Devan * clippy Signed-off-by: Devan --------- Signed-off-by: Devan --- datafusion/functions/src/string/common.rs | 195 +++++++++++++++++- datafusion/functions/src/string/concat.rs | 184 ++++++++++++++--- .../sqllogictest/test_files/string_view.slt | 71 ++++++- 3 files changed, 416 insertions(+), 34 deletions(-) diff --git a/datafusion/functions/src/string/common.rs b/datafusion/functions/src/string/common.rs index 9738cb812f90..6ebcc4ee6cd3 100644 --- a/datafusion/functions/src/string/common.rs +++ b/datafusion/functions/src/string/common.rs @@ -22,12 +22,11 @@ use std::sync::Arc; use arrow::array::{ new_null_array, Array, ArrayAccessor, ArrayDataBuilder, ArrayIter, ArrayRef, - GenericStringArray, GenericStringBuilder, OffsetSizeTrait, StringArray, - StringBuilder, StringViewArray, + GenericStringArray, GenericStringBuilder, LargeStringArray, OffsetSizeTrait, + StringArray, StringBuilder, StringViewArray, StringViewBuilder, }; use arrow::buffer::{Buffer, MutableBuffer, NullBuffer}; use arrow::datatypes::DataType; - use datafusion_common::cast::{as_generic_string_array, as_string_view_array}; use datafusion_common::Result; use datafusion_common::{exec_err, ScalarValue}; @@ -249,26 +248,41 @@ where } } +#[derive(Debug)] pub(crate) enum ColumnarValueRef<'a> { Scalar(&'a [u8]), NullableArray(&'a StringArray), NonNullableArray(&'a StringArray), + NullableLargeStringArray(&'a LargeStringArray), + NonNullableLargeStringArray(&'a LargeStringArray), + NullableStringViewArray(&'a StringViewArray), + NonNullableStringViewArray(&'a StringViewArray), } impl<'a> ColumnarValueRef<'a> { #[inline] pub fn is_valid(&self, i: usize) -> bool { match &self { - Self::Scalar(_) | Self::NonNullableArray(_) => true, + Self::Scalar(_) + | Self::NonNullableArray(_) + | Self::NonNullableLargeStringArray(_) + | Self::NonNullableStringViewArray(_) => true, Self::NullableArray(array) => array.is_valid(i), + Self::NullableStringViewArray(array) => array.is_valid(i), + Self::NullableLargeStringArray(array) => array.is_valid(i), } } #[inline] pub fn nulls(&self) -> Option { match &self { - Self::Scalar(_) | Self::NonNullableArray(_) => None, + Self::Scalar(_) + | Self::NonNullableArray(_) + | Self::NonNullableStringViewArray(_) + | Self::NonNullableLargeStringArray(_) => None, Self::NullableArray(array) => array.nulls().cloned(), + Self::NullableStringViewArray(array) => array.nulls().cloned(), + Self::NullableLargeStringArray(array) => array.nulls().cloned(), } } } @@ -387,10 +401,30 @@ impl StringArrayBuilder { .extend_from_slice(array.value(i).as_bytes()); } } + ColumnarValueRef::NullableLargeStringArray(array) => { + if !CHECK_VALID || array.is_valid(i) { + self.value_buffer + .extend_from_slice(array.value(i).as_bytes()); + } + } + ColumnarValueRef::NullableStringViewArray(array) => { + if !CHECK_VALID || array.is_valid(i) { + self.value_buffer + .extend_from_slice(array.value(i).as_bytes()); + } + } ColumnarValueRef::NonNullableArray(array) => { self.value_buffer .extend_from_slice(array.value(i).as_bytes()); } + ColumnarValueRef::NonNullableLargeStringArray(array) => { + self.value_buffer + .extend_from_slice(array.value(i).as_bytes()); + } + ColumnarValueRef::NonNullableStringViewArray(array) => { + self.value_buffer + .extend_from_slice(array.value(i).as_bytes()); + } } } @@ -416,6 +450,157 @@ impl StringArrayBuilder { } } +pub(crate) struct StringViewArrayBuilder { + builder: StringViewBuilder, + block: String, +} + +impl StringViewArrayBuilder { + pub fn with_capacity(_item_capacity: usize, data_capacity: usize) -> Self { + let builder = StringViewBuilder::with_capacity(data_capacity); + Self { + builder, + block: String::new(), + } + } + + pub fn write( + &mut self, + column: &ColumnarValueRef, + i: usize, + ) { + match column { + ColumnarValueRef::Scalar(s) => { + self.block.push_str(std::str::from_utf8(s).unwrap()); + } + ColumnarValueRef::NullableArray(array) => { + if !CHECK_VALID || array.is_valid(i) { + self.block.push_str( + std::str::from_utf8(array.value(i).as_bytes()).unwrap(), + ); + } + } + ColumnarValueRef::NullableLargeStringArray(array) => { + if !CHECK_VALID || array.is_valid(i) { + self.block.push_str( + std::str::from_utf8(array.value(i).as_bytes()).unwrap(), + ); + } + } + ColumnarValueRef::NullableStringViewArray(array) => { + if !CHECK_VALID || array.is_valid(i) { + self.block.push_str( + std::str::from_utf8(array.value(i).as_bytes()).unwrap(), + ); + } + } + ColumnarValueRef::NonNullableArray(array) => { + self.block + .push_str(std::str::from_utf8(array.value(i).as_bytes()).unwrap()); + } + ColumnarValueRef::NonNullableLargeStringArray(array) => { + self.block + .push_str(std::str::from_utf8(array.value(i).as_bytes()).unwrap()); + } + ColumnarValueRef::NonNullableStringViewArray(array) => { + self.block + .push_str(std::str::from_utf8(array.value(i).as_bytes()).unwrap()); + } + } + } + + pub fn append_offset(&mut self) { + self.builder.append_value(&self.block); + self.block = String::new(); + } + + pub fn finish(mut self) -> StringViewArray { + self.builder.finish() + } +} + +pub(crate) struct LargeStringArrayBuilder { + offsets_buffer: MutableBuffer, + value_buffer: MutableBuffer, +} + +impl LargeStringArrayBuilder { + pub fn with_capacity(item_capacity: usize, data_capacity: usize) -> Self { + let mut offsets_buffer = MutableBuffer::with_capacity( + (item_capacity + 1) * std::mem::size_of::(), + ); + // SAFETY: the first offset value is definitely not going to exceed the bounds. + unsafe { offsets_buffer.push_unchecked(0_i64) }; + Self { + offsets_buffer, + value_buffer: MutableBuffer::with_capacity(data_capacity), + } + } + + pub fn write( + &mut self, + column: &ColumnarValueRef, + i: usize, + ) { + match column { + ColumnarValueRef::Scalar(s) => { + self.value_buffer.extend_from_slice(s); + } + ColumnarValueRef::NullableArray(array) => { + if !CHECK_VALID || array.is_valid(i) { + self.value_buffer + .extend_from_slice(array.value(i).as_bytes()); + } + } + ColumnarValueRef::NullableLargeStringArray(array) => { + if !CHECK_VALID || array.is_valid(i) { + self.value_buffer + .extend_from_slice(array.value(i).as_bytes()); + } + } + ColumnarValueRef::NullableStringViewArray(array) => { + if !CHECK_VALID || array.is_valid(i) { + self.value_buffer + .extend_from_slice(array.value(i).as_bytes()); + } + } + ColumnarValueRef::NonNullableArray(array) => { + self.value_buffer + .extend_from_slice(array.value(i).as_bytes()); + } + ColumnarValueRef::NonNullableLargeStringArray(array) => { + self.value_buffer + .extend_from_slice(array.value(i).as_bytes()); + } + ColumnarValueRef::NonNullableStringViewArray(array) => { + self.value_buffer + .extend_from_slice(array.value(i).as_bytes()); + } + } + } + + pub fn append_offset(&mut self) { + let next_offset: i64 = self + .value_buffer + .len() + .try_into() + .expect("byte array offset overflow"); + unsafe { self.offsets_buffer.push_unchecked(next_offset) }; + } + + pub fn finish(self, null_buffer: Option) -> LargeStringArray { + let array_builder = ArrayDataBuilder::new(DataType::LargeUtf8) + .len(self.offsets_buffer.len() / std::mem::size_of::() - 1) + .add_buffer(self.offsets_buffer.into()) + .add_buffer(self.value_buffer.into()) + .nulls(null_buffer); + // SAFETY: all data that was appended was valid Large UTF8 and the values + // and offsets were created correctly + let array_data = unsafe { array_builder.build_unchecked() }; + LargeStringArray::from(array_data) + } +} + fn case_conversion_array<'a, O, F>(array: &'a ArrayRef, op: F) -> Result where O: OffsetSizeTrait, diff --git a/datafusion/functions/src/string/concat.rs b/datafusion/functions/src/string/concat.rs index 6d15e2206721..00fe69b0bd33 100644 --- a/datafusion/functions/src/string/concat.rs +++ b/datafusion/functions/src/string/concat.rs @@ -15,14 +15,13 @@ // specific language governing permissions and limitations // under the License. +use arrow::array::{as_largestring_array, Array}; +use arrow::datatypes::DataType; use std::any::Any; use std::sync::Arc; -use arrow::datatypes::DataType; -use arrow::datatypes::DataType::Utf8; - -use datafusion_common::cast::as_string_array; -use datafusion_common::{internal_err, Result, ScalarValue}; +use datafusion_common::cast::{as_string_array, as_string_view_array}; +use datafusion_common::{internal_err, plan_err, Result, ScalarValue}; use datafusion_expr::expr::ScalarFunction; use datafusion_expr::simplify::{ExprSimplifyResult, SimplifyInfo}; use datafusion_expr::{lit, ColumnarValue, Expr, Volatility}; @@ -46,7 +45,10 @@ impl ConcatFunc { pub fn new() -> Self { use DataType::*; Self { - signature: Signature::variadic(vec![Utf8], Volatility::Immutable), + signature: Signature::variadic( + vec![Utf8, Utf8View, LargeUtf8], + Volatility::Immutable, + ), } } } @@ -64,13 +66,36 @@ impl ScalarUDFImpl for ConcatFunc { &self.signature } - fn return_type(&self, _arg_types: &[DataType]) -> Result { - Ok(Utf8) + fn return_type(&self, arg_types: &[DataType]) -> Result { + use DataType::*; + let mut dt = &Utf8; + arg_types.iter().for_each(|data_type| { + if data_type == &Utf8View { + dt = data_type; + } + if data_type == &LargeUtf8 && dt != &Utf8View { + dt = data_type; + } + }); + + Ok(dt.to_owned()) } /// Concatenates the text representations of all the arguments. NULL arguments are ignored. /// concat('abcde', 2, NULL, 22) = 'abcde222' fn invoke(&self, args: &[ColumnarValue]) -> Result { + let mut return_datatype = DataType::Utf8; + args.iter().for_each(|col| { + if col.data_type() == DataType::Utf8View { + return_datatype = col.data_type(); + } + if col.data_type() == DataType::LargeUtf8 + && return_datatype != DataType::Utf8View + { + return_datatype = col.data_type(); + } + }); + let array_len = args .iter() .filter_map(|x| match x { @@ -87,7 +112,21 @@ impl ScalarUDFImpl for ConcatFunc { result.push_str(v); } } - return Ok(ColumnarValue::Scalar(ScalarValue::Utf8(Some(result)))); + + return match return_datatype { + DataType::Utf8View => { + Ok(ColumnarValue::Scalar(ScalarValue::Utf8View(Some(result)))) + } + DataType::Utf8 => { + Ok(ColumnarValue::Scalar(ScalarValue::Utf8(Some(result)))) + } + DataType::LargeUtf8 => { + Ok(ColumnarValue::Scalar(ScalarValue::LargeUtf8(Some(result)))) + } + other => { + plan_err!("Concat function does not support datatype of {other}") + } + }; } // Array @@ -103,28 +142,95 @@ impl ScalarUDFImpl for ConcatFunc { columns.push(ColumnarValueRef::Scalar(s.as_bytes())); } } + ColumnarValue::Scalar(ScalarValue::Utf8View(maybe_value)) => { + if let Some(s) = maybe_value { + data_size += s.len() * len; + columns.push(ColumnarValueRef::Scalar(s.as_bytes())); + } + } ColumnarValue::Array(array) => { - let string_array = as_string_array(array)?; - data_size += string_array.values().len(); - let column = if array.is_nullable() { - ColumnarValueRef::NullableArray(string_array) - } else { - ColumnarValueRef::NonNullableArray(string_array) + match array.data_type() { + DataType::Utf8 => { + let string_array = as_string_array(array)?; + + data_size += string_array.values().len(); + let column = if array.is_nullable() { + ColumnarValueRef::NullableArray(string_array) + } else { + ColumnarValueRef::NonNullableArray(string_array) + }; + columns.push(column); + }, + DataType::LargeUtf8 => { + let string_array = as_largestring_array(array); + + data_size += string_array.values().len(); + let column = if array.is_nullable() { + ColumnarValueRef::NullableLargeStringArray(string_array) + } else { + ColumnarValueRef::NonNullableLargeStringArray(string_array) + }; + columns.push(column); + }, + DataType::Utf8View => { + let string_array = as_string_view_array(array)?; + + data_size += string_array.len(); + let column = if array.is_nullable() { + ColumnarValueRef::NullableStringViewArray(string_array) + } else { + ColumnarValueRef::NonNullableStringViewArray(string_array) + }; + columns.push(column); + }, + other => { + return plan_err!("Input was {other} which is not a supported datatype for concat function") + } }; - columns.push(column); } _ => unreachable!(), } } - let mut builder = StringArrayBuilder::with_capacity(len, data_size); - for i in 0..len { - columns - .iter() - .for_each(|column| builder.write::(column, i)); - builder.append_offset(); + match return_datatype { + DataType::Utf8 => { + let mut builder = StringArrayBuilder::with_capacity(len, data_size); + for i in 0..len { + columns + .iter() + .for_each(|column| builder.write::(column, i)); + builder.append_offset(); + } + + let string_array = builder.finish(None); + Ok(ColumnarValue::Array(Arc::new(string_array))) + } + DataType::Utf8View => { + let mut builder = StringViewArrayBuilder::with_capacity(len, data_size); + for i in 0..len { + columns + .iter() + .for_each(|column| builder.write::(column, i)); + builder.append_offset(); + } + + let string_array = builder.finish(); + Ok(ColumnarValue::Array(Arc::new(string_array))) + } + DataType::LargeUtf8 => { + let mut builder = LargeStringArrayBuilder::with_capacity(len, data_size); + for i in 0..len { + columns + .iter() + .for_each(|column| builder.write::(column, i)); + builder.append_offset(); + } + + let string_array = builder.finish(None); + Ok(ColumnarValue::Array(Arc::new(string_array))) + } + _ => unreachable!(), } - Ok(ColumnarValue::Array(Arc::new(builder.finish(None)))) } /// Simplify the `concat` function by @@ -151,11 +257,11 @@ pub fn simplify_concat(args: Vec) -> Result { for arg in args.clone() { match arg { // filter out `null` args - Expr::Literal(ScalarValue::Utf8(None) | ScalarValue::LargeUtf8(None)) => {} + Expr::Literal(ScalarValue::Utf8(None) | ScalarValue::LargeUtf8(None) | ScalarValue::Utf8View(None)) => {} // All literals have been converted to Utf8 or LargeUtf8 in type_coercion. // Concatenate it with the `contiguous_scalar`. Expr::Literal( - ScalarValue::Utf8(Some(v)) | ScalarValue::LargeUtf8(Some(v)), + ScalarValue::Utf8(Some(v)) | ScalarValue::LargeUtf8(Some(v)) | ScalarValue::Utf8View(Some(v)), ) => contiguous_scalar += &v, Expr::Literal(x) => { return internal_err!( @@ -195,8 +301,9 @@ pub fn simplify_concat(args: Vec) -> Result { mod tests { use super::*; use crate::utils::test::test_function; - use arrow::array::Array; + use arrow::array::{Array, LargeStringArray, StringViewArray}; use arrow::array::{ArrayRef, StringArray}; + use DataType::*; #[test] fn test_functions() -> Result<()> { @@ -232,6 +339,31 @@ mod tests { Utf8, StringArray ); + test_function!( + ConcatFunc::new(), + &[ + ColumnarValue::Scalar(ScalarValue::from("aa")), + ColumnarValue::Scalar(ScalarValue::Utf8View(None)), + ColumnarValue::Scalar(ScalarValue::LargeUtf8(None)), + ColumnarValue::Scalar(ScalarValue::from("cc")), + ], + Ok(Some("aacc")), + &str, + Utf8View, + StringViewArray + ); + test_function!( + ConcatFunc::new(), + &[ + ColumnarValue::Scalar(ScalarValue::from("aa")), + ColumnarValue::Scalar(ScalarValue::LargeUtf8(None)), + ColumnarValue::Scalar(ScalarValue::from("cc")), + ], + Ok(Some("aacc")), + &str, + LargeUtf8, + LargeStringArray + ); Ok(()) } diff --git a/datafusion/sqllogictest/test_files/string_view.slt b/datafusion/sqllogictest/test_files/string_view.slt index 83c75b8df38c..eb625e530b66 100644 --- a/datafusion/sqllogictest/test_files/string_view.slt +++ b/datafusion/sqllogictest/test_files/string_view.slt @@ -768,17 +768,26 @@ logical_plan 01)Projection: character_length(test.column1_utf8view) AS l 02)--TableScan: test projection=[column1_utf8view] -## Ensure no casts for CONCAT -## TODO https://github.com/apache/datafusion/issues/11836 +## Ensure no casts for CONCAT Utf8View query TT EXPLAIN SELECT concat(column1_utf8view, column2_utf8view) as c FROM test; ---- logical_plan -01)Projection: concat(CAST(test.column1_utf8view AS Utf8), CAST(test.column2_utf8view AS Utf8)) AS c +01)Projection: concat(test.column1_utf8view, test.column2_utf8view) AS c 02)--TableScan: test projection=[column1_utf8view, column2_utf8view] +## Ensure no casts for CONCAT LargeUtf8 +query TT +EXPLAIN SELECT + concat(column1_large_utf8, column2_large_utf8) as c +FROM test; +---- +logical_plan +01)Projection: concat(test.column1_large_utf8, test.column2_large_utf8) AS c +02)--TableScan: test projection=[column1_large_utf8, column2_large_utf8] + ## Ensure no casts for CONCAT_WS ## TODO https://github.com/apache/datafusion/issues/11837 query TT @@ -863,6 +872,61 @@ XIANGPENG RAPHAEL NULL +## Should run CONCAT successfully with utf8view +query T +SELECT + concat(column1_utf8view, column2_utf8view) as c +FROM test; +---- +AndrewX +XiangpengXiangpeng +RaphaelR +R + +## Should run CONCAT successfully with utf8 +query T +SELECT + concat(column1_utf8, column2_utf8) as c +FROM test; +---- +AndrewX +XiangpengXiangpeng +RaphaelR +R + +## Should run CONCAT successfully with utf8 and utf8view +query T +SELECT + concat(column1_utf8view, column2_utf8) as c +FROM test; +---- +AndrewX +XiangpengXiangpeng +RaphaelR +R + +## Should run CONCAT successfully with utf8 utf8view and largeutf8 +query T +SELECT + concat(column1_utf8view, column2_utf8, column2_large_utf8) as c +FROM test; +---- +AndrewXX +XiangpengXiangpengXiangpeng +RaphaelRR +RR + +## Should run CONCAT successfully with utf8large +query T +SELECT + concat(column1_large_utf8, column2_large_utf8) as c +FROM test; +---- +AndrewX +XiangpengXiangpeng +RaphaelR +R + ## Ensure no casts for LPAD query TT EXPLAIN SELECT @@ -1307,3 +1371,4 @@ select column2|| ' ' ||column3 from temp; ---- rust fast datafusion cool + From 169fc1ef01e83393e9fde52519e820838592b96c Mon Sep 17 00:00:00 2001 From: Emil Ejbyfeldt Date: Tue, 3 Sep 2024 22:57:45 +0200 Subject: [PATCH 080/154] chore: Fix warnings produced by shellcheck on bench.sh (#12303) --- benchmarks/bench.sh | 38 +++++++++++++++++++------------------- 1 file changed, 19 insertions(+), 19 deletions(-) diff --git a/benchmarks/bench.sh b/benchmarks/bench.sh index 3b854f6dc147..c02b08576eaa 100755 --- a/benchmarks/bench.sh +++ b/benchmarks/bench.sh @@ -77,7 +77,7 @@ parquet: Benchmark of parquet reader's filtering speed sort: Benchmark of sorting speed clickbench_1: ClickBench queries against a single parquet file clickbench_partitioned: ClickBench queries against a partitioned (100 files) parquet -clickbench_extended: ClickBench "inspired" queries against a single parquet (DataFusion specific) +clickbench_extended: ClickBench \"inspired\" queries against a single parquet (DataFusion specific) ********** * Supported Configuration (Environment Variables) @@ -106,7 +106,7 @@ while [[ $# -gt 0 ]]; do shift # past argument usage ;; - -*|--*) + -*) echo "Unknown option $1" exit 1 ;; @@ -175,7 +175,7 @@ main() { run) # Parse positional parameters BENCHMARK=${ARG2:-"${BENCHMARK}"} - BRANCH_NAME=$(cd ${DATAFUSION_DIR} && git rev-parse --abbrev-ref HEAD) + BRANCH_NAME=$(cd "${DATAFUSION_DIR}" && git rev-parse --abbrev-ref HEAD) BRANCH_NAME=${BRANCH_NAME//\//_} # mind blowing syntax to replace / with _ RESULTS_NAME=${RESULTS_NAME:-"${BRANCH_NAME}"} RESULTS_DIR=${RESULTS_DIR:-"$SCRIPT_DIR/results/$RESULTS_NAME"} @@ -189,7 +189,7 @@ main() { echo "DATA_DIR: ${DATA_DIR}" echo "RESULTS_DIR: ${RESULTS_DIR}" echo "CARGO_COMMAND: ${CARGO_COMMAND}" - echo "PREFER_HASH_JOIN": ${PREFER_HASH_JOIN} + echo "PREFER_HASH_JOIN: ${PREFER_HASH_JOIN}" echo "***************************" # navigate to the appropriate directory @@ -288,7 +288,7 @@ data_tpch() { echo " tbl files exist ($FILE exists)." else echo " creating tbl files with tpch_dbgen..." - docker run -v "${TPCH_DIR}":/data -it --rm ghcr.io/scalytics/tpch-docker:main -vf -s ${SCALE_FACTOR} + docker run -v "${TPCH_DIR}":/data -it --rm ghcr.io/scalytics/tpch-docker:main -vf -s "${SCALE_FACTOR}" fi # Copy expected answers into the ./data/answers directory if it does not already exist @@ -325,7 +325,7 @@ run_tpch() { RESULTS_FILE="${RESULTS_DIR}/tpch_sf${SCALE_FACTOR}.json" echo "RESULTS_FILE: ${RESULTS_FILE}" echo "Running tpch benchmark..." - $CARGO_COMMAND --bin tpch -- benchmark datafusion --iterations 5 --path "${TPCH_DIR}" --prefer_hash_join ${PREFER_HASH_JOIN} --format parquet -o ${RESULTS_FILE} + $CARGO_COMMAND --bin tpch -- benchmark datafusion --iterations 5 --path "${TPCH_DIR}" --prefer_hash_join "${PREFER_HASH_JOIN}" --format parquet -o "${RESULTS_FILE}" } # Runs the tpch in memory @@ -341,7 +341,7 @@ run_tpch_mem() { echo "RESULTS_FILE: ${RESULTS_FILE}" echo "Running tpch_mem benchmark..." # -m means in memory - $CARGO_COMMAND --bin tpch -- benchmark datafusion --iterations 5 --path "${TPCH_DIR}" --prefer_hash_join ${PREFER_HASH_JOIN} -m --format parquet -o ${RESULTS_FILE} + $CARGO_COMMAND --bin tpch -- benchmark datafusion --iterations 5 --path "${TPCH_DIR}" --prefer_hash_join "${PREFER_HASH_JOIN}" -m --format parquet -o "${RESULTS_FILE}" } # Runs the parquet filter benchmark @@ -349,7 +349,7 @@ run_parquet() { RESULTS_FILE="${RESULTS_DIR}/parquet.json" echo "RESULTS_FILE: ${RESULTS_FILE}" echo "Running parquet filter benchmark..." - $CARGO_COMMAND --bin parquet -- filter --path "${DATA_DIR}" --prefer_hash_join ${PREFER_HASH_JOIN} --scale-factor 1.0 --iterations 5 -o ${RESULTS_FILE} + $CARGO_COMMAND --bin parquet -- filter --path "${DATA_DIR}" --prefer_hash_join "${PREFER_HASH_JOIN}" --scale-factor 1.0 --iterations 5 -o "${RESULTS_FILE}" } # Runs the sort benchmark @@ -357,7 +357,7 @@ run_sort() { RESULTS_FILE="${RESULTS_DIR}/sort.json" echo "RESULTS_FILE: ${RESULTS_FILE}" echo "Running sort benchmark..." - $CARGO_COMMAND --bin parquet -- sort --path "${DATA_DIR}" --prefer_hash_join ${PREFER_HASH_JOIN} --scale-factor 1.0 --iterations 5 -o ${RESULTS_FILE} + $CARGO_COMMAND --bin parquet -- sort --path "${DATA_DIR}" --prefer_hash_join "${PREFER_HASH_JOIN}" --scale-factor 1.0 --iterations 5 -o "${RESULTS_FILE}" } @@ -369,7 +369,7 @@ data_clickbench_1() { pushd "${DATA_DIR}" > /dev/null # Avoid downloading if it already exists and is the right size - OUTPUT_SIZE=`wc -c hits.parquet 2>/dev/null | awk '{print $1}' || true` + OUTPUT_SIZE=$(wc -c hits.parquet 2>/dev/null | awk '{print $1}' || true) echo -n "Checking hits.parquet..." if test "${OUTPUT_SIZE}" = "14779976446"; then echo -n "... found ${OUTPUT_SIZE} bytes ..." @@ -393,7 +393,7 @@ data_clickbench_partitioned() { pushd "${DATA_DIR}/hits_partitioned" > /dev/null echo -n "Checking hits_partitioned..." - OUTPUT_SIZE=`wc -c * 2>/dev/null | tail -n 1 | awk '{print $1}' || true` + OUTPUT_SIZE=$(wc -c -- * 2>/dev/null | tail -n 1 | awk '{print $1}' || true) if test "${OUTPUT_SIZE}" = "14737666736"; then echo -n "... found ${OUTPUT_SIZE} bytes ..." else @@ -411,7 +411,7 @@ run_clickbench_1() { RESULTS_FILE="${RESULTS_DIR}/clickbench_1.json" echo "RESULTS_FILE: ${RESULTS_FILE}" echo "Running clickbench (1 file) benchmark..." - $CARGO_COMMAND --bin dfbench -- clickbench --iterations 5 --path "${DATA_DIR}/hits.parquet" --queries-path "${SCRIPT_DIR}/queries/clickbench/queries.sql" -o ${RESULTS_FILE} + $CARGO_COMMAND --bin dfbench -- clickbench --iterations 5 --path "${DATA_DIR}/hits.parquet" --queries-path "${SCRIPT_DIR}/queries/clickbench/queries.sql" -o "${RESULTS_FILE}" } # Runs the clickbench benchmark with the partitioned parquet files @@ -419,7 +419,7 @@ run_clickbench_partitioned() { RESULTS_FILE="${RESULTS_DIR}/clickbench_partitioned.json" echo "RESULTS_FILE: ${RESULTS_FILE}" echo "Running clickbench (partitioned, 100 files) benchmark..." - $CARGO_COMMAND --bin dfbench -- clickbench --iterations 5 --path "${DATA_DIR}/hits_partitioned" --queries-path "${SCRIPT_DIR}/queries/clickbench/queries.sql" -o ${RESULTS_FILE} + $CARGO_COMMAND --bin dfbench -- clickbench --iterations 5 --path "${DATA_DIR}/hits_partitioned" --queries-path "${SCRIPT_DIR}/queries/clickbench/queries.sql" -o "${RESULTS_FILE}" } # Runs the clickbench "extended" benchmark with a single large parquet file @@ -427,7 +427,7 @@ run_clickbench_extended() { RESULTS_FILE="${RESULTS_DIR}/clickbench_extended.json" echo "RESULTS_FILE: ${RESULTS_FILE}" echo "Running clickbench (1 file) extended benchmark..." - $CARGO_COMMAND --bin dfbench -- clickbench --iterations 5 --path "${DATA_DIR}/hits.parquet" --queries-path "${SCRIPT_DIR}/queries/clickbench/extended.sql" -o ${RESULTS_FILE} + $CARGO_COMMAND --bin dfbench -- clickbench --iterations 5 --path "${DATA_DIR}/hits.parquet" --queries-path "${SCRIPT_DIR}/queries/clickbench/extended.sql" -o "${RESULTS_FILE}" } compare_benchmarks() { @@ -447,12 +447,12 @@ compare_benchmarks() { fi echo "Comparing ${BRANCH1} and ${BRANCH2}" - for bench in `ls ${BASE_RESULTS_DIR}/${BRANCH1}` ; do - RESULTS_FILE1="${BASE_RESULTS_DIR}/${BRANCH1}/${bench}" - RESULTS_FILE2="${BASE_RESULTS_DIR}/${BRANCH2}/${bench}" + for RESULTS_FILE1 in "${BASE_RESULTS_DIR}/${BRANCH1}"/*.json ; do + BENCH=$(basename "${RESULTS_FILE1}") + RESULTS_FILE2="${BASE_RESULTS_DIR}/${BRANCH2}/${BENCH}" if test -f "${RESULTS_FILE2}" ; then echo "--------------------" - echo "Benchmark ${bench}" + echo "Benchmark ${BENCH}" echo "--------------------" PATH=$VIRTUAL_ENV/bin:$PATH python3 "${SCRIPT_DIR}"/compare.py "${RESULTS_FILE1}" "${RESULTS_FILE2}" else @@ -463,7 +463,7 @@ compare_benchmarks() { } setup_venv() { - python3 -m venv $VIRTUAL_ENV + python3 -m venv "$VIRTUAL_ENV" PATH=$VIRTUAL_ENV/bin:$PATH python3 -m pip install -r requirements.txt } From 7ca98104fe0238e9a3991b291d98dc0a2210df7e Mon Sep 17 00:00:00 2001 From: Eduard Karacharov Date: Tue, 3 Sep 2024 23:58:40 +0300 Subject: [PATCH 081/154] test: re-enable merge join test with forced collisions (#12276) --- .../test_files/sort_merge_join.slt | 20 +++++++++---------- 1 file changed, 9 insertions(+), 11 deletions(-) diff --git a/datafusion/sqllogictest/test_files/sort_merge_join.slt b/datafusion/sqllogictest/test_files/sort_merge_join.slt index 2c28a5feadba..ebd53e9690fc 100644 --- a/datafusion/sqllogictest/test_files/sort_merge_join.slt +++ b/datafusion/sqllogictest/test_files/sort_merge_join.slt @@ -233,18 +233,16 @@ SELECT * FROM t1 FULL JOIN t2 ON t1_id = t2_id 44 d 4 44 x 3 NULL NULL NULL 55 w 3 -# FIXME(#TODO) fails with feature `force_hash_collisions` -# https://github.com/apache/datafusion/issues/11659 # equijoin_full_and_condition_from_both -# query ITIITI rowsort -# SELECT * FROM t1 FULL JOIN t2 ON t1_id = t2_id AND t2_int <= t1_int -# ---- -# 11 a 1 NULL NULL NULL -# 22 b 2 22 y 1 -# 33 c 3 NULL NULL NULL -# 44 d 4 44 x 3 -# NULL NULL NULL 11 z 3 -# NULL NULL NULL 55 w 3 +query ITIITI rowsort +SELECT * FROM t1 FULL JOIN t2 ON t1_id = t2_id AND t2_int <= t1_int +---- +11 a 1 NULL NULL NULL +22 b 2 22 y 1 +33 c 3 NULL NULL NULL +44 d 4 44 x 3 +NULL NULL NULL 11 z 3 +NULL NULL NULL 55 w 3 statement ok DROP TABLE t1; From 0cd7c25a37ae5d7fa27e5cf49a5aa258d73e9efa Mon Sep 17 00:00:00 2001 From: peasee <98815791+peasee@users.noreply.github.com> Date: Wed, 4 Sep 2024 06:59:19 +1000 Subject: [PATCH 082/154] feat: Add DateFieldExtractStyle::Strftime support for SqliteDialect unparser (#12161) * feat: Add DateFieldExtractStyle::Strftime support for SqliteDialect (#26) * feat: Add DateFieldExtractStyle::Strftime support for SqliteDialect * refactor: Refactor DateFieldExtractStyle if checks into if/match * Fix merge issue --------- Co-authored-by: Andrew Lamb --- datafusion/sql/src/unparser/dialect.rs | 5 ++ datafusion/sql/src/unparser/expr.rs | 97 ++++++++++++++++++++------ 2 files changed, 81 insertions(+), 21 deletions(-) diff --git a/datafusion/sql/src/unparser/dialect.rs b/datafusion/sql/src/unparser/dialect.rs index 706b7633cb90..2a8e61add1d0 100644 --- a/datafusion/sql/src/unparser/dialect.rs +++ b/datafusion/sql/src/unparser/dialect.rs @@ -130,6 +130,7 @@ pub enum IntervalStyle { pub enum DateFieldExtractStyle { DatePart, Extract, + Strftime, } pub struct DefaultDialect {} @@ -213,6 +214,10 @@ impl Dialect for SqliteDialect { Some('`') } + fn date_field_extract_style(&self) -> DateFieldExtractStyle { + DateFieldExtractStyle::Strftime + } + fn date32_cast_dtype(&self) -> sqlparser::ast::DataType { sqlparser::ast::DataType::Text } diff --git a/datafusion/sql/src/unparser/expr.rs b/datafusion/sql/src/unparser/expr.rs index 8ff4773e8e94..fe34d87bfeae 100644 --- a/datafusion/sql/src/unparser/expr.rs +++ b/datafusion/sql/src/unparser/expr.rs @@ -524,30 +524,78 @@ impl Unparser<'_> { _func: &Arc, args: &[Expr], ) -> Option { - if func_name.to_lowercase() == "date_part" - && self.dialect.date_field_extract_style() == DateFieldExtractStyle::Extract - && args.len() == 2 - { - let date_expr = self.expr_to_sql(&args[1]).ok()?; - - if let Expr::Literal(ScalarValue::Utf8(Some(field))) = &args[0] { - let field = match field.to_lowercase().as_str() { - "year" => ast::DateTimeField::Year, - "month" => ast::DateTimeField::Month, - "day" => ast::DateTimeField::Day, - "hour" => ast::DateTimeField::Hour, - "minute" => ast::DateTimeField::Minute, - "second" => ast::DateTimeField::Second, - _ => return None, - }; + if func_name.to_lowercase() == "date_part" { + match (self.dialect.date_field_extract_style(), args.len()) { + (DateFieldExtractStyle::Extract, 2) => { + let date_expr = self.expr_to_sql(&args[1]).ok()?; + + if let Expr::Literal(ScalarValue::Utf8(Some(field))) = &args[0] { + let field = match field.to_lowercase().as_str() { + "year" => ast::DateTimeField::Year, + "month" => ast::DateTimeField::Month, + "day" => ast::DateTimeField::Day, + "hour" => ast::DateTimeField::Hour, + "minute" => ast::DateTimeField::Minute, + "second" => ast::DateTimeField::Second, + _ => return None, + }; + + return Some(ast::Expr::Extract { + field, + expr: Box::new(date_expr), + syntax: ast::ExtractSyntax::From, + }); + } + } + (DateFieldExtractStyle::Strftime, 2) => { + let column = self.expr_to_sql(&args[1]).ok()?; + + if let Expr::Literal(ScalarValue::Utf8(Some(field))) = &args[0] { + let field = match field.to_lowercase().as_str() { + "year" => "%Y", + "month" => "%m", + "day" => "%d", + "hour" => "%H", + "minute" => "%M", + "second" => "%S", + _ => return None, + }; - return Some(ast::Expr::Extract { - field, - expr: Box::new(date_expr), - syntax: ast::ExtractSyntax::From, - }); + return Some(ast::Expr::Function(ast::Function { + name: ast::ObjectName(vec![ast::Ident { + value: "strftime".to_string(), + quote_style: None, + }]), + args: ast::FunctionArguments::List( + ast::FunctionArgumentList { + duplicate_treatment: None, + args: vec![ + ast::FunctionArg::Unnamed( + ast::FunctionArgExpr::Expr(ast::Expr::Value( + ast::Value::SingleQuotedString( + field.to_string(), + ), + )), + ), + ast::FunctionArg::Unnamed( + ast::FunctionArgExpr::Expr(column), + ), + ], + clauses: vec![], + }, + ), + filter: None, + null_treatment: None, + over: None, + within_group: vec![], + parameters: ast::FunctionArguments::None, + })); + } + } + _ => {} // no overrides for DateFieldExtractStyle::DatePart, because it's already a date_part } } + None } @@ -2178,6 +2226,7 @@ mod tests { "YEAR", "EXTRACT(YEAR FROM x)", ), + (DateFieldExtractStyle::Strftime, "YEAR", "strftime('%Y', x)"), ( DateFieldExtractStyle::DatePart, "MONTH", @@ -2188,11 +2237,17 @@ mod tests { "MONTH", "EXTRACT(MONTH FROM x)", ), + ( + DateFieldExtractStyle::Strftime, + "MONTH", + "strftime('%m', x)", + ), ( DateFieldExtractStyle::DatePart, "DAY", "date_part('DAY', x)", ), + (DateFieldExtractStyle::Strftime, "DAY", "strftime('%d', x)"), (DateFieldExtractStyle::Extract, "DAY", "EXTRACT(DAY FROM x)"), ] { let dialect = CustomDialectBuilder::new() From f72622d6fcd27adac0bce5e87cccc83bad215448 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=BC=A0=E6=9E=97=E4=BC=9F?= Date: Wed, 4 Sep 2024 09:19:43 +0800 Subject: [PATCH 083/154] Fix various typos in aggregation doc (#12301) --- datafusion/expr-common/src/groups_accumulator.rs | 4 ++-- datafusion/physical-plan/src/aggregates/mod.rs | 3 ++- datafusion/physical-plan/src/aggregates/order/full.rs | 4 ++-- datafusion/physical-plan/src/aggregates/order/partial.rs | 2 +- datafusion/physical-plan/src/aggregates/row_hash.rs | 2 +- 5 files changed, 8 insertions(+), 7 deletions(-) diff --git a/datafusion/expr-common/src/groups_accumulator.rs b/datafusion/expr-common/src/groups_accumulator.rs index e66b27d073d1..156e21d9ae20 100644 --- a/datafusion/expr-common/src/groups_accumulator.rs +++ b/datafusion/expr-common/src/groups_accumulator.rs @@ -29,7 +29,7 @@ pub enum EmitTo { /// indexes down by `n`. /// /// For example, if `n=10`, group_index `0, 1, ... 9` are emitted - /// and group indexes '`10, 11, 12, ...` become `0, 1, 2, ...`. + /// and group indexes `10, 11, 12, ...` become `0, 1, 2, ...`. First(usize), } @@ -113,7 +113,7 @@ pub trait GroupsAccumulator: Send { /// each group, and `evaluate` will produce that running sum as /// its output for all groups, in group_index order /// - /// If `emit_to`` is [`EmitTo::All`], the accumulator should + /// If `emit_to` is [`EmitTo::All`], the accumulator should /// return all groups and release / reset its internal state /// equivalent to when it was first created. /// diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 764227e5e717..375c6421b0d9 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -1006,7 +1006,8 @@ pub fn get_finer_aggregate_exprs_requirement( Ok(PhysicalSortRequirement::from_sort_exprs(&requirement)) } -/// returns physical expressions for arguments to evaluate against a batch +/// Returns physical expressions for arguments to evaluate against a batch. +/// /// The expressions are different depending on `mode`: /// * Partial: AggregateFunctionExpr::expressions /// * Final: columns of `AggregateFunctionExpr::state_fields()` diff --git a/datafusion/physical-plan/src/aggregates/order/full.rs b/datafusion/physical-plan/src/aggregates/order/full.rs index e86d7677479a..d64c99ba1bee 100644 --- a/datafusion/physical-plan/src/aggregates/order/full.rs +++ b/datafusion/physical-plan/src/aggregates/order/full.rs @@ -21,7 +21,7 @@ use datafusion_expr::EmitTo; /// group keys /// /// When the group values are sorted, as soon as we see group `n+1` we -/// know we will never see any rows for group `n again and thus they +/// know we will never see any rows for group `n` again and thus they /// can be emitted. /// /// For example, given `SUM(amt) GROUP BY id` if the input is sorted @@ -63,7 +63,7 @@ enum State { /// Seen no input yet Start, - /// Data is in progress. `current is the current group for which + /// Data is in progress. `current` is the current group for which /// values are being generated. Can emit `current` - 1 InProgress { current: usize }, diff --git a/datafusion/physical-plan/src/aggregates/order/partial.rs b/datafusion/physical-plan/src/aggregates/order/partial.rs index 73a157f3aa96..2cbe3bbb784e 100644 --- a/datafusion/physical-plan/src/aggregates/order/partial.rs +++ b/datafusion/physical-plan/src/aggregates/order/partial.rs @@ -32,7 +32,7 @@ use std::sync::Arc; /// key and earlier. /// /// For example, given `SUM(amt) GROUP BY id, state` if the input is -/// sorted by `state, when a new value of `state` is seen, all groups +/// sorted by `state`, when a new value of `state` is seen, all groups /// with prior values of `state` can be emitted. /// /// The state is tracked like this: diff --git a/datafusion/physical-plan/src/aggregates/row_hash.rs b/datafusion/physical-plan/src/aggregates/row_hash.rs index d022bb007d9b..c38137994d44 100644 --- a/datafusion/physical-plan/src/aggregates/row_hash.rs +++ b/datafusion/physical-plan/src/aggregates/row_hash.rs @@ -95,7 +95,7 @@ struct SpillState { // ======================================================================== // STATES: // Fields changes during execution. Can be buffer, or state flags that - // influence the exeuction in parent `GroupedHashAggregateStream` + // influence the execution in parent `GroupedHashAggregateStream` // ======================================================================== /// If data has previously been spilled, the locations of the /// spill files (in Arrow IPC format) From 5ff5a6c924c0d3e2f2c2959d1a348be7913431c6 Mon Sep 17 00:00:00 2001 From: Jax Liu Date: Wed, 4 Sep 2024 15:19:38 +0800 Subject: [PATCH 084/154] Implement `kurtosis_pop` UDAF (#12273) * implement kurtosis_pop udaf * add tests * add empty end line * fix MSRV check * fix the null input and enhance tests * refactor the aggregation * address the review comments * add the doc for kurtois_pop * fix the doc style * use coercible signature * remove unused cast --- .../functions-aggregate/src/kurtosis_pop.rs | 190 ++++++++++++++++++ datafusion/functions-aggregate/src/lib.rs | 2 + .../tests/cases/roundtrip_logical_plan.rs | 2 + .../sqllogictest/test_files/aggregate.slt | 61 ++++++ .../user-guide/sql/aggregate_functions.md | 14 ++ 5 files changed, 269 insertions(+) create mode 100644 datafusion/functions-aggregate/src/kurtosis_pop.rs diff --git a/datafusion/functions-aggregate/src/kurtosis_pop.rs b/datafusion/functions-aggregate/src/kurtosis_pop.rs new file mode 100644 index 000000000000..ac173a0ee579 --- /dev/null +++ b/datafusion/functions-aggregate/src/kurtosis_pop.rs @@ -0,0 +1,190 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +use arrow::array::{Array, ArrayRef, Float64Array, UInt64Array}; +use arrow_schema::{DataType, Field}; +use datafusion_common::cast::as_float64_array; +use datafusion_common::{downcast_value, DataFusionError, Result, ScalarValue}; +use datafusion_expr::{Accumulator, AggregateUDFImpl, Signature, Volatility}; +use datafusion_functions_aggregate_common::accumulator::{ + AccumulatorArgs, StateFieldsArgs, +}; +use std::any::Any; +use std::fmt::Debug; + +make_udaf_expr_and_func!( + KurtosisPopFunction, + kurtosis_pop, + x, + "Calculates the excess kurtosis (Fisher’s definition) without bias correction.", + kurtosis_pop_udaf +); + +pub struct KurtosisPopFunction { + signature: Signature, +} + +impl Debug for KurtosisPopFunction { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + f.debug_struct("KurtosisPopFunction") + .field("signature", &self.signature) + .finish() + } +} + +impl Default for KurtosisPopFunction { + fn default() -> Self { + Self::new() + } +} + +impl KurtosisPopFunction { + pub fn new() -> Self { + Self { + signature: Signature::coercible( + vec![DataType::Float64], + Volatility::Immutable, + ), + } + } +} + +impl AggregateUDFImpl for KurtosisPopFunction { + fn as_any(&self) -> &dyn Any { + self + } + + fn name(&self) -> &str { + "kurtosis_pop" + } + + fn signature(&self) -> &Signature { + &self.signature + } + + fn return_type(&self, _arg_types: &[DataType]) -> Result { + Ok(DataType::Float64) + } + + fn state_fields(&self, _args: StateFieldsArgs) -> Result> { + Ok(vec![ + Field::new("count", DataType::UInt64, true), + Field::new("sum", DataType::Float64, true), + Field::new("sum_sqr", DataType::Float64, true), + Field::new("sum_cub", DataType::Float64, true), + Field::new("sum_four", DataType::Float64, true), + ]) + } + + fn accumulator(&self, _acc_args: AccumulatorArgs) -> Result> { + Ok(Box::new(KurtosisPopAccumulator::new())) + } +} + +/// Accumulator for calculating the excess kurtosis (Fisher’s definition) without bias correction. +/// This implementation follows the [DuckDB implementation]: +/// +#[derive(Debug, Default)] +pub struct KurtosisPopAccumulator { + count: u64, + sum: f64, + sum_sqr: f64, + sum_cub: f64, + sum_four: f64, +} + +impl KurtosisPopAccumulator { + pub fn new() -> Self { + Self { + count: 0, + sum: 0.0, + sum_sqr: 0.0, + sum_cub: 0.0, + sum_four: 0.0, + } + } +} + +impl Accumulator for KurtosisPopAccumulator { + fn update_batch(&mut self, values: &[ArrayRef]) -> Result<()> { + let array = as_float64_array(&values[0])?; + for value in array.iter().flatten() { + self.count += 1; + self.sum += value; + self.sum_sqr += value.powi(2); + self.sum_cub += value.powi(3); + self.sum_four += value.powi(4); + } + Ok(()) + } + + fn merge_batch(&mut self, states: &[ArrayRef]) -> Result<()> { + let counts = downcast_value!(states[0], UInt64Array); + let sums = downcast_value!(states[1], Float64Array); + let sum_sqrs = downcast_value!(states[2], Float64Array); + let sum_cubs = downcast_value!(states[3], Float64Array); + let sum_fours = downcast_value!(states[4], Float64Array); + + for i in 0..counts.len() { + let c = counts.value(i); + if c == 0 { + continue; + } + self.count += c; + self.sum += sums.value(i); + self.sum_sqr += sum_sqrs.value(i); + self.sum_cub += sum_cubs.value(i); + self.sum_four += sum_fours.value(i); + } + + Ok(()) + } + + fn evaluate(&mut self) -> Result { + if self.count < 1 { + return Ok(ScalarValue::Float64(None)); + } + + let count_64 = 1_f64 / self.count as f64; + let m4 = count_64 + * (self.sum_four - 4.0 * self.sum_cub * self.sum * count_64 + + 6.0 * self.sum_sqr * self.sum.powi(2) * count_64.powi(2) + - 3.0 * self.sum.powi(4) * count_64.powi(3)); + + let m2 = (self.sum_sqr - self.sum.powi(2) * count_64) * count_64; + if m2 <= 0.0 { + return Ok(ScalarValue::Float64(None)); + } + + let target = m4 / (m2.powi(2)) - 3.0; + Ok(ScalarValue::Float64(Some(target))) + } + + fn size(&self) -> usize { + std::mem::size_of_val(self) + } + + fn state(&mut self) -> Result> { + Ok(vec![ + ScalarValue::from(self.count), + ScalarValue::from(self.sum), + ScalarValue::from(self.sum_sqr), + ScalarValue::from(self.sum_cub), + ScalarValue::from(self.sum_four), + ]) + } +} diff --git a/datafusion/functions-aggregate/src/lib.rs b/datafusion/functions-aggregate/src/lib.rs index ca0276d326a4..60e2602eb6ed 100644 --- a/datafusion/functions-aggregate/src/lib.rs +++ b/datafusion/functions-aggregate/src/lib.rs @@ -78,6 +78,7 @@ pub mod average; pub mod bit_and_or_xor; pub mod bool_and_or; pub mod grouping; +pub mod kurtosis_pop; pub mod nth_value; pub mod string_agg; @@ -170,6 +171,7 @@ pub fn all_default_aggregate_functions() -> Vec> { average::avg_udaf(), grouping::grouping_udaf(), nth_value::nth_value_udaf(), + kurtosis_pop::kurtosis_pop_udaf(), ] } diff --git a/datafusion/proto/tests/cases/roundtrip_logical_plan.rs b/datafusion/proto/tests/cases/roundtrip_logical_plan.rs index 994ed8ad2352..dd3b99b0768b 100644 --- a/datafusion/proto/tests/cases/roundtrip_logical_plan.rs +++ b/datafusion/proto/tests/cases/roundtrip_logical_plan.rs @@ -73,6 +73,7 @@ use datafusion_functions_aggregate::expr_fn::{ approx_distinct, array_agg, avg, bit_and, bit_or, bit_xor, bool_and, bool_or, corr, nth_value, }; +use datafusion_functions_aggregate::kurtosis_pop::kurtosis_pop; use datafusion_functions_aggregate::string_agg::string_agg; use datafusion_proto::bytes::{ logical_plan_from_bytes, logical_plan_from_bytes_with_extension_codec, @@ -904,6 +905,7 @@ async fn roundtrip_expr_api() -> Result<()> { vec![lit(10), lit(20), lit(30)], ), row_number(), + kurtosis_pop(lit(1)), nth_value(col("b"), 1, vec![]), nth_value( col("b"), diff --git a/datafusion/sqllogictest/test_files/aggregate.slt b/datafusion/sqllogictest/test_files/aggregate.slt index 45cb4d4615d7..c52445c561ee 100644 --- a/datafusion/sqllogictest/test_files/aggregate.slt +++ b/datafusion/sqllogictest/test_files/aggregate.slt @@ -5863,3 +5863,64 @@ ORDER BY k; ---- 1 1.8125 6.8007813 Float16 Float16 2 8.5 8.5 Float16 Float16 + +# The result is 0.19432323191699075 actually +query R +SELECT kurtosis_pop(col) FROM VALUES (1), (10), (100), (10), (1) as tab(col); +---- +0.194323231917 + +# The result is -1.153061224489787 actually +query R +SELECT kurtosis_pop(col) FROM VALUES (1), (2), (3), (2), (1) as tab(col); +---- +-1.15306122449 + +query R +SELECT kurtosis_pop(col) FROM VALUES (1.0), (10.0), (100.0), (10.0), (1.0) as tab(col); +---- +0.194323231917 + +query R +SELECT kurtosis_pop(col) FROM VALUES ('1'), ('10'), ('100'), ('10'), ('1') as tab(col); +---- +0.194323231917 + +query R +SELECT kurtosis_pop(col) FROM VALUES (1.0) as tab(col); +---- +NULL + +query R +SELECT kurtosis_pop(1) +---- +NULL + +query R +SELECT kurtosis_pop(1.0) +---- +NULL + +query R +SELECT kurtosis_pop(null) +---- +NULL + +statement ok +CREATE TABLE t1(c1 int); + +query R +SELECT kurtosis_pop(c1) FROM t1; +---- +NULL + +statement ok +INSERT INTO t1 VALUES (1), (10), (100), (10), (1); + +query R +SELECT kurtosis_pop(c1) FROM t1; +---- +0.194323231917 + +statement ok +DROP TABLE t1; diff --git a/docs/source/user-guide/sql/aggregate_functions.md b/docs/source/user-guide/sql/aggregate_functions.md index edb0e1d0c9f0..1c214084b3fa 100644 --- a/docs/source/user-guide/sql/aggregate_functions.md +++ b/docs/source/user-guide/sql/aggregate_functions.md @@ -252,6 +252,7 @@ last_value(expression [ORDER BY expression]) - [regr_sxx](#regr_sxx) - [regr_syy](#regr_syy) - [regr_sxy](#regr_sxy) +- [kurtosis_pop](#kurtosis_pop) ### `corr` @@ -527,6 +528,19 @@ regr_sxy(expression_y, expression_x) - **expression_x**: Independent variable. Can be a constant, column, or function, and any combination of arithmetic operators. +### `kurtosis_pop` + +Computes the excess kurtosis (Fisher’s definition) without bias correction. + +``` +kurtois_pop(expression) +``` + +#### Arguments + +- **expression**: Expression to operate on. + Can be a constant, column, or function, and any combination of arithmetic operators. + ## Approximate - [approx_distinct](#approx_distinct) From 6bbad7eda15cd1adcfb85182f6a20df6b5035842 Mon Sep 17 00:00:00 2001 From: Huaijin Date: Wed, 4 Sep 2024 16:41:11 +0800 Subject: [PATCH 085/154] perf: avoid repeat format in calc_func_dependencies_for_project (#12305) --- datafusion/expr/src/logical_plan/plan.rs | 28 ++++++++++++++---------- 1 file changed, 17 insertions(+), 11 deletions(-) diff --git a/datafusion/expr/src/logical_plan/plan.rs b/datafusion/expr/src/logical_plan/plan.rs index 19be9143fd23..d0c1c3b2b3d6 100644 --- a/datafusion/expr/src/logical_plan/plan.rs +++ b/datafusion/expr/src/logical_plan/plan.rs @@ -2793,22 +2793,28 @@ fn calc_func_dependencies_for_project( .filter_map(|(qualifier, f)| { let flat_name = qualifier .map(|t| format!("{}.{}", t, f.name())) - .unwrap_or(f.name().clone()); + .unwrap_or_else(|| f.name().clone()); input_fields.iter().position(|item| *item == flat_name) }) .collect::>(), ) } - Expr::Alias(alias) => Ok(input_fields - .iter() - .position(|item| *item == format!("{}", alias.expr)) - .map(|i| vec![i]) - .unwrap_or(vec![])), - _ => Ok(input_fields - .iter() - .position(|item| *item == format!("{}", expr)) - .map(|i| vec![i]) - .unwrap_or(vec![])), + Expr::Alias(alias) => { + let name = format!("{}", alias.expr); + Ok(input_fields + .iter() + .position(|item| *item == name) + .map(|i| vec![i]) + .unwrap_or(vec![])) + } + _ => { + let name = format!("{}", expr); + Ok(input_fields + .iter() + .position(|item| *item == name) + .map(|i| vec![i]) + .unwrap_or(vec![])) + } }) .collect::>>()? .into_iter() From 9ab27248240bd798a4e814e4fe05f644c8ea3766 Mon Sep 17 00:00:00 2001 From: Alex Huang Date: Wed, 4 Sep 2024 17:54:29 +0800 Subject: [PATCH 086/154] feat: Enforce the uniqueness of map key name for the map/make_map function (#12153) * feat: Enforce the uniqueness of map key name for the map/make_map function * chore: Update tests * chore * chore: Update tests for nested type * refactor * chore * fix: Check unique key for the make_map function earlier * fix: Update bench * chore: Clean UP --- datafusion/common/src/utils/mod.rs | 7 +- datafusion/functions-nested/benches/map.rs | 20 +++--- datafusion/functions-nested/src/map.rs | 79 ++++++++++++++++------ datafusion/sqllogictest/test_files/map.slt | 39 +++++++++++ 4 files changed, 115 insertions(+), 30 deletions(-) diff --git a/datafusion/common/src/utils/mod.rs b/datafusion/common/src/utils/mod.rs index 418ea380bc2c..83f98ff9aff6 100644 --- a/datafusion/common/src/utils/mod.rs +++ b/datafusion/common/src/utils/mod.rs @@ -444,10 +444,15 @@ pub fn arrays_into_list_array( } /// Helper function to convert a ListArray into a vector of ArrayRefs. -pub fn list_to_arrays(a: ArrayRef) -> Vec { +pub fn list_to_arrays(a: &ArrayRef) -> Vec { a.as_list::().iter().flatten().collect::>() } +/// Helper function to convert a FixedSizeListArray into a vector of ArrayRefs. +pub fn fixed_size_list_to_arrays(a: &ArrayRef) -> Vec { + a.as_fixed_size_list().iter().flatten().collect::>() +} + /// Get the base type of a data type. /// /// Example diff --git a/datafusion/functions-nested/benches/map.rs b/datafusion/functions-nested/benches/map.rs index c9a12eefa4fa..ca23d8b7ff4c 100644 --- a/datafusion/functions-nested/benches/map.rs +++ b/datafusion/functions-nested/benches/map.rs @@ -23,6 +23,7 @@ use arrow_schema::{DataType, Field}; use criterion::{black_box, criterion_group, criterion_main, Criterion}; use rand::prelude::ThreadRng; use rand::Rng; +use std::collections::HashSet; use std::sync::Arc; use datafusion_common::ScalarValue; @@ -32,19 +33,22 @@ use datafusion_functions_nested::map::map_udf; use datafusion_functions_nested::planner::NestedFunctionPlanner; fn keys(rng: &mut ThreadRng) -> Vec { - let mut keys = vec![]; - for _ in 0..1000 { - keys.push(rng.gen_range(0..9999).to_string()); + let mut keys = HashSet::with_capacity(1000); + + while keys.len() < 1000 { + keys.insert(rng.gen_range(0..10000).to_string()); } - keys + + keys.into_iter().collect() } fn values(rng: &mut ThreadRng) -> Vec { - let mut values = vec![]; - for _ in 0..1000 { - values.push(rng.gen_range(0..9999)); + let mut values = HashSet::with_capacity(1000); + + while values.len() < 1000 { + values.insert(rng.gen_range(0..10000)); } - values + values.into_iter().collect() } fn criterion_benchmark(c: &mut Criterion) { diff --git a/datafusion/functions-nested/src/map.rs b/datafusion/functions-nested/src/map.rs index b6068fdff0d5..29afe4a7f3be 100644 --- a/datafusion/functions-nested/src/map.rs +++ b/datafusion/functions-nested/src/map.rs @@ -16,7 +16,7 @@ // under the License. use std::any::Any; -use std::collections::VecDeque; +use std::collections::{HashSet, VecDeque}; use std::sync::Arc; use arrow::array::ArrayData; @@ -24,7 +24,8 @@ use arrow_array::{Array, ArrayRef, MapArray, OffsetSizeTrait, StructArray}; use arrow_buffer::{Buffer, ToByteSlice}; use arrow_schema::{DataType, Field, SchemaBuilder}; -use datafusion_common::{exec_err, ScalarValue}; +use datafusion_common::utils::{fixed_size_list_to_arrays, list_to_arrays}; +use datafusion_common::{exec_err, Result, ScalarValue}; use datafusion_expr::expr::ScalarFunction; use datafusion_expr::{ColumnarValue, Expr, ScalarUDFImpl, Signature, Volatility}; @@ -51,7 +52,7 @@ fn can_evaluate_to_const(args: &[ColumnarValue]) -> bool { .all(|arg| matches!(arg, ColumnarValue::Scalar(_))) } -fn make_map_batch(args: &[ColumnarValue]) -> datafusion_common::Result { +fn make_map_batch(args: &[ColumnarValue]) -> Result { if args.len() != 2 { return exec_err!( "make_map requires exactly 2 arguments, got {} instead", @@ -59,16 +60,56 @@ fn make_map_batch(args: &[ColumnarValue]) -> datafusion_common::Result 0 { + return exec_err!("map key cannot be null"); + } + let key_array = keys.as_ref(); + + match &args[0] { + ColumnarValue::Array(_) => { + let row_keys = match key_array.data_type() { + DataType::List(_) => list_to_arrays::(&keys), + DataType::LargeList(_) => list_to_arrays::(&keys), + DataType::FixedSizeList(_, _) => fixed_size_list_to_arrays(&keys), + data_type => { + return exec_err!( + "Expected list, large_list or fixed_size_list, got {:?}", + data_type + ); + } + }; + + row_keys + .iter() + .try_for_each(|key| check_unique_keys(key.as_ref()))?; + } + ColumnarValue::Scalar(_) => { + check_unique_keys(key_array)?; + } + } + + let values = get_first_array_ref(&args[1])?; + make_map_batch_internal(keys, values, can_evaluate_to_const, args[0].data_type()) } -fn get_first_array_ref( - columnar_value: &ColumnarValue, -) -> datafusion_common::Result { +fn check_unique_keys(array: &dyn Array) -> Result<()> { + let mut seen_keys = HashSet::with_capacity(array.len()); + + for i in 0..array.len() { + let key = ScalarValue::try_from_array(array, i)?; + if seen_keys.contains(&key) { + return exec_err!("map key must be unique, duplicate key found: {}", key); + } + seen_keys.insert(key); + } + Ok(()) +} + +fn get_first_array_ref(columnar_value: &ColumnarValue) -> Result { match columnar_value { ColumnarValue::Scalar(value) => match value { ScalarValue::List(array) => Ok(array.value(0)), @@ -85,11 +126,7 @@ fn make_map_batch_internal( values: ArrayRef, can_evaluate_to_const: bool, data_type: DataType, -) -> datafusion_common::Result { - if keys.null_count() > 0 { - return exec_err!("map key cannot be null"); - } - +) -> Result { if keys.len() != values.len() { return exec_err!("map requires key and value lists to have the same length"); } @@ -173,7 +210,7 @@ impl ScalarUDFImpl for MapFunc { &self.signature } - fn return_type(&self, arg_types: &[DataType]) -> datafusion_common::Result { + fn return_type(&self, arg_types: &[DataType]) -> Result { if arg_types.len() % 2 != 0 { return exec_err!( "map requires an even number of arguments, got {} instead", @@ -198,11 +235,11 @@ impl ScalarUDFImpl for MapFunc { )) } - fn invoke(&self, args: &[ColumnarValue]) -> datafusion_common::Result { + fn invoke(&self, args: &[ColumnarValue]) -> Result { make_map_batch(args) } } -fn get_element_type(data_type: &DataType) -> datafusion_common::Result<&DataType> { +fn get_element_type(data_type: &DataType) -> Result<&DataType> { match data_type { DataType::List(element) => Ok(element.data_type()), DataType::LargeList(element) => Ok(element.data_type()), @@ -273,12 +310,12 @@ fn get_element_type(data_type: &DataType) -> datafusion_common::Result<&DataType fn make_map_array_internal( keys: ArrayRef, values: ArrayRef, -) -> datafusion_common::Result { +) -> Result { let mut offset_buffer = vec![O::zero()]; let mut running_offset = O::zero(); - let keys = datafusion_common::utils::list_to_arrays::(keys); - let values = datafusion_common::utils::list_to_arrays::(values); + let keys = list_to_arrays::(&keys); + let values = list_to_arrays::(&values); let mut key_array_vec = vec![]; let mut value_array_vec = vec![]; diff --git a/datafusion/sqllogictest/test_files/map.slt b/datafusion/sqllogictest/test_files/map.slt index c66334c4de2a..45e1b51a09b4 100644 --- a/datafusion/sqllogictest/test_files/map.slt +++ b/datafusion/sqllogictest/test_files/map.slt @@ -198,6 +198,45 @@ SELECT MAP(['POST', 'HEAD', 'PATCH'], [41, 33]); query error DataFusion error: Execution error: map key cannot be null SELECT MAP(['POST', 'HEAD', null], [41, 33, 30]); +statement error DataFusion error: Execution error: map key cannot be null +CREATE TABLE duplicated_keys_table +AS VALUES + (MAP {1: [1, NULL, 3], NULL: [4, NULL, 6]}); + +# Test duplicate keys +# key is a scalar type +query error DataFusion error: Execution error: map key must be unique, duplicate key found: POST +SELECT MAP(['POST', 'HEAD', 'POST'], [41, 33, null]); + +query error DataFusion error: Execution error: map key must be unique, duplicate key found: POST +SELECT MAP(make_array('POST', 'HEAD', 'POST'), make_array(41, 33, 30)); + +query error DataFusion error: Execution error: map key must be unique, duplicate key found: POST +SELECT make_map('POST', 41, 'HEAD', 33, 'POST', 30); + +statement error DataFusion error: Execution error: map key must be unique, duplicate key found: 1 +CREATE TABLE duplicated_keys_table +AS VALUES + (MAP {1: [1, NULL, 3], 1: [4, NULL, 6]}); + +statement ok +create table duplicate_keys_table as values +('a', 1, 'a', 10, ['k1', 'k1'], [1, 2]); + +query error DataFusion error: Execution error: map key must be unique, duplicate key found: a +SELECT make_map(column1, column2, column3, column4) FROM duplicate_keys_table; + +query error DataFusion error: Execution error: map key must be unique, duplicate key found: k1 +SELECT map(column5, column6) FROM duplicate_keys_table; + +# key is a nested type +query error DataFusion error: Execution error: map key must be unique, duplicate key found: \[1, 2\] +SELECT MAP([[1,2], [1,2], []], [41, 33, null]); + +query error DataFusion error: Execution error: map key must be unique, duplicate key found: \[\{1:1\}\] +SELECT MAP([Map {1:'1'}, Map {1:'1'}, Map {2:'2'}], [41, 33, null]); + + query ? SELECT MAP(make_array('POST', 'HEAD', 'PATCH'), make_array(41, 33, 30)); ---- From 1531c5d43b0327ec9b2b225ca7b3018b3b94342e Mon Sep 17 00:00:00 2001 From: Phillip LeBlanc Date: Wed, 4 Sep 2024 19:42:21 +0900 Subject: [PATCH 087/154] Update download page to reflect latest version (v41) (#12310) * Update download page to reflect latest version * Add note to update the download page --- dev/release/README.md | 4 ++++ docs/source/download.md | 8 ++++---- 2 files changed, 8 insertions(+), 4 deletions(-) diff --git a/dev/release/README.md b/dev/release/README.md index 397369a41aa3..bd9c0621fdbc 100644 --- a/dev/release/README.md +++ b/dev/release/README.md @@ -407,3 +407,7 @@ Here is an example blog post PR: - https://github.com/apache/arrow-site/pull/217 Once the PR is merged, a GitHub action will publish the new blog post to https://arrow.apache.org/blog/. + +### Update the version on the download page + +Update the version on the [download page](https://datafusion.apache.org/download) to point to the latest release [here](../../docs/source/download.md). diff --git a/docs/source/download.md b/docs/source/download.md index 0c279dd033f2..33a6d7008877 100644 --- a/docs/source/download.md +++ b/docs/source/download.md @@ -26,12 +26,12 @@ official Apache DataFusion releases are provided as source artifacts. ## Releases -The latest source release is [37.0.0][source-link] ([asc][asc-link], +The latest source release is [41.0.0][source-link] ([asc][asc-link], [sha512][sha512-link]). -[source-link]: https://www.apache.org/dyn/closer.lua/arrow/arrow-datafusion-37.0.0/apache-arrow-datafusion-37.0.0.tar.gz?action=download -[asc-link]: https://downloads.apache.org/arrow/arrow-datafusion-37.0.0/apache-arrow-datafusion-37.0.0.tar.gz.asc -[sha512-link]: https://downloads.apache.org/arrow/arrow-datafusion-37.0.0/apache-arrow-datafusion-37.0.0.tar.gz.sha512 +[source-link]: https://www.apache.org/dyn/closer.lua/datafusion/datafusion-41.0.0/apache-datafusion-41.0.0.tar.gz?action=download +[asc-link]: https://downloads.apache.org/datafusion/datafusion-41.0.0/apache-datafusion-41.0.0.tar.gz.asc +[sha512-link]: https://downloads.apache.org/datafusion/datafusion-41.0.0/apache-datafusion-41.0.0.tar.gz.sha512 For previous releases, please check the [archive](https://archive.apache.org/dist/datafusion/). From e5a6cd5f6784f1d1b3d559d5356a6154a73e077c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=BC=A0=E6=9E=97=E4=BC=9F?= Date: Wed, 4 Sep 2024 20:20:46 +0800 Subject: [PATCH 088/154] Improve binary scalars display (#12192) * Improve binary scalar display * Reduce allocation & add test * Display binary scalar as hex string --- datafusion/common/src/scalar/mod.rs | 79 ++++++++++++++++++++++++----- 1 file changed, 66 insertions(+), 13 deletions(-) diff --git a/datafusion/common/src/scalar/mod.rs b/datafusion/common/src/scalar/mod.rs index 5acc2b6f188e..3cff0731dcee 100644 --- a/datafusion/common/src/scalar/mod.rs +++ b/datafusion/common/src/scalar/mod.rs @@ -3517,14 +3517,14 @@ impl fmt::Display for ScalarValue { | ScalarValue::FixedSizeBinary(_, e) | ScalarValue::LargeBinary(e) | ScalarValue::BinaryView(e) => match e { - Some(l) => write!( - f, - "{}", - l.iter() - .map(|v| format!("{v}")) - .collect::>() - .join(",") - )?, + Some(l) => { + let data = l.iter().map(|v| format!("{v:02X}")).collect::>(); + if data.len() > 10 { + write!(f, "{}...", data[..10].join(""))?; + } else { + write!(f, "{}", data.join(""))?; + } + } None => write!(f, "NULL")?, }, ScalarValue::List(arr) => fmt_list(arr.to_owned() as ArrayRef, f)?, @@ -3648,6 +3648,16 @@ fn fmt_list(arr: ArrayRef, f: &mut fmt::Formatter) -> fmt::Result { impl fmt::Debug for ScalarValue { fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { + fn fmt_binary(data: &[u8], f: &mut fmt::Formatter) -> fmt::Result { + write!( + f, + "{}", + data.iter() + .map(|v| format!("{v}")) + .collect::>() + .join(",") + ) + } match self { ScalarValue::Decimal128(_, _, _) => write!(f, "Decimal128({self})"), ScalarValue::Decimal256(_, _, _) => write!(f, "Decimal256({self})"), @@ -3682,17 +3692,31 @@ impl fmt::Debug for ScalarValue { ScalarValue::LargeUtf8(None) => write!(f, "LargeUtf8({self})"), ScalarValue::LargeUtf8(Some(_)) => write!(f, "LargeUtf8(\"{self}\")"), ScalarValue::Binary(None) => write!(f, "Binary({self})"), - ScalarValue::Binary(Some(_)) => write!(f, "Binary(\"{self}\")"), + ScalarValue::Binary(Some(b)) => { + write!(f, "Binary(\"")?; + fmt_binary(b.as_slice(), f)?; + write!(f, "\")") + } ScalarValue::BinaryView(None) => write!(f, "BinaryView({self})"), - ScalarValue::BinaryView(Some(_)) => write!(f, "BinaryView(\"{self}\")"), + ScalarValue::BinaryView(Some(b)) => { + write!(f, "BinaryView(\"")?; + fmt_binary(b.as_slice(), f)?; + write!(f, "\")") + } ScalarValue::FixedSizeBinary(size, None) => { write!(f, "FixedSizeBinary({size}, {self})") } - ScalarValue::FixedSizeBinary(size, Some(_)) => { - write!(f, "FixedSizeBinary({size}, \"{self}\")") + ScalarValue::FixedSizeBinary(size, Some(b)) => { + write!(f, "Binary({size}, \"")?; + fmt_binary(b.as_slice(), f)?; + write!(f, "\")") } ScalarValue::LargeBinary(None) => write!(f, "LargeBinary({self})"), - ScalarValue::LargeBinary(Some(_)) => write!(f, "LargeBinary(\"{self}\")"), + ScalarValue::LargeBinary(Some(b)) => { + write!(f, "LargeBinary(\"")?; + fmt_binary(b.as_slice(), f)?; + write!(f, "\")") + } ScalarValue::FixedSizeList(_) => write!(f, "FixedSizeList({self})"), ScalarValue::List(_) => write!(f, "List({self})"), ScalarValue::LargeList(_) => write!(f, "LargeList({self})"), @@ -6518,6 +6542,35 @@ mod tests { assert_batches_eq!(&expected, &[batch]); } + #[test] + fn test_binary_display() { + let small_binary_value = ScalarValue::Binary(Some(vec![1u8, 2, 3])); + assert_eq!(format!("{small_binary_value}"), "010203"); + let large_binary_value = + ScalarValue::Binary(Some(vec![1u8, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11])); + assert_eq!(format!("{large_binary_value}"), "0102030405060708090A..."); + + let small_binary_value = ScalarValue::BinaryView(Some(vec![1u8, 2, 3])); + assert_eq!(format!("{small_binary_value}"), "010203"); + let large_binary_value = + ScalarValue::BinaryView(Some(vec![1u8, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11])); + assert_eq!(format!("{large_binary_value}"), "0102030405060708090A..."); + + let small_binary_value = ScalarValue::LargeBinary(Some(vec![1u8, 2, 3])); + assert_eq!(format!("{small_binary_value}"), "010203"); + let large_binary_value = + ScalarValue::LargeBinary(Some(vec![1u8, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11])); + assert_eq!(format!("{large_binary_value}"), "0102030405060708090A..."); + + let small_binary_value = ScalarValue::FixedSizeBinary(3, Some(vec![1u8, 2, 3])); + assert_eq!(format!("{small_binary_value}"), "010203"); + let large_binary_value = ScalarValue::FixedSizeBinary( + 11, + Some(vec![1u8, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11]), + ); + assert_eq!(format!("{large_binary_value}"), "0102030405060708090A..."); + } + #[test] fn test_build_timestamp_millisecond_list() { let values = vec![ScalarValue::TimestampMillisecond(Some(1), None)]; From e6caf6206b516fe97fc2d2962900030d1782e7d5 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Wed, 4 Sep 2024 12:25:43 -0400 Subject: [PATCH 089/154] Minor: reduce allocations in ScalarValue::*Binary display (#12322) --- datafusion/common/src/scalar/mod.rs | 21 +++++++++++++++------ 1 file changed, 15 insertions(+), 6 deletions(-) diff --git a/datafusion/common/src/scalar/mod.rs b/datafusion/common/src/scalar/mod.rs index 3cff0731dcee..88802af23a4b 100644 --- a/datafusion/common/src/scalar/mod.rs +++ b/datafusion/common/src/scalar/mod.rs @@ -3517,12 +3517,13 @@ impl fmt::Display for ScalarValue { | ScalarValue::FixedSizeBinary(_, e) | ScalarValue::LargeBinary(e) | ScalarValue::BinaryView(e) => match e { - Some(l) => { - let data = l.iter().map(|v| format!("{v:02X}")).collect::>(); - if data.len() > 10 { - write!(f, "{}...", data[..10].join(""))?; - } else { - write!(f, "{}", data.join(""))?; + Some(bytes) => { + // print up to first 10 bytes, with trailing ... if needed + for b in bytes.iter().take(10) { + write!(f, "{b:02X}")?; + } + if bytes.len() > 10 { + write!(f, "...")?; } } None => write!(f, "NULL")?, @@ -6544,24 +6545,32 @@ mod tests { #[test] fn test_binary_display() { + let no_binary_value = ScalarValue::Binary(None); + assert_eq!(format!("{no_binary_value}"), "NULL"); let small_binary_value = ScalarValue::Binary(Some(vec![1u8, 2, 3])); assert_eq!(format!("{small_binary_value}"), "010203"); let large_binary_value = ScalarValue::Binary(Some(vec![1u8, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11])); assert_eq!(format!("{large_binary_value}"), "0102030405060708090A..."); + let no_binary_value = ScalarValue::BinaryView(None); + assert_eq!(format!("{no_binary_value}"), "NULL"); let small_binary_value = ScalarValue::BinaryView(Some(vec![1u8, 2, 3])); assert_eq!(format!("{small_binary_value}"), "010203"); let large_binary_value = ScalarValue::BinaryView(Some(vec![1u8, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11])); assert_eq!(format!("{large_binary_value}"), "0102030405060708090A..."); + let no_binary_value = ScalarValue::LargeBinary(None); + assert_eq!(format!("{no_binary_value}"), "NULL"); let small_binary_value = ScalarValue::LargeBinary(Some(vec![1u8, 2, 3])); assert_eq!(format!("{small_binary_value}"), "010203"); let large_binary_value = ScalarValue::LargeBinary(Some(vec![1u8, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11])); assert_eq!(format!("{large_binary_value}"), "0102030405060708090A..."); + let no_binary_value = ScalarValue::FixedSizeBinary(3, None); + assert_eq!(format!("{no_binary_value}"), "NULL"); let small_binary_value = ScalarValue::FixedSizeBinary(3, Some(vec![1u8, 2, 3])); assert_eq!(format!("{small_binary_value}"), "010203"); let large_binary_value = ScalarValue::FixedSizeBinary( From 91b1d2bfe8f603df94e846b91d8475a0af2e5240 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Wed, 4 Sep 2024 10:27:21 -0600 Subject: [PATCH 090/154] Add PartialEq, Eq traits to StatsType (#12327) --- datafusion/functions-aggregate-common/src/stats.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/functions-aggregate-common/src/stats.rs b/datafusion/functions-aggregate-common/src/stats.rs index 6a11ebe36c5f..bcd004db7831 100644 --- a/datafusion/functions-aggregate-common/src/stats.rs +++ b/datafusion/functions-aggregate-common/src/stats.rs @@ -17,7 +17,7 @@ /// TODO: Move this to functions-aggregate module /// Enum used for differentiating population and sample for statistical functions -#[derive(Debug, Clone, Copy)] +#[derive(PartialEq, Eq, Debug, Clone, Copy)] pub enum StatsType { /// Population Population, From 6034be42808b43e3f48f6e58ec38cc35fa253abb Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Thu, 5 Sep 2024 06:05:40 -0400 Subject: [PATCH 091/154] Update to `arrow`/`parquet` `53.0.0`, `tonic`, `prost`, `object_store`, `pyo3` (#12032) * Update prost, prost-derive, pbjson * udpate more * Update datafusion/substrait/Cargo.toml Co-authored-by: tison * Update vendored code * revert upgrade in datafusion-examples until arrow-flight is updated * Pin to pre-release arrow-rs * update pyo3 * Update to use new arrow apis * update for new api * Update tonic in examples * update prost * update datafusion-cli/cargo * update test output * update * updates * updates * update math * update more * fix scalar tests * Port statistics to use new API * factor into a function * update generated files * Update test * add new test * update tests * tapelo format * Update other tests * Update datafusion pin * Update for API change * Update to arrow 53.0.0 sha * Update cli deps * update cargo.lock * Update expected output * Remove patch * update datafusion-cli cargo * Pin some aws sdks whose update caused CI failures --------- Co-authored-by: tison --- Cargo.toml | 24 +- datafusion-cli/Cargo.lock | 216 +++++++++--------- datafusion-cli/Cargo.toml | 13 +- datafusion-cli/src/functions.rs | 113 +++++---- datafusion-examples/Cargo.toml | 4 +- datafusion/common/Cargo.toml | 2 +- datafusion/common/src/scalar/mod.rs | 6 +- .../src/datasource/file_format/parquet.rs | 2 +- .../physical_plan/parquet/page_filter.rs | 5 +- .../physical_plan/parquet/row_group_filter.rs | 150 +++++++++--- .../functions/src/regex/regexpreplace.rs | 3 +- .../src/binary_view_map.rs | 2 +- datafusion/physical-plan/src/coalesce/mod.rs | 5 +- datafusion/proto-common/Cargo.toml | 2 +- datafusion/proto-common/gen/Cargo.toml | 4 +- datafusion/proto/Cargo.toml | 2 +- datafusion/proto/gen/Cargo.toml | 4 +- .../src/generated/datafusion_proto_common.rs | 54 ++--- datafusion/proto/src/generated/pbjson.rs | 31 +++ datafusion/proto/src/generated/prost.rs | 14 +- datafusion/sql/src/unparser/expr.rs | 18 +- datafusion/sql/tests/cases/plan_to_sql.rs | 4 +- .../sqllogictest/test_files/aggregate.slt | 6 +- .../sqllogictest/test_files/arrow_typeof.slt | 12 +- datafusion/sqllogictest/test_files/ddl.slt | 2 +- datafusion/sqllogictest/test_files/expr.slt | 80 +++---- .../sqllogictest/test_files/interval.slt | 102 ++++----- datafusion/sqllogictest/test_files/math.slt | 2 +- .../test_files/repartition_scan.slt | 8 +- .../sqllogictest/test_files/timestamps.slt | 16 +- 30 files changed, 526 insertions(+), 380 deletions(-) diff --git a/Cargo.toml b/Cargo.toml index 877cead93673..c155e475a026 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -69,22 +69,22 @@ version = "41.0.0" ahash = { version = "0.8", default-features = false, features = [ "runtime-rng", ] } -arrow = { version = "52.2.0", features = [ +arrow = { version = "53.0.0", features = [ "prettyprint", ] } -arrow-array = { version = "52.2.0", default-features = false, features = [ +arrow-array = { version = "53.0.0", default-features = false, features = [ "chrono-tz", ] } -arrow-buffer = { version = "52.2.0", default-features = false } -arrow-flight = { version = "52.2.0", features = [ +arrow-buffer = { version = "53.0.0", default-features = false } +arrow-flight = { version = "53.0.0", features = [ "flight-sql-experimental", ] } -arrow-ipc = { version = "52.2.0", default-features = false, features = [ +arrow-ipc = { version = "53.0.0", default-features = false, features = [ "lz4", ] } -arrow-ord = { version = "52.2.0", default-features = false } -arrow-schema = { version = "52.2.0", default-features = false } -arrow-string = { version = "52.2.0", default-features = false } +arrow-ord = { version = "53.0.0", default-features = false } +arrow-schema = { version = "53.0.0", default-features = false } +arrow-string = { version = "53.0.0", default-features = false } async-trait = "0.1.73" bigdecimal = "=0.4.1" bytes = "1.4" @@ -122,15 +122,17 @@ indexmap = "2.0.0" itertools = "0.13" log = "^0.4" num_cpus = "1.13.0" -object_store = { version = "0.10.2", default-features = false } +object_store = { version = "0.11.0", default-features = false } parking_lot = "0.12" -parquet = { version = "52.2.0", default-features = false, features = [ +parquet = { version = "53.0.0", default-features = false, features = [ "arrow", "async", "object_store", ] } +pbjson = { version = "0.7.0" } # Should match arrow-flight's version of prost. -prost = "0.12.3" +prost = "0.13.1" +prost-derive = "0.13.1" rand = "0.8" regex = "1.8" rstest = "0.22.0" diff --git a/datafusion-cli/Cargo.lock b/datafusion-cli/Cargo.lock index ddc6242977d3..039f3fb9a6aa 100644 --- a/datafusion-cli/Cargo.lock +++ b/datafusion-cli/Cargo.lock @@ -179,9 +179,9 @@ checksum = "7c02d123df017efcdfbd739ef81735b36c5ba83ec3c59c80a9d7ecc718f92e50" [[package]] name = "arrow" -version = "52.2.0" +version = "53.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "05048a8932648b63f21c37d88b552ccc8a65afb6dfe9fc9f30ce79174c2e7a85" +checksum = "45aef0d9cf9a039bf6cd1acc451b137aca819977b0928dece52bd92811b640ba" dependencies = [ "arrow-arith", "arrow-array", @@ -200,9 +200,9 @@ dependencies = [ [[package]] name = "arrow-arith" -version = "52.2.0" +version = "53.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1d8a57966e43bfe9a3277984a14c24ec617ad874e4c0e1d2a1b083a39cfbf22c" +checksum = "03675e42d1560790f3524800e41403b40d0da1c793fe9528929fde06d8c7649a" dependencies = [ "arrow-array", "arrow-buffer", @@ -215,9 +215,9 @@ dependencies = [ [[package]] name = "arrow-array" -version = "52.2.0" +version = "53.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "16f4a9468c882dc66862cef4e1fd8423d47e67972377d85d80e022786427768c" +checksum = "cd2bf348cf9f02a5975c5962c7fa6dee107a2009a7b41ac5fb1a027e12dc033f" dependencies = [ "ahash", "arrow-buffer", @@ -232,9 +232,9 @@ dependencies = [ [[package]] name = "arrow-buffer" -version = "52.2.0" +version = "53.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c975484888fc95ec4a632cdc98be39c085b1bb518531b0c80c5d462063e5daa1" +checksum = "3092e37715f168976012ce52273c3989b5793b0db5f06cbaa246be25e5f0924d" dependencies = [ "bytes", "half", @@ -243,9 +243,9 @@ dependencies = [ [[package]] name = "arrow-cast" -version = "52.2.0" +version = "53.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "da26719e76b81d8bc3faad1d4dbdc1bcc10d14704e63dc17fc9f3e7e1e567c8e" +checksum = "7ce1018bb710d502f9db06af026ed3561552e493e989a79d0d0f5d9cf267a785" dependencies = [ "arrow-array", "arrow-buffer", @@ -264,9 +264,9 @@ dependencies = [ [[package]] name = "arrow-csv" -version = "52.2.0" +version = "53.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c13c36dc5ddf8c128df19bab27898eea64bf9da2b555ec1cd17a8ff57fba9ec2" +checksum = "fd178575f45624d045e4ebee714e246a05d9652e41363ee3f57ec18cca97f740" dependencies = [ "arrow-array", "arrow-buffer", @@ -283,9 +283,9 @@ dependencies = [ [[package]] name = "arrow-data" -version = "52.2.0" +version = "53.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "dd9d6f18c65ef7a2573ab498c374d8ae364b4a4edf67105357491c031f716ca5" +checksum = "4e4ac0c4ee79150afe067dc4857154b3ee9c1cd52b5f40d59a77306d0ed18d65" dependencies = [ "arrow-buffer", "arrow-schema", @@ -295,9 +295,9 @@ dependencies = [ [[package]] name = "arrow-ipc" -version = "52.2.0" +version = "53.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e786e1cdd952205d9a8afc69397b317cfbb6e0095e445c69cda7e8da5c1eeb0f" +checksum = "bb307482348a1267f91b0912e962cd53440e5de0f7fb24c5f7b10da70b38c94a" dependencies = [ "arrow-array", "arrow-buffer", @@ -310,9 +310,9 @@ dependencies = [ [[package]] name = "arrow-json" -version = "52.2.0" +version = "53.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fb22284c5a2a01d73cebfd88a33511a3234ab45d66086b2ca2d1228c3498e445" +checksum = "d24805ba326758effdd6f2cbdd482fcfab749544f21b134701add25b33f474e6" dependencies = [ "arrow-array", "arrow-buffer", @@ -330,9 +330,9 @@ dependencies = [ [[package]] name = "arrow-ord" -version = "52.2.0" +version = "53.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "42745f86b1ab99ef96d1c0bcf49180848a64fe2c7a7a0d945bc64fa2b21ba9bc" +checksum = "644046c479d80ae8ed02a7f1e1399072ea344ca6a7b0e293ab2d5d9ed924aa3b" dependencies = [ "arrow-array", "arrow-buffer", @@ -345,9 +345,9 @@ dependencies = [ [[package]] name = "arrow-row" -version = "52.2.0" +version = "53.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4cd09a518c602a55bd406bcc291a967b284cfa7a63edfbf8b897ea4748aad23c" +checksum = "a29791f8eb13b340ce35525b723f5f0df17ecb955599e11f65c2a94ab34e2efb" dependencies = [ "ahash", "arrow-array", @@ -359,15 +359,15 @@ dependencies = [ [[package]] name = "arrow-schema" -version = "52.2.0" +version = "53.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9e972cd1ff4a4ccd22f86d3e53e835c2ed92e0eea6a3e8eadb72b4f1ac802cf8" +checksum = "c85320a3a2facf2b2822b57aa9d6d9d55edb8aee0b6b5d3b8df158e503d10858" [[package]] name = "arrow-select" -version = "52.2.0" +version = "53.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "600bae05d43483d216fb3494f8c32fdbefd8aa4e1de237e790dbb3d9f44690a3" +checksum = "9cc7e6b582e23855fd1625ce46e51647aa440c20ea2e71b1d748e0839dd73cba" dependencies = [ "ahash", "arrow-array", @@ -379,9 +379,9 @@ dependencies = [ [[package]] name = "arrow-string" -version = "52.2.0" +version = "53.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f0dc1985b67cb45f6606a248ac2b4a288849f196bab8c657ea5589f47cdd55e6" +checksum = "0775b6567c66e56ded19b87a954b6b1beffbdd784ef95a3a2b03f59570c1d230" dependencies = [ "arrow-array", "arrow-buffer", @@ -430,13 +430,13 @@ dependencies = [ [[package]] name = "async-trait" -version = "0.1.81" +version = "0.1.82" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6e0c28dcc82d7c8ead5cb13beb15405b57b8546e93215673ff8ca0349a028107" +checksum = "a27b8a3a6e1a44fa4c8baf1f653e4172e81486d4941f2237e20dc2d0cf4ddff1" dependencies = [ "proc-macro2", "quote", - "syn 2.0.75", + "syn 2.0.77", ] [[package]] @@ -503,9 +503,9 @@ dependencies = [ [[package]] name = "aws-credential-types" -version = "1.2.0" +version = "1.2.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e16838e6c9e12125face1c1eff1343c75e3ff540de98ff7ebd61874a89bcfeb9" +checksum = "60e8f6b615cb5fc60a98132268508ad104310f0cfb25a1c22eee76efdf9154da" dependencies = [ "aws-smithy-async", "aws-smithy-runtime-api", @@ -515,14 +515,15 @@ dependencies = [ [[package]] name = "aws-runtime" -version = "1.4.0" +version = "1.4.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f42c2d4218de4dcd890a109461e2f799a1a2ba3bcd2cde9af88360f5df9266c6" +checksum = "2424565416eef55906f9f8cece2072b6b6a76075e3ff81483ebe938a89a4c05f" dependencies = [ "aws-credential-types", "aws-sigv4", "aws-smithy-async", "aws-smithy-http", + "aws-smithy-runtime", "aws-smithy-runtime-api", "aws-smithy-types", "aws-types", @@ -730,6 +731,7 @@ dependencies = [ "base64-simd", "bytes", "bytes-utils", + "futures-core", "http 0.2.12", "http 1.1.0", "http-body 0.4.6", @@ -742,6 +744,8 @@ dependencies = [ "ryu", "serde", "time", + "tokio", + "tokio-util", ] [[package]] @@ -930,9 +934,9 @@ dependencies = [ [[package]] name = "cc" -version = "1.1.13" +version = "1.1.15" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "72db2f7947ecee9b03b510377e8bb9077afa27176fdbff55c51027e976fdcc48" +checksum = "57b6a275aa2903740dc87da01c62040406b8812552e97129a63ea8850a17c6e6" dependencies = [ "jobserver", "libc", @@ -1011,7 +1015,7 @@ dependencies = [ "heck 0.5.0", "proc-macro2", "quote", - "syn 2.0.75", + "syn 2.0.77", ] [[package]] @@ -1070,9 +1074,9 @@ dependencies = [ [[package]] name = "constant_time_eq" -version = "0.3.0" +version = "0.3.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f7144d30dcf0fafbce74250a3963025d8d52177934239851c917d29f1df280c2" +checksum = "7c74b8349d32d297c9134b8c88677813a227df8f779daa29bfc29c183fe3dca6" [[package]] name = "core-foundation" @@ -1167,7 +1171,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "edb49164822f3ee45b17acd4a208cfc1251410cf0cad9a833234c9890774dd9f" dependencies = [ "quote", - "syn 2.0.75", + "syn 2.0.77", ] [[package]] @@ -1268,6 +1272,9 @@ dependencies = [ "async-trait", "aws-config", "aws-credential-types", + "aws-sdk-sso", + "aws-sdk-ssooidc", + "aws-sdk-sts", "clap", "ctor", "datafusion", @@ -1697,9 +1704,9 @@ dependencies = [ [[package]] name = "fastrand" -version = "2.1.0" +version = "2.1.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9fc0510504f03c51ada170672ac806f1f105a88aa97a5281117e1ddc3368e51a" +checksum = "e8c02a5121d4ea3eb16a80748c74f5549a5665e4c21333c6098f283870fbdea6" [[package]] name = "fd-lock" @@ -1730,9 +1737,9 @@ dependencies = [ [[package]] name = "flate2" -version = "1.0.32" +version = "1.0.33" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9c0596c1eac1f9e04ed902702e9878208b336edc9d6fddc8a48387349bab3666" +checksum = "324a1be68054ef05ad64b861cc9eaf1d623d2d8cb25b4bf2cb9cdd902b4bf253" dependencies = [ "crc32fast", "miniz_oxide 0.8.0", @@ -1818,7 +1825,7 @@ checksum = "87750cf4b7a4c0625b1529e4c543c2182106e4dedc60a2a6455e00d212c489ac" dependencies = [ "proc-macro2", "quote", - "syn 2.0.75", + "syn 2.0.77", ] [[package]] @@ -2136,7 +2143,7 @@ dependencies = [ "hyper 1.4.1", "hyper-util", "rustls 0.23.12", - "rustls-native-certs 0.7.2", + "rustls-native-certs 0.7.3", "rustls-pki-types", "tokio", "tokio-rustls 0.26.0", @@ -2198,9 +2205,9 @@ dependencies = [ [[package]] name = "indexmap" -version = "2.4.0" +version = "2.5.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "93ead53efc7ea8ed3cfb0c79fc8023fbb782a5432b52830b6518941cebe6505c" +checksum = "68b900aa2f7301e21c36462b170ee99994de34dff39a4a6a528e80e7376d07e5" dependencies = [ "equivalent", "hashbrown", @@ -2616,18 +2623,18 @@ dependencies = [ [[package]] name = "object" -version = "0.36.3" +version = "0.36.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "27b64972346851a39438c60b341ebc01bba47464ae329e55cf343eb93964efd9" +checksum = "084f1a5821ac4c651660a94a7153d27ac9d8a53736203f58b31945ded098070a" dependencies = [ "memchr", ] [[package]] name = "object_store" -version = "0.10.2" +version = "0.11.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e6da452820c715ce78221e8202ccc599b4a52f3e1eb3eedb487b680c81a8e3f3" +checksum = "25a0c4b3a0e31f8b66f71ad8064521efa773910196e2cde791436f13409f3b45" dependencies = [ "async-trait", "base64 0.22.1", @@ -2706,9 +2713,9 @@ dependencies = [ [[package]] name = "parquet" -version = "52.2.0" +version = "53.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e977b9066b4d3b03555c22bdc442f3fadebd96a39111249113087d0edb2691cd" +checksum = "f0fbf928021131daaa57d334ca8e3904fe9ae22f73c56244fc7db9b04eedc3d8" dependencies = [ "ahash", "arrow-array", @@ -2826,7 +2833,7 @@ checksum = "2f38a4412a78282e09a2cf38d195ea5420d15ba0602cb375210efbc877243965" dependencies = [ "proc-macro2", "quote", - "syn 2.0.75", + "syn 2.0.77", ] [[package]] @@ -2919,9 +2926,9 @@ dependencies = [ [[package]] name = "quinn" -version = "0.11.3" +version = "0.11.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b22d8e7369034b9a7132bc2008cac12f2013c8132b45e0554e6e20e2617f2156" +checksum = "8c7c5fdde3cdae7203427dc4f0a68fe0ed09833edc525a03456b153b79828684" dependencies = [ "bytes", "pin-project-lite", @@ -2937,9 +2944,9 @@ dependencies = [ [[package]] name = "quinn-proto" -version = "0.11.6" +version = "0.11.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ba92fb39ec7ad06ca2582c0ca834dfeadcaf06ddfc8e635c80aa7e1c05315fdd" +checksum = "fadfaed2cd7f389d0161bb73eeb07b7b78f8691047a6f3e73caaeae55310a4a6" dependencies = [ "bytes", "rand", @@ -2954,22 +2961,22 @@ dependencies = [ [[package]] name = "quinn-udp" -version = "0.5.4" +version = "0.5.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8bffec3605b73c6f1754535084a85229fa8a30f86014e6c81aeec4abb68b0285" +checksum = "4fe68c2e9e1a1234e218683dbdf9f9dfcb094113c5ac2b938dfcb9bab4c4140b" dependencies = [ "libc", "once_cell", "socket2", "tracing", - "windows-sys 0.52.0", + "windows-sys 0.59.0", ] [[package]] name = "quote" -version = "1.0.36" +version = "1.0.37" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0fa76aaf39101c457836aec0ce2316dbdc3ab723cdda1c6bd4e6ad4208acaca7" +checksum = "b5b9d34b8991d19d98081b46eacdd8eb58c6f2b201139f7c5f643cc155a633af" dependencies = [ "proc-macro2", ] @@ -3095,7 +3102,7 @@ dependencies = [ "pin-project-lite", "quinn", "rustls 0.23.12", - "rustls-native-certs 0.7.2", + "rustls-native-certs 0.7.3", "rustls-pemfile 2.1.3", "rustls-pki-types", "serde", @@ -3175,18 +3182,18 @@ checksum = "583034fd73374156e66797ed8e5b0d5690409c9226b22d87cb7f19821c05d152" [[package]] name = "rustc_version" -version = "0.4.0" +version = "0.4.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bfa0f585226d2e68097d4f95d113b15b83a82e819ab25717ec0590d9584ef366" +checksum = "cfcb3a22ef46e85b45de6ee7e79d063319ebb6594faafcf1c225ea92ab6e9b92" dependencies = [ "semver", ] [[package]] name = "rustix" -version = "0.38.34" +version = "0.38.35" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "70dc5ec042f7a43c4a73241207cecc9873a06d45debb38b329f8541d85c2730f" +checksum = "a85d50532239da68e9addb745ba38ff4612a242c1c7ceea689c4bc7c2f43c36f" dependencies = [ "bitflags 2.6.0", "errno", @@ -3216,7 +3223,7 @@ dependencies = [ "once_cell", "ring", "rustls-pki-types", - "rustls-webpki 0.102.6", + "rustls-webpki 0.102.7", "subtle", "zeroize", ] @@ -3235,9 +3242,9 @@ dependencies = [ [[package]] name = "rustls-native-certs" -version = "0.7.2" +version = "0.7.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "04182dffc9091a404e0fc069ea5cd60e5b866c3adf881eff99a32d048242dffa" +checksum = "e5bfb394eeed242e909609f56089eecfe5fda225042e8b171791b9c95f5931e5" dependencies = [ "openssl-probe", "rustls-pemfile 2.1.3", @@ -3283,9 +3290,9 @@ dependencies = [ [[package]] name = "rustls-webpki" -version = "0.102.6" +version = "0.102.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8e6b52d4fda176fd835fdc55a835d4a89b8499cad995885a21149d5ad62f852e" +checksum = "84678086bd54edf2b415183ed7a94d0efb049f1b646a33e22a36f3794be6ae56" dependencies = [ "ring", "rustls-pki-types", @@ -3398,29 +3405,29 @@ checksum = "a3f0bf26fd526d2a95683cd0f87bf103b8539e2ca1ef48ce002d67aad59aa0b4" [[package]] name = "serde" -version = "1.0.208" +version = "1.0.209" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cff085d2cb684faa248efb494c39b68e522822ac0de72ccf08109abde717cfb2" +checksum = "99fce0ffe7310761ca6bf9faf5115afbc19688edd00171d81b1bb1b116c63e09" dependencies = [ "serde_derive", ] [[package]] name = "serde_derive" -version = "1.0.208" +version = "1.0.209" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "24008e81ff7613ed8e5ba0cfaf24e2c2f1e5b8a0495711e44fcd4882fca62bcf" +checksum = "a5831b979fd7b5439637af1752d535ff49f4860c0f341d1baeb6faf0f4242170" dependencies = [ "proc-macro2", "quote", - "syn 2.0.75", + "syn 2.0.77", ] [[package]] name = "serde_json" -version = "1.0.125" +version = "1.0.127" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "83c8e735a073ccf5be70aa8066aa984eaf2fa000db6c8d0100ae605b366d31ed" +checksum = "8043c06d9f82bd7271361ed64f415fe5e12a77fdb52e573e7f06a516dea329ad" dependencies = [ "itoa", "memchr", @@ -3489,24 +3496,23 @@ checksum = "3c5e1a9a646d36c3599cd173a41282daf47c44583ad367b8e6837255952e5c67" [[package]] name = "snafu" -version = "0.7.5" +version = "0.8.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e4de37ad025c587a29e8f3f5605c00f70b98715ef90b9061a815b9e59e9042d6" +checksum = "2b835cb902660db3415a672d862905e791e54d306c6e8189168c7f3d9ae1c79d" dependencies = [ - "doc-comment", "snafu-derive", ] [[package]] name = "snafu-derive" -version = "0.7.5" +version = "0.8.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "990079665f075b699031e9c08fd3ab99be5029b96f3b78dc0709e8f77e4efebf" +checksum = "38d1e02fca405f6280643174a50c942219f0bbf4dbf7d480f1dd864d6f211ae5" dependencies = [ - "heck 0.4.1", + "heck 0.5.0", "proc-macro2", "quote", - "syn 1.0.109", + "syn 2.0.77", ] [[package]] @@ -3549,7 +3555,7 @@ checksum = "01b2e185515564f15375f593fb966b5718bc624ba77fe49fa4616ad619690554" dependencies = [ "proc-macro2", "quote", - "syn 2.0.75", + "syn 2.0.77", ] [[package]] @@ -3595,7 +3601,7 @@ dependencies = [ "proc-macro2", "quote", "rustversion", - "syn 2.0.75", + "syn 2.0.77", ] [[package]] @@ -3608,7 +3614,7 @@ dependencies = [ "proc-macro2", "quote", "rustversion", - "syn 2.0.75", + "syn 2.0.77", ] [[package]] @@ -3630,9 +3636,9 @@ dependencies = [ [[package]] name = "syn" -version = "2.0.75" +version = "2.0.77" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f6af063034fc1935ede7be0122941bafa9bacb949334d090b77ca98b5817c7d9" +checksum = "9f35bcdf61fd8e7be6caf75f429fdca8beb3ed76584befb503b1569faee373ed" dependencies = [ "proc-macro2", "quote", @@ -3693,7 +3699,7 @@ checksum = "a4558b58466b9ad7ca0f102865eccc95938dca1a74a856f2b57b6629050da261" dependencies = [ "proc-macro2", "quote", - "syn 2.0.75", + "syn 2.0.77", ] [[package]] @@ -3763,9 +3769,9 @@ checksum = "1f3ccbac311fea05f86f61904b462b55fb3df8837a366dfc601a0161d0532f20" [[package]] name = "tokio" -version = "1.39.3" +version = "1.40.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9babc99b9923bfa4804bd74722ff02c0381021eafa4db9949217e3be8e84fff5" +checksum = "e2b070231665d27ad9ec9b8df639893f46727666c6767db40317fbe920a5d998" dependencies = [ "backtrace", "bytes", @@ -3787,7 +3793,7 @@ checksum = "693d596312e88961bc67d7f1f97af8a70227d9f90c31bba5806eec004978d752" dependencies = [ "proc-macro2", "quote", - "syn 2.0.75", + "syn 2.0.77", ] [[package]] @@ -3870,7 +3876,7 @@ checksum = "34704c8d6ebcbc939824180af020566b01a7c01f80641264eba0999f6c2b6be7" dependencies = [ "proc-macro2", "quote", - "syn 2.0.75", + "syn 2.0.77", ] [[package]] @@ -3915,7 +3921,7 @@ checksum = "f03ca4cb38206e2bef0700092660bb74d696f808514dae47fa1467cbfe26e96e" dependencies = [ "proc-macro2", "quote", - "syn 2.0.75", + "syn 2.0.77", ] [[package]] @@ -4064,7 +4070,7 @@ dependencies = [ "once_cell", "proc-macro2", "quote", - "syn 2.0.75", + "syn 2.0.77", "wasm-bindgen-shared", ] @@ -4098,7 +4104,7 @@ checksum = "afc340c74d9005395cf9dd098506f7f44e38f2b4a21c6aaacf9a105ea5e1e836" dependencies = [ "proc-macro2", "quote", - "syn 2.0.75", + "syn 2.0.77", "wasm-bindgen-backend", "wasm-bindgen-shared", ] @@ -4383,7 +4389,7 @@ checksum = "fa4f8080344d4671fb4e831a13ad1e68092748387dfc4f55e356242fae12ce3e" dependencies = [ "proc-macro2", "quote", - "syn 2.0.75", + "syn 2.0.77", ] [[package]] @@ -4431,9 +4437,9 @@ dependencies = [ [[package]] name = "zstd-sys" -version = "2.0.12+zstd.1.5.6" +version = "2.0.13+zstd.1.5.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0a4e40c320c3cb459d9a9ff6de98cff88f4751ee9275d140e2be94a2b74e4c13" +checksum = "38ff0f21cfee8f97d94cef41359e0c89aa6113028ab0291aa8ca0038995a95aa" dependencies = [ "cc", "pkg-config", diff --git a/datafusion-cli/Cargo.toml b/datafusion-cli/Cargo.toml index f477bad69a2c..f2f52846ab54 100644 --- a/datafusion-cli/Cargo.toml +++ b/datafusion-cli/Cargo.toml @@ -30,9 +30,16 @@ rust-version = "1.76" readme = "README.md" [dependencies] -arrow = { version = "52.2.0" } +arrow = { version = "53.0.0" } async-trait = "0.1.73" aws-config = "1.5.5" +# begin pin aws-sdk crates otherwise CI MSRV check fails +# We can't update these libraries yet as it requires Rust 1.78, which is not available until Nov 2024 +# per https://github.com/apache/datafusion?tab=readme-ov-file#rust-version-compatibility-policy +aws-sdk-sso = "=1.39.0" +aws-sdk-ssooidc = "=1.40.0" +aws-sdk-sts = "=1.39.0" +# end pin aws-sdk crates aws-credential-types = "1.2.0" clap = { version = "4.5.16", features = ["derive", "cargo"] } datafusion = { path = "../datafusion/core", version = "41.0.0", features = [ @@ -49,9 +56,9 @@ dirs = "4.0.0" env_logger = "0.9" futures = "0.3" mimalloc = { version = "0.1", default-features = false } -object_store = { version = "0.10.1", features = ["aws", "gcp", "http"] } +object_store = { version = "0.11.0", features = ["aws", "gcp", "http"] } parking_lot = { version = "0.12" } -parquet = { version = "52.2.0", default-features = false } +parquet = { version = "53.0.0", default-features = false } regex = "1.8" rustyline = "11.0" tokio = { version = "1.24", features = ["macros", "rt", "rt-multi-thread", "sync", "parking_lot", "signal"] } diff --git a/datafusion-cli/src/functions.rs b/datafusion-cli/src/functions.rs index a85c43f3576f..99511e969386 100644 --- a/datafusion-cli/src/functions.rs +++ b/datafusion-cli/src/functions.rs @@ -32,6 +32,7 @@ use datafusion::physical_plan::memory::MemoryExec; use datafusion::physical_plan::ExecutionPlan; use datafusion::scalar::ScalarValue; use parquet::basic::ConvertedType; +use parquet::data_type::{ByteArray, FixedLenByteArray}; use parquet::file::reader::FileReader; use parquet::file::serialized_reader::SerializedFileReader; use parquet::file::statistics::Statistics; @@ -250,49 +251,69 @@ impl TableProvider for ParquetMetadataTable { fn convert_parquet_statistics( value: &Statistics, converted_type: ConvertedType, -) -> (String, String) { +) -> (Option, Option) { match (value, converted_type) { - (Statistics::Boolean(val), _) => (val.min().to_string(), val.max().to_string()), - (Statistics::Int32(val), _) => (val.min().to_string(), val.max().to_string()), - (Statistics::Int64(val), _) => (val.min().to_string(), val.max().to_string()), - (Statistics::Int96(val), _) => (val.min().to_string(), val.max().to_string()), - (Statistics::Float(val), _) => (val.min().to_string(), val.max().to_string()), - (Statistics::Double(val), _) => (val.min().to_string(), val.max().to_string()), - (Statistics::ByteArray(val), ConvertedType::UTF8) => { - let min_bytes = val.min(); - let max_bytes = val.max(); - let min = min_bytes - .as_utf8() - .map(|v| v.to_string()) - .unwrap_or_else(|_| min_bytes.to_string()); - - let max = max_bytes - .as_utf8() - .map(|v| v.to_string()) - .unwrap_or_else(|_| max_bytes.to_string()); - (min, max) - } - (Statistics::ByteArray(val), _) => (val.min().to_string(), val.max().to_string()), - (Statistics::FixedLenByteArray(val), ConvertedType::UTF8) => { - let min_bytes = val.min(); - let max_bytes = val.max(); - let min = min_bytes - .as_utf8() - .map(|v| v.to_string()) - .unwrap_or_else(|_| min_bytes.to_string()); - - let max = max_bytes - .as_utf8() - .map(|v| v.to_string()) - .unwrap_or_else(|_| max_bytes.to_string()); - (min, max) - } - (Statistics::FixedLenByteArray(val), _) => { - (val.min().to_string(), val.max().to_string()) - } + (Statistics::Boolean(val), _) => ( + val.min_opt().map(|v| v.to_string()), + val.max_opt().map(|v| v.to_string()), + ), + (Statistics::Int32(val), _) => ( + val.min_opt().map(|v| v.to_string()), + val.max_opt().map(|v| v.to_string()), + ), + (Statistics::Int64(val), _) => ( + val.min_opt().map(|v| v.to_string()), + val.max_opt().map(|v| v.to_string()), + ), + (Statistics::Int96(val), _) => ( + val.min_opt().map(|v| v.to_string()), + val.max_opt().map(|v| v.to_string()), + ), + (Statistics::Float(val), _) => ( + val.min_opt().map(|v| v.to_string()), + val.max_opt().map(|v| v.to_string()), + ), + (Statistics::Double(val), _) => ( + val.min_opt().map(|v| v.to_string()), + val.max_opt().map(|v| v.to_string()), + ), + (Statistics::ByteArray(val), ConvertedType::UTF8) => ( + byte_array_to_string(val.min_opt()), + byte_array_to_string(val.max_opt()), + ), + (Statistics::ByteArray(val), _) => ( + val.min_opt().map(|v| v.to_string()), + val.max_opt().map(|v| v.to_string()), + ), + (Statistics::FixedLenByteArray(val), ConvertedType::UTF8) => ( + fixed_len_byte_array_to_string(val.min_opt()), + fixed_len_byte_array_to_string(val.max_opt()), + ), + (Statistics::FixedLenByteArray(val), _) => ( + val.min_opt().map(|v| v.to_string()), + val.max_opt().map(|v| v.to_string()), + ), } } +/// Convert to a string if it has utf8 encoding, otherwise print bytes directly +fn byte_array_to_string(val: Option<&ByteArray>) -> Option { + val.map(|v| { + v.as_utf8() + .map(|s| s.to_string()) + .unwrap_or_else(|_e| v.to_string()) + }) +} + +/// Convert to a string if it has utf8 encoding, otherwise print bytes directly +fn fixed_len_byte_array_to_string(val: Option<&FixedLenByteArray>) -> Option { + val.map(|v| { + v.as_utf8() + .map(|s| s.to_string()) + .unwrap_or_else(|_e| v.to_string()) + }) +} + pub struct ParquetMetadataFunc {} impl TableFunctionImpl for ParquetMetadataFunc { @@ -376,17 +397,13 @@ impl TableFunctionImpl for ParquetMetadataFunc { let converted_type = column.column_descr().converted_type(); if let Some(s) = column.statistics() { - let (min_val, max_val) = if s.has_min_max_set() { - let (min_val, max_val) = - convert_parquet_statistics(s, converted_type); - (Some(min_val), Some(max_val)) - } else { - (None, None) - }; + let (min_val, max_val) = + convert_parquet_statistics(s, converted_type); stats_min_arr.push(min_val.clone()); stats_max_arr.push(max_val.clone()); - stats_null_count_arr.push(Some(s.null_count() as i64)); - stats_distinct_count_arr.push(s.distinct_count().map(|c| c as i64)); + stats_null_count_arr.push(s.null_count_opt().map(|c| c as i64)); + stats_distinct_count_arr + .push(s.distinct_count_opt().map(|c| c as i64)); stats_min_value_arr.push(min_val); stats_max_value_arr.push(max_val); } else { diff --git a/datafusion-examples/Cargo.toml b/datafusion-examples/Cargo.toml index 57b9930177d4..f430a87e190d 100644 --- a/datafusion-examples/Cargo.toml +++ b/datafusion-examples/Cargo.toml @@ -73,13 +73,13 @@ mimalloc = { version = "0.1", default-features = false } num_cpus = { workspace = true } object_store = { workspace = true, features = ["aws", "http"] } prost = { workspace = true } -prost-derive = { version = "0.13", default-features = false } +prost-derive = { workspace = true } serde = { version = "1.0.136", features = ["derive"] } serde_json = { workspace = true } tempfile = { workspace = true } test-utils = { path = "../test-utils" } tokio = { workspace = true, features = ["rt-multi-thread", "parking_lot"] } -tonic = "0.11" +tonic = "0.12.1" url = { workspace = true } uuid = "1.7" diff --git a/datafusion/common/Cargo.toml b/datafusion/common/Cargo.toml index 79e20ba1215c..1ac27b40c219 100644 --- a/datafusion/common/Cargo.toml +++ b/datafusion/common/Cargo.toml @@ -61,7 +61,7 @@ num_cpus = { workspace = true } object_store = { workspace = true, optional = true } parquet = { workspace = true, optional = true, default-features = true } paste = "1.0.15" -pyo3 = { version = "0.21.0", optional = true } +pyo3 = { version = "0.22.0", optional = true } sqlparser = { workspace = true } tokio = { workspace = true } diff --git a/datafusion/common/src/scalar/mod.rs b/datafusion/common/src/scalar/mod.rs index 88802af23a4b..22e39404cdb5 100644 --- a/datafusion/common/src/scalar/mod.rs +++ b/datafusion/common/src/scalar/mod.rs @@ -4356,7 +4356,7 @@ mod tests { .strip_backtrace(); assert_eq!( err, - "Arrow error: Compute error: Overflow happened on: 2147483647 - -2147483648" + "Arrow error: Arithmetic overflow: Overflow happened on: 2147483647 - -2147483648" ) } @@ -4377,7 +4377,7 @@ mod tests { .sub_checked(&int_value_2) .unwrap_err() .strip_backtrace(); - assert_eq!(err, "Arrow error: Compute error: Overflow happened on: 9223372036854775807 - -9223372036854775808") + assert_eq!(err, "Arrow error: Arithmetic overflow: Overflow happened on: 9223372036854775807 - -9223372036854775808") } #[test] @@ -5893,7 +5893,7 @@ mod tests { let root_err = err.find_root(); match root_err{ DataFusionError::ArrowError( - ArrowError::ComputeError(_), + ArrowError::ArithmeticOverflow(_), _, ) => {} _ => return Err(err), diff --git a/datafusion/core/src/datasource/file_format/parquet.rs b/datafusion/core/src/datasource/file_format/parquet.rs index 23e765f0f2cd..76e8ad9da559 100644 --- a/datafusion/core/src/datasource/file_format/parquet.rs +++ b/datafusion/core/src/datasource/file_format/parquet.rs @@ -2010,7 +2010,7 @@ mod tests { // test result in int_col let int_col_index = page_index.get(4).unwrap(); - let int_col_offset = offset_index.get(4).unwrap(); + let int_col_offset = offset_index.get(4).unwrap().page_locations(); // 325 pages in int_col assert_eq!(int_col_offset.len(), 325); diff --git a/datafusion/core/src/datasource/physical_plan/parquet/page_filter.rs b/datafusion/core/src/datasource/physical_plan/parquet/page_filter.rs index e4d26a460ecd..4e71993b5153 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet/page_filter.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet/page_filter.rs @@ -392,13 +392,16 @@ impl<'a> PagesPruningStatistics<'a> { trace!("No page offsets for row group {row_group_index}, skipping"); return None; }; - let Some(page_offsets) = row_group_page_offsets.get(parquet_column_index) else { + let Some(offset_index_metadata) = + row_group_page_offsets.get(parquet_column_index) + else { trace!( "No page offsets for column {:?} in row group {row_group_index}, skipping", converter.arrow_field() ); return None; }; + let page_offsets = offset_index_metadata.page_locations(); Some(Self { row_group_index, diff --git a/datafusion/core/src/datasource/physical_plan/parquet/row_group_filter.rs b/datafusion/core/src/datasource/physical_plan/parquet/row_group_filter.rs index 6a6910748fc8..ccd77d90be57 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet/row_group_filter.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet/row_group_filter.rs @@ -487,11 +487,23 @@ mod tests { let schema_descr = get_test_schema_descr(vec![field]); let rgm1 = get_row_group_meta_data( &schema_descr, - vec![ParquetStatistics::int32(Some(1), Some(10), None, 0, false)], + vec![ParquetStatistics::int32( + Some(1), + Some(10), + None, + Some(0), + false, + )], ); let rgm2 = get_row_group_meta_data( &schema_descr, - vec![ParquetStatistics::int32(Some(11), Some(20), None, 0, false)], + vec![ParquetStatistics::int32( + Some(11), + Some(20), + None, + Some(0), + false, + )], ); let metrics = parquet_file_metrics(); @@ -520,11 +532,17 @@ mod tests { let schema_descr = get_test_schema_descr(vec![field]); let rgm1 = get_row_group_meta_data( &schema_descr, - vec![ParquetStatistics::int32(None, None, None, 0, false)], + vec![ParquetStatistics::int32(None, None, None, Some(0), false)], ); let rgm2 = get_row_group_meta_data( &schema_descr, - vec![ParquetStatistics::int32(Some(11), Some(20), None, 0, false)], + vec![ParquetStatistics::int32( + Some(11), + Some(20), + None, + Some(0), + false, + )], ); let metrics = parquet_file_metrics(); // missing statistics for first row group mean that the result from the predicate expression @@ -560,15 +578,15 @@ mod tests { let rgm1 = get_row_group_meta_data( &schema_descr, vec![ - ParquetStatistics::int32(Some(1), Some(10), None, 0, false), - ParquetStatistics::int32(Some(1), Some(10), None, 0, false), + ParquetStatistics::int32(Some(1), Some(10), None, Some(0), false), + ParquetStatistics::int32(Some(1), Some(10), None, Some(0), false), ], ); let rgm2 = get_row_group_meta_data( &schema_descr, vec![ - ParquetStatistics::int32(Some(11), Some(20), None, 0, false), - ParquetStatistics::int32(Some(11), Some(20), None, 0, false), + ParquetStatistics::int32(Some(11), Some(20), None, Some(0), false), + ParquetStatistics::int32(Some(11), Some(20), None, Some(0), false), ], ); @@ -633,16 +651,16 @@ mod tests { let rgm1 = get_row_group_meta_data( &schema_descr, vec![ - ParquetStatistics::int32(Some(-10), Some(-1), None, 0, false), // c2 - ParquetStatistics::int32(Some(1), Some(10), None, 0, false), + ParquetStatistics::int32(Some(-10), Some(-1), None, Some(0), false), // c2 + ParquetStatistics::int32(Some(1), Some(10), None, Some(0), false), ], ); // rg1 has c2 greater than zero, c1 less than zero let rgm2 = get_row_group_meta_data( &schema_descr, vec![ - ParquetStatistics::int32(Some(1), Some(10), None, 0, false), - ParquetStatistics::int32(Some(-10), Some(-1), None, 0, false), + ParquetStatistics::int32(Some(1), Some(10), None, Some(0), false), + ParquetStatistics::int32(Some(-10), Some(-1), None, Some(0), false), ], ); @@ -669,15 +687,15 @@ mod tests { let rgm1 = get_row_group_meta_data( &schema_descr, vec![ - ParquetStatistics::int32(Some(1), Some(10), None, 0, false), - ParquetStatistics::boolean(Some(false), Some(true), None, 0, false), + ParquetStatistics::int32(Some(1), Some(10), None, Some(0), false), + ParquetStatistics::boolean(Some(false), Some(true), None, Some(0), false), ], ); let rgm2 = get_row_group_meta_data( &schema_descr, vec![ - ParquetStatistics::int32(Some(11), Some(20), None, 0, false), - ParquetStatistics::boolean(Some(false), Some(true), None, 1, false), + ParquetStatistics::int32(Some(11), Some(20), None, Some(0), false), + ParquetStatistics::boolean(Some(false), Some(true), None, Some(1), false), ], ); vec![rgm1, rgm2] @@ -775,7 +793,7 @@ mod tests { Some(100), Some(600), None, - 0, + Some(0), false, )], ); @@ -783,13 +801,25 @@ mod tests { &schema_descr, // [0.1, 0.2] // c1 > 5, this row group will not be included in the results. - vec![ParquetStatistics::int32(Some(10), Some(20), None, 0, false)], + vec![ParquetStatistics::int32( + Some(10), + Some(20), + None, + Some(0), + false, + )], ); let rgm3 = get_row_group_meta_data( &schema_descr, // [1, None] // c1 > 5, this row group can not be filtered out, so will be included in the results. - vec![ParquetStatistics::int32(Some(100), None, None, 0, false)], + vec![ParquetStatistics::int32( + Some(100), + None, + None, + Some(0), + false, + )], ); let metrics = parquet_file_metrics(); let mut row_groups = RowGroupAccessPlanFilter::new(ParquetAccessPlan::new_all(3)); @@ -837,7 +867,7 @@ mod tests { Some(100), Some(600), None, - 0, + Some(0), false, )], ); @@ -845,30 +875,62 @@ mod tests { &schema_descr, // [10, 20] // c1 > 5, this row group will be included in the results. - vec![ParquetStatistics::int32(Some(10), Some(20), None, 0, false)], + vec![ParquetStatistics::int32( + Some(10), + Some(20), + None, + Some(0), + false, + )], ); let rgm3 = get_row_group_meta_data( &schema_descr, // [0, 2] // c1 > 5, this row group will not be included in the results. - vec![ParquetStatistics::int32(Some(0), Some(2), None, 0, false)], + vec![ParquetStatistics::int32( + Some(0), + Some(2), + None, + Some(0), + false, + )], ); let rgm4 = get_row_group_meta_data( &schema_descr, // [None, 2] - // c1 > 5, this row group can not be filtered out, so will be included in the results. - vec![ParquetStatistics::int32(None, Some(2), None, 0, false)], + // c1 > 5, this row group will also not be included in the results + // (the min value is unknown, but the max value is 2, so no values can be greater than 5) + vec![ParquetStatistics::int32( + None, + Some(2), + None, + Some(0), + false, + )], + ); + let rgm5 = get_row_group_meta_data( + &schema_descr, + // [2, None] + // c1 > 5, this row group must be included + // (the min value is 2, but the max value is unknown, so it may have values greater than 5) + vec![ParquetStatistics::int32( + Some(2), + None, + None, + Some(0), + false, + )], ); let metrics = parquet_file_metrics(); - let mut row_groups = RowGroupAccessPlanFilter::new(ParquetAccessPlan::new_all(4)); + let mut row_groups = RowGroupAccessPlanFilter::new(ParquetAccessPlan::new_all(5)); row_groups.prune_by_statistics( &schema, &schema_descr, - &[rgm1, rgm2, rgm3, rgm4], + &[rgm1, rgm2, rgm3, rgm4, rgm5], &pruning_predicate, &metrics, ); - assert_pruned(row_groups, ExpectedPruning::Some(vec![0, 1, 3])); + assert_pruned(row_groups, ExpectedPruning::Some(vec![0, 1, 4])); } #[test] fn row_group_pruning_predicate_decimal_type3() { @@ -896,19 +958,25 @@ mod tests { Some(600), Some(800), None, - 0, + Some(0), false, )], ); let rgm2 = get_row_group_meta_data( &schema_descr, // [0.1, 0.2] - vec![ParquetStatistics::int64(Some(10), Some(20), None, 0, false)], + vec![ParquetStatistics::int64( + Some(10), + Some(20), + None, + Some(0), + false, + )], ); let rgm3 = get_row_group_meta_data( &schema_descr, // [0.1, 0.2] - vec![ParquetStatistics::int64(None, None, None, 0, false)], + vec![ParquetStatistics::int64(None, None, None, Some(0), false)], ); let metrics = parquet_file_metrics(); let mut row_groups = RowGroupAccessPlanFilter::new(ParquetAccessPlan::new_all(3)); @@ -957,7 +1025,7 @@ mod tests { 8000i128.to_be_bytes().to_vec(), ))), None, - 0, + Some(0), false, )], ); @@ -973,7 +1041,7 @@ mod tests { 20000i128.to_be_bytes().to_vec(), ))), None, - 0, + Some(0), false, )], ); @@ -981,7 +1049,11 @@ mod tests { let rgm3 = get_row_group_meta_data( &schema_descr, vec![ParquetStatistics::fixed_len_byte_array( - None, None, None, 0, false, + None, + None, + None, + Some(0), + false, )], ); let metrics = parquet_file_metrics(); @@ -1027,7 +1099,7 @@ mod tests { // 80.00 Some(ByteArray::from(8000i128.to_be_bytes().to_vec())), None, - 0, + Some(0), false, )], ); @@ -1039,13 +1111,19 @@ mod tests { // 200.00 Some(ByteArray::from(20000i128.to_be_bytes().to_vec())), None, - 0, + Some(0), false, )], ); let rgm3 = get_row_group_meta_data( &schema_descr, - vec![ParquetStatistics::byte_array(None, None, None, 0, false)], + vec![ParquetStatistics::byte_array( + None, + None, + None, + Some(0), + false, + )], ); let metrics = parquet_file_metrics(); let mut row_groups = RowGroupAccessPlanFilter::new(ParquetAccessPlan::new_all(3)); diff --git a/datafusion/functions/src/regex/regexpreplace.rs b/datafusion/functions/src/regex/regexpreplace.rs index d28c6cd36d65..0b0f7287e1ec 100644 --- a/datafusion/functions/src/regex/regexpreplace.rs +++ b/datafusion/functions/src/regex/regexpreplace.rs @@ -401,8 +401,7 @@ fn _regexp_replace_static_pattern_replace( DataType::Utf8View => { let string_view_array = as_string_view_array(&args[0])?; - let mut builder = StringViewBuilder::with_capacity(string_view_array.len()) - .with_block_size(1024 * 1024 * 2); + let mut builder = StringViewBuilder::with_capacity(string_view_array.len()); for val in string_view_array.iter() { if let Some(val) = val { diff --git a/datafusion/physical-expr-common/src/binary_view_map.rs b/datafusion/physical-expr-common/src/binary_view_map.rs index 18bc6801aa60..bdcf7bbacc69 100644 --- a/datafusion/physical-expr-common/src/binary_view_map.rs +++ b/datafusion/physical-expr-common/src/binary_view_map.rs @@ -149,7 +149,7 @@ where output_type, map: hashbrown::raw::RawTable::with_capacity(INITIAL_MAP_CAPACITY), map_size: 0, - builder: GenericByteViewBuilder::new().with_block_size(2 * 1024 * 1024), + builder: GenericByteViewBuilder::new(), random_state: RandomState::new(), hashes_buffer: vec![], null: None, diff --git a/datafusion/physical-plan/src/coalesce/mod.rs b/datafusion/physical-plan/src/coalesce/mod.rs index ce5a1e53abfd..46875fae94fc 100644 --- a/datafusion/physical-plan/src/coalesce/mod.rs +++ b/datafusion/physical-plan/src/coalesce/mod.rs @@ -248,7 +248,7 @@ fn gc_string_view_batch(batch: &RecordBatch) -> RecordBatch { // See https://github.com/apache/arrow-rs/issues/6094 for more details. let mut builder = StringViewBuilder::with_capacity(s.len()); if ideal_buffer_size > 0 { - builder = builder.with_block_size(ideal_buffer_size as u32); + builder = builder.with_fixed_block_size(ideal_buffer_size as u32); } for v in s.iter() { @@ -580,7 +580,8 @@ mod tests { impl StringViewTest { /// Create a `StringViewArray` with the parameters specified in this struct fn build(self) -> StringViewArray { - let mut builder = StringViewBuilder::with_capacity(100).with_block_size(8192); + let mut builder = + StringViewBuilder::with_capacity(100).with_fixed_block_size(8192); loop { for &v in self.strings.iter() { builder.append_option(v); diff --git a/datafusion/proto-common/Cargo.toml b/datafusion/proto-common/Cargo.toml index 33a7ecd2daab..7ba503171520 100644 --- a/datafusion/proto-common/Cargo.toml +++ b/datafusion/proto-common/Cargo.toml @@ -44,7 +44,7 @@ arrow = { workspace = true } chrono = { workspace = true } datafusion-common = { workspace = true } object_store = { workspace = true } -pbjson = { version = "0.6.0", optional = true } +pbjson = { workspace = true, optional = true } prost = { workspace = true } serde = { version = "1.0", optional = true } serde_json = { workspace = true, optional = true } diff --git a/datafusion/proto-common/gen/Cargo.toml b/datafusion/proto-common/gen/Cargo.toml index 54ec0e44694b..cca49dba7ed3 100644 --- a/datafusion/proto-common/gen/Cargo.toml +++ b/datafusion/proto-common/gen/Cargo.toml @@ -34,5 +34,5 @@ workspace = true [dependencies] # Pin these dependencies so that the generated output is deterministic -pbjson-build = "=0.6.2" -prost-build = "=0.12.6" +pbjson-build = "=0.7.0" +prost-build = "=0.13.1" diff --git a/datafusion/proto/Cargo.toml b/datafusion/proto/Cargo.toml index 2804ed019b61..32678246c005 100644 --- a/datafusion/proto/Cargo.toml +++ b/datafusion/proto/Cargo.toml @@ -52,7 +52,7 @@ datafusion-common = { workspace = true, default-features = true } datafusion-expr = { workspace = true } datafusion-proto-common = { workspace = true } object_store = { workspace = true } -pbjson = { version = "0.6.0", optional = true } +pbjson = { workspace = true, optional = true } prost = { workspace = true } serde = { version = "1.0", optional = true } serde_json = { workspace = true, optional = true } diff --git a/datafusion/proto/gen/Cargo.toml b/datafusion/proto/gen/Cargo.toml index 401c51c94563..1dc5f7e0dddc 100644 --- a/datafusion/proto/gen/Cargo.toml +++ b/datafusion/proto/gen/Cargo.toml @@ -34,5 +34,5 @@ workspace = true [dependencies] # Pin these dependencies so that the generated output is deterministic -pbjson-build = "=0.6.2" -prost-build = "=0.12.6" +pbjson-build = "=0.7.0" +prost-build = "=0.13.1" diff --git a/datafusion/proto/src/generated/datafusion_proto_common.rs b/datafusion/proto/src/generated/datafusion_proto_common.rs index ebc05718a458..3d7b1007b04e 100644 --- a/datafusion/proto/src/generated/datafusion_proto_common.rs +++ b/datafusion/proto/src/generated/datafusion_proto_common.rs @@ -45,10 +45,10 @@ pub struct ParquetFormat { pub options: ::core::option::Option, } #[allow(clippy::derive_partial_eq_without_eq)] -#[derive(Clone, PartialEq, ::prost::Message)] +#[derive(Clone, Copy, PartialEq, ::prost::Message)] pub struct AvroFormat {} #[allow(clippy::derive_partial_eq_without_eq)] -#[derive(Clone, PartialEq, ::prost::Message)] +#[derive(Clone, Copy, PartialEq, ::prost::Message)] pub struct NdJsonFormat { #[prost(message, optional, tag = "1")] pub options: ::core::option::Option, @@ -89,10 +89,10 @@ pub struct Constraints { pub constraints: ::prost::alloc::vec::Vec, } #[allow(clippy::derive_partial_eq_without_eq)] -#[derive(Clone, PartialEq, ::prost::Message)] +#[derive(Clone, Copy, PartialEq, ::prost::Message)] pub struct AvroOptions {} #[allow(clippy::derive_partial_eq_without_eq)] -#[derive(Clone, PartialEq, ::prost::Message)] +#[derive(Clone, Copy, PartialEq, ::prost::Message)] pub struct ArrowOptions {} #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] @@ -137,7 +137,7 @@ pub struct Timestamp { pub timezone: ::prost::alloc::string::String, } #[allow(clippy::derive_partial_eq_without_eq)] -#[derive(Clone, PartialEq, ::prost::Message)] +#[derive(Clone, Copy, PartialEq, ::prost::Message)] pub struct Decimal { #[prost(uint32, tag = "3")] pub precision: u32, @@ -145,7 +145,7 @@ pub struct Decimal { pub scale: i32, } #[allow(clippy::derive_partial_eq_without_eq)] -#[derive(Clone, PartialEq, ::prost::Message)] +#[derive(Clone, Copy, PartialEq, ::prost::Message)] pub struct Decimal256Type { #[prost(uint32, tag = "3")] pub precision: u32, @@ -223,7 +223,7 @@ pub mod scalar_nested_value { } } #[allow(clippy::derive_partial_eq_without_eq)] -#[derive(Clone, PartialEq, ::prost::Message)] +#[derive(Clone, Copy, PartialEq, ::prost::Message)] pub struct ScalarTime32Value { #[prost(oneof = "scalar_time32_value::Value", tags = "1, 2")] pub value: ::core::option::Option, @@ -231,7 +231,7 @@ pub struct ScalarTime32Value { /// Nested message and enum types in `ScalarTime32Value`. pub mod scalar_time32_value { #[allow(clippy::derive_partial_eq_without_eq)] - #[derive(Clone, PartialEq, ::prost::Oneof)] + #[derive(Clone, Copy, PartialEq, ::prost::Oneof)] pub enum Value { #[prost(int32, tag = "1")] Time32SecondValue(i32), @@ -240,7 +240,7 @@ pub mod scalar_time32_value { } } #[allow(clippy::derive_partial_eq_without_eq)] -#[derive(Clone, PartialEq, ::prost::Message)] +#[derive(Clone, Copy, PartialEq, ::prost::Message)] pub struct ScalarTime64Value { #[prost(oneof = "scalar_time64_value::Value", tags = "1, 2")] pub value: ::core::option::Option, @@ -248,7 +248,7 @@ pub struct ScalarTime64Value { /// Nested message and enum types in `ScalarTime64Value`. pub mod scalar_time64_value { #[allow(clippy::derive_partial_eq_without_eq)] - #[derive(Clone, PartialEq, ::prost::Oneof)] + #[derive(Clone, Copy, PartialEq, ::prost::Oneof)] pub enum Value { #[prost(int64, tag = "1")] Time64MicrosecondValue(i64), @@ -267,7 +267,7 @@ pub struct ScalarTimestampValue { /// Nested message and enum types in `ScalarTimestampValue`. pub mod scalar_timestamp_value { #[allow(clippy::derive_partial_eq_without_eq)] - #[derive(Clone, PartialEq, ::prost::Oneof)] + #[derive(Clone, Copy, PartialEq, ::prost::Oneof)] pub enum Value { #[prost(int64, tag = "1")] TimeMicrosecondValue(i64), @@ -288,7 +288,7 @@ pub struct ScalarDictionaryValue { pub value: ::core::option::Option<::prost::alloc::boxed::Box>, } #[allow(clippy::derive_partial_eq_without_eq)] -#[derive(Clone, PartialEq, ::prost::Message)] +#[derive(Clone, Copy, PartialEq, ::prost::Message)] pub struct IntervalDayTimeValue { #[prost(int32, tag = "1")] pub days: i32, @@ -296,7 +296,7 @@ pub struct IntervalDayTimeValue { pub milliseconds: i32, } #[allow(clippy::derive_partial_eq_without_eq)] -#[derive(Clone, PartialEq, ::prost::Message)] +#[derive(Clone, Copy, PartialEq, ::prost::Message)] pub struct IntervalMonthDayNanoValue { #[prost(int32, tag = "1")] pub months: i32, @@ -558,10 +558,10 @@ pub mod arrow_type { /// } /// } #[allow(clippy::derive_partial_eq_without_eq)] -#[derive(Clone, PartialEq, ::prost::Message)] +#[derive(Clone, Copy, PartialEq, ::prost::Message)] pub struct EmptyMessage {} #[allow(clippy::derive_partial_eq_without_eq)] -#[derive(Clone, PartialEq, ::prost::Message)] +#[derive(Clone, Copy, PartialEq, ::prost::Message)] pub struct JsonWriterOptions { #[prost(enumeration = "CompressionTypeVariant", tag = "1")] pub compression: i32, @@ -655,7 +655,7 @@ pub struct CsvOptions { } /// Options controlling CSV format #[allow(clippy::derive_partial_eq_without_eq)] -#[derive(Clone, PartialEq, ::prost::Message)] +#[derive(Clone, Copy, PartialEq, ::prost::Message)] pub struct JsonOptions { /// Compression type #[prost(enumeration = "CompressionTypeVariant", tag = "1")] @@ -720,7 +720,7 @@ pub struct ParquetColumnOptions { /// Nested message and enum types in `ParquetColumnOptions`. pub mod parquet_column_options { #[allow(clippy::derive_partial_eq_without_eq)] - #[derive(Clone, PartialEq, ::prost::Oneof)] + #[derive(Clone, Copy, PartialEq, ::prost::Oneof)] pub enum BloomFilterEnabledOpt { #[prost(bool, tag = "1")] BloomFilterEnabled(bool), @@ -732,7 +732,7 @@ pub mod parquet_column_options { Encoding(::prost::alloc::string::String), } #[allow(clippy::derive_partial_eq_without_eq)] - #[derive(Clone, PartialEq, ::prost::Oneof)] + #[derive(Clone, Copy, PartialEq, ::prost::Oneof)] pub enum DictionaryEnabledOpt { #[prost(bool, tag = "3")] DictionaryEnabled(bool), @@ -750,19 +750,19 @@ pub mod parquet_column_options { StatisticsEnabled(::prost::alloc::string::String), } #[allow(clippy::derive_partial_eq_without_eq)] - #[derive(Clone, PartialEq, ::prost::Oneof)] + #[derive(Clone, Copy, PartialEq, ::prost::Oneof)] pub enum BloomFilterFppOpt { #[prost(double, tag = "6")] BloomFilterFpp(f64), } #[allow(clippy::derive_partial_eq_without_eq)] - #[derive(Clone, PartialEq, ::prost::Oneof)] + #[derive(Clone, Copy, PartialEq, ::prost::Oneof)] pub enum BloomFilterNdvOpt { #[prost(uint64, tag = "7")] BloomFilterNdv(u64), } #[allow(clippy::derive_partial_eq_without_eq)] - #[derive(Clone, PartialEq, ::prost::Oneof)] + #[derive(Clone, Copy, PartialEq, ::prost::Oneof)] pub enum MaxStatisticsSizeOpt { #[prost(uint32, tag = "8")] MaxStatisticsSize(u32), @@ -857,7 +857,7 @@ pub struct ParquetOptions { /// Nested message and enum types in `ParquetOptions`. pub mod parquet_options { #[allow(clippy::derive_partial_eq_without_eq)] - #[derive(Clone, PartialEq, ::prost::Oneof)] + #[derive(Clone, Copy, PartialEq, ::prost::Oneof)] pub enum MetadataSizeHintOpt { #[prost(uint64, tag = "4")] MetadataSizeHint(u64), @@ -869,7 +869,7 @@ pub mod parquet_options { Compression(::prost::alloc::string::String), } #[allow(clippy::derive_partial_eq_without_eq)] - #[derive(Clone, PartialEq, ::prost::Oneof)] + #[derive(Clone, Copy, PartialEq, ::prost::Oneof)] pub enum DictionaryEnabledOpt { #[prost(bool, tag = "11")] DictionaryEnabled(bool), @@ -881,13 +881,13 @@ pub mod parquet_options { StatisticsEnabled(::prost::alloc::string::String), } #[allow(clippy::derive_partial_eq_without_eq)] - #[derive(Clone, PartialEq, ::prost::Oneof)] + #[derive(Clone, Copy, PartialEq, ::prost::Oneof)] pub enum MaxStatisticsSizeOpt { #[prost(uint64, tag = "14")] MaxStatisticsSize(u64), } #[allow(clippy::derive_partial_eq_without_eq)] - #[derive(Clone, PartialEq, ::prost::Oneof)] + #[derive(Clone, Copy, PartialEq, ::prost::Oneof)] pub enum ColumnIndexTruncateLengthOpt { #[prost(uint64, tag = "17")] ColumnIndexTruncateLength(u64), @@ -899,13 +899,13 @@ pub mod parquet_options { Encoding(::prost::alloc::string::String), } #[allow(clippy::derive_partial_eq_without_eq)] - #[derive(Clone, PartialEq, ::prost::Oneof)] + #[derive(Clone, Copy, PartialEq, ::prost::Oneof)] pub enum BloomFilterFppOpt { #[prost(double, tag = "21")] BloomFilterFpp(f64), } #[allow(clippy::derive_partial_eq_without_eq)] - #[derive(Clone, PartialEq, ::prost::Oneof)] + #[derive(Clone, Copy, PartialEq, ::prost::Oneof)] pub enum BloomFilterNdvOpt { #[prost(uint64, tag = "22")] BloomFilterNdv(u64), diff --git a/datafusion/proto/src/generated/pbjson.rs b/datafusion/proto/src/generated/pbjson.rs index cff58d3ddc4a..1c433c2572c4 100644 --- a/datafusion/proto/src/generated/pbjson.rs +++ b/datafusion/proto/src/generated/pbjson.rs @@ -12,6 +12,7 @@ impl serde::Serialize for AggLimit { let mut struct_ser = serializer.serialize_struct("datafusion.AggLimit", len)?; if self.limit != 0 { #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] struct_ser.serialize_field("limit", ToString::to_string(&self.limit).as_str())?; } struct_ser.end() @@ -613,6 +614,7 @@ impl serde::Serialize for AggregateUdfExprNode { } if let Some(v) = self.fun_definition.as_ref() { #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] struct_ser.serialize_field("funDefinition", pbjson::private::base64::encode(&v).as_str())?; } struct_ser.end() @@ -2348,6 +2350,7 @@ impl serde::Serialize for CopyToNode { } if !self.file_type.is_empty() { #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] struct_ser.serialize_field("fileType", pbjson::private::base64::encode(&self.file_type).as_str())?; } if !self.partition_by.is_empty() { @@ -3953,6 +3956,7 @@ impl serde::Serialize for CustomTableScanNode { } if !self.custom_table_data.is_empty() { #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] struct_ser.serialize_field("customTableData", pbjson::private::base64::encode(&self.custom_table_data).as_str())?; } struct_ser.end() @@ -5037,10 +5041,12 @@ impl serde::Serialize for FileRange { let mut struct_ser = serializer.serialize_struct("datafusion.FileRange", len)?; if self.start != 0 { #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] struct_ser.serialize_field("start", ToString::to_string(&self.start).as_str())?; } if self.end != 0 { #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] struct_ser.serialize_field("end", ToString::to_string(&self.end).as_str())?; } struct_ser.end() @@ -5922,6 +5928,7 @@ impl serde::Serialize for GlobalLimitExecNode { } if self.fetch != 0 { #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] struct_ser.serialize_field("fetch", ToString::to_string(&self.fetch).as_str())?; } struct_ser.end() @@ -6357,6 +6364,7 @@ impl serde::Serialize for HashRepartition { } if self.partition_count != 0 { #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] struct_ser.serialize_field("partitionCount", ToString::to_string(&self.partition_count).as_str())?; } struct_ser.end() @@ -8409,10 +8417,12 @@ impl serde::Serialize for LimitNode { } if self.skip != 0 { #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] struct_ser.serialize_field("skip", ToString::to_string(&self.skip).as_str())?; } if self.fetch != 0 { #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] struct_ser.serialize_field("fetch", ToString::to_string(&self.fetch).as_str())?; } struct_ser.end() @@ -9860,6 +9870,7 @@ impl serde::Serialize for LogicalExtensionNode { let mut struct_ser = serializer.serialize_struct("datafusion.LogicalExtensionNode", len)?; if !self.node.is_empty() { #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] struct_ser.serialize_field("node", pbjson::private::base64::encode(&self.node).as_str())?; } if !self.inputs.is_empty() { @@ -11982,14 +11993,17 @@ impl serde::Serialize for PartitionStats { let mut struct_ser = serializer.serialize_struct("datafusion.PartitionStats", len)?; if self.num_rows != 0 { #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] struct_ser.serialize_field("numRows", ToString::to_string(&self.num_rows).as_str())?; } if self.num_batches != 0 { #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] struct_ser.serialize_field("numBatches", ToString::to_string(&self.num_batches).as_str())?; } if self.num_bytes != 0 { #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] struct_ser.serialize_field("numBytes", ToString::to_string(&self.num_bytes).as_str())?; } if !self.column_stats.is_empty() { @@ -12146,10 +12160,12 @@ impl serde::Serialize for PartitionedFile { } if self.size != 0 { #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] struct_ser.serialize_field("size", ToString::to_string(&self.size).as_str())?; } if self.last_modified_ns != 0 { #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] struct_ser.serialize_field("lastModifiedNs", ToString::to_string(&self.last_modified_ns).as_str())?; } if !self.partition_values.is_empty() { @@ -12314,6 +12330,7 @@ impl serde::Serialize for Partitioning { match v { partitioning::PartitionMethod::RoundRobin(v) => { #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] struct_ser.serialize_field("roundRobin", ToString::to_string(&v).as_str())?; } partitioning::PartitionMethod::Hash(v) => { @@ -12321,6 +12338,7 @@ impl serde::Serialize for Partitioning { } partitioning::PartitionMethod::Unknown(v) => { #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] struct_ser.serialize_field("unknown", ToString::to_string(&v).as_str())?; } } @@ -12462,6 +12480,7 @@ impl serde::Serialize for PhysicalAggregateExprNode { } if let Some(v) = self.fun_definition.as_ref() { #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] struct_ser.serialize_field("funDefinition", pbjson::private::base64::encode(&v).as_str())?; } if let Some(v) = self.aggregate_function.as_ref() { @@ -13644,6 +13663,7 @@ impl serde::Serialize for PhysicalExtensionExprNode { let mut struct_ser = serializer.serialize_struct("datafusion.PhysicalExtensionExprNode", len)?; if !self.expr.is_empty() { #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] struct_ser.serialize_field("expr", pbjson::private::base64::encode(&self.expr).as_str())?; } if !self.inputs.is_empty() { @@ -13755,6 +13775,7 @@ impl serde::Serialize for PhysicalExtensionNode { let mut struct_ser = serializer.serialize_struct("datafusion.PhysicalExtensionNode", len)?; if !self.node.is_empty() { #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] struct_ser.serialize_field("node", pbjson::private::base64::encode(&self.node).as_str())?; } if !self.inputs.is_empty() { @@ -13869,6 +13890,7 @@ impl serde::Serialize for PhysicalHashRepartition { } if self.partition_count != 0 { #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] struct_ser.serialize_field("partitionCount", ToString::to_string(&self.partition_count).as_str())?; } struct_ser.end() @@ -15085,6 +15107,7 @@ impl serde::Serialize for PhysicalScalarUdfNode { } if let Some(v) = self.fun_definition.as_ref() { #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] struct_ser.serialize_field("funDefinition", pbjson::private::base64::encode(&v).as_str())?; } if let Some(v) = self.return_type.as_ref() { @@ -15687,6 +15710,7 @@ impl serde::Serialize for PhysicalWindowExprNode { } if let Some(v) = self.fun_definition.as_ref() { #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] struct_ser.serialize_field("funDefinition", pbjson::private::base64::encode(&v).as_str())?; } if let Some(v) = self.window_function.as_ref() { @@ -16901,6 +16925,7 @@ impl serde::Serialize for RepartitionNode { match v { repartition_node::PartitionMethod::RoundRobin(v) => { #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] struct_ser.serialize_field("roundRobin", ToString::to_string(&v).as_str())?; } repartition_node::PartitionMethod::Hash(v) => { @@ -17123,6 +17148,7 @@ impl serde::Serialize for ScalarUdfExprNode { } if let Some(v) = self.fun_definition.as_ref() { #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] struct_ser.serialize_field("funDefinition", pbjson::private::base64::encode(&v).as_str())?; } struct_ser.end() @@ -17691,6 +17717,7 @@ impl serde::Serialize for SortExecNode { } if self.fetch != 0 { #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] struct_ser.serialize_field("fetch", ToString::to_string(&self.fetch).as_str())?; } if self.preserve_partitioning { @@ -18052,6 +18079,7 @@ impl serde::Serialize for SortNode { } if self.fetch != 0 { #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] struct_ser.serialize_field("fetch", ToString::to_string(&self.fetch).as_str())?; } struct_ser.end() @@ -18180,6 +18208,7 @@ impl serde::Serialize for SortPreservingMergeExecNode { } if self.fetch != 0 { #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] struct_ser.serialize_field("fetch", ToString::to_string(&self.fetch).as_str())?; } struct_ser.end() @@ -19625,6 +19654,7 @@ impl serde::Serialize for ValuesNode { let mut struct_ser = serializer.serialize_struct("datafusion.ValuesNode", len)?; if self.n_cols != 0 { #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] struct_ser.serialize_field("nCols", ToString::to_string(&self.n_cols).as_str())?; } if !self.values_list.is_empty() { @@ -20299,6 +20329,7 @@ impl serde::Serialize for WindowExprNode { } if let Some(v) = self.fun_definition.as_ref() { #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] struct_ser.serialize_field("funDefinition", pbjson::private::base64::encode(&v).as_str())?; } if let Some(v) = self.window_function.as_ref() { diff --git a/datafusion/proto/src/generated/prost.rs b/datafusion/proto/src/generated/prost.rs index 2ce8004e3248..dbcf7672a48c 100644 --- a/datafusion/proto/src/generated/prost.rs +++ b/datafusion/proto/src/generated/prost.rs @@ -239,7 +239,7 @@ pub struct HashRepartition { pub partition_count: u64, } #[allow(clippy::derive_partial_eq_without_eq)] -#[derive(Clone, PartialEq, ::prost::Message)] +#[derive(Clone, Copy, PartialEq, ::prost::Message)] pub struct EmptyRelationNode { #[prost(bool, tag = "1")] pub produce_one_row: bool, @@ -443,7 +443,7 @@ pub struct UnnestNode { pub options: ::core::option::Option, } #[allow(clippy::derive_partial_eq_without_eq)] -#[derive(Clone, PartialEq, ::prost::Message)] +#[derive(Clone, Copy, PartialEq, ::prost::Message)] pub struct UnnestOptions { #[prost(bool, tag = "1")] pub preserve_nulls: bool, @@ -912,7 +912,7 @@ pub struct WindowFrameBound { pub bound_value: ::core::option::Option, } #[allow(clippy::derive_partial_eq_without_eq)] -#[derive(Clone, PartialEq, ::prost::Message)] +#[derive(Clone, Copy, PartialEq, ::prost::Message)] pub struct FixedSizeBinary { #[prost(int32, tag = "1")] pub length: i32, @@ -1474,7 +1474,7 @@ pub struct FileGroup { pub files: ::prost::alloc::vec::Vec, } #[allow(clippy::derive_partial_eq_without_eq)] -#[derive(Clone, PartialEq, ::prost::Message)] +#[derive(Clone, Copy, PartialEq, ::prost::Message)] pub struct ScanLimit { /// wrap into a message to make it optional #[prost(uint32, tag = "1")] @@ -1721,7 +1721,7 @@ pub struct MaybePhysicalSortExprs { pub sort_expr: ::prost::alloc::vec::Vec, } #[allow(clippy::derive_partial_eq_without_eq)] -#[derive(Clone, PartialEq, ::prost::Message)] +#[derive(Clone, Copy, PartialEq, ::prost::Message)] pub struct AggLimit { /// wrap into a message to make it optional #[prost(uint64, tag = "1")] @@ -1877,7 +1877,7 @@ pub struct JoinFilter { pub schema: ::core::option::Option, } #[allow(clippy::derive_partial_eq_without_eq)] -#[derive(Clone, PartialEq, ::prost::Message)] +#[derive(Clone, Copy, PartialEq, ::prost::Message)] pub struct ColumnIndex { #[prost(uint32, tag = "1")] pub index: u32, @@ -1903,7 +1903,7 @@ pub struct PartitionedFile { pub statistics: ::core::option::Option, } #[allow(clippy::derive_partial_eq_without_eq)] -#[derive(Clone, PartialEq, ::prost::Message)] +#[derive(Clone, Copy, PartialEq, ::prost::Message)] pub struct FileRange { #[prost(int64, tag = "1")] pub start: i64, diff --git a/datafusion/sql/src/unparser/expr.rs b/datafusion/sql/src/unparser/expr.rs index fe34d87bfeae..516833a39f1e 100644 --- a/datafusion/sql/src/unparser/expr.rs +++ b/datafusion/sql/src/unparser/expr.rs @@ -2083,49 +2083,49 @@ mod tests { "1 YEAR 1 MONTH 1 DAY 3 HOUR 10 MINUTE 20 SECOND", ), IntervalStyle::PostgresVerbose, - r#"INTERVAL '0 YEARS 13 MONS 1 DAYS 3 HOURS 10 MINS 20.000000000 SECS'"#, + r#"INTERVAL '13 MONS 1 DAYS 3 HOURS 10 MINS 20.000000000 SECS'"#, ), ( interval_month_day_nano_lit("1.5 MONTH"), IntervalStyle::PostgresVerbose, - r#"INTERVAL '0 YEARS 1 MONS 15 DAYS 0 HOURS 0 MINS 0.000000000 SECS'"#, + r#"INTERVAL '1 MONS 15 DAYS'"#, ), ( interval_month_day_nano_lit("-3 MONTH"), IntervalStyle::PostgresVerbose, - r#"INTERVAL '0 YEARS -3 MONS 0 DAYS 0 HOURS 0 MINS 0.000000000 SECS'"#, + r#"INTERVAL '-3 MONS'"#, ), ( interval_month_day_nano_lit("1 MONTH") .add(interval_month_day_nano_lit("1 DAY")), IntervalStyle::PostgresVerbose, - r#"(INTERVAL '0 YEARS 1 MONS 0 DAYS 0 HOURS 0 MINS 0.000000000 SECS' + INTERVAL '0 YEARS 0 MONS 1 DAYS 0 HOURS 0 MINS 0.000000000 SECS')"#, + r#"(INTERVAL '1 MONS' + INTERVAL '1 DAYS')"#, ), ( interval_month_day_nano_lit("1 MONTH") .sub(interval_month_day_nano_lit("1 DAY")), IntervalStyle::PostgresVerbose, - r#"(INTERVAL '0 YEARS 1 MONS 0 DAYS 0 HOURS 0 MINS 0.000000000 SECS' - INTERVAL '0 YEARS 0 MONS 1 DAYS 0 HOURS 0 MINS 0.000000000 SECS')"#, + r#"(INTERVAL '1 MONS' - INTERVAL '1 DAYS')"#, ), ( interval_datetime_lit("10 DAY 1 HOUR 10 MINUTE 20 SECOND"), IntervalStyle::PostgresVerbose, - r#"INTERVAL '0 YEARS 0 MONS 10 DAYS 1 HOURS 10 MINS 20.000 SECS'"#, + r#"INTERVAL '10 DAYS 1 HOURS 10 MINS 20.000 SECS'"#, ), ( interval_datetime_lit("10 DAY 1.5 HOUR 10 MINUTE 20 SECOND"), IntervalStyle::PostgresVerbose, - r#"INTERVAL '0 YEARS 0 MONS 10 DAYS 1 HOURS 40 MINS 20.000 SECS'"#, + r#"INTERVAL '10 DAYS 1 HOURS 40 MINS 20.000 SECS'"#, ), ( interval_year_month_lit("1 YEAR 1 MONTH"), IntervalStyle::PostgresVerbose, - r#"INTERVAL '1 YEARS 1 MONS 0 DAYS 0 HOURS 0 MINS 0.00 SECS'"#, + r#"INTERVAL '1 YEARS 1 MONS'"#, ), ( interval_year_month_lit("1.5 YEAR 1 MONTH"), IntervalStyle::PostgresVerbose, - r#"INTERVAL '1 YEARS 7 MONS 0 DAYS 0 HOURS 0 MINS 0.00 SECS'"#, + r#"INTERVAL '1 YEARS 7 MONS'"#, ), ( interval_year_month_lit("1 YEAR 1 MONTH"), diff --git a/datafusion/sql/tests/cases/plan_to_sql.rs b/datafusion/sql/tests/cases/plan_to_sql.rs index cdc7bef06afd..d4e189f5f66a 100644 --- a/datafusion/sql/tests/cases/plan_to_sql.rs +++ b/datafusion/sql/tests/cases/plan_to_sql.rs @@ -611,7 +611,7 @@ fn sql_round_trip(query: &str, expect: &str) { fn test_interval_lhs_eq() { sql_round_trip( "select interval '2 seconds' = interval '2 seconds'", - "SELECT (INTERVAL '0 YEARS 0 MONS 0 DAYS 0 HOURS 0 MINS 2.000000000 SECS' = INTERVAL '0 YEARS 0 MONS 0 DAYS 0 HOURS 0 MINS 2.000000000 SECS')", + "SELECT (INTERVAL '2.000000000 SECS' = INTERVAL '2.000000000 SECS')", ); } @@ -619,6 +619,6 @@ fn test_interval_lhs_eq() { fn test_interval_lhs_lt() { sql_round_trip( "select interval '2 seconds' < interval '2 seconds'", - "SELECT (INTERVAL '0 YEARS 0 MONS 0 DAYS 0 HOURS 0 MINS 2.000000000 SECS' < INTERVAL '0 YEARS 0 MONS 0 DAYS 0 HOURS 0 MINS 2.000000000 SECS')", + "SELECT (INTERVAL '2.000000000 SECS' < INTERVAL '2.000000000 SECS')", ); } diff --git a/datafusion/sqllogictest/test_files/aggregate.slt b/datafusion/sqllogictest/test_files/aggregate.slt index c52445c561ee..83f4e4f03055 100644 --- a/datafusion/sqllogictest/test_files/aggregate.slt +++ b/datafusion/sqllogictest/test_files/aggregate.slt @@ -1916,7 +1916,7 @@ from values (interval '2 month 15 days'), (interval '-2 month') ---- -Interval(MonthDayNano) 0 years -2 mons 0 days 0 hours 0 mins 0.000000000 secs 0 years 2 mons 15 days 0 hours 0 mins 0.000000000 secs +Interval(MonthDayNano) -2 mons 2 mons 15 days # aggregate Interval(DayTime) min/max query T?? @@ -1927,7 +1927,7 @@ from values (arrow_cast('-3 minutes', 'Interval(DayTime)')), (arrow_cast('30 minutes', 'Interval(DayTime)')); ---- -Interval(DayTime) 0 years 0 mons 0 days 0 hours -3 mins 0.000 secs 0 years 0 mons 0 days 1 hours 0 mins 0.000 secs +Interval(DayTime) -3 mins 1 hours # aggregate Interval(YearMonth) min/max query T?? @@ -1938,7 +1938,7 @@ from values (arrow_cast('13 months', 'Interval(YearMonth)')), (arrow_cast('1 year', 'Interval(YearMonth)')); ---- -Interval(YearMonth) -1 years 0 mons 0 days 0 hours 0 mins 0.00 secs 1 years 1 mons 0 days 0 hours 0 mins 0.00 secs +Interval(YearMonth) -1 years 0 mons 1 years 1 mons # aggregate query II diff --git a/datafusion/sqllogictest/test_files/arrow_typeof.slt b/datafusion/sqllogictest/test_files/arrow_typeof.slt index bae6dc33c8cf..77b10b41ccb3 100644 --- a/datafusion/sqllogictest/test_files/arrow_typeof.slt +++ b/datafusion/sqllogictest/test_files/arrow_typeof.slt @@ -290,22 +290,22 @@ query ? --- select arrow_cast(interval '30 minutes', 'Interval(MonthDayNano)'); ---- -0 years 0 mons 0 days 0 hours 30 mins 0.000000000 secs +30 mins query ? select arrow_cast('30 minutes', 'Interval(DayTime)'); ---- -0 years 0 mons 0 days 0 hours 30 mins 0.000 secs +30 mins query ? select arrow_cast('1 year 5 months', 'Interval(YearMonth)'); ---- -1 years 5 mons 0 days 0 hours 0 mins 0.00 secs +1 years 5 mons query ? select arrow_cast('30 minutes', 'Interval(MonthDayNano)'); ---- -0 years 0 mons 0 days 0 hours 30 mins 0.000000000 secs +30 mins ## Duration @@ -432,5 +432,7 @@ MyAwesomeString Utf8View # Fails until we update to use the arrow-cast release with support for casting utf8 types to BinaryView # refer to merge commit https://github.com/apache/arrow-rs/commit/4bd737dab2aa17aca200259347909d48ed793ba1 -query error DataFusion error: This feature is not implemented: Unsupported CAST from Utf8 to BinaryView +query ?T select arrow_cast('MyAwesomeString', 'BinaryView'), arrow_typeof(arrow_cast('MyAwesomeString', 'BinaryView')) +---- +4d79417765736f6d65537472696e67 BinaryView diff --git a/datafusion/sqllogictest/test_files/ddl.slt b/datafusion/sqllogictest/test_files/ddl.slt index 7164425fc0f5..21edb458fe56 100644 --- a/datafusion/sqllogictest/test_files/ddl.slt +++ b/datafusion/sqllogictest/test_files/ddl.slt @@ -710,7 +710,7 @@ create table t (i interval, x int) as values (interval '5 days 3 nanoseconds', C query ?I select * from t; ---- -0 years 0 mons 5 days 0 hours 0 mins 0.000000003 secs 1 +5 days 0.000000003 secs 1 statement ok drop table t; diff --git a/datafusion/sqllogictest/test_files/expr.slt b/datafusion/sqllogictest/test_files/expr.slt index 81ae60f3ba93..002e8db2132d 100644 --- a/datafusion/sqllogictest/test_files/expr.slt +++ b/datafusion/sqllogictest/test_files/expr.slt @@ -122,203 +122,203 @@ SELECT query ? SELECT interval '1' ---- -0 years 0 mons 0 days 0 hours 0 mins 1.000000000 secs +1.000000000 secs query ? SELECT interval '1 second' ---- -0 years 0 mons 0 days 0 hours 0 mins 1.000000000 secs +1.000000000 secs query ? SELECT interval '500 milliseconds' ---- -0 years 0 mons 0 days 0 hours 0 mins 0.500000000 secs +0.500000000 secs query ? SELECT interval '5 second' ---- -0 years 0 mons 0 days 0 hours 0 mins 5.000000000 secs +5.000000000 secs query ? SELECT interval '0.5 minute' ---- -0 years 0 mons 0 days 0 hours 0 mins 30.000000000 secs +30.000000000 secs query ? SELECT interval '.5 minute' ---- -0 years 0 mons 0 days 0 hours 0 mins 30.000000000 secs +30.000000000 secs query ? SELECT interval '5 minute' ---- -0 years 0 mons 0 days 0 hours 5 mins 0.000000000 secs +5 mins query ? SELECT interval '5 minute 1 second' ---- -0 years 0 mons 0 days 0 hours 5 mins 1.000000000 secs +5 mins 1.000000000 secs query ? SELECT interval '1 hour' ---- -0 years 0 mons 0 days 1 hours 0 mins 0.000000000 secs +1 hours query ? SELECT interval '5 hour' ---- -0 years 0 mons 0 days 5 hours 0 mins 0.000000000 secs +5 hours query ? SELECT interval '1 day' ---- -0 years 0 mons 1 days 0 hours 0 mins 0.000000000 secs +1 days query ? SELECT interval '1 week' ---- -0 years 0 mons 7 days 0 hours 0 mins 0.000000000 secs +7 days query ? SELECT interval '2 weeks' ---- -0 years 0 mons 14 days 0 hours 0 mins 0.000000000 secs +14 days query ? SELECT interval '1 day 1' ---- -0 years 0 mons 1 days 0 hours 0 mins 1.000000000 secs +1 days 1.000000000 secs query ? SELECT interval '0.5' ---- -0 years 0 mons 0 days 0 hours 0 mins 0.500000000 secs +0.500000000 secs query ? SELECT interval '0.5 day 1' ---- -0 years 0 mons 0 days 12 hours 0 mins 1.000000000 secs +12 hours 1.000000000 secs query ? SELECT interval '0.49 day' ---- -0 years 0 mons 0 days 11 hours 45 mins 36.000000000 secs +11 hours 45 mins 36.000000000 secs query ? SELECT interval '0.499 day' ---- -0 years 0 mons 0 days 11 hours 58 mins 33.600000000 secs +11 hours 58 mins 33.600000000 secs query ? SELECT interval '0.4999 day' ---- -0 years 0 mons 0 days 11 hours 59 mins 51.360000000 secs +11 hours 59 mins 51.360000000 secs query ? SELECT interval '0.49999 day' ---- -0 years 0 mons 0 days 11 hours 59 mins 59.136000000 secs +11 hours 59 mins 59.136000000 secs query ? SELECT interval '0.49999999999 day' ---- -0 years 0 mons 0 days 11 hours 59 mins 59.999999136 secs +11 hours 59 mins 59.999999136 secs query ? SELECT interval '5 day' ---- -0 years 0 mons 5 days 0 hours 0 mins 0.000000000 secs +5 days # Hour is ignored, this matches PostgreSQL query ? SELECT interval '5 day' hour ---- -0 years 0 mons 5 days 0 hours 0 mins 0.000000000 secs +5 days query ? SELECT interval '5 day 4 hours 3 minutes 2 seconds 100 milliseconds' ---- -0 years 0 mons 5 days 4 hours 3 mins 2.100000000 secs +5 days 4 hours 3 mins 2.100000000 secs query ? SELECT interval '0.5 month' ---- -0 years 0 mons 15 days 0 hours 0 mins 0.000000000 secs +15 days query ? SELECT interval '0.5' month ---- -0 years 0 mons 15 days 0 hours 0 mins 0.000000000 secs +15 days query ? SELECT interval '1 month' ---- -0 years 1 mons 0 days 0 hours 0 mins 0.000000000 secs +1 mons query ? SELECT interval '1' MONTH ---- -0 years 1 mons 0 days 0 hours 0 mins 0.000000000 secs +1 mons query ? SELECT interval '5 month' ---- -0 years 5 mons 0 days 0 hours 0 mins 0.000000000 secs +5 mons query ? SELECT interval '13 month' ---- -0 years 13 mons 0 days 0 hours 0 mins 0.000000000 secs +13 mons query ? SELECT interval '0.5 year' ---- -0 years 6 mons 0 days 0 hours 0 mins 0.000000000 secs +6 mons query ? SELECT interval '1 year' ---- -0 years 12 mons 0 days 0 hours 0 mins 0.000000000 secs +12 mons query ? SELECT interval '1 decade' ---- -0 years 120 mons 0 days 0 hours 0 mins 0.000000000 secs +120 mons query ? SELECT interval '2 decades' ---- -0 years 240 mons 0 days 0 hours 0 mins 0.000000000 secs +240 mons query ? SELECT interval '1 century' ---- -0 years 1200 mons 0 days 0 hours 0 mins 0.000000000 secs +1200 mons query ? SELECT interval '2 year' ---- -0 years 24 mons 0 days 0 hours 0 mins 0.000000000 secs +24 mons query ? SELECT interval '1 year 1 day' ---- -0 years 12 mons 1 days 0 hours 0 mins 0.000000000 secs +12 mons 1 days query ? SELECT interval '1 year 1 day 1 hour' ---- -0 years 12 mons 1 days 1 hours 0 mins 0.000000000 secs +12 mons 1 days 1 hours query ? SELECT interval '1 year 1 day 1 hour 1 minute' ---- -0 years 12 mons 1 days 1 hours 1 mins 0.000000000 secs +12 mons 1 days 1 hours 1 mins query ? SELECT interval '1 year 1 day 1 hour 1 minute 1 second' ---- -0 years 12 mons 1 days 1 hours 1 mins 1.000000000 secs +12 mons 1 days 1 hours 1 mins 1.000000000 secs query I SELECT ascii('') diff --git a/datafusion/sqllogictest/test_files/interval.slt b/datafusion/sqllogictest/test_files/interval.slt index afb262cf95a5..077f38d5d5bb 100644 --- a/datafusion/sqllogictest/test_files/interval.slt +++ b/datafusion/sqllogictest/test_files/interval.slt @@ -45,250 +45,250 @@ Interval(MonthDayNano) Interval(MonthDayNano) query ? select interval '5' years ---- -0 years 0 mons 0 days 0 hours 0 mins 5.000000000 secs +5.000000000 secs # check all different kinds of intervals query ? select interval '5' year ---- -0 years 60 mons 0 days 0 hours 0 mins 0.000000000 secs +60 mons query ? select interval '5' month ---- -0 years 5 mons 0 days 0 hours 0 mins 0.000000000 secs +5 mons query ? select interval '5' months ---- -0 years 0 mons 0 days 0 hours 0 mins 5.000000000 secs +5.000000000 secs query ? select interval '5' week ---- -0 years 0 mons 35 days 0 hours 0 mins 0.000000000 secs +35 days query ? select interval '5' day ---- -0 years 0 mons 5 days 0 hours 0 mins 0.000000000 secs +5 days query ? select interval '5' hour ---- -0 years 0 mons 0 days 5 hours 0 mins 0.000000000 secs +5 hours ## This seems wrong (5 mons) query ? select interval '5' hours ---- -0 years 0 mons 0 days 0 hours 0 mins 5.000000000 secs +5.000000000 secs query ? select interval '5' minute ---- -0 years 0 mons 0 days 0 hours 5 mins 0.000000000 secs +5 mins query ? select interval '5' second ---- -0 years 0 mons 0 days 0 hours 0 mins 5.000000000 secs +5.000000000 secs query ? select interval '5' millisecond ---- -0 years 0 mons 0 days 0 hours 0 mins 0.005000000 secs +0.005000000 secs query ? select interval '5' milliseconds ---- -0 years 0 mons 0 days 0 hours 0 mins 0.005000000 secs +0.005000000 secs query ? select interval '5' microsecond ---- -0 years 0 mons 0 days 0 hours 0 mins 0.000005000 secs +0.000005000 secs query ? select interval '5' microseconds ---- -0 years 0 mons 0 days 0 hours 0 mins 0.000005000 secs +0.000005000 secs query ? select interval '5' nanosecond ---- -0 years 0 mons 0 days 0 hours 0 mins 0.000000005 secs +0.000000005 secs query ? select interval '5' nanoseconds ---- -0 years 0 mons 0 days 0 hours 0 mins 0.000000005 secs +0.000000005 secs query ? select interval '5 YEAR' ---- -0 years 60 mons 0 days 0 hours 0 mins 0.000000000 secs +60 mons query ? select interval '5 MONTH' ---- -0 years 5 mons 0 days 0 hours 0 mins 0.000000000 secs +5 mons query ? select interval '5 WEEK' ---- -0 years 0 mons 35 days 0 hours 0 mins 0.000000000 secs +35 days query ? select interval '5 DAY' ---- -0 years 0 mons 5 days 0 hours 0 mins 0.000000000 secs +5 days query ? select interval '5 HOUR' ---- -0 years 0 mons 0 days 5 hours 0 mins 0.000000000 secs +5 hours query ? select interval '5 HOURS' ---- -0 years 0 mons 0 days 5 hours 0 mins 0.000000000 secs +5 hours query ? select interval '5 MINUTE' ---- -0 years 0 mons 0 days 0 hours 5 mins 0.000000000 secs +5 mins query ? select interval '5 SECOND' ---- -0 years 0 mons 0 days 0 hours 0 mins 5.000000000 secs +5.000000000 secs query ? select interval '5 SECONDS' ---- -0 years 0 mons 0 days 0 hours 0 mins 5.000000000 secs +5.000000000 secs query ? select interval '5 MILLISECOND' ---- -0 years 0 mons 0 days 0 hours 0 mins 0.005000000 secs +0.005000000 secs query ? select interval '5 MILLISECONDS' ---- -0 years 0 mons 0 days 0 hours 0 mins 0.005000000 secs +0.005000000 secs query ? select interval '5 MICROSECOND' ---- -0 years 0 mons 0 days 0 hours 0 mins 0.000005000 secs +0.000005000 secs query ? select interval '5 MICROSECONDS' ---- -0 years 0 mons 0 days 0 hours 0 mins 0.000005000 secs +0.000005000 secs query ? select interval '5 NANOSECOND' ---- -0 years 0 mons 0 days 0 hours 0 mins 0.000000005 secs +0.000000005 secs query ? select interval '5 NANOSECONDS' ---- -0 years 0 mons 0 days 0 hours 0 mins 0.000000005 secs +0.000000005 secs query ? select interval '5 YEAR 5 MONTH 5 DAY 5 HOUR 5 MINUTE 5 SECOND 5 MILLISECOND 5 MICROSECOND 5 NANOSECOND' ---- -0 years 65 mons 5 days 5 hours 5 mins 5.005005005 secs +65 mons 5 days 5 hours 5 mins 5.005005005 secs # Interval with string literal addition query ? select interval '1 month' + '1 month' ---- -0 years 2 mons 0 days 0 hours 0 mins 0.000000000 secs +2 mons # Interval with string literal addition and leading field query ? select interval '1' + '1' month ---- -0 years 2 mons 0 days 0 hours 0 mins 0.000000000 secs +2 mons # Interval with nested string literal addition query ? select interval '1 month' + '1 month' + '1 month' ---- -0 years 3 mons 0 days 0 hours 0 mins 0.000000000 secs +3 mons # Interval with nested string literal addition and leading field query ? select interval '1' + '1' + '1' month ---- -0 years 3 mons 0 days 0 hours 0 mins 0.000000000 secs +3 mons # Interval mega nested string literal addition query ? select interval '1 year' + '1 month' + '1 day' + '1 hour' + '1 minute' + '1 second' + '1 millisecond' + '1 microsecond' + '1 nanosecond' ---- -0 years 13 mons 1 days 1 hours 1 mins 1.001001001 secs +13 mons 1 days 1 hours 1 mins 1.001001001 secs # Interval with string literal subtraction query ? select interval '1 month' - '1 day'; ---- -0 years 1 mons -1 days 0 hours 0 mins 0.000000000 secs +1 mons -1 days # Interval with string literal subtraction and leading field query ? select interval '5' - '1' - '2' year; ---- -0 years 24 mons 0 days 0 hours 0 mins 0.000000000 secs +24 mons # Interval with nested string literal subtraction query ? select interval '1 month' - '1 day' - '1 hour'; ---- -0 years 1 mons -1 days -1 hours 0 mins 0.000000000 secs +1 mons -1 days -1 hours # Interval with nested string literal subtraction and leading field query ? select interval '10' - '1' - '1' month; ---- -0 years 8 mons 0 days 0 hours 0 mins 0.000000000 secs +8 mons # Interval mega nested string literal subtraction query ? select interval '1 year' - '1 month' - '1 day' - '1 hour' - '1 minute' - '1 second' - '1 millisecond' - '1 microsecond' - '1 nanosecond' ---- -0 years 11 mons -1 days -1 hours -1 mins -1.001001001 secs +11 mons -1 days -1 hours -1 mins -1.001001001 secs # Interval with string literal negation and leading field query ? select -interval '5' - '1' - '2' year; ---- -0 years -96 mons 0 days 0 hours 0 mins 0.000000000 secs +-96 mons # Interval with nested string literal negation query ? select -interval '1 month' + '1 day' + '1 hour'; ---- -0 years -1 mons 1 days 1 hours 0 mins 0.000000000 secs +-1 mons 1 days 1 hours # Interval with nested string literal negation and leading field query ? select -interval '10' - '1' - '1' month; ---- -0 years -12 mons 0 days 0 hours 0 mins 0.000000000 secs +-12 mons # Interval mega nested string literal negation query ? select -interval '1 year' - '1 month' - '1 day' - '1 hour' - '1 minute' - '1 second' - '1 millisecond' - '1 microsecond' - '1 nanosecond' ---- -0 years -13 mons -1 days -1 hours -1 mins -1.001001001 secs +-13 mons -1 days -1 hours -1 mins -1.001001001 secs # Interval string literal + date query D @@ -343,7 +343,7 @@ select arrow_typeof(i) from t; ---- -0 years 0 mons 5 days 0 hours 0 mins 0.000000003 secs Interval(MonthDayNano) +5 days 0.000000003 secs Interval(MonthDayNano) statement ok @@ -359,8 +359,8 @@ insert into t values ('6 days 7 nanoseconds'::interval) query ? rowsort select -i from t order by 1; ---- -0 years 0 mons -5 days 0 hours 0 mins -0.000000003 secs -0 years 0 mons -6 days 0 hours 0 mins -0.000000007 secs +-5 days -0.000000003 secs +-6 days -0.000000007 secs query ?T rowsort select @@ -368,8 +368,8 @@ select arrow_typeof(i) from t; ---- -0 years 0 mons 5 days 0 hours 0 mins 0.000000003 secs Interval(MonthDayNano) -0 years 0 mons 6 days 0 hours 0 mins 0.000000007 secs Interval(MonthDayNano) +5 days 0.000000003 secs Interval(MonthDayNano) +6 days 0.000000007 secs Interval(MonthDayNano) statement ok drop table t; diff --git a/datafusion/sqllogictest/test_files/math.slt b/datafusion/sqllogictest/test_files/math.slt index 6884d762612d..eece56942317 100644 --- a/datafusion/sqllogictest/test_files/math.slt +++ b/datafusion/sqllogictest/test_files/math.slt @@ -673,7 +673,7 @@ query error DataFusion error: Arrow error: Compute error: Signed integer overflo select lcm(2, 9223372036854775803); -query error DataFusion error: Arrow error: Compute error: Overflow happened on: 2107754225 \^ 1221660777 +query error DataFusion error: Arrow error: Arithmetic overflow: Overflow happened on: 2107754225 \^ 1221660777 select power(2107754225, 1221660777); # factorial overflow diff --git a/datafusion/sqllogictest/test_files/repartition_scan.slt b/datafusion/sqllogictest/test_files/repartition_scan.slt index 6b9cb521f5f8..4c86312f9e51 100644 --- a/datafusion/sqllogictest/test_files/repartition_scan.slt +++ b/datafusion/sqllogictest/test_files/repartition_scan.slt @@ -61,7 +61,7 @@ logical_plan physical_plan 01)CoalesceBatchesExec: target_batch_size=8192 02)--FilterExec: column1@0 != 42 -03)----ParquetExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:0..104], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:104..208], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:208..312], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:312..414]]}, projection=[column1], predicate=column1@0 != 42, pruning_predicate=CASE WHEN column1_null_count@2 = column1_row_count@3 THEN false ELSE column1_min@0 != 42 OR 42 != column1_max@1 END, required_guarantees=[column1 not in (42)] +03)----ParquetExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:0..87], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:87..174], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:174..261], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:261..347]]}, projection=[column1], predicate=column1@0 != 42, pruning_predicate=CASE WHEN column1_null_count@2 = column1_row_count@3 THEN false ELSE column1_min@0 != 42 OR 42 != column1_max@1 END, required_guarantees=[column1 not in (42)] # disable round robin repartitioning statement ok @@ -77,7 +77,7 @@ logical_plan physical_plan 01)CoalesceBatchesExec: target_batch_size=8192 02)--FilterExec: column1@0 != 42 -03)----ParquetExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:0..104], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:104..208], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:208..312], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:312..414]]}, projection=[column1], predicate=column1@0 != 42, pruning_predicate=CASE WHEN column1_null_count@2 = column1_row_count@3 THEN false ELSE column1_min@0 != 42 OR 42 != column1_max@1 END, required_guarantees=[column1 not in (42)] +03)----ParquetExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:0..87], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:87..174], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:174..261], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:261..347]]}, projection=[column1], predicate=column1@0 != 42, pruning_predicate=CASE WHEN column1_null_count@2 = column1_row_count@3 THEN false ELSE column1_min@0 != 42 OR 42 != column1_max@1 END, required_guarantees=[column1 not in (42)] # enable round robin repartitioning again statement ok @@ -102,7 +102,7 @@ physical_plan 02)--SortExec: expr=[column1@0 ASC NULLS LAST], preserve_partitioning=[true] 03)----CoalesceBatchesExec: target_batch_size=8192 04)------FilterExec: column1@0 != 42 -05)--------ParquetExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/1.parquet:0..205], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/1.parquet:205..405, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:0..5], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:5..210], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:210..414]]}, projection=[column1], predicate=column1@0 != 42, pruning_predicate=CASE WHEN column1_null_count@2 = column1_row_count@3 THEN false ELSE column1_min@0 != 42 OR 42 != column1_max@1 END, required_guarantees=[column1 not in (42)] +05)--------ParquetExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/1.parquet:0..172], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/1.parquet:172..338, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:0..6], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:6..178], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:178..347]]}, projection=[column1], predicate=column1@0 != 42, pruning_predicate=CASE WHEN column1_null_count@2 = column1_row_count@3 THEN false ELSE column1_min@0 != 42 OR 42 != column1_max@1 END, required_guarantees=[column1 not in (42)] ## Read the files as though they are ordered @@ -138,7 +138,7 @@ physical_plan 01)SortPreservingMergeExec: [column1@0 ASC NULLS LAST] 02)--CoalesceBatchesExec: target_batch_size=8192 03)----FilterExec: column1@0 != 42 -04)------ParquetExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/1.parquet:0..202], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:0..207], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:207..414], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/1.parquet:202..405]]}, projection=[column1], output_ordering=[column1@0 ASC NULLS LAST], predicate=column1@0 != 42, pruning_predicate=CASE WHEN column1_null_count@2 = column1_row_count@3 THEN false ELSE column1_min@0 != 42 OR 42 != column1_max@1 END, required_guarantees=[column1 not in (42)] +04)------ParquetExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/1.parquet:0..169], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:0..173], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:173..347], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/1.parquet:169..338]]}, projection=[column1], output_ordering=[column1@0 ASC NULLS LAST], predicate=column1@0 != 42, pruning_predicate=CASE WHEN column1_null_count@2 = column1_row_count@3 THEN false ELSE column1_min@0 != 42 OR 42 != column1_max@1 END, required_guarantees=[column1 not in (42)] # Cleanup statement ok diff --git a/datafusion/sqllogictest/test_files/timestamps.slt b/datafusion/sqllogictest/test_files/timestamps.slt index fb0fd8397f2d..4b11e338da70 100644 --- a/datafusion/sqllogictest/test_files/timestamps.slt +++ b/datafusion/sqllogictest/test_files/timestamps.slt @@ -1509,19 +1509,19 @@ SELECT val, ts1 - ts2 FROM foo ORDER BY ts2 - ts1; query ? SELECT i1 - i2 FROM bar; ---- -0 years 0 mons -1 days 0 hours 0 mins 0.000000000 secs -0 years 2 mons -13 days 0 hours 0 mins 0.000000000 secs -0 years 0 mons 1 days 2 hours 56 mins 0.000000000 secs -0 years 0 mons 1 days 0 hours 0 mins -3.999999993 secs +-1 days +2 mons -13 days +1 days 2 hours 56 mins +1 days -3.999999993 secs # Interval + Interval query ? SELECT i1 + i2 FROM bar; ---- -0 years 0 mons 3 days 0 hours 0 mins 0.000000000 secs -0 years 2 mons 13 days 0 hours 0 mins 0.000000000 secs -0 years 0 mons 1 days 3 hours 4 mins 0.000000000 secs -0 years 0 mons 1 days 0 hours 0 mins 4.000000007 secs +3 days +2 mons 13 days +1 days 3 hours 4 mins +1 days 4.000000007 secs # Timestamp - Interval query P From 7561cbc057ddecea220cc052011d9a8f5e2eef02 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Berkay=20=C5=9Eahin?= <124376117+berkaysynnada@users.noreply.github.com> Date: Thu, 5 Sep 2024 13:58:44 +0300 Subject: [PATCH 092/154] Minor: Update Sanity Checker Error Messages (#12333) * Update sanity_checker.rs * Update datafusion/core/src/physical_optimizer/sanity_checker.rs Co-authored-by: Marco Neumann --------- Co-authored-by: Marco Neumann --- .../src/physical_optimizer/sanity_checker.rs | 30 +++++++++++-------- 1 file changed, 17 insertions(+), 13 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/sanity_checker.rs b/datafusion/core/src/physical_optimizer/sanity_checker.rs index bd80d31224ef..e392105fbcb7 100644 --- a/datafusion/core/src/physical_optimizer/sanity_checker.rs +++ b/datafusion/core/src/physical_optimizer/sanity_checker.rs @@ -120,32 +120,36 @@ pub fn check_plan_sanity( ) -> Result>> { check_finiteness_requirements(plan.clone(), optimizer_options)?; - for (child, child_sort_req, child_dist_req) in izip!( - plan.children().iter(), + for ((idx, child), sort_req, dist_req) in izip!( + plan.children().iter().enumerate(), plan.required_input_ordering().iter(), plan.required_input_distribution().iter() ) { let child_eq_props = child.equivalence_properties(); - if let Some(child_sort_req) = child_sort_req { - if !child_eq_props.ordering_satisfy_requirement(child_sort_req) { - let child_plan_str = get_plan_string(child); + if let Some(sort_req) = sort_req { + if !child_eq_props.ordering_satisfy_requirement(sort_req) { + let plan_str = get_plan_string(&plan); return plan_err!( - "Child: {:?} does not satisfy parent order requirements: {:?}", - child_plan_str, - child_sort_req + "Plan: {:?} does not satisfy order requirements: {:?}. Child-{} order: {:?}", + plan_str, + sort_req, + idx, + child_eq_props.oeq_class ); } } if !child .output_partitioning() - .satisfy(child_dist_req, child_eq_props) + .satisfy(dist_req, child_eq_props) { - let child_plan_str = get_plan_string(child); + let plan_str = get_plan_string(&plan); return plan_err!( - "Child: {:?} does not satisfy parent distribution requirements: {:?}", - child_plan_str, - child_dist_req + "Plan: {:?} does not satisfy distribution requirements: {:?}. Child-{} output partitioning: {:?}", + plan_str, + dist_req, + idx, + child.output_partitioning() ); } } From ab1e3e29402d0ab8467d29665ba090d8300270d5 Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Thu, 5 Sep 2024 13:19:37 +0200 Subject: [PATCH 093/154] Improve & unify validation in LogicalPlan::with_new_exprs (#12264) * Improve & unify validation in LogicalPlan::with_new_exprs When adding new plan node type, `LogicalPlan::with_new_exprs` needs to be updated. Different code branches apply different inputs validation style (no validation, just assert, or assert with messages), so it's unclear which code pattern to follow. This commit unifies the validation and adds it to the branches where there was none. * #inline * rename and return Result * Add doc line --- datafusion/expr/src/logical_plan/plan.rs | 280 ++++++++++++++++------- 1 file changed, 193 insertions(+), 87 deletions(-) diff --git a/datafusion/expr/src/logical_plan/plan.rs b/datafusion/expr/src/logical_plan/plan.rs index d0c1c3b2b3d6..18a624dd9cb2 100644 --- a/datafusion/expr/src/logical_plan/plan.rs +++ b/datafusion/expr/src/logical_plan/plan.rs @@ -799,40 +799,49 @@ impl LogicalPlan { pub fn with_new_exprs( &self, mut expr: Vec, - mut inputs: Vec, + inputs: Vec, ) -> Result { match self { // Since expr may be different than the previous expr, schema of the projection // may change. We need to use try_new method instead of try_new_with_schema method. LogicalPlan::Projection(Projection { .. }) => { - Projection::try_new(expr, Arc::new(inputs.swap_remove(0))) - .map(LogicalPlan::Projection) + let input = self.only_input(inputs)?; + Projection::try_new(expr, Arc::new(input)).map(LogicalPlan::Projection) } LogicalPlan::Dml(DmlStatement { table_name, table_schema, op, .. - }) => Ok(LogicalPlan::Dml(DmlStatement::new( - table_name.clone(), - Arc::clone(table_schema), - op.clone(), - Arc::new(inputs.swap_remove(0)), - ))), + }) => { + self.assert_no_expressions(expr)?; + let input = self.only_input(inputs)?; + Ok(LogicalPlan::Dml(DmlStatement::new( + table_name.clone(), + Arc::clone(table_schema), + op.clone(), + Arc::new(input), + ))) + } LogicalPlan::Copy(CopyTo { input: _, output_url, file_type, options, partition_by, - }) => Ok(LogicalPlan::Copy(CopyTo { - input: Arc::new(inputs.swap_remove(0)), - output_url: output_url.clone(), - file_type: Arc::clone(file_type), - options: options.clone(), - partition_by: partition_by.clone(), - })), + }) => { + self.assert_no_expressions(expr)?; + let input = self.only_input(inputs)?; + Ok(LogicalPlan::Copy(CopyTo { + input: Arc::new(input), + output_url: output_url.clone(), + file_type: Arc::clone(file_type), + options: options.clone(), + partition_by: partition_by.clone(), + })) + } LogicalPlan::Values(Values { schema, .. }) => { + self.assert_no_inputs(inputs)?; Ok(LogicalPlan::Values(Values { schema: Arc::clone(schema), values: expr @@ -842,54 +851,63 @@ impl LogicalPlan { })) } LogicalPlan::Filter { .. } => { - assert_eq!(1, expr.len()); - let predicate = expr.pop().unwrap(); + let predicate = self.only_expr(expr)?; + let input = self.only_input(inputs)?; - Filter::try_new(predicate, Arc::new(inputs.swap_remove(0))) - .map(LogicalPlan::Filter) + Filter::try_new(predicate, Arc::new(input)).map(LogicalPlan::Filter) } LogicalPlan::Repartition(Repartition { partitioning_scheme, .. }) => match partitioning_scheme { Partitioning::RoundRobinBatch(n) => { + self.assert_no_expressions(expr)?; + let input = self.only_input(inputs)?; Ok(LogicalPlan::Repartition(Repartition { partitioning_scheme: Partitioning::RoundRobinBatch(*n), - input: Arc::new(inputs.swap_remove(0)), + input: Arc::new(input), + })) + } + Partitioning::Hash(_, n) => { + let input = self.only_input(inputs)?; + Ok(LogicalPlan::Repartition(Repartition { + partitioning_scheme: Partitioning::Hash(expr, *n), + input: Arc::new(input), })) } - Partitioning::Hash(_, n) => Ok(LogicalPlan::Repartition(Repartition { - partitioning_scheme: Partitioning::Hash(expr, *n), - input: Arc::new(inputs.swap_remove(0)), - })), Partitioning::DistributeBy(_) => { + let input = self.only_input(inputs)?; Ok(LogicalPlan::Repartition(Repartition { partitioning_scheme: Partitioning::DistributeBy(expr), - input: Arc::new(inputs.swap_remove(0)), + input: Arc::new(input), })) } }, LogicalPlan::Window(Window { window_expr, .. }) => { assert_eq!(window_expr.len(), expr.len()); - Window::try_new(expr, Arc::new(inputs.swap_remove(0))) - .map(LogicalPlan::Window) + let input = self.only_input(inputs)?; + Window::try_new(expr, Arc::new(input)).map(LogicalPlan::Window) } LogicalPlan::Aggregate(Aggregate { group_expr, .. }) => { + let input = self.only_input(inputs)?; // group exprs are the first expressions let agg_expr = expr.split_off(group_expr.len()); - Aggregate::try_new(Arc::new(inputs.swap_remove(0)), expr, agg_expr) + Aggregate::try_new(Arc::new(input), expr, agg_expr) .map(LogicalPlan::Aggregate) } LogicalPlan::Sort(Sort { expr: sort_expr, fetch, .. - }) => Ok(LogicalPlan::Sort(Sort { - expr: replace_sort_expressions(sort_expr.clone(), expr), - input: Arc::new(inputs.swap_remove(0)), - fetch: *fetch, - })), + }) => { + let input = self.only_input(inputs)?; + Ok(LogicalPlan::Sort(Sort { + expr: replace_sort_expressions(sort_expr.clone(), expr), + input: Arc::new(input), + fetch: *fetch, + })) + } LogicalPlan::Join(Join { join_type, join_constraint, @@ -897,8 +915,8 @@ impl LogicalPlan { null_equals_null, .. }) => { - let schema = - build_join_schema(inputs[0].schema(), inputs[1].schema(), join_type)?; + let (left, right) = self.only_two_inputs(inputs)?; + let schema = build_join_schema(left.schema(), right.schema(), join_type)?; let equi_expr_count = on.len(); assert!(expr.len() >= equi_expr_count); @@ -927,8 +945,8 @@ impl LogicalPlan { }).collect::>>()?; Ok(LogicalPlan::Join(Join { - left: Arc::new(inputs.swap_remove(0)), - right: Arc::new(inputs.swap_remove(0)), + left: Arc::new(left), + right: Arc::new(right), join_type: *join_type, join_constraint: *join_constraint, on: new_on, @@ -938,28 +956,34 @@ impl LogicalPlan { })) } LogicalPlan::CrossJoin(_) => { - let left = inputs.swap_remove(0); - let right = inputs.swap_remove(0); + self.assert_no_expressions(expr)?; + let (left, right) = self.only_two_inputs(inputs)?; LogicalPlanBuilder::from(left).cross_join(right)?.build() } LogicalPlan::Subquery(Subquery { outer_ref_columns, .. }) => { - let subquery = LogicalPlanBuilder::from(inputs.swap_remove(0)).build()?; + self.assert_no_expressions(expr)?; + let input = self.only_input(inputs)?; + let subquery = LogicalPlanBuilder::from(input).build()?; Ok(LogicalPlan::Subquery(Subquery { subquery: Arc::new(subquery), outer_ref_columns: outer_ref_columns.clone(), })) } LogicalPlan::SubqueryAlias(SubqueryAlias { alias, .. }) => { - SubqueryAlias::try_new(Arc::new(inputs.swap_remove(0)), alias.clone()) + self.assert_no_expressions(expr)?; + let input = self.only_input(inputs)?; + SubqueryAlias::try_new(Arc::new(input), alias.clone()) .map(LogicalPlan::SubqueryAlias) } LogicalPlan::Limit(Limit { skip, fetch, .. }) => { + self.assert_no_expressions(expr)?; + let input = self.only_input(inputs)?; Ok(LogicalPlan::Limit(Limit { skip: *skip, fetch: *fetch, - input: Arc::new(inputs.swap_remove(0)), + input: Arc::new(input), })) } LogicalPlan::Ddl(DdlStatement::CreateMemoryTable(CreateMemoryTable { @@ -968,31 +992,40 @@ impl LogicalPlan { or_replace, column_defaults, .. - })) => Ok(LogicalPlan::Ddl(DdlStatement::CreateMemoryTable( - CreateMemoryTable { - input: Arc::new(inputs.swap_remove(0)), - constraints: Constraints::empty(), - name: name.clone(), - if_not_exists: *if_not_exists, - or_replace: *or_replace, - column_defaults: column_defaults.clone(), - }, - ))), + })) => { + self.assert_no_expressions(expr)?; + let input = self.only_input(inputs)?; + Ok(LogicalPlan::Ddl(DdlStatement::CreateMemoryTable( + CreateMemoryTable { + input: Arc::new(input), + constraints: Constraints::empty(), + name: name.clone(), + if_not_exists: *if_not_exists, + or_replace: *or_replace, + column_defaults: column_defaults.clone(), + }, + ))) + } LogicalPlan::Ddl(DdlStatement::CreateView(CreateView { name, or_replace, definition, .. - })) => Ok(LogicalPlan::Ddl(DdlStatement::CreateView(CreateView { - input: Arc::new(inputs.swap_remove(0)), - name: name.clone(), - or_replace: *or_replace, - definition: definition.clone(), - }))), + })) => { + self.assert_no_expressions(expr)?; + let input = self.only_input(inputs)?; + Ok(LogicalPlan::Ddl(DdlStatement::CreateView(CreateView { + input: Arc::new(input), + name: name.clone(), + or_replace: *or_replace, + definition: definition.clone(), + }))) + } LogicalPlan::Extension(e) => Ok(LogicalPlan::Extension(Extension { node: e.node.with_exprs_and_inputs(expr, inputs)?, })), LogicalPlan::Union(Union { schema, .. }) => { + self.assert_no_expressions(expr)?; let input_schema = inputs[0].schema(); // If inputs are not pruned do not change schema. let schema = if schema.fields().len() == input_schema.fields().len() { @@ -1007,12 +1040,17 @@ impl LogicalPlan { } LogicalPlan::Distinct(distinct) => { let distinct = match distinct { - Distinct::All(_) => Distinct::All(Arc::new(inputs.swap_remove(0))), + Distinct::All(_) => { + self.assert_no_expressions(expr)?; + let input = self.only_input(inputs)?; + Distinct::All(Arc::new(input)) + } Distinct::On(DistinctOn { on_expr, select_expr, .. }) => { + let input = self.only_input(inputs)?; let sort_expr = expr.split_off(on_expr.len() + select_expr.len()); let select_expr = expr.split_off(on_expr.len()); assert!(sort_expr.is_empty(), "with_new_exprs for Distinct does not support sort expressions"); @@ -1020,7 +1058,7 @@ impl LogicalPlan { expr, select_expr, None, // no sort expressions accepted - Arc::new(inputs.swap_remove(0)), + Arc::new(input), )?) } }; @@ -1028,30 +1066,31 @@ impl LogicalPlan { } LogicalPlan::RecursiveQuery(RecursiveQuery { name, is_distinct, .. - }) => Ok(LogicalPlan::RecursiveQuery(RecursiveQuery { - name: name.clone(), - static_term: Arc::new(inputs.swap_remove(0)), - recursive_term: Arc::new(inputs.swap_remove(0)), - is_distinct: *is_distinct, - })), + }) => { + self.assert_no_expressions(expr)?; + let (static_term, recursive_term) = self.only_two_inputs(inputs)?; + Ok(LogicalPlan::RecursiveQuery(RecursiveQuery { + name: name.clone(), + static_term: Arc::new(static_term), + recursive_term: Arc::new(recursive_term), + is_distinct: *is_distinct, + })) + } LogicalPlan::Analyze(a) => { - assert!(expr.is_empty()); - assert_eq!(inputs.len(), 1); + self.assert_no_expressions(expr)?; + let input = self.only_input(inputs)?; Ok(LogicalPlan::Analyze(Analyze { verbose: a.verbose, schema: Arc::clone(&a.schema), - input: Arc::new(inputs.swap_remove(0)), + input: Arc::new(input), })) } LogicalPlan::Explain(e) => { - assert!( - expr.is_empty(), - "Invalid EXPLAIN command. Expression should empty" - ); - assert_eq!(inputs.len(), 1, "Invalid EXPLAIN command. Inputs are empty"); + self.assert_no_expressions(expr)?; + let input = self.only_input(inputs)?; Ok(LogicalPlan::Explain(Explain { verbose: e.verbose, - plan: Arc::new(inputs.swap_remove(0)), + plan: Arc::new(input), stringified_plans: e.stringified_plans.clone(), schema: Arc::clone(&e.schema), logical_optimization_succeeded: e.logical_optimization_succeeded, @@ -1059,13 +1098,17 @@ impl LogicalPlan { } LogicalPlan::Prepare(Prepare { name, data_types, .. - }) => Ok(LogicalPlan::Prepare(Prepare { - name: name.clone(), - data_types: data_types.clone(), - input: Arc::new(inputs.swap_remove(0)), - })), + }) => { + self.assert_no_expressions(expr)?; + let input = self.only_input(inputs)?; + Ok(LogicalPlan::Prepare(Prepare { + name: name.clone(), + data_types: data_types.clone(), + input: Arc::new(input), + })) + } LogicalPlan::TableScan(ts) => { - assert!(inputs.is_empty(), "{self:?} should have no inputs"); + self.assert_no_inputs(inputs)?; Ok(LogicalPlan::TableScan(TableScan { filters: expr, ..ts.clone() @@ -1073,26 +1116,89 @@ impl LogicalPlan { } LogicalPlan::EmptyRelation(_) | LogicalPlan::Ddl(_) - | LogicalPlan::Statement(_) => { + | LogicalPlan::Statement(_) + | LogicalPlan::DescribeTable(_) => { // All of these plan types have no inputs / exprs so should not be called - assert!(expr.is_empty(), "{self:?} should have no exprs"); - assert!(inputs.is_empty(), "{self:?} should have no inputs"); + self.assert_no_expressions(expr)?; + self.assert_no_inputs(inputs)?; Ok(self.clone()) } - LogicalPlan::DescribeTable(_) => Ok(self.clone()), LogicalPlan::Unnest(Unnest { exec_columns: columns, options, .. }) => { + self.assert_no_expressions(expr)?; + let input = self.only_input(inputs)?; // Update schema with unnested column type. - let input = inputs.swap_remove(0); let new_plan = unnest_with_options(input, columns.clone(), options.clone())?; Ok(new_plan) } } } + + /// Helper for [Self::with_new_exprs] to use when no expressions are expected. + #[inline] + #[allow(clippy::needless_pass_by_value)] // expr is moved intentionally to ensure it's not used again + fn assert_no_expressions(&self, expr: Vec) -> Result<()> { + if !expr.is_empty() { + return internal_err!("{self:?} should have no exprs, got {:?}", expr); + } + Ok(()) + } + + /// Helper for [Self::with_new_exprs] to use when no inputs are expected. + #[inline] + #[allow(clippy::needless_pass_by_value)] // inputs is moved intentionally to ensure it's not used again + fn assert_no_inputs(&self, inputs: Vec) -> Result<()> { + if !inputs.is_empty() { + return internal_err!("{self:?} should have no inputs, got: {:?}", inputs); + } + Ok(()) + } + + /// Helper for [Self::with_new_exprs] to use when exactly one expression is expected. + #[inline] + fn only_expr(&self, mut expr: Vec) -> Result { + if expr.len() != 1 { + return internal_err!( + "{self:?} should have exactly one expr, got {:?}", + expr + ); + } + Ok(expr.remove(0)) + } + + /// Helper for [Self::with_new_exprs] to use when exactly one input is expected. + #[inline] + fn only_input(&self, mut inputs: Vec) -> Result { + if inputs.len() != 1 { + return internal_err!( + "{self:?} should have exactly one input, got {:?}", + inputs + ); + } + Ok(inputs.remove(0)) + } + + /// Helper for [Self::with_new_exprs] to use when exactly two inputs are expected. + #[inline] + fn only_two_inputs( + &self, + mut inputs: Vec, + ) -> Result<(LogicalPlan, LogicalPlan)> { + if inputs.len() != 2 { + return internal_err!( + "{self:?} should have exactly two inputs, got {:?}", + inputs + ); + } + let right = inputs.remove(1); + let left = inputs.remove(0); + Ok((left, right)) + } + /// Replaces placeholder param values (like `$1`, `$2`) in [`LogicalPlan`] /// with the specified `param_values`. /// From 008c94210c1342d6183ca3d4936a68623759980b Mon Sep 17 00:00:00 2001 From: Jax Liu Date: Thu, 5 Sep 2024 19:20:30 +0800 Subject: [PATCH 094/154] Support the custom terminator for the CSV file format (#12263) * add terminator config to CsvConfig * add test and fix missing builder * remove the debug message and fix the doc * support EscapedStringLiteral * add create external table tests * refactor the error assertion * add issue reference --- datafusion-examples/examples/csv_opener.rs | 1 + datafusion/common/src/config.rs | 13 ++ .../core/src/datasource/file_format/csv.rs | 8 + .../src/datasource/file_format/options.rs | 10 ++ .../core/src/datasource/physical_plan/csv.rs | 139 +++++++++++++++++- datafusion/core/tests/data/cr_terminator.csv | 1 + .../data/newlines_in_values_cr_terminator.csv | 1 + .../proto/datafusion_common.proto | 1 + datafusion/proto-common/src/from_proto/mod.rs | 1 + .../proto-common/src/generated/pbjson.rs | 20 +++ .../proto-common/src/generated/prost.rs | 3 + datafusion/proto-common/src/to_proto/mod.rs | 1 + .../src/generated/datafusion_proto_common.rs | 3 + .../proto/src/logical_plan/file_formats.rs | 6 + datafusion/sql/src/utils.rs | 2 +- .../sqllogictest/test_files/csv_files.slt | 21 +++ 16 files changed, 228 insertions(+), 3 deletions(-) create mode 100644 datafusion/core/tests/data/cr_terminator.csv create mode 100644 datafusion/core/tests/data/newlines_in_values_cr_terminator.csv diff --git a/datafusion-examples/examples/csv_opener.rs b/datafusion-examples/examples/csv_opener.rs index 1f45026a214d..e7b7ead109bc 100644 --- a/datafusion-examples/examples/csv_opener.rs +++ b/datafusion-examples/examples/csv_opener.rs @@ -47,6 +47,7 @@ async fn main() -> Result<()> { true, b',', b'"', + None, object_store, Some(b'#'), ); diff --git a/datafusion/common/src/config.rs b/datafusion/common/src/config.rs index 7c247103f6e7..19978e102cc8 100644 --- a/datafusion/common/src/config.rs +++ b/datafusion/common/src/config.rs @@ -1604,6 +1604,7 @@ config_namespace! { pub has_header: Option, default = None pub delimiter: u8, default = b',' pub quote: u8, default = b'"' + pub terminator: Option, default = None pub escape: Option, default = None pub double_quote: Option, default = None /// Specifies whether newlines in (quoted) values are supported. @@ -1672,6 +1673,13 @@ impl CsvOptions { self } + /// The character that terminates a row. + /// - default to None (CRLF) + pub fn with_terminator(mut self, terminator: Option) -> Self { + self.terminator = terminator; + self + } + /// The escape character in a row. /// - default is None pub fn with_escape(mut self, escape: Option) -> Self { @@ -1718,6 +1726,11 @@ impl CsvOptions { self.quote } + /// The terminator character. + pub fn terminator(&self) -> Option { + self.terminator + } + /// The escape character. pub fn escape(&self) -> Option { self.escape diff --git a/datafusion/core/src/datasource/file_format/csv.rs b/datafusion/core/src/datasource/file_format/csv.rs index e43f6ab29abc..99e8f13776fc 100644 --- a/datafusion/core/src/datasource/file_format/csv.rs +++ b/datafusion/core/src/datasource/file_format/csv.rs @@ -247,6 +247,13 @@ impl CsvFormat { self } + /// The character used to indicate the end of a row. + /// - default to None (CRLF) + pub fn with_terminator(mut self, terminator: Option) -> Self { + self.options.terminator = terminator; + self + } + /// Specifies whether newlines in (quoted) values are supported. /// /// Parsing newlines in quoted values may be affected by execution behaviour such as @@ -359,6 +366,7 @@ impl FileFormat for CsvFormat { .with_has_header(has_header) .with_delimeter(self.options.delimiter) .with_quote(self.options.quote) + .with_terminator(self.options.terminator) .with_escape(self.options.escape) .with_comment(self.options.comment) .with_newlines_in_values(newlines_in_values) diff --git a/datafusion/core/src/datasource/file_format/options.rs b/datafusion/core/src/datasource/file_format/options.rs index db90262edbf8..7ef5a2182d1c 100644 --- a/datafusion/core/src/datasource/file_format/options.rs +++ b/datafusion/core/src/datasource/file_format/options.rs @@ -59,6 +59,8 @@ pub struct CsvReadOptions<'a> { pub delimiter: u8, /// An optional quote character. Defaults to `b'"'`. pub quote: u8, + /// An optional terminator character. Defaults to None (CRLF). + pub terminator: Option, /// An optional escape character. Defaults to None. pub escape: Option, /// If enabled, lines beginning with this byte are ignored. @@ -102,6 +104,7 @@ impl<'a> CsvReadOptions<'a> { schema_infer_max_records: DEFAULT_SCHEMA_INFER_MAX_RECORD, delimiter: b',', quote: b'"', + terminator: None, escape: None, newlines_in_values: false, file_extension: DEFAULT_CSV_EXTENSION, @@ -136,6 +139,12 @@ impl<'a> CsvReadOptions<'a> { self } + /// Specify terminator to use for CSV read + pub fn terminator(mut self, terminator: Option) -> Self { + self.terminator = terminator; + self + } + /// Specify delimiter to use for CSV read pub fn escape(mut self, escape: u8) -> Self { self.escape = Some(escape); @@ -511,6 +520,7 @@ impl ReadOptions<'_> for CsvReadOptions<'_> { .with_delimiter(self.delimiter) .with_quote(self.quote) .with_escape(self.escape) + .with_terminator(self.terminator) .with_newlines_in_values(self.newlines_in_values) .with_schema_infer_max_rec(self.schema_infer_max_records) .with_file_compression_type(self.file_compression_type.to_owned()); diff --git a/datafusion/core/src/datasource/physical_plan/csv.rs b/datafusion/core/src/datasource/physical_plan/csv.rs index 5ab32ed36e53..6cd1864deb1d 100644 --- a/datafusion/core/src/datasource/physical_plan/csv.rs +++ b/datafusion/core/src/datasource/physical_plan/csv.rs @@ -77,6 +77,7 @@ pub struct CsvExec { has_header: bool, delimiter: u8, quote: u8, + terminator: Option, escape: Option, comment: Option, newlines_in_values: bool, @@ -98,6 +99,7 @@ pub struct CsvExecBuilder { has_header: bool, delimiter: u8, quote: u8, + terminator: Option, escape: Option, comment: Option, newlines_in_values: bool, @@ -112,6 +114,7 @@ impl CsvExecBuilder { has_header: false, delimiter: b',', quote: b'"', + terminator: None, escape: None, comment: None, newlines_in_values: false, @@ -143,6 +146,14 @@ impl CsvExecBuilder { self } + /// Set the line terminator. If not set, the default is CRLF. + /// + /// The default is None. + pub fn with_terminator(mut self, terminator: Option) -> Self { + self.terminator = terminator; + self + } + /// Set the escape character. /// /// The default is `None` (i.e. quotes cannot be escaped). @@ -191,6 +202,7 @@ impl CsvExecBuilder { has_header, delimiter, quote, + terminator, escape, comment, newlines_in_values, @@ -210,6 +222,7 @@ impl CsvExecBuilder { has_header, delimiter, quote, + terminator, escape, newlines_in_values, metrics: ExecutionPlanMetricsSet::new(), @@ -229,6 +242,7 @@ impl CsvExec { has_header: bool, delimiter: u8, quote: u8, + terminator: Option, escape: Option, comment: Option, newlines_in_values: bool, @@ -238,6 +252,7 @@ impl CsvExec { .with_has_header(has_header) .with_delimeter(delimiter) .with_quote(quote) + .with_terminator(terminator) .with_escape(escape) .with_comment(comment) .with_newlines_in_values(newlines_in_values) @@ -270,6 +285,11 @@ impl CsvExec { self.quote } + /// The line terminator + pub fn terminator(&self) -> Option { + self.terminator + } + /// Lines beginning with this byte are ignored. pub fn comment(&self) -> Option { self.comment @@ -406,10 +426,10 @@ impl ExecutionPlan for CsvExec { delimiter: self.delimiter, quote: self.quote, escape: self.escape, + terminator: self.terminator, object_store, comment: self.comment, }); - let opener = CsvOpener { config, file_compression_type: self.file_compression_type.to_owned(), @@ -441,6 +461,7 @@ impl ExecutionPlan for CsvExec { delimiter: self.delimiter, quote: self.quote, escape: self.escape, + terminator: self.terminator, comment: self.comment, newlines_in_values: self.newlines_in_values, metrics: self.metrics.clone(), @@ -459,6 +480,7 @@ pub struct CsvConfig { has_header: bool, delimiter: u8, quote: u8, + terminator: Option, escape: Option, object_store: Arc, comment: Option, @@ -474,6 +496,7 @@ impl CsvConfig { has_header: bool, delimiter: u8, quote: u8, + terminator: Option, object_store: Arc, comment: Option, ) -> Self { @@ -484,6 +507,7 @@ impl CsvConfig { has_header, delimiter, quote, + terminator, escape: None, object_store, comment, @@ -502,7 +526,9 @@ impl CsvConfig { .with_batch_size(self.batch_size) .with_header(self.has_header) .with_quote(self.quote); - + if let Some(terminator) = self.terminator { + builder = builder.with_terminator(terminator); + } if let Some(proj) = &self.file_projection { builder = builder.with_projection(proj.clone()); } @@ -775,6 +801,7 @@ mod tests { .with_has_header(true) .with_delimeter(b',') .with_quote(b'"') + .with_terminator(None) .with_escape(None) .with_comment(None) .with_newlines_in_values(false) @@ -844,6 +871,7 @@ mod tests { .with_has_header(true) .with_delimeter(b',') .with_quote(b'"') + .with_terminator(None) .with_escape(None) .with_comment(None) .with_newlines_in_values(false) @@ -913,6 +941,7 @@ mod tests { .with_has_header(true) .with_delimeter(b',') .with_quote(b'"') + .with_terminator(None) .with_escape(None) .with_comment(None) .with_newlines_in_values(false) @@ -979,6 +1008,7 @@ mod tests { .with_has_header(true) .with_delimeter(b',') .with_quote(b'"') + .with_terminator(None) .with_escape(None) .with_comment(None) .with_newlines_in_values(false) @@ -1044,6 +1074,7 @@ mod tests { .with_has_header(true) .with_delimeter(b',') .with_quote(b'"') + .with_terminator(None) .with_escape(None) .with_comment(None) .with_newlines_in_values(false) @@ -1139,6 +1170,7 @@ mod tests { .with_has_header(true) .with_delimeter(b',') .with_quote(b'"') + .with_terminator(None) .with_escape(None) .with_comment(None) .with_newlines_in_values(false) @@ -1210,6 +1242,107 @@ mod tests { crate::assert_batches_eq!(expected, &result); } + #[tokio::test] + async fn test_terminator() { + let session_ctx = SessionContext::new(); + let store = object_store::memory::InMemory::new(); + + let data = bytes::Bytes::from("a,b\r1,2\r3,4"); + let path = object_store::path::Path::from("a.csv"); + store.put(&path, data.into()).await.unwrap(); + + let url = Url::parse("memory://").unwrap(); + session_ctx.register_object_store(&url, Arc::new(store)); + + let df = session_ctx + .read_csv("memory:///", CsvReadOptions::new().terminator(Some(b'\r'))) + .await + .unwrap(); + + let result = df.collect().await.unwrap(); + + let expected = [ + "+---+---+", + "| a | b |", + "+---+---+", + "| 1 | 2 |", + "| 3 | 4 |", + "+---+---+", + ]; + + crate::assert_batches_eq!(expected, &result); + + let e = session_ctx + .read_csv("memory:///", CsvReadOptions::new().terminator(Some(b'\n'))) + .await + .unwrap() + .collect() + .await + .unwrap_err(); + assert_eq!(e.strip_backtrace(), "Arrow error: Csv error: incorrect number of fields for line 1, expected 2 got more than 2") + } + + #[tokio::test] + async fn test_create_external_table_with_terminator() -> Result<()> { + let ctx = SessionContext::new(); + ctx.sql( + r#" + CREATE EXTERNAL TABLE t1 ( + col1 TEXT, + col2 TEXT + ) STORED AS CSV + LOCATION 'tests/data/cr_terminator.csv' + OPTIONS ('format.terminator' E'\r', 'format.has_header' 'true'); + "#, + ) + .await? + .collect() + .await?; + + let df = ctx.sql(r#"select * from t1"#).await?.collect().await?; + let expected = [ + "+------+--------+", + "| col1 | col2 |", + "+------+--------+", + "| id0 | value0 |", + "| id1 | value1 |", + "| id2 | value2 |", + "| id3 | value3 |", + "+------+--------+", + ]; + crate::assert_batches_eq!(expected, &df); + Ok(()) + } + + #[tokio::test] + async fn test_create_external_table_with_terminator_with_newlines_in_values( + ) -> Result<()> { + let ctx = SessionContext::new(); + ctx.sql(r#" + CREATE EXTERNAL TABLE t1 ( + col1 TEXT, + col2 TEXT + ) STORED AS CSV + LOCATION 'tests/data/newlines_in_values_cr_terminator.csv' + OPTIONS ('format.terminator' E'\r', 'format.has_header' 'true', 'format.newlines_in_values' 'true'); + "#).await?.collect().await?; + + let df = ctx.sql(r#"select * from t1"#).await?.collect().await?; + let expected = [ + "+-------+-----------------------------+", + "| col1 | col2 |", + "+-------+-----------------------------+", + "| 1 | hello\rworld |", + "| 2 | something\relse |", + "| 3 | \rmany\rlines\rmake\rgood test\r |", + "| 4 | unquoted |", + "| value | end |", + "+-------+-----------------------------+", + ]; + crate::assert_batches_eq!(expected, &df); + Ok(()) + } + #[tokio::test] async fn write_csv_results_error_handling() -> Result<()> { let ctx = SessionContext::new(); @@ -1365,6 +1498,7 @@ mod tests { has_header, delimiter, quote, + terminator, escape, comment, newlines_in_values, @@ -1374,6 +1508,7 @@ mod tests { assert_eq!(has_header, default_options.has_header.unwrap_or(false)); assert_eq!(delimiter, default_options.delimiter); assert_eq!(quote, default_options.quote); + assert_eq!(terminator, default_options.terminator); assert_eq!(escape, default_options.escape); assert_eq!(comment, default_options.comment); assert_eq!( diff --git a/datafusion/core/tests/data/cr_terminator.csv b/datafusion/core/tests/data/cr_terminator.csv new file mode 100644 index 000000000000..f2a5d09a4c19 --- /dev/null +++ b/datafusion/core/tests/data/cr_terminator.csv @@ -0,0 +1 @@ +c1,c2 id0,value0 id1,value1 id2,value2 id3,value3 \ No newline at end of file diff --git a/datafusion/core/tests/data/newlines_in_values_cr_terminator.csv b/datafusion/core/tests/data/newlines_in_values_cr_terminator.csv new file mode 100644 index 000000000000..2f6557d60ec5 --- /dev/null +++ b/datafusion/core/tests/data/newlines_in_values_cr_terminator.csv @@ -0,0 +1 @@ +id,message 1,"hello world" 2,"something else" 3," many lines make good test " 4,unquoted value,end \ No newline at end of file diff --git a/datafusion/proto-common/proto/datafusion_common.proto b/datafusion/proto-common/proto/datafusion_common.proto index 9268ccca0b70..51e94d2caaf4 100644 --- a/datafusion/proto-common/proto/datafusion_common.proto +++ b/datafusion/proto-common/proto/datafusion_common.proto @@ -423,6 +423,7 @@ message CsvOptions { bytes comment = 13; // Optional comment character as a byte bytes double_quote = 14; // Indicates if quotes are doubled bytes newlines_in_values = 15; // Indicates if newlines are supported in values + bytes terminator = 16; // Optional terminator character as a byte } // Options controlling CSV format diff --git a/datafusion/proto-common/src/from_proto/mod.rs b/datafusion/proto-common/src/from_proto/mod.rs index feb4c11aa809..45d275fb488e 100644 --- a/datafusion/proto-common/src/from_proto/mod.rs +++ b/datafusion/proto-common/src/from_proto/mod.rs @@ -863,6 +863,7 @@ impl TryFrom<&protobuf::CsvOptions> for CsvOptions { has_header: proto_opts.has_header.first().map(|h| *h != 0), delimiter: proto_opts.delimiter[0], quote: proto_opts.quote[0], + terminator: proto_opts.terminator.first().copied(), escape: proto_opts.escape.first().copied(), double_quote: proto_opts.has_header.first().map(|h| *h != 0), newlines_in_values: proto_opts.newlines_in_values.first().map(|h| *h != 0), diff --git a/datafusion/proto-common/src/generated/pbjson.rs b/datafusion/proto-common/src/generated/pbjson.rs index 05e57f5585a6..78ba829f8c50 100644 --- a/datafusion/proto-common/src/generated/pbjson.rs +++ b/datafusion/proto-common/src/generated/pbjson.rs @@ -1542,6 +1542,9 @@ impl serde::Serialize for CsvOptions { if !self.newlines_in_values.is_empty() { len += 1; } + if !self.terminator.is_empty() { + len += 1; + } let mut struct_ser = serializer.serialize_struct("datafusion_common.CsvOptions", len)?; if !self.has_header.is_empty() { #[allow(clippy::needless_borrow)] @@ -1598,6 +1601,10 @@ impl serde::Serialize for CsvOptions { #[allow(clippy::needless_borrow)] struct_ser.serialize_field("newlinesInValues", pbjson::private::base64::encode(&self.newlines_in_values).as_str())?; } + if !self.terminator.is_empty() { + #[allow(clippy::needless_borrow)] + struct_ser.serialize_field("terminator", pbjson::private::base64::encode(&self.terminator).as_str())?; + } struct_ser.end() } } @@ -1633,6 +1640,7 @@ impl<'de> serde::Deserialize<'de> for CsvOptions { "doubleQuote", "newlines_in_values", "newlinesInValues", + "terminator", ]; #[allow(clippy::enum_variant_names)] @@ -1652,6 +1660,7 @@ impl<'de> serde::Deserialize<'de> for CsvOptions { Comment, DoubleQuote, NewlinesInValues, + Terminator, } impl<'de> serde::Deserialize<'de> for GeneratedField { fn deserialize(deserializer: D) -> std::result::Result @@ -1688,6 +1697,7 @@ impl<'de> serde::Deserialize<'de> for CsvOptions { "comment" => Ok(GeneratedField::Comment), "doubleQuote" | "double_quote" => Ok(GeneratedField::DoubleQuote), "newlinesInValues" | "newlines_in_values" => Ok(GeneratedField::NewlinesInValues), + "terminator" => Ok(GeneratedField::Terminator), _ => Err(serde::de::Error::unknown_field(value, FIELDS)), } } @@ -1722,6 +1732,7 @@ impl<'de> serde::Deserialize<'de> for CsvOptions { let mut comment__ = None; let mut double_quote__ = None; let mut newlines_in_values__ = None; + let mut terminator__ = None; while let Some(k) = map_.next_key()? { match k { GeneratedField::HasHeader => { @@ -1830,6 +1841,14 @@ impl<'de> serde::Deserialize<'de> for CsvOptions { Some(map_.next_value::<::pbjson::private::BytesDeserialize<_>>()?.0) ; } + GeneratedField::Terminator => { + if terminator__.is_some() { + return Err(serde::de::Error::duplicate_field("terminator")); + } + terminator__ = + Some(map_.next_value::<::pbjson::private::BytesDeserialize<_>>()?.0) + ; + } } } Ok(CsvOptions { @@ -1848,6 +1867,7 @@ impl<'de> serde::Deserialize<'de> for CsvOptions { comment: comment__.unwrap_or_default(), double_quote: double_quote__.unwrap_or_default(), newlines_in_values: newlines_in_values__.unwrap_or_default(), + terminator: terminator__.unwrap_or_default(), }) } } diff --git a/datafusion/proto-common/src/generated/prost.rs b/datafusion/proto-common/src/generated/prost.rs index ebc05718a458..cb8f86a022a6 100644 --- a/datafusion/proto-common/src/generated/prost.rs +++ b/datafusion/proto-common/src/generated/prost.rs @@ -652,6 +652,9 @@ pub struct CsvOptions { /// Indicates if newlines are supported in values #[prost(bytes = "vec", tag = "15")] pub newlines_in_values: ::prost::alloc::vec::Vec, + /// Optional terminator character as a byte + #[prost(bytes = "vec", tag = "16")] + pub terminator: ::prost::alloc::vec::Vec, } /// Options controlling CSV format #[allow(clippy::derive_partial_eq_without_eq)] diff --git a/datafusion/proto-common/src/to_proto/mod.rs b/datafusion/proto-common/src/to_proto/mod.rs index 4cf7e73ac912..3718ccbb0f85 100644 --- a/datafusion/proto-common/src/to_proto/mod.rs +++ b/datafusion/proto-common/src/to_proto/mod.rs @@ -910,6 +910,7 @@ impl TryFrom<&CsvOptions> for protobuf::CsvOptions { has_header: opts.has_header.map_or_else(Vec::new, |h| vec![h as u8]), delimiter: vec![opts.delimiter], quote: vec![opts.quote], + terminator: opts.terminator.map_or_else(Vec::new, |e| vec![e]), escape: opts.escape.map_or_else(Vec::new, |e| vec![e]), double_quote: opts.double_quote.map_or_else(Vec::new, |h| vec![h as u8]), newlines_in_values: opts diff --git a/datafusion/proto/src/generated/datafusion_proto_common.rs b/datafusion/proto/src/generated/datafusion_proto_common.rs index 3d7b1007b04e..dc8d0017d3fd 100644 --- a/datafusion/proto/src/generated/datafusion_proto_common.rs +++ b/datafusion/proto/src/generated/datafusion_proto_common.rs @@ -652,6 +652,9 @@ pub struct CsvOptions { /// Indicates if newlines are supported in values #[prost(bytes = "vec", tag = "15")] pub newlines_in_values: ::prost::alloc::vec::Vec, + /// Optional terminator character as a byte + #[prost(bytes = "vec", tag = "16")] + pub terminator: ::prost::alloc::vec::Vec, } /// Options controlling CSV format #[allow(clippy::derive_partial_eq_without_eq)] diff --git a/datafusion/proto/src/logical_plan/file_formats.rs b/datafusion/proto/src/logical_plan/file_formats.rs index 607a3d8642fd..2e3476da6ac0 100644 --- a/datafusion/proto/src/logical_plan/file_formats.rs +++ b/datafusion/proto/src/logical_plan/file_formats.rs @@ -53,6 +53,7 @@ impl CsvOptionsProto { has_header: options.has_header.map_or(vec![], |v| vec![v as u8]), delimiter: vec![options.delimiter], quote: vec![options.quote], + terminator: options.terminator.map_or(vec![], |v| vec![v]), escape: options.escape.map_or(vec![], |v| vec![v]), double_quote: options.double_quote.map_or(vec![], |v| vec![v as u8]), compression: options.compression as i32, @@ -87,6 +88,11 @@ impl From<&CsvOptionsProto> for CsvOptions { }, delimiter: proto.delimiter.first().copied().unwrap_or(b','), quote: proto.quote.first().copied().unwrap_or(b'"'), + terminator: if !proto.terminator.is_empty() { + Some(proto.terminator[0]) + } else { + None + }, escape: if !proto.escape.is_empty() { Some(proto.escape[0]) } else { diff --git a/datafusion/sql/src/utils.rs b/datafusion/sql/src/utils.rs index c32acecaae5f..2531795a1630 100644 --- a/datafusion/sql/src/utils.rs +++ b/datafusion/sql/src/utils.rs @@ -268,8 +268,8 @@ pub(crate) fn value_to_string(value: &Value) -> Option { Value::DollarQuotedString(s) => Some(s.to_string()), Value::Number(_, _) | Value::Boolean(_) => Some(value.to_string()), Value::UnicodeStringLiteral(s) => Some(s.to_string()), + Value::EscapedStringLiteral(s) => Some(s.to_string()), Value::DoubleQuotedString(_) - | Value::EscapedStringLiteral(_) | Value::NationalStringLiteral(_) | Value::SingleQuotedByteStringLiteral(_) | Value::DoubleQuotedByteStringLiteral(_) diff --git a/datafusion/sqllogictest/test_files/csv_files.slt b/datafusion/sqllogictest/test_files/csv_files.slt index 7cb21abdba10..d6600e06dc1c 100644 --- a/datafusion/sqllogictest/test_files/csv_files.slt +++ b/datafusion/sqllogictest/test_files/csv_files.slt @@ -336,3 +336,24 @@ id message 05)good test 4 unquoted value end + +statement ok +CREATE EXTERNAL TABLE stored_table_with_cr_terminator ( +col1 TEXT, +col2 TEXT +) STORED AS CSV +LOCATION '../core/tests/data/cr_terminator.csv' +OPTIONS ('format.terminator' E'\r', 'format.has_header' 'true'); + +# TODO: It should be passed but got the error: External error: query failed: DataFusion error: Object Store error: Generic LocalFileSystem error: Requested range was invalid +# See the issue: https://github.com/apache/datafusion/issues/12328 +# query TT +# select * from stored_table_with_cr_terminator; +# ---- +# id0 value0 +# id1 value1 +# id2 value2 +# id3 value3 + +statement ok +drop table stored_table_with_cr_terminator; From d6f3f738e88988daba725468eb552a40ba98a1de Mon Sep 17 00:00:00 2001 From: Georgi Krastev Date: Thu, 5 Sep 2024 14:20:47 +0300 Subject: [PATCH 095/154] Support try_from_array and eq_array for ScalarValue::Union (#12208) --- datafusion/common/src/scalar/mod.rs | 124 +++++++++++++++++++++++++++- 1 file changed, 122 insertions(+), 2 deletions(-) diff --git a/datafusion/common/src/scalar/mod.rs b/datafusion/common/src/scalar/mod.rs index 22e39404cdb5..0cb325e0b02b 100644 --- a/datafusion/common/src/scalar/mod.rs +++ b/datafusion/common/src/scalar/mod.rs @@ -2800,6 +2800,13 @@ impl ScalarValue { let a = array.slice(index, 1); Self::Map(Arc::new(a.as_map().to_owned())) } + DataType::Union(fields, mode) => { + let array = as_union_array(array); + let ti = array.type_id(index); + let index = array.value_offset(index); + let value = ScalarValue::try_from_array(array.child(ti), index)?; + ScalarValue::Union(Some((ti, Box::new(value))), fields.clone(), *mode) + } other => { return _not_impl_err!( "Can't create a scalar from array of type \"{other:?}\"" @@ -3035,8 +3042,15 @@ impl ScalarValue { ScalarValue::DurationNanosecond(val) => { eq_array_primitive!(array, index, DurationNanosecondArray, val)? } - ScalarValue::Union(_, _, _) => { - return _not_impl_err!("Union is not supported yet") + ScalarValue::Union(value, _, _) => { + let array = as_union_array(array); + let ti = array.type_id(index); + let index = array.value_offset(index); + if let Some((ti_v, value)) = value { + ti_v == &ti && value.eq_array(array.child(ti), index)? + } else { + array.child(ti).is_null(index) + } } ScalarValue::Dictionary(key_type, v) => { let (values_array, values_index) = match key_type.as_ref() { @@ -5536,6 +5550,112 @@ mod tests { assert_eq!(&array, &expected); } + #[test] + fn test_scalar_union_sparse() { + let field_a = Arc::new(Field::new("A", DataType::Int32, true)); + let field_b = Arc::new(Field::new("B", DataType::Boolean, true)); + let field_c = Arc::new(Field::new("C", DataType::Utf8, true)); + let fields = UnionFields::from_iter([(0, field_a), (1, field_b), (2, field_c)]); + + let mut values_a = vec![None; 6]; + values_a[0] = Some(42); + let mut values_b = vec![None; 6]; + values_b[1] = Some(true); + let mut values_c = vec![None; 6]; + values_c[2] = Some("foo"); + let children: Vec = vec![ + Arc::new(Int32Array::from(values_a)), + Arc::new(BooleanArray::from(values_b)), + Arc::new(StringArray::from(values_c)), + ]; + + let type_ids = ScalarBuffer::from(vec![0, 1, 2, 0, 1, 2]); + let array: ArrayRef = Arc::new( + UnionArray::try_new(fields.clone(), type_ids, None, children) + .expect("UnionArray"), + ); + + let expected = [ + (0, ScalarValue::from(42)), + (1, ScalarValue::from(true)), + (2, ScalarValue::from("foo")), + (0, ScalarValue::Int32(None)), + (1, ScalarValue::Boolean(None)), + (2, ScalarValue::Utf8(None)), + ]; + + for (i, (ti, value)) in expected.into_iter().enumerate() { + let is_null = value.is_null(); + let value = Some((ti, Box::new(value))); + let expected = ScalarValue::Union(value, fields.clone(), UnionMode::Sparse); + let actual = ScalarValue::try_from_array(&array, i).expect("try_from_array"); + + assert_eq!( + actual, expected, + "[{i}] {actual} was not equal to {expected}" + ); + + assert!( + expected.eq_array(&array, i).expect("eq_array"), + "[{i}] {expected}.eq_array was false" + ); + + if is_null { + assert!(actual.is_null(), "[{i}] {actual} was not null") + } + } + } + + #[test] + fn test_scalar_union_dense() { + let field_a = Arc::new(Field::new("A", DataType::Int32, true)); + let field_b = Arc::new(Field::new("B", DataType::Boolean, true)); + let field_c = Arc::new(Field::new("C", DataType::Utf8, true)); + let fields = UnionFields::from_iter([(0, field_a), (1, field_b), (2, field_c)]); + let children: Vec = vec![ + Arc::new(Int32Array::from(vec![Some(42), None])), + Arc::new(BooleanArray::from(vec![Some(true), None])), + Arc::new(StringArray::from(vec![Some("foo"), None])), + ]; + + let type_ids = ScalarBuffer::from(vec![0, 1, 2, 0, 1, 2]); + let offsets = ScalarBuffer::from(vec![0, 0, 0, 1, 1, 1]); + let array: ArrayRef = Arc::new( + UnionArray::try_new(fields.clone(), type_ids, Some(offsets), children) + .expect("UnionArray"), + ); + + let expected = [ + (0, ScalarValue::from(42)), + (1, ScalarValue::from(true)), + (2, ScalarValue::from("foo")), + (0, ScalarValue::Int32(None)), + (1, ScalarValue::Boolean(None)), + (2, ScalarValue::Utf8(None)), + ]; + + for (i, (ti, value)) in expected.into_iter().enumerate() { + let is_null = value.is_null(); + let value = Some((ti, Box::new(value))); + let expected = ScalarValue::Union(value, fields.clone(), UnionMode::Dense); + let actual = ScalarValue::try_from_array(&array, i).expect("try_from_array"); + + assert_eq!( + actual, expected, + "[{i}] {actual} was not equal to {expected}" + ); + + assert!( + expected.eq_array(&array, i).expect("eq_array"), + "[{i}] {expected}.eq_array was false" + ); + + if is_null { + assert!(actual.is_null(), "[{i}] {actual} was not null") + } + } + } + #[test] fn test_lists_in_struct() { let field_a = Arc::new(Field::new("A", DataType::Utf8, false)); From f638df3e4173b07e96cb0fcf944f71d19d4fcd21 Mon Sep 17 00:00:00 2001 From: Matthijs Brobbel Date: Thu, 5 Sep 2024 22:29:43 +0200 Subject: [PATCH 096/154] Fix some clippy warnings (#12346) --- datafusion/functions/benches/repeat.rs | 12 ++++++------ datafusion/functions/benches/substr.rs | 18 +++++++++--------- 2 files changed, 15 insertions(+), 15 deletions(-) diff --git a/datafusion/functions/benches/repeat.rs b/datafusion/functions/benches/repeat.rs index 916c8374e5fb..e45313660ea2 100644 --- a/datafusion/functions/benches/repeat.rs +++ b/datafusion/functions/benches/repeat.rs @@ -67,7 +67,7 @@ fn criterion_benchmark(c: &mut Criterion) { let args = create_args::(size, 32, repeat_times, true); group.bench_function( - &format!( + format!( "repeat_string_view [size={}, repeat_times={}]", size, repeat_times ), @@ -76,7 +76,7 @@ fn criterion_benchmark(c: &mut Criterion) { let args = create_args::(size, 32, repeat_times, false); group.bench_function( - &format!( + format!( "repeat_string [size={}, repeat_times={}]", size, repeat_times ), @@ -85,7 +85,7 @@ fn criterion_benchmark(c: &mut Criterion) { let args = create_args::(size, 32, repeat_times, false); group.bench_function( - &format!( + format!( "repeat_large_string [size={}, repeat_times={}]", size, repeat_times ), @@ -103,7 +103,7 @@ fn criterion_benchmark(c: &mut Criterion) { let args = create_args::(size, 32, repeat_times, true); group.bench_function( - &format!( + format!( "repeat_string_view [size={}, repeat_times={}]", size, repeat_times ), @@ -112,7 +112,7 @@ fn criterion_benchmark(c: &mut Criterion) { let args = create_args::(size, 32, repeat_times, false); group.bench_function( - &format!( + format!( "repeat_string [size={}, repeat_times={}]", size, repeat_times ), @@ -121,7 +121,7 @@ fn criterion_benchmark(c: &mut Criterion) { let args = create_args::(size, 32, repeat_times, false); group.bench_function( - &format!( + format!( "repeat_large_string [size={}, repeat_times={}]", size, repeat_times ), diff --git a/datafusion/functions/benches/substr.rs b/datafusion/functions/benches/substr.rs index 14a3389da380..1a696520c3ad 100644 --- a/datafusion/functions/benches/substr.rs +++ b/datafusion/functions/benches/substr.rs @@ -106,19 +106,19 @@ fn criterion_benchmark(c: &mut Criterion) { let args = create_args_without_count::(size, len, true, true); group.bench_function( - &format!("substr_string_view [size={}, strlen={}]", size, len), + format!("substr_string_view [size={}, strlen={}]", size, len), |b| b.iter(|| black_box(substr.invoke(&args))), ); let args = create_args_without_count::(size, len, false, false); group.bench_function( - &format!("substr_string [size={}, strlen={}]", size, len), + format!("substr_string [size={}, strlen={}]", size, len), |b| b.iter(|| black_box(substr.invoke(&args))), ); let args = create_args_without_count::(size, len, true, false); group.bench_function( - &format!("substr_large_string [size={}, strlen={}]", size, len), + format!("substr_large_string [size={}, strlen={}]", size, len), |b| b.iter(|| black_box(substr.invoke(&args))), ); @@ -133,7 +133,7 @@ fn criterion_benchmark(c: &mut Criterion) { let args = create_args_with_count::(size, len, count, true); group.bench_function( - &format!( + format!( "substr_string_view [size={}, count={}, strlen={}]", size, count, len, ), @@ -142,7 +142,7 @@ fn criterion_benchmark(c: &mut Criterion) { let args = create_args_with_count::(size, len, count, false); group.bench_function( - &format!( + format!( "substr_string [size={}, count={}, strlen={}]", size, count, len, ), @@ -151,7 +151,7 @@ fn criterion_benchmark(c: &mut Criterion) { let args = create_args_with_count::(size, len, count, false); group.bench_function( - &format!( + format!( "substr_large_string [size={}, count={}, strlen={}]", size, count, len, ), @@ -169,7 +169,7 @@ fn criterion_benchmark(c: &mut Criterion) { let args = create_args_with_count::(size, len, count, true); group.bench_function( - &format!( + format!( "substr_string_view [size={}, count={}, strlen={}]", size, count, len, ), @@ -178,7 +178,7 @@ fn criterion_benchmark(c: &mut Criterion) { let args = create_args_with_count::(size, len, count, false); group.bench_function( - &format!( + format!( "substr_string [size={}, count={}, strlen={}]", size, count, len, ), @@ -187,7 +187,7 @@ fn criterion_benchmark(c: &mut Criterion) { let args = create_args_with_count::(size, len, count, false); group.bench_function( - &format!( + format!( "substr_large_string [size={}, count={}, strlen={}]", size, count, len, ), From 9d819e1e237f5b10a197bc0a6548c1ad9c4c467c Mon Sep 17 00:00:00 2001 From: Oleks V Date: Thu, 5 Sep 2024 17:28:21 -0700 Subject: [PATCH 097/154] minor: reuse SessionStateBuilder methods for default builder (#12330) --- .../core/src/execution/session_state.rs | 20 +++++++++---------- 1 file changed, 9 insertions(+), 11 deletions(-) diff --git a/datafusion/core/src/execution/session_state.rs b/datafusion/core/src/execution/session_state.rs index 90f175b59385..675ac798bf4e 100644 --- a/datafusion/core/src/execution/session_state.rs +++ b/datafusion/core/src/execution/session_state.rs @@ -1035,17 +1035,15 @@ impl SessionStateBuilder { } } - /// Set defaults for table_factories, file formats, expr_planners and builtin - /// scalar and aggregate functions. - pub fn with_default_features(mut self) -> Self { - self.table_factories = Some(SessionStateDefaults::default_table_factories()); - self.file_formats = Some(SessionStateDefaults::default_file_formats()); - self.expr_planners = Some(SessionStateDefaults::default_expr_planners()); - self.scalar_functions = Some(SessionStateDefaults::default_scalar_functions()); - self.aggregate_functions = - Some(SessionStateDefaults::default_aggregate_functions()); - self.window_functions = Some(SessionStateDefaults::default_window_functions()); - self + /// Create default builder with defaults for table_factories, file formats, expr_planners and builtin + /// scalar, aggregate and windows functions. + pub fn with_default_features(self) -> Self { + self.with_table_factories(SessionStateDefaults::default_table_factories()) + .with_file_formats(SessionStateDefaults::default_file_formats()) + .with_expr_planners(SessionStateDefaults::default_expr_planners()) + .with_scalar_functions(SessionStateDefaults::default_scalar_functions()) + .with_aggregate_functions(SessionStateDefaults::default_aggregate_functions()) + .with_window_functions(SessionStateDefaults::default_window_functions()) } /// Set the session id. From 41b10ca44e6ca1e6d4b3e63d71c383ad0af0e05a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Dani=C3=ABl=20Heres?= Date: Fri, 6 Sep 2024 09:43:45 +0200 Subject: [PATCH 098/154] Push down null filters for more join types (#12348) * Push down null filters for more join types * fix tests * Fix docs --- .../optimizer/src/filter_null_join_keys.rs | 54 +++++++++++++------ datafusion/optimizer/src/push_down_filter.rs | 34 ++++++------ .../optimizer/tests/optimizer_integration.rs | 28 ++++++---- 3 files changed, 75 insertions(+), 41 deletions(-) diff --git a/datafusion/optimizer/src/filter_null_join_keys.rs b/datafusion/optimizer/src/filter_null_join_keys.rs index 381713662f10..01e3d27c580f 100644 --- a/datafusion/optimizer/src/filter_null_join_keys.rs +++ b/datafusion/optimizer/src/filter_null_join_keys.rs @@ -18,18 +18,16 @@ //! [`FilterNullJoinKeys`] adds filters to join inputs when input isn't nullable use crate::optimizer::ApplyOrder; +use crate::push_down_filter::on_lr_is_preserved; use crate::{OptimizerConfig, OptimizerRule}; use datafusion_common::tree_node::Transformed; use datafusion_common::Result; use datafusion_expr::utils::conjunction; -use datafusion_expr::{ - logical_plan::Filter, logical_plan::JoinType, Expr, ExprSchemable, LogicalPlan, -}; +use datafusion_expr::{logical_plan::Filter, Expr, ExprSchemable, LogicalPlan}; use std::sync::Arc; -/// The FilterNullJoinKeys rule will identify inner joins with equi-join conditions -/// where the join key is nullable on one side and non-nullable on the other side -/// and then insert an `IsNotNull` filter on the nullable side since null values +/// The FilterNullJoinKeys rule will identify joins with equi-join conditions +/// where the join key is nullable and then insert an `IsNotNull` filter on the nullable side since null values /// can never match. #[derive(Default)] pub struct FilterNullJoinKeys {} @@ -51,9 +49,11 @@ impl OptimizerRule for FilterNullJoinKeys { if !config.options().optimizer.filter_null_join_keys { return Ok(Transformed::no(plan)); } - match plan { - LogicalPlan::Join(mut join) if join.join_type == JoinType::Inner => { + LogicalPlan::Join(mut join) if !join.on.is_empty() => { + let (left_preserved, right_preserved) = + on_lr_is_preserved(join.join_type); + let left_schema = join.left.schema(); let right_schema = join.right.schema(); @@ -61,11 +61,11 @@ impl OptimizerRule for FilterNullJoinKeys { let mut right_filters = vec![]; for (l, r) in &join.on { - if l.nullable(left_schema)? { + if left_preserved && l.nullable(left_schema)? { left_filters.push(l.clone()); } - if r.nullable(right_schema)? { + if right_preserved && r.nullable(right_schema)? { right_filters.push(r.clone()); } } @@ -109,7 +109,7 @@ mod tests { use arrow::datatypes::{DataType, Field, Schema}; use datafusion_common::Column; use datafusion_expr::logical_plan::table_scan; - use datafusion_expr::{col, lit, LogicalPlanBuilder}; + use datafusion_expr::{col, lit, JoinType, LogicalPlanBuilder}; fn assert_optimized_plan_equal(plan: LogicalPlan, expected: &str) -> Result<()> { assert_optimized_plan_eq(Arc::new(FilterNullJoinKeys {}), plan, expected) @@ -118,7 +118,7 @@ mod tests { #[test] fn left_nullable() -> Result<()> { let (t1, t2) = test_tables()?; - let plan = build_plan(t1, t2, "t1.optional_id", "t2.id")?; + let plan = build_plan(t1, t2, "t1.optional_id", "t2.id", JoinType::Inner)?; let expected = "Inner Join: t1.optional_id = t2.id\ \n Filter: t1.optional_id IS NOT NULL\ \n TableScan: t1\ @@ -126,10 +126,33 @@ mod tests { assert_optimized_plan_equal(plan, expected) } + #[test] + fn left_nullable_left_join() -> Result<()> { + let (t1, t2) = test_tables()?; + let plan = build_plan(t1, t2, "t1.optional_id", "t2.id", JoinType::Left)?; + let expected = "Left Join: t1.optional_id = t2.id\ + \n TableScan: t1\ + \n TableScan: t2"; + assert_optimized_plan_equal(plan, expected) + } + + #[test] + fn left_nullable_left_join_reordered() -> Result<()> { + let (t_left, t_right) = test_tables()?; + // Note: order of tables is reversed + let plan = + build_plan(t_right, t_left, "t2.id", "t1.optional_id", JoinType::Left)?; + let expected = "Left Join: t2.id = t1.optional_id\ + \n TableScan: t2\ + \n Filter: t1.optional_id IS NOT NULL\ + \n TableScan: t1"; + assert_optimized_plan_equal(plan, expected) + } + #[test] fn left_nullable_on_condition_reversed() -> Result<()> { let (t1, t2) = test_tables()?; - let plan = build_plan(t1, t2, "t2.id", "t1.optional_id")?; + let plan = build_plan(t1, t2, "t2.id", "t1.optional_id", JoinType::Inner)?; let expected = "Inner Join: t1.optional_id = t2.id\ \n Filter: t1.optional_id IS NOT NULL\ \n TableScan: t1\ @@ -140,7 +163,7 @@ mod tests { #[test] fn nested_join_multiple_filter_expr() -> Result<()> { let (t1, t2) = test_tables()?; - let plan = build_plan(t1, t2, "t1.optional_id", "t2.id")?; + let plan = build_plan(t1, t2, "t1.optional_id", "t2.id", JoinType::Inner)?; let schema = Schema::new(vec![ Field::new("id", DataType::UInt32, false), Field::new("t1_id", DataType::UInt32, true), @@ -244,11 +267,12 @@ mod tests { right_table: LogicalPlan, left_key: &str, right_key: &str, + join_type: JoinType, ) -> Result { LogicalPlanBuilder::from(left_table) .join( right_table, - JoinType::Inner, + join_type, ( vec![Column::from_qualified_name(left_key)], vec![Column::from_qualified_name(right_key)], diff --git a/datafusion/optimizer/src/push_down_filter.rs b/datafusion/optimizer/src/push_down_filter.rs index 0625652a6be1..6f0a64b85cb6 100644 --- a/datafusion/optimizer/src/push_down_filter.rs +++ b/datafusion/optimizer/src/push_down_filter.rs @@ -157,18 +157,18 @@ pub struct PushDownFilter {} /// the right is not, because there may be rows in the output that don't /// directly map to a row in the right input (due to nulls filling where there /// is no match on the right). -fn lr_is_preserved(join_type: JoinType) -> Result<(bool, bool)> { +pub(crate) fn lr_is_preserved(join_type: JoinType) -> (bool, bool) { match join_type { - JoinType::Inner => Ok((true, true)), - JoinType::Left => Ok((true, false)), - JoinType::Right => Ok((false, true)), - JoinType::Full => Ok((false, false)), + JoinType::Inner => (true, true), + JoinType::Left => (true, false), + JoinType::Right => (false, true), + JoinType::Full => (false, false), // No columns from the right side of the join can be referenced in output // predicates for semi/anti joins, so whether we specify t/f doesn't matter. - JoinType::LeftSemi | JoinType::LeftAnti => Ok((true, false)), + JoinType::LeftSemi | JoinType::LeftAnti => (true, false), // No columns from the left side of the join can be referenced in output // predicates for semi/anti joins, so whether we specify t/f doesn't matter. - JoinType::RightSemi | JoinType::RightAnti => Ok((false, true)), + JoinType::RightSemi | JoinType::RightAnti => (false, true), } } @@ -181,15 +181,15 @@ fn lr_is_preserved(join_type: JoinType) -> Result<(bool, bool)> { /// A tuple of booleans - (left_preserved, right_preserved). /// /// See [`lr_is_preserved`] for a definition of "preserved". -fn on_lr_is_preserved(join_type: JoinType) -> Result<(bool, bool)> { +pub(crate) fn on_lr_is_preserved(join_type: JoinType) -> (bool, bool) { match join_type { - JoinType::Inner => Ok((true, true)), - JoinType::Left => Ok((false, true)), - JoinType::Right => Ok((true, false)), - JoinType::Full => Ok((false, false)), - JoinType::LeftSemi | JoinType::RightSemi => Ok((true, true)), - JoinType::LeftAnti => Ok((false, true)), - JoinType::RightAnti => Ok((true, false)), + JoinType::Inner => (true, true), + JoinType::Left => (false, true), + JoinType::Right => (true, false), + JoinType::Full => (false, false), + JoinType::LeftSemi | JoinType::RightSemi => (true, true), + JoinType::LeftAnti => (false, true), + JoinType::RightAnti => (true, false), } } @@ -420,7 +420,7 @@ fn push_down_all_join( ) -> Result> { let is_inner_join = join.join_type == JoinType::Inner; // Get pushable predicates from current optimizer state - let (left_preserved, right_preserved) = lr_is_preserved(join.join_type)?; + let (left_preserved, right_preserved) = lr_is_preserved(join.join_type); // The predicates can be divided to three categories: // 1) can push through join to its children(left or right) @@ -457,7 +457,7 @@ fn push_down_all_join( } let mut on_filter_join_conditions = vec![]; - let (on_left_preserved, on_right_preserved) = on_lr_is_preserved(join.join_type)?; + let (on_left_preserved, on_right_preserved) = on_lr_is_preserved(join.join_type); if !on_filter.is_empty() { for on in on_filter { diff --git a/datafusion/optimizer/tests/optimizer_integration.rs b/datafusion/optimizer/tests/optimizer_integration.rs index 93dd49b17492..5292b66197f6 100644 --- a/datafusion/optimizer/tests/optimizer_integration.rs +++ b/datafusion/optimizer/tests/optimizer_integration.rs @@ -124,10 +124,12 @@ fn semi_join_with_join_filter() -> Result<()> { let plan = test_sql(sql)?; let expected = "Projection: test.col_utf8\ \n LeftSemi Join: test.col_int32 = __correlated_sq_1.col_int32 Filter: test.col_uint32 != __correlated_sq_1.col_uint32\ - \n TableScan: test projection=[col_int32, col_uint32, col_utf8]\ + \n Filter: test.col_int32 IS NOT NULL\ + \n TableScan: test projection=[col_int32, col_uint32, col_utf8]\ \n SubqueryAlias: __correlated_sq_1\ \n SubqueryAlias: t2\ - \n TableScan: test projection=[col_int32, col_uint32]"; + \n Filter: test.col_int32 IS NOT NULL\ + \n TableScan: test projection=[col_int32, col_uint32]"; assert_eq!(expected, format!("{plan}")); Ok(()) } @@ -144,7 +146,8 @@ fn anti_join_with_join_filter() -> Result<()> { \n TableScan: test projection=[col_int32, col_uint32, col_utf8]\ \n SubqueryAlias: __correlated_sq_1\ \n SubqueryAlias: t2\ - \n TableScan: test projection=[col_int32, col_uint32]"; + \n Filter: test.col_int32 IS NOT NULL\ + \n TableScan: test projection=[col_int32, col_uint32]"; assert_eq!(expected, format!("{plan}")); Ok(()) } @@ -155,11 +158,13 @@ fn where_exists_distinct() -> Result<()> { SELECT DISTINCT col_int32 FROM test t2 WHERE test.col_int32 = t2.col_int32)"; let plan = test_sql(sql)?; let expected = "LeftSemi Join: test.col_int32 = __correlated_sq_1.col_int32\ - \n TableScan: test projection=[col_int32]\ + \n Filter: test.col_int32 IS NOT NULL\ + \n TableScan: test projection=[col_int32]\ \n SubqueryAlias: __correlated_sq_1\ \n Aggregate: groupBy=[[t2.col_int32]], aggr=[[]]\ \n SubqueryAlias: t2\ - \n TableScan: test projection=[col_int32]"; + \n Filter: test.col_int32 IS NOT NULL\ + \n TableScan: test projection=[col_int32]"; assert_eq!(expected, format!("{plan}")); Ok(()) } @@ -175,9 +180,12 @@ fn intersect() -> Result<()> { \n Aggregate: groupBy=[[test.col_int32, test.col_utf8]], aggr=[[]]\ \n LeftSemi Join: test.col_int32 = test.col_int32, test.col_utf8 = test.col_utf8\ \n Aggregate: groupBy=[[test.col_int32, test.col_utf8]], aggr=[[]]\ + \n Filter: test.col_int32 IS NOT NULL AND test.col_utf8 IS NOT NULL\ + \n TableScan: test projection=[col_int32, col_utf8]\ + \n Filter: test.col_int32 IS NOT NULL AND test.col_utf8 IS NOT NULL\ \n TableScan: test projection=[col_int32, col_utf8]\ - \n TableScan: test projection=[col_int32, col_utf8]\ - \n TableScan: test projection=[col_int32, col_utf8]"; + \n Filter: test.col_int32 IS NOT NULL AND test.col_utf8 IS NOT NULL\ + \n TableScan: test projection=[col_int32, col_utf8]"; assert_eq!(expected, format!("{plan}")); Ok(()) } @@ -273,9 +281,11 @@ fn test_same_name_but_not_ambiguous() { let expected = "LeftSemi Join: t1.col_int32 = t2.col_int32\ \n Aggregate: groupBy=[[t1.col_int32]], aggr=[[]]\ \n SubqueryAlias: t1\ - \n TableScan: test projection=[col_int32]\ + \n Filter: test.col_int32 IS NOT NULL\ + \n TableScan: test projection=[col_int32]\ \n SubqueryAlias: t2\ - \n TableScan: test projection=[col_int32]"; + \n Filter: test.col_int32 IS NOT NULL\ + \n TableScan: test projection=[col_int32]"; assert_eq!(expected, format!("{plan}")); } From a4445283dbff1b74a6b4d9ecfa1016857dc6207e Mon Sep 17 00:00:00 2001 From: Jonah Gao Date: Fri, 6 Sep 2024 15:44:13 +0800 Subject: [PATCH 099/154] fix: preserve qualifiers when rewriting expressions (#12341) * fix: preserve qualifiers in `NamePreserver` * Add test * Review feedback --- datafusion/expr/src/expr.rs | 15 +++- datafusion/expr/src/expr_rewriter/mod.rs | 85 +++++++++++-------- datafusion/expr/src/expr_schema.rs | 35 ++------ datafusion/sqllogictest/test_files/select.slt | 15 ++++ 4 files changed, 84 insertions(+), 66 deletions(-) diff --git a/datafusion/expr/src/expr.rs b/datafusion/expr/src/expr.rs index 8914214d084f..161e29e5925b 100644 --- a/datafusion/expr/src/expr.rs +++ b/datafusion/expr/src/expr.rs @@ -227,7 +227,7 @@ use sqlparser::ast::{ pub enum Expr { /// An expression with a specific name. Alias(Alias), - /// A named reference to a qualified filed in a schema. + /// A named reference to a qualified field in a schema. Column(Column), /// A named reference to a variable in a registry. ScalarVariable(DataType, Vec), @@ -1115,6 +1115,19 @@ impl Expr { SchemaDisplay(self) } + /// Returns the qualifier and the schema name of this expression. + /// + /// Used when the expression forms the output field of a certain plan. + /// The result is the field's qualifier and field name in the plan's + /// output schema. We can use this qualified name to reference the field. + pub fn qualified_name(&self) -> (Option, String) { + match self { + Expr::Column(Column { relation, name }) => (relation.clone(), name.clone()), + Expr::Alias(Alias { relation, name, .. }) => (relation.clone(), name.clone()), + _ => (None, self.schema_name().to_string()), + } + } + /// Returns a full and complete string representation of this expression. #[deprecated(note = "use format! instead")] pub fn canonical_name(&self) -> String { diff --git a/datafusion/expr/src/expr_rewriter/mod.rs b/datafusion/expr/src/expr_rewriter/mod.rs index 35c573836419..f0f02f398642 100644 --- a/datafusion/expr/src/expr_rewriter/mod.rs +++ b/datafusion/expr/src/expr_rewriter/mod.rs @@ -26,9 +26,7 @@ use crate::logical_plan::Projection; use crate::{Expr, ExprSchemable, LogicalPlan, LogicalPlanBuilder}; use datafusion_common::config::ConfigOptions; -use datafusion_common::tree_node::{ - Transformed, TransformedResult, TreeNode, TreeNodeRewriter, -}; +use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; use datafusion_common::TableReference; use datafusion_common::{Column, DFSchema, Result}; @@ -279,22 +277,10 @@ pub fn unalias(expr: Expr) -> Expr { } } -/// Rewrites `expr` using `rewriter`, ensuring that the output has the -/// same name as `expr` prior to rewrite, adding an alias if necessary. -/// -/// This is important when optimizing plans to ensure the output -/// schema of plan nodes don't change after optimization -pub fn rewrite_preserving_name(expr: Expr, rewriter: &mut R) -> Result -where - R: TreeNodeRewriter, -{ - let original_name = expr.name_for_alias()?; - let expr = expr.rewrite(rewriter)?.data; - expr.alias_if_changed(original_name) -} - /// Handles ensuring the name of rewritten expressions is not changed. /// +/// This is important when optimizing plans to ensure the output +/// schema of plan nodes don't change after optimization. /// For example, if an expression `1 + 2` is rewritten to `3`, the name of the /// expression should be preserved: `3 as "1 + 2"` /// @@ -303,9 +289,17 @@ pub struct NamePreserver { use_alias: bool, } -/// If the name of an expression is remembered, it will be preserved when -/// rewriting the expression -pub struct SavedName(Option); +/// If the qualified name of an expression is remembered, it will be preserved +/// when rewriting the expression +pub enum SavedName { + /// Saved qualified name to be preserved + Saved { + relation: Option, + name: String, + }, + /// Name is not preserved + None, +} impl NamePreserver { /// Create a new NamePreserver for rewriting the `expr` that is part of the specified plan @@ -326,23 +320,30 @@ impl NamePreserver { pub fn save(&self, expr: &Expr) -> Result { let original_name = if self.use_alias { - Some(expr.name_for_alias()?) + let (relation, name) = expr.qualified_name(); + SavedName::Saved { relation, name } } else { - None + SavedName::None }; - - Ok(SavedName(original_name)) + Ok(original_name) } } impl SavedName { - /// Ensures the name of the rewritten expression is preserved + /// Ensures the qualified name of the rewritten expression is preserved pub fn restore(self, expr: Expr) -> Result { - let Self(original_name) = self; - match original_name { - Some(name) => expr.alias_if_changed(name), - None => Ok(expr), - } + let expr = match self { + SavedName::Saved { relation, name } => { + let (new_relation, new_name) = expr.qualified_name(); + if new_relation != relation || new_name != name { + expr.alias_qualified(relation, name) + } else { + expr + } + } + SavedName::None => expr, + }; + Ok(expr) } } @@ -353,6 +354,7 @@ mod test { use super::*; use crate::{col, lit, Cast}; use arrow::datatypes::{DataType, Field, Schema}; + use datafusion_common::tree_node::TreeNodeRewriter; use datafusion_common::ScalarValue; #[derive(Default)] @@ -511,10 +513,20 @@ mod test { // change literal type from i32 to i64 test_rewrite(col("a").add(lit(1i32)), col("a").add(lit(1i64))); + + // test preserve qualifier + test_rewrite( + Expr::Column(Column::new(Some("test"), "a")), + Expr::Column(Column::new_unqualified("test.a")), + ); + test_rewrite( + Expr::Column(Column::new_unqualified("test.a")), + Expr::Column(Column::new(Some("test"), "a")), + ); } - /// rewrites `expr_from` to `rewrite_to` using - /// `rewrite_preserving_name` verifying the result is `expected_expr` + /// rewrites `expr_from` to `rewrite_to` while preserving the original qualified name + /// by using the `NamePreserver` fn test_rewrite(expr_from: Expr, rewrite_to: Expr) { struct TestRewriter { rewrite_to: Expr, @@ -531,11 +543,12 @@ mod test { let mut rewriter = TestRewriter { rewrite_to: rewrite_to.clone(), }; - let expr = rewrite_preserving_name(expr_from.clone(), &mut rewriter).unwrap(); - - let original_name = expr_from.schema_name().to_string(); - let new_name = expr.schema_name().to_string(); + let saved_name = NamePreserver { use_alias: true }.save(&expr_from).unwrap(); + let new_expr = expr_from.clone().rewrite(&mut rewriter).unwrap().data; + let new_expr = saved_name.restore(new_expr).unwrap(); + let original_name = expr_from.qualified_name(); + let new_name = new_expr.qualified_name(); assert_eq!( original_name, new_name, "mismatch rewriting expr_from: {expr_from} to {rewrite_to}" diff --git a/datafusion/expr/src/expr_schema.rs b/datafusion/expr/src/expr_schema.rs index 894b7e58d954..598d172d30a0 100644 --- a/datafusion/expr/src/expr_schema.rs +++ b/datafusion/expr/src/expr_schema.rs @@ -462,35 +462,12 @@ impl ExprSchemable for Expr { &self, input_schema: &dyn ExprSchema, ) -> Result<(Option, Arc)> { - match self { - Expr::Column(c) => { - let (data_type, nullable) = self.data_type_and_nullable(input_schema)?; - Ok(( - c.relation.clone(), - Field::new(&c.name, data_type, nullable) - .with_metadata(self.metadata(input_schema)?) - .into(), - )) - } - Expr::Alias(Alias { relation, name, .. }) => { - let (data_type, nullable) = self.data_type_and_nullable(input_schema)?; - Ok(( - relation.clone(), - Field::new(name, data_type, nullable) - .with_metadata(self.metadata(input_schema)?) - .into(), - )) - } - _ => { - let (data_type, nullable) = self.data_type_and_nullable(input_schema)?; - Ok(( - None, - Field::new(self.schema_name().to_string(), data_type, nullable) - .with_metadata(self.metadata(input_schema)?) - .into(), - )) - } - } + let (relation, schema_name) = self.qualified_name(); + let (data_type, nullable) = self.data_type_and_nullable(input_schema)?; + let field = Field::new(schema_name, data_type, nullable) + .with_metadata(self.metadata(input_schema)?) + .into(); + Ok((relation, field)) } /// Wraps this expression in a cast to a target [arrow::datatypes::DataType]. diff --git a/datafusion/sqllogictest/test_files/select.slt b/datafusion/sqllogictest/test_files/select.slt index 9832f97ae862..bdd8deff1873 100644 --- a/datafusion/sqllogictest/test_files/select.slt +++ b/datafusion/sqllogictest/test_files/select.slt @@ -1685,6 +1685,9 @@ SELECT i + i FROM test WHERE i > 2; ---- 6 +statement ok +DROP TABLE test; + query error DataFusion error: Arrow error: Parser error: Error parsing timestamp from 'I AM NOT A TIMESTAMP': error parsing date SELECT to_timestamp('I AM NOT A TIMESTAMP'); @@ -1741,3 +1744,15 @@ select a from t; statement ok set datafusion.optimizer.max_passes=3; + +# Test issue: https://github.com/apache/datafusion/issues/12183 +statement ok +CREATE TABLE test(a BIGINT) AS VALUES (1); + +query I +SELECT "test.a" FROM (SELECT a AS "test.a" FROM test) +---- +1 + +statement ok +DROP TABLE test; \ No newline at end of file From dd04929d8abab8254afd778d6e4e47fbc8dd1933 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=BC=A0=E6=9E=97=E4=BC=9F?= Date: Fri, 6 Sep 2024 19:04:51 +0800 Subject: [PATCH 100/154] Move `TopKAggregation` rule into `physical-optimizer` crate (#12334) * Move TopKAggregation rule into physical-optimizer crate * cargo update for cli * Update cli msrv * Fix clippy lints --- datafusion-cli/Cargo.lock | 50 ++++++++++++------- datafusion/core/src/physical_optimizer/mod.rs | 1 - datafusion/physical-optimizer/Cargo.toml | 1 + datafusion/physical-optimizer/src/lib.rs | 1 + .../src}/topk_aggregation.rs | 20 ++++---- 5 files changed, 44 insertions(+), 29 deletions(-) rename datafusion/{core/src/physical_optimizer => physical-optimizer/src}/topk_aggregation.rs (90%) diff --git a/datafusion-cli/Cargo.lock b/datafusion-cli/Cargo.lock index 039f3fb9a6aa..ddc33b43d134 100644 --- a/datafusion-cli/Cargo.lock +++ b/datafusion-cli/Cargo.lock @@ -934,9 +934,9 @@ dependencies = [ [[package]] name = "cc" -version = "1.1.15" +version = "1.1.16" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "57b6a275aa2903740dc87da01c62040406b8812552e97129a63ea8850a17c6e6" +checksum = "e9d013ecb737093c0e86b151a7b837993cf9ec6c502946cfb44bedc392421e0b" dependencies = [ "jobserver", "libc", @@ -986,9 +986,9 @@ dependencies = [ [[package]] name = "clap" -version = "4.5.16" +version = "4.5.17" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ed6719fffa43d0d87e5fd8caeab59be1554fb028cd30edc88fc4369b17971019" +checksum = "3e5a21b8495e732f1b3c364c9949b201ca7bae518c502c80256c96ad79eaf6ac" dependencies = [ "clap_builder", "clap_derive", @@ -996,9 +996,9 @@ dependencies = [ [[package]] name = "clap_builder" -version = "4.5.15" +version = "4.5.17" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "216aec2b177652e3846684cbfe25c9964d18ec45234f0f5da5157b207ed1aab6" +checksum = "8cf2dd12af7a047ad9d6da2b6b249759a22a7abc0f474c1dae1777afa4b21a73" dependencies = [ "anstream", "anstyle", @@ -1182,9 +1182,9 @@ checksum = "7762d17f1241643615821a8455a0b2c3e803784b058693d990b11f2dce25a0ca" [[package]] name = "dashmap" -version = "6.0.1" +version = "6.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "804c8821570c3f8b70230c2ba75ffa5c0f9a4189b9a432b6656c536712acae28" +checksum = "5041cc499144891f3790297212f32a74fb938e5136a14943f338ef9e0ae276cf" dependencies = [ "cfg-if", "crossbeam-utils", @@ -1523,6 +1523,7 @@ dependencies = [ name = "datafusion-physical-optimizer" version = "41.0.0" dependencies = [ + "arrow-schema", "datafusion-common", "datafusion-execution", "datafusion-physical-expr", @@ -2134,16 +2135,16 @@ dependencies = [ [[package]] name = "hyper-rustls" -version = "0.27.2" +version = "0.27.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5ee4be2c948921a1a5320b629c4193916ed787a7f7f293fd3f7f5a6c9de74155" +checksum = "08afdbb5c31130e3034af566421053ab03787c640246a446327f550d11bcb333" dependencies = [ "futures-util", "http 1.1.0", "hyper 1.4.1", "hyper-util", "rustls 0.23.12", - "rustls-native-certs 0.7.3", + "rustls-native-certs 0.8.0", "rustls-pki-types", "tokio", "tokio-rustls 0.26.0", @@ -3091,7 +3092,7 @@ dependencies = [ "http-body 1.0.1", "http-body-util", "hyper 1.4.1", - "hyper-rustls 0.27.2", + "hyper-rustls 0.27.3", "hyper-util", "ipnet", "js-sys", @@ -3191,9 +3192,9 @@ dependencies = [ [[package]] name = "rustix" -version = "0.38.35" +version = "0.38.36" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a85d50532239da68e9addb745ba38ff4612a242c1c7ceea689c4bc7c2f43c36f" +checksum = "3f55e80d50763938498dd5ebb18647174e0c76dc38c5505294bb224624f30f36" dependencies = [ "bitflags 2.6.0", "errno", @@ -3253,6 +3254,19 @@ dependencies = [ "security-framework", ] +[[package]] +name = "rustls-native-certs" +version = "0.8.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "fcaf18a4f2be7326cd874a5fa579fae794320a0f388d365dca7e480e55f83f8a" +dependencies = [ + "openssl-probe", + "rustls-pemfile 2.1.3", + "rustls-pki-types", + "schannel", + "security-framework", +] + [[package]] name = "rustls-pemfile" version = "1.0.4" @@ -3425,9 +3439,9 @@ dependencies = [ [[package]] name = "serde_json" -version = "1.0.127" +version = "1.0.128" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8043c06d9f82bd7271361ed64f415fe5e12a77fdb52e573e7f06a516dea329ad" +checksum = "6ff5456707a1de34e7e37f2a6fd3d3f808c318259cbd01ab6377795054b483d8" dependencies = [ "itoa", "memchr", @@ -3819,9 +3833,9 @@ dependencies = [ [[package]] name = "tokio-util" -version = "0.7.11" +version = "0.7.12" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9cf6b47b3771c49ac75ad09a6162f53ad4b8088b76ac60e8ec1455b31a189fe1" +checksum = "61e7c3654c13bcd040d4a03abee2c75b1d14a37b423cf5a813ceae1cc903ec6a" dependencies = [ "bytes", "futures-core", diff --git a/datafusion/core/src/physical_optimizer/mod.rs b/datafusion/core/src/physical_optimizer/mod.rs index 46d86ead183b..efdd3148d03f 100644 --- a/datafusion/core/src/physical_optimizer/mod.rs +++ b/datafusion/core/src/physical_optimizer/mod.rs @@ -32,7 +32,6 @@ pub mod replace_with_order_preserving_variants; pub mod sanity_checker; #[cfg(test)] pub mod test_utils; -pub mod topk_aggregation; pub mod update_aggr_exprs; mod sort_pushdown; diff --git a/datafusion/physical-optimizer/Cargo.toml b/datafusion/physical-optimizer/Cargo.toml index a7f675b37a60..acf3eee105d4 100644 --- a/datafusion/physical-optimizer/Cargo.toml +++ b/datafusion/physical-optimizer/Cargo.toml @@ -32,6 +32,7 @@ rust-version = { workspace = true } workspace = true [dependencies] +arrow-schema = { workspace = true } datafusion-common = { workspace = true, default-features = true } datafusion-execution = { workspace = true } datafusion-physical-expr = { workspace = true } diff --git a/datafusion/physical-optimizer/src/lib.rs b/datafusion/physical-optimizer/src/lib.rs index 41dfdb84a9f4..439f1dc873d1 100644 --- a/datafusion/physical-optimizer/src/lib.rs +++ b/datafusion/physical-optimizer/src/lib.rs @@ -23,5 +23,6 @@ pub mod limit_pushdown; pub mod limited_distinct_aggregation; mod optimizer; pub mod output_requirements; +pub mod topk_aggregation; pub use optimizer::PhysicalOptimizerRule; diff --git a/datafusion/core/src/physical_optimizer/topk_aggregation.rs b/datafusion/physical-optimizer/src/topk_aggregation.rs similarity index 90% rename from datafusion/core/src/physical_optimizer/topk_aggregation.rs rename to datafusion/physical-optimizer/src/topk_aggregation.rs index b2c8f640fda0..0cb76d9c2393 100644 --- a/datafusion/core/src/physical_optimizer/topk_aggregation.rs +++ b/datafusion/physical-optimizer/src/topk_aggregation.rs @@ -19,12 +19,12 @@ use std::sync::Arc; -use crate::physical_plan::aggregates::AggregateExec; -use crate::physical_plan::coalesce_batches::CoalesceBatchesExec; -use crate::physical_plan::filter::FilterExec; -use crate::physical_plan::repartition::RepartitionExec; -use crate::physical_plan::sorts::sort::SortExec; -use crate::physical_plan::ExecutionPlan; +use datafusion_physical_plan::aggregates::AggregateExec; +use datafusion_physical_plan::coalesce_batches::CoalesceBatchesExec; +use datafusion_physical_plan::filter::FilterExec; +use datafusion_physical_plan::repartition::RepartitionExec; +use datafusion_physical_plan::sorts::sort::SortExec; +use datafusion_physical_plan::ExecutionPlan; use arrow_schema::DataType; use datafusion_common::config::ConfigOptions; @@ -33,7 +33,7 @@ use datafusion_common::Result; use datafusion_physical_expr::expressions::Column; use datafusion_physical_expr::PhysicalSortExpr; -use datafusion_physical_optimizer::PhysicalOptimizerRule; +use crate::PhysicalOptimizerRule; use itertools::Itertools; /// An optimizer rule that passes a `limit` hint to aggregations if the whole result is not needed @@ -76,7 +76,7 @@ impl TopKAggregation { aggr.group_expr().clone(), aggr.aggr_expr().to_vec(), aggr.filter_expr().to_vec(), - aggr.input().clone(), + Arc::clone(aggr.input()), aggr.input_schema(), ) .expect("Unable to copy Aggregate!") @@ -114,13 +114,13 @@ impl TopKAggregation { } } else { // or we continue down whitelisted nodes of other types - if !is_cardinality_preserving(plan.clone()) { + if !is_cardinality_preserving(Arc::clone(&plan)) { cardinality_preserved = false; } } Ok(Transformed::no(plan)) }; - let child = child.clone().transform_down(closure).data().ok()?; + let child = Arc::clone(child).transform_down(closure).data().ok()?; let sort = SortExec::new(sort.expr().to_vec(), child) .with_fetch(sort.fetch()) .with_preserve_partitioning(sort.preserve_partitioning()); From aed84c27461dd5c123ad86829200c0bb6a33db98 Mon Sep 17 00:00:00 2001 From: wiedld Date: Fri, 6 Sep 2024 01:10:37 -1000 Subject: [PATCH 101/154] Support Utf8View and BinaryView in substrait serialization. (#12199) * feat(12118): logical plan support for Utf8View * feat(12118): physical plan support for Utf8View * feat(12118): logical plan support for BinaryView * feat(12118): physical plan support for BinaryView * refactor(12118): remove BinaryView work-arounds, now that upstream arrow changes are in --------- Co-authored-by: Andrew Lamb --- .../substrait/src/logical_plan/consumer.rs | 6 ++- .../substrait/src/logical_plan/producer.rs | 24 +++++++++++- .../substrait/src/physical_plan/consumer.rs | 27 ++++++++++++- .../substrait/src/physical_plan/producer.rs | 39 ++++++++++++++++++- datafusion/substrait/src/variation_const.rs | 1 + .../tests/cases/roundtrip_logical_plan.rs | 6 ++- 6 files changed, 97 insertions(+), 6 deletions(-) diff --git a/datafusion/substrait/src/logical_plan/consumer.rs b/datafusion/substrait/src/logical_plan/consumer.rs index 21bef3c2c98e..01a854ffbdf2 100644 --- a/datafusion/substrait/src/logical_plan/consumer.rs +++ b/datafusion/substrait/src/logical_plan/consumer.rs @@ -42,7 +42,7 @@ use crate::variation_const::{ DECIMAL_128_TYPE_VARIATION_REF, DECIMAL_256_TYPE_VARIATION_REF, DEFAULT_CONTAINER_TYPE_VARIATION_REF, DEFAULT_TYPE_VARIATION_REF, INTERVAL_MONTH_DAY_NANO_TYPE_NAME, LARGE_CONTAINER_TYPE_VARIATION_REF, - UNSIGNED_INTEGER_TYPE_VARIATION_REF, + UNSIGNED_INTEGER_TYPE_VARIATION_REF, VIEW_CONTAINER_TYPE_VARIATION_REF, }; #[allow(deprecated)] use crate::variation_const::{ @@ -1432,6 +1432,7 @@ fn from_substrait_type( r#type::Kind::Binary(binary) => match binary.type_variation_reference { DEFAULT_CONTAINER_TYPE_VARIATION_REF => Ok(DataType::Binary), LARGE_CONTAINER_TYPE_VARIATION_REF => Ok(DataType::LargeBinary), + VIEW_CONTAINER_TYPE_VARIATION_REF => Ok(DataType::BinaryView), v => not_impl_err!( "Unsupported Substrait type variation {v} of type {s_kind:?}" ), @@ -1442,6 +1443,7 @@ fn from_substrait_type( r#type::Kind::String(string) => match string.type_variation_reference { DEFAULT_CONTAINER_TYPE_VARIATION_REF => Ok(DataType::Utf8), LARGE_CONTAINER_TYPE_VARIATION_REF => Ok(DataType::LargeUtf8), + VIEW_CONTAINER_TYPE_VARIATION_REF => Ok(DataType::Utf8View), v => not_impl_err!( "Unsupported Substrait type variation {v} of type {s_kind:?}" ), @@ -1759,6 +1761,7 @@ fn from_substrait_literal( Some(LiteralType::String(s)) => match lit.type_variation_reference { DEFAULT_CONTAINER_TYPE_VARIATION_REF => ScalarValue::Utf8(Some(s.clone())), LARGE_CONTAINER_TYPE_VARIATION_REF => ScalarValue::LargeUtf8(Some(s.clone())), + VIEW_CONTAINER_TYPE_VARIATION_REF => ScalarValue::Utf8View(Some(s.clone())), others => { return substrait_err!("Unknown type variation reference {others}"); } @@ -1768,6 +1771,7 @@ fn from_substrait_literal( LARGE_CONTAINER_TYPE_VARIATION_REF => { ScalarValue::LargeBinary(Some(b.clone())) } + VIEW_CONTAINER_TYPE_VARIATION_REF => ScalarValue::BinaryView(Some(b.clone())), others => { return substrait_err!("Unknown type variation reference {others}"); } diff --git a/datafusion/substrait/src/logical_plan/producer.rs b/datafusion/substrait/src/logical_plan/producer.rs index e71cf04cd341..f323ae146600 100644 --- a/datafusion/substrait/src/logical_plan/producer.rs +++ b/datafusion/substrait/src/logical_plan/producer.rs @@ -37,7 +37,7 @@ use crate::variation_const::{ DECIMAL_128_TYPE_VARIATION_REF, DECIMAL_256_TYPE_VARIATION_REF, DEFAULT_CONTAINER_TYPE_VARIATION_REF, DEFAULT_TYPE_VARIATION_REF, INTERVAL_MONTH_DAY_NANO_TYPE_NAME, LARGE_CONTAINER_TYPE_VARIATION_REF, - UNSIGNED_INTEGER_TYPE_VARIATION_REF, + UNSIGNED_INTEGER_TYPE_VARIATION_REF, VIEW_CONTAINER_TYPE_VARIATION_REF, }; use datafusion::arrow::array::{Array, GenericListArray, OffsetSizeTrait}; use datafusion::common::{ @@ -1450,6 +1450,12 @@ fn to_substrait_type( nullability, })), }), + DataType::BinaryView => Ok(substrait::proto::Type { + kind: Some(r#type::Kind::Binary(r#type::Binary { + type_variation_reference: VIEW_CONTAINER_TYPE_VARIATION_REF, + nullability, + })), + }), DataType::Utf8 => Ok(substrait::proto::Type { kind: Some(r#type::Kind::String(r#type::String { type_variation_reference: DEFAULT_CONTAINER_TYPE_VARIATION_REF, @@ -1462,6 +1468,12 @@ fn to_substrait_type( nullability, })), }), + DataType::Utf8View => Ok(substrait::proto::Type { + kind: Some(r#type::Kind::String(r#type::String { + type_variation_reference: VIEW_CONTAINER_TYPE_VARIATION_REF, + nullability, + })), + }), DataType::List(inner) => { let inner_type = to_substrait_type(inner.data_type(), inner.is_nullable(), extensions)?; @@ -1902,6 +1914,10 @@ fn to_substrait_literal( LiteralType::Binary(b.clone()), LARGE_CONTAINER_TYPE_VARIATION_REF, ), + ScalarValue::BinaryView(Some(b)) => ( + LiteralType::Binary(b.clone()), + VIEW_CONTAINER_TYPE_VARIATION_REF, + ), ScalarValue::FixedSizeBinary(_, Some(b)) => ( LiteralType::FixedBinary(b.clone()), DEFAULT_TYPE_VARIATION_REF, @@ -1914,6 +1930,10 @@ fn to_substrait_literal( LiteralType::String(s.clone()), LARGE_CONTAINER_TYPE_VARIATION_REF, ), + ScalarValue::Utf8View(Some(s)) => ( + LiteralType::String(s.clone()), + VIEW_CONTAINER_TYPE_VARIATION_REF, + ), ScalarValue::Decimal128(v, p, s) if v.is_some() => ( LiteralType::Decimal(Decimal { value: v.unwrap().to_le_bytes().to_vec(), @@ -2335,8 +2355,10 @@ mod test { round_trip_type(DataType::Binary)?; round_trip_type(DataType::FixedSizeBinary(10))?; round_trip_type(DataType::LargeBinary)?; + round_trip_type(DataType::BinaryView)?; round_trip_type(DataType::Utf8)?; round_trip_type(DataType::LargeUtf8)?; + round_trip_type(DataType::Utf8View)?; round_trip_type(DataType::Decimal128(10, 2))?; round_trip_type(DataType::Decimal256(30, 2))?; diff --git a/datafusion/substrait/src/physical_plan/consumer.rs b/datafusion/substrait/src/physical_plan/consumer.rs index 5a8b888ef1cc..a8f8ce048e0f 100644 --- a/datafusion/substrait/src/physical_plan/consumer.rs +++ b/datafusion/substrait/src/physical_plan/consumer.rs @@ -37,6 +37,11 @@ use substrait::proto::{ expression::MaskExpression, read_rel::ReadType, rel::RelType, Rel, }; +use crate::variation_const::{ + DEFAULT_CONTAINER_TYPE_VARIATION_REF, LARGE_CONTAINER_TYPE_VARIATION_REF, + VIEW_CONTAINER_TYPE_VARIATION_REF, +}; + /// Convert Substrait Rel to DataFusion ExecutionPlan #[async_recursion] pub async fn from_substrait_rel( @@ -177,7 +182,27 @@ fn to_field(name: &String, r#type: &Type) -> Result { } Kind::String(string) => { nullable = is_nullable(string.nullability); - Ok(DataType::Utf8) + match string.type_variation_reference { + DEFAULT_CONTAINER_TYPE_VARIATION_REF => Ok(DataType::Utf8), + LARGE_CONTAINER_TYPE_VARIATION_REF => Ok(DataType::LargeUtf8), + VIEW_CONTAINER_TYPE_VARIATION_REF => Ok(DataType::Utf8View), + _ => substrait_err!( + "Invalid type variation found for substrait string type class: {}", + string.type_variation_reference + ), + } + } + Kind::Binary(binary) => { + nullable = is_nullable(binary.nullability); + match binary.type_variation_reference { + DEFAULT_CONTAINER_TYPE_VARIATION_REF => Ok(DataType::Binary), + LARGE_CONTAINER_TYPE_VARIATION_REF => Ok(DataType::LargeBinary), + VIEW_CONTAINER_TYPE_VARIATION_REF => Ok(DataType::BinaryView), + _ => substrait_err!( + "Invalid type variation found for substrait binary type class: {}", + binary.type_variation_reference + ), + } } _ => substrait_err!( "Unsupported kind: {:?} in the type with name {}", diff --git a/datafusion/substrait/src/physical_plan/producer.rs b/datafusion/substrait/src/physical_plan/producer.rs index 57fe68c4a780..7279785ae873 100644 --- a/datafusion/substrait/src/physical_plan/producer.rs +++ b/datafusion/substrait/src/physical_plan/producer.rs @@ -23,7 +23,7 @@ use std::collections::HashMap; use substrait::proto::expression::mask_expression::{StructItem, StructSelect}; use substrait::proto::expression::MaskExpression; use substrait::proto::r#type::{ - Boolean, Fp64, Kind, Nullability, String as SubstraitString, Struct, I64, + Binary, Boolean, Fp64, Kind, Nullability, String as SubstraitString, Struct, I64, }; use substrait::proto::read_rel::local_files::file_or_files::ParquetReadOptions; use substrait::proto::read_rel::local_files::file_or_files::{FileFormat, PathType}; @@ -35,6 +35,11 @@ use substrait::proto::ReadRel; use substrait::proto::Rel; use substrait::proto::{extensions, NamedStruct, Type}; +use crate::variation_const::{ + DEFAULT_CONTAINER_TYPE_VARIATION_REF, LARGE_CONTAINER_TYPE_VARIATION_REF, + VIEW_CONTAINER_TYPE_VARIATION_REF, +}; + /// Convert DataFusion ExecutionPlan to Substrait Rel pub fn to_substrait_rel( plan: &dyn ExecutionPlan, @@ -155,7 +160,37 @@ fn to_substrait_type(data_type: &DataType, nullable: bool) -> Result { }), DataType::Utf8 => Ok(Type { kind: Some(Kind::String(SubstraitString { - type_variation_reference: 0, + type_variation_reference: DEFAULT_CONTAINER_TYPE_VARIATION_REF, + nullability, + })), + }), + DataType::LargeUtf8 => Ok(Type { + kind: Some(Kind::String(SubstraitString { + type_variation_reference: LARGE_CONTAINER_TYPE_VARIATION_REF, + nullability, + })), + }), + DataType::Utf8View => Ok(Type { + kind: Some(Kind::String(SubstraitString { + type_variation_reference: VIEW_CONTAINER_TYPE_VARIATION_REF, + nullability, + })), + }), + DataType::Binary => Ok(Type { + kind: Some(Kind::Binary(Binary { + type_variation_reference: DEFAULT_CONTAINER_TYPE_VARIATION_REF, + nullability, + })), + }), + DataType::LargeBinary => Ok(Type { + kind: Some(Kind::Binary(Binary { + type_variation_reference: LARGE_CONTAINER_TYPE_VARIATION_REF, + nullability, + })), + }), + DataType::BinaryView => Ok(Type { + kind: Some(Kind::Binary(Binary { + type_variation_reference: VIEW_CONTAINER_TYPE_VARIATION_REF, nullability, })), }), diff --git a/datafusion/substrait/src/variation_const.rs b/datafusion/substrait/src/variation_const.rs index 1525da764509..a3e76389d510 100644 --- a/datafusion/substrait/src/variation_const.rs +++ b/datafusion/substrait/src/variation_const.rs @@ -52,6 +52,7 @@ pub const DATE_32_TYPE_VARIATION_REF: u32 = 0; pub const DATE_64_TYPE_VARIATION_REF: u32 = 1; pub const DEFAULT_CONTAINER_TYPE_VARIATION_REF: u32 = 0; pub const LARGE_CONTAINER_TYPE_VARIATION_REF: u32 = 1; +pub const VIEW_CONTAINER_TYPE_VARIATION_REF: u32 = 2; pub const DECIMAL_128_TYPE_VARIATION_REF: u32 = 0; pub const DECIMAL_256_TYPE_VARIATION_REF: u32 = 1; diff --git a/datafusion/substrait/tests/cases/roundtrip_logical_plan.rs b/datafusion/substrait/tests/cases/roundtrip_logical_plan.rs index 083a589fce26..98daac65e1cf 100644 --- a/datafusion/substrait/tests/cases/roundtrip_logical_plan.rs +++ b/datafusion/substrait/tests/cases/roundtrip_logical_plan.rs @@ -716,8 +716,10 @@ async fn all_type_literal() -> Result<()> { date32_col = arrow_cast('2020-01-01', 'Date32') AND binary_col = arrow_cast('binary', 'Binary') AND large_binary_col = arrow_cast('large_binary', 'LargeBinary') AND + view_binary_col = arrow_cast('binary_view', 'BinaryView') AND utf8_col = arrow_cast('utf8', 'Utf8') AND - large_utf8_col = arrow_cast('large_utf8', 'LargeUtf8');", + large_utf8_col = arrow_cast('large_utf8', 'LargeUtf8') AND + view_utf8_col = arrow_cast('utf8_view', 'Utf8View');", ) .await } @@ -1231,9 +1233,11 @@ async fn create_all_type_context() -> Result { Field::new("date64_col", DataType::Date64, true), Field::new("binary_col", DataType::Binary, true), Field::new("large_binary_col", DataType::LargeBinary, true), + Field::new("view_binary_col", DataType::BinaryView, true), Field::new("fixed_size_binary_col", DataType::FixedSizeBinary(42), true), Field::new("utf8_col", DataType::Utf8, true), Field::new("large_utf8_col", DataType::LargeUtf8, true), + Field::new("view_utf8_col", DataType::Utf8View, true), Field::new_list("list_col", Field::new("item", DataType::Int64, true), true), Field::new_list( "large_list_col", From f2a8b072a70fce9061874ef99fe670564d197740 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Berkay=20=C5=9Eahin?= <124376117+berkaysynnada@users.noreply.github.com> Date: Fri, 6 Sep 2024 17:10:02 +0300 Subject: [PATCH 102/154] Fix Possible Congestion Scenario in `SortPreservingMergeExec` (#12302) * Ready Update merge_fuzz.rs Update merge.rs Update merge.rs Update merge.rs Update merge_fuzz.rs Update merge.rs Update merge_fuzz.rs Update merge.rs Counter is not enough Termination logic with counter * Add comments * Increase threshold * Debug * Remove merge.rs changes * Use waker * Update merge.rs * Simplify the test * Update merge_fuzz.rs * Update merge.rs * Addresses the latest review * fix clippy * Use VecDeque for rotation --- .../core/tests/fuzz_cases/merge_fuzz.rs | 1 + datafusion/physical-plan/src/sorts/merge.rs | 55 ++++-- .../src/sorts/sort_preserving_merge.rs | 171 +++++++++++++++++- 3 files changed, 213 insertions(+), 14 deletions(-) diff --git a/datafusion/core/tests/fuzz_cases/merge_fuzz.rs b/datafusion/core/tests/fuzz_cases/merge_fuzz.rs index 95cd75f50a00..4eb1070e6c85 100644 --- a/datafusion/core/tests/fuzz_cases/merge_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/merge_fuzz.rs @@ -16,6 +16,7 @@ // under the License. //! Fuzz Test for various corner cases merging streams of RecordBatches + use std::sync::Arc; use arrow::{ diff --git a/datafusion/physical-plan/src/sorts/merge.rs b/datafusion/physical-plan/src/sorts/merge.rs index 85418ff36119..875922ac34b5 100644 --- a/datafusion/physical-plan/src/sorts/merge.rs +++ b/datafusion/physical-plan/src/sorts/merge.rs @@ -18,19 +18,23 @@ //! Merge that deals with an arbitrary size of streaming inputs. //! This is an order-preserving merge. +use std::collections::VecDeque; +use std::pin::Pin; +use std::sync::Arc; +use std::task::{ready, Context, Poll}; + use crate::metrics::BaselineMetrics; use crate::sorts::builder::BatchBuilder; use crate::sorts::cursor::{Cursor, CursorValues}; use crate::sorts::stream::PartitionedStream; use crate::RecordBatchStream; + use arrow::datatypes::SchemaRef; use arrow::record_batch::RecordBatch; use datafusion_common::Result; use datafusion_execution::memory_pool::MemoryReservation; + use futures::Stream; -use std::pin::Pin; -use std::sync::Arc; -use std::task::{ready, Context, Poll}; /// A fallible [`PartitionedStream`] of [`Cursor`] and [`RecordBatch`] type CursorStream = Box>>; @@ -86,7 +90,7 @@ pub(crate) struct SortPreservingMergeStream { /// been updated loser_tree_adjusted: bool, - /// target batch size + /// Target batch size batch_size: usize, /// Cursors for each input partition. `None` means the input is exhausted @@ -97,6 +101,12 @@ pub(crate) struct SortPreservingMergeStream { /// number of rows produced produced: usize, + + /// This queue contains partition indices in order. When a partition is polled and returns `Poll::Ready`, + /// it is removed from the vector. If a partition returns `Poll::Pending`, it is moved to the end of the + /// vector to ensure the next iteration starts with a different partition, preventing the same partition + /// from being continuously polled. + uninitiated_partitions: VecDeque, } impl SortPreservingMergeStream { @@ -121,6 +131,7 @@ impl SortPreservingMergeStream { batch_size, fetch, produced: 0, + uninitiated_partitions: (0..stream_count).collect(), } } @@ -154,14 +165,36 @@ impl SortPreservingMergeStream { if self.aborted { return Poll::Ready(None); } - // try to initialize the loser tree + // Once all partitions have set their corresponding cursors for the loser tree, + // we skip the following block. Until then, this function may be called multiple + // times and can return Poll::Pending if any partition returns Poll::Pending. if self.loser_tree.is_empty() { - // Ensure all non-exhausted streams have a cursor from which - // rows can be pulled - for i in 0..self.streams.partitions() { - if let Err(e) = ready!(self.maybe_poll_stream(cx, i)) { - self.aborted = true; - return Poll::Ready(Some(Err(e))); + let remaining_partitions = self.uninitiated_partitions.clone(); + for i in remaining_partitions { + match self.maybe_poll_stream(cx, i) { + Poll::Ready(Err(e)) => { + self.aborted = true; + return Poll::Ready(Some(Err(e))); + } + Poll::Pending => { + // If a partition returns Poll::Pending, to avoid continuously polling it + // and potentially increasing upstream buffer sizes, we move it to the + // back of the polling queue. + if let Some(front) = self.uninitiated_partitions.pop_front() { + // This pop_front can never return `None`. + self.uninitiated_partitions.push_back(front); + } + // This function could remain in a pending state, so we manually wake it here. + // However, this approach can be investigated further to find a more natural way + // to avoid disrupting the runtime scheduler. + cx.waker().wake_by_ref(); + return Poll::Pending; + } + _ => { + // If the polling result is Poll::Ready(Some(batch)) or Poll::Ready(None), + // we remove this partition from the queue so it is not polled again. + self.uninitiated_partitions.retain(|idx| *idx != i); + } } } self.init_loser_tree(); diff --git a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs index 131fa71217cc..4d333175bf75 100644 --- a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs +++ b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs @@ -300,6 +300,11 @@ impl ExecutionPlan for SortPreservingMergeExec { #[cfg(test)] mod tests { + use std::fmt::Formatter; + use std::pin::Pin; + use std::sync::Mutex; + use std::task::{Context, Poll}; + use std::time::Duration; use super::*; use crate::coalesce_partitions::CoalescePartitionsExec; @@ -310,16 +315,23 @@ mod tests { use crate::stream::RecordBatchReceiverStream; use crate::test::exec::{assert_strong_count_converges_to_zero, BlockingExec}; use crate::test::{self, assert_is_pending, make_partition}; - use crate::{collect, common}; + use crate::{collect, common, ExecutionMode}; use arrow::array::{ArrayRef, Int32Array, StringArray, TimestampNanosecondArray}; use arrow::compute::SortOptions; use arrow::datatypes::{DataType, Field, Schema}; use arrow::record_batch::RecordBatch; - use datafusion_common::{assert_batches_eq, assert_contains}; + use arrow_schema::SchemaRef; + use datafusion_common::{assert_batches_eq, assert_contains, DataFusionError}; + use datafusion_common_runtime::SpawnedTask; use datafusion_execution::config::SessionConfig; + use datafusion_execution::RecordBatchStream; + use datafusion_physical_expr::expressions::Column; + use datafusion_physical_expr::EquivalenceProperties; + use datafusion_physical_expr_common::physical_expr::PhysicalExpr; - use futures::{FutureExt, StreamExt}; + use futures::{FutureExt, Stream, StreamExt}; + use tokio::time::timeout; #[tokio::test] async fn test_merge_interleave() { @@ -1141,4 +1153,157 @@ mod tests { collected.as_slice() ); } + + /// It returns pending for the 2nd partition until the 3rd partition is polled. The 1st + /// partition is exhausted from the start, and if it is polled more than one, it panics. + #[derive(Debug, Clone)] + struct CongestedExec { + schema: Schema, + cache: PlanProperties, + congestion_cleared: Arc>, + } + + impl CongestedExec { + fn compute_properties(schema: SchemaRef) -> PlanProperties { + let columns = schema + .fields + .iter() + .enumerate() + .map(|(i, f)| Arc::new(Column::new(f.name(), i)) as Arc) + .collect::>(); + let mut eq_properties = EquivalenceProperties::new(schema); + eq_properties.add_new_orderings(vec![columns + .iter() + .map(|expr| { + PhysicalSortExpr::new(Arc::clone(expr), SortOptions::default()) + }) + .collect::>()]); + let mode = ExecutionMode::Unbounded; + PlanProperties::new(eq_properties, Partitioning::Hash(columns, 3), mode) + } + } + + impl ExecutionPlan for CongestedExec { + fn name(&self) -> &'static str { + Self::static_name() + } + fn as_any(&self) -> &dyn Any { + self + } + fn properties(&self) -> &PlanProperties { + &self.cache + } + fn children(&self) -> Vec<&Arc> { + vec![] + } + fn with_new_children( + self: Arc, + _: Vec>, + ) -> Result> { + Ok(self) + } + fn execute( + &self, + partition: usize, + _context: Arc, + ) -> Result { + Ok(Box::pin(CongestedStream { + schema: Arc::new(self.schema.clone()), + none_polled_once: false, + congestion_cleared: Arc::clone(&self.congestion_cleared), + partition, + })) + } + } + + impl DisplayAs for CongestedExec { + fn fmt_as(&self, t: DisplayFormatType, f: &mut Formatter) -> std::fmt::Result { + match t { + DisplayFormatType::Default | DisplayFormatType::Verbose => { + write!(f, "CongestedExec",).unwrap() + } + } + Ok(()) + } + } + + /// It returns pending for the 2nd partition until the 3rd partition is polled. The 1st + /// partition is exhausted from the start, and if it is polled more than once, it panics. + #[derive(Debug)] + pub struct CongestedStream { + schema: SchemaRef, + none_polled_once: bool, + congestion_cleared: Arc>, + partition: usize, + } + + impl Stream for CongestedStream { + type Item = Result; + fn poll_next( + mut self: Pin<&mut Self>, + _cx: &mut Context<'_>, + ) -> Poll> { + match self.partition { + 0 => { + if self.none_polled_once { + panic!("Exhausted stream is polled more than one") + } else { + self.none_polled_once = true; + Poll::Ready(None) + } + } + 1 => { + let cleared = self.congestion_cleared.lock().unwrap(); + if *cleared { + Poll::Ready(None) + } else { + Poll::Pending + } + } + 2 => { + let mut cleared = self.congestion_cleared.lock().unwrap(); + *cleared = true; + Poll::Ready(None) + } + _ => unreachable!(), + } + } + } + + impl RecordBatchStream for CongestedStream { + fn schema(&self) -> SchemaRef { + Arc::clone(&self.schema) + } + } + + #[tokio::test] + async fn test_spm_congestion() -> Result<()> { + let task_ctx = Arc::new(TaskContext::default()); + let schema = Schema::new(vec![Field::new("c1", DataType::UInt64, false)]); + let source = CongestedExec { + schema: schema.clone(), + cache: CongestedExec::compute_properties(Arc::new(schema.clone())), + congestion_cleared: Arc::new(Mutex::new(false)), + }; + let spm = SortPreservingMergeExec::new( + vec![PhysicalSortExpr::new( + Arc::new(Column::new("c1", 0)), + SortOptions::default(), + )], + Arc::new(source), + ); + let spm_task = SpawnedTask::spawn(collect(Arc::new(spm), task_ctx)); + + let result = timeout(Duration::from_secs(3), spm_task.join()).await; + match result { + Ok(Ok(Ok(_batches))) => Ok(()), + Ok(Ok(Err(e))) => Err(e), + Ok(Err(_)) => Err(DataFusionError::Execution( + "SortPreservingMerge task panicked or was cancelled".to_string(), + )), + Err(_) => Err(DataFusionError::Execution( + "SortPreservingMerge caused a deadlock".to_string(), + )), + } + } } From 22e06b5c3c5f27eec7171bcc96a117ac0b19bc65 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=BC=A0=E6=9E=97=E4=BC=9F?= Date: Fri, 6 Sep 2024 22:11:21 +0800 Subject: [PATCH 103/154] Re-export variable provider (#12351) --- datafusion/core/src/lib.rs | 6 +++++- datafusion/core/src/variable/mod.rs | 20 -------------------- 2 files changed, 5 insertions(+), 21 deletions(-) delete mode 100644 datafusion/core/src/variable/mod.rs diff --git a/datafusion/core/src/lib.rs b/datafusion/core/src/lib.rs index 67f3cb01c0a4..9c368415bb05 100644 --- a/datafusion/core/src/lib.rs +++ b/datafusion/core/src/lib.rs @@ -509,7 +509,6 @@ pub mod physical_optimizer; pub mod physical_planner; pub mod prelude; pub mod scalar; -pub mod variable; // re-export dependencies from arrow-rs to minimize version maintenance for crate users pub use arrow; @@ -602,6 +601,11 @@ pub mod functions_window { pub use datafusion_functions_window::*; } +/// re-export of variable provider for `@name` and `@@name` style runtime values. +pub mod variable { + pub use datafusion_expr::var_provider::{VarProvider, VarType}; +} + #[cfg(test)] pub mod test; pub mod test_util; diff --git a/datafusion/core/src/variable/mod.rs b/datafusion/core/src/variable/mod.rs deleted file mode 100644 index 475f7570a8ee..000000000000 --- a/datafusion/core/src/variable/mod.rs +++ /dev/null @@ -1,20 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -//! Variable provider for `@name` and `@@name` style runtime values. - -pub use datafusion_expr::var_provider::{VarProvider, VarType}; From ddfbf7a26b691a33bbd35c775cc3178c7372a41a Mon Sep 17 00:00:00 2001 From: Georgi Krastev Date: Fri, 6 Sep 2024 17:12:07 +0300 Subject: [PATCH 104/154] Support encoding and decoding UnnestExec (#12344) --- datafusion/physical-plan/src/unnest.rs | 19 ++ datafusion/proto/proto/datafusion.proto | 11 +- datafusion/proto/src/generated/pbjson.rs | 180 ++++++++++++++++++ datafusion/proto/src/generated/prost.rs | 18 +- datafusion/proto/src/physical_plan/mod.rs | 46 ++++- .../tests/cases/roundtrip_physical_plan.rs | 33 +++- 6 files changed, 303 insertions(+), 4 deletions(-) diff --git a/datafusion/physical-plan/src/unnest.rs b/datafusion/physical-plan/src/unnest.rs index 67c2aaedbebf..b99d0d838870 100644 --- a/datafusion/physical-plan/src/unnest.rs +++ b/datafusion/physical-plan/src/unnest.rs @@ -109,6 +109,25 @@ impl UnnestExec { input.execution_mode(), ) } + + /// Input execution plan + pub fn input(&self) -> &Arc { + &self.input + } + + /// indices of the list-typed columns in the input schema + pub fn list_column_indices(&self) -> &[usize] { + &self.list_column_indices + } + + /// indices of the struct-typed columns in the input schema + pub fn struct_column_indices(&self) -> &[usize] { + &self.struct_column_indices + } + + pub fn options(&self) -> &UnnestOptions { + &self.options + } } impl DisplayAs for UnnestExec { diff --git a/datafusion/proto/proto/datafusion.proto b/datafusion/proto/proto/datafusion.proto index 19759a897068..2d911537bc85 100644 --- a/datafusion/proto/proto/datafusion.proto +++ b/datafusion/proto/proto/datafusion.proto @@ -693,10 +693,11 @@ message PhysicalPlanNode { AnalyzeExecNode analyze = 23; JsonSinkExecNode json_sink = 24; SymmetricHashJoinExecNode symmetric_hash_join = 25; - InterleaveExecNode interleave = 26; + InterleaveExecNode interleave = 26; PlaceholderRowExecNode placeholder_row = 27; CsvSinkExecNode csv_sink = 28; ParquetSinkExecNode parquet_sink = 29; + UnnestExecNode unnest = 30; } } @@ -754,6 +755,14 @@ message ParquetSinkExecNode { PhysicalSortExprNodeCollection sort_order = 4; } +message UnnestExecNode { + PhysicalPlanNode input = 1; + datafusion_common.Schema schema = 2; + repeated uint64 list_type_columns = 3; + repeated uint64 struct_type_columns = 4; + UnnestOptions options = 5; +} + message PhysicalExtensionNode { bytes node = 1; repeated PhysicalPlanNode inputs = 2; diff --git a/datafusion/proto/src/generated/pbjson.rs b/datafusion/proto/src/generated/pbjson.rs index 1c433c2572c4..ef2f8a92fc85 100644 --- a/datafusion/proto/src/generated/pbjson.rs +++ b/datafusion/proto/src/generated/pbjson.rs @@ -14714,6 +14714,9 @@ impl serde::Serialize for PhysicalPlanNode { physical_plan_node::PhysicalPlanType::ParquetSink(v) => { struct_ser.serialize_field("parquetSink", v)?; } + physical_plan_node::PhysicalPlanType::Unnest(v) => { + struct_ser.serialize_field("unnest", v)?; + } } } struct_ser.end() @@ -14769,6 +14772,7 @@ impl<'de> serde::Deserialize<'de> for PhysicalPlanNode { "csvSink", "parquet_sink", "parquetSink", + "unnest", ]; #[allow(clippy::enum_variant_names)] @@ -14801,6 +14805,7 @@ impl<'de> serde::Deserialize<'de> for PhysicalPlanNode { PlaceholderRow, CsvSink, ParquetSink, + Unnest, } impl<'de> serde::Deserialize<'de> for GeneratedField { fn deserialize(deserializer: D) -> std::result::Result @@ -14850,6 +14855,7 @@ impl<'de> serde::Deserialize<'de> for PhysicalPlanNode { "placeholderRow" | "placeholder_row" => Ok(GeneratedField::PlaceholderRow), "csvSink" | "csv_sink" => Ok(GeneratedField::CsvSink), "parquetSink" | "parquet_sink" => Ok(GeneratedField::ParquetSink), + "unnest" => Ok(GeneratedField::Unnest), _ => Err(serde::de::Error::unknown_field(value, FIELDS)), } } @@ -15066,6 +15072,13 @@ impl<'de> serde::Deserialize<'de> for PhysicalPlanNode { return Err(serde::de::Error::duplicate_field("parquetSink")); } physical_plan_type__ = map_.next_value::<::std::option::Option<_>>()?.map(physical_plan_node::PhysicalPlanType::ParquetSink) +; + } + GeneratedField::Unnest => { + if physical_plan_type__.is_some() { + return Err(serde::de::Error::duplicate_field("unnest")); + } + physical_plan_type__ = map_.next_value::<::std::option::Option<_>>()?.map(physical_plan_node::PhysicalPlanType::Unnest) ; } } @@ -19339,6 +19352,173 @@ impl<'de> serde::Deserialize<'de> for Unnest { deserializer.deserialize_struct("datafusion.Unnest", FIELDS, GeneratedVisitor) } } +impl serde::Serialize for UnnestExecNode { + #[allow(deprecated)] + fn serialize(&self, serializer: S) -> std::result::Result + where + S: serde::Serializer, + { + use serde::ser::SerializeStruct; + let mut len = 0; + if self.input.is_some() { + len += 1; + } + if self.schema.is_some() { + len += 1; + } + if !self.list_type_columns.is_empty() { + len += 1; + } + if !self.struct_type_columns.is_empty() { + len += 1; + } + if self.options.is_some() { + len += 1; + } + let mut struct_ser = serializer.serialize_struct("datafusion.UnnestExecNode", len)?; + if let Some(v) = self.input.as_ref() { + struct_ser.serialize_field("input", v)?; + } + if let Some(v) = self.schema.as_ref() { + struct_ser.serialize_field("schema", v)?; + } + if !self.list_type_columns.is_empty() { + struct_ser.serialize_field("listTypeColumns", &self.list_type_columns.iter().map(ToString::to_string).collect::>())?; + } + if !self.struct_type_columns.is_empty() { + struct_ser.serialize_field("structTypeColumns", &self.struct_type_columns.iter().map(ToString::to_string).collect::>())?; + } + if let Some(v) = self.options.as_ref() { + struct_ser.serialize_field("options", v)?; + } + struct_ser.end() + } +} +impl<'de> serde::Deserialize<'de> for UnnestExecNode { + #[allow(deprecated)] + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + const FIELDS: &[&str] = &[ + "input", + "schema", + "list_type_columns", + "listTypeColumns", + "struct_type_columns", + "structTypeColumns", + "options", + ]; + + #[allow(clippy::enum_variant_names)] + enum GeneratedField { + Input, + Schema, + ListTypeColumns, + StructTypeColumns, + Options, + } + impl<'de> serde::Deserialize<'de> for GeneratedField { + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + struct GeneratedVisitor; + + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = GeneratedField; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(formatter, "expected one of: {:?}", &FIELDS) + } + + #[allow(unused_variables)] + fn visit_str(self, value: &str) -> std::result::Result + where + E: serde::de::Error, + { + match value { + "input" => Ok(GeneratedField::Input), + "schema" => Ok(GeneratedField::Schema), + "listTypeColumns" | "list_type_columns" => Ok(GeneratedField::ListTypeColumns), + "structTypeColumns" | "struct_type_columns" => Ok(GeneratedField::StructTypeColumns), + "options" => Ok(GeneratedField::Options), + _ => Err(serde::de::Error::unknown_field(value, FIELDS)), + } + } + } + deserializer.deserialize_identifier(GeneratedVisitor) + } + } + struct GeneratedVisitor; + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = UnnestExecNode; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + formatter.write_str("struct datafusion.UnnestExecNode") + } + + fn visit_map(self, mut map_: V) -> std::result::Result + where + V: serde::de::MapAccess<'de>, + { + let mut input__ = None; + let mut schema__ = None; + let mut list_type_columns__ = None; + let mut struct_type_columns__ = None; + let mut options__ = None; + while let Some(k) = map_.next_key()? { + match k { + GeneratedField::Input => { + if input__.is_some() { + return Err(serde::de::Error::duplicate_field("input")); + } + input__ = map_.next_value()?; + } + GeneratedField::Schema => { + if schema__.is_some() { + return Err(serde::de::Error::duplicate_field("schema")); + } + schema__ = map_.next_value()?; + } + GeneratedField::ListTypeColumns => { + if list_type_columns__.is_some() { + return Err(serde::de::Error::duplicate_field("listTypeColumns")); + } + list_type_columns__ = + Some(map_.next_value::>>()? + .into_iter().map(|x| x.0).collect()) + ; + } + GeneratedField::StructTypeColumns => { + if struct_type_columns__.is_some() { + return Err(serde::de::Error::duplicate_field("structTypeColumns")); + } + struct_type_columns__ = + Some(map_.next_value::>>()? + .into_iter().map(|x| x.0).collect()) + ; + } + GeneratedField::Options => { + if options__.is_some() { + return Err(serde::de::Error::duplicate_field("options")); + } + options__ = map_.next_value()?; + } + } + } + Ok(UnnestExecNode { + input: input__, + schema: schema__, + list_type_columns: list_type_columns__.unwrap_or_default(), + struct_type_columns: struct_type_columns__.unwrap_or_default(), + options: options__, + }) + } + } + deserializer.deserialize_struct("datafusion.UnnestExecNode", FIELDS, GeneratedVisitor) + } +} impl serde::Serialize for UnnestNode { #[allow(deprecated)] fn serialize(&self, serializer: S) -> std::result::Result diff --git a/datafusion/proto/src/generated/prost.rs b/datafusion/proto/src/generated/prost.rs index dbcf7672a48c..fc8a5736531a 100644 --- a/datafusion/proto/src/generated/prost.rs +++ b/datafusion/proto/src/generated/prost.rs @@ -1034,7 +1034,7 @@ pub mod table_reference { pub struct PhysicalPlanNode { #[prost( oneof = "physical_plan_node::PhysicalPlanType", - tags = "1, 2, 3, 4, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29" + tags = "1, 2, 3, 4, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30" )] pub physical_plan_type: ::core::option::Option, } @@ -1101,6 +1101,8 @@ pub mod physical_plan_node { CsvSink(::prost::alloc::boxed::Box), #[prost(message, tag = "29")] ParquetSink(::prost::alloc::boxed::Box), + #[prost(message, tag = "30")] + Unnest(::prost::alloc::boxed::Box), } } #[allow(clippy::derive_partial_eq_without_eq)] @@ -1197,6 +1199,20 @@ pub struct ParquetSinkExecNode { } #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] +pub struct UnnestExecNode { + #[prost(message, optional, boxed, tag = "1")] + pub input: ::core::option::Option<::prost::alloc::boxed::Box>, + #[prost(message, optional, tag = "2")] + pub schema: ::core::option::Option, + #[prost(uint64, repeated, tag = "3")] + pub list_type_columns: ::prost::alloc::vec::Vec, + #[prost(uint64, repeated, tag = "4")] + pub struct_type_columns: ::prost::alloc::vec::Vec, + #[prost(message, optional, tag = "5")] + pub options: ::core::option::Option, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] pub struct PhysicalExtensionNode { #[prost(bytes = "vec", tag = "1")] pub node: ::prost::alloc::vec::Vec, diff --git a/datafusion/proto/src/physical_plan/mod.rs b/datafusion/proto/src/physical_plan/mod.rs index e622af745062..a0934e54005e 100644 --- a/datafusion/proto/src/physical_plan/mod.rs +++ b/datafusion/proto/src/physical_plan/mod.rs @@ -58,6 +58,7 @@ use datafusion::physical_plan::repartition::RepartitionExec; use datafusion::physical_plan::sorts::sort::SortExec; use datafusion::physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; use datafusion::physical_plan::union::{InterleaveExec, UnionExec}; +use datafusion::physical_plan::unnest::UnnestExec; use datafusion::physical_plan::windows::{BoundedWindowAggExec, WindowAggExec}; use datafusion::physical_plan::{ ExecutionPlan, InputOrderMode, PhysicalExpr, WindowExpr, @@ -66,7 +67,6 @@ use datafusion_common::{internal_err, not_impl_err, DataFusionError, Result}; use datafusion_expr::{AggregateUDF, ScalarUDF}; use crate::common::{byte_to_string, str_to_byte}; -use crate::convert_required; use crate::physical_plan::from_proto::{ parse_physical_expr, parse_physical_sort_expr, parse_physical_sort_exprs, parse_physical_window_expr, parse_protobuf_file_scan_config, @@ -79,6 +79,7 @@ use crate::protobuf::physical_aggregate_expr_node::AggregateFunction; use crate::protobuf::physical_expr_node::ExprType; use crate::protobuf::physical_plan_node::PhysicalPlanType; use crate::protobuf::{self, proto_error, window_agg_exec_node}; +use crate::{convert_required, into_required}; use self::from_proto::parse_protobuf_partitioning; use self::to_proto::{serialize_partitioning, serialize_physical_expr}; @@ -1085,6 +1086,22 @@ impl AsExecutionPlan for protobuf::PhysicalPlanNode { sort_order, ))) } + PhysicalPlanType::Unnest(unnest) => { + let input = into_physical_plan( + &unnest.input, + registry, + runtime, + extension_codec, + )?; + + Ok(Arc::new(UnnestExec::new( + input, + unnest.list_type_columns.iter().map(|c| *c as _).collect(), + unnest.struct_type_columns.iter().map(|c| *c as _).collect(), + Arc::new(convert_required!(unnest.schema)?), + into_required!(unnest.options)?, + ))) + } } } @@ -1935,6 +1952,33 @@ impl AsExecutionPlan for protobuf::PhysicalPlanNode { // If unknown DataSink then let extension handle it } + if let Some(exec) = plan.downcast_ref::() { + let input = protobuf::PhysicalPlanNode::try_from_physical_plan( + exec.input().to_owned(), + extension_codec, + )?; + + return Ok(protobuf::PhysicalPlanNode { + physical_plan_type: Some(PhysicalPlanType::Unnest(Box::new( + protobuf::UnnestExecNode { + input: Some(Box::new(input)), + schema: Some(exec.schema().try_into()?), + list_type_columns: exec + .list_column_indices() + .iter() + .map(|c| *c as _) + .collect(), + struct_type_columns: exec + .struct_column_indices() + .iter() + .map(|c| *c as _) + .collect(), + options: Some(exec.options().into()), + }, + ))), + }); + } + let mut buf: Vec = vec![]; match extension_codec.try_encode(Arc::clone(&plan_clone), &mut buf) { Ok(_) => { diff --git a/datafusion/proto/tests/cases/roundtrip_physical_plan.rs b/datafusion/proto/tests/cases/roundtrip_physical_plan.rs index b2ded88dfaf4..97d65ba87197 100644 --- a/datafusion/proto/tests/cases/roundtrip_physical_plan.rs +++ b/datafusion/proto/tests/cases/roundtrip_physical_plan.rs @@ -24,6 +24,7 @@ use std::vec; use arrow::array::RecordBatch; use arrow::csv::WriterBuilder; +use arrow::datatypes::{Fields, TimeUnit}; use datafusion::physical_expr::aggregate::AggregateExprBuilder; use datafusion::physical_plan::coalesce_batches::CoalesceBatchesExec; use datafusion_functions_aggregate::approx_percentile_cont::approx_percentile_cont_udaf; @@ -71,6 +72,7 @@ use datafusion::physical_plan::repartition::RepartitionExec; use datafusion::physical_plan::sorts::sort::SortExec; use datafusion::physical_plan::udaf::AggregateFunctionExpr; use datafusion::physical_plan::union::{InterleaveExec, UnionExec}; +use datafusion::physical_plan::unnest::UnnestExec; use datafusion::physical_plan::windows::{ BuiltInWindowExpr, PlainAggregateWindowExpr, WindowAggExec, }; @@ -82,7 +84,9 @@ use datafusion_common::file_options::csv_writer::CsvWriterOptions; use datafusion_common::file_options::json_writer::JsonWriterOptions; use datafusion_common::parsers::CompressionTypeVariant; use datafusion_common::stats::Precision; -use datafusion_common::{internal_err, not_impl_err, DataFusionError, Result}; +use datafusion_common::{ + internal_err, not_impl_err, DataFusionError, Result, UnnestOptions, +}; use datafusion_expr::{ Accumulator, AccumulatorFactoryFunction, AggregateUDF, ColumnarValue, ScalarUDF, Signature, SimpleAggregateUDF, WindowFrame, WindowFrameBound, @@ -1345,3 +1349,30 @@ fn roundtrip_interleave() -> Result<()> { let interleave = InterleaveExec::try_new(inputs)?; roundtrip_test(Arc::new(interleave)) } + +#[test] +fn roundtrip_unnest() -> Result<()> { + let fa = Field::new("a", DataType::Int64, true); + let fb0 = Field::new_list_field(DataType::Utf8, true); + let fb = Field::new_list("b", fb0.clone(), false); + let fc1 = Field::new("c1", DataType::Boolean, false); + let fc2 = Field::new("c2", DataType::Date64, true); + let fc = Field::new_struct("c", Fields::from(vec![fc1.clone(), fc2.clone()]), true); + let fd0 = Field::new_list_field(DataType::Float32, false); + let fd = Field::new_list("d", fd0.clone(), true); + let fe1 = Field::new("e1", DataType::UInt16, false); + let fe2 = Field::new("e2", DataType::Duration(TimeUnit::Millisecond), true); + let fe3 = Field::new("e3", DataType::Timestamp(TimeUnit::Millisecond, None), true); + let fe_fields = Fields::from(vec![fe1.clone(), fe2.clone(), fe3.clone()]); + let fe = Field::new_struct("e", fe_fields, false); + + let fb0 = fb0.with_name("b"); + let fd0 = fd0.with_name("d"); + let input_schema = Arc::new(Schema::new(vec![fa.clone(), fb, fc, fd, fe])); + let output_schema = + Arc::new(Schema::new(vec![fa, fb0, fc1, fc2, fd0, fe1, fe2, fe3])); + let input = Arc::new(EmptyExec::new(input_schema)); + let options = UnnestOptions::default(); + let unnest = UnnestExec::new(input, vec![1, 3], vec![2, 4], output_schema, options); + roundtrip_test(Arc::new(unnest)) +} From 4659096f9c0d30bbea59256af8f8e9b73b921cb7 Mon Sep 17 00:00:00 2001 From: Emil Ejbyfeldt Date: Fri, 6 Sep 2024 16:27:52 +0200 Subject: [PATCH 105/154] feat: Add projection to FilterExec (#12281) * Implement Partitioning::project to avoid code duplication * feat: Projection inside FilterExec * Fix proto serialization * Update tpch plans * PR comments, improved doc string and spelling --- .../physical_optimizer/projection_pushdown.rs | 43 ++++-- .../src/equivalence/projection.rs | 25 ++++ datafusion/physical-expr/src/partitioning.rs | 28 +++- .../physical-plan/src/aggregates/mod.rs | 27 ++-- datafusion/physical-plan/src/filter.rs | 128 ++++++++++++++++-- .../physical-plan/src/joins/hash_join.rs | 41 +----- datafusion/physical-plan/src/projection.rs | 24 +--- datafusion/proto/proto/datafusion.proto | 1 + datafusion/proto/src/generated/pbjson.rs | 20 +++ datafusion/proto/src/generated/prost.rs | 2 + datafusion/proto/src/physical_plan/mod.rs | 20 ++- .../sqllogictest/test_files/aggregate.slt | 17 ++- .../sqllogictest/test_files/explain.slt | 9 +- datafusion/sqllogictest/test_files/joins.slt | 24 ++-- .../sqllogictest/test_files/predicates.slt | 15 +- .../sqllogictest/test_files/tpch/q1.slt.part | 2 +- .../sqllogictest/test_files/tpch/q10.slt.part | 26 ++-- .../sqllogictest/test_files/tpch/q11.slt.part | 56 ++++---- .../sqllogictest/test_files/tpch/q12.slt.part | 13 +- .../sqllogictest/test_files/tpch/q13.slt.part | 7 +- .../sqllogictest/test_files/tpch/q14.slt.part | 15 +- .../sqllogictest/test_files/tpch/q15.slt.part | 34 +++-- .../sqllogictest/test_files/tpch/q16.slt.part | 9 +- .../sqllogictest/test_files/tpch/q17.slt.part | 21 ++- .../sqllogictest/test_files/tpch/q18.slt.part | 15 +- .../sqllogictest/test_files/tpch/q19.slt.part | 19 ++- .../sqllogictest/test_files/tpch/q2.slt.part | 113 ++++++++-------- .../sqllogictest/test_files/tpch/q20.slt.part | 61 ++++----- .../sqllogictest/test_files/tpch/q21.slt.part | 48 +++---- .../sqllogictest/test_files/tpch/q22.slt.part | 9 +- .../sqllogictest/test_files/tpch/q3.slt.part | 30 ++-- .../sqllogictest/test_files/tpch/q4.slt.part | 18 ++- .../sqllogictest/test_files/tpch/q5.slt.part | 42 +++--- .../sqllogictest/test_files/tpch/q6.slt.part | 7 +- .../sqllogictest/test_files/tpch/q8.slt.part | 70 +++++----- .../sqllogictest/test_files/tpch/q9.slt.part | 43 +++--- datafusion/sqllogictest/test_files/union.slt | 21 ++- datafusion/sqllogictest/test_files/window.slt | 9 +- 38 files changed, 615 insertions(+), 497 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/projection_pushdown.rs b/datafusion/core/src/physical_optimizer/projection_pushdown.rs index e09332e8c36a..61463b6ea6d5 100644 --- a/datafusion/core/src/physical_optimizer/projection_pushdown.rs +++ b/datafusion/core/src/physical_optimizer/projection_pushdown.rs @@ -123,7 +123,10 @@ pub fn remove_unnecessary_projections( } else if input.is::() { try_swapping_with_coalesce_partitions(projection)? } else if let Some(filter) = input.downcast_ref::() { - try_swapping_with_filter(projection, filter)? + try_swapping_with_filter(projection, filter)?.map_or_else( + || try_embed_projection(projection, filter), + |e| Ok(Some(e)), + )? } else if let Some(repartition) = input.downcast_ref::() { try_swapping_with_repartition(projection, repartition)? } else if let Some(sort) = input.downcast_ref::() { @@ -134,7 +137,7 @@ pub fn remove_unnecessary_projections( try_pushdown_through_union(projection, union)? } else if let Some(hash_join) = input.downcast_ref::() { try_pushdown_through_hash_join(projection, hash_join)?.map_or_else( - || try_embed_to_hash_join(projection, hash_join), + || try_embed_projection(projection, hash_join), |e| Ok(Some(e)), )? } else if let Some(cross_join) = input.downcast_ref::() { @@ -535,11 +538,27 @@ fn try_pushdown_through_union( Ok(Some(Arc::new(UnionExec::new(new_children)))) } +trait EmbeddedProjection: ExecutionPlan + Sized { + fn with_projection(&self, projection: Option>) -> Result; +} + +impl EmbeddedProjection for HashJoinExec { + fn with_projection(&self, projection: Option>) -> Result { + self.with_projection(projection) + } +} + +impl EmbeddedProjection for FilterExec { + fn with_projection(&self, projection: Option>) -> Result { + self.with_projection(projection) + } +} + /// Some projection can't be pushed down left input or right input of hash join because filter or on need may need some columns that won't be used in later. /// By embed those projection to hash join, we can reduce the cost of build_batch_from_indices in hash join (build_batch_from_indices need to can compute::take() for each column) and avoid unnecessary output creation. -fn try_embed_to_hash_join( +fn try_embed_projection( projection: &ProjectionExec, - hash_join: &HashJoinExec, + execution_plan: &Exec, ) -> Result>> { // Collect all column indices from the given projection expressions. let projection_index = collect_column_indices(projection.expr()); @@ -549,20 +568,20 @@ fn try_embed_to_hash_join( }; // If the projection indices is the same as the input columns, we don't need to embed the projection to hash join. - // Check the projection_index is 0..n-1 and the length of projection_index is the same as the length of hash_join schema fields. + // Check the projection_index is 0..n-1 and the length of projection_index is the same as the length of execution_plan schema fields. if projection_index.len() == projection_index.last().unwrap() + 1 - && projection_index.len() == hash_join.schema().fields().len() + && projection_index.len() == execution_plan.schema().fields().len() { return Ok(None); } - let new_hash_join = - Arc::new(hash_join.with_projection(Some(projection_index.to_vec()))?); + let new_execution_plan = + Arc::new(execution_plan.with_projection(Some(projection_index.to_vec()))?); - // Build projection expressions for update_expr. Zip the projection_index with the new_hash_join output schema fields. + // Build projection expressions for update_expr. Zip the projection_index with the new_execution_plan output schema fields. let embed_project_exprs = projection_index .iter() - .zip(new_hash_join.schema().fields()) + .zip(new_execution_plan.schema().fields()) .map(|(index, field)| { ( Arc::new(Column::new(field.name(), *index)) as Arc, @@ -583,10 +602,10 @@ fn try_embed_to_hash_join( // Old projection may contain some alias or expression such as `a + 1` and `CAST('true' AS BOOLEAN)`, but our projection_exprs in hash join just contain column, so we need to create the new projection to keep the original projection. let new_projection = Arc::new(ProjectionExec::try_new( new_projection_exprs, - new_hash_join.clone(), + new_execution_plan.clone(), )?); if is_projection_removable(&new_projection) { - Ok(Some(new_hash_join)) + Ok(Some(new_execution_plan)) } else { Ok(Some(new_projection)) } diff --git a/datafusion/physical-expr/src/equivalence/projection.rs b/datafusion/physical-expr/src/equivalence/projection.rs index f1ce3f04489e..ebf26d3262aa 100644 --- a/datafusion/physical-expr/src/equivalence/projection.rs +++ b/datafusion/physical-expr/src/equivalence/projection.rs @@ -82,6 +82,15 @@ impl ProjectionMapping { .map(|map| Self { map }) } + /// Constructs a subset mapping using the provided indices. + /// + /// This is used when the output is a subset of the input without any + /// other transformations. The indices are for columns in the schema. + pub fn from_indices(indices: &[usize], schema: &SchemaRef) -> Result { + let projection_exprs = project_index_to_exprs(indices, schema); + ProjectionMapping::try_new(&projection_exprs, schema) + } + /// Iterate over pairs of (source, target) expressions pub fn iter( &self, @@ -110,6 +119,22 @@ impl ProjectionMapping { } } +fn project_index_to_exprs( + projection_index: &[usize], + schema: &SchemaRef, +) -> Vec<(Arc, String)> { + projection_index + .iter() + .map(|index| { + let field = schema.field(*index); + ( + Arc::new(Column::new(field.name(), *index)) as Arc, + field.name().to_owned(), + ) + }) + .collect::>() +} + #[cfg(test)] mod tests { use super::*; diff --git a/datafusion/physical-expr/src/partitioning.rs b/datafusion/physical-expr/src/partitioning.rs index 45beeb7b81af..272585f89815 100644 --- a/datafusion/physical-expr/src/partitioning.rs +++ b/datafusion/physical-expr/src/partitioning.rs @@ -20,7 +20,10 @@ use std::fmt; use std::sync::Arc; -use crate::{physical_exprs_equal, EquivalenceProperties, PhysicalExpr}; +use crate::{ + equivalence::ProjectionMapping, expressions::UnKnownColumn, physical_exprs_equal, + EquivalenceProperties, PhysicalExpr, +}; /// Output partitioning supported by [`ExecutionPlan`]s. /// @@ -191,6 +194,29 @@ impl Partitioning { _ => false, } } + + /// Calculate the output partitioning after applying the given projection. + pub fn project( + &self, + projection_mapping: &ProjectionMapping, + input_eq_properties: &EquivalenceProperties, + ) -> Self { + if let Partitioning::Hash(exprs, part) = self { + let normalized_exprs = exprs + .iter() + .map(|expr| { + input_eq_properties + .project_expr(expr, projection_mapping) + .unwrap_or_else(|| { + Arc::new(UnKnownColumn::new(&expr.to_string())) + }) + }) + .collect(); + Partitioning::Hash(normalized_exprs, *part) + } else { + self.clone() + } + } } impl PartialEq for Partitioning { diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 375c6421b0d9..bf6d71308cab 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -28,7 +28,7 @@ use crate::aggregates::{ use crate::metrics::{ExecutionPlanMetricsSet, MetricsSet}; use crate::windows::get_ordered_partition_by_indices; use crate::{ - DisplayFormatType, Distribution, ExecutionPlan, InputOrderMode, Partitioning, + DisplayFormatType, Distribution, ExecutionPlan, InputOrderMode, SendableRecordBatchStream, Statistics, }; @@ -41,7 +41,7 @@ use datafusion_execution::TaskContext; use datafusion_expr::Accumulator; use datafusion_physical_expr::{ equivalence::{collapse_lex_req, ProjectionMapping}, - expressions::{Column, UnKnownColumn}, + expressions::Column, physical_exprs_contains, EquivalenceProperties, LexOrdering, LexRequirement, PhysicalExpr, PhysicalSortRequirement, }; @@ -567,26 +567,16 @@ impl AggregateExec { .project(projection_mapping, schema); // Get output partitioning: - let mut output_partitioning = input.output_partitioning().clone(); - if mode.is_first_stage() { + let input_partitioning = input.output_partitioning().clone(); + let output_partitioning = if mode.is_first_stage() { // First stage aggregation will not change the output partitioning, // but needs to respect aliases (e.g. mapping in the GROUP BY // expression). let input_eq_properties = input.equivalence_properties(); - if let Partitioning::Hash(exprs, part) = output_partitioning { - let normalized_exprs = exprs - .iter() - .map(|expr| { - input_eq_properties - .project_expr(expr, projection_mapping) - .unwrap_or_else(|| { - Arc::new(UnKnownColumn::new(&expr.to_string())) - }) - }) - .collect(); - output_partitioning = Partitioning::Hash(normalized_exprs, part); - } - } + input_partitioning.project(projection_mapping, input_eq_properties) + } else { + input_partitioning.clone() + }; // Determine execution mode: let mut exec_mode = input.execution_mode(); @@ -1233,6 +1223,7 @@ mod tests { use crate::common::collect; use datafusion_physical_expr::aggregate::AggregateExprBuilder; use datafusion_physical_expr::expressions::Literal; + use datafusion_physical_expr::Partitioning; use futures::{FutureExt, Stream}; // Generate a schema which consists of 5 columns (a, b, c, d, e) diff --git a/datafusion/physical-plan/src/filter.rs b/datafusion/physical-plan/src/filter.rs index 6aba3d817710..3da0f21156d9 100644 --- a/datafusion/physical-plan/src/filter.rs +++ b/datafusion/physical-plan/src/filter.rs @@ -24,6 +24,7 @@ use super::{ ColumnStatistics, DisplayAs, ExecutionPlanProperties, PlanProperties, RecordBatchStream, SendableRecordBatchStream, Statistics, }; +use crate::common::can_project; use crate::{ metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet}, DisplayFormatType, ExecutionPlan, @@ -34,9 +35,12 @@ use arrow::datatypes::{DataType, SchemaRef}; use arrow::record_batch::RecordBatch; use datafusion_common::cast::as_boolean_array; use datafusion_common::stats::Precision; -use datafusion_common::{internal_err, plan_err, DataFusionError, Result}; +use datafusion_common::{ + internal_err, plan_err, project_schema, DataFusionError, Result, +}; use datafusion_execution::TaskContext; use datafusion_expr::Operator; +use datafusion_physical_expr::equivalence::ProjectionMapping; use datafusion_physical_expr::expressions::BinaryExpr; use datafusion_physical_expr::intervals::utils::check_support; use datafusion_physical_expr::utils::collect_columns; @@ -61,6 +65,8 @@ pub struct FilterExec { default_selectivity: u8, /// Properties equivalence properties, partitioning, etc. cache: PlanProperties, + /// The projection indices of the columns in the output schema of join + projection: Option>, } impl FilterExec { @@ -72,14 +78,19 @@ impl FilterExec { match predicate.data_type(input.schema().as_ref())? { DataType::Boolean => { let default_selectivity = 20; - let cache = - Self::compute_properties(&input, &predicate, default_selectivity)?; + let cache = Self::compute_properties( + &input, + &predicate, + default_selectivity, + None, + )?; Ok(Self { predicate, input: Arc::clone(&input), metrics: ExecutionPlanMetricsSet::new(), default_selectivity, cache, + projection: None, }) } other => { @@ -101,6 +112,35 @@ impl FilterExec { Ok(self) } + /// Return new instance of [FilterExec] with the given projection. + pub fn with_projection(&self, projection: Option>) -> Result { + // check if the projection is valid + can_project(&self.schema(), projection.as_ref())?; + + let projection = match projection { + Some(projection) => match &self.projection { + Some(p) => Some(projection.iter().map(|i| p[*i]).collect()), + None => Some(projection), + }, + None => None, + }; + + let cache = Self::compute_properties( + &self.input, + &self.predicate, + self.default_selectivity, + projection.as_ref(), + )?; + Ok(Self { + predicate: Arc::clone(&self.predicate), + input: Arc::clone(&self.input), + metrics: self.metrics.clone(), + default_selectivity: self.default_selectivity, + cache, + projection, + }) + } + /// The expression to filter on. This expression must evaluate to a boolean value. pub fn predicate(&self) -> &Arc { &self.predicate @@ -116,6 +156,11 @@ impl FilterExec { self.default_selectivity } + /// projection + pub fn projection(&self) -> Option<&Vec> { + self.projection.as_ref() + } + /// Calculates `Statistics` for `FilterExec`, by applying selectivity (either default, or estimated) to input statistics. fn statistics_helper( input: &Arc, @@ -190,6 +235,7 @@ impl FilterExec { input: &Arc, predicate: &Arc, default_selectivity: u8, + projection: Option<&Vec>, ) -> Result { // Combine the equal predicates with the input equivalence properties // to construct the equivalence properties: @@ -214,10 +260,21 @@ impl FilterExec { // to do: how to deal with multiple situation to represent = (for example c1 between 0 and 0) eq_properties = eq_properties.add_constants(Self::extend_constants(input, predicate)); + + let mut output_partitioning = input.output_partitioning().clone(); + // If contains projection, update the PlanProperties. + if let Some(projection) = projection { + let schema = eq_properties.schema(); + let projection_mapping = ProjectionMapping::from_indices(projection, schema)?; + let out_schema = project_schema(schema, Some(projection))?; + output_partitioning = + output_partitioning.project(&projection_mapping, &eq_properties); + eq_properties = eq_properties.project(&projection_mapping, out_schema); + } Ok(PlanProperties::new( eq_properties, - input.output_partitioning().clone(), // Output Partitioning - input.execution_mode(), // Execution Mode + output_partitioning, + input.execution_mode(), )) } } @@ -230,7 +287,25 @@ impl DisplayAs for FilterExec { ) -> std::fmt::Result { match t { DisplayFormatType::Default | DisplayFormatType::Verbose => { - write!(f, "FilterExec: {}", self.predicate) + let display_projections = if let Some(projection) = + self.projection.as_ref() + { + format!( + ", projection=[{}]", + projection + .iter() + .map(|index| format!( + "{}@{}", + self.input.schema().fields().get(*index).unwrap().name(), + index + )) + .collect::>() + .join(", ") + ) + } else { + "".to_string() + }; + write!(f, "FilterExec: {}{}", self.predicate, display_projections) } } } @@ -268,6 +343,7 @@ impl ExecutionPlan for FilterExec { let selectivity = e.default_selectivity(); e.with_default_selectivity(selectivity) }) + .and_then(|e| e.with_projection(self.projection().cloned())) .map(|e| Arc::new(e) as _) } @@ -279,10 +355,11 @@ impl ExecutionPlan for FilterExec { trace!("Start FilterExec::execute for partition {} of context session_id {} and task_id {:?}", partition, context.session_id(), context.task_id()); let baseline_metrics = BaselineMetrics::new(&self.metrics, partition); Ok(Box::pin(FilterExecStream { - schema: self.input.schema(), + schema: self.schema(), predicate: Arc::clone(&self.predicate), input: self.input.execute(partition, context)?, baseline_metrics, + projection: self.projection.clone(), })) } @@ -337,7 +414,7 @@ fn collect_new_statistics( /// The FilterExec streams wraps the input iterator and applies the predicate expression to /// determine which rows to include in its output batches struct FilterExecStream { - /// Output schema, which is the same as the input schema for this operator + /// Output schema after the projection schema: SchemaRef, /// The expression to filter on. This expression must evaluate to a boolean value. predicate: Arc, @@ -345,20 +422,42 @@ struct FilterExecStream { input: SendableRecordBatchStream, /// runtime metrics recording baseline_metrics: BaselineMetrics, + /// The projection indices of the columns in the input schema + projection: Option>, } pub fn batch_filter( batch: &RecordBatch, predicate: &Arc, +) -> Result { + filter_and_project(batch, predicate, None, &batch.schema()) +} + +fn filter_and_project( + batch: &RecordBatch, + predicate: &Arc, + projection: Option<&Vec>, + output_schema: &SchemaRef, ) -> Result { predicate .evaluate(batch) .and_then(|v| v.into_array(batch.num_rows())) .and_then(|array| { - Ok(match as_boolean_array(&array) { + Ok(match (as_boolean_array(&array), projection) { // apply filter array to record batch - Ok(filter_array) => filter_record_batch(batch, filter_array)?, - Err(_) => { + (Ok(filter_array), None) => filter_record_batch(batch, filter_array)?, + (Ok(filter_array), Some(projection)) => { + let projected_columns = projection + .iter() + .map(|i| Arc::clone(batch.column(*i))) + .collect(); + let projected_batch = RecordBatch::try_new( + Arc::clone(output_schema), + projected_columns, + )?; + filter_record_batch(&projected_batch, filter_array)? + } + (Err(_), _) => { return internal_err!( "Cannot create filter_array from non-boolean predicates" ); @@ -379,7 +478,12 @@ impl Stream for FilterExecStream { match ready!(self.input.poll_next_unpin(cx)) { Some(Ok(batch)) => { let timer = self.baseline_metrics.elapsed_compute().timer(); - let filtered_batch = batch_filter(&batch, &self.predicate)?; + let filtered_batch = filter_and_project( + &batch, + &self.predicate, + self.projection.as_ref(), + &self.schema, + )?; timer.done(); // skip entirely filtered batches if filtered_batch.num_rows() == 0 { diff --git a/datafusion/physical-plan/src/joins/hash_join.rs b/datafusion/physical-plan/src/joins/hash_join.rs index 38827108e815..48d648c89a35 100644 --- a/datafusion/physical-plan/src/joins/hash_join.rs +++ b/datafusion/physical-plan/src/joins/hash_join.rs @@ -68,8 +68,7 @@ use datafusion_execution::TaskContext; use datafusion_physical_expr::equivalence::{ join_equivalence_properties, ProjectionMapping, }; -use datafusion_physical_expr::expressions::UnKnownColumn; -use datafusion_physical_expr::{PhysicalExpr, PhysicalExprRef}; +use datafusion_physical_expr::PhysicalExprRef; use ahash::RandomState; use datafusion_expr::Operator; @@ -528,24 +527,12 @@ impl HashJoinExec { // If contains projection, update the PlanProperties. if let Some(projection) = projection { - let projection_exprs = project_index_to_exprs(projection, &schema); // construct a map from the input expressions to the output expression of the Projection let projection_mapping = - ProjectionMapping::try_new(&projection_exprs, &schema)?; + ProjectionMapping::from_indices(projection, &schema)?; let out_schema = project_schema(&schema, Some(projection))?; - if let Partitioning::Hash(exprs, part) = output_partitioning { - let normalized_exprs = exprs - .iter() - .map(|expr| { - eq_properties - .project_expr(expr, &projection_mapping) - .unwrap_or_else(|| { - Arc::new(UnKnownColumn::new(&expr.to_string())) - }) - }) - .collect(); - output_partitioning = Partitioning::Hash(normalized_exprs, part); - } + output_partitioning = + output_partitioning.project(&projection_mapping, &eq_properties); eq_properties = eq_properties.project(&projection_mapping, out_schema); } Ok(PlanProperties::new( @@ -598,25 +585,6 @@ impl DisplayAs for HashJoinExec { } } -fn project_index_to_exprs( - projection_index: &[usize], - schema: &SchemaRef, -) -> Vec<(Arc, String)> { - projection_index - .iter() - .map(|index| { - let field = schema.field(*index); - ( - Arc::new(datafusion_physical_expr::expressions::Column::new( - field.name(), - *index, - )) as Arc, - field.name().to_owned(), - ) - }) - .collect::>() -} - impl ExecutionPlan for HashJoinExec { fn name(&self) -> &'static str { "HashJoinExec" @@ -1575,6 +1543,7 @@ mod tests { use datafusion_execution::runtime_env::RuntimeEnvBuilder; use datafusion_expr::Operator; use datafusion_physical_expr::expressions::{BinaryExpr, Literal}; + use datafusion_physical_expr::PhysicalExpr; use hashbrown::raw::RawTable; use rstest::*; diff --git a/datafusion/physical-plan/src/projection.rs b/datafusion/physical-plan/src/projection.rs index d2bb8f2b0ead..f1b9cdaf728f 100644 --- a/datafusion/physical-plan/src/projection.rs +++ b/datafusion/physical-plan/src/projection.rs @@ -32,9 +32,7 @@ use super::{ DisplayAs, ExecutionPlanProperties, PlanProperties, RecordBatchStream, SendableRecordBatchStream, Statistics, }; -use crate::{ - ColumnStatistics, DisplayFormatType, ExecutionPlan, Partitioning, PhysicalExpr, -}; +use crate::{ColumnStatistics, DisplayFormatType, ExecutionPlan, PhysicalExpr}; use arrow::datatypes::{Field, Schema, SchemaRef}; use arrow::record_batch::{RecordBatch, RecordBatchOptions}; @@ -42,7 +40,7 @@ use datafusion_common::stats::Precision; use datafusion_common::Result; use datafusion_execution::TaskContext; use datafusion_physical_expr::equivalence::ProjectionMapping; -use datafusion_physical_expr::expressions::{Literal, UnKnownColumn}; +use datafusion_physical_expr::expressions::Literal; use futures::stream::{Stream, StreamExt}; use log::trace; @@ -127,22 +125,8 @@ impl ProjectionExec { // Calculate output partitioning, which needs to respect aliases: let input_partition = input.output_partitioning(); - let output_partitioning = if let Partitioning::Hash(exprs, part) = input_partition - { - let normalized_exprs = exprs - .iter() - .map(|expr| { - input_eq_properties - .project_expr(expr, projection_mapping) - .unwrap_or_else(|| { - Arc::new(UnKnownColumn::new(&expr.to_string())) - }) - }) - .collect(); - Partitioning::Hash(normalized_exprs, *part) - } else { - input_partition.clone() - }; + let output_partitioning = + input_partition.project(projection_mapping, &input_eq_properties); Ok(PlanProperties::new( eq_properties, diff --git a/datafusion/proto/proto/datafusion.proto b/datafusion/proto/proto/datafusion.proto index 2d911537bc85..645df14a0337 100644 --- a/datafusion/proto/proto/datafusion.proto +++ b/datafusion/proto/proto/datafusion.proto @@ -922,6 +922,7 @@ message FilterExecNode { PhysicalPlanNode input = 1; PhysicalExprNode expr = 2; uint32 default_filter_selectivity = 3; + repeated uint32 projection = 9; } message FileGroup { diff --git a/datafusion/proto/src/generated/pbjson.rs b/datafusion/proto/src/generated/pbjson.rs index ef2f8a92fc85..e493f761b51f 100644 --- a/datafusion/proto/src/generated/pbjson.rs +++ b/datafusion/proto/src/generated/pbjson.rs @@ -5573,6 +5573,9 @@ impl serde::Serialize for FilterExecNode { if self.default_filter_selectivity != 0 { len += 1; } + if !self.projection.is_empty() { + len += 1; + } let mut struct_ser = serializer.serialize_struct("datafusion.FilterExecNode", len)?; if let Some(v) = self.input.as_ref() { struct_ser.serialize_field("input", v)?; @@ -5583,6 +5586,9 @@ impl serde::Serialize for FilterExecNode { if self.default_filter_selectivity != 0 { struct_ser.serialize_field("defaultFilterSelectivity", &self.default_filter_selectivity)?; } + if !self.projection.is_empty() { + struct_ser.serialize_field("projection", &self.projection)?; + } struct_ser.end() } } @@ -5597,6 +5603,7 @@ impl<'de> serde::Deserialize<'de> for FilterExecNode { "expr", "default_filter_selectivity", "defaultFilterSelectivity", + "projection", ]; #[allow(clippy::enum_variant_names)] @@ -5604,6 +5611,7 @@ impl<'de> serde::Deserialize<'de> for FilterExecNode { Input, Expr, DefaultFilterSelectivity, + Projection, } impl<'de> serde::Deserialize<'de> for GeneratedField { fn deserialize(deserializer: D) -> std::result::Result @@ -5628,6 +5636,7 @@ impl<'de> serde::Deserialize<'de> for FilterExecNode { "input" => Ok(GeneratedField::Input), "expr" => Ok(GeneratedField::Expr), "defaultFilterSelectivity" | "default_filter_selectivity" => Ok(GeneratedField::DefaultFilterSelectivity), + "projection" => Ok(GeneratedField::Projection), _ => Err(serde::de::Error::unknown_field(value, FIELDS)), } } @@ -5650,6 +5659,7 @@ impl<'de> serde::Deserialize<'de> for FilterExecNode { let mut input__ = None; let mut expr__ = None; let mut default_filter_selectivity__ = None; + let mut projection__ = None; while let Some(k) = map_.next_key()? { match k { GeneratedField::Input => { @@ -5672,12 +5682,22 @@ impl<'de> serde::Deserialize<'de> for FilterExecNode { Some(map_.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) ; } + GeneratedField::Projection => { + if projection__.is_some() { + return Err(serde::de::Error::duplicate_field("projection")); + } + projection__ = + Some(map_.next_value::>>()? + .into_iter().map(|x| x.0).collect()) + ; + } } } Ok(FilterExecNode { input: input__, expr: expr__, default_filter_selectivity: default_filter_selectivity__.unwrap_or_default(), + projection: projection__.unwrap_or_default(), }) } } diff --git a/datafusion/proto/src/generated/prost.rs b/datafusion/proto/src/generated/prost.rs index fc8a5736531a..1d086a610ce4 100644 --- a/datafusion/proto/src/generated/prost.rs +++ b/datafusion/proto/src/generated/prost.rs @@ -1482,6 +1482,8 @@ pub struct FilterExecNode { pub expr: ::core::option::Option, #[prost(uint32, tag = "3")] pub default_filter_selectivity: u32, + #[prost(uint32, repeated, tag = "9")] + pub projection: ::prost::alloc::vec::Vec, } #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] diff --git a/datafusion/proto/src/physical_plan/mod.rs b/datafusion/proto/src/physical_plan/mod.rs index a0934e54005e..74b6073a415e 100644 --- a/datafusion/proto/src/physical_plan/mod.rs +++ b/datafusion/proto/src/physical_plan/mod.rs @@ -179,7 +179,19 @@ impl AsExecutionPlan for protobuf::PhysicalPlanNode { ) })?; let filter_selectivity = filter.default_filter_selectivity.try_into(); - let filter = FilterExec::try_new(predicate, input)?; + let projection = if !filter.projection.is_empty() { + Some( + filter + .projection + .iter() + .map(|i| *i as usize) + .collect::>(), + ) + } else { + None + }; + let filter = + FilterExec::try_new(predicate, input)?.with_projection(projection)?; match filter_selectivity { Ok(filter_selectivity) => Ok(Arc::new( filter.with_default_selectivity(filter_selectivity)?, @@ -1184,6 +1196,12 @@ impl AsExecutionPlan for protobuf::PhysicalPlanNode { extension_codec, )?), default_filter_selectivity: exec.default_selectivity() as u32, + projection: exec + .projection() + .as_ref() + .map_or_else(Vec::new, |v| { + v.iter().map(|x| *x as u32).collect::>() + }), }, ))), }); diff --git a/datafusion/sqllogictest/test_files/aggregate.slt b/datafusion/sqllogictest/test_files/aggregate.slt index 83f4e4f03055..576abe5c6f5a 100644 --- a/datafusion/sqllogictest/test_files/aggregate.slt +++ b/datafusion/sqllogictest/test_files/aggregate.slt @@ -5772,15 +5772,14 @@ logical_plan 03)----Aggregate: groupBy=[[having_test.v1, having_test.v2]], aggr=[[max(having_test.v1)]] 04)------TableScan: having_test projection=[v1, v2] physical_plan -01)ProjectionExec: expr=[v1@0 as v1, v2@1 as v2] -02)--CoalesceBatchesExec: target_batch_size=8192 -03)----FilterExec: max(having_test.v1)@2 = 3 -04)------AggregateExec: mode=FinalPartitioned, gby=[v1@0 as v1, v2@1 as v2], aggr=[max(having_test.v1)] -05)--------CoalesceBatchesExec: target_batch_size=8192 -06)----------RepartitionExec: partitioning=Hash([v1@0, v2@1], 4), input_partitions=4 -07)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -08)--------------AggregateExec: mode=Partial, gby=[v1@0 as v1, v2@1 as v2], aggr=[max(having_test.v1)] -09)----------------MemoryExec: partitions=1, partition_sizes=[1] +01)CoalesceBatchesExec: target_batch_size=8192 +02)--FilterExec: max(having_test.v1)@2 = 3, projection=[v1@0, v2@1] +03)----AggregateExec: mode=FinalPartitioned, gby=[v1@0 as v1, v2@1 as v2], aggr=[max(having_test.v1)] +04)------CoalesceBatchesExec: target_batch_size=8192 +05)--------RepartitionExec: partitioning=Hash([v1@0, v2@1], 4), input_partitions=4 +06)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +07)------------AggregateExec: mode=Partial, gby=[v1@0 as v1, v2@1 as v2], aggr=[max(having_test.v1)] +08)--------------MemoryExec: partitions=1, partition_sizes=[1] query error diff --git a/datafusion/sqllogictest/test_files/explain.slt b/datafusion/sqllogictest/test_files/explain.slt index 94b915cebe14..6dc92bae828b 100644 --- a/datafusion/sqllogictest/test_files/explain.slt +++ b/datafusion/sqllogictest/test_files/explain.slt @@ -43,11 +43,10 @@ logical_plan 02)--Filter: aggregate_test_100.c2 > Int8(10) 03)----TableScan: aggregate_test_100 projection=[c1, c2], partial_filters=[aggregate_test_100.c2 > Int8(10)] physical_plan -01)ProjectionExec: expr=[c1@0 as c1] -02)--CoalesceBatchesExec: target_batch_size=8192 -03)----FilterExec: c2@1 > 10 -04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -05)--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2], has_header=true +01)CoalesceBatchesExec: target_batch_size=8192 +02)--FilterExec: c2@1 > 10, projection=[c1@0] +03)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +04)------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2], has_header=true # explain_csv_exec_scan_config diff --git a/datafusion/sqllogictest/test_files/joins.slt b/datafusion/sqllogictest/test_files/joins.slt index 0ef745a6b8e6..7d0262952b31 100644 --- a/datafusion/sqllogictest/test_files/joins.slt +++ b/datafusion/sqllogictest/test_files/joins.slt @@ -2082,15 +2082,14 @@ physical_plan 01)ProjectionExec: expr=[t1_id@1 as t1_id, t2_id@0 as t2_id] 02)--NestedLoopJoinExec: join_type=Inner, filter=t1_id@0 > t2_id@1 03)----CoalescePartitionsExec -04)------ProjectionExec: expr=[t2_id@0 as t2_id] -05)--------CoalesceBatchesExec: target_batch_size=2 -06)----------FilterExec: t2_int@1 > 1 -07)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -08)--------------MemoryExec: partitions=1, partition_sizes=[1] -09)----CoalesceBatchesExec: target_batch_size=2 -10)------FilterExec: t1_id@0 > 10 -11)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -12)----------MemoryExec: partitions=1, partition_sizes=[1] +04)------CoalesceBatchesExec: target_batch_size=2 +05)--------FilterExec: t2_int@1 > 1, projection=[t2_id@0] +06)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +07)------------MemoryExec: partitions=1, partition_sizes=[1] +08)----CoalesceBatchesExec: target_batch_size=2 +09)------FilterExec: t1_id@0 > 10 +10)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +11)----------MemoryExec: partitions=1, partition_sizes=[1] query II SELECT join_t1.t1_id, join_t2.t2_id @@ -4143,10 +4142,9 @@ physical_plan 04)------CoalesceBatchesExec: target_batch_size=3 05)--------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(currency@3, currency_from@1)], filter=ts@0 >= ts@1, projection=[ts@0, sn@1, amount@2, currency@3, rate@6] 06)----------MemoryExec: partitions=1, partition_sizes=[0] -07)----------ProjectionExec: expr=[ts@0 as ts, currency_from@1 as currency_from, rate@3 as rate] -08)------------CoalesceBatchesExec: target_batch_size=3 -09)--------------FilterExec: currency_to@2 = USD -10)----------------MemoryExec: partitions=1, partition_sizes=[0] +07)----------CoalesceBatchesExec: target_batch_size=3 +08)------------FilterExec: currency_to@2 = USD, projection=[ts@0, currency_from@1, rate@3] +09)--------------MemoryExec: partitions=1, partition_sizes=[0] statement ok DROP TABLE sales_global; diff --git a/datafusion/sqllogictest/test_files/predicates.slt b/datafusion/sqllogictest/test_files/predicates.slt index 4695e37aa560..878d7c8a4dfb 100644 --- a/datafusion/sqllogictest/test_files/predicates.slt +++ b/datafusion/sqllogictest/test_files/predicates.slt @@ -770,14 +770,13 @@ physical_plan 09)----------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/tpch-csv/lineitem.csv]]}, projection=[l_partkey, l_extendedprice, l_discount], has_header=true 10)----------CoalesceBatchesExec: target_batch_size=8192 11)------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 -12)--------------ProjectionExec: expr=[p_partkey@0 as p_partkey] -13)----------------CoalesceBatchesExec: target_batch_size=8192 -14)------------------FilterExec: p_brand@1 = Brand#12 OR p_brand@1 = Brand#23 -15)--------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -16)----------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/tpch-csv/part.csv]]}, projection=[p_partkey, p_brand], has_header=true -17)------CoalesceBatchesExec: target_batch_size=8192 -18)--------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=1 -19)----------MemoryExec: partitions=1, partition_sizes=[1] +12)--------------CoalesceBatchesExec: target_batch_size=8192 +13)----------------FilterExec: p_brand@1 = Brand#12 OR p_brand@1 = Brand#23, projection=[p_partkey@0] +14)------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +15)--------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/tpch-csv/part.csv]]}, projection=[p_partkey, p_brand], has_header=true +16)------CoalesceBatchesExec: target_batch_size=8192 +17)--------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=1 +18)----------MemoryExec: partitions=1, partition_sizes=[1] # Inlist simplification diff --git a/datafusion/sqllogictest/test_files/tpch/q1.slt.part b/datafusion/sqllogictest/test_files/tpch/q1.slt.part index 6c3e7dd3618a..8cfd25d26c07 100644 --- a/datafusion/sqllogictest/test_files/tpch/q1.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q1.slt.part @@ -56,7 +56,7 @@ physical_plan 07)------------AggregateExec: mode=Partial, gby=[l_returnflag@5 as l_returnflag, l_linestatus@6 as l_linestatus], aggr=[sum(lineitem.l_quantity), sum(lineitem.l_extendedprice), sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount), sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount * Int64(1) + lineitem.l_tax), avg(lineitem.l_quantity), avg(lineitem.l_extendedprice), avg(lineitem.l_discount), count(*)] 08)--------------ProjectionExec: expr=[l_extendedprice@1 * (Some(1),20,0 - l_discount@2) as __common_expr_1, l_quantity@0 as l_quantity, l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount, l_tax@3 as l_tax, l_returnflag@4 as l_returnflag, l_linestatus@5 as l_linestatus] 09)----------------CoalesceBatchesExec: target_batch_size=8192 -10)------------------FilterExec: l_shipdate@6 <= 1998-09-02 +10)------------------FilterExec: l_shipdate@6 <= 1998-09-02, projection=[l_quantity@0, l_extendedprice@1, l_discount@2, l_tax@3, l_returnflag@4, l_linestatus@5] 11)--------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate], has_header=false query TTRRRRRRRI diff --git a/datafusion/sqllogictest/test_files/tpch/q10.slt.part b/datafusion/sqllogictest/test_files/tpch/q10.slt.part index 23ae70d7ec97..73593a470c9a 100644 --- a/datafusion/sqllogictest/test_files/tpch/q10.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q10.slt.part @@ -92,20 +92,18 @@ physical_plan 21)----------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_comment], has_header=false 22)----------------------------------CoalesceBatchesExec: target_batch_size=8192 23)------------------------------------RepartitionExec: partitioning=Hash([o_custkey@1], 4), input_partitions=4 -24)--------------------------------------ProjectionExec: expr=[o_orderkey@0 as o_orderkey, o_custkey@1 as o_custkey] -25)----------------------------------------CoalesceBatchesExec: target_batch_size=8192 -26)------------------------------------------FilterExec: o_orderdate@2 >= 1993-10-01 AND o_orderdate@2 < 1994-01-01 -27)--------------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_custkey, o_orderdate], has_header=false -28)--------------------------CoalesceBatchesExec: target_batch_size=8192 -29)----------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 -30)------------------------------ProjectionExec: expr=[l_orderkey@0 as l_orderkey, l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount] -31)--------------------------------CoalesceBatchesExec: target_batch_size=8192 -32)----------------------------------FilterExec: l_returnflag@3 = R -33)------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_extendedprice, l_discount, l_returnflag], has_header=false -34)------------------CoalesceBatchesExec: target_batch_size=8192 -35)--------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 -36)----------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -37)------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false +24)--------------------------------------CoalesceBatchesExec: target_batch_size=8192 +25)----------------------------------------FilterExec: o_orderdate@2 >= 1993-10-01 AND o_orderdate@2 < 1994-01-01, projection=[o_orderkey@0, o_custkey@1] +26)------------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_custkey, o_orderdate], has_header=false +27)--------------------------CoalesceBatchesExec: target_batch_size=8192 +28)----------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 +29)------------------------------CoalesceBatchesExec: target_batch_size=8192 +30)--------------------------------FilterExec: l_returnflag@3 = R, projection=[l_orderkey@0, l_extendedprice@1, l_discount@2] +31)----------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_extendedprice, l_discount, l_returnflag], has_header=false +32)------------------CoalesceBatchesExec: target_batch_size=8192 +33)--------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 +34)----------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +35)------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/q11.slt.part b/datafusion/sqllogictest/test_files/tpch/q11.slt.part index 0d66b2f2f2a9..adaf391de0a2 100644 --- a/datafusion/sqllogictest/test_files/tpch/q11.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q11.slt.part @@ -96,35 +96,33 @@ physical_plan 21)----------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], has_header=false 22)--------------------CoalesceBatchesExec: target_batch_size=8192 23)----------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 -24)------------------------ProjectionExec: expr=[n_nationkey@0 as n_nationkey] -25)--------------------------CoalesceBatchesExec: target_batch_size=8192 -26)----------------------------FilterExec: n_name@1 = GERMANY -27)------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -28)--------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false -29)------ProjectionExec: expr=[CAST(CAST(sum(partsupp.ps_supplycost * partsupp.ps_availqty)@0 AS Float64) * 0.0001 AS Decimal128(38, 15)) as sum(partsupp.ps_supplycost * partsupp.ps_availqty) * Float64(0.0001)] -30)--------AggregateExec: mode=Final, gby=[], aggr=[sum(partsupp.ps_supplycost * partsupp.ps_availqty)] -31)----------CoalescePartitionsExec -32)------------AggregateExec: mode=Partial, gby=[], aggr=[sum(partsupp.ps_supplycost * partsupp.ps_availqty)] -33)--------------CoalesceBatchesExec: target_batch_size=8192 -34)----------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@2, n_nationkey@0)], projection=[ps_availqty@0, ps_supplycost@1] -35)------------------CoalesceBatchesExec: target_batch_size=8192 -36)--------------------RepartitionExec: partitioning=Hash([s_nationkey@2], 4), input_partitions=4 -37)----------------------CoalesceBatchesExec: target_batch_size=8192 -38)------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_suppkey@0, s_suppkey@0)], projection=[ps_availqty@1, ps_supplycost@2, s_nationkey@4] -39)--------------------------CoalesceBatchesExec: target_batch_size=8192 -40)----------------------------RepartitionExec: partitioning=Hash([ps_suppkey@0], 4), input_partitions=4 -41)------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:0..2932049], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:2932049..5864098], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:5864098..8796147], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:8796147..11728193]]}, projection=[ps_suppkey, ps_availqty, ps_supplycost], has_header=false -42)--------------------------CoalesceBatchesExec: target_batch_size=8192 -43)----------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 -44)------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -45)--------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], has_header=false -46)------------------CoalesceBatchesExec: target_batch_size=8192 -47)--------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 -48)----------------------ProjectionExec: expr=[n_nationkey@0 as n_nationkey] -49)------------------------CoalesceBatchesExec: target_batch_size=8192 -50)--------------------------FilterExec: n_name@1 = GERMANY -51)----------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -52)------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false +24)------------------------CoalesceBatchesExec: target_batch_size=8192 +25)--------------------------FilterExec: n_name@1 = GERMANY, projection=[n_nationkey@0] +26)----------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +27)------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false +28)------ProjectionExec: expr=[CAST(CAST(sum(partsupp.ps_supplycost * partsupp.ps_availqty)@0 AS Float64) * 0.0001 AS Decimal128(38, 15)) as sum(partsupp.ps_supplycost * partsupp.ps_availqty) * Float64(0.0001)] +29)--------AggregateExec: mode=Final, gby=[], aggr=[sum(partsupp.ps_supplycost * partsupp.ps_availqty)] +30)----------CoalescePartitionsExec +31)------------AggregateExec: mode=Partial, gby=[], aggr=[sum(partsupp.ps_supplycost * partsupp.ps_availqty)] +32)--------------CoalesceBatchesExec: target_batch_size=8192 +33)----------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@2, n_nationkey@0)], projection=[ps_availqty@0, ps_supplycost@1] +34)------------------CoalesceBatchesExec: target_batch_size=8192 +35)--------------------RepartitionExec: partitioning=Hash([s_nationkey@2], 4), input_partitions=4 +36)----------------------CoalesceBatchesExec: target_batch_size=8192 +37)------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_suppkey@0, s_suppkey@0)], projection=[ps_availqty@1, ps_supplycost@2, s_nationkey@4] +38)--------------------------CoalesceBatchesExec: target_batch_size=8192 +39)----------------------------RepartitionExec: partitioning=Hash([ps_suppkey@0], 4), input_partitions=4 +40)------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:0..2932049], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:2932049..5864098], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:5864098..8796147], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:8796147..11728193]]}, projection=[ps_suppkey, ps_availqty, ps_supplycost], has_header=false +41)--------------------------CoalesceBatchesExec: target_batch_size=8192 +42)----------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 +43)------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +44)--------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], has_header=false +45)------------------CoalesceBatchesExec: target_batch_size=8192 +46)--------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 +47)----------------------CoalesceBatchesExec: target_batch_size=8192 +48)------------------------FilterExec: n_name@1 = GERMANY, projection=[n_nationkey@0] +49)--------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +50)----------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/q12.slt.part b/datafusion/sqllogictest/test_files/tpch/q12.slt.part index 3f14f8905f72..b0d0baba90b0 100644 --- a/datafusion/sqllogictest/test_files/tpch/q12.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q12.slt.part @@ -70,13 +70,12 @@ physical_plan 09)----------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_orderkey@0, o_orderkey@0)], projection=[l_shipmode@1, o_orderpriority@3] 10)------------------CoalesceBatchesExec: target_batch_size=8192 11)--------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 -12)----------------------ProjectionExec: expr=[l_orderkey@0 as l_orderkey, l_shipmode@4 as l_shipmode] -13)------------------------CoalesceBatchesExec: target_batch_size=8192 -14)--------------------------FilterExec: (l_shipmode@4 = MAIL OR l_shipmode@4 = SHIP) AND l_receiptdate@3 > l_commitdate@2 AND l_shipdate@1 < l_commitdate@2 AND l_receiptdate@3 >= 1994-01-01 AND l_receiptdate@3 < 1995-01-01 -15)----------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_shipdate, l_commitdate, l_receiptdate, l_shipmode], has_header=false -16)------------------CoalesceBatchesExec: target_batch_size=8192 -17)--------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 -18)----------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_orderpriority], has_header=false +12)----------------------CoalesceBatchesExec: target_batch_size=8192 +13)------------------------FilterExec: (l_shipmode@4 = MAIL OR l_shipmode@4 = SHIP) AND l_receiptdate@3 > l_commitdate@2 AND l_shipdate@1 < l_commitdate@2 AND l_receiptdate@3 >= 1994-01-01 AND l_receiptdate@3 < 1995-01-01, projection=[l_orderkey@0, l_shipmode@4] +14)--------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_shipdate, l_commitdate, l_receiptdate, l_shipmode], has_header=false +15)------------------CoalesceBatchesExec: target_batch_size=8192 +16)--------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 +17)----------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_orderpriority], has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/q13.slt.part b/datafusion/sqllogictest/test_files/tpch/q13.slt.part index 011bd761d760..bb32fb209700 100644 --- a/datafusion/sqllogictest/test_files/tpch/q13.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q13.slt.part @@ -70,10 +70,9 @@ physical_plan 15)----------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey], has_header=false 16)----------------------CoalesceBatchesExec: target_batch_size=8192 17)------------------------RepartitionExec: partitioning=Hash([o_custkey@1], 4), input_partitions=4 -18)--------------------------ProjectionExec: expr=[o_orderkey@0 as o_orderkey, o_custkey@1 as o_custkey] -19)----------------------------CoalesceBatchesExec: target_batch_size=8192 -20)------------------------------FilterExec: o_comment@2 NOT LIKE %special%requests% -21)--------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_custkey, o_comment], has_header=false +18)--------------------------CoalesceBatchesExec: target_batch_size=8192 +19)----------------------------FilterExec: o_comment@2 NOT LIKE %special%requests%, projection=[o_orderkey@0, o_custkey@1] +20)------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_custkey, o_comment], has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/q14.slt.part b/datafusion/sqllogictest/test_files/tpch/q14.slt.part index 3743c201ff2e..eee10cb3f8e2 100644 --- a/datafusion/sqllogictest/test_files/tpch/q14.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q14.slt.part @@ -50,14 +50,13 @@ physical_plan 07)------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_partkey@0, p_partkey@0)], projection=[l_extendedprice@1, l_discount@2, p_type@4] 08)--------------CoalesceBatchesExec: target_batch_size=8192 09)----------------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=4 -10)------------------ProjectionExec: expr=[l_partkey@0 as l_partkey, l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount] -11)--------------------CoalesceBatchesExec: target_batch_size=8192 -12)----------------------FilterExec: l_shipdate@3 >= 1995-09-01 AND l_shipdate@3 < 1995-10-01 -13)------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_partkey, l_extendedprice, l_discount, l_shipdate], has_header=false -14)--------------CoalesceBatchesExec: target_batch_size=8192 -15)----------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 -16)------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -17)--------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_type], has_header=false +10)------------------CoalesceBatchesExec: target_batch_size=8192 +11)--------------------FilterExec: l_shipdate@3 >= 1995-09-01 AND l_shipdate@3 < 1995-10-01, projection=[l_partkey@0, l_extendedprice@1, l_discount@2] +12)----------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_partkey, l_extendedprice, l_discount, l_shipdate], has_header=false +13)--------------CoalesceBatchesExec: target_batch_size=8192 +14)----------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 +15)------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +16)--------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_type], has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/q15.slt.part b/datafusion/sqllogictest/test_files/tpch/q15.slt.part index 087ad3ffbba0..2374fd8430a4 100644 --- a/datafusion/sqllogictest/test_files/tpch/q15.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q15.slt.part @@ -88,24 +88,22 @@ physical_plan 15)--------------------CoalesceBatchesExec: target_batch_size=8192 16)----------------------RepartitionExec: partitioning=Hash([l_suppkey@0], 4), input_partitions=4 17)------------------------AggregateExec: mode=Partial, gby=[l_suppkey@0 as l_suppkey], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] -18)--------------------------ProjectionExec: expr=[l_suppkey@0 as l_suppkey, l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount] -19)----------------------------CoalesceBatchesExec: target_batch_size=8192 -20)------------------------------FilterExec: l_shipdate@3 >= 1996-01-01 AND l_shipdate@3 < 1996-04-01 -21)--------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_suppkey, l_extendedprice, l_discount, l_shipdate], has_header=false -22)--------CoalesceBatchesExec: target_batch_size=8192 -23)----------RepartitionExec: partitioning=Hash([max(revenue0.total_revenue)@0], 4), input_partitions=1 -24)------------AggregateExec: mode=Final, gby=[], aggr=[max(revenue0.total_revenue)] -25)--------------CoalescePartitionsExec -26)----------------AggregateExec: mode=Partial, gby=[], aggr=[max(revenue0.total_revenue)] -27)------------------ProjectionExec: expr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@1 as total_revenue] -28)--------------------AggregateExec: mode=FinalPartitioned, gby=[l_suppkey@0 as l_suppkey], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] -29)----------------------CoalesceBatchesExec: target_batch_size=8192 -30)------------------------RepartitionExec: partitioning=Hash([l_suppkey@0], 4), input_partitions=4 -31)--------------------------AggregateExec: mode=Partial, gby=[l_suppkey@0 as l_suppkey], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] -32)----------------------------ProjectionExec: expr=[l_suppkey@0 as l_suppkey, l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount] -33)------------------------------CoalesceBatchesExec: target_batch_size=8192 -34)--------------------------------FilterExec: l_shipdate@3 >= 1996-01-01 AND l_shipdate@3 < 1996-04-01 -35)----------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_suppkey, l_extendedprice, l_discount, l_shipdate], has_header=false +18)--------------------------CoalesceBatchesExec: target_batch_size=8192 +19)----------------------------FilterExec: l_shipdate@3 >= 1996-01-01 AND l_shipdate@3 < 1996-04-01, projection=[l_suppkey@0, l_extendedprice@1, l_discount@2] +20)------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_suppkey, l_extendedprice, l_discount, l_shipdate], has_header=false +21)--------CoalesceBatchesExec: target_batch_size=8192 +22)----------RepartitionExec: partitioning=Hash([max(revenue0.total_revenue)@0], 4), input_partitions=1 +23)------------AggregateExec: mode=Final, gby=[], aggr=[max(revenue0.total_revenue)] +24)--------------CoalescePartitionsExec +25)----------------AggregateExec: mode=Partial, gby=[], aggr=[max(revenue0.total_revenue)] +26)------------------ProjectionExec: expr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@1 as total_revenue] +27)--------------------AggregateExec: mode=FinalPartitioned, gby=[l_suppkey@0 as l_suppkey], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] +28)----------------------CoalesceBatchesExec: target_batch_size=8192 +29)------------------------RepartitionExec: partitioning=Hash([l_suppkey@0], 4), input_partitions=4 +30)--------------------------AggregateExec: mode=Partial, gby=[l_suppkey@0 as l_suppkey], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] +31)----------------------------CoalesceBatchesExec: target_batch_size=8192 +32)------------------------------FilterExec: l_shipdate@3 >= 1996-01-01 AND l_shipdate@3 < 1996-04-01, projection=[l_suppkey@0, l_extendedprice@1, l_discount@2] +33)--------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_suppkey, l_extendedprice, l_discount, l_shipdate], has_header=false query ITTTR with revenue0 (supplier_no, total_revenue) as ( diff --git a/datafusion/sqllogictest/test_files/tpch/q16.slt.part b/datafusion/sqllogictest/test_files/tpch/q16.slt.part index b847db14f2b2..8058371764f2 100644 --- a/datafusion/sqllogictest/test_files/tpch/q16.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q16.slt.part @@ -93,11 +93,10 @@ physical_plan 26)--------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_brand, p_type, p_size], has_header=false 27)--------------------------CoalesceBatchesExec: target_batch_size=8192 28)----------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 -29)------------------------------ProjectionExec: expr=[s_suppkey@0 as s_suppkey] -30)--------------------------------CoalesceBatchesExec: target_batch_size=8192 -31)----------------------------------FilterExec: s_comment@1 LIKE %Customer%Complaints% -32)------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -33)--------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_comment], has_header=false +29)------------------------------CoalesceBatchesExec: target_batch_size=8192 +30)--------------------------------FilterExec: s_comment@1 LIKE %Customer%Complaints%, projection=[s_suppkey@0] +31)----------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +32)------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_comment], has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/q17.slt.part b/datafusion/sqllogictest/test_files/tpch/q17.slt.part index ecb54e97b910..058bcb5f4962 100644 --- a/datafusion/sqllogictest/test_files/tpch/q17.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q17.slt.part @@ -64,17 +64,16 @@ physical_plan 11)--------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_partkey, l_quantity, l_extendedprice], has_header=false 12)----------------CoalesceBatchesExec: target_batch_size=8192 13)------------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 -14)--------------------ProjectionExec: expr=[p_partkey@0 as p_partkey] -15)----------------------CoalesceBatchesExec: target_batch_size=8192 -16)------------------------FilterExec: p_brand@1 = Brand#23 AND p_container@2 = MED BOX -17)--------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -18)----------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_brand, p_container], has_header=false -19)------------ProjectionExec: expr=[CAST(0.2 * CAST(avg(lineitem.l_quantity)@1 AS Float64) AS Decimal128(30, 15)) as Float64(0.2) * avg(lineitem.l_quantity), l_partkey@0 as l_partkey] -20)--------------AggregateExec: mode=FinalPartitioned, gby=[l_partkey@0 as l_partkey], aggr=[avg(lineitem.l_quantity)] -21)----------------CoalesceBatchesExec: target_batch_size=8192 -22)------------------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=4 -23)--------------------AggregateExec: mode=Partial, gby=[l_partkey@0 as l_partkey], aggr=[avg(lineitem.l_quantity)] -24)----------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_partkey, l_quantity], has_header=false +14)--------------------CoalesceBatchesExec: target_batch_size=8192 +15)----------------------FilterExec: p_brand@1 = Brand#23 AND p_container@2 = MED BOX, projection=[p_partkey@0] +16)------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +17)--------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_brand, p_container], has_header=false +18)------------ProjectionExec: expr=[CAST(0.2 * CAST(avg(lineitem.l_quantity)@1 AS Float64) AS Decimal128(30, 15)) as Float64(0.2) * avg(lineitem.l_quantity), l_partkey@0 as l_partkey] +19)--------------AggregateExec: mode=FinalPartitioned, gby=[l_partkey@0 as l_partkey], aggr=[avg(lineitem.l_quantity)] +20)----------------CoalesceBatchesExec: target_batch_size=8192 +21)------------------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=4 +22)--------------------AggregateExec: mode=Partial, gby=[l_partkey@0 as l_partkey], aggr=[avg(lineitem.l_quantity)] +23)----------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_partkey, l_quantity], has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/q18.slt.part b/datafusion/sqllogictest/test_files/tpch/q18.slt.part index 77f11e0d3e5a..e78b0d87f651 100644 --- a/datafusion/sqllogictest/test_files/tpch/q18.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q18.slt.part @@ -91,14 +91,13 @@ physical_plan 22)--------------------CoalesceBatchesExec: target_batch_size=8192 23)----------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 24)------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_quantity], has_header=false -25)----------------ProjectionExec: expr=[l_orderkey@0 as l_orderkey] -26)------------------CoalesceBatchesExec: target_batch_size=8192 -27)--------------------FilterExec: sum(lineitem.l_quantity)@1 > Some(30000),25,2 -28)----------------------AggregateExec: mode=FinalPartitioned, gby=[l_orderkey@0 as l_orderkey], aggr=[sum(lineitem.l_quantity)] -29)------------------------CoalesceBatchesExec: target_batch_size=8192 -30)--------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 -31)----------------------------AggregateExec: mode=Partial, gby=[l_orderkey@0 as l_orderkey], aggr=[sum(lineitem.l_quantity)] -32)------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_quantity], has_header=false +25)----------------CoalesceBatchesExec: target_batch_size=8192 +26)------------------FilterExec: sum(lineitem.l_quantity)@1 > Some(30000),25,2, projection=[l_orderkey@0] +27)--------------------AggregateExec: mode=FinalPartitioned, gby=[l_orderkey@0 as l_orderkey], aggr=[sum(lineitem.l_quantity)] +28)----------------------CoalesceBatchesExec: target_batch_size=8192 +29)------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 +30)--------------------------AggregateExec: mode=Partial, gby=[l_orderkey@0 as l_orderkey], aggr=[sum(lineitem.l_quantity)] +31)----------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_quantity], has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/q19.slt.part b/datafusion/sqllogictest/test_files/tpch/q19.slt.part index 41080ff68fc6..70465ea065a1 100644 --- a/datafusion/sqllogictest/test_files/tpch/q19.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q19.slt.part @@ -72,16 +72,15 @@ physical_plan 06)----------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_partkey@0, p_partkey@0)], filter=p_brand@1 = Brand#12 AND Use p_container@3 IN (SET) ([Literal { value: Utf8("SM CASE") }, Literal { value: Utf8("SM BOX") }, Literal { value: Utf8("SM PACK") }, Literal { value: Utf8("SM PKG") }]) AND l_quantity@0 >= Some(100),15,2 AND l_quantity@0 <= Some(1100),15,2 AND p_size@2 <= 5 OR p_brand@1 = Brand#23 AND Use p_container@3 IN (SET) ([Literal { value: Utf8("MED BAG") }, Literal { value: Utf8("MED BOX") }, Literal { value: Utf8("MED PKG") }, Literal { value: Utf8("MED PACK") }]) AND l_quantity@0 >= Some(1000),15,2 AND l_quantity@0 <= Some(2000),15,2 AND p_size@2 <= 10 OR p_brand@1 = Brand#34 AND Use p_container@3 IN (SET) ([Literal { value: Utf8("LG CASE") }, Literal { value: Utf8("LG BOX") }, Literal { value: Utf8("LG PACK") }, Literal { value: Utf8("LG PKG") }]) AND l_quantity@0 >= Some(2000),15,2 AND l_quantity@0 <= Some(3000),15,2 AND p_size@2 <= 15, projection=[l_extendedprice@2, l_discount@3] 07)------------CoalesceBatchesExec: target_batch_size=8192 08)--------------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=4 -09)----------------ProjectionExec: expr=[l_partkey@0 as l_partkey, l_quantity@1 as l_quantity, l_extendedprice@2 as l_extendedprice, l_discount@3 as l_discount] -10)------------------CoalesceBatchesExec: target_batch_size=8192 -11)--------------------FilterExec: (l_quantity@1 >= Some(100),15,2 AND l_quantity@1 <= Some(1100),15,2 OR l_quantity@1 >= Some(1000),15,2 AND l_quantity@1 <= Some(2000),15,2 OR l_quantity@1 >= Some(2000),15,2 AND l_quantity@1 <= Some(3000),15,2) AND (l_shipmode@5 = AIR OR l_shipmode@5 = AIR REG) AND l_shipinstruct@4 = DELIVER IN PERSON -12)----------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_partkey, l_quantity, l_extendedprice, l_discount, l_shipinstruct, l_shipmode], has_header=false -13)------------CoalesceBatchesExec: target_batch_size=8192 -14)--------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 -15)----------------CoalesceBatchesExec: target_batch_size=8192 -16)------------------FilterExec: (p_brand@1 = Brand#12 AND Use p_container@3 IN (SET) ([Literal { value: Utf8("SM CASE") }, Literal { value: Utf8("SM BOX") }, Literal { value: Utf8("SM PACK") }, Literal { value: Utf8("SM PKG") }]) AND p_size@2 <= 5 OR p_brand@1 = Brand#23 AND Use p_container@3 IN (SET) ([Literal { value: Utf8("MED BAG") }, Literal { value: Utf8("MED BOX") }, Literal { value: Utf8("MED PKG") }, Literal { value: Utf8("MED PACK") }]) AND p_size@2 <= 10 OR p_brand@1 = Brand#34 AND Use p_container@3 IN (SET) ([Literal { value: Utf8("LG CASE") }, Literal { value: Utf8("LG BOX") }, Literal { value: Utf8("LG PACK") }, Literal { value: Utf8("LG PKG") }]) AND p_size@2 <= 15) AND p_size@2 >= 1 -17)--------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -18)----------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_brand, p_size, p_container], has_header=false +09)----------------CoalesceBatchesExec: target_batch_size=8192 +10)------------------FilterExec: (l_quantity@1 >= Some(100),15,2 AND l_quantity@1 <= Some(1100),15,2 OR l_quantity@1 >= Some(1000),15,2 AND l_quantity@1 <= Some(2000),15,2 OR l_quantity@1 >= Some(2000),15,2 AND l_quantity@1 <= Some(3000),15,2) AND (l_shipmode@5 = AIR OR l_shipmode@5 = AIR REG) AND l_shipinstruct@4 = DELIVER IN PERSON, projection=[l_partkey@0, l_quantity@1, l_extendedprice@2, l_discount@3] +11)--------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_partkey, l_quantity, l_extendedprice, l_discount, l_shipinstruct, l_shipmode], has_header=false +12)------------CoalesceBatchesExec: target_batch_size=8192 +13)--------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 +14)----------------CoalesceBatchesExec: target_batch_size=8192 +15)------------------FilterExec: (p_brand@1 = Brand#12 AND Use p_container@3 IN (SET) ([Literal { value: Utf8("SM CASE") }, Literal { value: Utf8("SM BOX") }, Literal { value: Utf8("SM PACK") }, Literal { value: Utf8("SM PKG") }]) AND p_size@2 <= 5 OR p_brand@1 = Brand#23 AND Use p_container@3 IN (SET) ([Literal { value: Utf8("MED BAG") }, Literal { value: Utf8("MED BOX") }, Literal { value: Utf8("MED PKG") }, Literal { value: Utf8("MED PACK") }]) AND p_size@2 <= 10 OR p_brand@1 = Brand#34 AND Use p_container@3 IN (SET) ([Literal { value: Utf8("LG CASE") }, Literal { value: Utf8("LG BOX") }, Literal { value: Utf8("LG PACK") }, Literal { value: Utf8("LG PKG") }]) AND p_size@2 <= 15) AND p_size@2 >= 1 +16)--------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +17)----------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_brand, p_size, p_container], has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/q2.slt.part b/datafusion/sqllogictest/test_files/tpch/q2.slt.part index 223a011c9e37..17f3b78a089d 100644 --- a/datafusion/sqllogictest/test_files/tpch/q2.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q2.slt.part @@ -123,64 +123,61 @@ physical_plan 22)------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(p_partkey@0, ps_partkey@0)], projection=[p_partkey@0, p_mfgr@1, ps_suppkey@3, ps_supplycost@4] 23)--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 24)----------------------------------------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 -25)------------------------------------------------ProjectionExec: expr=[p_partkey@0 as p_partkey, p_mfgr@1 as p_mfgr] -26)--------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -27)----------------------------------------------------FilterExec: p_size@3 = 15 AND p_type@2 LIKE %BRASS -28)------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -29)--------------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_mfgr, p_type, p_size], has_header=false -30)--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -31)----------------------------------------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4 -32)------------------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:0..2932049], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:2932049..5864098], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:5864098..8796147], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:8796147..11728193]]}, projection=[ps_partkey, ps_suppkey, ps_supplycost], has_header=false -33)------------------------------------CoalesceBatchesExec: target_batch_size=8192 -34)--------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 -35)----------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -36)------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment], has_header=false -37)--------------------------CoalesceBatchesExec: target_batch_size=8192 -38)----------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 -39)------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -40)--------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name, n_regionkey], has_header=false -41)------------------CoalesceBatchesExec: target_batch_size=8192 -42)--------------------RepartitionExec: partitioning=Hash([r_regionkey@0], 4), input_partitions=4 -43)----------------------ProjectionExec: expr=[r_regionkey@0 as r_regionkey] -44)------------------------CoalesceBatchesExec: target_batch_size=8192 -45)--------------------------FilterExec: r_name@1 = EUROPE -46)----------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -47)------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/region.tbl]]}, projection=[r_regionkey, r_name], has_header=false -48)----------CoalesceBatchesExec: target_batch_size=8192 -49)------------RepartitionExec: partitioning=Hash([ps_partkey@1, min(partsupp.ps_supplycost)@0], 4), input_partitions=4 -50)--------------ProjectionExec: expr=[min(partsupp.ps_supplycost)@1 as min(partsupp.ps_supplycost), ps_partkey@0 as ps_partkey] -51)----------------AggregateExec: mode=FinalPartitioned, gby=[ps_partkey@0 as ps_partkey], aggr=[min(partsupp.ps_supplycost)] -52)------------------CoalesceBatchesExec: target_batch_size=8192 -53)--------------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4 -54)----------------------AggregateExec: mode=Partial, gby=[ps_partkey@0 as ps_partkey], aggr=[min(partsupp.ps_supplycost)] -55)------------------------CoalesceBatchesExec: target_batch_size=8192 -56)--------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(n_regionkey@2, r_regionkey@0)], projection=[ps_partkey@0, ps_supplycost@1] -57)----------------------------CoalesceBatchesExec: target_batch_size=8192 -58)------------------------------RepartitionExec: partitioning=Hash([n_regionkey@2], 4), input_partitions=4 -59)--------------------------------CoalesceBatchesExec: target_batch_size=8192 -60)----------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@2, n_nationkey@0)], projection=[ps_partkey@0, ps_supplycost@1, n_regionkey@4] -61)------------------------------------CoalesceBatchesExec: target_batch_size=8192 -62)--------------------------------------RepartitionExec: partitioning=Hash([s_nationkey@2], 4), input_partitions=4 -63)----------------------------------------CoalesceBatchesExec: target_batch_size=8192 -64)------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_suppkey@1, s_suppkey@0)], projection=[ps_partkey@0, ps_supplycost@2, s_nationkey@4] -65)--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -66)----------------------------------------------RepartitionExec: partitioning=Hash([ps_suppkey@1], 4), input_partitions=4 -67)------------------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:0..2932049], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:2932049..5864098], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:5864098..8796147], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:8796147..11728193]]}, projection=[ps_partkey, ps_suppkey, ps_supplycost], has_header=false -68)--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -69)----------------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 -70)------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -71)--------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], has_header=false -72)------------------------------------CoalesceBatchesExec: target_batch_size=8192 -73)--------------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 -74)----------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -75)------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_regionkey], has_header=false -76)----------------------------CoalesceBatchesExec: target_batch_size=8192 -77)------------------------------RepartitionExec: partitioning=Hash([r_regionkey@0], 4), input_partitions=4 -78)--------------------------------ProjectionExec: expr=[r_regionkey@0 as r_regionkey] -79)----------------------------------CoalesceBatchesExec: target_batch_size=8192 -80)------------------------------------FilterExec: r_name@1 = EUROPE -81)--------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -82)----------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/region.tbl]]}, projection=[r_regionkey, r_name], has_header=false +25)------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +26)--------------------------------------------------FilterExec: p_size@3 = 15 AND p_type@2 LIKE %BRASS, projection=[p_partkey@0, p_mfgr@1] +27)----------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +28)------------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_mfgr, p_type, p_size], has_header=false +29)--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +30)----------------------------------------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4 +31)------------------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:0..2932049], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:2932049..5864098], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:5864098..8796147], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:8796147..11728193]]}, projection=[ps_partkey, ps_suppkey, ps_supplycost], has_header=false +32)------------------------------------CoalesceBatchesExec: target_batch_size=8192 +33)--------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 +34)----------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +35)------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment], has_header=false +36)--------------------------CoalesceBatchesExec: target_batch_size=8192 +37)----------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 +38)------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +39)--------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name, n_regionkey], has_header=false +40)------------------CoalesceBatchesExec: target_batch_size=8192 +41)--------------------RepartitionExec: partitioning=Hash([r_regionkey@0], 4), input_partitions=4 +42)----------------------CoalesceBatchesExec: target_batch_size=8192 +43)------------------------FilterExec: r_name@1 = EUROPE, projection=[r_regionkey@0] +44)--------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +45)----------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/region.tbl]]}, projection=[r_regionkey, r_name], has_header=false +46)----------CoalesceBatchesExec: target_batch_size=8192 +47)------------RepartitionExec: partitioning=Hash([ps_partkey@1, min(partsupp.ps_supplycost)@0], 4), input_partitions=4 +48)--------------ProjectionExec: expr=[min(partsupp.ps_supplycost)@1 as min(partsupp.ps_supplycost), ps_partkey@0 as ps_partkey] +49)----------------AggregateExec: mode=FinalPartitioned, gby=[ps_partkey@0 as ps_partkey], aggr=[min(partsupp.ps_supplycost)] +50)------------------CoalesceBatchesExec: target_batch_size=8192 +51)--------------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4 +52)----------------------AggregateExec: mode=Partial, gby=[ps_partkey@0 as ps_partkey], aggr=[min(partsupp.ps_supplycost)] +53)------------------------CoalesceBatchesExec: target_batch_size=8192 +54)--------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(n_regionkey@2, r_regionkey@0)], projection=[ps_partkey@0, ps_supplycost@1] +55)----------------------------CoalesceBatchesExec: target_batch_size=8192 +56)------------------------------RepartitionExec: partitioning=Hash([n_regionkey@2], 4), input_partitions=4 +57)--------------------------------CoalesceBatchesExec: target_batch_size=8192 +58)----------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@2, n_nationkey@0)], projection=[ps_partkey@0, ps_supplycost@1, n_regionkey@4] +59)------------------------------------CoalesceBatchesExec: target_batch_size=8192 +60)--------------------------------------RepartitionExec: partitioning=Hash([s_nationkey@2], 4), input_partitions=4 +61)----------------------------------------CoalesceBatchesExec: target_batch_size=8192 +62)------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_suppkey@1, s_suppkey@0)], projection=[ps_partkey@0, ps_supplycost@2, s_nationkey@4] +63)--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +64)----------------------------------------------RepartitionExec: partitioning=Hash([ps_suppkey@1], 4), input_partitions=4 +65)------------------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:0..2932049], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:2932049..5864098], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:5864098..8796147], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:8796147..11728193]]}, projection=[ps_partkey, ps_suppkey, ps_supplycost], has_header=false +66)--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +67)----------------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 +68)------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +69)--------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], has_header=false +70)------------------------------------CoalesceBatchesExec: target_batch_size=8192 +71)--------------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 +72)----------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +73)------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_regionkey], has_header=false +74)----------------------------CoalesceBatchesExec: target_batch_size=8192 +75)------------------------------RepartitionExec: partitioning=Hash([r_regionkey@0], 4), input_partitions=4 +76)--------------------------------CoalesceBatchesExec: target_batch_size=8192 +77)----------------------------------FilterExec: r_name@1 = EUROPE, projection=[r_regionkey@0] +78)------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +79)--------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/region.tbl]]}, projection=[r_regionkey, r_name], has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/q20.slt.part b/datafusion/sqllogictest/test_files/tpch/q20.slt.part index d35d6886e0d8..67ea87b6ee61 100644 --- a/datafusion/sqllogictest/test_files/tpch/q20.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q20.slt.part @@ -95,38 +95,35 @@ physical_plan 12)----------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_name, s_address, s_nationkey], has_header=false 13)----------------CoalesceBatchesExec: target_batch_size=8192 14)------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 -15)--------------------ProjectionExec: expr=[n_nationkey@0 as n_nationkey] -16)----------------------CoalesceBatchesExec: target_batch_size=8192 -17)------------------------FilterExec: n_name@1 = CANADA -18)--------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -19)----------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false -20)--------CoalesceBatchesExec: target_batch_size=8192 -21)----------RepartitionExec: partitioning=Hash([ps_suppkey@0], 4), input_partitions=4 -22)------------CoalesceBatchesExec: target_batch_size=8192 -23)--------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_partkey@0, l_partkey@1), (ps_suppkey@1, l_suppkey@2)], filter=CAST(ps_availqty@0 AS Float64) > Float64(0.5) * sum(lineitem.l_quantity)@1, projection=[ps_suppkey@1] -24)----------------CoalesceBatchesExec: target_batch_size=8192 -25)------------------RepartitionExec: partitioning=Hash([ps_partkey@0, ps_suppkey@1], 4), input_partitions=4 -26)--------------------CoalesceBatchesExec: target_batch_size=8192 -27)----------------------HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(ps_partkey@0, p_partkey@0)] -28)------------------------CoalesceBatchesExec: target_batch_size=8192 -29)--------------------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4 -30)----------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:0..2932049], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:2932049..5864098], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:5864098..8796147], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:8796147..11728193]]}, projection=[ps_partkey, ps_suppkey, ps_availqty], has_header=false -31)------------------------CoalesceBatchesExec: target_batch_size=8192 -32)--------------------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 -33)----------------------------ProjectionExec: expr=[p_partkey@0 as p_partkey] -34)------------------------------CoalesceBatchesExec: target_batch_size=8192 -35)--------------------------------FilterExec: p_name@1 LIKE forest% -36)----------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -37)------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_name], has_header=false -38)----------------ProjectionExec: expr=[0.5 * CAST(sum(lineitem.l_quantity)@2 AS Float64) as Float64(0.5) * sum(lineitem.l_quantity), l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey] -39)------------------AggregateExec: mode=FinalPartitioned, gby=[l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey], aggr=[sum(lineitem.l_quantity)] -40)--------------------CoalesceBatchesExec: target_batch_size=8192 -41)----------------------RepartitionExec: partitioning=Hash([l_partkey@0, l_suppkey@1], 4), input_partitions=4 -42)------------------------AggregateExec: mode=Partial, gby=[l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey], aggr=[sum(lineitem.l_quantity)] -43)--------------------------ProjectionExec: expr=[l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey, l_quantity@2 as l_quantity] -44)----------------------------CoalesceBatchesExec: target_batch_size=8192 -45)------------------------------FilterExec: l_shipdate@3 >= 1994-01-01 AND l_shipdate@3 < 1995-01-01 -46)--------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_partkey, l_suppkey, l_quantity, l_shipdate], has_header=false +15)--------------------CoalesceBatchesExec: target_batch_size=8192 +16)----------------------FilterExec: n_name@1 = CANADA, projection=[n_nationkey@0] +17)------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +18)--------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false +19)--------CoalesceBatchesExec: target_batch_size=8192 +20)----------RepartitionExec: partitioning=Hash([ps_suppkey@0], 4), input_partitions=4 +21)------------CoalesceBatchesExec: target_batch_size=8192 +22)--------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_partkey@0, l_partkey@1), (ps_suppkey@1, l_suppkey@2)], filter=CAST(ps_availqty@0 AS Float64) > Float64(0.5) * sum(lineitem.l_quantity)@1, projection=[ps_suppkey@1] +23)----------------CoalesceBatchesExec: target_batch_size=8192 +24)------------------RepartitionExec: partitioning=Hash([ps_partkey@0, ps_suppkey@1], 4), input_partitions=4 +25)--------------------CoalesceBatchesExec: target_batch_size=8192 +26)----------------------HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(ps_partkey@0, p_partkey@0)] +27)------------------------CoalesceBatchesExec: target_batch_size=8192 +28)--------------------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4 +29)----------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:0..2932049], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:2932049..5864098], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:5864098..8796147], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:8796147..11728193]]}, projection=[ps_partkey, ps_suppkey, ps_availqty], has_header=false +30)------------------------CoalesceBatchesExec: target_batch_size=8192 +31)--------------------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 +32)----------------------------CoalesceBatchesExec: target_batch_size=8192 +33)------------------------------FilterExec: p_name@1 LIKE forest%, projection=[p_partkey@0] +34)--------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +35)----------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_name], has_header=false +36)----------------ProjectionExec: expr=[0.5 * CAST(sum(lineitem.l_quantity)@2 AS Float64) as Float64(0.5) * sum(lineitem.l_quantity), l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey] +37)------------------AggregateExec: mode=FinalPartitioned, gby=[l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey], aggr=[sum(lineitem.l_quantity)] +38)--------------------CoalesceBatchesExec: target_batch_size=8192 +39)----------------------RepartitionExec: partitioning=Hash([l_partkey@0, l_suppkey@1], 4), input_partitions=4 +40)------------------------AggregateExec: mode=Partial, gby=[l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey], aggr=[sum(lineitem.l_quantity)] +41)--------------------------CoalesceBatchesExec: target_batch_size=8192 +42)----------------------------FilterExec: l_shipdate@3 >= 1994-01-01 AND l_shipdate@3 < 1995-01-01, projection=[l_partkey@0, l_suppkey@1, l_quantity@2] +43)------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_partkey, l_suppkey, l_quantity, l_shipdate], has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/q21.slt.part b/datafusion/sqllogictest/test_files/tpch/q21.slt.part index 74c1c2fa77d7..5cf069ec7248 100644 --- a/datafusion/sqllogictest/test_files/tpch/q21.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q21.slt.part @@ -119,32 +119,28 @@ physical_plan 27)----------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_name, s_nationkey], has_header=false 28)----------------------------------------------CoalesceBatchesExec: target_batch_size=8192 29)------------------------------------------------RepartitionExec: partitioning=Hash([l_suppkey@1], 4), input_partitions=4 -30)--------------------------------------------------ProjectionExec: expr=[l_orderkey@0 as l_orderkey, l_suppkey@1 as l_suppkey] -31)----------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -32)------------------------------------------------------FilterExec: l_receiptdate@3 > l_commitdate@2 -33)--------------------------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_suppkey, l_commitdate, l_receiptdate], has_header=false -34)--------------------------------------CoalesceBatchesExec: target_batch_size=8192 -35)----------------------------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 -36)------------------------------------------ProjectionExec: expr=[o_orderkey@0 as o_orderkey] -37)--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -38)----------------------------------------------FilterExec: o_orderstatus@1 = F -39)------------------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_orderstatus], has_header=false -40)------------------------------CoalesceBatchesExec: target_batch_size=8192 -41)--------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 -42)----------------------------------ProjectionExec: expr=[n_nationkey@0 as n_nationkey] -43)------------------------------------CoalesceBatchesExec: target_batch_size=8192 -44)--------------------------------------FilterExec: n_name@1 = SAUDI ARABIA -45)----------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -46)------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false -47)----------------------CoalesceBatchesExec: target_batch_size=8192 -48)------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 -49)--------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_suppkey], has_header=false -50)------------------CoalesceBatchesExec: target_batch_size=8192 -51)--------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 -52)----------------------ProjectionExec: expr=[l_orderkey@0 as l_orderkey, l_suppkey@1 as l_suppkey] -53)------------------------CoalesceBatchesExec: target_batch_size=8192 -54)--------------------------FilterExec: l_receiptdate@3 > l_commitdate@2 -55)----------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_suppkey, l_commitdate, l_receiptdate], has_header=false +30)--------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +31)----------------------------------------------------FilterExec: l_receiptdate@3 > l_commitdate@2, projection=[l_orderkey@0, l_suppkey@1] +32)------------------------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_suppkey, l_commitdate, l_receiptdate], has_header=false +33)--------------------------------------CoalesceBatchesExec: target_batch_size=8192 +34)----------------------------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 +35)------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +36)--------------------------------------------FilterExec: o_orderstatus@1 = F, projection=[o_orderkey@0] +37)----------------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_orderstatus], has_header=false +38)------------------------------CoalesceBatchesExec: target_batch_size=8192 +39)--------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 +40)----------------------------------CoalesceBatchesExec: target_batch_size=8192 +41)------------------------------------FilterExec: n_name@1 = SAUDI ARABIA, projection=[n_nationkey@0] +42)--------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +43)----------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false +44)----------------------CoalesceBatchesExec: target_batch_size=8192 +45)------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 +46)--------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_suppkey], has_header=false +47)------------------CoalesceBatchesExec: target_batch_size=8192 +48)--------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 +49)----------------------CoalesceBatchesExec: target_batch_size=8192 +50)------------------------FilterExec: l_receiptdate@3 > l_commitdate@2, projection=[l_orderkey@0, l_suppkey@1] +51)--------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_suppkey, l_commitdate, l_receiptdate], has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/q22.slt.part b/datafusion/sqllogictest/test_files/tpch/q22.slt.part index b3bfc329642f..d2168b0136ba 100644 --- a/datafusion/sqllogictest/test_files/tpch/q22.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q22.slt.part @@ -99,11 +99,10 @@ physical_plan 23)--------------------AggregateExec: mode=Final, gby=[], aggr=[avg(customer.c_acctbal)] 24)----------------------CoalescePartitionsExec 25)------------------------AggregateExec: mode=Partial, gby=[], aggr=[avg(customer.c_acctbal)] -26)--------------------------ProjectionExec: expr=[c_acctbal@1 as c_acctbal] -27)----------------------------CoalesceBatchesExec: target_batch_size=8192 -28)------------------------------FilterExec: c_acctbal@1 > Some(0),15,2 AND Use substr(c_phone@0, 1, 2) IN (SET) ([Literal { value: Utf8("13") }, Literal { value: Utf8("31") }, Literal { value: Utf8("23") }, Literal { value: Utf8("29") }, Literal { value: Utf8("30") }, Literal { value: Utf8("18") }, Literal { value: Utf8("17") }]) -29)--------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -30)----------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_phone, c_acctbal], has_header=false +26)--------------------------CoalesceBatchesExec: target_batch_size=8192 +27)----------------------------FilterExec: c_acctbal@1 > Some(0),15,2 AND Use substr(c_phone@0, 1, 2) IN (SET) ([Literal { value: Utf8("13") }, Literal { value: Utf8("31") }, Literal { value: Utf8("23") }, Literal { value: Utf8("29") }, Literal { value: Utf8("30") }, Literal { value: Utf8("18") }, Literal { value: Utf8("17") }]), projection=[c_acctbal@1] +28)------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +29)--------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_phone, c_acctbal], has_header=false query TIR diff --git a/datafusion/sqllogictest/test_files/tpch/q3.slt.part b/datafusion/sqllogictest/test_files/tpch/q3.slt.part index 1a8512372d37..16a1c2b6ebb1 100644 --- a/datafusion/sqllogictest/test_files/tpch/q3.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q3.slt.part @@ -73,22 +73,20 @@ physical_plan 13)------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_custkey@0, o_custkey@1)], projection=[o_orderkey@1, o_orderdate@3, o_shippriority@4] 14)--------------------------CoalesceBatchesExec: target_batch_size=8192 15)----------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 -16)------------------------------ProjectionExec: expr=[c_custkey@0 as c_custkey] -17)--------------------------------CoalesceBatchesExec: target_batch_size=8192 -18)----------------------------------FilterExec: c_mktsegment@1 = BUILDING -19)------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -20)--------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_mktsegment], has_header=false -21)--------------------------CoalesceBatchesExec: target_batch_size=8192 -22)----------------------------RepartitionExec: partitioning=Hash([o_custkey@1], 4), input_partitions=4 -23)------------------------------CoalesceBatchesExec: target_batch_size=8192 -24)--------------------------------FilterExec: o_orderdate@2 < 1995-03-15 -25)----------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_custkey, o_orderdate, o_shippriority], has_header=false -26)------------------CoalesceBatchesExec: target_batch_size=8192 -27)--------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 -28)----------------------ProjectionExec: expr=[l_orderkey@0 as l_orderkey, l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount] -29)------------------------CoalesceBatchesExec: target_batch_size=8192 -30)--------------------------FilterExec: l_shipdate@3 > 1995-03-15 -31)----------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_extendedprice, l_discount, l_shipdate], has_header=false +16)------------------------------CoalesceBatchesExec: target_batch_size=8192 +17)--------------------------------FilterExec: c_mktsegment@1 = BUILDING, projection=[c_custkey@0] +18)----------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +19)------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_mktsegment], has_header=false +20)--------------------------CoalesceBatchesExec: target_batch_size=8192 +21)----------------------------RepartitionExec: partitioning=Hash([o_custkey@1], 4), input_partitions=4 +22)------------------------------CoalesceBatchesExec: target_batch_size=8192 +23)--------------------------------FilterExec: o_orderdate@2 < 1995-03-15 +24)----------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_custkey, o_orderdate, o_shippriority], has_header=false +25)------------------CoalesceBatchesExec: target_batch_size=8192 +26)--------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 +27)----------------------CoalesceBatchesExec: target_batch_size=8192 +28)------------------------FilterExec: l_shipdate@3 > 1995-03-15, projection=[l_orderkey@0, l_extendedprice@1, l_discount@2] +29)--------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_extendedprice, l_discount, l_shipdate], has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/q4.slt.part b/datafusion/sqllogictest/test_files/tpch/q4.slt.part index b5a40e5b62d1..a68b745c366c 100644 --- a/datafusion/sqllogictest/test_files/tpch/q4.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q4.slt.part @@ -64,16 +64,14 @@ physical_plan 09)----------------HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(o_orderkey@0, l_orderkey@0)], projection=[o_orderpriority@1] 10)------------------CoalesceBatchesExec: target_batch_size=8192 11)--------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 -12)----------------------ProjectionExec: expr=[o_orderkey@0 as o_orderkey, o_orderpriority@2 as o_orderpriority] -13)------------------------CoalesceBatchesExec: target_batch_size=8192 -14)--------------------------FilterExec: o_orderdate@1 >= 1993-07-01 AND o_orderdate@1 < 1993-10-01 -15)----------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_orderdate, o_orderpriority], has_header=false -16)------------------CoalesceBatchesExec: target_batch_size=8192 -17)--------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 -18)----------------------ProjectionExec: expr=[l_orderkey@0 as l_orderkey] -19)------------------------CoalesceBatchesExec: target_batch_size=8192 -20)--------------------------FilterExec: l_receiptdate@2 > l_commitdate@1 -21)----------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_commitdate, l_receiptdate], has_header=false +12)----------------------CoalesceBatchesExec: target_batch_size=8192 +13)------------------------FilterExec: o_orderdate@1 >= 1993-07-01 AND o_orderdate@1 < 1993-10-01, projection=[o_orderkey@0, o_orderpriority@2] +14)--------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_orderdate, o_orderpriority], has_header=false +15)------------------CoalesceBatchesExec: target_batch_size=8192 +16)--------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 +17)----------------------CoalesceBatchesExec: target_batch_size=8192 +18)------------------------FilterExec: l_receiptdate@2 > l_commitdate@1, projection=[l_orderkey@0] +19)--------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_commitdate, l_receiptdate], has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/q5.slt.part b/datafusion/sqllogictest/test_files/tpch/q5.slt.part index 57cdace3ff90..e59daf4943e8 100644 --- a/datafusion/sqllogictest/test_files/tpch/q5.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q5.slt.part @@ -98,28 +98,26 @@ physical_plan 29)--------------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_nationkey], has_header=false 30)--------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 31)----------------------------------------------------RepartitionExec: partitioning=Hash([o_custkey@1], 4), input_partitions=4 -32)------------------------------------------------------ProjectionExec: expr=[o_orderkey@0 as o_orderkey, o_custkey@1 as o_custkey] -33)--------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -34)----------------------------------------------------------FilterExec: o_orderdate@2 >= 1994-01-01 AND o_orderdate@2 < 1995-01-01 -35)------------------------------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_custkey, o_orderdate], has_header=false -36)------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -37)--------------------------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 -38)----------------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_suppkey, l_extendedprice, l_discount], has_header=false -39)----------------------------------CoalesceBatchesExec: target_batch_size=8192 -40)------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0, s_nationkey@1], 4), input_partitions=4 -41)--------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -42)----------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], has_header=false -43)--------------------------CoalesceBatchesExec: target_batch_size=8192 -44)----------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 -45)------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -46)--------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name, n_regionkey], has_header=false -47)------------------CoalesceBatchesExec: target_batch_size=8192 -48)--------------------RepartitionExec: partitioning=Hash([r_regionkey@0], 4), input_partitions=4 -49)----------------------ProjectionExec: expr=[r_regionkey@0 as r_regionkey] -50)------------------------CoalesceBatchesExec: target_batch_size=8192 -51)--------------------------FilterExec: r_name@1 = ASIA -52)----------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -53)------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/region.tbl]]}, projection=[r_regionkey, r_name], has_header=false +32)------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +33)--------------------------------------------------------FilterExec: o_orderdate@2 >= 1994-01-01 AND o_orderdate@2 < 1995-01-01, projection=[o_orderkey@0, o_custkey@1] +34)----------------------------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_custkey, o_orderdate], has_header=false +35)------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +36)--------------------------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 +37)----------------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_suppkey, l_extendedprice, l_discount], has_header=false +38)----------------------------------CoalesceBatchesExec: target_batch_size=8192 +39)------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0, s_nationkey@1], 4), input_partitions=4 +40)--------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +41)----------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], has_header=false +42)--------------------------CoalesceBatchesExec: target_batch_size=8192 +43)----------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 +44)------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +45)--------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name, n_regionkey], has_header=false +46)------------------CoalesceBatchesExec: target_batch_size=8192 +47)--------------------RepartitionExec: partitioning=Hash([r_regionkey@0], 4), input_partitions=4 +48)----------------------CoalesceBatchesExec: target_batch_size=8192 +49)------------------------FilterExec: r_name@1 = ASIA, projection=[r_regionkey@0] +50)--------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +51)----------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/region.tbl]]}, projection=[r_regionkey, r_name], has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/q6.slt.part b/datafusion/sqllogictest/test_files/tpch/q6.slt.part index 5b1031f60a5d..548d26972f14 100644 --- a/datafusion/sqllogictest/test_files/tpch/q6.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q6.slt.part @@ -38,10 +38,9 @@ physical_plan 02)--AggregateExec: mode=Final, gby=[], aggr=[sum(lineitem.l_extendedprice * lineitem.l_discount)] 03)----CoalescePartitionsExec 04)------AggregateExec: mode=Partial, gby=[], aggr=[sum(lineitem.l_extendedprice * lineitem.l_discount)] -05)--------ProjectionExec: expr=[l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount] -06)----------CoalesceBatchesExec: target_batch_size=8192 -07)------------FilterExec: l_shipdate@3 >= 1994-01-01 AND l_shipdate@3 < 1995-01-01 AND l_discount@2 >= Some(5),15,2 AND l_discount@2 <= Some(7),15,2 AND l_quantity@0 < Some(2400),15,2 -08)--------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_quantity, l_extendedprice, l_discount, l_shipdate], has_header=false +05)--------CoalesceBatchesExec: target_batch_size=8192 +06)----------FilterExec: l_shipdate@3 >= 1994-01-01 AND l_shipdate@3 < 1995-01-01 AND l_discount@2 >= Some(5),15,2 AND l_discount@2 <= Some(7),15,2 AND l_quantity@0 < Some(2400),15,2, projection=[l_extendedprice@1, l_discount@2] +07)------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_quantity, l_extendedprice, l_discount, l_shipdate], has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/q8.slt.part b/datafusion/sqllogictest/test_files/tpch/q8.slt.part index c13eee3e785b..fd5773438466 100644 --- a/datafusion/sqllogictest/test_files/tpch/q8.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q8.slt.part @@ -126,42 +126,40 @@ physical_plan 34)------------------------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(p_partkey@0, l_partkey@1)], projection=[l_orderkey@1, l_suppkey@3, l_extendedprice@4, l_discount@5] 35)--------------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 36)----------------------------------------------------------------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 -37)------------------------------------------------------------------------ProjectionExec: expr=[p_partkey@0 as p_partkey] -38)--------------------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -39)----------------------------------------------------------------------------FilterExec: p_type@1 = ECONOMY ANODIZED STEEL -40)------------------------------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -41)--------------------------------------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_type], has_header=false -42)--------------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -43)----------------------------------------------------------------------RepartitionExec: partitioning=Hash([l_partkey@1], 4), input_partitions=4 -44)------------------------------------------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_partkey, l_suppkey, l_extendedprice, l_discount], has_header=false -45)------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -46)--------------------------------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 -47)----------------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -48)------------------------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], has_header=false -49)----------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -50)------------------------------------------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 -51)--------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -52)----------------------------------------------------------FilterExec: o_orderdate@2 >= 1995-01-01 AND o_orderdate@2 <= 1996-12-31 -53)------------------------------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_custkey, o_orderdate], has_header=false -54)--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -55)----------------------------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 -56)------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -57)--------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_nationkey], has_header=false -58)------------------------------------CoalesceBatchesExec: target_batch_size=8192 -59)--------------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 -60)----------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -61)------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_regionkey], has_header=false -62)----------------------------CoalesceBatchesExec: target_batch_size=8192 -63)------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 -64)--------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -65)----------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false -66)--------------------CoalesceBatchesExec: target_batch_size=8192 -67)----------------------RepartitionExec: partitioning=Hash([r_regionkey@0], 4), input_partitions=4 -68)------------------------ProjectionExec: expr=[r_regionkey@0 as r_regionkey] -69)--------------------------CoalesceBatchesExec: target_batch_size=8192 -70)----------------------------FilterExec: r_name@1 = AMERICA -71)------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -72)--------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/region.tbl]]}, projection=[r_regionkey, r_name], has_header=false +37)------------------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +38)--------------------------------------------------------------------------FilterExec: p_type@1 = ECONOMY ANODIZED STEEL, projection=[p_partkey@0] +39)----------------------------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +40)------------------------------------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_type], has_header=false +41)--------------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +42)----------------------------------------------------------------------RepartitionExec: partitioning=Hash([l_partkey@1], 4), input_partitions=4 +43)------------------------------------------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_partkey, l_suppkey, l_extendedprice, l_discount], has_header=false +44)------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +45)--------------------------------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 +46)----------------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +47)------------------------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], has_header=false +48)----------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +49)------------------------------------------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 +50)--------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +51)----------------------------------------------------------FilterExec: o_orderdate@2 >= 1995-01-01 AND o_orderdate@2 <= 1996-12-31 +52)------------------------------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_custkey, o_orderdate], has_header=false +53)--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +54)----------------------------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 +55)------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +56)--------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_nationkey], has_header=false +57)------------------------------------CoalesceBatchesExec: target_batch_size=8192 +58)--------------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 +59)----------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +60)------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_regionkey], has_header=false +61)----------------------------CoalesceBatchesExec: target_batch_size=8192 +62)------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 +63)--------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +64)----------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false +65)--------------------CoalesceBatchesExec: target_batch_size=8192 +66)----------------------RepartitionExec: partitioning=Hash([r_regionkey@0], 4), input_partitions=4 +67)------------------------CoalesceBatchesExec: target_batch_size=8192 +68)--------------------------FilterExec: r_name@1 = AMERICA, projection=[r_regionkey@0] +69)----------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +70)------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/region.tbl]]}, projection=[r_regionkey, r_name], has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/q9.slt.part b/datafusion/sqllogictest/test_files/tpch/q9.slt.part index a3fe2e3c675b..b3631f07cc8f 100644 --- a/datafusion/sqllogictest/test_files/tpch/q9.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q9.slt.part @@ -103,28 +103,27 @@ physical_plan 26)--------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(p_partkey@0, l_partkey@1)], projection=[l_orderkey@1, l_partkey@2, l_suppkey@3, l_quantity@4, l_extendedprice@5, l_discount@6] 27)----------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 28)------------------------------------------------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 -29)--------------------------------------------------------ProjectionExec: expr=[p_partkey@0 as p_partkey] -30)----------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -31)------------------------------------------------------------FilterExec: p_name@1 LIKE %green% -32)--------------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -33)----------------------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_name], has_header=false -34)----------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -35)------------------------------------------------------RepartitionExec: partitioning=Hash([l_partkey@1], 4), input_partitions=4 -36)--------------------------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_partkey, l_suppkey, l_quantity, l_extendedprice, l_discount], has_header=false -37)--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -38)----------------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 -39)------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -40)--------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], has_header=false -41)------------------------------------CoalesceBatchesExec: target_batch_size=8192 -42)--------------------------------------RepartitionExec: partitioning=Hash([ps_suppkey@1, ps_partkey@0], 4), input_partitions=4 -43)----------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:0..2932049], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:2932049..5864098], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:5864098..8796147], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:8796147..11728193]]}, projection=[ps_partkey, ps_suppkey, ps_supplycost], has_header=false -44)----------------------------CoalesceBatchesExec: target_batch_size=8192 -45)------------------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 -46)--------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_orderdate], has_header=false -47)--------------------CoalesceBatchesExec: target_batch_size=8192 -48)----------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 -49)------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -50)--------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false +29)--------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +30)----------------------------------------------------------FilterExec: p_name@1 LIKE %green%, projection=[p_partkey@0] +31)------------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +32)--------------------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_name], has_header=false +33)----------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +34)------------------------------------------------------RepartitionExec: partitioning=Hash([l_partkey@1], 4), input_partitions=4 +35)--------------------------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_partkey, l_suppkey, l_quantity, l_extendedprice, l_discount], has_header=false +36)--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +37)----------------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 +38)------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +39)--------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], has_header=false +40)------------------------------------CoalesceBatchesExec: target_batch_size=8192 +41)--------------------------------------RepartitionExec: partitioning=Hash([ps_suppkey@1, ps_partkey@0], 4), input_partitions=4 +42)----------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:0..2932049], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:2932049..5864098], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:5864098..8796147], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:8796147..11728193]]}, projection=[ps_partkey, ps_suppkey, ps_supplycost], has_header=false +43)----------------------------CoalesceBatchesExec: target_batch_size=8192 +44)------------------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 +45)--------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_orderdate], has_header=false +46)--------------------CoalesceBatchesExec: target_batch_size=8192 +47)----------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 +48)------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +49)--------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false diff --git a/datafusion/sqllogictest/test_files/union.slt b/datafusion/sqllogictest/test_files/union.slt index d2c013373d0c..a3d0ff4383ae 100644 --- a/datafusion/sqllogictest/test_files/union.slt +++ b/datafusion/sqllogictest/test_files/union.slt @@ -522,17 +522,16 @@ physical_plan 10)------------------CoalesceBatchesExec: target_batch_size=2 11)--------------------RepartitionExec: partitioning=Hash([c1@0], 4), input_partitions=4 12)----------------------AggregateExec: mode=Partial, gby=[c1@0 as c1], aggr=[] -13)------------------------ProjectionExec: expr=[c1@0 as c1] -14)--------------------------CoalesceBatchesExec: target_batch_size=2 -15)----------------------------FilterExec: c13@1 != C2GT5KVyOPZpgKVl110TyZO0NcJ434 -16)------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -17)--------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c13], has_header=true -18)------ProjectionExec: expr=[1 as cnt] -19)--------PlaceholderRowExec -20)------ProjectionExec: expr=[LEAD(b.c1,Int64(1)) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@1 as cnt] -21)--------BoundedWindowAggExec: wdw=[LEAD(b.c1,Int64(1)) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "LEAD(b.c1,Int64(1)) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)), is_causal: false }], mode=[Sorted] -22)----------ProjectionExec: expr=[1 as c1] -23)------------PlaceholderRowExec +13)------------------------CoalesceBatchesExec: target_batch_size=2 +14)--------------------------FilterExec: c13@1 != C2GT5KVyOPZpgKVl110TyZO0NcJ434, projection=[c1@0] +15)----------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +16)------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c13], has_header=true +17)------ProjectionExec: expr=[1 as cnt] +18)--------PlaceholderRowExec +19)------ProjectionExec: expr=[LEAD(b.c1,Int64(1)) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@1 as cnt] +20)--------BoundedWindowAggExec: wdw=[LEAD(b.c1,Int64(1)) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "LEAD(b.c1,Int64(1)) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)), is_causal: false }], mode=[Sorted] +21)----------ProjectionExec: expr=[1 as c1] +22)------------PlaceholderRowExec ######## diff --git a/datafusion/sqllogictest/test_files/window.slt b/datafusion/sqllogictest/test_files/window.slt index 5bf5cf83284f..505c66aef058 100644 --- a/datafusion/sqllogictest/test_files/window.slt +++ b/datafusion/sqllogictest/test_files/window.slt @@ -1782,11 +1782,10 @@ physical_plan 07)------------CoalesceBatchesExec: target_batch_size=4096 08)--------------RepartitionExec: partitioning=Hash([c1@0], 2), input_partitions=2 09)----------------AggregateExec: mode=Partial, gby=[c1@0 as c1], aggr=[] -10)------------------ProjectionExec: expr=[c1@0 as c1] -11)--------------------CoalesceBatchesExec: target_batch_size=4096 -12)----------------------FilterExec: c13@1 != C2GT5KVyOPZpgKVl110TyZO0NcJ434 -13)------------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -14)--------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c13], has_header=true +10)------------------CoalesceBatchesExec: target_batch_size=4096 +11)--------------------FilterExec: c13@1 != C2GT5KVyOPZpgKVl110TyZO0NcJ434, projection=[c1@0] +12)----------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +13)------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c13], has_header=true query I From cad41464fb2be574bfe9e72b4279c56ec0ebd132 Mon Sep 17 00:00:00 2001 From: Sergei Grebnov Date: Fri, 6 Sep 2024 07:29:01 -0700 Subject: [PATCH 106/154] Fix subquery alias table definition unparsing for SQLite (#12331) * Fix subquery alias table definition unparsing for SQLite * Fix lint * Update datafusion/sql/src/unparser/plan.rs Co-authored-by: Phillip LeBlanc * Update datafusion/sql/src/unparser/plan.rs Co-authored-by: Phillip LeBlanc * Refactor inject_column_aliases --------- Co-authored-by: Phillip LeBlanc --- datafusion/sql/src/unparser/dialect.rs | 29 ++++++++++++++++++ datafusion/sql/src/unparser/plan.rs | 36 +++++++++++++++++++---- datafusion/sql/src/unparser/rewrite.rs | 36 ++++++++++++++++++++++- datafusion/sql/tests/cases/plan_to_sql.rs | 16 ++++++++-- 4 files changed, 109 insertions(+), 8 deletions(-) diff --git a/datafusion/sql/src/unparser/dialect.rs b/datafusion/sql/src/unparser/dialect.rs index 2a8e61add1d0..d8a4fb254264 100644 --- a/datafusion/sql/src/unparser/dialect.rs +++ b/datafusion/sql/src/unparser/dialect.rs @@ -102,6 +102,12 @@ pub trait Dialect: Send + Sync { fn date32_cast_dtype(&self) -> sqlparser::ast::DataType { sqlparser::ast::DataType::Date } + + /// Does the dialect support specifying column aliases as part of alias table definition? + /// (SELECT col1, col2 from my_table) AS my_table_alias(col1_alias, col2_alias) + fn supports_column_alias_in_table_alias(&self) -> bool { + true + } } /// `IntervalStyle` to use for unparsing @@ -221,6 +227,10 @@ impl Dialect for SqliteDialect { fn date32_cast_dtype(&self) -> sqlparser::ast::DataType { sqlparser::ast::DataType::Text } + + fn supports_column_alias_in_table_alias(&self) -> bool { + false + } } pub struct CustomDialect { @@ -236,6 +246,7 @@ pub struct CustomDialect { timestamp_cast_dtype: ast::DataType, timestamp_tz_cast_dtype: ast::DataType, date32_cast_dtype: sqlparser::ast::DataType, + supports_column_alias_in_table_alias: bool, } impl Default for CustomDialect { @@ -256,6 +267,7 @@ impl Default for CustomDialect { TimezoneInfo::WithTimeZone, ), date32_cast_dtype: sqlparser::ast::DataType::Date, + supports_column_alias_in_table_alias: true, } } } @@ -323,6 +335,10 @@ impl Dialect for CustomDialect { fn date32_cast_dtype(&self) -> sqlparser::ast::DataType { self.date32_cast_dtype.clone() } + + fn supports_column_alias_in_table_alias(&self) -> bool { + self.supports_column_alias_in_table_alias + } } /// `CustomDialectBuilder` to build `CustomDialect` using builder pattern @@ -352,6 +368,7 @@ pub struct CustomDialectBuilder { timestamp_cast_dtype: ast::DataType, timestamp_tz_cast_dtype: ast::DataType, date32_cast_dtype: ast::DataType, + supports_column_alias_in_table_alias: bool, } impl Default for CustomDialectBuilder { @@ -378,6 +395,7 @@ impl CustomDialectBuilder { TimezoneInfo::WithTimeZone, ), date32_cast_dtype: sqlparser::ast::DataType::Date, + supports_column_alias_in_table_alias: true, } } @@ -395,6 +413,8 @@ impl CustomDialectBuilder { timestamp_cast_dtype: self.timestamp_cast_dtype, timestamp_tz_cast_dtype: self.timestamp_tz_cast_dtype, date32_cast_dtype: self.date32_cast_dtype, + supports_column_alias_in_table_alias: self + .supports_column_alias_in_table_alias, } } @@ -482,4 +502,13 @@ impl CustomDialectBuilder { self.date32_cast_dtype = date32_cast_dtype; self } + + /// Customize the dialect to supports column aliases as part of alias table definition + pub fn with_supports_column_alias_in_table_alias( + mut self, + supports_column_alias_in_table_alias: bool, + ) -> Self { + self.supports_column_alias_in_table_alias = supports_column_alias_in_table_alias; + self + } } diff --git a/datafusion/sql/src/unparser/plan.rs b/datafusion/sql/src/unparser/plan.rs index 509c5dd52cd4..478cfd0ecd90 100644 --- a/datafusion/sql/src/unparser/plan.rs +++ b/datafusion/sql/src/unparser/plan.rs @@ -15,7 +15,9 @@ // specific language governing permissions and limitations // under the License. -use datafusion_common::{internal_err, not_impl_err, Column, DataFusionError, Result}; +use datafusion_common::{ + internal_err, not_impl_err, plan_err, Column, DataFusionError, Result, +}; use datafusion_expr::{ expr::Alias, Distinct, Expr, JoinConstraint, JoinType, LogicalPlan, Projection, SortExpr, @@ -30,7 +32,8 @@ use super::{ SelectBuilder, TableRelationBuilder, TableWithJoinsBuilder, }, rewrite::{ - normalize_union_schema, rewrite_plan_for_sort_on_non_projected_fields, + inject_column_aliases, normalize_union_schema, + rewrite_plan_for_sort_on_non_projected_fields, subquery_alias_inner_query_and_columns, }, utils::{find_agg_node_within_select, unproject_window_exprs, AggVariant}, @@ -450,10 +453,33 @@ impl Unparser<'_> { Ok(()) } LogicalPlan::SubqueryAlias(plan_alias) => { - // Handle bottom-up to allocate relation - let (plan, columns) = subquery_alias_inner_query_and_columns(plan_alias); + let (plan, mut columns) = + subquery_alias_inner_query_and_columns(plan_alias); + + if !columns.is_empty() + && !self.dialect.supports_column_alias_in_table_alias() + { + // if columns are returned then the plan corresponds to a projection + let LogicalPlan::Projection(inner_p) = plan else { + return plan_err!( + "Inner projection for subquery alias is expected" + ); + }; + + // Instead of specifying column aliases as part of the outer table, inject them directly into the inner projection + let rewritten_plan = inject_column_aliases(inner_p, columns); + columns = vec![]; + + self.select_to_sql_recursively( + &rewritten_plan, + query, + select, + relation, + )?; + } else { + self.select_to_sql_recursively(plan, query, select, relation)?; + } - self.select_to_sql_recursively(plan, query, select, relation)?; relation.alias(Some( self.new_table_alias(plan_alias.alias.table().to_string(), columns), )); diff --git a/datafusion/sql/src/unparser/rewrite.rs b/datafusion/sql/src/unparser/rewrite.rs index 2529385849e0..0f5cf5abe64b 100644 --- a/datafusion/sql/src/unparser/rewrite.rs +++ b/datafusion/sql/src/unparser/rewrite.rs @@ -24,7 +24,7 @@ use datafusion_common::{ tree_node::{Transformed, TransformedResult, TreeNode}, Result, }; -use datafusion_expr::tree_node::transform_sort_vec; +use datafusion_expr::{expr::Alias, tree_node::transform_sort_vec}; use datafusion_expr::{Expr, LogicalPlan, Projection, Sort, SortExpr}; use sqlparser::ast::Ident; @@ -257,6 +257,40 @@ pub(super) fn subquery_alias_inner_query_and_columns( (outer_projections.input.as_ref(), columns) } +/// Injects column aliases into the projection of a logical plan by wrapping `Expr::Column` expressions +/// with `Expr::Alias` using the provided list of aliases. Non-column expressions are left unchanged. +/// +/// Example: +/// - `SELECT col1, col2 FROM table` with aliases `["alias_1", "some_alias_2"]` will be transformed to +/// - `SELECT col1 AS alias_1, col2 AS some_alias_2 FROM table` +pub(super) fn inject_column_aliases( + projection: &datafusion_expr::Projection, + aliases: impl IntoIterator, +) -> LogicalPlan { + let mut updated_projection = projection.clone(); + + let new_exprs = updated_projection + .expr + .into_iter() + .zip(aliases) + .map(|(expr, col_alias)| match expr { + Expr::Column(col) => { + let relation = col.relation.clone(); + Expr::Alias(Alias { + expr: Box::new(Expr::Column(col)), + relation, + name: col_alias.value, + }) + } + _ => expr, + }) + .collect::>(); + + updated_projection.expr = new_exprs; + + LogicalPlan::Projection(updated_projection) +} + fn find_projection(logical_plan: &LogicalPlan) -> Option<&Projection> { match logical_plan { LogicalPlan::Projection(p) => Some(p), diff --git a/datafusion/sql/tests/cases/plan_to_sql.rs b/datafusion/sql/tests/cases/plan_to_sql.rs index d4e189f5f66a..4a6923593fc9 100644 --- a/datafusion/sql/tests/cases/plan_to_sql.rs +++ b/datafusion/sql/tests/cases/plan_to_sql.rs @@ -25,7 +25,7 @@ use datafusion_expr::{col, table_scan}; use datafusion_sql::planner::{ContextProvider, PlannerContext, SqlToRel}; use datafusion_sql::unparser::dialect::{ DefaultDialect as UnparserDefaultDialect, Dialect as UnparserDialect, - MySqlDialect as UnparserMySqlDialect, + MySqlDialect as UnparserMySqlDialect, SqliteDialect, }; use datafusion_sql::unparser::{expr_to_sql, plan_to_sql, Unparser}; @@ -406,7 +406,19 @@ fn roundtrip_statement_with_dialect() -> Result<()> { expected: r#"SELECT c.id FROM (SELECT (CAST(j1.j1_id AS BIGINT) + 1) FROM j1 ORDER BY j1.j1_id ASC NULLS LAST LIMIT 1) AS c (id)"#, parser_dialect: Box::new(GenericDialect {}), unparser_dialect: Box::new(UnparserDefaultDialect {}), - } + }, + TestStatementWithDialect { + sql: "SELECT temp_j.id2 FROM (SELECT j1_id, j1_string FROM j1) AS temp_j(id2, string2)", + expected: r#"SELECT temp_j.id2 FROM (SELECT j1.j1_id, j1.j1_string FROM j1) AS temp_j (id2, string2)"#, + parser_dialect: Box::new(GenericDialect {}), + unparser_dialect: Box::new(UnparserDefaultDialect {}), + }, + TestStatementWithDialect { + sql: "SELECT temp_j.id2 FROM (SELECT j1_id, j1_string FROM j1) AS temp_j(id2, string2)", + expected: r#"SELECT `temp_j`.`id2` FROM (SELECT `j1`.`j1_id` AS `id2`, `j1`.`j1_string` AS `string2` FROM `j1`) AS `temp_j`"#, + parser_dialect: Box::new(GenericDialect {}), + unparser_dialect: Box::new(SqliteDialect {}), + }, ]; for query in tests { From 5756f39d97c2dbedbf8323e54c35514604eeb8e2 Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Fri, 6 Sep 2024 22:36:35 +0200 Subject: [PATCH 107/154] Remove deprecated ScalarValue::get_datatype (#12361) It's deprecated since v 31. --- datafusion/common/src/scalar/mod.rs | 8 -------- 1 file changed, 8 deletions(-) diff --git a/datafusion/common/src/scalar/mod.rs b/datafusion/common/src/scalar/mod.rs index 0cb325e0b02b..c98a4428a81b 100644 --- a/datafusion/common/src/scalar/mod.rs +++ b/datafusion/common/src/scalar/mod.rs @@ -1321,14 +1321,6 @@ impl ScalarValue { } } - /// Getter for the `DataType` of the value. - /// - /// Suggest using [`Self::data_type`] as a more standard API - #[deprecated(since = "31.0.0", note = "use data_type instead")] - pub fn get_datatype(&self) -> DataType { - self.data_type() - } - /// Calculate arithmetic negation for a scalar value pub fn arithmetic_negate(&self) -> Result { fn neg_checked_with_ctx( From 7ead2ad9588636001022e357ffb7adb80ba1a542 Mon Sep 17 00:00:00 2001 From: kf zheng <100595273+Kev1n8@users.noreply.github.com> Date: Sat, 7 Sep 2024 04:39:58 +0800 Subject: [PATCH 108/154] Improve StringView support for SUBSTR (#12044) * operate stringview instead of generating string in SUBSTR * treat Utf8View as Text in sqllogictests output * add bench to see enhancement of utf8view against utf8 and large_utf8 * fix a tiny bug * make clippy happy * add tests to cover stringview larger than 12B and correct the code * better comments * fix lint * correct feature setting * avoid expensive utf8 and some other checks * fix lint * remove unnecessary indirection * add optimized_utf8_to_str_type * Simplify type check * Use ByteView * update datafusion-cli.lock * Remove duration override * format toml * refactor the code, using append_view_u128 from arrow * manually collect the views and nulls * remove bench file and fix some comments * fix tiny mistake * Update Cargo.lock --------- Co-authored-by: Andrew Lamb --- datafusion/functions/src/unicode/substr.rs | 242 ++++++++++++++++++--- 1 file changed, 216 insertions(+), 26 deletions(-) diff --git a/datafusion/functions/src/unicode/substr.rs b/datafusion/functions/src/unicode/substr.rs index db218f9127ae..8a70b380669c 100644 --- a/datafusion/functions/src/unicode/substr.rs +++ b/datafusion/functions/src/unicode/substr.rs @@ -19,18 +19,18 @@ use std::any::Any; use std::cmp::max; use std::sync::Arc; +use crate::utils::{make_scalar_function, utf8_to_str_type}; use arrow::array::{ - ArrayAccessor, ArrayIter, ArrayRef, AsArray, GenericStringArray, OffsetSizeTrait, + make_view, Array, ArrayAccessor, ArrayIter, ArrayRef, AsArray, ByteView, + GenericStringArray, OffsetSizeTrait, StringViewArray, }; use arrow::datatypes::DataType; - +use arrow_buffer::{NullBufferBuilder, ScalarBuffer}; use datafusion_common::cast::as_int64_array; use datafusion_common::{exec_datafusion_err, exec_err, Result}; use datafusion_expr::TypeSignature::Exact; use datafusion_expr::{ColumnarValue, ScalarUDFImpl, Signature, Volatility}; -use crate::utils::{make_scalar_function, utf8_to_str_type}; - #[derive(Debug)] pub struct SubstrFunc { signature: Signature, @@ -77,7 +77,11 @@ impl ScalarUDFImpl for SubstrFunc { } fn return_type(&self, arg_types: &[DataType]) -> Result { - utf8_to_str_type(&arg_types[0], "substr") + if arg_types[0] == DataType::Utf8View { + Ok(DataType::Utf8View) + } else { + utf8_to_str_type(&arg_types[0], "substr") + } } fn invoke(&self, args: &[ColumnarValue]) -> Result { @@ -89,29 +93,188 @@ impl ScalarUDFImpl for SubstrFunc { } } +/// Extracts the substring of string starting at the start'th character, and extending for count characters if that is specified. (Same as substring(string from start for count).) +/// substr('alphabet', 3) = 'phabet' +/// substr('alphabet', 3, 2) = 'ph' +/// The implementation uses UTF-8 code points as characters pub fn substr(args: &[ArrayRef]) -> Result { match args[0].data_type() { DataType::Utf8 => { let string_array = args[0].as_string::(); - calculate_substr::<_, i32>(string_array, &args[1..]) + string_substr::<_, i32>(string_array, &args[1..]) } DataType::LargeUtf8 => { let string_array = args[0].as_string::(); - calculate_substr::<_, i64>(string_array, &args[1..]) + string_substr::<_, i64>(string_array, &args[1..]) } DataType::Utf8View => { let string_array = args[0].as_string_view(); - calculate_substr::<_, i32>(string_array, &args[1..]) + string_view_substr(string_array, &args[1..]) } - other => exec_err!("Unsupported data type {other:?} for function substr"), + other => exec_err!( + "Unsupported data type {other:?} for function substr,\ + expected Utf8View, Utf8 or LargeUtf8." + ), } } -/// Extracts the substring of string starting at the start'th character, and extending for count characters if that is specified. (Same as substring(string from start for count).) -/// substr('alphabet', 3) = 'phabet' -/// substr('alphabet', 3, 2) = 'ph' -/// The implementation uses UTF-8 code points as characters -fn calculate_substr<'a, V, T>(string_array: V, args: &[ArrayRef]) -> Result +// Return the exact byte index for [start, end), set count to -1 to ignore count +fn get_true_start_end(input: &str, start: usize, count: i64) -> (usize, usize) { + let (mut st, mut ed) = (input.len(), input.len()); + let mut start_counting = false; + let mut cnt = 0; + for (char_cnt, (byte_cnt, _)) in input.char_indices().enumerate() { + if char_cnt == start { + st = byte_cnt; + if count != -1 { + start_counting = true; + } else { + break; + } + } + if start_counting { + if cnt == count { + ed = byte_cnt; + break; + } + cnt += 1; + } + } + (st, ed) +} + +/// Make a `u128` based on the given substr, start(offset to view.offset), and +/// push into to the given buffers +fn make_and_append_view( + views_buffer: &mut Vec, + null_builder: &mut NullBufferBuilder, + raw: &u128, + substr: &str, + start: u32, +) { + let substr_len = substr.len(); + if substr_len == 0 { + null_builder.append_null(); + views_buffer.push(0); + } else { + let sub_view = if substr_len > 12 { + let view = ByteView::from(*raw); + make_view(substr.as_bytes(), view.buffer_index, view.offset + start) + } else { + // inline value does not need block id or offset + make_view(substr.as_bytes(), 0, 0) + }; + views_buffer.push(sub_view); + null_builder.append_non_null(); + } +} + +// The decoding process refs the trait at: arrow/arrow-data/src/byte_view.rs:44 +// From for ByteView +fn string_view_substr( + string_view_array: &StringViewArray, + args: &[ArrayRef], +) -> Result { + let mut views_buf = Vec::with_capacity(string_view_array.len()); + let mut null_builder = NullBufferBuilder::new(string_view_array.len()); + + let start_array = as_int64_array(&args[0])?; + + match args.len() { + 1 => { + for (idx, (raw, start)) in string_view_array + .views() + .iter() + .zip(start_array.iter()) + .enumerate() + { + if let Some(start) = start { + let start = (start - 1).max(0) as usize; + + // Safety: + // idx is always smaller or equal to string_view_array.views.len() + unsafe { + let str = string_view_array.value_unchecked(idx); + let (start, end) = get_true_start_end(str, start, -1); + let substr = &str[start..end]; + + make_and_append_view( + &mut views_buf, + &mut null_builder, + raw, + substr, + start as u32, + ); + } + } else { + null_builder.append_null(); + views_buf.push(0); + } + } + } + 2 => { + let count_array = as_int64_array(&args[1])?; + for (idx, ((raw, start), count)) in string_view_array + .views() + .iter() + .zip(start_array.iter()) + .zip(count_array.iter()) + .enumerate() + { + if let (Some(start), Some(count)) = (start, count) { + let start = (start - 1).max(0) as usize; + if count < 0 { + return exec_err!( + "negative substring length not allowed: substr(, {start}, {count})" + ); + } else { + // Safety: + // idx is always smaller or equal to string_view_array.views.len() + unsafe { + let str = string_view_array.value_unchecked(idx); + let (start, end) = get_true_start_end(str, start, count); + let substr = &str[start..end]; + + make_and_append_view( + &mut views_buf, + &mut null_builder, + raw, + substr, + start as u32, + ); + } + } + } else { + null_builder.append_null(); + views_buf.push(0); + } + } + } + other => { + return exec_err!( + "substr was called with {other} arguments. It requires 2 or 3." + ) + } + } + + let views_buf = ScalarBuffer::from(views_buf); + let nulls_buf = null_builder.finish(); + + // Safety: + // (1) The blocks of the given views are all provided + // (2) Each of the range `view.offset+start..end` of view in views_buf is within + // the bounds of each of the blocks + unsafe { + let array = StringViewArray::new_unchecked( + views_buf, + string_view_array.data_buffers().to_vec(), + nulls_buf, + ); + Ok(Arc::new(array) as ArrayRef) + } +} + +fn string_substr<'a, V, T>(string_array: V, args: &[ArrayRef]) -> Result where V: ArrayAccessor, T: OffsetSizeTrait, @@ -174,8 +337,8 @@ where #[cfg(test)] mod tests { - use arrow::array::{Array, StringArray}; - use arrow::datatypes::DataType::Utf8; + use arrow::array::{Array, StringArray, StringViewArray}; + use arrow::datatypes::DataType::{Utf8, Utf8View}; use datafusion_common::{exec_err, Result, ScalarValue}; use datafusion_expr::{ColumnarValue, ScalarUDFImpl}; @@ -193,8 +356,8 @@ mod tests { ], Ok(None), &str, - Utf8, - StringArray + Utf8View, + StringViewArray ); test_function!( SubstrFunc::new(), @@ -206,8 +369,35 @@ mod tests { ], Ok(Some("alphabet")), &str, - Utf8, - StringArray + Utf8View, + StringViewArray + ); + test_function!( + SubstrFunc::new(), + &[ + ColumnarValue::Scalar(ScalarValue::Utf8View(Some(String::from( + "this és longer than 12B" + )))), + ColumnarValue::Scalar(ScalarValue::from(5i64)), + ColumnarValue::Scalar(ScalarValue::from(2i64)), + ], + Ok(Some(" é")), + &str, + Utf8View, + StringViewArray + ); + test_function!( + SubstrFunc::new(), + &[ + ColumnarValue::Scalar(ScalarValue::Utf8View(Some(String::from( + "this is longer than 12B" + )))), + ColumnarValue::Scalar(ScalarValue::from(5i64)), + ], + Ok(Some(" is longer than 12B")), + &str, + Utf8View, + StringViewArray ); test_function!( SubstrFunc::new(), @@ -219,8 +409,8 @@ mod tests { ], Ok(Some("ésoj")), &str, - Utf8, - StringArray + Utf8View, + StringViewArray ); test_function!( SubstrFunc::new(), @@ -233,8 +423,8 @@ mod tests { ], Ok(Some("ph")), &str, - Utf8, - StringArray + Utf8View, + StringViewArray ); test_function!( SubstrFunc::new(), @@ -247,8 +437,8 @@ mod tests { ], Ok(Some("phabet")), &str, - Utf8, - StringArray + Utf8View, + StringViewArray ); test_function!( SubstrFunc::new(), From 1a44ad2408dfe09ce62acf23d7743c79b17cf469 Mon Sep 17 00:00:00 2001 From: Martin Kolb Date: Fri, 6 Sep 2024 22:41:07 +0200 Subject: [PATCH 109/154] Fix issue with "to_date" failing to process dates later than year 2262 (#12227) * Fix to_date: Support date values greater than year 2262 Due to using nanoseconds as an intermediate state, the values for dates processed via "to_date" cannot be later than the year 2262. The Arrow datatype for Date32 and Date64 supports much larger values. The statements in some areas in the code that the usage of nanoseconds is imposed by Arrow is simply wrong. The Date32 type stores the number of days since epoch. The Date64 type stores the milliseconds (NOT nanoseconds) since epoch. Both Date32 and Date64 can therefore massively exceed the year 2262. See: https://arrow.apache.org/docs/cpp/api/datatype.html * Improved checks in unit tests * to_date: Enable handling of nagative years and add more tests * to_date: Fix test failures and update doc in MarkDown * to_date: Revert special handling for negative years, focus on years up to 9999 * to_date: Fix clippy warnings * Move string_to_timestamp_millis_formatted to common.rs --------- Co-authored-by: Andrew Lamb --- datafusion/functions/src/datetime/common.rs | 25 ++ datafusion/functions/src/datetime/to_date.rs | 233 +++++++++++++++++- datafusion/sqllogictest/test_files/dates.slt | 10 +- .../source/user-guide/sql/scalar_functions.md | 4 +- 4 files changed, 256 insertions(+), 16 deletions(-) diff --git a/datafusion/functions/src/datetime/common.rs b/datafusion/functions/src/datetime/common.rs index 6048eeeaa554..89b40a3534d3 100644 --- a/datafusion/functions/src/datetime/common.rs +++ b/datafusion/functions/src/datetime/common.rs @@ -153,6 +153,31 @@ pub(crate) fn string_to_timestamp_nanos_formatted( }) } +/// Accepts a string with a `chrono` format and converts it to a +/// millisecond precision timestamp. +/// +/// See [`chrono::format::strftime`] for the full set of supported formats. +/// +/// Internally, this function uses the `chrono` library for the +/// datetime parsing +/// +/// ## Timezone / Offset Handling +/// +/// Numerical values of timestamps are stored compared to offset UTC. +/// +/// Any timestamp in the formatting string is handled according to the rules +/// defined by `chrono`. +/// +/// [`chrono::format::strftime`]: https://docs.rs/chrono/latest/chrono/format/strftime/index.html +/// +#[inline] +pub(crate) fn string_to_timestamp_millis_formatted(s: &str, format: &str) -> Result { + Ok(string_to_datetime_formatted(&Utc, s, format)? + .naive_utc() + .and_utc() + .timestamp_millis()) +} + pub(crate) fn handle<'a, O, F, S>( args: &'a [ColumnarValue], op: F, diff --git a/datafusion/functions/src/datetime/to_date.rs b/datafusion/functions/src/datetime/to_date.rs index cc5ffa73c8f1..288641b84dd7 100644 --- a/datafusion/functions/src/datetime/to_date.rs +++ b/datafusion/functions/src/datetime/to_date.rs @@ -17,12 +17,14 @@ use std::any::Any; -use arrow::array::types::Date32Type; use arrow::datatypes::DataType; use arrow::datatypes::DataType::Date32; +use arrow::error::ArrowError::ParseError; +use arrow::{array::types::Date32Type, compute::kernels::cast_utils::Parser}; use crate::datetime::common::*; -use datafusion_common::{exec_err, internal_datafusion_err, Result}; +use datafusion_common::error::DataFusionError; +use datafusion_common::{arrow_err, exec_err, internal_datafusion_err, Result}; use datafusion_expr::{ColumnarValue, ScalarUDFImpl, Signature, Volatility}; #[derive(Debug)] @@ -47,22 +49,20 @@ impl ToDateFunc { match args.len() { 1 => handle::( args, - |s| { - string_to_timestamp_nanos_shim(s) - .map(|n| n / (1_000_000 * 24 * 60 * 60 * 1_000)) - .and_then(|v| { - v.try_into().map_err(|_| { - internal_datafusion_err!("Unable to cast to Date32 for converting from i64 to i32 failed") - }) - }) + |s| match Date32Type::parse(s) { + Some(v) => Ok(v), + None => arrow_err!(ParseError( + "Unable to cast to Date32 for converting from i64 to i32 failed" + .to_string() + )), }, "to_date", ), 2.. => handle_multiple::( args, |s, format| { - string_to_timestamp_nanos_formatted(s, format) - .map(|n| n / (1_000_000 * 24 * 60 * 60 * 1_000)) + string_to_timestamp_millis_formatted(s, format) + .map(|n| n / (24 * 60 * 60 * 1_000)) .and_then(|v| { v.try_into().map_err(|_| { internal_datafusion_err!("Unable to cast to Date32 for converting from i64 to i32 failed") @@ -118,3 +118,212 @@ impl ScalarUDFImpl for ToDateFunc { } } } + +#[cfg(test)] +mod tests { + use arrow::{compute::kernels::cast_utils::Parser, datatypes::Date32Type}; + use datafusion_common::ScalarValue; + use datafusion_expr::{ColumnarValue, ScalarUDFImpl}; + + use super::ToDateFunc; + + #[test] + fn test_to_date_without_format() { + struct TestCase { + name: &'static str, + date_str: &'static str, + } + + let test_cases = vec![ + TestCase { + name: "Largest four-digit year (9999)", + date_str: "9999-12-31", + }, + TestCase { + name: "Year 1 (0001)", + date_str: "0001-12-31", + }, + TestCase { + name: "Year before epoch (1969)", + date_str: "1969-01-01", + }, + TestCase { + name: "Switch Julian/Gregorian calendar (1582-10-10)", + date_str: "1582-10-10", + }, + ]; + + for tc in &test_cases { + let date_scalar = ScalarValue::Utf8(Some(tc.date_str.to_string())); + let to_date_result = + ToDateFunc::new().invoke(&[ColumnarValue::Scalar(date_scalar)]); + + match to_date_result { + Ok(ColumnarValue::Scalar(ScalarValue::Date32(date_val))) => { + let expected = Date32Type::parse_formatted(tc.date_str, "%Y-%m-%d"); + assert_eq!( + date_val, expected, + "{}: to_date created wrong value", + tc.name + ); + } + _ => panic!("Could not convert '{}' to Date", tc.date_str), + } + } + } + + #[test] + fn test_to_date_with_format() { + struct TestCase { + name: &'static str, + date_str: &'static str, + format_str: &'static str, + formatted_date: &'static str, + } + + let test_cases = vec![ + TestCase { + name: "Largest four-digit year (9999)", + date_str: "9999-12-31", + format_str: "%Y%m%d", + formatted_date: "99991231", + }, + TestCase { + name: "Smallest four-digit year (-9999)", + date_str: "-9999-12-31", + format_str: "%Y/%m/%d", + formatted_date: "-9999/12/31", + }, + TestCase { + name: "Year 1 (0001)", + date_str: "0001-12-31", + format_str: "%Y%m%d", + formatted_date: "00011231", + }, + TestCase { + name: "Year before epoch (1969)", + date_str: "1969-01-01", + format_str: "%Y%m%d", + formatted_date: "19690101", + }, + TestCase { + name: "Switch Julian/Gregorian calendar (1582-10-10)", + date_str: "1582-10-10", + format_str: "%Y%m%d", + formatted_date: "15821010", + }, + TestCase { + name: "Negative Year, BC (-42-01-01)", + date_str: "-42-01-01", + format_str: "%Y/%m/%d", + formatted_date: "-42/01/01", + }, + ]; + + for tc in &test_cases { + let formatted_date_scalar = + ScalarValue::Utf8(Some(tc.formatted_date.to_string())); + let format_scalar = ScalarValue::Utf8(Some(tc.format_str.to_string())); + + let to_date_result = ToDateFunc::new().invoke(&[ + ColumnarValue::Scalar(formatted_date_scalar), + ColumnarValue::Scalar(format_scalar), + ]); + + match to_date_result { + Ok(ColumnarValue::Scalar(ScalarValue::Date32(date_val))) => { + let expected = Date32Type::parse_formatted(tc.date_str, "%Y-%m-%d"); + assert_eq!(date_val, expected, "{}: to_date created wrong value for date '{}' with format string '{}'", tc.name, tc.formatted_date, tc.format_str); + } + _ => panic!( + "Could not convert '{}' with format string '{}'to Date", + tc.date_str, tc.format_str + ), + } + } + } + + #[test] + fn test_to_date_multiple_format_strings() { + let formatted_date_scalar = ScalarValue::Utf8(Some("2023/01/31".into())); + let format1_scalar = ScalarValue::Utf8(Some("%Y-%m-%d".into())); + let format2_scalar = ScalarValue::Utf8(Some("%Y/%m/%d".into())); + + let to_date_result = ToDateFunc::new().invoke(&[ + ColumnarValue::Scalar(formatted_date_scalar), + ColumnarValue::Scalar(format1_scalar), + ColumnarValue::Scalar(format2_scalar), + ]); + + match to_date_result { + Ok(ColumnarValue::Scalar(ScalarValue::Date32(date_val))) => { + let expected = Date32Type::parse_formatted("2023-01-31", "%Y-%m-%d"); + assert_eq!( + date_val, expected, + "to_date created wrong value for date with 2 format strings" + ); + } + _ => panic!("Conversion failed",), + } + } + + #[test] + fn test_to_date_from_timestamp() { + let test_cases = vec![ + "2020-09-08T13:42:29Z", + "2020-09-08T13:42:29.190855-05:00", + "2020-09-08 12:13:29", + ]; + for date_str in test_cases { + let formatted_date_scalar = ScalarValue::Utf8(Some(date_str.into())); + + let to_date_result = + ToDateFunc::new().invoke(&[ColumnarValue::Scalar(formatted_date_scalar)]); + + match to_date_result { + Ok(ColumnarValue::Scalar(ScalarValue::Date32(date_val))) => { + let expected = Date32Type::parse_formatted("2020-09-08", "%Y-%m-%d"); + assert_eq!(date_val, expected, "to_date created wrong value"); + } + _ => panic!("Conversion of {} failed", date_str), + } + } + } + + #[test] + fn test_to_date_string_with_valid_number() { + let date_str = "20241231"; + let date_scalar = ScalarValue::Utf8(Some(date_str.into())); + + let to_date_result = + ToDateFunc::new().invoke(&[ColumnarValue::Scalar(date_scalar)]); + + match to_date_result { + Ok(ColumnarValue::Scalar(ScalarValue::Date32(date_val))) => { + let expected = Date32Type::parse_formatted("2024-12-31", "%Y-%m-%d"); + assert_eq!( + date_val, expected, + "to_date created wrong value for {}", + date_str + ); + } + _ => panic!("Conversion of {} failed", date_str), + } + } + + #[test] + fn test_to_date_string_with_invalid_number() { + let date_str = "202412311"; + let date_scalar = ScalarValue::Utf8(Some(date_str.into())); + + let to_date_result = + ToDateFunc::new().invoke(&[ColumnarValue::Scalar(date_scalar)]); + + if let Ok(ColumnarValue::Scalar(ScalarValue::Date32(_))) = to_date_result { + panic!( + "Conversion of {} succeded, but should have failed, ", + date_str + ); + } + } +} diff --git a/datafusion/sqllogictest/test_files/dates.slt b/datafusion/sqllogictest/test_files/dates.slt index e21637bd8913..3950a165a004 100644 --- a/datafusion/sqllogictest/test_files/dates.slt +++ b/datafusion/sqllogictest/test_files/dates.slt @@ -123,6 +123,7 @@ SELECT to_date(ts / 100000000) FROM to_date_t1 LIMIT 3 2003-11-02 2003-11-29 +# verify date with time zone, where the time zone date is already the next day, but result date in UTC is day before query D SELECT to_date('01-14-2023 01:01:30+05:30', '%q', '%d-%m-%Y %H/%M/%S', '%+', '%m-%d-%Y %H:%M:%S%#z'); ---- @@ -137,8 +138,15 @@ select to_date(arrow_cast(123, 'Int64')) ---- 1970-05-04 -statement error DataFusion error: Arrow error: +# Parse sequence of digits which yield a valid date, e.g. "21311111" would be "2131-11-11" +query D SELECT to_date('21311111'); +---- +2131-11-11 + +# Parse sequence of digits which do not make up a valid date +statement error DataFusion error: Arrow error: +SELECT to_date('213111111'); # verify date cast with integer input query DDDDDD diff --git a/docs/source/user-guide/sql/scalar_functions.md b/docs/source/user-guide/sql/scalar_functions.md index 80b61f8242ef..c3d3ab7a64a7 100644 --- a/docs/source/user-guide/sql/scalar_functions.md +++ b/docs/source/user-guide/sql/scalar_functions.md @@ -1735,9 +1735,7 @@ Strings are parsed as YYYY-MM-DD (e.g. '2023-07-20') if no [Chrono format]s are Integers and doubles are interpreted as days since the unix epoch (`1970-01-01T00:00:00Z`). Returns the corresponding date. -Note: `to_date` returns Date32. The supported range for integer input is between `-96465293` and `95026237`. -Supported range for string input is between `1677-09-21` and `2262-04-11` exclusive. To parse dates outside of -that range use a [Chrono format]. +Note: `to_date` returns Date32, which represents its values as the number of days since unix epoch(`1970-01-01`) stored as signed 32 bit value. The largest supported date value is `9999-12-31`. ``` to_date(expression[, ..., format_n]) From 923f098cab00933d7b09fe57343b38d3cda6d9a8 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Fri, 6 Sep 2024 17:15:52 -0400 Subject: [PATCH 110/154] Minor: improve performance of `ScalarValue::Binary*` debug (#12323) * Minor: improve performance of ScalarValue debug * reduce allocations * simplify loop * Add test for single list * Add another test for display --- datafusion/common/src/scalar/mod.rs | 80 +++++++++++++++++++++++++---- 1 file changed, 69 insertions(+), 11 deletions(-) diff --git a/datafusion/common/src/scalar/mod.rs b/datafusion/common/src/scalar/mod.rs index c98a4428a81b..5cec2e56bd00 100644 --- a/datafusion/common/src/scalar/mod.rs +++ b/datafusion/common/src/scalar/mod.rs @@ -3653,18 +3653,20 @@ fn fmt_list(arr: ArrayRef, f: &mut fmt::Formatter) -> fmt::Result { write!(f, "{value_formatter}") } +/// writes a byte array to formatter. `[1, 2, 3]` ==> `"1,2,3"` +fn fmt_binary(data: &[u8], f: &mut fmt::Formatter) -> fmt::Result { + let mut iter = data.iter(); + if let Some(b) = iter.next() { + write!(f, "{b}")?; + } + for b in iter { + write!(f, ",{b}")?; + } + Ok(()) +} + impl fmt::Debug for ScalarValue { fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { - fn fmt_binary(data: &[u8], f: &mut fmt::Formatter) -> fmt::Result { - write!( - f, - "{}", - data.iter() - .map(|v| format!("{v}")) - .collect::>() - .join(",") - ) - } match self { ScalarValue::Decimal128(_, _, _) => write!(f, "Decimal128({self})"), ScalarValue::Decimal256(_, _, _) => write!(f, "Decimal256({self})"), @@ -3714,7 +3716,7 @@ impl fmt::Debug for ScalarValue { write!(f, "FixedSizeBinary({size}, {self})") } ScalarValue::FixedSizeBinary(size, Some(b)) => { - write!(f, "Binary({size}, \"")?; + write!(f, "FixedSizeBinary({size}, \"")?; fmt_binary(b.as_slice(), f)?; write!(f, "\")") } @@ -6659,6 +6661,8 @@ mod tests { fn test_binary_display() { let no_binary_value = ScalarValue::Binary(None); assert_eq!(format!("{no_binary_value}"), "NULL"); + let single_binary_value = ScalarValue::Binary(Some(vec![42u8])); + assert_eq!(format!("{single_binary_value}"), "2A"); let small_binary_value = ScalarValue::Binary(Some(vec![1u8, 2, 3])); assert_eq!(format!("{small_binary_value}"), "010203"); let large_binary_value = @@ -6692,6 +6696,60 @@ mod tests { assert_eq!(format!("{large_binary_value}"), "0102030405060708090A..."); } + #[test] + fn test_binary_debug() { + let no_binary_value = ScalarValue::Binary(None); + assert_eq!(format!("{no_binary_value:?}"), "Binary(NULL)"); + let single_binary_value = ScalarValue::Binary(Some(vec![42u8])); + assert_eq!(format!("{single_binary_value:?}"), "Binary(\"42\")"); + let small_binary_value = ScalarValue::Binary(Some(vec![1u8, 2, 3])); + assert_eq!(format!("{small_binary_value:?}"), "Binary(\"1,2,3\")"); + let large_binary_value = + ScalarValue::Binary(Some(vec![1u8, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11])); + assert_eq!( + format!("{large_binary_value:?}"), + "Binary(\"1,2,3,4,5,6,7,8,9,10,11\")" + ); + + let no_binary_value = ScalarValue::BinaryView(None); + assert_eq!(format!("{no_binary_value:?}"), "BinaryView(NULL)"); + let small_binary_value = ScalarValue::BinaryView(Some(vec![1u8, 2, 3])); + assert_eq!(format!("{small_binary_value:?}"), "BinaryView(\"1,2,3\")"); + let large_binary_value = + ScalarValue::BinaryView(Some(vec![1u8, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11])); + assert_eq!( + format!("{large_binary_value:?}"), + "BinaryView(\"1,2,3,4,5,6,7,8,9,10,11\")" + ); + + let no_binary_value = ScalarValue::LargeBinary(None); + assert_eq!(format!("{no_binary_value:?}"), "LargeBinary(NULL)"); + let small_binary_value = ScalarValue::LargeBinary(Some(vec![1u8, 2, 3])); + assert_eq!(format!("{small_binary_value:?}"), "LargeBinary(\"1,2,3\")"); + let large_binary_value = + ScalarValue::LargeBinary(Some(vec![1u8, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11])); + assert_eq!( + format!("{large_binary_value:?}"), + "LargeBinary(\"1,2,3,4,5,6,7,8,9,10,11\")" + ); + + let no_binary_value = ScalarValue::FixedSizeBinary(3, None); + assert_eq!(format!("{no_binary_value:?}"), "FixedSizeBinary(3, NULL)"); + let small_binary_value = ScalarValue::FixedSizeBinary(3, Some(vec![1u8, 2, 3])); + assert_eq!( + format!("{small_binary_value:?}"), + "FixedSizeBinary(3, \"1,2,3\")" + ); + let large_binary_value = ScalarValue::FixedSizeBinary( + 11, + Some(vec![1u8, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11]), + ); + assert_eq!( + format!("{large_binary_value:?}"), + "FixedSizeBinary(11, \"1,2,3,4,5,6,7,8,9,10,11\")" + ); + } + #[test] fn test_build_timestamp_millisecond_list() { let values = vec![ScalarValue::TimestampMillisecond(Some(1), None)]; From f2bbf25cfbfc04719afe8d7a8342c20e4d595648 Mon Sep 17 00:00:00 2001 From: Arttu Date: Sat, 7 Sep 2024 00:17:40 +0200 Subject: [PATCH 111/154] fix: support Substrait VirtualTables with no columns (#12339) * add a test case for a virtual table w/o columns should fail with: ArrowError(InvalidArgumentError("must either specify a row count or at least one column"), None) * fix: feed Arrow's RecordBatch builder the row count * use a roundtrip test instead * fix clippy --- datafusion/physical-plan/src/values.rs | 6 +++- .../tests/cases/roundtrip_logical_plan.rs | 32 +++++++++++++++---- 2 files changed, 30 insertions(+), 8 deletions(-) diff --git a/datafusion/physical-plan/src/values.rs b/datafusion/physical-plan/src/values.rs index 3ea27d62d80b..e01aea1fdd6b 100644 --- a/datafusion/physical-plan/src/values.rs +++ b/datafusion/physical-plan/src/values.rs @@ -88,7 +88,11 @@ impl ValuesExec { .and_then(ScalarValue::iter_to_array) }) .collect::>>()?; - let batch = RecordBatch::try_new(Arc::clone(&schema), arr)?; + let batch = RecordBatch::try_new_with_options( + Arc::clone(&schema), + arr, + &RecordBatchOptions::new().with_row_count(Some(n_row)), + )?; let data: Vec = vec![batch]; Self::try_new_from_batches(schema, data) } diff --git a/datafusion/substrait/tests/cases/roundtrip_logical_plan.rs b/datafusion/substrait/tests/cases/roundtrip_logical_plan.rs index 98daac65e1cf..cc353ab36d97 100644 --- a/datafusion/substrait/tests/cases/roundtrip_logical_plan.rs +++ b/datafusion/substrait/tests/cases/roundtrip_logical_plan.rs @@ -22,9 +22,6 @@ use datafusion_substrait::logical_plan::{ consumer::from_substrait_plan, producer::to_substrait_plan, }; -use std::hash::Hash; -use std::sync::Arc; - use datafusion::arrow::datatypes::{DataType, Field, IntervalUnit, Schema, TimeUnit}; use datafusion::common::{not_impl_err, plan_err, DFSchema, DFSchemaRef}; use datafusion::error::Result; @@ -32,10 +29,12 @@ use datafusion::execution::registry::SerializerRegistry; use datafusion::execution::runtime_env::RuntimeEnv; use datafusion::logical_expr::{ Extension, LogicalPlan, PartitionEvaluator, Repartition, UserDefinedLogicalNode, - Volatility, + Values, Volatility, }; use datafusion::optimizer::simplify_expressions::expr_simplifier::THRESHOLD_INLINE_INLIST; use datafusion::prelude::*; +use std::hash::Hash; +use std::sync::Arc; use datafusion::execution::session_state::SessionStateBuilder; use substrait::proto::extensions::simple_extension_declaration::{ @@ -767,6 +766,18 @@ async fn roundtrip_values() -> Result<()> { true).await } +#[tokio::test] +async fn roundtrip_values_no_columns() -> Result<()> { + let ctx = create_context().await?; + // "VALUES ()" is not yet supported by the SQL parser, so we construct the plan manually + let plan = LogicalPlan::Values(Values { + values: vec![vec![], vec![]], // two rows, no columns + schema: DFSchemaRef::new(DFSchema::empty()), + }); + roundtrip_logical_plan_with_ctx(plan, ctx).await?; + Ok(()) +} + #[tokio::test] async fn roundtrip_values_empty_relation() -> Result<()> { roundtrip("SELECT * FROM (VALUES ('a')) LIMIT 0").await @@ -1121,9 +1132,10 @@ async fn test_alias(sql_with_alias: &str, sql_no_alias: &str) -> Result<()> { Ok(()) } -async fn roundtrip_with_ctx(sql: &str, ctx: SessionContext) -> Result> { - let df = ctx.sql(sql).await?; - let plan = df.into_optimized_plan()?; +async fn roundtrip_logical_plan_with_ctx( + plan: LogicalPlan, + ctx: SessionContext, +) -> Result> { let proto = to_substrait_plan(&plan, &ctx)?; let plan2 = from_substrait_plan(&ctx, &proto).await?; let plan2 = ctx.state().optimize(&plan2)?; @@ -1143,6 +1155,12 @@ async fn roundtrip_with_ctx(sql: &str, ctx: SessionContext) -> Result> Ok(proto) } +async fn roundtrip_with_ctx(sql: &str, ctx: SessionContext) -> Result> { + let df = ctx.sql(sql).await?; + let plan = df.into_optimized_plan()?; + roundtrip_logical_plan_with_ctx(plan, ctx).await +} + async fn roundtrip(sql: &str) -> Result<()> { roundtrip_with_ctx(sql, create_context().await?).await?; Ok(()) From 83ca713c64adfca5e0e15dad74a69c2ab35b0d62 Mon Sep 17 00:00:00 2001 From: Jay Zhan Date: Sat, 7 Sep 2024 13:07:44 +0800 Subject: [PATCH 112/154] Apply non-nested kernel for non-nested in `array_has` and `inlist` (#12164) * use non-nested kernel for non-nested Signed-off-by: jayzhan211 * check is_nested outside of func Signed-off-by: jayzhan211 --------- Signed-off-by: jayzhan211 --- datafusion/functions-nested/src/array_has.rs | 11 ++++++----- datafusion/physical-expr-common/src/datum.rs | 16 +++++++++++++++- .../physical-expr/src/expressions/in_list.rs | 19 +++++++++---------- 3 files changed, 30 insertions(+), 16 deletions(-) diff --git a/datafusion/functions-nested/src/array_has.rs b/datafusion/functions-nested/src/array_has.rs index 9b4357d0d14f..7f66eba5df08 100644 --- a/datafusion/functions-nested/src/array_has.rs +++ b/datafusion/functions-nested/src/array_has.rs @@ -24,9 +24,8 @@ use arrow_array::{Datum, GenericListArray, Scalar}; use datafusion_common::cast::as_generic_list_array; use datafusion_common::utils::string_utils::string_array_to_vec; use datafusion_common::{exec_err, Result, ScalarValue}; -use datafusion_expr::{ColumnarValue, Operator, ScalarUDFImpl, Signature, Volatility}; - -use datafusion_physical_expr_common::datum::compare_op_for_nested; +use datafusion_expr::{ColumnarValue, ScalarUDFImpl, Signature, Volatility}; +use datafusion_physical_expr_common::datum::compare_with_eq; use itertools::Itertools; use crate::utils::make_scalar_function; @@ -180,8 +179,9 @@ fn array_has_dispatch_for_array( continue; } let arr = arr.unwrap(); + let is_nested = arr.data_type().is_nested(); let needle_row = Scalar::new(needle.slice(i, 1)); - let eq_array = compare_op_for_nested(Operator::Eq, &arr, &needle_row)?; + let eq_array = compare_with_eq(&arr, &needle_row, is_nested)?; let is_contained = eq_array.true_count() > 0; boolean_builder.append_value(is_contained) } @@ -195,13 +195,14 @@ fn array_has_dispatch_for_scalar( ) -> Result { let haystack = as_generic_list_array::(haystack)?; let values = haystack.values(); + let is_nested = values.data_type().is_nested(); let offsets = haystack.value_offsets(); // If first argument is empty list (second argument is non-null), return false // i.e. array_has([], non-null element) -> false if values.len() == 0 { return Ok(Arc::new(BooleanArray::from(vec![Some(false)]))); } - let eq_array = compare_op_for_nested(Operator::Eq, values, needle)?; + let eq_array = compare_with_eq(values, needle, is_nested)?; let mut final_contained = vec![None; haystack.len()]; for (i, offset) in offsets.windows(2).enumerate() { let start = offset[0].to_usize().unwrap(); diff --git a/datafusion/physical-expr-common/src/datum.rs b/datafusion/physical-expr-common/src/datum.rs index 96c08d0d3a5b..c47ec9d75d50 100644 --- a/datafusion/physical-expr-common/src/datum.rs +++ b/datafusion/physical-expr-common/src/datum.rs @@ -20,7 +20,8 @@ use arrow::array::{make_comparator, ArrayRef, Datum}; use arrow::buffer::NullBuffer; use arrow::compute::SortOptions; use arrow::error::ArrowError; -use datafusion_common::internal_err; +use datafusion_common::DataFusionError; +use datafusion_common::{arrow_datafusion_err, internal_err}; use datafusion_common::{Result, ScalarValue}; use datafusion_expr_common::columnar_value::ColumnarValue; use datafusion_expr_common::operator::Operator; @@ -87,6 +88,19 @@ pub fn apply_cmp_for_nested( } } +/// Compare with eq with either nested or non-nested +pub fn compare_with_eq( + lhs: &dyn Datum, + rhs: &dyn Datum, + is_nested: bool, +) -> Result { + if is_nested { + compare_op_for_nested(Operator::Eq, lhs, rhs) + } else { + arrow::compute::kernels::cmp::eq(lhs, rhs).map_err(|e| arrow_datafusion_err!(e)) + } +} + /// Compare on nested type List, Struct, and so on pub fn compare_op_for_nested( op: Operator, diff --git a/datafusion/physical-expr/src/expressions/in_list.rs b/datafusion/physical-expr/src/expressions/in_list.rs index dfc70551ccf6..0a3e5fcefcf6 100644 --- a/datafusion/physical-expr/src/expressions/in_list.rs +++ b/datafusion/physical-expr/src/expressions/in_list.rs @@ -40,8 +40,8 @@ use datafusion_common::hash_utils::HashValue; use datafusion_common::{ exec_err, internal_err, not_impl_err, DFSchema, Result, ScalarValue, }; -use datafusion_expr::{ColumnarValue, Operator}; -use datafusion_physical_expr_common::datum::compare_op_for_nested; +use datafusion_expr::ColumnarValue; +use datafusion_physical_expr_common::datum::compare_with_eq; use ahash::RandomState; use hashbrown::hash_map::RawEntryMut; @@ -356,17 +356,16 @@ impl PhysicalExpr for InListExpr { Some(f) => f.contains(value.into_array(num_rows)?.as_ref(), self.negated)?, None => { let value = value.into_array(num_rows)?; + let is_nested = value.data_type().is_nested(); let found = self.list.iter().map(|expr| expr.evaluate(batch)).try_fold( BooleanArray::new(BooleanBuffer::new_unset(num_rows), None), |result, expr| -> Result { - Ok(or_kleene( - &result, - &compare_op_for_nested( - Operator::Eq, - &value, - &expr?.into_array(num_rows)?, - )?, - )?) + let rhs = compare_with_eq( + &value, + &expr?.into_array(num_rows)?, + is_nested, + )?; + Ok(or_kleene(&result, &rhs)?) }, )?; From 453c026328ca8747ebcbaa6d2b0df939f3123de1 Mon Sep 17 00:00:00 2001 From: Jonah Gao Date: Sat, 7 Sep 2024 13:13:57 +0800 Subject: [PATCH 113/154] Remove unnecessary `Result` from return type in `NamePreserver` (#12358) --- datafusion/expr/src/expr_rewriter/mod.rs | 18 ++++++++---------- datafusion/expr/src/logical_plan/plan.rs | 4 ++-- .../src/analyzer/count_wildcard_rule.rs | 4 ++-- .../optimizer/src/analyzer/function_rewrite.rs | 4 ++-- .../optimizer/src/analyzer/type_coercion.rs | 6 +++--- .../optimizer/src/common_subexpr_eliminate.rs | 12 ++++++------ .../optimizer/src/optimize_projections/mod.rs | 6 +++--- .../src/simplify_expressions/simplify_exprs.rs | 4 ++-- .../optimizer/src/unwrap_cast_in_comparison.rs | 6 +++--- 9 files changed, 31 insertions(+), 33 deletions(-) diff --git a/datafusion/expr/src/expr_rewriter/mod.rs b/datafusion/expr/src/expr_rewriter/mod.rs index f0f02f398642..05ee3e112d62 100644 --- a/datafusion/expr/src/expr_rewriter/mod.rs +++ b/datafusion/expr/src/expr_rewriter/mod.rs @@ -318,21 +318,20 @@ impl NamePreserver { Self { use_alias: true } } - pub fn save(&self, expr: &Expr) -> Result { - let original_name = if self.use_alias { + pub fn save(&self, expr: &Expr) -> SavedName { + if self.use_alias { let (relation, name) = expr.qualified_name(); SavedName::Saved { relation, name } } else { SavedName::None - }; - Ok(original_name) + } } } impl SavedName { /// Ensures the qualified name of the rewritten expression is preserved - pub fn restore(self, expr: Expr) -> Result { - let expr = match self { + pub fn restore(self, expr: Expr) -> Expr { + match self { SavedName::Saved { relation, name } => { let (new_relation, new_name) = expr.qualified_name(); if new_relation != relation || new_name != name { @@ -342,8 +341,7 @@ impl SavedName { } } SavedName::None => expr, - }; - Ok(expr) + } } } @@ -543,9 +541,9 @@ mod test { let mut rewriter = TestRewriter { rewrite_to: rewrite_to.clone(), }; - let saved_name = NamePreserver { use_alias: true }.save(&expr_from).unwrap(); + let saved_name = NamePreserver { use_alias: true }.save(&expr_from); let new_expr = expr_from.clone().rewrite(&mut rewriter).unwrap().data; - let new_expr = saved_name.restore(new_expr).unwrap(); + let new_expr = saved_name.restore(new_expr); let original_name = expr_from.qualified_name(); let new_name = new_expr.qualified_name(); diff --git a/datafusion/expr/src/logical_plan/plan.rs b/datafusion/expr/src/logical_plan/plan.rs index 18a624dd9cb2..20adeb7cc808 100644 --- a/datafusion/expr/src/logical_plan/plan.rs +++ b/datafusion/expr/src/logical_plan/plan.rs @@ -1441,7 +1441,7 @@ impl LogicalPlan { let schema = Arc::clone(plan.schema()); let name_preserver = NamePreserver::new(&plan); plan.map_expressions(|e| { - let original_name = name_preserver.save(&e)?; + let original_name = name_preserver.save(&e); let transformed_expr = e.infer_placeholder_types(&schema)?.transform_up(|e| { if let Expr::Placeholder(Placeholder { id, .. }) = e { @@ -1452,7 +1452,7 @@ impl LogicalPlan { } })?; // Preserve name to avoid breaking column references to this expression - transformed_expr.map_data(|expr| original_name.restore(expr)) + Ok(transformed_expr.update_data(|expr| original_name.restore(expr))) }) }) .map(|res| res.data) diff --git a/datafusion/optimizer/src/analyzer/count_wildcard_rule.rs b/datafusion/optimizer/src/analyzer/count_wildcard_rule.rs index 0036f6df43f6..32ca790b0094 100644 --- a/datafusion/optimizer/src/analyzer/count_wildcard_rule.rs +++ b/datafusion/optimizer/src/analyzer/count_wildcard_rule.rs @@ -76,7 +76,7 @@ fn is_count_star_window_aggregate(window_function: &WindowFunction) -> bool { fn analyze_internal(plan: LogicalPlan) -> Result> { let name_preserver = NamePreserver::new(&plan); plan.map_expressions(|expr| { - let original_name = name_preserver.save(&expr)?; + let original_name = name_preserver.save(&expr); let transformed_expr = expr.transform_up(|expr| match expr { Expr::WindowFunction(mut window_function) if is_count_star_window_aggregate(&window_function) => @@ -94,7 +94,7 @@ fn analyze_internal(plan: LogicalPlan) -> Result> { } _ => Ok(Transformed::no(expr)), })?; - transformed_expr.map_data(|data| original_name.restore(data)) + Ok(transformed_expr.update_data(|data| original_name.restore(data))) }) } diff --git a/datafusion/optimizer/src/analyzer/function_rewrite.rs b/datafusion/optimizer/src/analyzer/function_rewrite.rs index 94f5657b899b..ec3626b2c899 100644 --- a/datafusion/optimizer/src/analyzer/function_rewrite.rs +++ b/datafusion/optimizer/src/analyzer/function_rewrite.rs @@ -61,7 +61,7 @@ impl ApplyFunctionRewrites { let name_preserver = NamePreserver::new(&plan); plan.map_expressions(|expr| { - let original_name = name_preserver.save(&expr)?; + let original_name = name_preserver.save(&expr); // recursively transform the expression, applying the rewrites at each step let transformed_expr = expr.transform_up(|expr| { @@ -74,7 +74,7 @@ impl ApplyFunctionRewrites { Ok(result) })?; - transformed_expr.map_data(|expr| original_name.restore(expr)) + Ok(transformed_expr.update_data(|expr| original_name.restore(expr))) }) } } diff --git a/datafusion/optimizer/src/analyzer/type_coercion.rs b/datafusion/optimizer/src/analyzer/type_coercion.rs index 284650c3d64e..a0b5c9552c83 100644 --- a/datafusion/optimizer/src/analyzer/type_coercion.rs +++ b/datafusion/optimizer/src/analyzer/type_coercion.rs @@ -119,9 +119,9 @@ fn analyze_internal( let name_preserver = NamePreserver::new(&plan); // apply coercion rewrite all expressions in the plan individually plan.map_expressions(|expr| { - let original_name = name_preserver.save(&expr)?; - expr.rewrite(&mut expr_rewrite)? - .map_data(|expr| original_name.restore(expr)) + let original_name = name_preserver.save(&expr); + expr.rewrite(&mut expr_rewrite) + .map(|transformed| transformed.update_data(|e| original_name.restore(e))) })? // some plans need extra coercion after their expressions are coerced .map_data(|plan| expr_rewrite.coerce_plan(plan))? diff --git a/datafusion/optimizer/src/common_subexpr_eliminate.rs b/datafusion/optimizer/src/common_subexpr_eliminate.rs index 583c6cf50de3..14b23a4524fb 100644 --- a/datafusion/optimizer/src/common_subexpr_eliminate.rs +++ b/datafusion/optimizer/src/common_subexpr_eliminate.rs @@ -414,9 +414,9 @@ impl CommonSubexprEliminate { exprs .iter() .map(|expr| name_preserver.save(expr)) - .collect::>>() + .collect::>() }) - .collect::>>()?; + .collect::>(); new_window_expr_list.into_iter().zip(saved_names).try_rfold( new_input, |plan, (new_window_expr, saved_names)| { @@ -426,7 +426,7 @@ impl CommonSubexprEliminate { .map(|(new_window_expr, saved_name)| { saved_name.restore(new_window_expr) }) - .collect::>>()?; + .collect::>(); Window::try_new(new_window_expr, Arc::new(plan)) .map(LogicalPlan::Window) }, @@ -604,14 +604,14 @@ impl CommonSubexprEliminate { let saved_names = aggr_expr .iter() .map(|expr| name_perserver.save(expr)) - .collect::>>()?; + .collect::>(); let new_aggr_expr = rewritten_aggr_expr .into_iter() - .zip(saved_names.into_iter()) + .zip(saved_names) .map(|(new_expr, saved_name)| { saved_name.restore(new_expr) }) - .collect::>>()?; + .collect::>(); // Since `group_expr` may have changed, schema may also. // Use `try_new()` method. diff --git a/datafusion/optimizer/src/optimize_projections/mod.rs b/datafusion/optimizer/src/optimize_projections/mod.rs index 96772d3f2864..0623be504b9b 100644 --- a/datafusion/optimizer/src/optimize_projections/mod.rs +++ b/datafusion/optimizer/src/optimize_projections/mod.rs @@ -497,7 +497,7 @@ fn merge_consecutive_projections(proj: Projection) -> Result Result>>()?; + .collect::>(); Projection::try_new(new_exprs, prev_projection.input).map(Transformed::yes) } else { // not rewritten, so put the projection back together diff --git a/datafusion/optimizer/src/simplify_expressions/simplify_exprs.rs b/datafusion/optimizer/src/simplify_expressions/simplify_exprs.rs index 877989c02d74..c4dba4c8df73 100644 --- a/datafusion/optimizer/src/simplify_expressions/simplify_exprs.rs +++ b/datafusion/optimizer/src/simplify_expressions/simplify_exprs.rs @@ -123,10 +123,10 @@ impl SimplifyExpressions { // Preserve expression names to avoid changing the schema of the plan. let name_preserver = NamePreserver::new(&plan); plan.map_expressions(|e| { - let original_name = name_preserver.save(&e)?; + let original_name = name_preserver.save(&e); let new_e = simplifier .simplify(e) - .and_then(|expr| original_name.restore(expr))?; + .map(|expr| original_name.restore(expr))?; // TODO it would be nice to have a way to know if the expression was simplified // or not. For now conservatively return Transformed::yes Ok(Transformed::yes(new_e)) diff --git a/datafusion/optimizer/src/unwrap_cast_in_comparison.rs b/datafusion/optimizer/src/unwrap_cast_in_comparison.rs index 6043f0d7c8b5..318badc08823 100644 --- a/datafusion/optimizer/src/unwrap_cast_in_comparison.rs +++ b/datafusion/optimizer/src/unwrap_cast_in_comparison.rs @@ -117,9 +117,9 @@ impl OptimizerRule for UnwrapCastInComparison { let name_preserver = NamePreserver::new(&plan); plan.map_expressions(|expr| { - let original_name = name_preserver.save(&expr)?; - expr.rewrite(&mut expr_rewriter)? - .map_data(|expr| original_name.restore(expr)) + let original_name = name_preserver.save(&expr); + expr.rewrite(&mut expr_rewriter) + .map(|transformed| transformed.update_data(|e| original_name.restore(e))) }) } } From 82fb5b94ae1fe97029f54b9ba29517b3fec30280 Mon Sep 17 00:00:00 2001 From: Yongting You <2010youy01@gmail.com> Date: Sat, 7 Sep 2024 13:15:49 +0800 Subject: [PATCH 114/154] Faster `character_length()` string function for ASCII-only case (#12356) * charcter_length() benchmark * char_length() ascii fast path * use usize_as --- datafusion/functions/Cargo.toml | 5 + .../functions/benches/character_length.rs | 114 ++++++++++++++++++ datafusion/functions/src/string/common.rs | 11 ++ .../functions/src/unicode/character_length.rs | 20 ++- 4 files changed, 144 insertions(+), 6 deletions(-) create mode 100644 datafusion/functions/benches/character_length.rs diff --git a/datafusion/functions/Cargo.toml b/datafusion/functions/Cargo.toml index 337379a74670..3c95c03896e2 100644 --- a/datafusion/functions/Cargo.toml +++ b/datafusion/functions/Cargo.toml @@ -166,3 +166,8 @@ required-features = ["math_expressions"] harness = false name = "substr" required-features = ["unicode_expressions"] + +[[bench]] +harness = false +name = "character_length" +required-features = ["unicode_expressions"] diff --git a/datafusion/functions/benches/character_length.rs b/datafusion/functions/benches/character_length.rs new file mode 100644 index 000000000000..17c4dd1f8912 --- /dev/null +++ b/datafusion/functions/benches/character_length.rs @@ -0,0 +1,114 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +extern crate criterion; + +use arrow::array::{StringArray, StringViewArray}; +use criterion::{black_box, criterion_group, criterion_main, Criterion}; +use datafusion_expr::ColumnarValue; +use rand::distributions::Alphanumeric; +use rand::{rngs::StdRng, Rng, SeedableRng}; +use std::sync::Arc; + +/// gen_arr(4096, 128, 0.1, 0.1, true) will generate a StringViewArray with +/// 4096 rows, each row containing a string with 128 random characters. +/// around 10% of the rows are null, around 10% of the rows are non-ASCII. +fn gen_string_array( + n_rows: usize, + str_len_chars: usize, + null_density: f32, + utf8_density: f32, + is_string_view: bool, // false -> StringArray, true -> StringViewArray +) -> Vec { + let mut rng = StdRng::seed_from_u64(42); + let rng_ref = &mut rng; + + let corpus = "DataFusionДатаФусион数据融合📊🔥"; // includes utf8 encoding with 1~4 bytes + let corpus_char_count = corpus.chars().count(); + + let mut output_string_vec: Vec> = Vec::with_capacity(n_rows); + for _ in 0..n_rows { + let rand_num = rng_ref.gen::(); // [0.0, 1.0) + if rand_num < null_density { + output_string_vec.push(None); + } else if rand_num < null_density + utf8_density { + // Generate random UTF8 string + let mut generated_string = String::with_capacity(str_len_chars); + for _ in 0..str_len_chars { + let idx = rng_ref.gen_range(0..corpus_char_count); + let char = corpus.chars().nth(idx).unwrap(); + generated_string.push(char); + } + output_string_vec.push(Some(generated_string)); + } else { + // Generate random ASCII-only string + let value = rng_ref + .sample_iter(&Alphanumeric) + .take(str_len_chars) + .collect(); + let value = String::from_utf8(value).unwrap(); + output_string_vec.push(Some(value)); + } + } + + if is_string_view { + let string_view_array: StringViewArray = output_string_vec.into_iter().collect(); + vec![ColumnarValue::Array(Arc::new(string_view_array))] + } else { + let string_array: StringArray = output_string_vec.clone().into_iter().collect(); + vec![ColumnarValue::Array(Arc::new(string_array))] + } +} + +fn criterion_benchmark(c: &mut Criterion) { + // All benches are single batch run with 8192 rows + let character_length = datafusion_functions::unicode::character_length(); + + let n_rows = 8192; + for str_len in [8, 32, 128, 4096] { + // StringArray ASCII only + let args_string_ascii = gen_string_array(n_rows, str_len, 0.1, 0.0, false); + c.bench_function( + &format!("character_length_StringArray_ascii_str_len_{}", str_len), + |b| b.iter(|| black_box(character_length.invoke(&args_string_ascii))), + ); + + // StringArray UTF8 + let args_string_utf8 = gen_string_array(n_rows, str_len, 0.1, 0.5, false); + c.bench_function( + &format!("character_length_StringArray_utf8_str_len_{}", str_len), + |b| b.iter(|| black_box(character_length.invoke(&args_string_utf8))), + ); + + // StringViewArray ASCII only + let args_string_view_ascii = gen_string_array(n_rows, str_len, 0.1, 0.0, true); + c.bench_function( + &format!("character_length_StringViewArray_ascii_str_len_{}", str_len), + |b| b.iter(|| black_box(character_length.invoke(&args_string_view_ascii))), + ); + + // StringViewArray UTF8 + let args_string_view_utf8 = gen_string_array(n_rows, str_len, 0.1, 0.5, true); + c.bench_function( + &format!("character_length_StringViewArray_utf8_str_len_{}", str_len), + |b| b.iter(|| black_box(character_length.invoke(&args_string_view_utf8))), + ); + } +} + +criterion_group!(benches, criterion_benchmark); +criterion_main!(benches); diff --git a/datafusion/functions/src/string/common.rs b/datafusion/functions/src/string/common.rs index 6ebcc4ee6cd3..9365a6d83331 100644 --- a/datafusion/functions/src/string/common.rs +++ b/datafusion/functions/src/string/common.rs @@ -351,18 +351,29 @@ pub trait StringArrayType<'a>: ArrayAccessor + Sized { /// /// This iterator iterates returns `Option<&str>` for each item in the array. fn iter(&self) -> ArrayIter; + + /// Check if the array is ASCII only. + fn is_ascii(&self) -> bool; } impl<'a, T: OffsetSizeTrait> StringArrayType<'a> for &'a GenericStringArray { fn iter(&self) -> ArrayIter { GenericStringArray::::iter(self) } + + fn is_ascii(&self) -> bool { + GenericStringArray::::is_ascii(self) + } } impl<'a> StringArrayType<'a> for &'a StringViewArray { fn iter(&self) -> ArrayIter { StringViewArray::iter(self) } + + fn is_ascii(&self) -> bool { + StringViewArray::is_ascii(self) + } } /// Optimized version of the StringBuilder in Arrow that: diff --git a/datafusion/functions/src/unicode/character_length.rs b/datafusion/functions/src/unicode/character_length.rs index e46ee162ff12..c9dc96b2a935 100644 --- a/datafusion/functions/src/unicode/character_length.rs +++ b/datafusion/functions/src/unicode/character_length.rs @@ -15,10 +15,10 @@ // specific language governing permissions and limitations // under the License. +use crate::string::common::StringArrayType; use crate::utils::{make_scalar_function, utf8_to_int_type}; use arrow::array::{ - Array, ArrayAccessor, ArrayIter, ArrayRef, ArrowPrimitiveType, AsArray, - OffsetSizeTrait, PrimitiveArray, + Array, ArrayRef, ArrowPrimitiveType, AsArray, OffsetSizeTrait, PrimitiveArray, }; use arrow::datatypes::{ArrowNativeType, DataType, Int32Type, Int64Type}; use datafusion_common::Result; @@ -99,18 +99,26 @@ fn character_length(args: &[ArrayRef]) -> Result { } } -fn character_length_general<'a, T: ArrowPrimitiveType, V: ArrayAccessor>( +fn character_length_general<'a, T: ArrowPrimitiveType, V: StringArrayType<'a>>( array: V, ) -> Result where T::Native: OffsetSizeTrait, { - let iter = ArrayIter::new(array); + // String characters are variable length encoded in UTF-8, counting the + // number of chars requires expensive decoding, however checking if the + // string is ASCII only is relatively cheap. + // If strings are ASCII only, count bytes instead. + let is_array_ascii_only = array.is_ascii(); + let iter = array.iter(); let result = iter .map(|string| { string.map(|string: &str| { - T::Native::from_usize(string.chars().count()) - .expect("should not fail as string.chars will always return integer") + if is_array_ascii_only { + T::Native::usize_as(string.len()) + } else { + T::Native::usize_as(string.chars().count()) + } }) }) .collect::>(); From ddbdf4bf7f215c01aca435f8edd00412e84aab60 Mon Sep 17 00:00:00 2001 From: Jax Liu Date: Sat, 7 Sep 2024 18:18:22 +0800 Subject: [PATCH 115/154] Unparse TableScan with projections, filters or fetch to SQL string (#12158) * support to unparse table scan with projection * support to unparse table_scan with filter and fetch * add the doc for public function * clippy * remove pub get method for table scan * fix merge conflict * add issue reference * fix typo --- datafusion/expr/src/logical_plan/builder.rs | 36 +++++ datafusion/sql/src/unparser/plan.rs | 99 ++++++++++++- datafusion/sql/tests/cases/plan_to_sql.rs | 154 +++++++++++++++++++- 3 files changed, 278 insertions(+), 11 deletions(-) diff --git a/datafusion/expr/src/logical_plan/builder.rs b/datafusion/expr/src/logical_plan/builder.rs index fc961b83f7b5..f4dad085422b 100644 --- a/datafusion/expr/src/logical_plan/builder.rs +++ b/datafusion/expr/src/logical_plan/builder.rs @@ -333,6 +333,19 @@ impl LogicalPlanBuilder { .map(Self::new) } + /// Convert a table provider into a builder with a TableScan with filter and fetch + pub fn scan_with_filters_fetch( + table_name: impl Into, + table_source: Arc, + projection: Option>, + filters: Vec, + fetch: Option, + ) -> Result { + TableScan::try_new(table_name, table_source, projection, filters, fetch) + .map(LogicalPlan::TableScan) + .map(Self::new) + } + /// Wrap a plan in a window pub fn window_plan( input: LogicalPlan, @@ -1424,6 +1437,29 @@ pub fn table_scan_with_filters( LogicalPlanBuilder::scan_with_filters(name, table_source, projection, filters) } +/// Create a LogicalPlanBuilder representing a scan of a table with the provided name and schema, +/// filters, and inlined fetch. +/// This is mostly used for testing and documentation. +pub fn table_scan_with_filter_and_fetch( + name: Option>, + table_schema: &Schema, + projection: Option>, + filters: Vec, + fetch: Option, +) -> Result { + let table_source = table_source(table_schema); + let name = name + .map(|n| n.into()) + .unwrap_or_else(|| TableReference::bare(UNNAMED_TABLE)); + LogicalPlanBuilder::scan_with_filters_fetch( + name, + table_source, + projection, + filters, + fetch, + ) +} + fn table_source(table_schema: &Schema) -> Arc { let table_schema = Arc::new(table_schema.clone()); Arc::new(LogicalTableSource { table_schema }) diff --git a/datafusion/sql/src/unparser/plan.rs b/datafusion/sql/src/unparser/plan.rs index 478cfd0ecd90..802d4762574d 100644 --- a/datafusion/sql/src/unparser/plan.rs +++ b/datafusion/sql/src/unparser/plan.rs @@ -15,16 +15,16 @@ // specific language governing permissions and limitations // under the License. +use crate::unparser::utils::unproject_agg_exprs; use datafusion_common::{ - internal_err, not_impl_err, plan_err, Column, DataFusionError, Result, + internal_err, not_impl_err, plan_err, Column, DataFusionError, Result, TableReference, }; use datafusion_expr::{ - expr::Alias, Distinct, Expr, JoinConstraint, JoinType, LogicalPlan, Projection, - SortExpr, + expr::Alias, Distinct, Expr, JoinConstraint, JoinType, LogicalPlan, + LogicalPlanBuilder, Projection, SortExpr, }; use sqlparser::ast::{self, Ident, SetExpr}; - -use crate::unparser::utils::unproject_agg_exprs; +use std::sync::Arc; use super::{ ast::{ @@ -240,6 +240,19 @@ impl Unparser<'_> { ) -> Result<()> { match plan { LogicalPlan::TableScan(scan) => { + if scan.projection.is_some() + || !scan.filters.is_empty() + || scan.fetch.is_some() + { + let unparsed_table_scan = + Self::unparse_table_scan_pushdown(plan, None)?; + return self.select_to_sql_recursively( + &unparsed_table_scan, + query, + select, + relation, + ); + } let mut builder = TableRelationBuilder::default(); let mut table_parts = vec![]; if let Some(catalog_name) = scan.table_name.catalog() { @@ -455,7 +468,10 @@ impl Unparser<'_> { LogicalPlan::SubqueryAlias(plan_alias) => { let (plan, mut columns) = subquery_alias_inner_query_and_columns(plan_alias); - + let plan = Self::unparse_table_scan_pushdown( + plan, + Some(plan_alias.alias.clone()), + )?; if !columns.is_empty() && !self.dialect.supports_column_alias_in_table_alias() { @@ -467,7 +483,7 @@ impl Unparser<'_> { }; // Instead of specifying column aliases as part of the outer table, inject them directly into the inner projection - let rewritten_plan = inject_column_aliases(inner_p, columns); + let rewritten_plan = inject_column_aliases(&inner_p, columns); columns = vec![]; self.select_to_sql_recursively( @@ -477,7 +493,7 @@ impl Unparser<'_> { relation, )?; } else { - self.select_to_sql_recursively(plan, query, select, relation)?; + self.select_to_sql_recursively(&plan, query, select, relation)?; } relation.alias(Some( @@ -532,6 +548,73 @@ impl Unparser<'_> { } } + fn unparse_table_scan_pushdown( + plan: &LogicalPlan, + alias: Option, + ) -> Result { + match plan { + LogicalPlan::TableScan(table_scan) => { + // TODO: support filters for table scan with alias. Remove this check after #12368 issue. + // see the issue: https://github.com/apache/datafusion/issues/12368 + if alias.is_some() && !table_scan.filters.is_empty() { + return not_impl_err!( + "Subquery alias is not supported for table scan with pushdown filters" + ); + } + + let mut builder = LogicalPlanBuilder::scan( + table_scan.table_name.clone(), + Arc::clone(&table_scan.source), + None, + )?; + if let Some(project_vec) = &table_scan.projection { + let project_columns = project_vec + .iter() + .cloned() + .map(|i| { + let (qualifier, field) = + table_scan.projected_schema.qualified_field(i); + if alias.is_some() { + Column::new(alias.clone(), field.name().clone()) + } else { + Column::new(qualifier.cloned(), field.name().clone()) + } + }) + .collect::>(); + if let Some(alias) = alias { + builder = builder.alias(alias)?; + } + builder = builder.project(project_columns)?; + } + + let filter_expr = table_scan + .filters + .iter() + .cloned() + .reduce(|acc, expr| acc.and(expr)); + if let Some(filter) = filter_expr { + builder = builder.filter(filter)?; + } + + if let Some(fetch) = table_scan.fetch { + builder = builder.limit(0, Some(fetch))?; + } + + builder.build() + } + LogicalPlan::SubqueryAlias(subquery_alias) => { + let new_plan = Self::unparse_table_scan_pushdown( + &subquery_alias.input, + Some(subquery_alias.alias.clone()), + )?; + LogicalPlanBuilder::from(new_plan) + .alias(subquery_alias.alias.clone())? + .build() + } + _ => Ok(plan.clone()), + } + } + fn select_item_to_sql(&self, expr: &Expr) -> Result { match expr { Expr::Alias(Alias { expr, name, .. }) => { diff --git a/datafusion/sql/tests/cases/plan_to_sql.rs b/datafusion/sql/tests/cases/plan_to_sql.rs index 4a6923593fc9..fa95d05c3275 100644 --- a/datafusion/sql/tests/cases/plan_to_sql.rs +++ b/datafusion/sql/tests/cases/plan_to_sql.rs @@ -21,7 +21,7 @@ use std::vec; use arrow_schema::*; use datafusion_common::{DFSchema, Result, TableReference}; use datafusion_expr::test::function_stub::{count_udaf, max_udaf, min_udaf, sum_udaf}; -use datafusion_expr::{col, table_scan}; +use datafusion_expr::{col, lit, table_scan, wildcard, LogicalPlanBuilder}; use datafusion_sql::planner::{ContextProvider, PlannerContext, SqlToRel}; use datafusion_sql::unparser::dialect::{ DefaultDialect as UnparserDefaultDialect, Dialect as UnparserDialect, @@ -29,12 +29,14 @@ use datafusion_sql::unparser::dialect::{ }; use datafusion_sql::unparser::{expr_to_sql, plan_to_sql, Unparser}; +use crate::common::{MockContextProvider, MockSessionState}; +use datafusion_expr::builder::{ + table_scan_with_filter_and_fetch, table_scan_with_filters, +}; use datafusion_functions::core::planner::CoreFunctionPlanner; use sqlparser::dialect::{Dialect, GenericDialect, MySqlDialect}; use sqlparser::parser::Parser; -use crate::common::{MockContextProvider, MockSessionState}; - #[test] fn roundtrip_expr() { let tests: Vec<(TableReference, &str, &str)> = vec![ @@ -619,6 +621,152 @@ fn sql_round_trip(query: &str, expect: &str) { assert_eq!(roundtrip_statement.to_string(), expect); } +#[test] +fn test_table_scan_pushdown() -> Result<()> { + let schema = Schema::new(vec![ + Field::new("id", DataType::Utf8, false), + Field::new("age", DataType::Utf8, false), + ]); + + let scan_with_projection = + table_scan(Some("t1"), &schema, Some(vec![0, 1]))?.build()?; + let scan_with_projection = plan_to_sql(&scan_with_projection)?; + assert_eq!( + format!("{}", scan_with_projection), + "SELECT t1.id, t1.age FROM t1" + ); + + let scan_with_no_projection = table_scan(Some("t1"), &schema, None)?.build()?; + let scan_with_no_projection = plan_to_sql(&scan_with_no_projection)?; + assert_eq!(format!("{}", scan_with_no_projection), "SELECT * FROM t1"); + + let table_scan_with_projection_alias = + table_scan(Some("t1"), &schema, Some(vec![0, 1]))? + .alias("ta")? + .build()?; + let table_scan_with_projection_alias = + plan_to_sql(&table_scan_with_projection_alias)?; + assert_eq!( + format!("{}", table_scan_with_projection_alias), + "SELECT ta.id, ta.age FROM t1 AS ta" + ); + + let table_scan_with_no_projection_alias = table_scan(Some("t1"), &schema, None)? + .alias("ta")? + .build()?; + let table_scan_with_no_projection_alias = + plan_to_sql(&table_scan_with_no_projection_alias)?; + assert_eq!( + format!("{}", table_scan_with_no_projection_alias), + "SELECT * FROM t1 AS ta" + ); + + let query_from_table_scan_with_projection = LogicalPlanBuilder::from( + table_scan(Some("t1"), &schema, Some(vec![0, 1]))?.build()?, + ) + .project(vec![wildcard()])? + .build()?; + let query_from_table_scan_with_projection = + plan_to_sql(&query_from_table_scan_with_projection)?; + assert_eq!( + format!("{}", query_from_table_scan_with_projection), + "SELECT * FROM (SELECT t1.id, t1.age FROM t1)" + ); + + let table_scan_with_filter = table_scan_with_filters( + Some("t1"), + &schema, + None, + vec![col("id").gt(col("age"))], + )? + .build()?; + let table_scan_with_filter = plan_to_sql(&table_scan_with_filter)?; + assert_eq!( + format!("{}", table_scan_with_filter), + "SELECT * FROM t1 WHERE (t1.id > t1.age)" + ); + + let table_scan_with_two_filter = table_scan_with_filters( + Some("t1"), + &schema, + None, + vec![col("id").gt(lit(1)), col("age").lt(lit(2))], + )? + .build()?; + let table_scan_with_two_filter = plan_to_sql(&table_scan_with_two_filter)?; + assert_eq!( + format!("{}", table_scan_with_two_filter), + "SELECT * FROM t1 WHERE ((t1.id > 1) AND (t1.age < 2))" + ); + + // TODO: support filters for table scan with alias. Enable this test after #12368 issue is fixed + // see the issue: https://github.com/apache/datafusion/issues/12368 + // let table_scan_with_filter_alias = table_scan_with_filters( + // Some("t1"), + // &schema, + // None, + // vec![col("id").gt(col("age"))], + // )?.alias("ta")?.build()?; + // let table_scan_with_filter_alias = plan_to_sql(&table_scan_with_filter_alias)?; + // assert_eq!( + // format!("{}", table_scan_with_filter_alias), + // "SELECT * FROM t1 AS ta WHERE (ta.id > ta.age)" + // ); + + let table_scan_with_projection_and_filter = table_scan_with_filters( + Some("t1"), + &schema, + Some(vec![0, 1]), + vec![col("id").gt(col("age"))], + )? + .build()?; + let table_scan_with_projection_and_filter = + plan_to_sql(&table_scan_with_projection_and_filter)?; + assert_eq!( + format!("{}", table_scan_with_projection_and_filter), + "SELECT t1.id, t1.age FROM t1 WHERE (t1.id > t1.age)" + ); + + let table_scan_with_inline_fetch = + table_scan_with_filter_and_fetch(Some("t1"), &schema, None, vec![], Some(10))? + .build()?; + let table_scan_with_inline_fetch = plan_to_sql(&table_scan_with_inline_fetch)?; + assert_eq!( + format!("{}", table_scan_with_inline_fetch), + "SELECT * FROM t1 LIMIT 10" + ); + + let table_scan_with_projection_and_inline_fetch = table_scan_with_filter_and_fetch( + Some("t1"), + &schema, + Some(vec![0, 1]), + vec![], + Some(10), + )? + .build()?; + let table_scan_with_projection_and_inline_fetch = + plan_to_sql(&table_scan_with_projection_and_inline_fetch)?; + assert_eq!( + format!("{}", table_scan_with_projection_and_inline_fetch), + "SELECT t1.id, t1.age FROM t1 LIMIT 10" + ); + + let table_scan_with_all = table_scan_with_filter_and_fetch( + Some("t1"), + &schema, + Some(vec![0, 1]), + vec![col("id").gt(col("age"))], + Some(10), + )? + .build()?; + let table_scan_with_all = plan_to_sql(&table_scan_with_all)?; + assert_eq!( + format!("{}", table_scan_with_all), + "SELECT t1.id, t1.age FROM t1 WHERE (t1.id > t1.age) LIMIT 10" + ); + Ok(()) +} + #[test] fn test_interval_lhs_eq() { sql_round_trip( From 73f308605ea1b8f0e784278332cbf013ea77a8b9 Mon Sep 17 00:00:00 2001 From: Lordworms <48054792+Lordworms@users.noreply.github.com> Date: Sat, 7 Sep 2024 03:49:54 -0700 Subject: [PATCH 116/154] Minor: Support protobuf serialization for Utf8View and BinaryView (#12165) * Minor: Support protobuf serialization for ScalarValue::Utf8View and ScalarValue::BinaryView * Use correct protobuf type, add coverage for DataType serialization --------- Co-authored-by: Andrew Lamb --- datafusion/proto-common/src/to_proto/mod.rs | 4 ++- .../tests/cases/roundtrip_logical_plan.rs | 33 +++++++++++++++---- 2 files changed, 29 insertions(+), 8 deletions(-) diff --git a/datafusion/proto-common/src/to_proto/mod.rs b/datafusion/proto-common/src/to_proto/mod.rs index 3718ccbb0f85..e608caf0ecf8 100644 --- a/datafusion/proto-common/src/to_proto/mod.rs +++ b/datafusion/proto-common/src/to_proto/mod.rs @@ -153,9 +153,11 @@ impl TryFrom<&DataType> for protobuf::arrow_type::ArrowTypeEnum { Self::Interval(protobuf::IntervalUnit::from(interval_unit) as i32) } DataType::Binary => Self::Binary(EmptyMessage {}), + DataType::BinaryView => Self::BinaryView(EmptyMessage {}), DataType::FixedSizeBinary(size) => Self::FixedSizeBinary(*size), DataType::LargeBinary => Self::LargeBinary(EmptyMessage {}), DataType::Utf8 => Self::Utf8(EmptyMessage {}), + DataType::Utf8View => Self::Utf8View(EmptyMessage {}), DataType::LargeUtf8 => Self::LargeUtf8(EmptyMessage {}), DataType::List(item_type) => Self::List(Box::new(protobuf::List { field_type: Some(Box::new(item_type.as_ref().try_into()?)), @@ -210,7 +212,7 @@ impl TryFrom<&DataType> for protobuf::arrow_type::ArrowTypeEnum { "Proto serialization error: The RunEndEncoded data type is not yet supported".to_owned() )) } - DataType::Utf8View | DataType::BinaryView | DataType::ListView(_) | DataType::LargeListView(_) => { + DataType::ListView(_) | DataType::LargeListView(_) => { return Err(Error::General(format!("Proto serialization error: {val} not yet supported"))) } }; diff --git a/datafusion/proto/tests/cases/roundtrip_logical_plan.rs b/datafusion/proto/tests/cases/roundtrip_logical_plan.rs index dd3b99b0768b..c550847250a0 100644 --- a/datafusion/proto/tests/cases/roundtrip_logical_plan.rs +++ b/datafusion/proto/tests/cases/roundtrip_logical_plan.rs @@ -1191,7 +1191,7 @@ impl LogicalExtensionCodec for UDFExtensionCodec { } #[test] -fn round_trip_scalar_values() { +fn round_trip_scalar_values_and_data_types() { let should_pass: Vec = vec![ ScalarValue::Boolean(None), ScalarValue::Float32(None), @@ -1245,6 +1245,8 @@ fn round_trip_scalar_values() { ScalarValue::UInt64(Some(0)), ScalarValue::Utf8(Some(String::from("Test string "))), ScalarValue::LargeUtf8(Some(String::from("Test Large utf8"))), + ScalarValue::Utf8View(Some(String::from("Test stringview"))), + ScalarValue::BinaryView(Some(b"binaryview".to_vec())), ScalarValue::Date32(Some(0)), ScalarValue::Date32(Some(i32::MAX)), ScalarValue::Date32(None), @@ -1471,21 +1473,38 @@ fn round_trip_scalar_values() { ScalarValue::FixedSizeBinary(5, None), ]; - for test_case in should_pass.into_iter() { - let proto: protobuf::ScalarValue = (&test_case) - .try_into() - .expect("failed conversion to protobuf"); - + // ScalarValue directly + for test_case in should_pass.iter() { + let proto: protobuf::ScalarValue = + test_case.try_into().expect("failed conversion to protobuf"); let roundtrip: ScalarValue = (&proto) .try_into() .expect("failed conversion from protobuf"); assert_eq!( - test_case, roundtrip, + test_case, &roundtrip, "ScalarValue was not the same after round trip!\n\n\ Input: {test_case:?}\n\nRoundtrip: {roundtrip:?}" ); } + + // DataType conversion + for test_case in should_pass.iter() { + let dt = test_case.data_type(); + + let proto: protobuf::ArrowType = (&dt) + .try_into() + .expect("datatype failed conversion to protobuf"); + let roundtrip: DataType = (&proto) + .try_into() + .expect("datatype failed conversion from protobuf"); + + assert_eq!( + dt, roundtrip, + "DataType was not the same after round trip!\n\n\ + Input: {dt:?}\n\nRoundtrip: {roundtrip:?}" + ); + } } // create a map array [{joe:1}, {blogs:2, foo:4}, {}, null] for testing From 5e37bb929e69a44f45d91a0898412dba3ac42700 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Sat, 7 Sep 2024 07:19:56 -0400 Subject: [PATCH 117/154] Minor: Add tests for using FilterExec when parquet was pushed down (#12362) * Add test for parquet filter pushdown * Set configuration explicitly --- .../test_files/parquet_filter_pushdown.slt | 174 ++++++++++++++++++ 1 file changed, 174 insertions(+) create mode 100644 datafusion/sqllogictest/test_files/parquet_filter_pushdown.slt diff --git a/datafusion/sqllogictest/test_files/parquet_filter_pushdown.slt b/datafusion/sqllogictest/test_files/parquet_filter_pushdown.slt new file mode 100644 index 000000000000..d69662f75da4 --- /dev/null +++ b/datafusion/sqllogictest/test_files/parquet_filter_pushdown.slt @@ -0,0 +1,174 @@ +# 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. + + +########## +# Tests for parquet filter pushdown (filtering on data in the +# scan not just the metadata) +########## + +# File1 has only columns a and b +statement ok +COPY ( + SELECT column1 as a, column2 as b + FROM ( VALUES ('foo', 1), ('bar', 2), ('foo', 3), ('baz', 50) ) + ) TO 'test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet' +STORED AS PARQUET; + +# File2 has only b +statement ok +COPY ( + SELECT column1 as b + FROM ( VALUES (10), (20), (30) ) + ) TO 'test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet' +STORED AS PARQUET; + + +## Create table without filter pushdown +## (pushdown setting is part of the table, but is copied from the session settings) + +# pushdown_filters (currently) defaults to false, but we set it here to be explicit +statement ok +set datafusion.execution.parquet.pushdown_filters = false; + +statement ok +CREATE EXTERNAL TABLE t(a varchar, b int, c float) STORED AS PARQUET +LOCATION 'test_files/scratch/parquet_filter_pushdown/parquet_table/'; + +## Create table with pushdown enabled (pushdown setting is part of the table) + +statement ok +set datafusion.execution.parquet.pushdown_filters = true; + +## Create table without pushdown +statement ok +CREATE EXTERNAL TABLE t_pushdown(a varchar, b int, c float) STORED AS PARQUET +LOCATION 'test_files/scratch/parquet_filter_pushdown/parquet_table/'; + +# restore defaults +statement ok +set datafusion.execution.parquet.pushdown_filters = false; + +# When filter pushdown is not enabled, ParquetExec only filters based on +# metadata, so a FilterExec is required to filter the +# output of the `ParquetExec` + +query T +select a from t where b > 2 ORDER BY a; +---- +baz +foo +NULL +NULL +NULL + +query TT +EXPLAIN select a from t_pushdown where b > 2 ORDER BY a; +---- +logical_plan +01)Sort: t_pushdown.a ASC NULLS LAST +02)--Projection: t_pushdown.a +03)----Filter: t_pushdown.b > Int32(2) +04)------TableScan: t_pushdown projection=[a, b], partial_filters=[t_pushdown.b > Int32(2)] +physical_plan +01)SortPreservingMergeExec: [a@0 ASC NULLS LAST] +02)--SortExec: expr=[a@0 ASC NULLS LAST], preserve_partitioning=[true] +03)----CoalesceBatchesExec: target_batch_size=8192 +04)------FilterExec: b@1 > 2, projection=[a@0] +05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=2 +06)----------ParquetExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet]]}, projection=[a, b], predicate=b@1 > 2, pruning_predicate=CASE WHEN b_null_count@1 = b_row_count@2 THEN false ELSE b_max@0 > 2 END, required_guarantees=[] + + +# When filter pushdown *is* enabled, ParquetExec can filter exactly, +# not just metadata, so we expect to see no FilterExec +# once https://github.com/apache/datafusion/issues/4028 is fixed +query T +select a from t_pushdown where b > 2 ORDER BY a; +---- +baz +foo +NULL +NULL +NULL + +query TT +EXPLAIN select a from t where b > 2 ORDER BY a; +---- +logical_plan +01)Sort: t.a ASC NULLS LAST +02)--Projection: t.a +03)----Filter: t.b > Int32(2) +04)------TableScan: t projection=[a, b], partial_filters=[t.b > Int32(2)] +physical_plan +01)SortPreservingMergeExec: [a@0 ASC NULLS LAST] +02)--SortExec: expr=[a@0 ASC NULLS LAST], preserve_partitioning=[true] +03)----CoalesceBatchesExec: target_batch_size=8192 +04)------FilterExec: b@1 > 2, projection=[a@0] +05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=2 +06)----------ParquetExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet]]}, projection=[a, b], predicate=b@1 > 2, pruning_predicate=CASE WHEN b_null_count@1 = b_row_count@2 THEN false ELSE b_max@0 > 2 END, required_guarantees=[] + +# also test querying on columns that are not in all the files +query T +select a from t_pushdown where b > 2 AND a IS NOT NULL order by a; +---- +baz +foo + +query TT +EXPLAIN select a from t_pushdown where b > 2 AND a IS NOT NULL order by a; +---- +logical_plan +01)Sort: t_pushdown.a ASC NULLS LAST +02)--Projection: t_pushdown.a +03)----Filter: t_pushdown.b > Int32(2) AND t_pushdown.a IS NOT NULL +04)------TableScan: t_pushdown projection=[a, b], partial_filters=[t_pushdown.b > Int32(2), t_pushdown.a IS NOT NULL] +physical_plan +01)SortPreservingMergeExec: [a@0 ASC NULLS LAST] +02)--SortExec: expr=[a@0 ASC NULLS LAST], preserve_partitioning=[true] +03)----CoalesceBatchesExec: target_batch_size=8192 +04)------FilterExec: b@1 > 2 AND a@0 IS NOT NULL, projection=[a@0] +05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=2 +06)----------ParquetExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet]]}, projection=[a, b], predicate=b@1 > 2 AND a@0 IS NOT NULL, pruning_predicate=CASE WHEN b_null_count@1 = b_row_count@2 THEN false ELSE b_max@0 > 2 END AND a_null_count@4 != a_row_count@3, required_guarantees=[] + + +query I +select b from t_pushdown where a = 'bar' order by b; +---- +2 + +query TT +EXPLAIN select b from t_pushdown where a = 'bar' order by b; +---- +logical_plan +01)Sort: t_pushdown.b ASC NULLS LAST +02)--Projection: t_pushdown.b +03)----Filter: t_pushdown.a = Utf8("bar") +04)------TableScan: t_pushdown projection=[a, b], partial_filters=[t_pushdown.a = Utf8("bar")] +physical_plan +01)SortPreservingMergeExec: [b@0 ASC NULLS LAST] +02)--SortExec: expr=[b@0 ASC NULLS LAST], preserve_partitioning=[true] +03)----CoalesceBatchesExec: target_batch_size=8192 +04)------FilterExec: a@0 = bar, projection=[b@1] +05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=2 +06)----------ParquetExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet]]}, projection=[a, b], predicate=a@0 = bar, pruning_predicate=CASE WHEN a_null_count@2 = a_row_count@3 THEN false ELSE a_min@0 <= bar AND bar <= a_max@1 END, required_guarantees=[a in (bar)] + +## cleanup +statement ok +DROP TABLE t; + +statement ok +DROP TABLE t_pushdown; From 3e1850db1b7effea16b9130bd329a8240028f8a2 Mon Sep 17 00:00:00 2001 From: Maron Montano Date: Sun, 8 Sep 2024 11:51:06 +0800 Subject: [PATCH 118/154] Minor: Add getter for logical optimizer rules (#12379) * feat: new getter method for optimizer rules in parity with its physical counterpart * style: use rust_lint * chore: make struct local to the test scope --- .../core/src/execution/session_state.rs | 34 +++++++++++++++++++ 1 file changed, 34 insertions(+) diff --git a/datafusion/core/src/execution/session_state.rs b/datafusion/core/src/execution/session_state.rs index 675ac798bf4e..5e8d22b6ccbc 100644 --- a/datafusion/core/src/execution/session_state.rs +++ b/datafusion/core/src/execution/session_state.rs @@ -801,6 +801,11 @@ impl SessionState { &mut self.config } + /// Return the logical optimizers + pub fn optimizers(&self) -> &[Arc] { + &self.optimizer.rules + } + /// Return the physical optimizers pub fn physical_optimizers(&self) -> &[Arc] { &self.physical_optimizers.rules @@ -1826,6 +1831,8 @@ mod tests { use datafusion_common::Result; use datafusion_execution::config::SessionConfig; use datafusion_expr::Expr; + use datafusion_optimizer::optimizer::OptimizerRule; + use datafusion_optimizer::Optimizer; use datafusion_sql::planner::{PlannerContext, SqlToRel}; use std::collections::HashMap; use std::sync::Arc; @@ -1922,4 +1929,31 @@ mod tests { assert!(new_state.catalog_list().catalog(&default_catalog).is_none()); Ok(()) } + + #[test] + fn test_session_state_with_optimizer_rules() { + struct DummyRule {} + + impl OptimizerRule for DummyRule { + fn name(&self) -> &str { + "dummy_rule" + } + } + // test building sessions with fresh set of rules + let state = SessionStateBuilder::new() + .with_optimizer_rules(vec![Arc::new(DummyRule {})]) + .build(); + + assert_eq!(state.optimizers().len(), 1); + + // test adding rules to default recommendations + let state = SessionStateBuilder::new() + .with_optimizer_rule(Arc::new(DummyRule {})) + .build(); + + assert_eq!( + state.optimizers().len(), + Optimizer::default().rules.len() + 1 + ); + } } From f5c47fa274d53c1d524a1fb788d9a063bf5240ef Mon Sep 17 00:00:00 2001 From: Athul T R Date: Sun, 8 Sep 2024 19:50:45 +0530 Subject: [PATCH 119/154] Removed Arc wrapping for AggregateFunctionExpr (#12353) --- .../physical_optimizer/update_aggr_exprs.rs | 6 +- datafusion/core/src/physical_planner.rs | 2 +- datafusion/core/src/test_util/mod.rs | 2 +- .../combine_partial_final_agg.rs | 6 +- datafusion/physical-expr/src/aggregate.rs | 16 +-- .../physical-expr/src/window/aggregate.rs | 6 +- .../src/window/sliding_aggregate.rs | 6 +- .../src/combine_partial_final_agg.rs | 2 +- .../physical-plan/src/aggregates/mod.rs | 97 ++++++++++--------- .../physical-plan/src/aggregates/row_hash.rs | 4 +- datafusion/physical-plan/src/windows/mod.rs | 2 +- datafusion/proto/src/physical_plan/mod.rs | 2 +- .../proto/src/physical_plan/to_proto.rs | 6 +- .../tests/cases/roundtrip_physical_plan.rs | 10 +- 14 files changed, 84 insertions(+), 83 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/update_aggr_exprs.rs b/datafusion/core/src/physical_optimizer/update_aggr_exprs.rs index a2726d62e9f6..8b5084c67e42 100644 --- a/datafusion/core/src/physical_optimizer/update_aggr_exprs.rs +++ b/datafusion/core/src/physical_optimizer/update_aggr_exprs.rs @@ -118,7 +118,7 @@ impl PhysicalOptimizerRule for OptimizeAggregateOrder { /// /// # Parameters /// -/// * `aggr_exprs` - A vector of `Arc` representing the +/// * `aggr_exprs` - A vector of `AggregateFunctionExpr` representing the /// aggregate expressions to be optimized. /// * `prefix_requirement` - An array slice representing the ordering /// requirements preceding the aggregate expressions. @@ -131,10 +131,10 @@ impl PhysicalOptimizerRule for OptimizeAggregateOrder { /// successfully. Any errors occurring during the conversion process are /// passed through. fn try_convert_aggregate_if_better( - aggr_exprs: Vec>, + aggr_exprs: Vec, prefix_requirement: &[PhysicalSortRequirement], eq_properties: &EquivalenceProperties, -) -> Result>> { +) -> Result> { aggr_exprs .into_iter() .map(|aggr_expr| { diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index 82405dd98e30..cc35255dfe29 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -1541,7 +1541,7 @@ pub fn create_window_expr( } type AggregateExprWithOptionalArgs = ( - Arc, + AggregateFunctionExpr, // The filter clause, if any Option>, // Ordering requirements, if any diff --git a/datafusion/core/src/test_util/mod.rs b/datafusion/core/src/test_util/mod.rs index dd8b697666ee..edccb3844c84 100644 --- a/datafusion/core/src/test_util/mod.rs +++ b/datafusion/core/src/test_util/mod.rs @@ -427,7 +427,7 @@ impl TestAggregate { } /// Return appropriate expr depending if COUNT is for col or table (*) - pub fn count_expr(&self, schema: &Schema) -> Arc { + pub fn count_expr(&self, schema: &Schema) -> AggregateFunctionExpr { AggregateExprBuilder::new(count_udaf(), vec![self.column()]) .schema(Arc::new(schema.clone())) .alias(self.column_name()) diff --git a/datafusion/core/tests/physical_optimizer/combine_partial_final_agg.rs b/datafusion/core/tests/physical_optimizer/combine_partial_final_agg.rs index 5152afa6c050..24e46b3ad97c 100644 --- a/datafusion/core/tests/physical_optimizer/combine_partial_final_agg.rs +++ b/datafusion/core/tests/physical_optimizer/combine_partial_final_agg.rs @@ -84,7 +84,7 @@ fn parquet_exec(schema: &SchemaRef) -> Arc { fn partial_aggregate_exec( input: Arc, group_by: PhysicalGroupBy, - aggr_expr: Vec>, + aggr_expr: Vec, ) -> Arc { let schema = input.schema(); let n_aggr = aggr_expr.len(); @@ -104,7 +104,7 @@ fn partial_aggregate_exec( fn final_aggregate_exec( input: Arc, group_by: PhysicalGroupBy, - aggr_expr: Vec>, + aggr_expr: Vec, ) -> Arc { let schema = input.schema(); let n_aggr = aggr_expr.len(); @@ -130,7 +130,7 @@ fn count_expr( expr: Arc, name: &str, schema: &Schema, -) -> Arc { +) -> AggregateFunctionExpr { AggregateExprBuilder::new(count_udaf(), vec![expr]) .schema(Arc::new(schema.clone())) .alias(name) diff --git a/datafusion/physical-expr/src/aggregate.rs b/datafusion/physical-expr/src/aggregate.rs index d62dc27ece86..866596d0b690 100644 --- a/datafusion/physical-expr/src/aggregate.rs +++ b/datafusion/physical-expr/src/aggregate.rs @@ -88,7 +88,7 @@ impl AggregateExprBuilder { } } - pub fn build(self) -> Result> { + pub fn build(self) -> Result { let Self { fun, args, @@ -132,7 +132,7 @@ impl AggregateExprBuilder { Some(alias) => alias, }; - Ok(Arc::new(AggregateFunctionExpr { + Ok(AggregateFunctionExpr { fun: Arc::unwrap_or_clone(fun), args, data_type, @@ -145,7 +145,7 @@ impl AggregateExprBuilder { input_types: input_exprs_types, is_reversed, is_nullable, - })) + }) } pub fn alias(mut self, alias: impl Into) -> Self { @@ -328,9 +328,9 @@ impl AggregateFunctionExpr { /// not implement the method, returns an error. Order insensitive and hard /// requirement aggregators return `Ok(None)`. pub fn with_beneficial_ordering( - self: Arc, + self, beneficial_ordering: bool, - ) -> Result>> { + ) -> Result> { let Some(updated_fn) = self .fun .clone() @@ -457,10 +457,10 @@ impl AggregateFunctionExpr { /// Typically the "reverse" expression is itself (e.g. SUM, COUNT). /// For aggregates that do not support calculation in reverse, /// returns None (which is the default value). - pub fn reverse_expr(&self) -> Option> { + pub fn reverse_expr(&self) -> Option { match self.fun.reverse_udf() { ReversedUDAF::NotSupported => None, - ReversedUDAF::Identical => Some(Arc::new(self.clone())), + ReversedUDAF::Identical => Some(self.clone()), ReversedUDAF::Reversed(reverse_udf) => { let reverse_ordering_req = reverse_order_bys(&self.ordering_req); let mut name = self.name().to_string(); @@ -507,7 +507,7 @@ impl AggregateFunctionExpr { &self, _args: Vec>, _order_by_exprs: Vec>, - ) -> Option> { + ) -> Option { None } diff --git a/datafusion/physical-expr/src/window/aggregate.rs b/datafusion/physical-expr/src/window/aggregate.rs index 1cc08a4e99aa..d012fef93b67 100644 --- a/datafusion/physical-expr/src/window/aggregate.rs +++ b/datafusion/physical-expr/src/window/aggregate.rs @@ -41,7 +41,7 @@ use crate::{expressions::PhysicalSortExpr, reverse_order_bys, PhysicalExpr}; /// See comments on [`WindowExpr`] for more details. #[derive(Debug)] pub struct PlainAggregateWindowExpr { - aggregate: Arc, + aggregate: AggregateFunctionExpr, partition_by: Vec>, order_by: Vec, window_frame: Arc, @@ -50,7 +50,7 @@ pub struct PlainAggregateWindowExpr { impl PlainAggregateWindowExpr { /// Create a new aggregate window function expression pub fn new( - aggregate: Arc, + aggregate: AggregateFunctionExpr, partition_by: &[Arc], order_by: &[PhysicalSortExpr], window_frame: Arc, @@ -64,7 +64,7 @@ impl PlainAggregateWindowExpr { } /// Get aggregate expr of AggregateWindowExpr - pub fn get_aggregate_expr(&self) -> &Arc { + pub fn get_aggregate_expr(&self) -> &AggregateFunctionExpr { &self.aggregate } } diff --git a/datafusion/physical-expr/src/window/sliding_aggregate.rs b/datafusion/physical-expr/src/window/sliding_aggregate.rs index b3848e15ee42..143d59eb4495 100644 --- a/datafusion/physical-expr/src/window/sliding_aggregate.rs +++ b/datafusion/physical-expr/src/window/sliding_aggregate.rs @@ -41,7 +41,7 @@ use crate::{expressions::PhysicalSortExpr, reverse_order_bys, PhysicalExpr}; /// See comments on [`WindowExpr`] for more details. #[derive(Debug)] pub struct SlidingAggregateWindowExpr { - aggregate: Arc, + aggregate: AggregateFunctionExpr, partition_by: Vec>, order_by: Vec, window_frame: Arc, @@ -50,7 +50,7 @@ pub struct SlidingAggregateWindowExpr { impl SlidingAggregateWindowExpr { /// Create a new (sliding) aggregate window function expression. pub fn new( - aggregate: Arc, + aggregate: AggregateFunctionExpr, partition_by: &[Arc], order_by: &[PhysicalSortExpr], window_frame: Arc, @@ -64,7 +64,7 @@ impl SlidingAggregateWindowExpr { } /// Get the [AggregateFunctionExpr] of this object. - pub fn get_aggregate_expr(&self) -> &Arc { + pub fn get_aggregate_expr(&self) -> &AggregateFunctionExpr { &self.aggregate } } diff --git a/datafusion/physical-optimizer/src/combine_partial_final_agg.rs b/datafusion/physical-optimizer/src/combine_partial_final_agg.rs index 12ff13f8f6ae..67e40c9b507e 100644 --- a/datafusion/physical-optimizer/src/combine_partial_final_agg.rs +++ b/datafusion/physical-optimizer/src/combine_partial_final_agg.rs @@ -125,7 +125,7 @@ impl PhysicalOptimizerRule for CombinePartialFinalAggregate { type GroupExprsRef<'a> = ( &'a PhysicalGroupBy, - &'a [Arc], + &'a [AggregateFunctionExpr], &'a [Option>], ); diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index bf6d71308cab..ad0e43631afd 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -262,7 +262,7 @@ pub struct AggregateExec { /// Group by expressions group_by: PhysicalGroupBy, /// Aggregate expressions - aggr_expr: Vec>, + aggr_expr: Vec, /// FILTER (WHERE clause) expression for each aggregate expression filter_expr: Vec>>, /// Set if the output of this aggregation is truncated by a upstream sort/limit clause @@ -289,10 +289,7 @@ impl AggregateExec { /// Function used in `OptimizeAggregateOrder` optimizer rule, /// where we need parts of the new value, others cloned from the old one /// Rewrites aggregate exec with new aggregate expressions. - pub fn with_new_aggr_exprs( - &self, - aggr_expr: Vec>, - ) -> Self { + pub fn with_new_aggr_exprs(&self, aggr_expr: Vec) -> Self { Self { aggr_expr, // clone the rest of the fields @@ -318,7 +315,7 @@ impl AggregateExec { pub fn try_new( mode: AggregateMode, group_by: PhysicalGroupBy, - aggr_expr: Vec>, + aggr_expr: Vec, filter_expr: Vec>>, input: Arc, input_schema: SchemaRef, @@ -355,7 +352,7 @@ impl AggregateExec { fn try_new_with_schema( mode: AggregateMode, group_by: PhysicalGroupBy, - mut aggr_expr: Vec>, + mut aggr_expr: Vec, filter_expr: Vec>>, input: Arc, input_schema: SchemaRef, @@ -463,7 +460,7 @@ impl AggregateExec { } /// Aggregate expressions - pub fn aggr_expr(&self) -> &[Arc] { + pub fn aggr_expr(&self) -> &[AggregateFunctionExpr] { &self.aggr_expr } @@ -790,7 +787,7 @@ impl ExecutionPlan for AggregateExec { fn create_schema( input_schema: &Schema, group_expr: &[(Arc, String)], - aggr_expr: &[Arc], + aggr_expr: &[AggregateFunctionExpr], group_expr_nullable: Vec, mode: AggregateMode, ) -> Result { @@ -836,7 +833,7 @@ fn group_schema(schema: &Schema, group_count: usize) -> SchemaRef { /// /// # Parameters /// -/// - `aggr_expr`: A reference to an `Arc` representing the +/// - `aggr_expr`: A reference to an `AggregateFunctionExpr` representing the /// aggregate expression. /// - `group_by`: A reference to a `PhysicalGroupBy` instance representing the /// physical GROUP BY expression. @@ -848,7 +845,7 @@ fn group_schema(schema: &Schema, group_count: usize) -> SchemaRef { /// A `LexOrdering` instance indicating the lexical ordering requirement for /// the aggregate expression. fn get_aggregate_expr_req( - aggr_expr: &Arc, + aggr_expr: &AggregateFunctionExpr, group_by: &PhysicalGroupBy, agg_mode: &AggregateMode, ) -> LexOrdering { @@ -896,7 +893,7 @@ fn get_aggregate_expr_req( /// the aggregator requirement is incompatible. fn finer_ordering( existing_req: &LexOrdering, - aggr_expr: &Arc, + aggr_expr: &AggregateFunctionExpr, group_by: &PhysicalGroupBy, eq_properties: &EquivalenceProperties, agg_mode: &AggregateMode, @@ -914,7 +911,7 @@ pub fn concat_slices(lhs: &[T], rhs: &[T]) -> Vec { /// /// # Parameters /// -/// - `aggr_exprs`: A slice of `Arc` containing all the +/// - `aggr_exprs`: A slice of `AggregateFunctionExpr` containing all the /// aggregate expressions. /// - `group_by`: A reference to a `PhysicalGroupBy` instance representing the /// physical GROUP BY expression. @@ -928,7 +925,7 @@ pub fn concat_slices(lhs: &[T], rhs: &[T]) -> Vec { /// A `LexRequirement` instance, which is the requirement that satisfies all the /// aggregate requirements. Returns an error in case of conflicting requirements. pub fn get_finer_aggregate_exprs_requirement( - aggr_exprs: &mut [Arc], + aggr_exprs: &mut [AggregateFunctionExpr], group_by: &PhysicalGroupBy, eq_properties: &EquivalenceProperties, agg_mode: &AggregateMode, @@ -1002,7 +999,7 @@ pub fn get_finer_aggregate_exprs_requirement( /// * Partial: AggregateFunctionExpr::expressions /// * Final: columns of `AggregateFunctionExpr::state_fields()` pub fn aggregate_expressions( - aggr_expr: &[Arc], + aggr_expr: &[AggregateFunctionExpr], mode: &AggregateMode, col_idx_base: usize, ) -> Result>>> { @@ -1043,7 +1040,7 @@ pub fn aggregate_expressions( /// `index_base` is the starting physical column index for the next expanded state field. fn merge_expressions( index_base: usize, - expr: &Arc, + expr: &AggregateFunctionExpr, ) -> Result>> { expr.state_fields().map(|fields| { fields @@ -1057,7 +1054,7 @@ fn merge_expressions( pub type AccumulatorItem = Box; pub fn create_accumulators( - aggr_expr: &[Arc], + aggr_expr: &[AggregateFunctionExpr], ) -> Result> { aggr_expr .iter() @@ -1497,12 +1494,13 @@ mod tests { groups: vec![vec![false]], }; - let aggregates: Vec> = vec![ - AggregateExprBuilder::new(avg_udaf(), vec![col("b", &input_schema)?]) - .schema(Arc::clone(&input_schema)) - .alias("AVG(b)") - .build()?, - ]; + let aggregates: Vec = + vec![ + AggregateExprBuilder::new(avg_udaf(), vec![col("b", &input_schema)?]) + .schema(Arc::clone(&input_schema)) + .alias("AVG(b)") + .build()?, + ]; let task_ctx = if spill { // set to an appropriate value to trigger spill @@ -1793,7 +1791,7 @@ mod tests { } // Median(a) - fn test_median_agg_expr(schema: SchemaRef) -> Result> { + fn test_median_agg_expr(schema: SchemaRef) -> Result { AggregateExprBuilder::new(median_udaf(), vec![col("a", &schema)?]) .schema(schema) .alias("MEDIAN(a)") @@ -1819,16 +1817,17 @@ mod tests { }; // something that allocates within the aggregator - let aggregates_v0: Vec> = + let aggregates_v0: Vec = vec![test_median_agg_expr(Arc::clone(&input_schema))?]; // use fast-path in `row_hash.rs`. - let aggregates_v2: Vec> = vec![ - AggregateExprBuilder::new(avg_udaf(), vec![col("b", &input_schema)?]) - .schema(Arc::clone(&input_schema)) - .alias("AVG(b)") - .build()?, - ]; + let aggregates_v2: Vec = + vec![ + AggregateExprBuilder::new(avg_udaf(), vec![col("b", &input_schema)?]) + .schema(Arc::clone(&input_schema)) + .alias("AVG(b)") + .build()?, + ]; for (version, groups, aggregates) in [ (0, groups_none, aggregates_v0), @@ -1882,12 +1881,13 @@ mod tests { let groups = PhysicalGroupBy::default(); - let aggregates: Vec> = vec![ - AggregateExprBuilder::new(avg_udaf(), vec![col("a", &schema)?]) - .schema(Arc::clone(&schema)) - .alias("AVG(a)") - .build()?, - ]; + let aggregates: Vec = + vec![ + AggregateExprBuilder::new(avg_udaf(), vec![col("a", &schema)?]) + .schema(Arc::clone(&schema)) + .alias("AVG(a)") + .build()?, + ]; let blocking_exec = Arc::new(BlockingExec::new(Arc::clone(&schema), 1)); let refs = blocking_exec.refs(); @@ -1921,12 +1921,13 @@ mod tests { let groups = PhysicalGroupBy::new_single(vec![(col("a", &schema)?, "a".to_string())]); - let aggregates: Vec> = vec![ - AggregateExprBuilder::new(avg_udaf(), vec![col("b", &schema)?]) - .schema(Arc::clone(&schema)) - .alias("AVG(b)") - .build()?, - ]; + let aggregates: Vec = + vec![ + AggregateExprBuilder::new(avg_udaf(), vec![col("b", &schema)?]) + .schema(Arc::clone(&schema)) + .alias("AVG(b)") + .build()?, + ]; let blocking_exec = Arc::new(BlockingExec::new(Arc::clone(&schema), 1)); let refs = blocking_exec.refs(); @@ -1971,7 +1972,7 @@ mod tests { fn test_first_value_agg_expr( schema: &Schema, sort_options: SortOptions, - ) -> Result> { + ) -> Result { let ordering_req = [PhysicalSortExpr { expr: col("b", schema)?, options: sort_options, @@ -1989,7 +1990,7 @@ mod tests { fn test_last_value_agg_expr( schema: &Schema, sort_options: SortOptions, - ) -> Result> { + ) -> Result { let ordering_req = [PhysicalSortExpr { expr: col("b", schema)?, options: sort_options, @@ -2044,7 +2045,7 @@ mod tests { descending: false, nulls_first: false, }; - let aggregates: Vec> = if is_first_acc { + let aggregates: Vec = if is_first_acc { vec![test_first_value_agg_expr(&schema, sort_options)?] } else { vec![test_last_value_agg_expr(&schema, sort_options)?] @@ -2209,7 +2210,7 @@ mod tests { }; let groups = PhysicalGroupBy::new_single(vec![(col_a, "a".to_string())]); - let aggregates: Vec> = vec![ + let aggregates: Vec = vec![ test_first_value_agg_expr(&schema, option_desc)?, test_last_value_agg_expr(&schema, option_desc)?, ]; @@ -2267,7 +2268,7 @@ mod tests { ], ); - let aggregates: Vec> = + let aggregates: Vec = vec![AggregateExprBuilder::new(count_udaf(), vec![lit(1)]) .schema(Arc::clone(&schema)) .alias("1") diff --git a/datafusion/physical-plan/src/aggregates/row_hash.rs b/datafusion/physical-plan/src/aggregates/row_hash.rs index c38137994d44..fd2d26d9b49e 100644 --- a/datafusion/physical-plan/src/aggregates/row_hash.rs +++ b/datafusion/physical-plan/src/aggregates/row_hash.rs @@ -578,7 +578,7 @@ impl GroupedHashAggregateStream { /// that is supported by the aggregate, or a /// [`GroupsAccumulatorAdapter`] if not. pub(crate) fn create_group_accumulator( - agg_expr: &Arc, + agg_expr: &AggregateFunctionExpr, ) -> Result> { if agg_expr.groups_accumulator_supported() { agg_expr.create_groups_accumulator() @@ -588,7 +588,7 @@ pub(crate) fn create_group_accumulator( "Creating GroupsAccumulatorAdapter for {}: {agg_expr:?}", agg_expr.name() ); - let agg_expr_captured = Arc::clone(agg_expr); + let agg_expr_captured = agg_expr.clone(); let factory = move || agg_expr_captured.create_accumulator(); Ok(Box::new(GroupsAccumulatorAdapter::new(factory))) } diff --git a/datafusion/physical-plan/src/windows/mod.rs b/datafusion/physical-plan/src/windows/mod.rs index 56823e6dec2d..0275cd2441a9 100644 --- a/datafusion/physical-plan/src/windows/mod.rs +++ b/datafusion/physical-plan/src/windows/mod.rs @@ -141,7 +141,7 @@ fn window_expr_from_aggregate_expr( partition_by: &[Arc], order_by: &[PhysicalSortExpr], window_frame: Arc, - aggregate: Arc, + aggregate: AggregateFunctionExpr, ) -> Arc { // Is there a potentially unlimited sized window frame? let unbounded_window = window_frame.start_bound.is_unbounded(); diff --git a/datafusion/proto/src/physical_plan/mod.rs b/datafusion/proto/src/physical_plan/mod.rs index 74b6073a415e..e1cc37091bf7 100644 --- a/datafusion/proto/src/physical_plan/mod.rs +++ b/datafusion/proto/src/physical_plan/mod.rs @@ -481,7 +481,7 @@ impl AsExecutionPlan for protobuf::PhysicalPlanNode { }) .collect::, _>>()?; - let physical_aggr_expr: Vec> = hash_agg + let physical_aggr_expr: Vec = hash_agg .aggr_expr .iter() .zip(hash_agg.aggr_expr_name.iter()) diff --git a/datafusion/proto/src/physical_plan/to_proto.rs b/datafusion/proto/src/physical_plan/to_proto.rs index 25be7de61cc3..6981c77228a8 100644 --- a/datafusion/proto/src/physical_plan/to_proto.rs +++ b/datafusion/proto/src/physical_plan/to_proto.rs @@ -49,7 +49,7 @@ use crate::protobuf::{ use super::PhysicalExtensionCodec; pub fn serialize_physical_aggr_expr( - aggr_expr: Arc, + aggr_expr: AggregateFunctionExpr, codec: &dyn PhysicalExtensionCodec, ) -> Result { let expressions = serialize_physical_exprs(&aggr_expr.expressions(), codec)?; @@ -171,7 +171,7 @@ pub fn serialize_physical_window_expr( expr.downcast_ref::() { serialize_physical_window_aggr_expr( - plain_aggr_window_expr.get_aggregate_expr().as_ref(), + plain_aggr_window_expr.get_aggregate_expr(), window_frame, codec, )? @@ -179,7 +179,7 @@ pub fn serialize_physical_window_expr( expr.downcast_ref::() { serialize_physical_window_aggr_expr( - sliding_aggr_window_expr.get_aggregate_expr().as_ref(), + sliding_aggr_window_expr.get_aggregate_expr(), window_frame, codec, )? diff --git a/datafusion/proto/tests/cases/roundtrip_physical_plan.rs b/datafusion/proto/tests/cases/roundtrip_physical_plan.rs index 97d65ba87197..58f6015ee336 100644 --- a/datafusion/proto/tests/cases/roundtrip_physical_plan.rs +++ b/datafusion/proto/tests/cases/roundtrip_physical_plan.rs @@ -364,7 +364,7 @@ fn rountrip_aggregate() -> Result<()> { .alias("NTH_VALUE(b, 1)") .build()?; - let test_cases: Vec>> = vec![ + let test_cases: Vec> = vec![ // AVG vec![avg_expr], // NTH_VALUE @@ -397,7 +397,7 @@ fn rountrip_aggregate_with_limit() -> Result<()> { let groups: Vec<(Arc, String)> = vec![(col("a", &schema)?, "unused".to_string())]; - let aggregates: Vec> = + let aggregates: Vec = vec![ AggregateExprBuilder::new(avg_udaf(), vec![col("b", &schema)?]) .schema(Arc::clone(&schema)) @@ -426,7 +426,7 @@ fn rountrip_aggregate_with_approx_pencentile_cont() -> Result<()> { let groups: Vec<(Arc, String)> = vec![(col("a", &schema)?, "unused".to_string())]; - let aggregates: Vec> = vec![AggregateExprBuilder::new( + let aggregates: Vec = vec![AggregateExprBuilder::new( approx_percentile_cont_udaf(), vec![col("b", &schema)?, lit(0.5)], ) @@ -461,7 +461,7 @@ fn rountrip_aggregate_with_sort() -> Result<()> { }, }]; - let aggregates: Vec> = + let aggregates: Vec = vec![ AggregateExprBuilder::new(array_agg_udaf(), vec![col("b", &schema)?]) .schema(Arc::clone(&schema)) @@ -528,7 +528,7 @@ fn roundtrip_aggregate_udaf() -> Result<()> { let groups: Vec<(Arc, String)> = vec![(col("a", &schema)?, "unused".to_string())]; - let aggregates: Vec> = + let aggregates: Vec = vec![ AggregateExprBuilder::new(Arc::new(udaf), vec![col("b", &schema)?]) .schema(Arc::clone(&schema)) From 383bca379649d06f0fb11fd29f2a86998c23398e Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 9 Sep 2024 07:19:21 -0400 Subject: [PATCH 120/154] Update sqllogictest requirement from 0.21.0 to 0.22.0 (#12388) Updates the requirements on [sqllogictest](https://github.com/risinglightdb/sqllogictest-rs) to permit the latest version. - [Release notes](https://github.com/risinglightdb/sqllogictest-rs/releases) - [Changelog](https://github.com/risinglightdb/sqllogictest-rs/blob/main/CHANGELOG.md) - [Commits](https://github.com/risinglightdb/sqllogictest-rs/compare/v0.21.0...v0.22.0) --- updated-dependencies: - dependency-name: sqllogictest dependency-type: direct:production ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- datafusion/sqllogictest/Cargo.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/sqllogictest/Cargo.toml b/datafusion/sqllogictest/Cargo.toml index 36aff613962b..07dbc60e86bc 100644 --- a/datafusion/sqllogictest/Cargo.toml +++ b/datafusion/sqllogictest/Cargo.toml @@ -51,7 +51,7 @@ 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.21.0" +sqllogictest = "0.22.0" sqlparser = { workspace = true } tempfile = { workspace = true } thiserror = { workspace = true } From 6316849166732d2439c45dfb1281a2b2f58a72c8 Mon Sep 17 00:00:00 2001 From: Tai Le Manh <49281946+tlm365@users.noreply.github.com> Date: Mon, 9 Sep 2024 18:19:52 +0700 Subject: [PATCH 121/154] Support StringView for binary operators (#12212) * Support StringView for binary operators Signed-off-by: Tai Le Manh * Fix cargo fmt * Add and update unit tests Signed-off-by: Tai Le Manh * Fix clippy warning --------- Signed-off-by: Tai Le Manh --- .../expr-common/src/type_coercion/binary.rs | 24 +++++- datafusion/functions/src/regex/regexpmatch.rs | 6 +- .../src/unwrap_cast_in_comparison.rs | 52 ++++++----- .../physical-expr/src/expressions/binary.rs | 60 +++++++++++-- .../sqllogictest/test_files/string_view.slt | 86 ++++++++++++++++++- 5 files changed, 195 insertions(+), 33 deletions(-) diff --git a/datafusion/expr-common/src/type_coercion/binary.rs b/datafusion/expr-common/src/type_coercion/binary.rs index 3617f56905a9..d1144ed40759 100644 --- a/datafusion/expr-common/src/type_coercion/binary.rs +++ b/datafusion/expr-common/src/type_coercion/binary.rs @@ -984,6 +984,26 @@ fn string_coercion(lhs_type: &DataType, rhs_type: &DataType) -> Option } } +/// This will be deprecated when binary operators native support +/// for Utf8View (use `string_coercion` instead). +fn regex_comparison_string_coercion( + lhs_type: &DataType, + rhs_type: &DataType, +) -> Option { + use arrow::datatypes::DataType::*; + match (lhs_type, rhs_type) { + // If Utf8View is in any side, we coerce to Utf8. + (Utf8View, Utf8View | Utf8 | LargeUtf8) | (Utf8 | LargeUtf8, Utf8View) => { + Some(Utf8) + } + // Then, if LargeUtf8 is in any side, we coerce to LargeUtf8. + (LargeUtf8, Utf8 | LargeUtf8) | (Utf8, LargeUtf8) => Some(LargeUtf8), + // Utf8 coerces to Utf8 + (Utf8, Utf8) => Some(Utf8), + _ => None, + } +} + fn numeric_string_coercion(lhs_type: &DataType, rhs_type: &DataType) -> Option { use arrow::datatypes::DataType::*; match (lhs_type, rhs_type) { @@ -1072,10 +1092,10 @@ fn regex_null_coercion(lhs_type: &DataType, rhs_type: &DataType) -> Option Option { - string_coercion(lhs_type, rhs_type) + regex_comparison_string_coercion(lhs_type, rhs_type) .or_else(|| dictionary_comparison_coercion(lhs_type, rhs_type, false)) .or_else(|| regex_null_coercion(lhs_type, rhs_type)) } diff --git a/datafusion/functions/src/regex/regexpmatch.rs b/datafusion/functions/src/regex/regexpmatch.rs index 764acd7de757..bf40eff11d30 100644 --- a/datafusion/functions/src/regex/regexpmatch.rs +++ b/datafusion/functions/src/regex/regexpmatch.rs @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -//! Regx expressions +//! Regex expressions use arrow::array::{Array, ArrayRef, OffsetSizeTrait}; use arrow::compute::kernels::regexp; use arrow::datatypes::DataType; @@ -49,6 +49,10 @@ impl RegexpMatchFunc { Self { signature: Signature::one_of( vec![ + // Planner attempts coercion to the target type starting with the most preferred candidate. + // For example, given input `(Utf8View, Utf8)`, it first tries coercing to `(Utf8, Utf8)`. + // If that fails, it proceeds to `(LargeUtf8, Utf8)`. + // TODO: Native support Utf8View for regexp_match. Exact(vec![Utf8, Utf8]), Exact(vec![LargeUtf8, Utf8]), Exact(vec![Utf8, Utf8, Utf8]), diff --git a/datafusion/optimizer/src/unwrap_cast_in_comparison.rs b/datafusion/optimizer/src/unwrap_cast_in_comparison.rs index 318badc08823..2118ae96a198 100644 --- a/datafusion/optimizer/src/unwrap_cast_in_comparison.rs +++ b/datafusion/optimizer/src/unwrap_cast_in_comparison.rs @@ -159,20 +159,26 @@ impl TreeNodeRewriter for UnwrapCastExprRewriter { expr: right_expr, .. }), ) => { - // if the left_lit_value can be casted to the type of expr + // if the left_lit_value can be cast to the type of expr // we need to unwrap the cast for cast/try_cast expr, and add cast to the literal let Ok(expr_type) = right_expr.get_type(&self.schema) else { return Ok(Transformed::no(expr)); }; - let Some(value) = - try_cast_literal_to_type(left_lit_value, &expr_type) - else { - return Ok(Transformed::no(expr)); - }; - **left = lit(value); - // unwrap the cast/try_cast for the right expr - **right = mem::take(right_expr); - Ok(Transformed::yes(expr)) + match expr_type { + // https://github.com/apache/datafusion/issues/12180 + DataType::Utf8View => Ok(Transformed::no(expr)), + _ => { + let Some(value) = + try_cast_literal_to_type(left_lit_value, &expr_type) + else { + return Ok(Transformed::no(expr)); + }; + **left = lit(value); + // unwrap the cast/try_cast for the right expr + **right = mem::take(right_expr); + Ok(Transformed::yes(expr)) + } + } } ( Expr::TryCast(TryCast { @@ -183,20 +189,26 @@ impl TreeNodeRewriter for UnwrapCastExprRewriter { }), Expr::Literal(right_lit_value), ) => { - // if the right_lit_value can be casted to the type of expr + // if the right_lit_value can be cast to the type of expr // we need to unwrap the cast for cast/try_cast expr, and add cast to the literal let Ok(expr_type) = left_expr.get_type(&self.schema) else { return Ok(Transformed::no(expr)); }; - let Some(value) = - try_cast_literal_to_type(right_lit_value, &expr_type) - else { - return Ok(Transformed::no(expr)); - }; - // unwrap the cast/try_cast for the left expr - **left = mem::take(left_expr); - **right = lit(value); - Ok(Transformed::yes(expr)) + match expr_type { + // https://github.com/apache/datafusion/issues/12180 + DataType::Utf8View => Ok(Transformed::no(expr)), + _ => { + let Some(value) = + try_cast_literal_to_type(right_lit_value, &expr_type) + else { + return Ok(Transformed::no(expr)); + }; + // unwrap the cast/try_cast for the left expr + **left = mem::take(left_expr); + **right = lit(value); + Ok(Transformed::yes(expr)) + } + } } _ => Ok(Transformed::no(expr)), } diff --git a/datafusion/physical-expr/src/expressions/binary.rs b/datafusion/physical-expr/src/expressions/binary.rs index 08c133d7193a..7500c3e2651d 100644 --- a/datafusion/physical-expr/src/expressions/binary.rs +++ b/datafusion/physical-expr/src/expressions/binary.rs @@ -148,12 +148,12 @@ fn boolean_op( macro_rules! binary_string_array_flag_op { ($LEFT:expr, $RIGHT:expr, $OP:ident, $NOT:expr, $FLAG:expr) => {{ match $LEFT.data_type() { - DataType::Utf8 => { + DataType::Utf8View | DataType::Utf8 => { compute_utf8_flag_op!($LEFT, $RIGHT, $OP, StringArray, $NOT, $FLAG) - } + }, DataType::LargeUtf8 => { compute_utf8_flag_op!($LEFT, $RIGHT, $OP, LargeStringArray, $NOT, $FLAG) - } + }, other => internal_err!( "Data type {:?} not supported for binary_string_array_flag_op operation '{}' on string array", other, stringify!($OP) @@ -190,12 +190,12 @@ macro_rules! compute_utf8_flag_op { macro_rules! binary_string_array_flag_op_scalar { ($LEFT:expr, $RIGHT:expr, $OP:ident, $NOT:expr, $FLAG:expr) => {{ let result: Result> = match $LEFT.data_type() { - DataType::Utf8 => { + DataType::Utf8View | DataType::Utf8 => { compute_utf8_flag_op_scalar!($LEFT, $RIGHT, $OP, StringArray, $NOT, $FLAG) - } + }, DataType::LargeUtf8 => { compute_utf8_flag_op_scalar!($LEFT, $RIGHT, $OP, LargeStringArray, $NOT, $FLAG) - } + }, other => internal_err!( "Data type {:?} not supported for binary_string_array_flag_op_scalar operation '{}' on string array", other, stringify!($OP) @@ -937,6 +937,54 @@ mod tests { DataType::Boolean, [true, false], ); + test_coercion!( + StringViewArray, + DataType::Utf8View, + vec!["abc"; 5], + StringArray, + DataType::Utf8, + vec!["^a", "^A", "(b|d)", "(B|D)", "^(b|c)"], + Operator::RegexMatch, + BooleanArray, + DataType::Boolean, + [true, false, true, false, false], + ); + test_coercion!( + StringViewArray, + DataType::Utf8View, + vec!["abc"; 5], + StringArray, + DataType::Utf8, + vec!["^a", "^A", "(b|d)", "(B|D)", "^(b|c)"], + Operator::RegexIMatch, + BooleanArray, + DataType::Boolean, + [true, true, true, true, false], + ); + test_coercion!( + StringArray, + DataType::Utf8, + vec!["abc"; 5], + StringViewArray, + DataType::Utf8View, + vec!["^a", "^A", "(b|d)", "(B|D)", "^(b|c)"], + Operator::RegexNotMatch, + BooleanArray, + DataType::Boolean, + [false, true, false, true, true], + ); + test_coercion!( + StringArray, + DataType::Utf8, + vec!["abc"; 5], + StringViewArray, + DataType::Utf8View, + vec!["^a", "^A", "(b|d)", "(B|D)", "^(b|c)"], + Operator::RegexNotIMatch, + BooleanArray, + DataType::Boolean, + [false, false, false, false, true], + ); test_coercion!( StringArray, DataType::Utf8, diff --git a/datafusion/sqllogictest/test_files/string_view.slt b/datafusion/sqllogictest/test_files/string_view.slt index eb625e530b66..f478ba47aa4c 100644 --- a/datafusion/sqllogictest/test_files/string_view.slt +++ b/datafusion/sqllogictest/test_files/string_view.slt @@ -1104,7 +1104,6 @@ Rap (empty) Raph NULL NULL NULL ## Ensure no casts for RPAD -## TODO file ticket query TT EXPLAIN SELECT RPAD(column1_utf8view, 1) as c1, @@ -1134,7 +1133,6 @@ logical_plan 02)--TableScan: test projection=[column1_utf8view, column2_utf8view] ## Ensure no casts for SPLIT_PART -## TODO file ticket query TT EXPLAIN SELECT SPLIT_PART(column1_utf8view, 'f', 1) as c1, @@ -1146,7 +1144,6 @@ logical_plan 02)--TableScan: test projection=[column1_utf8view] ## Ensure no casts for STRPOS -## TODO file ticket query TT EXPLAIN SELECT STRPOS(column1_utf8view, 'f') as c, @@ -1158,7 +1155,6 @@ logical_plan 02)--TableScan: test projection=[column1_utf8view, column2_utf8view] ## Ensure no casts for SUBSTR -## TODO file ticket query TT EXPLAIN SELECT SUBSTR(column1_utf8view, 1) as c, @@ -1289,6 +1285,88 @@ XiangpengXiangpeng XiangpengXiangpeng XiangpengXiangpeng RaphaelRaphael RaphaelRaphael RaphaelRaphael NULL NULL NULL +## Ensure no casts for binary operators +## TODO: https://github.com/apache/datafusion/issues/12180 +# `~` operator (regex match) +query TT +EXPLAIN SELECT + column1_utf8view ~ 'an' AS c1 +FROM test; +---- +logical_plan +01)Projection: CAST(test.column1_utf8view AS Utf8) LIKE Utf8("%an%") AS c1 +02)--TableScan: test projection=[column1_utf8view] + +query B +SELECT + column1_utf8view ~ 'an' AS c1 +FROM test; +---- +false +true +false +NULL + +# `~*` operator (regex match case-insensitive) +query TT +EXPLAIN SELECT + column1_utf8view ~* '^a.{3}e' AS c1 +FROM test; +---- +logical_plan +01)Projection: CAST(test.column1_utf8view AS Utf8) ~* Utf8("^a.{3}e") AS c1 +02)--TableScan: test projection=[column1_utf8view] + +query B +SELECT + column1_utf8view ~* '^a.{3}e' AS c1 +FROM test; +---- +true +false +false +NULL + +# `!~~` operator (not like match) +query TT +EXPLAIN SELECT + column1_utf8view !~~ 'xia_g%g' AS c1 +FROM test; +---- +logical_plan +01)Projection: CAST(test.column1_utf8view AS Utf8) !~~ Utf8("xia_g%g") AS c1 +02)--TableScan: test projection=[column1_utf8view] + +query B +SELECT + column1_utf8view !~~ 'xia_g%g' AS c1 +FROM test; +---- +true +true +true +NULL + +# `!~~*` operator (not like match case-insensitive) +query TT +EXPLAIN SELECT + column1_utf8view !~~* 'xia_g%g' AS c1 +FROM test; +---- +logical_plan +01)Projection: CAST(test.column1_utf8view AS Utf8) !~~* Utf8("xia_g%g") AS c1 +02)--TableScan: test projection=[column1_utf8view] + +query B +SELECT + column1_utf8view !~~* 'xia_g%g' AS c1 +FROM test; +---- +true +false +true +NULL + statement ok drop table test; From 391f5cb96e224c3234aadc922fcb33f37d2af5c8 Mon Sep 17 00:00:00 2001 From: wiedld Date: Mon, 9 Sep 2024 01:20:10 -1000 Subject: [PATCH 122/154] Coerce BinaryView/Utf8View to LargeBinary/LargeUtf8 on output. (#12271) * chore(12119): add expand_views_at_output optimizer config option, default to false * feat(12119): coerce output based upon expand_views_at_output * test(12119): test coercion of view types, to non-view types, on output. * chore(12119): config.md for new config option * refactor(12119): code cleanup * test(12119): more clearly define scenarios in viewtype-coercion test coverage, and expand with PlanD scenario --- datafusion/common/src/config.rs | 5 + datafusion/expr/src/logical_plan/plan.rs | 2 +- .../optimizer/src/analyzer/type_coercion.rs | 231 +++++++++++++++++- datafusion/optimizer/src/test/mod.rs | 11 + .../test_files/information_schema.slt | 2 + docs/source/user-guide/configs.md | 1 + 6 files changed, 245 insertions(+), 7 deletions(-) diff --git a/datafusion/common/src/config.rs b/datafusion/common/src/config.rs index 19978e102cc8..d77655e8a77f 100644 --- a/datafusion/common/src/config.rs +++ b/datafusion/common/src/config.rs @@ -606,6 +606,11 @@ config_namespace! { /// When set to true, the optimizer will not attempt to convert Union to Interleave pub prefer_existing_union: bool, default = false + + /// When set to true, if the returned type is a view type + /// then the output will be coerced to a non-view. + /// Coerces `Utf8View` to `LargeUtf8`, and `BinaryView` to `LargeBinary`. + pub expand_views_at_output: bool, default = false } } diff --git a/datafusion/expr/src/logical_plan/plan.rs b/datafusion/expr/src/logical_plan/plan.rs index 20adeb7cc808..975bfc9feebf 100644 --- a/datafusion/expr/src/logical_plan/plan.rs +++ b/datafusion/expr/src/logical_plan/plan.rs @@ -1393,7 +1393,7 @@ impl LogicalPlan { /// referenced expressions into columns. /// /// See also: [`crate::utils::columnize_expr`] - pub(crate) fn columnized_output_exprs(&self) -> Result> { + pub fn columnized_output_exprs(&self) -> Result> { match self { LogicalPlan::Aggregate(aggregate) => Ok(aggregate .output_expressions()? diff --git a/datafusion/optimizer/src/analyzer/type_coercion.rs b/datafusion/optimizer/src/analyzer/type_coercion.rs index a0b5c9552c83..7a8746572cfd 100644 --- a/datafusion/optimizer/src/analyzer/type_coercion.rs +++ b/datafusion/optimizer/src/analyzer/type_coercion.rs @@ -21,7 +21,7 @@ use std::sync::Arc; use itertools::izip; -use arrow::datatypes::{DataType, Field, IntervalUnit}; +use arrow::datatypes::{DataType, Field, IntervalUnit, Schema}; use crate::analyzer::AnalyzerRule; use crate::utils::NamePreserver; @@ -29,7 +29,7 @@ use datafusion_common::config::ConfigOptions; use datafusion_common::tree_node::{Transformed, TreeNode, TreeNodeRewriter}; use datafusion_common::{ exec_err, internal_err, not_impl_err, plan_datafusion_err, plan_err, Column, - DFSchema, DFSchemaRef, DataFusionError, Result, ScalarValue, + DFSchema, DFSchemaRef, DataFusionError, Result, ScalarValue, TableReference, }; use datafusion_expr::expr::{ self, Alias, Between, BinaryExpr, Case, Exists, InList, InSubquery, Like, @@ -66,19 +66,39 @@ impl TypeCoercion { } } +/// Coerce output schema based upon optimizer config. +fn coerce_output(plan: LogicalPlan, config: &ConfigOptions) -> Result { + if !config.optimizer.expand_views_at_output { + return Ok(plan); + } + + let outer_refs = plan.expressions(); + if outer_refs.is_empty() { + return Ok(plan); + } + + if let Some(dfschema) = transform_schema_to_nonview(plan.schema()) { + coerce_plan_expr_for_schema(plan, &dfschema?) + } else { + Ok(plan) + } +} + impl AnalyzerRule for TypeCoercion { fn name(&self) -> &str { "type_coercion" } - fn analyze(&self, plan: LogicalPlan, _: &ConfigOptions) -> Result { + fn analyze(&self, plan: LogicalPlan, config: &ConfigOptions) -> Result { let empty_schema = DFSchema::empty(); + // recurse let transformed_plan = plan .transform_up_with_subqueries(|plan| analyze_internal(&empty_schema, plan))? .data; - Ok(transformed_plan) + // finish + coerce_output(transformed_plan, config) } } @@ -514,6 +534,55 @@ impl<'a> TreeNodeRewriter for TypeCoercionRewriter<'a> { } } +/// Transform a schema to use non-view types for Utf8View and BinaryView +fn transform_schema_to_nonview(dfschema: &DFSchemaRef) -> Option> { + let metadata = dfschema.as_arrow().metadata.clone(); + let mut transformed = false; + + let (qualifiers, transformed_fields): (Vec>, Vec>) = + dfschema + .iter() + .map(|(qualifier, field)| match field.data_type() { + DataType::Utf8View => { + transformed = true; + ( + qualifier.cloned() as Option, + Arc::new(Field::new( + field.name(), + DataType::LargeUtf8, + field.is_nullable(), + )), + ) + } + DataType::BinaryView => { + transformed = true; + ( + qualifier.cloned() as Option, + Arc::new(Field::new( + field.name(), + DataType::LargeBinary, + field.is_nullable(), + )), + ) + } + _ => ( + qualifier.cloned() as Option, + Arc::clone(field), + ), + }) + .unzip(); + + if !transformed { + return None; + } + + let schema = Schema::new_with_metadata(transformed_fields, metadata); + Some(DFSchema::from_field_specific_qualified_schema( + qualifiers, + &Arc::new(schema), + )) +} + /// Casts the given `value` to `target_type`. Note that this function /// only considers `Null` or `Utf8` values. fn coerce_scalar(target_type: &DataType, value: &ScalarValue) -> Result { @@ -935,10 +1004,11 @@ mod test { use arrow::datatypes::DataType::Utf8; use arrow::datatypes::{DataType, Field, TimeUnit}; + use datafusion_common::config::ConfigOptions; use datafusion_common::tree_node::{TransformedResult, TreeNode}; use datafusion_common::{DFSchema, DFSchemaRef, Result, ScalarValue}; use datafusion_expr::expr::{self, InSubquery, Like, ScalarFunction}; - use datafusion_expr::logical_plan::{EmptyRelation, Projection}; + use datafusion_expr::logical_plan::{EmptyRelation, Projection, Sort}; use datafusion_expr::test::function_stub::avg_udaf; use datafusion_expr::{ cast, col, create_udaf, is_true, lit, AccumulatorFactoryFunction, AggregateUDF, @@ -951,7 +1021,7 @@ mod test { use crate::analyzer::type_coercion::{ coerce_case_expression, TypeCoercion, TypeCoercionRewriter, }; - use crate::test::assert_analyzed_plan_eq; + use crate::test::{assert_analyzed_plan_eq, assert_analyzed_plan_with_config_eq}; fn empty() -> Arc { Arc::new(LogicalPlan::EmptyRelation(EmptyRelation { @@ -982,6 +1052,155 @@ mod test { assert_analyzed_plan_eq(Arc::new(TypeCoercion::new()), plan, expected) } + fn coerce_on_output_if_viewtype(plan: LogicalPlan, expected: &str) -> Result<()> { + let mut options = ConfigOptions::default(); + options.optimizer.expand_views_at_output = true; + + assert_analyzed_plan_with_config_eq( + options, + Arc::new(TypeCoercion::new()), + plan.clone(), + expected, + ) + } + + fn do_not_coerce_on_output(plan: LogicalPlan, expected: &str) -> Result<()> { + assert_analyzed_plan_with_config_eq( + ConfigOptions::default(), + Arc::new(TypeCoercion::new()), + plan.clone(), + expected, + ) + } + + #[test] + fn coerce_utf8view_output() -> Result<()> { + // Plan A + // scenario: outermost utf8view projection + let expr = col("a"); + let empty = empty_with_type(DataType::Utf8View); + let plan = LogicalPlan::Projection(Projection::try_new( + vec![expr.clone()], + Arc::clone(&empty), + )?); + // Plan A: no coerce + let if_not_coerced = "Projection: a\n EmptyRelation"; + do_not_coerce_on_output(plan.clone(), if_not_coerced)?; + // Plan A: coerce requested: Utf8View => LargeUtf8 + let if_coerced = "Projection: CAST(a AS LargeUtf8)\n EmptyRelation"; + coerce_on_output_if_viewtype(plan.clone(), if_coerced)?; + + // Plan B + // scenario: outermost bool projection + let bool_expr = col("a").lt(lit("foo")); + let bool_plan = LogicalPlan::Projection(Projection::try_new( + vec![bool_expr], + Arc::clone(&empty), + )?); + // Plan B: no coerce + let if_not_coerced = + "Projection: a < CAST(Utf8(\"foo\") AS Utf8View)\n EmptyRelation"; + do_not_coerce_on_output(bool_plan.clone(), if_not_coerced)?; + // Plan B: coerce requested: no coercion applied + let if_coerced = if_not_coerced; + coerce_on_output_if_viewtype(bool_plan, if_coerced)?; + + // Plan C + // scenario: with a non-projection root logical plan node + let sort_expr = expr.sort(true, true); + let sort_plan = LogicalPlan::Sort(Sort { + expr: vec![sort_expr], + input: Arc::new(plan), + fetch: None, + }); + // Plan C: no coerce + let if_not_coerced = + "Sort: a ASC NULLS FIRST\n Projection: a\n EmptyRelation"; + do_not_coerce_on_output(sort_plan.clone(), if_not_coerced)?; + // Plan C: coerce requested: Utf8View => LargeUtf8 + let if_coerced = "Projection: CAST(a AS LargeUtf8)\n Sort: a ASC NULLS FIRST\n Projection: a\n EmptyRelation"; + coerce_on_output_if_viewtype(sort_plan.clone(), if_coerced)?; + + // Plan D + // scenario: two layers of projections with view types + let plan = LogicalPlan::Projection(Projection::try_new( + vec![col("a")], + Arc::new(sort_plan), + )?); + // Plan D: no coerce + let if_not_coerced = "Projection: a\n Sort: a ASC NULLS FIRST\n Projection: a\n EmptyRelation"; + do_not_coerce_on_output(plan.clone(), if_not_coerced)?; + // Plan B: coerce requested: Utf8View => LargeUtf8 only on outermost + let if_coerced = "Projection: CAST(a AS LargeUtf8)\n Sort: a ASC NULLS FIRST\n Projection: a\n EmptyRelation"; + coerce_on_output_if_viewtype(plan.clone(), if_coerced)?; + + Ok(()) + } + + #[test] + fn coerce_binaryview_output() -> Result<()> { + // Plan A + // scenario: outermost binaryview projection + let expr = col("a"); + let empty = empty_with_type(DataType::BinaryView); + let plan = LogicalPlan::Projection(Projection::try_new( + vec![expr.clone()], + Arc::clone(&empty), + )?); + // Plan A: no coerce + let if_not_coerced = "Projection: a\n EmptyRelation"; + do_not_coerce_on_output(plan.clone(), if_not_coerced)?; + // Plan A: coerce requested: BinaryView => LargeBinary + let if_coerced = "Projection: CAST(a AS LargeBinary)\n EmptyRelation"; + coerce_on_output_if_viewtype(plan.clone(), if_coerced)?; + + // Plan B + // scenario: outermost bool projection + let bool_expr = col("a").lt(lit(vec![8, 1, 8, 1])); + let bool_plan = LogicalPlan::Projection(Projection::try_new( + vec![bool_expr], + Arc::clone(&empty), + )?); + // Plan B: no coerce + let if_not_coerced = + "Projection: a < CAST(Binary(\"8,1,8,1\") AS BinaryView)\n EmptyRelation"; + do_not_coerce_on_output(bool_plan.clone(), if_not_coerced)?; + // Plan B: coerce requested: no coercion applied + let if_coerced = if_not_coerced; + coerce_on_output_if_viewtype(bool_plan, if_coerced)?; + + // Plan C + // scenario: with a non-projection root logical plan node + let sort_expr = expr.sort(true, true); + let sort_plan = LogicalPlan::Sort(Sort { + expr: vec![sort_expr], + input: Arc::new(plan), + fetch: None, + }); + // Plan C: no coerce + let if_not_coerced = + "Sort: a ASC NULLS FIRST\n Projection: a\n EmptyRelation"; + do_not_coerce_on_output(sort_plan.clone(), if_not_coerced)?; + // Plan C: coerce requested: BinaryView => LargeBinary + let if_coerced = "Projection: CAST(a AS LargeBinary)\n Sort: a ASC NULLS FIRST\n Projection: a\n EmptyRelation"; + coerce_on_output_if_viewtype(sort_plan.clone(), if_coerced)?; + + // Plan D + // scenario: two layers of projections with view types + let plan = LogicalPlan::Projection(Projection::try_new( + vec![col("a")], + Arc::new(sort_plan), + )?); + // Plan D: no coerce + let if_not_coerced = "Projection: a\n Sort: a ASC NULLS FIRST\n Projection: a\n EmptyRelation"; + do_not_coerce_on_output(plan.clone(), if_not_coerced)?; + // Plan B: coerce requested: BinaryView => LargeBinary only on outermost + let if_coerced = "Projection: CAST(a AS LargeBinary)\n Sort: a ASC NULLS FIRST\n Projection: a\n EmptyRelation"; + coerce_on_output_if_viewtype(plan.clone(), if_coerced)?; + + Ok(()) + } + #[test] fn nested_case() -> Result<()> { let expr = col("a").lt(lit(2_u32)); diff --git a/datafusion/optimizer/src/test/mod.rs b/datafusion/optimizer/src/test/mod.rs index 1266b548ab05..cabeafd8e7de 100644 --- a/datafusion/optimizer/src/test/mod.rs +++ b/datafusion/optimizer/src/test/mod.rs @@ -114,6 +114,17 @@ pub fn assert_analyzed_plan_eq( expected: &str, ) -> Result<()> { let options = ConfigOptions::default(); + assert_analyzed_plan_with_config_eq(options, rule, plan, expected)?; + + Ok(()) +} + +pub fn assert_analyzed_plan_with_config_eq( + options: ConfigOptions, + rule: Arc, + plan: LogicalPlan, + expected: &str, +) -> Result<()> { let analyzed_plan = Analyzer::with_rules(vec![rule]).execute_and_check(plan, &options, |_, _| {})?; let formatted_plan = format!("{analyzed_plan}"); diff --git a/datafusion/sqllogictest/test_files/information_schema.slt b/datafusion/sqllogictest/test_files/information_schema.slt index f797a7a6539d..beefa24ba4c6 100644 --- a/datafusion/sqllogictest/test_files/information_schema.slt +++ b/datafusion/sqllogictest/test_files/information_schema.slt @@ -225,6 +225,7 @@ datafusion.optimizer.default_filter_selectivity 20 datafusion.optimizer.enable_distinct_aggregation_soft_limit true datafusion.optimizer.enable_round_robin_repartition true datafusion.optimizer.enable_topk_aggregation true +datafusion.optimizer.expand_views_at_output false datafusion.optimizer.filter_null_join_keys false datafusion.optimizer.hash_join_single_partition_threshold 1048576 datafusion.optimizer.hash_join_single_partition_threshold_rows 131072 @@ -314,6 +315,7 @@ datafusion.optimizer.default_filter_selectivity 20 The default filter selectivit datafusion.optimizer.enable_distinct_aggregation_soft_limit true When set to true, the optimizer will push a limit operation into grouped aggregations which have no aggregate expressions, as a soft limit, emitting groups once the limit is reached, before all rows in the group are read. datafusion.optimizer.enable_round_robin_repartition true When set to true, the physical plan optimizer will try to add round robin repartitioning to increase parallelism to leverage more CPU cores datafusion.optimizer.enable_topk_aggregation true When set to true, the optimizer will attempt to perform limit operations during aggregations, if possible +datafusion.optimizer.expand_views_at_output false When set to true, if the returned type is a view type then the output will be coerced to a non-view. Coerces `Utf8View` to `LargeUtf8`, and `BinaryView` to `LargeBinary`. datafusion.optimizer.filter_null_join_keys false When set to true, the optimizer will insert filters before a join between a nullable and non-nullable column to filter out nulls on the nullable side. This filter can add additional overhead when the file format does not fully support predicate push down. datafusion.optimizer.hash_join_single_partition_threshold 1048576 The maximum estimated size in bytes for one input side of a HashJoin will be collected into a single partition datafusion.optimizer.hash_join_single_partition_threshold_rows 131072 The maximum estimated size in rows for one input side of a HashJoin will be collected into a single partition diff --git a/docs/source/user-guide/configs.md b/docs/source/user-guide/configs.md index 8514fb1fbd93..f7e25bd55850 100644 --- a/docs/source/user-guide/configs.md +++ b/docs/source/user-guide/configs.md @@ -111,6 +111,7 @@ Environment variables are read during `SessionConfig` initialisation so they mus | datafusion.optimizer.hash_join_single_partition_threshold_rows | 131072 | The maximum estimated size in rows for one input side of a HashJoin will be collected into a single partition | | datafusion.optimizer.default_filter_selectivity | 20 | The default filter selectivity used by Filter Statistics when an exact selectivity cannot be determined. Valid values are between 0 (no selectivity) and 100 (all rows are selected). | | datafusion.optimizer.prefer_existing_union | false | When set to true, the optimizer will not attempt to convert Union to Interleave | +| datafusion.optimizer.expand_views_at_output | false | When set to true, if the returned type is a view type then the output will be coerced to a non-view. Coerces `Utf8View` to `LargeUtf8`, and `BinaryView` to `LargeBinary`. | | datafusion.explain.logical_plan_only | false | When set to true, the explain statement will only print logical plans | | datafusion.explain.physical_plan_only | false | When set to true, the explain statement will only print physical plans | | datafusion.explain.show_statistics | false | When set to true, the explain statement will print operator statistics for physical plans | From 9001b73c064ee221ba51615fdcf635a20d4bd067 Mon Sep 17 00:00:00 2001 From: theirix Date: Mon, 9 Sep 2024 12:20:49 +0100 Subject: [PATCH 123/154] Support for SIMILAR TO for physical plan (#12350) * Handle SIMILAR TO for physical plan * Add sqllogictest test cases * Add unit tests for similar to * Fix type error --- .../physical-expr/src/expressions/binary.rs | 74 +++++++++++++++++++ .../physical-expr/src/expressions/mod.rs | 2 +- datafusion/physical-expr/src/planner.rs | 18 ++++- .../sqllogictest/test_files/strings.slt | 45 +++++++++++ 4 files changed, 137 insertions(+), 2 deletions(-) diff --git a/datafusion/physical-expr/src/expressions/binary.rs b/datafusion/physical-expr/src/expressions/binary.rs index 7500c3e2651d..e115ec3c74fe 100644 --- a/datafusion/physical-expr/src/expressions/binary.rs +++ b/datafusion/physical-expr/src/expressions/binary.rs @@ -681,6 +681,22 @@ pub fn binary( Ok(Arc::new(BinaryExpr::new(lhs, op, rhs))) } +/// Create a similar to expression +pub fn similar_to( + negated: bool, + case_insensitive: bool, + expr: Arc, + pattern: Arc, +) -> Result> { + let binary_op = match (negated, case_insensitive) { + (false, false) => Operator::RegexMatch, + (false, true) => Operator::RegexIMatch, + (true, false) => Operator::RegexNotMatch, + (true, true) => Operator::RegexNotIMatch, + }; + Ok(Arc::new(BinaryExpr::new(expr, binary_op, pattern))) +} + #[cfg(test)] mod tests { use super::*; @@ -4274,4 +4290,62 @@ mod tests { .contains("Overflow happened on: 2147483647 * 2")); Ok(()) } + + /// Test helper for SIMILAR TO binary operation + fn apply_similar_to( + schema: &SchemaRef, + va: Vec<&str>, + vb: Vec<&str>, + negated: bool, + case_insensitive: bool, + expected: &BooleanArray, + ) -> Result<()> { + let a = StringArray::from(va); + let b = StringArray::from(vb); + let op = similar_to( + negated, + case_insensitive, + col("a", schema)?, + col("b", schema)?, + )?; + let batch = + RecordBatch::try_new(Arc::clone(schema), vec![Arc::new(a), Arc::new(b)])?; + let result = op + .evaluate(&batch)? + .into_array(batch.num_rows()) + .expect("Failed to convert to array"); + assert_eq!(result.as_ref(), expected); + + Ok(()) + } + + #[test] + fn test_similar_to() { + let schema = Arc::new(Schema::new(vec![ + Field::new("a", DataType::Utf8, false), + Field::new("b", DataType::Utf8, false), + ])); + + let expected = [Some(true), Some(false)].iter().collect(); + // case-sensitive + apply_similar_to( + &schema, + vec!["hello world", "Hello World"], + vec!["hello.*", "hello.*"], + false, + false, + &expected, + ) + .unwrap(); + // case-insensitive + apply_similar_to( + &schema, + vec!["hello world", "bye"], + vec!["hello.*", "hello.*"], + false, + true, + &expected, + ) + .unwrap(); + } } diff --git a/datafusion/physical-expr/src/expressions/mod.rs b/datafusion/physical-expr/src/expressions/mod.rs index 87d8f04a6858..177fd799ae79 100644 --- a/datafusion/physical-expr/src/expressions/mod.rs +++ b/datafusion/physical-expr/src/expressions/mod.rs @@ -42,7 +42,7 @@ pub use crate::window::ntile::Ntile; pub use crate::window::rank::{dense_rank, percent_rank, rank, Rank, RankType}; pub use crate::PhysicalSortExpr; -pub use binary::{binary, BinaryExpr}; +pub use binary::{binary, similar_to, BinaryExpr}; pub use case::{case, CaseExpr}; pub use cast::{cast, CastExpr}; pub use column::{col, with_new_schema, Column}; diff --git a/datafusion/physical-expr/src/planner.rs b/datafusion/physical-expr/src/planner.rs index d015f545bf9d..bffc2c46fc1e 100644 --- a/datafusion/physical-expr/src/planner.rs +++ b/datafusion/physical-expr/src/planner.rs @@ -19,7 +19,7 @@ use std::sync::Arc; use crate::scalar_function; use crate::{ - expressions::{self, binary, like, Column, Literal}, + expressions::{self, binary, like, similar_to, Column, Literal}, PhysicalExpr, }; @@ -215,6 +215,22 @@ pub fn create_physical_expr( input_schema, ) } + Expr::SimilarTo(Like { + negated, + expr, + pattern, + escape_char, + case_insensitive, + }) => { + if escape_char.is_some() { + return exec_err!("SIMILAR TO does not support escape_char yet"); + } + let physical_expr = + create_physical_expr(expr, input_dfschema, execution_props)?; + let physical_pattern = + create_physical_expr(pattern, input_dfschema, execution_props)?; + similar_to(*negated, *case_insensitive, physical_expr, physical_pattern) + } Expr::Case(case) => { let expr: Option> = if let Some(e) = &case.expr { Some(create_physical_expr( diff --git a/datafusion/sqllogictest/test_files/strings.slt b/datafusion/sqllogictest/test_files/strings.slt index 30fb2d750d95..81b8f4b2da9a 100644 --- a/datafusion/sqllogictest/test_files/strings.slt +++ b/datafusion/sqllogictest/test_files/strings.slt @@ -46,6 +46,51 @@ P1m1e1 p1m1e1 p2m1e1 +# REGEX +query T rowsort +SELECT s FROM test WHERE s ~ 'p[12].*'; +---- +p1 +p1e1 +p1m1e1 +p2 +p2e1 +p2m1e1 + +# REGEX nocase +query T rowsort +SELECT s FROM test WHERE s ~* 'p[12].*'; +---- +P1 +P1e1 +P1m1e1 +p1 +p1e1 +p1m1e1 +p2 +p2e1 +p2m1e1 + +# SIMILAR TO +query T rowsort +SELECT s FROM test WHERE s SIMILAR TO 'p[12].*'; +---- +p1 +p1e1 +p1m1e1 +p2 +p2e1 +p2m1e1 + +# NOT SIMILAR TO +query T rowsort +SELECT s FROM test WHERE s NOT SIMILAR TO 'p[12].*'; +---- +P1 +P1e1 +P1m1e1 +e1 + # NOT LIKE query T rowsort SELECT s FROM test WHERE s NOT LIKE 'p1%'; From 6e790cf1f363652c3f2ed64286da75a1d5e88cb9 Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Mon, 9 Sep 2024 13:32:56 +0200 Subject: [PATCH 124/154] Remove deprecated expression optimizer's utils (#12390) They were deprecated since v 34. --- datafusion/optimizer/src/utils.rs | 181 +----------------------------- 1 file changed, 1 insertion(+), 180 deletions(-) diff --git a/datafusion/optimizer/src/utils.rs b/datafusion/optimizer/src/utils.rs index 9e602ad49715..b4d292f88313 100644 --- a/datafusion/optimizer/src/utils.rs +++ b/datafusion/optimizer/src/utils.rs @@ -23,8 +23,7 @@ use crate::{OptimizerConfig, OptimizerRule}; use datafusion_common::{Column, DFSchema, Result}; use datafusion_expr::expr_rewriter::replace_col; -use datafusion_expr::utils as expr_utils; -use datafusion_expr::{logical_plan::LogicalPlan, Expr, Operator}; +use datafusion_expr::{logical_plan::LogicalPlan, Expr}; use log::{debug, trace}; @@ -120,181 +119,3 @@ pub fn log_plan(description: &str, plan: &LogicalPlan) { debug!("{description}:\n{}\n", plan.display_indent()); trace!("{description}::\n{}\n", plan.display_indent_schema()); } - -/// Splits a conjunctive [`Expr`] such as `A AND B AND C` => `[A, B, C]` -/// -/// See [`split_conjunction_owned`] for more details and an example. -#[deprecated( - since = "34.0.0", - note = "use `datafusion_expr::utils::split_conjunction` instead" -)] -pub fn split_conjunction(expr: &Expr) -> Vec<&Expr> { - expr_utils::split_conjunction(expr) -} - -/// Splits an owned conjunctive [`Expr`] such as `A AND B AND C` => `[A, B, C]` -/// -/// This is often used to "split" filter expressions such as `col1 = 5 -/// AND col2 = 10` into [`col1 = 5`, `col2 = 10`]; -/// -/// # Example -/// ``` -/// # use datafusion_expr::{col, lit}; -/// # use datafusion_optimizer::utils::split_conjunction_owned; -/// // a=1 AND b=2 -/// let expr = col("a").eq(lit(1)).and(col("b").eq(lit(2))); -/// -/// // [a=1, b=2] -/// let split = vec![ -/// col("a").eq(lit(1)), -/// col("b").eq(lit(2)), -/// ]; -/// -/// // use split_conjunction_owned to split them -/// assert_eq!(split_conjunction_owned(expr), split); -/// ``` -#[deprecated( - since = "34.0.0", - note = "use `datafusion_expr::utils::split_conjunction_owned` instead" -)] -pub fn split_conjunction_owned(expr: Expr) -> Vec { - expr_utils::split_conjunction_owned(expr) -} - -/// Splits an owned binary operator tree [`Expr`] such as `A B C` => `[A, B, C]` -/// -/// This is often used to "split" expressions such as `col1 = 5 -/// AND col2 = 10` into [`col1 = 5`, `col2 = 10`]; -/// -/// # Example -/// ``` -/// # use datafusion_expr::{col, lit, Operator}; -/// # use datafusion_optimizer::utils::split_binary_owned; -/// # use std::ops::Add; -/// // a=1 + b=2 -/// let expr = col("a").eq(lit(1)).add(col("b").eq(lit(2))); -/// -/// // [a=1, b=2] -/// let split = vec![ -/// col("a").eq(lit(1)), -/// col("b").eq(lit(2)), -/// ]; -/// -/// // use split_binary_owned to split them -/// assert_eq!(split_binary_owned(expr, Operator::Plus), split); -/// ``` -#[deprecated( - since = "34.0.0", - note = "use `datafusion_expr::utils::split_binary_owned` instead" -)] -pub fn split_binary_owned(expr: Expr, op: Operator) -> Vec { - expr_utils::split_binary_owned(expr, op) -} - -/// Splits an binary operator tree [`Expr`] such as `A B C` => `[A, B, C]` -/// -/// See [`split_binary_owned`] for more details and an example. -#[deprecated( - since = "34.0.0", - note = "use `datafusion_expr::utils::split_binary` instead" -)] -pub fn split_binary(expr: &Expr, op: Operator) -> Vec<&Expr> { - expr_utils::split_binary(expr, op) -} - -/// Combines an array of filter expressions into a single filter -/// expression consisting of the input filter expressions joined with -/// logical AND. -/// -/// Returns None if the filters array is empty. -/// -/// # Example -/// ``` -/// # use datafusion_expr::{col, lit}; -/// # use datafusion_optimizer::utils::conjunction; -/// // a=1 AND b=2 -/// let expr = col("a").eq(lit(1)).and(col("b").eq(lit(2))); -/// -/// // [a=1, b=2] -/// let split = vec![ -/// col("a").eq(lit(1)), -/// col("b").eq(lit(2)), -/// ]; -/// -/// // use conjunction to join them together with `AND` -/// assert_eq!(conjunction(split), Some(expr)); -/// ``` -#[deprecated( - since = "34.0.0", - note = "use `datafusion_expr::utils::conjunction` instead" -)] -pub fn conjunction(filters: impl IntoIterator) -> Option { - expr_utils::conjunction(filters) -} - -/// Combines an array of filter expressions into a single filter -/// expression consisting of the input filter expressions joined with -/// logical OR. -/// -/// Returns None if the filters array is empty. -#[deprecated( - since = "34.0.0", - note = "use `datafusion_expr::utils::disjunction` instead" -)] -pub fn disjunction(filters: impl IntoIterator) -> Option { - expr_utils::disjunction(filters) -} - -/// returns a new [LogicalPlan] that wraps `plan` in a [LogicalPlan::Filter] with -/// its predicate be all `predicates` ANDed. -#[deprecated( - since = "34.0.0", - note = "use `datafusion_expr::utils::add_filter` instead" -)] -pub fn add_filter(plan: LogicalPlan, predicates: &[&Expr]) -> Result { - expr_utils::add_filter(plan, predicates) -} - -/// Looks for correlating expressions: for example, a binary expression with one field from the subquery, and -/// one not in the subquery (closed upon from outer scope) -/// -/// # Arguments -/// -/// * `exprs` - List of expressions that may or may not be joins -/// -/// # Return value -/// -/// Tuple of (expressions containing joins, remaining non-join expressions) -#[deprecated( - since = "34.0.0", - note = "use `datafusion_expr::utils::find_join_exprs` instead" -)] -pub fn find_join_exprs(exprs: Vec<&Expr>) -> Result<(Vec, Vec)> { - expr_utils::find_join_exprs(exprs) -} - -/// Returns the first (and only) element in a slice, or an error -/// -/// # Arguments -/// -/// * `slice` - The slice to extract from -/// -/// # Return value -/// -/// The first element, or an error -#[deprecated( - since = "34.0.0", - note = "use `datafusion_expr::utils::only_or_err` instead" -)] -pub fn only_or_err(slice: &[T]) -> Result<&T> { - expr_utils::only_or_err(slice) -} - -/// merge inputs schema into a single schema. -#[deprecated( - since = "34.0.0", - note = "use `datafusion_expr::utils::merge_schema` instead" -)] -pub fn merge_schema(inputs: Vec<&LogicalPlan>) -> DFSchema { - expr_utils::merge_schema(&inputs) -} From f56d6d033adc67c18eb5eca1d5429e90a231eb39 Mon Sep 17 00:00:00 2001 From: waruto Date: Mon, 9 Sep 2024 21:04:59 +0800 Subject: [PATCH 125/154] remove redundant usage of clone (#12392) --- .../core/src/datasource/physical_plan/parquet/opener.rs | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/datafusion/core/src/datasource/physical_plan/parquet/opener.rs b/datafusion/core/src/datasource/physical_plan/parquet/opener.rs index 4edc0ac525de..7e331293eb19 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet/opener.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet/opener.rs @@ -77,8 +77,9 @@ impl FileOpener for ParquetOpener { )?; let batch_size = self.batch_size; - let projection = self.projection.clone(); - let projected_schema = SchemaRef::from(self.table_schema.project(&projection)?); + + let projected_schema = + SchemaRef::from(self.table_schema.project(&self.projection)?); let schema_adapter = self.schema_adapter_factory.create(projected_schema); let predicate = self.predicate.clone(); let pruning_predicate = self.pruning_predicate.clone(); From 9bc39a0522840ed90de2a4d23157de2e192cd00f Mon Sep 17 00:00:00 2001 From: Jax Liu Date: Mon, 9 Sep 2024 21:09:07 +0800 Subject: [PATCH 126/154] Introduce the `DynamicFileCatalog` in `datafusion-catalog` (#11035) * early draft * fmt * add example for dynamic file query * add test and refactor * clippy and add doc * cargo fmt * extract substitute_tilde function * fix the error handling * fmt and clippy * fix test * fix sqllogictests * ignore dirs for windows test * enhance the test for every file format * disable the test for windows * make dynamic file query configurable * revert array_query.slt * modified the test and add example * make dirs be optional * enable dynamic file query in cli * cargo fmt * modified example * fix test * fix merge conflict * tmp * tmp * tmp * fix the catalog and schema * move dynamic file catalog to datafusion-catalog * add copyright * fix tests * rename catalog in cli and update lock * enable home_dir feature * update lock * fix compile * fix clippy * fmt toml * fix doc test and add more doc * fix clippy * add home_dir feature doc * rollback the unused changed * update lock * fix sqllogictest * separate dynamic file test to another slt * add test for querying url table but disabled this feature * add dynamic_file.slt * remove home_dir feature * update cli lock * fix msrv check * fix msrv check * rollback the lock change * address review comment and enhance the doc * remove the legacy comment * add missing doc --- datafusion-cli/Cargo.lock | 1 + datafusion-cli/src/catalog.rs | 76 ++++---- datafusion-cli/src/main.rs | 10 +- datafusion-examples/examples/dataframe.rs | 6 + .../external_dependency/query-aws-s3.rs | 9 + datafusion/catalog/Cargo.toml | 1 + .../catalog/src/dynamic_file/catalog.rs | 183 ++++++++++++++++++ datafusion/catalog/src/dynamic_file/mod.rs | 18 ++ datafusion/catalog/src/lib.rs | 2 + datafusion/catalog/src/session.rs | 34 +++- .../core/src/datasource/dynamic_file.rs | 80 ++++++++ datafusion/core/src/datasource/mod.rs | 1 + datafusion/core/src/execution/context/mod.rs | 90 ++++++++- datafusion/sqllogictest/src/test_context.rs | 3 + .../sqllogictest/test_files/arrow_files.slt | 5 + .../sqllogictest/test_files/csv_files.slt | 5 + .../sqllogictest/test_files/describe.slt | 2 +- .../sqllogictest/test_files/dynamic_file.slt | 106 ++++++++++ datafusion/sqllogictest/test_files/json.slt | 5 + .../sqllogictest/test_files/parquet.slt | 5 + 20 files changed, 590 insertions(+), 52 deletions(-) create mode 100644 datafusion/catalog/src/dynamic_file/catalog.rs create mode 100644 datafusion/catalog/src/dynamic_file/mod.rs create mode 100644 datafusion/core/src/datasource/dynamic_file.rs create mode 100644 datafusion/sqllogictest/test_files/dynamic_file.slt diff --git a/datafusion-cli/Cargo.lock b/datafusion-cli/Cargo.lock index ddc33b43d134..d3af3d5596dc 100644 --- a/datafusion-cli/Cargo.lock +++ b/datafusion-cli/Cargo.lock @@ -1261,6 +1261,7 @@ dependencies = [ "datafusion-execution", "datafusion-expr", "datafusion-physical-plan", + "parking_lot", ] [[package]] diff --git a/datafusion-cli/src/catalog.rs b/datafusion-cli/src/catalog.rs index c4636f1ce0e0..9b9afc1c2420 100644 --- a/datafusion-cli/src/catalog.rs +++ b/datafusion-cli/src/catalog.rs @@ -21,10 +21,9 @@ use std::sync::{Arc, Weak}; use crate::object_storage::{get_object_store, AwsOptions, GcpOptions}; use datafusion::catalog::{CatalogProvider, CatalogProviderList, SchemaProvider}; + use datafusion::common::plan_datafusion_err; -use datafusion::datasource::listing::{ - ListingTable, ListingTableConfig, ListingTableUrl, -}; +use datafusion::datasource::listing::ListingTableUrl; use datafusion::datasource::TableProvider; use datafusion::error::Result; use datafusion::execution::context::SessionState; @@ -34,14 +33,13 @@ use async_trait::async_trait; use dirs::home_dir; use parking_lot::RwLock; -/// Wraps another catalog, automatically creating table providers -/// for local files if needed -pub struct DynamicFileCatalog { +/// Wraps another catalog, automatically register require object stores for the file locations +pub struct DynamicObjectStoreCatalog { inner: Arc, state: Weak>, } -impl DynamicFileCatalog { +impl DynamicObjectStoreCatalog { pub fn new( inner: Arc, state: Weak>, @@ -50,7 +48,7 @@ impl DynamicFileCatalog { } } -impl CatalogProviderList for DynamicFileCatalog { +impl CatalogProviderList for DynamicObjectStoreCatalog { fn as_any(&self) -> &dyn Any { self } @@ -69,19 +67,19 @@ impl CatalogProviderList for DynamicFileCatalog { fn catalog(&self, name: &str) -> Option> { let state = self.state.clone(); - self.inner - .catalog(name) - .map(|catalog| Arc::new(DynamicFileCatalogProvider::new(catalog, state)) as _) + self.inner.catalog(name).map(|catalog| { + Arc::new(DynamicObjectStoreCatalogProvider::new(catalog, state)) as _ + }) } } /// Wraps another catalog provider -struct DynamicFileCatalogProvider { +struct DynamicObjectStoreCatalogProvider { inner: Arc, state: Weak>, } -impl DynamicFileCatalogProvider { +impl DynamicObjectStoreCatalogProvider { pub fn new( inner: Arc, state: Weak>, @@ -90,7 +88,7 @@ impl DynamicFileCatalogProvider { } } -impl CatalogProvider for DynamicFileCatalogProvider { +impl CatalogProvider for DynamicObjectStoreCatalogProvider { fn as_any(&self) -> &dyn Any { self } @@ -101,9 +99,9 @@ impl CatalogProvider for DynamicFileCatalogProvider { fn schema(&self, name: &str) -> Option> { let state = self.state.clone(); - self.inner - .schema(name) - .map(|schema| Arc::new(DynamicFileSchemaProvider::new(schema, state)) as _) + self.inner.schema(name).map(|schema| { + Arc::new(DynamicObjectStoreSchemaProvider::new(schema, state)) as _ + }) } fn register_schema( @@ -115,13 +113,14 @@ impl CatalogProvider for DynamicFileCatalogProvider { } } -/// Wraps another schema provider -struct DynamicFileSchemaProvider { +/// Wraps another schema provider. [DynamicObjectStoreSchemaProvider] is responsible for registering the required +/// object stores for the file locations. +struct DynamicObjectStoreSchemaProvider { inner: Arc, state: Weak>, } -impl DynamicFileSchemaProvider { +impl DynamicObjectStoreSchemaProvider { pub fn new( inner: Arc, state: Weak>, @@ -131,7 +130,7 @@ impl DynamicFileSchemaProvider { } #[async_trait] -impl SchemaProvider for DynamicFileSchemaProvider { +impl SchemaProvider for DynamicObjectStoreSchemaProvider { fn as_any(&self) -> &dyn Any { self } @@ -149,9 +148,11 @@ impl SchemaProvider for DynamicFileSchemaProvider { } async fn table(&self, name: &str) -> Result>> { - let inner_table = self.inner.table(name).await?; - if inner_table.is_some() { - return Ok(inner_table); + let inner_table = self.inner.table(name).await; + if inner_table.is_ok() { + if let Some(inner_table) = inner_table? { + return Ok(Some(inner_table)); + } } // if the inner schema provider didn't have a table by @@ -201,16 +202,7 @@ impl SchemaProvider for DynamicFileSchemaProvider { state.runtime_env().register_object_store(url, store); } } - - let config = match ListingTableConfig::new(table_url).infer(&state).await { - Ok(cfg) => cfg, - Err(_) => { - // treat as non-existing - return Ok(None); - } - }; - - Ok(Some(Arc::new(ListingTable::try_new(config)?))) + self.inner.table(name).await } fn deregister_table(&self, name: &str) -> Result>> { @@ -221,7 +213,8 @@ impl SchemaProvider for DynamicFileSchemaProvider { self.inner.table_exist(name) } } -fn substitute_tilde(cur: String) -> String { + +pub fn substitute_tilde(cur: String) -> String { if let Some(usr_dir_path) = home_dir() { if let Some(usr_dir) = usr_dir_path.to_str() { if cur.starts_with('~') && !usr_dir.is_empty() { @@ -231,9 +224,9 @@ fn substitute_tilde(cur: String) -> String { } cur } - #[cfg(test)] mod tests { + use super::*; use datafusion::catalog::SchemaProvider; @@ -241,12 +234,12 @@ mod tests { fn setup_context() -> (SessionContext, Arc) { let ctx = SessionContext::new(); - ctx.register_catalog_list(Arc::new(DynamicFileCatalog::new( + ctx.register_catalog_list(Arc::new(DynamicObjectStoreCatalog::new( ctx.state().catalog_list().clone(), ctx.state_weak_ref(), ))); - let provider = &DynamicFileCatalog::new( + let provider = &DynamicObjectStoreCatalog::new( ctx.state().catalog_list().clone(), ctx.state_weak_ref(), ) as &dyn CatalogProviderList; @@ -269,7 +262,7 @@ mod tests { let (ctx, schema) = setup_context(); // That's a non registered table so expecting None here - let table = schema.table(&location).await.unwrap(); + let table = schema.table(&location).await?; assert!(table.is_none()); // It should still create an object store for the location in the SessionState @@ -293,7 +286,7 @@ mod tests { let (ctx, schema) = setup_context(); - let table = schema.table(&location).await.unwrap(); + let table = schema.table(&location).await?; assert!(table.is_none()); let store = ctx @@ -315,7 +308,7 @@ mod tests { let (ctx, schema) = setup_context(); - let table = schema.table(&location).await.unwrap(); + let table = schema.table(&location).await?; assert!(table.is_none()); let store = ctx @@ -337,6 +330,7 @@ mod tests { assert!(schema.table(location).await.is_err()); } + #[cfg(not(target_os = "windows"))] #[test] fn test_substitute_tilde() { diff --git a/datafusion-cli/src/main.rs b/datafusion-cli/src/main.rs index 6e94e6ea4186..cdefada5e24a 100644 --- a/datafusion-cli/src/main.rs +++ b/datafusion-cli/src/main.rs @@ -26,7 +26,7 @@ use datafusion::execution::context::SessionConfig; use datafusion::execution::memory_pool::{FairSpillPool, GreedyMemoryPool}; use datafusion::execution::runtime_env::{RuntimeConfig, RuntimeEnv}; use datafusion::prelude::SessionContext; -use datafusion_cli::catalog::DynamicFileCatalog; +use datafusion_cli::catalog::DynamicObjectStoreCatalog; use datafusion_cli::functions::ParquetMetadataFunc; use datafusion_cli::{ exec, @@ -173,11 +173,13 @@ async fn main_inner() -> Result<()> { let runtime_env = create_runtime_env(rt_config.clone())?; + // enable dynamic file query let ctx = - SessionContext::new_with_config_rt(session_config.clone(), Arc::new(runtime_env)); + SessionContext::new_with_config_rt(session_config.clone(), Arc::new(runtime_env)) + .enable_url_table(); ctx.refresh_catalogs().await?; - // install dynamic catalog provider that knows how to open files - ctx.register_catalog_list(Arc::new(DynamicFileCatalog::new( + // install dynamic catalog provider that can register required object stores + ctx.register_catalog_list(Arc::new(DynamicObjectStoreCatalog::new( ctx.state().catalog_list().clone(), ctx.state_weak_ref(), ))); diff --git a/datafusion-examples/examples/dataframe.rs b/datafusion-examples/examples/dataframe.rs index ea01c53b1c62..d7e0068ef88f 100644 --- a/datafusion-examples/examples/dataframe.rs +++ b/datafusion-examples/examples/dataframe.rs @@ -64,6 +64,12 @@ async fn main() -> Result<()> { .await?; parquet_df.describe().await.unwrap().show().await?; + let dyn_ctx = ctx.enable_url_table(); + let df = dyn_ctx + .sql(&format!("SELECT * FROM '{}'", file_path.to_str().unwrap())) + .await?; + df.show().await?; + Ok(()) } diff --git a/datafusion-examples/examples/external_dependency/query-aws-s3.rs b/datafusion-examples/examples/external_dependency/query-aws-s3.rs index e32286e30e4f..9c4d76703c9c 100644 --- a/datafusion-examples/examples/external_dependency/query-aws-s3.rs +++ b/datafusion-examples/examples/external_dependency/query-aws-s3.rs @@ -63,5 +63,14 @@ async fn main() -> Result<()> { // print the results df.show().await?; + // dynamic query by the file path + ctx.enable_url_table(); + let df = ctx + .sql(format!(r#"SELECT * FROM '{}' LIMIT 10"#, &path).as_str()) + .await?; + + // print the results + df.show().await?; + Ok(()) } diff --git a/datafusion/catalog/Cargo.toml b/datafusion/catalog/Cargo.toml index 533bd1eeba08..f9801352087d 100644 --- a/datafusion/catalog/Cargo.toml +++ b/datafusion/catalog/Cargo.toml @@ -34,6 +34,7 @@ datafusion-common = { workspace = true } datafusion-execution = { workspace = true } datafusion-expr = { workspace = true } datafusion-physical-plan = { workspace = true } +parking_lot = { workspace = true } [lints] workspace = true diff --git a/datafusion/catalog/src/dynamic_file/catalog.rs b/datafusion/catalog/src/dynamic_file/catalog.rs new file mode 100644 index 000000000000..cd586446f82c --- /dev/null +++ b/datafusion/catalog/src/dynamic_file/catalog.rs @@ -0,0 +1,183 @@ +// 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. + +//! [`DynamicFileCatalog`] that creates tables from file paths + +use crate::{CatalogProvider, CatalogProviderList, SchemaProvider, TableProvider}; +use async_trait::async_trait; +use std::any::Any; +use std::sync::Arc; + +/// Wrap another catalog provider list +pub struct DynamicFileCatalog { + /// The inner catalog provider list + inner: Arc, + /// The factory that can create a table provider from the file path + factory: Arc, +} + +impl DynamicFileCatalog { + pub fn new( + inner: Arc, + factory: Arc, + ) -> Self { + Self { inner, factory } + } +} + +impl CatalogProviderList for DynamicFileCatalog { + fn as_any(&self) -> &dyn Any { + self + } + + fn register_catalog( + &self, + name: String, + catalog: Arc, + ) -> Option> { + self.inner.register_catalog(name, catalog) + } + + fn catalog_names(&self) -> Vec { + self.inner.catalog_names() + } + + fn catalog(&self, name: &str) -> Option> { + self.inner.catalog(name).map(|catalog| { + Arc::new(DynamicFileCatalogProvider::new( + catalog, + Arc::clone(&self.factory), + )) as _ + }) + } +} + +/// Wraps another catalog provider +struct DynamicFileCatalogProvider { + /// The inner catalog provider + inner: Arc, + /// The factory that can create a table provider from the file path + factory: Arc, +} + +impl DynamicFileCatalogProvider { + pub fn new( + inner: Arc, + factory: Arc, + ) -> Self { + Self { inner, factory } + } +} + +impl CatalogProvider for DynamicFileCatalogProvider { + fn as_any(&self) -> &dyn Any { + self + } + + fn schema_names(&self) -> Vec { + self.inner.schema_names() + } + + fn schema(&self, name: &str) -> Option> { + self.inner.schema(name).map(|schema| { + Arc::new(DynamicFileSchemaProvider::new( + schema, + Arc::clone(&self.factory), + )) as _ + }) + } + + fn register_schema( + &self, + name: &str, + schema: Arc, + ) -> datafusion_common::Result>> { + self.inner.register_schema(name, schema) + } +} + +/// Implements the [DynamicFileSchemaProvider] that can create tables provider from the file path. +/// +/// The provider will try to create a table provider from the file path if the table provider +/// isn't exist in the inner schema provider. +pub struct DynamicFileSchemaProvider { + /// The inner schema provider + inner: Arc, + /// The factory that can create a table provider from the file path + factory: Arc, +} + +impl DynamicFileSchemaProvider { + /// Create a new [DynamicFileSchemaProvider] with the given inner schema provider. + pub fn new( + inner: Arc, + factory: Arc, + ) -> Self { + Self { inner, factory } + } +} + +#[async_trait] +impl SchemaProvider for DynamicFileSchemaProvider { + fn as_any(&self) -> &dyn Any { + self + } + + fn table_names(&self) -> Vec { + self.inner.table_names() + } + + async fn table( + &self, + name: &str, + ) -> datafusion_common::Result>> { + if let Some(table) = self.inner.table(name).await? { + return Ok(Some(table)); + }; + + self.factory.try_new(name).await + } + + fn register_table( + &self, + name: String, + table: Arc, + ) -> datafusion_common::Result>> { + self.inner.register_table(name, table) + } + + fn deregister_table( + &self, + name: &str, + ) -> datafusion_common::Result>> { + self.inner.deregister_table(name) + } + + fn table_exist(&self, name: &str) -> bool { + self.inner.table_exist(name) + } +} + +/// [UrlTableFactory] is a factory that can create a table provider from the given url. +#[async_trait] +pub trait UrlTableFactory: Sync + Send { + /// create a new table provider from the provided url + async fn try_new( + &self, + url: &str, + ) -> datafusion_common::Result>>; +} diff --git a/datafusion/catalog/src/dynamic_file/mod.rs b/datafusion/catalog/src/dynamic_file/mod.rs new file mode 100644 index 000000000000..59142333dd54 --- /dev/null +++ b/datafusion/catalog/src/dynamic_file/mod.rs @@ -0,0 +1,18 @@ +// 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. + +pub(crate) mod catalog; diff --git a/datafusion/catalog/src/lib.rs b/datafusion/catalog/src/lib.rs index fe76b5dc9c64..21630f267d2c 100644 --- a/datafusion/catalog/src/lib.rs +++ b/datafusion/catalog/src/lib.rs @@ -16,11 +16,13 @@ // under the License. mod catalog; +mod dynamic_file; mod schema; mod session; mod table; pub use catalog::*; +pub use dynamic_file::catalog::*; pub use schema::*; pub use session::*; pub use table::*; diff --git a/datafusion/catalog/src/session.rs b/datafusion/catalog/src/session.rs index 05d2684ed3e0..61d9c2d8a71e 100644 --- a/datafusion/catalog/src/session.rs +++ b/datafusion/catalog/src/session.rs @@ -24,9 +24,10 @@ use datafusion_execution::TaskContext; use datafusion_expr::execution_props::ExecutionProps; use datafusion_expr::{AggregateUDF, Expr, LogicalPlan, ScalarUDF, WindowUDF}; use datafusion_physical_plan::{ExecutionPlan, PhysicalExpr}; +use parking_lot::{Mutex, RwLock}; use std::any::Any; use std::collections::HashMap; -use std::sync::Arc; +use std::sync::{Arc, Weak}; /// Interface for accessing [`SessionState`] from the catalog. /// @@ -136,3 +137,34 @@ impl From<&dyn Session> for TaskContext { ) } } +type SessionRefLock = Arc>>>>; +/// The state store that stores the reference of the runtime session state. +pub struct SessionStore { + session: SessionRefLock, +} + +impl SessionStore { + /// Create a new [SessionStore] + pub fn new() -> Self { + Self { + session: Arc::new(Mutex::new(None)), + } + } + + /// Set the session state of the store + pub fn with_state(&self, state: Weak>) { + let mut lock = self.session.lock(); + *lock = Some(state); + } + + /// Get the current session of the store + pub fn get_session(&self) -> Weak> { + self.session.lock().clone().unwrap() + } +} + +impl Default for SessionStore { + fn default() -> Self { + Self::new() + } +} diff --git a/datafusion/core/src/datasource/dynamic_file.rs b/datafusion/core/src/datasource/dynamic_file.rs new file mode 100644 index 000000000000..a95f3abb939b --- /dev/null +++ b/datafusion/core/src/datasource/dynamic_file.rs @@ -0,0 +1,80 @@ +// 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. + +//! dynamic_file_schema contains an [`UrlTableFactory`] implementation that +//! can create a [`ListingTable`] from the given url. + +use std::sync::Arc; + +use async_trait::async_trait; +use datafusion_catalog::{SessionStore, UrlTableFactory}; +use datafusion_common::plan_datafusion_err; + +use crate::datasource::listing::{ListingTable, ListingTableConfig, ListingTableUrl}; +use crate::datasource::TableProvider; +use crate::error::Result; +use crate::execution::context::SessionState; + +/// [DynamicListTableFactory] is a factory that can create a [ListingTable] from the given url. +#[derive(Default)] +pub struct DynamicListTableFactory { + /// The session store that contains the current session. + session_store: SessionStore, +} + +impl DynamicListTableFactory { + /// Create a new [DynamicListTableFactory] with the given state store. + pub fn new(session_store: SessionStore) -> Self { + Self { session_store } + } + + /// Get the session store. + pub fn session_store(&self) -> &SessionStore { + &self.session_store + } +} + +#[async_trait] +impl UrlTableFactory for DynamicListTableFactory { + async fn try_new(&self, url: &str) -> Result>> { + let Ok(table_url) = ListingTableUrl::parse(url) else { + return Ok(None); + }; + + let state = &self + .session_store() + .get_session() + .upgrade() + .and_then(|session| { + session + .read() + .as_any() + .downcast_ref::() + .cloned() + }) + .ok_or_else(|| plan_datafusion_err!("get current SessionStore error"))?; + + match ListingTableConfig::new(table_url.clone()) + .infer(state) + .await + { + Ok(cfg) => ListingTable::try_new(cfg) + .map(|table| Some(Arc::new(table) as Arc)), + Err(_) => Ok(None), + } + } +} diff --git a/datafusion/core/src/datasource/mod.rs b/datafusion/core/src/datasource/mod.rs index 529bb799e23d..0ed53418fe32 100644 --- a/datafusion/core/src/datasource/mod.rs +++ b/datafusion/core/src/datasource/mod.rs @@ -22,6 +22,7 @@ pub mod avro_to_arrow; pub mod cte_worktable; pub mod default_table_source; +pub mod dynamic_file; pub mod empty; pub mod file_format; pub mod function; diff --git a/datafusion/core/src/execution/context/mod.rs b/datafusion/core/src/execution/context/mod.rs index 06dc797ae27a..621b214818e9 100644 --- a/datafusion/core/src/execution/context/mod.rs +++ b/datafusion/core/src/execution/context/mod.rs @@ -69,17 +69,18 @@ use datafusion_expr::{ // backwards compatibility pub use crate::execution::session_state::SessionState; +use crate::datasource::dynamic_file::DynamicListTableFactory; +use crate::execution::session_state::SessionStateBuilder; use async_trait::async_trait; use chrono::{DateTime, Utc}; -use object_store::ObjectStore; -use parking_lot::RwLock; -use url::Url; - -use crate::execution::session_state::SessionStateBuilder; +use datafusion_catalog::{DynamicFileCatalog, SessionStore, UrlTableFactory}; pub use datafusion_execution::config::SessionConfig; pub use datafusion_execution::TaskContext; pub use datafusion_expr::execution_props::ExecutionProps; use datafusion_optimizer::{AnalyzerRule, OptimizerRule}; +use object_store::ObjectStore; +use parking_lot::RwLock; +use url::Url; mod avro; mod csv; @@ -356,6 +357,53 @@ impl SessionContext { } } + /// Enable dynamic file querying for the current session. + /// + /// This allows queries to directly access arbitrary file names via SQL like + /// `SELECT * from 'my_file.parquet'` + /// so it should only be enabled for systems that such access is not a security risk + /// + /// See [DynamicFileCatalog] for more details + /// + /// ``` + /// # use datafusion::prelude::*; + /// # use datafusion::{error::Result, assert_batches_eq}; + /// # #[tokio::main] + /// # async fn main() -> Result<()> { + /// let ctx = SessionContext::new().enable_url_table(); + /// let results = ctx + /// .sql("SELECT a, MIN(b) FROM 'tests/data/example.csv' as example GROUP BY a LIMIT 100") + /// .await? + /// .collect() + /// .await?; + /// assert_batches_eq!( + /// &[ + /// "+---+----------------+", + /// "| a | min(example.b) |", + /// "+---+----------------+", + /// "| 1 | 2 |", + /// "+---+----------------+", + /// ], + /// &results + /// ); + /// # Ok(()) + /// # } + /// ``` + pub fn enable_url_table(&self) -> Self { + let state_ref = self.state(); + let factory = Arc::new(DynamicListTableFactory::new(SessionStore::new())); + let catalog_list = Arc::new(DynamicFileCatalog::new( + Arc::clone(state_ref.catalog_list()), + Arc::clone(&factory) as Arc, + )); + let new_state = SessionStateBuilder::new_from_existing(self.state()) + .with_catalog_list(catalog_list) + .build(); + let ctx = SessionContext::new_with_state(new_state); + factory.session_store().with_state(ctx.state_weak_ref()); + ctx + } + /// Creates a new `SessionContext` using the provided [`SessionState`] #[deprecated(since = "32.0.0", note = "Use SessionContext::new_with_state")] pub fn with_state(state: SessionState) -> Self { @@ -1790,6 +1838,38 @@ mod tests { Ok(()) } + #[tokio::test] + async fn test_dynamic_file_query() -> Result<()> { + let path = PathBuf::from(env!("CARGO_MANIFEST_DIR")); + let path = path.join("tests/tpch-csv/customer.csv"); + let url = format!("file://{}", path.display()); + let cfg = SessionConfig::new(); + let session_state = SessionStateBuilder::new() + .with_default_features() + .with_config(cfg) + .build(); + let ctx = SessionContext::new_with_state(session_state).enable_url_table(); + let result = plan_and_collect( + &ctx, + format!("select c_name from '{}' limit 3;", &url).as_str(), + ) + .await?; + + let actual = arrow::util::pretty::pretty_format_batches(&result) + .unwrap() + .to_string(); + let expected = r#"+--------------------+ +| c_name | ++--------------------+ +| Customer#000000002 | +| Customer#000000003 | +| Customer#000000004 | ++--------------------+"#; + assert_eq!(actual, expected); + + Ok(()) + } + #[tokio::test] async fn custom_query_planner() -> Result<()> { let runtime = Arc::new(RuntimeEnv::default()); diff --git a/datafusion/sqllogictest/src/test_context.rs b/datafusion/sqllogictest/src/test_context.rs index 224a0e18eac4..ef2fa863e6b0 100644 --- a/datafusion/sqllogictest/src/test_context.rs +++ b/datafusion/sqllogictest/src/test_context.rs @@ -98,6 +98,9 @@ impl TestContext { return None; } } + "dynamic_file.slt" => { + test_ctx.ctx = test_ctx.ctx.enable_url_table(); + } "joins.slt" => { info!("Registering partition table tables"); let example_udf = create_example_udf(); diff --git a/datafusion/sqllogictest/test_files/arrow_files.slt b/datafusion/sqllogictest/test_files/arrow_files.slt index 8cf3550fdb25..e66ba7477fc4 100644 --- a/datafusion/sqllogictest/test_files/arrow_files.slt +++ b/datafusion/sqllogictest/test_files/arrow_files.slt @@ -43,6 +43,11 @@ SELECT * FROM arrow_simple 3 baz false 4 NULL true +# Ensure that local files can not be read by default (a potential security issue) +# (url table is only supported when DynamicFileCatalog is enabled) +statement error DataFusion error: Error during planning: table 'datafusion.public.../core/tests/data/example.arrow' not found +SELECT * FROM '../core/tests/data/example.arrow'; + # ARROW partitioned table statement ok CREATE EXTERNAL TABLE arrow_partitioned ( diff --git a/datafusion/sqllogictest/test_files/csv_files.slt b/datafusion/sqllogictest/test_files/csv_files.slt index d6600e06dc1c..01d0f4ac39bd 100644 --- a/datafusion/sqllogictest/test_files/csv_files.slt +++ b/datafusion/sqllogictest/test_files/csv_files.slt @@ -50,6 +50,11 @@ id7 value7 id8 value8 id9 value9 +# Ensure that local files can not be read by default (a potential security issue) +# (url table is only supported when DynamicFileCatalog is enabled) +statement error DataFusion error: Error during planning: table 'datafusion.public.../core/tests/data/quote.csv' not found +select * from '../core/tests/data/quote.csv'; + query TT select * from csv_with_escape; ---- diff --git a/datafusion/sqllogictest/test_files/describe.slt b/datafusion/sqllogictest/test_files/describe.slt index a15c3a109cab..077e8e6474d1 100644 --- a/datafusion/sqllogictest/test_files/describe.slt +++ b/datafusion/sqllogictest/test_files/describe.slt @@ -57,7 +57,7 @@ statement ok DROP TABLE aggregate_simple; ########## -# Describe file (currently we can only describe file in datafusion-cli, fix this after issue (#4850) has been done) +# Describe file (we can only describe file if the default catalog is `DynamicFileCatalog`) ########## statement error Error during planning: table 'datafusion.public.../core/tests/data/aggregate_simple.csv' not found diff --git a/datafusion/sqllogictest/test_files/dynamic_file.slt b/datafusion/sqllogictest/test_files/dynamic_file.slt new file mode 100644 index 000000000000..e177fd3de243 --- /dev/null +++ b/datafusion/sqllogictest/test_files/dynamic_file.slt @@ -0,0 +1,106 @@ +# 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. + +# +# Note: This file runs with a SessionContext that has the `enable_url_table` flag set +# +# dynamic select arrow file in the folder +query ITB +SELECT * FROM '../core/tests/data/partitioned_table_arrow/part=123' ORDER BY f0; +---- +1 foo true +2 bar false + +# dynamic file query doesn't support partitioned table +statement error DataFusion error: Error during planning: table 'datafusion.public.../core/tests/data/partitioned_table_arrow' not found +SELECT * FROM '../core/tests/data/partitioned_table_arrow' ORDER BY f0; + +# read avro file +query IT +SELECT id, CAST(string_col AS varchar) FROM '../../testing/data/avro/alltypes_plain.avro' +---- +4 0 +5 1 +6 0 +7 1 +2 0 +3 1 +0 0 +1 1 + +# dynamic query snappy avro file +query IT +SELECT id, CAST(string_col AS varchar) FROM '../../testing/data/avro/alltypes_plain.snappy.avro' +---- +4 0 +5 1 +6 0 +7 1 +2 0 +3 1 +0 0 +1 1 + +# query the csv file dynamically with the config of current session +query TT +select * from '../core/tests/data/quote.csv'; +---- +~id0~ ~value0~ +~id1~ ~value1~ +~id2~ ~value2~ +~id3~ ~value3~ +~id4~ ~value4~ +~id5~ ~value5~ +~id6~ ~value6~ +~id7~ ~value7~ +~id8~ ~value8~ +~id9~ ~value9~ + +query TTT +DESCRIBE '../core/tests/data/aggregate_simple.csv'; +---- +c1 Float64 YES +c2 Float64 YES +c3 Boolean YES + +query IR rowsort +SELECT a, b FROM '../core/tests/data/2.json' +---- +-10 -3.5 +1 -3.5 +1 0.6 +1 0.6 +1 2 +1 2 +1 2 +1 2 +100000000000000 0.6 +2 0.6 +5 -3.5 +7 -3.5 + +query IT +SELECT id, CAST(string_col AS varchar) FROM '../../parquet-testing/data/alltypes_plain.parquet'; +---- +4 0 +5 1 +6 0 +7 1 +2 0 +3 1 +0 0 +1 1 diff --git a/datafusion/sqllogictest/test_files/json.slt b/datafusion/sqllogictest/test_files/json.slt index 0b9508310b00..0903c2427649 100644 --- a/datafusion/sqllogictest/test_files/json.slt +++ b/datafusion/sqllogictest/test_files/json.slt @@ -45,6 +45,11 @@ SELECT a, b FROM json_test 5 -3.5 7 -3.5 +# Ensure that local files can not be read by default (a potential security issue) +# (url table is only supported when DynamicFileCatalog is enabled) +statement error DataFusion error: Error during planning: table 'datafusion.public.../core/tests/data/2.json' not found +SELECT a, b FROM '../core/tests/data/2.json' + query TT EXPLAIN SELECT count(*) from json_test ---- diff --git a/datafusion/sqllogictest/test_files/parquet.slt b/datafusion/sqllogictest/test_files/parquet.slt index 9a7b085312bb..f8b163adc796 100644 --- a/datafusion/sqllogictest/test_files/parquet.slt +++ b/datafusion/sqllogictest/test_files/parquet.slt @@ -202,6 +202,11 @@ SELECT id, CAST(string_col AS varchar) FROM alltypes_plain 0 0 1 1 +# Ensure that local files can not be read by default (a potential security issue) +# (url table is only supported when DynamicFileCatalog is enabled) +statement error DataFusion error: Error during planning: table 'datafusion.public.../../parquet-testing/data/alltypes_plain.parquet' not found +SELECT id, CAST(string_col AS varchar) FROM '../../parquet-testing/data/alltypes_plain.parquet'; + # Clean up statement ok DROP TABLE alltypes_plain; From 4569cbb252a130115627d55a59bc63755da8f57e Mon Sep 17 00:00:00 2001 From: Eduard Karacharov Date: Mon, 9 Sep 2024 18:51:07 +0300 Subject: [PATCH 127/154] tests: enable fuzz for filtered anti-semi NLJoin (#12360) * tests: enable fuzz for filtered anti-semi NLJoin * tests: update filters in join fuzz tests * tests: disable flaky tests for SortMergeJoin --- datafusion/core/tests/fuzz_cases/join_fuzz.rs | 192 ++++++++++-------- 1 file changed, 109 insertions(+), 83 deletions(-) diff --git a/datafusion/core/tests/fuzz_cases/join_fuzz.rs b/datafusion/core/tests/fuzz_cases/join_fuzz.rs index 1c2d8ece2f36..20c329915254 100644 --- a/datafusion/core/tests/fuzz_cases/join_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/join_fuzz.rs @@ -27,6 +27,7 @@ use datafusion_common::ScalarValue; use datafusion_physical_expr::expressions::Literal; use datafusion_physical_expr::PhysicalExprRef; +use itertools::Itertools; use rand::Rng; use datafusion::common::JoinSide; @@ -54,33 +55,6 @@ enum JoinTestType { // because if existing variants both passed that means SortMergeJoin and NestedLoopJoin also passes HjSmj, } -#[tokio::test] -async fn test_inner_join_1k() { - JoinFuzzTestCase::new( - make_staggered_batches(1000), - make_staggered_batches(1000), - JoinType::Inner, - None, - ) - .run_test(&[JoinTestType::HjSmj, JoinTestType::NljHj], false) - .await -} - -fn less_than_100_join_filter(schema1: Arc, _schema2: Arc) -> JoinFilter { - let less_than_100 = Arc::new(BinaryExpr::new( - Arc::new(Column::new("a", 0)), - Operator::Lt, - Arc::new(Literal::new(ScalarValue::from(100))), - )) as _; - let column_indices = vec![ColumnIndex { - index: 0, - side: JoinSide::Left, - }]; - let intermediate_schema = - Schema::new(vec![schema1.field_with_name("a").unwrap().to_owned()]); - - JoinFilter::new(less_than_100, column_indices, intermediate_schema) -} fn col_lt_col_filter(schema1: Arc, schema2: Arc) -> JoinFilter { let less_filter = Arc::new(BinaryExpr::new( @@ -120,14 +94,14 @@ async fn test_inner_join_1k_filtered() { make_staggered_batches(1000), make_staggered_batches(1000), JoinType::Inner, - Some(Box::new(less_than_100_join_filter)), + Some(Box::new(col_lt_col_filter)), ) .run_test(&[JoinTestType::HjSmj, JoinTestType::NljHj], false) .await } #[tokio::test] -async fn test_inner_join_1k_smjoin() { +async fn test_inner_join_1k() { JoinFuzzTestCase::new( make_staggered_batches(1000), make_staggered_batches(1000), @@ -151,14 +125,16 @@ async fn test_left_join_1k() { } #[tokio::test] +// flaky for HjSmj case +// https://github.com/apache/datafusion/issues/12359 async fn test_left_join_1k_filtered() { JoinFuzzTestCase::new( make_staggered_batches(1000), make_staggered_batches(1000), JoinType::Left, - Some(Box::new(less_than_100_join_filter)), + Some(Box::new(col_lt_col_filter)), ) - .run_test(&[JoinTestType::HjSmj, JoinTestType::NljHj], false) + .run_test(&[JoinTestType::NljHj], false) .await } @@ -173,17 +149,18 @@ async fn test_right_join_1k() { .run_test(&[JoinTestType::HjSmj, JoinTestType::NljHj], false) .await } -// Add support for Right filtered joins -#[ignore] + #[tokio::test] +// flaky for HjSmj case +// https://github.com/apache/datafusion/issues/12359 async fn test_right_join_1k_filtered() { JoinFuzzTestCase::new( make_staggered_batches(1000), make_staggered_batches(1000), JoinType::Right, - Some(Box::new(less_than_100_join_filter)), + Some(Box::new(col_lt_col_filter)), ) - .run_test(&[JoinTestType::HjSmj, JoinTestType::NljHj], false) + .run_test(&[JoinTestType::NljHj], false) .await } @@ -200,14 +177,16 @@ async fn test_full_join_1k() { } #[tokio::test] +// flaky for HjSmj case +// https://github.com/apache/datafusion/issues/12359 async fn test_full_join_1k_filtered() { JoinFuzzTestCase::new( make_staggered_batches(1000), make_staggered_batches(1000), JoinType::Full, - Some(Box::new(less_than_100_join_filter)), + Some(Box::new(col_lt_col_filter)), ) - .run_test(&[JoinTestType::HjSmj, JoinTestType::NljHj], false) + .run_test(&[JoinTestType::NljHj], false) .await } @@ -225,15 +204,13 @@ async fn test_semi_join_1k() { #[tokio::test] async fn test_semi_join_1k_filtered() { - // NLJ vs HJ gives wrong result - // Tracked in https://github.com/apache/datafusion/issues/11537 JoinFuzzTestCase::new( make_staggered_batches(1000), make_staggered_batches(1000), JoinType::LeftSemi, Some(Box::new(col_lt_col_filter)), ) - .run_test(&[JoinTestType::HjSmj], false) + .run_test(&[JoinTestType::HjSmj, JoinTestType::NljHj], false) .await } @@ -250,19 +227,16 @@ async fn test_anti_join_1k() { } #[tokio::test] -#[ignore] -// flaky test giving 1 rows difference sometimes +// flaky for HjSmj case, giving 1 rows difference sometimes // https://github.com/apache/datafusion/issues/11555 async fn test_anti_join_1k_filtered() { - // NLJ vs HJ gives wrong result - // Tracked in https://github.com/apache/datafusion/issues/11537 JoinFuzzTestCase::new( make_staggered_batches(1000), make_staggered_batches(1000), JoinType::LeftAnti, Some(Box::new(col_lt_col_filter)), ) - .run_test(&[JoinTestType::HjSmj], false) + .run_test(&[JoinTestType::NljHj], false) .await } @@ -292,27 +266,6 @@ impl JoinFuzzTestCase { } } - fn column_indices(&self) -> Vec { - vec![ - ColumnIndex { - index: 0, - side: JoinSide::Left, - }, - ColumnIndex { - index: 1, - side: JoinSide::Left, - }, - ColumnIndex { - index: 0, - side: JoinSide::Right, - }, - ColumnIndex { - index: 1, - side: JoinSide::Right, - }, - ] - } - fn on_columns(&self) -> Vec<(PhysicalExprRef, PhysicalExprRef)> { let schema1 = self.input1[0].schema(); let schema2 = self.input2[0].schema(); @@ -328,10 +281,20 @@ impl JoinFuzzTestCase { ] } + /// Helper function for building NLJoin filter, returning intermediate + /// schema as a union of origin filter intermediate schema and + /// on-condition schema fn intermediate_schema(&self) -> Schema { + let filter_schema = if let Some(filter) = self.join_filter() { + filter.schema().to_owned() + } else { + Schema::empty() + }; + let schema1 = self.input1[0].schema(); let schema2 = self.input2[0].schema(); - Schema::new(vec![ + + let on_schema = Schema::new(vec![ schema1 .field_with_name("a") .unwrap() @@ -344,7 +307,81 @@ impl JoinFuzzTestCase { .with_nullable(true), schema2.field_with_name("a").unwrap().to_owned(), schema2.field_with_name("b").unwrap().to_owned(), - ]) + ]); + + Schema::new( + filter_schema + .fields + .into_iter() + .cloned() + .chain(on_schema.fields.into_iter().cloned()) + .collect_vec(), + ) + } + + /// Helper function for building NLJoin filter, returns the union + /// of original filter expression and on-condition expression + fn composite_filter_expression(&self) -> PhysicalExprRef { + let (filter_expression, column_idx_offset) = + if let Some(filter) = self.join_filter() { + ( + filter.expression().to_owned(), + filter.schema().fields().len(), + ) + } else { + (Arc::new(Literal::new(ScalarValue::from(true))) as _, 0) + }; + + let equal_a = Arc::new(BinaryExpr::new( + Arc::new(Column::new("a", column_idx_offset)), + Operator::Eq, + Arc::new(Column::new("a", column_idx_offset + 2)), + )); + let equal_b = Arc::new(BinaryExpr::new( + Arc::new(Column::new("b", column_idx_offset + 1)), + Operator::Eq, + Arc::new(Column::new("b", column_idx_offset + 3)), + )); + let on_expression = Arc::new(BinaryExpr::new(equal_a, Operator::And, equal_b)); + + Arc::new(BinaryExpr::new( + filter_expression, + Operator::And, + on_expression, + )) + } + + /// Helper function for building NLJoin filter, returning the union + /// of original filter column indices and on-condition column indices. + /// Result must match intermediate schema. + fn column_indices(&self) -> Vec { + let mut column_indices = if let Some(filter) = self.join_filter() { + filter.column_indices().to_vec() + } else { + vec![] + }; + + let on_column_indices = vec![ + ColumnIndex { + index: 0, + side: JoinSide::Left, + }, + ColumnIndex { + index: 1, + side: JoinSide::Left, + }, + ColumnIndex { + index: 0, + side: JoinSide::Right, + }, + ColumnIndex { + index: 1, + side: JoinSide::Right, + }, + ]; + + column_indices.extend(on_column_indices); + column_indices } fn left_right(&self) -> (Arc, Arc) { @@ -400,26 +437,15 @@ impl JoinFuzzTestCase { fn nested_loop_join(&self) -> Arc { let (left, right) = self.left_right(); - // Nested loop join uses filter for joining records + let column_indices = self.column_indices(); let intermediate_schema = self.intermediate_schema(); + let expression = self.composite_filter_expression(); - let equal_a = Arc::new(BinaryExpr::new( - Arc::new(Column::new("a", 0)), - Operator::Eq, - Arc::new(Column::new("a", 2)), - )) as _; - let equal_b = Arc::new(BinaryExpr::new( - Arc::new(Column::new("b", 1)), - Operator::Eq, - Arc::new(Column::new("b", 3)), - )) as _; - let expression = Arc::new(BinaryExpr::new(equal_a, Operator::And, equal_b)) as _; - - let on_filter = JoinFilter::new(expression, column_indices, intermediate_schema); + let filter = JoinFilter::new(expression, column_indices, intermediate_schema); Arc::new( - NestedLoopJoinExec::try_new(left, right, Some(on_filter), &self.join_type) + NestedLoopJoinExec::try_new(left, right, Some(filter), &self.join_type) .unwrap(), ) } From 79b34336b464d49d86717541a10a5c5ab37fd90e Mon Sep 17 00:00:00 2001 From: Jeffrey Vo Date: Tue, 10 Sep 2024 01:52:34 +1000 Subject: [PATCH 128/154] Refactor `SqlToRel::sql_expr_to_logical_expr_internal` to reduce stack size (#12384) * Refactor sql_expr_to_logical_expr_internal to reduce stack size * Pass Expr by value instead of via Box * Update datafusion/sql/src/expr/mod.rs Co-authored-by: Andrew Lamb * Cargo fmt * Formatting --------- Co-authored-by: Andrew Lamb --- datafusion/sql/src/expr/mod.rs | 232 ++++++++++--------- datafusion/sqllogictest/bin/sqllogictests.rs | 2 - 2 files changed, 120 insertions(+), 114 deletions(-) diff --git a/datafusion/sql/src/expr/mod.rs b/datafusion/sql/src/expr/mod.rs index 035fd3816c6c..6e975d8557dc 100644 --- a/datafusion/sql/src/expr/mod.rs +++ b/datafusion/sql/src/expr/mod.rs @@ -21,8 +21,8 @@ use datafusion_expr::planner::{ PlannerResult, RawBinaryExpr, RawDictionaryExpr, RawFieldAccessExpr, }; use sqlparser::ast::{ - BinaryOperator, CastKind, DictionaryField, Expr as SQLExpr, MapEntry, StructField, - Subscript, TrimWhereField, Value, + BinaryOperator, CastFormat, CastKind, DataType as SQLDataType, DictionaryField, + Expr as SQLExpr, MapEntry, StructField, Subscript, TrimWhereField, Value, }; use datafusion_common::{ @@ -174,6 +174,10 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { schema: &DFSchema, planner_context: &mut PlannerContext, ) -> Result { + // NOTE: This function is called recusively, so each match arm body should be as + // small as possible to avoid stack overflows in debug builds. Follow the + // common pattern of extracting into a separate function for non-trivial + // arms. See https://github.com/apache/datafusion/pull/12384 for more context. match sql { SQLExpr::Value(value) => { self.parse_value(value, planner_context.prepare_param_data_types()) @@ -210,91 +214,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { // ["foo"], [4] or [4:5] SQLExpr::Subscript { expr, subscript } => { - let expr = - self.sql_expr_to_logical_expr(*expr, schema, planner_context)?; - - let field_access = match *subscript { - Subscript::Index { index } => { - // index can be a name, in which case it is a named field access - match index { - SQLExpr::Value( - Value::SingleQuotedString(s) - | Value::DoubleQuotedString(s), - ) => GetFieldAccess::NamedStructField { - name: ScalarValue::from(s), - }, - SQLExpr::JsonAccess { .. } => { - return not_impl_err!("JsonAccess"); - } - // otherwise treat like a list index - _ => GetFieldAccess::ListIndex { - key: Box::new(self.sql_expr_to_logical_expr( - index, - schema, - planner_context, - )?), - }, - } - } - Subscript::Slice { - lower_bound, - upper_bound, - stride, - } => { - // Means access like [:2] - let lower_bound = if let Some(lower_bound) = lower_bound { - self.sql_expr_to_logical_expr( - lower_bound, - schema, - planner_context, - ) - } else { - not_impl_err!("Slice subscript requires a lower bound") - }?; - - // means access like [2:] - let upper_bound = if let Some(upper_bound) = upper_bound { - self.sql_expr_to_logical_expr( - upper_bound, - schema, - planner_context, - ) - } else { - not_impl_err!("Slice subscript requires an upper bound") - }?; - - // stride, default to 1 - let stride = if let Some(stride) = stride { - self.sql_expr_to_logical_expr( - stride, - schema, - planner_context, - )? - } else { - lit(1i64) - }; - - GetFieldAccess::ListRange { - start: Box::new(lower_bound), - stop: Box::new(upper_bound), - stride: Box::new(stride), - } - } - }; - - let mut field_access_expr = RawFieldAccessExpr { expr, field_access }; - for planner in self.context_provider.get_expr_planners() { - match planner.plan_field_access(field_access_expr, schema)? { - PlannerResult::Planned(expr) => return Ok(expr), - PlannerResult::Original(expr) => { - field_access_expr = expr; - } - } - } - - not_impl_err!( - "GetFieldAccess not supported by ExprPlanner: {field_access_expr:?}" - ) + self.sql_subscript_to_expr(*expr, subscript, schema, planner_context) } SQLExpr::CompoundIdentifier(ids) => { @@ -320,31 +240,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { expr, data_type, format, - } => { - if let Some(format) = format { - return not_impl_err!("CAST with format is not supported: {format}"); - } - - let dt = self.convert_data_type(&data_type)?; - let expr = - self.sql_expr_to_logical_expr(*expr, schema, planner_context)?; - - // numeric constants are treated as seconds (rather as nanoseconds) - // to align with postgres / duckdb semantics - let expr = match &dt { - DataType::Timestamp(TimeUnit::Nanosecond, tz) - if expr.get_type(schema)? == DataType::Int64 => - { - Expr::Cast(Cast::new( - Box::new(expr), - DataType::Timestamp(TimeUnit::Second, tz.clone()), - )) - } - _ => expr, - }; - - Ok(Expr::Cast(Cast::new(Box::new(expr), dt))) - } + } => self.sql_cast_to_expr(*expr, data_type, format, schema, planner_context), SQLExpr::Cast { kind: CastKind::TryCast | CastKind::SafeCast, @@ -1016,6 +912,118 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { } not_impl_err!("Overlay not supported by ExprPlanner: {overlay_args:?}") } + + fn sql_cast_to_expr( + &self, + expr: SQLExpr, + data_type: SQLDataType, + format: Option, + schema: &DFSchema, + planner_context: &mut PlannerContext, + ) -> Result { + if let Some(format) = format { + return not_impl_err!("CAST with format is not supported: {format}"); + } + + let dt = self.convert_data_type(&data_type)?; + let expr = self.sql_expr_to_logical_expr(expr, schema, planner_context)?; + + // numeric constants are treated as seconds (rather as nanoseconds) + // to align with postgres / duckdb semantics + let expr = match &dt { + DataType::Timestamp(TimeUnit::Nanosecond, tz) + if expr.get_type(schema)? == DataType::Int64 => + { + Expr::Cast(Cast::new( + Box::new(expr), + DataType::Timestamp(TimeUnit::Second, tz.clone()), + )) + } + _ => expr, + }; + + Ok(Expr::Cast(Cast::new(Box::new(expr), dt))) + } + + fn sql_subscript_to_expr( + &self, + expr: SQLExpr, + subscript: Box, + schema: &DFSchema, + planner_context: &mut PlannerContext, + ) -> Result { + let expr = self.sql_expr_to_logical_expr(expr, schema, planner_context)?; + + let field_access = match *subscript { + Subscript::Index { index } => { + // index can be a name, in which case it is a named field access + match index { + SQLExpr::Value( + Value::SingleQuotedString(s) | Value::DoubleQuotedString(s), + ) => GetFieldAccess::NamedStructField { + name: ScalarValue::from(s), + }, + SQLExpr::JsonAccess { .. } => { + return not_impl_err!("JsonAccess"); + } + // otherwise treat like a list index + _ => GetFieldAccess::ListIndex { + key: Box::new(self.sql_expr_to_logical_expr( + index, + schema, + planner_context, + )?), + }, + } + } + Subscript::Slice { + lower_bound, + upper_bound, + stride, + } => { + // Means access like [:2] + let lower_bound = if let Some(lower_bound) = lower_bound { + self.sql_expr_to_logical_expr(lower_bound, schema, planner_context) + } else { + not_impl_err!("Slice subscript requires a lower bound") + }?; + + // means access like [2:] + let upper_bound = if let Some(upper_bound) = upper_bound { + self.sql_expr_to_logical_expr(upper_bound, schema, planner_context) + } else { + not_impl_err!("Slice subscript requires an upper bound") + }?; + + // stride, default to 1 + let stride = if let Some(stride) = stride { + self.sql_expr_to_logical_expr(stride, schema, planner_context)? + } else { + lit(1i64) + }; + + GetFieldAccess::ListRange { + start: Box::new(lower_bound), + stop: Box::new(upper_bound), + stride: Box::new(stride), + } + } + }; + + let mut field_access_expr = RawFieldAccessExpr { expr, field_access }; + for planner in self.context_provider.get_expr_planners() { + match planner.plan_field_access(field_access_expr, schema)? { + PlannerResult::Planned(expr) => return Ok(expr), + PlannerResult::Original(expr) => { + field_access_expr = expr; + } + } + } + + not_impl_err!( + "GetFieldAccess not supported by ExprPlanner: {field_access_expr:?}" + ) + } } #[cfg(test)] diff --git a/datafusion/sqllogictest/bin/sqllogictests.rs b/datafusion/sqllogictest/bin/sqllogictests.rs index 8c8ed2e58743..baa49057e1b9 100644 --- a/datafusion/sqllogictest/bin/sqllogictests.rs +++ b/datafusion/sqllogictest/bin/sqllogictests.rs @@ -30,11 +30,9 @@ use datafusion_common_runtime::SpawnedTask; const TEST_DIRECTORY: &str = "test_files/"; const PG_COMPAT_FILE_PREFIX: &str = "pg_compat_"; -const STACK_SIZE: usize = 2 * 1024 * 1024 + 512 * 1024; // 2.5 MBs, the default 2 MBs is currently too small pub fn main() -> Result<()> { tokio::runtime::Builder::new_multi_thread() - .thread_stack_size(STACK_SIZE) .enable_all() .build() .unwrap() From 09f7592bd6ebbeacaa1515d2f2d3a79dcc5a9178 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=BC=A0=E6=9E=97=E4=BC=9F?= Date: Tue, 10 Sep 2024 00:34:12 +0800 Subject: [PATCH 129/154] Reuse spill_record_batch_by_size function (#12389) --- .../physical-plan/src/aggregates/mod.rs | 8 +++---- .../physical-plan/src/aggregates/row_hash.rs | 21 +++++++------------ datafusion/physical-plan/src/spill.rs | 1 - 3 files changed, 11 insertions(+), 19 deletions(-) diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index ad0e43631afd..c3bc7b042e65 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -1136,12 +1136,12 @@ fn evaluate_optional( /// Evaluate a group by expression against a `RecordBatch` /// /// Arguments: -/// `group_by`: the expression to evaluate -/// `batch`: the `RecordBatch` to evaluate against +/// - `group_by`: the expression to evaluate +/// - `batch`: the `RecordBatch` to evaluate against /// /// Returns: A Vec of Vecs of Array of results -/// The outer Vect appears to be for grouping sets -/// The inner Vect contains the results per expression +/// The outer Vec appears to be for grouping sets +/// The inner Vec contains the results per expression /// The inner-inner Array contains the results per row pub(crate) fn evaluate_group_by( group_by: &PhysicalGroupBy, diff --git a/datafusion/physical-plan/src/aggregates/row_hash.rs b/datafusion/physical-plan/src/aggregates/row_hash.rs index fd2d26d9b49e..0332131d4b57 100644 --- a/datafusion/physical-plan/src/aggregates/row_hash.rs +++ b/datafusion/physical-plan/src/aggregates/row_hash.rs @@ -27,11 +27,10 @@ use crate::aggregates::{ evaluate_group_by, evaluate_many, evaluate_optional, group_schema, AggregateMode, PhysicalGroupBy, }; -use crate::common::IPCWriter; use crate::metrics::{BaselineMetrics, MetricBuilder, RecordOutput}; use crate::sorts::sort::sort_batch; use crate::sorts::streaming_merge; -use crate::spill::read_spill_as_stream; +use crate::spill::{read_spill_as_stream, spill_record_batch_by_size}; use crate::stream::RecordBatchStreamAdapter; use crate::{aggregates, metrics, ExecutionPlan, PhysicalExpr}; use crate::{RecordBatchStream, SendableRecordBatchStream}; @@ -905,19 +904,13 @@ impl GroupedHashAggregateStream { let emit = self.emit(EmitTo::All, true)?; let sorted = sort_batch(&emit, &self.spill_state.spill_expr, None)?; 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 - 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()?; + spill_record_batch_by_size( + &sorted, + spillfile.path().into(), + sorted.schema(), + self.batch_size, + )?; self.spill_state.spills.push(spillfile); Ok(()) } diff --git a/datafusion/physical-plan/src/spill.rs b/datafusion/physical-plan/src/spill.rs index 21ca58fa0a9f..de85a7c6f098 100644 --- a/datafusion/physical-plan/src/spill.rs +++ b/datafusion/physical-plan/src/spill.rs @@ -88,7 +88,6 @@ fn read_spill(sender: Sender>, path: &Path) -> Result<()> { /// Spill the `RecordBatch` to disk as smaller batches /// split by `batch_size_rows` -/// Return `total_rows` what is spilled pub fn spill_record_batch_by_size( batch: &RecordBatch, path: PathBuf, From 0a82ac3cd127d1cc8ab8baa3cf9886fc2714a62c Mon Sep 17 00:00:00 2001 From: Oleks V Date: Mon, 9 Sep 2024 10:48:11 -0700 Subject: [PATCH 130/154] minor: improve join fuzz tests debug kit (#12397) --- datafusion/core/tests/fuzz_cases/join_fuzz.rs | 123 +++++++++--------- 1 file changed, 63 insertions(+), 60 deletions(-) diff --git a/datafusion/core/tests/fuzz_cases/join_fuzz.rs b/datafusion/core/tests/fuzz_cases/join_fuzz.rs index 20c329915254..96aa1be181f5 100644 --- a/datafusion/core/tests/fuzz_cases/join_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/join_fuzz.rs @@ -15,13 +15,13 @@ // specific language governing permissions and limitations // under the License. -use std::sync::Arc; - use arrow::array::{ArrayRef, Int32Array}; use arrow::compute::SortOptions; use arrow::record_batch::RecordBatch; use arrow::util::pretty::pretty_format_batches; use arrow_schema::Schema; +use std::sync::Arc; +use std::time::SystemTime; use datafusion_common::ScalarValue; use datafusion_physical_expr::expressions::Literal; @@ -474,11 +474,34 @@ impl JoinFuzzTestCase { let smj_rows = smj_collected.iter().fold(0, |acc, b| acc + b.num_rows()); let nlj_rows = nlj_collected.iter().fold(0, |acc, b| acc + b.num_rows()); - if debug { + // compare + let smj_formatted = + pretty_format_batches(&smj_collected).unwrap().to_string(); + let hj_formatted = pretty_format_batches(&hj_collected).unwrap().to_string(); + let nlj_formatted = + pretty_format_batches(&nlj_collected).unwrap().to_string(); + + let mut smj_formatted_sorted: Vec<&str> = + smj_formatted.trim().lines().collect(); + smj_formatted_sorted.sort_unstable(); + + let mut hj_formatted_sorted: Vec<&str> = + hj_formatted.trim().lines().collect(); + hj_formatted_sorted.sort_unstable(); + + let mut nlj_formatted_sorted: Vec<&str> = + nlj_formatted.trim().lines().collect(); + nlj_formatted_sorted.sort_unstable(); + + if debug + && ((join_tests.contains(&JoinTestType::NljHj) && nlj_rows != hj_rows) + || (join_tests.contains(&JoinTestType::HjSmj) && smj_rows != hj_rows)) + { let fuzz_debug = "fuzz_test_debug"; std::fs::remove_dir_all(fuzz_debug).unwrap_or(()); std::fs::create_dir_all(fuzz_debug).unwrap(); let out_dir_name = &format!("{fuzz_debug}/batch_size_{batch_size}"); + println!("Test result data mismatch found. HJ rows {}, SMJ rows {}, NLJ rows {}", hj_rows, smj_rows, nlj_rows); println!("The debug is ON. Input data will be saved to {out_dir_name}"); Self::save_partitioned_batches_as_parquet( @@ -492,7 +515,15 @@ impl JoinFuzzTestCase { "input2", ); - if join_tests.contains(&JoinTestType::NljHj) { + if join_tests.contains(&JoinTestType::NljHj) + && join_tests.contains(&JoinTestType::NljHj) + && nlj_rows != hj_rows + { + println!("=============== HashJoinExec =================="); + hj_formatted_sorted.iter().for_each(|s| println!("{}", s)); + println!("=============== NestedLoopJoinExec =================="); + smj_formatted_sorted.iter().for_each(|s| println!("{}", s)); + Self::save_partitioned_batches_as_parquet( &nlj_collected, out_dir_name, @@ -505,7 +536,12 @@ impl JoinFuzzTestCase { ); } - if join_tests.contains(&JoinTestType::HjSmj) { + if join_tests.contains(&JoinTestType::HjSmj) && smj_rows != hj_rows { + println!("=============== HashJoinExec =================="); + hj_formatted_sorted.iter().for_each(|s| println!("{}", s)); + println!("=============== SortMergeJoinExec =================="); + smj_formatted_sorted.iter().for_each(|s| println!("{}", s)); + Self::save_partitioned_batches_as_parquet( &hj_collected, out_dir_name, @@ -519,25 +555,6 @@ impl JoinFuzzTestCase { } } - // compare - let smj_formatted = - pretty_format_batches(&smj_collected).unwrap().to_string(); - let hj_formatted = pretty_format_batches(&hj_collected).unwrap().to_string(); - let nlj_formatted = - pretty_format_batches(&nlj_collected).unwrap().to_string(); - - let mut smj_formatted_sorted: Vec<&str> = - smj_formatted.trim().lines().collect(); - smj_formatted_sorted.sort_unstable(); - - let mut hj_formatted_sorted: Vec<&str> = - hj_formatted.trim().lines().collect(); - hj_formatted_sorted.sort_unstable(); - - let mut nlj_formatted_sorted: Vec<&str> = - nlj_formatted.trim().lines().collect(); - nlj_formatted_sorted.sort_unstable(); - if join_tests.contains(&JoinTestType::NljHj) { let err_msg_rowcnt = format!("NestedLoopJoinExec and HashJoinExec produced different row counts, batch_size: {}", batch_size); assert_eq!(nlj_rows, hj_rows, "{}", err_msg_rowcnt.as_str()); @@ -602,34 +619,6 @@ impl JoinFuzzTestCase { /// ) /// .run_test(&[JoinTestType::HjSmj], false) /// .await; - /// - /// let ctx: SessionContext = SessionContext::new(); - /// let df = ctx - /// .read_parquet( - /// "/tmp/input1/*.parquet", - /// datafusion::prelude::ParquetReadOptions::default(), - /// ) - /// .await - /// .unwrap(); - /// let left = df.collect().await.unwrap(); - /// - /// let df = ctx - /// .read_parquet( - /// "/tmp/input2/*.parquet", - /// datafusion::prelude::ParquetReadOptions::default(), - /// ) - /// .await - /// .unwrap(); - /// - /// let right = df.collect().await.unwrap(); - /// JoinFuzzTestCase::new( - /// left, - /// right, - /// JoinType::LeftSemi, - /// Some(Box::new(less_than_100_join_filter)), - /// ) - /// .run_test() - /// .await /// } fn save_partitioned_batches_as_parquet( input: &[RecordBatch], @@ -641,9 +630,15 @@ impl JoinFuzzTestCase { std::fs::create_dir_all(out_path).unwrap(); input.iter().enumerate().for_each(|(idx, batch)| { - let mut file = - std::fs::File::create(format!("{out_path}/file_{}.parquet", idx)) - .unwrap(); + let file_path = format!("{out_path}/file_{}.parquet", idx); + let mut file = std::fs::File::create(&file_path).unwrap(); + println!( + "{}: Saving batch idx {} rows {} to parquet {}", + &out_name, + idx, + batch.num_rows(), + &file_path + ); let mut writer = parquet::arrow::ArrowWriter::try_new( &mut file, input.first().unwrap().schema(), @@ -653,8 +648,6 @@ impl JoinFuzzTestCase { writer.write(batch).unwrap(); writer.close().unwrap(); }); - - println!("The data {out_name} saved as parquet into {out_path}"); } /// Read parquet files preserving partitions, i.e. 1 file -> 1 partition @@ -667,10 +660,20 @@ impl JoinFuzzTestCase { ) -> std::io::Result> { let ctx: SessionContext = SessionContext::new(); let mut batches: Vec = vec![]; + let mut entries = std::fs::read_dir(dir)? + .map(|res| res.map(|e| e.path())) + .collect::, std::io::Error>>()?; + + // important to read files using the same order as they have been written + // sort by modification time + entries.sort_by_key(|path| { + std::fs::metadata(path) + .and_then(|metadata| metadata.modified()) + .unwrap_or(SystemTime::UNIX_EPOCH) + }); - for entry in std::fs::read_dir(dir)? { - let entry = entry?; - let path = entry.path(); + for entry in entries { + let path = entry.as_path(); if path.is_file() { let mut batch = ctx From 275fb1b32f9a6d716231db24441dcb7c1bba4b8d Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Mon, 9 Sep 2024 15:49:35 -0400 Subject: [PATCH 131/154] Add documentation about performance PRs, add (TBD) section on feature criteria (#12372) * Improve PR documentation: performance PRs and suitable changes * prettier --- docs/source/contributor-guide/index.md | 62 ++++++++++++++++++++++---- 1 file changed, 53 insertions(+), 9 deletions(-) diff --git a/docs/source/contributor-guide/index.md b/docs/source/contributor-guide/index.md index ad49b614c334..79a929879833 100644 --- a/docs/source/contributor-guide/index.md +++ b/docs/source/contributor-guide/index.md @@ -34,6 +34,10 @@ community as well as get more familiar with Rust and the relevant codebases. You can find a curated [good-first-issue] list to help you get started. +[good-first-issue]: https://github.com/apache/datafusion/issues?q=is%3Aissue+is%3Aopen+label%3A%22good+first+issue%22 + +### Open Contribution and Assigning tickets + DataFusion is an open contribution project, and thus there is no particular project imposed deadline for completing any issue or any restriction on who can work on an issue, nor how many people can work on an issue at the same time. @@ -54,13 +58,27 @@ unable to make progress you should unassign the issue by using the `unassign me` link at the top of the issue page (and ask for help if are stuck) so that someone else can get involved in the work. +### File Tickets to Discuss New Features + If you plan to work on a new feature that doesn't have an existing ticket, it is a good idea to open a ticket to discuss the feature. Advanced discussion often helps avoid wasted effort by determining early if the feature is a good fit for -DataFusion before too much time is invested. It also often helps to discuss your -ideas with the community to get feedback on implementation. +DataFusion before too much time is invested. Discussion on a ticket can help +gather feedback from the community and is likely easier to discuss than a 1000 +line PR. -[good-first-issue]: https://github.com/apache/datafusion/issues?q=is%3Aissue+is%3Aopen+label%3A%22good+first+issue%22 +If you open a ticket and it doesn't get any response, you can try `@`-mentioning +recently active community members in the ticket to get their attention. + +### What Features are Good Fits for DataFusion? + +DataFusion is designed to highly extensible, and many features can be implemented +as extensions without changing the core of DataFusion. + +We are [working on criteria for what features are good fits for DataFusion], and +will update this section when we have more to share. + +[working on criteria for what features are good fits for datafusion]: https://github.com/apache/datafusion/issues/12357 # Developer's guide @@ -88,7 +106,7 @@ committer who approved your PR to help remind them to merge it. ## Creating Pull Requests -We recommend splitting your contributions into multiple smaller focused PRs rather than large PRs (500+ lines) because: +When possible, we recommend splitting your contributions into multiple smaller focused PRs rather than large PRs (500+ lines) because: 1. The PR is more likely to be reviewed quickly -- our reviewers struggle to find the contiguous time needed to review large PRs. 2. The PR discussions tend to be more focused and less likely to get lost among several different threads. @@ -96,27 +114,53 @@ We recommend splitting your contributions into multiple smaller focused PRs rath If you are concerned that a larger design will be lost in a string of small PRs, creating a large draft PR that shows how they all work together can help. -Note all commits in a PR are squashed when merged to the `main` branch so there is one commit per PR. +Note all commits in a PR are squashed when merged to the `main` branch so there is one commit per PR after merge. # Reviewing Pull Requests Some helpful links: -- [PRs Waiting for Review] -- [Approved PRs Waiting for Merge] +- [PRs Waiting for Review] on GitHub +- [Approved PRs Waiting for Merge] on GitHub [prs waiting for review]: https://github.com/apache/datafusion/pulls?q=is%3Apr+is%3Aopen+-review%3Aapproved+-is%3Adraft+ [approved prs waiting for merge]: https://github.com/apache/datafusion/pulls?q=is%3Apr+is%3Aopen+review%3Aapproved+-is%3Adraft -When reviewing PRs, please remember our primary goal is to improve DataFusion and its community together. PR feedback should be constructive with the aim to help improve the code as well as the understanding of the contributor. +When reviewing PRs, our primary goal is to improve DataFusion and its community together. PR feedback should be constructive with the aim to help improve the code as well as the understanding of the contributor. Please ensure any issues you raise contains a rationale and suggested alternative -- it is frustrating to be told "don't do it this way" without any clear reason or alternate provided. Some things to specifically check: -1. Is the feature or fix covered sufficiently with tests (see `Test Organization` below)? +1. Is the feature or fix covered sufficiently with tests (see the [Testing](testing.md) section)? 2. Is the code clear, and fits the style of the existing codebase? +## Performance Improvements + +Performance improvements are always welcome: performance is a key DataFusion +feature. + +In general, the performance improvement from a change should be "enough" to +justify any added code complexity. How much is "enough" is a judgement made by +the committers, but generally means that the improvement should be noticeable in +a real-world scenario and is greater than the noise of the benchmarking system. + +To help committers evaluate the potential improvement, performance PRs should +in general be accompanied by benchmark results that demonstrate the improvement. + +The best way to demonstrate a performance improvement is with the existing +benchmarks: + +- [System level SQL Benchmarks](https://github.com/apache/datafusion/tree/main/benchmarks) +- Microbenchmarks such as those in [functions/benches](https://github.com/apache/datafusion/tree/main/datafusion/functions/benches) + +If there is no suitable existing benchmark, you can create a new one. It helps +to isolate the effects of your change by creating a separate PR with the +benchmark, and then a PR with the code change that improves the benchmark. + +[system level sql benchmarks]: https://github.com/apache/datafusion/tree/main/benchmarks +[functions/benches]: https://github.com/apache/datafusion/tree/main/datafusion/functions/benches + ## "Major" and "Minor" PRs Since we are a worldwide community, we have contributors in many timezones who review and comment. To ensure anyone who wishes has an opportunity to review a PR, our committers try to ensure that at least 24 hours passes between when a "major" PR is approved and when it is merged. From 23f3a8f07f2ec907cbf1fd55afa75196d9e09278 Mon Sep 17 00:00:00 2001 From: Dao Thanh Tung Date: Mon, 9 Sep 2024 20:51:30 +0100 Subject: [PATCH 132/154] Fix invalid ref in UserDefinedLogicalNodeCore doc (#12396) * Fix invalid ref in UserDefinedLogicalNodeCore doc Signed-off-by: dttung2905 * Update datafusion/expr/src/logical_plan/extension.rs Co-authored-by: Andrew Lamb --------- Signed-off-by: dttung2905 Co-authored-by: Andrew Lamb --- datafusion/expr/src/logical_plan/extension.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/datafusion/expr/src/logical_plan/extension.rs b/datafusion/expr/src/logical_plan/extension.rs index 2f581c1928f4..5514ec2925d9 100644 --- a/datafusion/expr/src/logical_plan/extension.rs +++ b/datafusion/expr/src/logical_plan/extension.rs @@ -212,8 +212,8 @@ impl Eq for dyn UserDefinedLogicalNode {} /// This trait facilitates implementation of the [`UserDefinedLogicalNode`]. /// /// See the example in -/// [user_defined_plan.rs](../../tests/user_defined_plan.rs) for an -/// example of how to use this extension API. +/// [user_defined_plan.rs](https://github.com/apache/datafusion/blob/main/datafusion/core/tests/user_defined/user_defined_plan.rs) +/// file for an example of how to use this extension API. pub trait UserDefinedLogicalNodeCore: fmt::Debug + Eq + Hash + Sized + Send + Sync + 'static { From 4b51bbecfffe7212519cc0b4d4ac4b787617f5ee Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Dani=C3=ABl=20Heres?= Date: Mon, 9 Sep 2024 22:55:46 +0200 Subject: [PATCH 133/154] Don't optimize null_equals_null case (#12404) --- .../optimizer/src/filter_null_join_keys.rs | 6 ++++-- .../optimizer/tests/optimizer_integration.rs | 21 +++++++------------ 2 files changed, 12 insertions(+), 15 deletions(-) diff --git a/datafusion/optimizer/src/filter_null_join_keys.rs b/datafusion/optimizer/src/filter_null_join_keys.rs index 01e3d27c580f..caf94a2c88bd 100644 --- a/datafusion/optimizer/src/filter_null_join_keys.rs +++ b/datafusion/optimizer/src/filter_null_join_keys.rs @@ -26,7 +26,7 @@ use datafusion_expr::utils::conjunction; use datafusion_expr::{logical_plan::Filter, Expr, ExprSchemable, LogicalPlan}; use std::sync::Arc; -/// The FilterNullJoinKeys rule will identify joins with equi-join conditions +/// The FilterNullJoinKeys rule will identify joins with equi-join conditions /// where the join key is nullable and then insert an `IsNotNull` filter on the nullable side since null values /// can never match. #[derive(Default)] @@ -50,7 +50,9 @@ impl OptimizerRule for FilterNullJoinKeys { return Ok(Transformed::no(plan)); } match plan { - LogicalPlan::Join(mut join) if !join.on.is_empty() => { + LogicalPlan::Join(mut join) + if !join.on.is_empty() && !join.null_equals_null => + { let (left_preserved, right_preserved) = on_lr_is_preserved(join.join_type); diff --git a/datafusion/optimizer/tests/optimizer_integration.rs b/datafusion/optimizer/tests/optimizer_integration.rs index 5292b66197f6..da5e92eafd11 100644 --- a/datafusion/optimizer/tests/optimizer_integration.rs +++ b/datafusion/optimizer/tests/optimizer_integration.rs @@ -177,15 +177,12 @@ fn intersect() -> Result<()> { let plan = test_sql(sql)?; let expected = "LeftSemi Join: test.col_int32 = test.col_int32, test.col_utf8 = test.col_utf8\ - \n Aggregate: groupBy=[[test.col_int32, test.col_utf8]], aggr=[[]]\ - \n LeftSemi Join: test.col_int32 = test.col_int32, test.col_utf8 = test.col_utf8\ - \n Aggregate: groupBy=[[test.col_int32, test.col_utf8]], aggr=[[]]\ - \n Filter: test.col_int32 IS NOT NULL AND test.col_utf8 IS NOT NULL\ - \n TableScan: test projection=[col_int32, col_utf8]\ - \n Filter: test.col_int32 IS NOT NULL AND test.col_utf8 IS NOT NULL\ - \n TableScan: test projection=[col_int32, col_utf8]\ - \n Filter: test.col_int32 IS NOT NULL AND test.col_utf8 IS NOT NULL\ - \n TableScan: test projection=[col_int32, col_utf8]"; + \n Aggregate: groupBy=[[test.col_int32, test.col_utf8]], aggr=[[]]\ + \n LeftSemi Join: test.col_int32 = test.col_int32, test.col_utf8 = test.col_utf8\ + \n Aggregate: groupBy=[[test.col_int32, test.col_utf8]], aggr=[[]]\ + \n TableScan: test projection=[col_int32, col_utf8]\ + \n TableScan: test projection=[col_int32, col_utf8]\ + \n TableScan: test projection=[col_int32, col_utf8]"; assert_eq!(expected, format!("{plan}")); Ok(()) } @@ -281,11 +278,9 @@ fn test_same_name_but_not_ambiguous() { let expected = "LeftSemi Join: t1.col_int32 = t2.col_int32\ \n Aggregate: groupBy=[[t1.col_int32]], aggr=[[]]\ \n SubqueryAlias: t1\ - \n Filter: test.col_int32 IS NOT NULL\ - \n TableScan: test projection=[col_int32]\ + \n TableScan: test projection=[col_int32]\ \n SubqueryAlias: t2\ - \n Filter: test.col_int32 IS NOT NULL\ - \n TableScan: test projection=[col_int32]"; + \n TableScan: test projection=[col_int32]"; assert_eq!(expected, format!("{plan}")); } From dbc78902582f4a63fc26d7359018ae85d2051343 Mon Sep 17 00:00:00 2001 From: Alex Huang Date: Tue, 10 Sep 2024 10:34:04 +0800 Subject: [PATCH 134/154] feat: Support `FixedSizedList` in `array_distance` function (#12381) --- datafusion/functions-nested/src/distance.rs | 18 ++++++- datafusion/sqllogictest/test_files/array.slt | 57 ++++++++++++++++++++ 2 files changed, 74 insertions(+), 1 deletion(-) diff --git a/datafusion/functions-nested/src/distance.rs b/datafusion/functions-nested/src/distance.rs index 4d779dc1fabf..fa9394c73bcb 100644 --- a/datafusion/functions-nested/src/distance.rs +++ b/datafusion/functions-nested/src/distance.rs @@ -28,6 +28,7 @@ use datafusion_common::cast::{ as_float32_array, as_float64_array, as_generic_list_array, as_int32_array, as_int64_array, }; +use datafusion_common::utils::coerced_fixed_size_list_to_list; use datafusion_common::DataFusionError; use datafusion_common::{exec_err, Result}; use datafusion_expr::{ColumnarValue, ScalarUDFImpl, Signature, Volatility}; @@ -51,7 +52,7 @@ pub(super) struct ArrayDistance { impl ArrayDistance { pub fn new() -> Self { Self { - signature: Signature::variadic_any(Volatility::Immutable), + signature: Signature::user_defined(Volatility::Immutable), aliases: vec!["list_distance".to_string()], } } @@ -77,6 +78,21 @@ impl ScalarUDFImpl for ArrayDistance { } } + fn coerce_types(&self, arg_types: &[DataType]) -> Result> { + if arg_types.len() != 2 { + return exec_err!("array_distance expects exactly two arguments"); + } + let mut result = Vec::new(); + for arg_type in arg_types { + match arg_type { + List(_) | LargeList(_) | FixedSizeList(_, _) => result.push(coerced_fixed_size_list_to_list(arg_type)), + _ => return exec_err!("The array_distance function can only accept List/LargeList/FixedSizeList."), + } + } + + Ok(result) + } + fn invoke(&self, args: &[ColumnarValue]) -> Result { make_scalar_function(array_distance_inner)(args) } diff --git a/datafusion/sqllogictest/test_files/array.slt b/datafusion/sqllogictest/test_files/array.slt index edc0cd7577e1..b20b59339b64 100644 --- a/datafusion/sqllogictest/test_files/array.slt +++ b/datafusion/sqllogictest/test_files/array.slt @@ -629,6 +629,38 @@ AS VALUES (arrow_cast(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]), 'FixedSizeList(10, List(Int64))'), [28, 29, 30], [28, 29, 30], 10) ; +statement ok +CREATE TABLE arrays_distance_table +AS VALUES + (make_array(1, 2, 3), make_array(1, 2, 3), make_array(1.1, 2.2, 3.3) , make_array(1.1, NULL, 3.3)), + (make_array(1, 2, 3), make_array(4, 5, 6), make_array(4.4, 5.5, 6.6), make_array(4.4, NULL, 6.6)), + (make_array(1, 2, 3), make_array(7, 8, 9), make_array(7.7, 8.8, 9.9), make_array(7.7, NULL, 9.9)), + (make_array(1, 2, 3), make_array(10, 11, 12), make_array(10.1, 11.2, 12.3), make_array(10.1, NULL, 12.3)) +; + +statement ok +CREATE TABLE large_arrays_distance_table +AS + SELECT + arrow_cast(column1, 'LargeList(Int64)') AS column1, + arrow_cast(column2, 'LargeList(Int64)') AS column2, + arrow_cast(column3, 'LargeList(Float64)') AS column3, + arrow_cast(column4, 'LargeList(Float64)') AS column4 +FROM arrays_distance_table +; + +statement ok +CREATE TABLE fixed_size_arrays_distance_table +AS + SELECT + arrow_cast(column1, 'FixedSizeList(3, Int64)') AS column1, + arrow_cast(column2, 'FixedSizeList(3, Int64)') AS column2, + arrow_cast(column3, 'FixedSizeList(3, Float64)') AS column3, + arrow_cast(column4, 'FixedSizeList(3, Float64)') AS column4 +FROM arrays_distance_table +; + + # Array literal ## boolean coercion is not supported @@ -4768,6 +4800,31 @@ select list_distance([1, 2, 3], [1, 2, 3]) AS distance; ---- 0 +# array_distance with columns +query RRR +select array_distance(column1, column2), array_distance(column1, column3), array_distance(column1, column4) from arrays_distance_table; +---- +0 0.374165738677 NULL +5.196152422707 6.063827174318 NULL +10.392304845413 11.778794505381 NULL +15.58845726812 15.935494971917 NULL + +query RRR +select array_distance(column1, column2), array_distance(column1, column3), array_distance(column1, column4) from large_arrays_distance_table; +---- +0 0.374165738677 NULL +5.196152422707 6.063827174318 NULL +10.392304845413 11.778794505381 NULL +15.58845726812 15.935494971917 NULL + +query RRR +select array_distance(column1, column2), array_distance(column1, column3), array_distance(column1, column4) from fixed_size_arrays_distance_table; +---- +0 0.374165738677 NULL +5.196152422707 6.063827174318 NULL +10.392304845413 11.778794505381 NULL +15.58845726812 15.935494971917 NULL + ## array_dims (aliases: `list_dims`) From 376a0b8d9badc6ac4ce446c0e31cebe605eb8724 Mon Sep 17 00:00:00 2001 From: Yongting You <2010youy01@gmail.com> Date: Tue, 10 Sep 2024 18:48:33 +0800 Subject: [PATCH 135/154] Fix substr() (#12383) --- datafusion/functions/src/unicode/substr.rs | 119 +++++++++-------- .../sqllogictest/test_files/functions.slt | 124 ++++++++++++++++-- 2 files changed, 178 insertions(+), 65 deletions(-) diff --git a/datafusion/functions/src/unicode/substr.rs b/datafusion/functions/src/unicode/substr.rs index 8a70b380669c..40d3a4d13e97 100644 --- a/datafusion/functions/src/unicode/substr.rs +++ b/datafusion/functions/src/unicode/substr.rs @@ -118,15 +118,37 @@ pub fn substr(args: &[ArrayRef]) -> Result { } } -// Return the exact byte index for [start, end), set count to -1 to ignore count -fn get_true_start_end(input: &str, start: usize, count: i64) -> (usize, usize) { +// Convert the given `start` and `count` to valid byte indices within `input` string +// Input `start` and `count` are equivalent to PostgreSQL's `substr(s, start, count)` +// `start` is 1-based, if `count` is not provided count to the end of the string +// Input indices are character-based, and return values are byte indices +// The input bounds can be outside string bounds, this function will return +// the intersection between input bounds and valid string bounds +// +// * Example +// 'Hi🌏' in-mem (`[]` for one char, `x` for one byte): [x][x][xxxx] +// `get_true_start_end('Hi🌏', 1, None) -> (0, 6)` +// `get_true_start_end('Hi🌏', 1, 1) -> (0, 1)` +// `get_true_start_end('Hi🌏', -10, 2) -> (0, 0)` +fn get_true_start_end(input: &str, start: i64, count: Option) -> (usize, usize) { + let start = start - 1; + let end = match count { + Some(count) => start + count as i64, + None => input.len() as i64, + }; + let count_to_end = count.is_some(); + + let start = start.clamp(0, input.len() as i64) as usize; + let end = end.clamp(0, input.len() as i64) as usize; + let count = end - start; + let (mut st, mut ed) = (input.len(), input.len()); let mut start_counting = false; let mut cnt = 0; for (char_cnt, (byte_cnt, _)) in input.char_indices().enumerate() { if char_cnt == start { st = byte_cnt; - if count != -1 { + if count_to_end { start_counting = true; } else { break; @@ -153,20 +175,15 @@ fn make_and_append_view( start: u32, ) { let substr_len = substr.len(); - if substr_len == 0 { - null_builder.append_null(); - views_buffer.push(0); + let sub_view = if substr_len > 12 { + let view = ByteView::from(*raw); + make_view(substr.as_bytes(), view.buffer_index, view.offset + start) } else { - let sub_view = if substr_len > 12 { - let view = ByteView::from(*raw); - make_view(substr.as_bytes(), view.buffer_index, view.offset + start) - } else { - // inline value does not need block id or offset - make_view(substr.as_bytes(), 0, 0) - }; - views_buffer.push(sub_view); - null_builder.append_non_null(); - } + // inline value does not need block id or offset + make_view(substr.as_bytes(), 0, 0) + }; + views_buffer.push(sub_view); + null_builder.append_non_null(); } // The decoding process refs the trait at: arrow/arrow-data/src/byte_view.rs:44 @@ -180,32 +197,26 @@ fn string_view_substr( let start_array = as_int64_array(&args[0])?; + // In either case of `substr(s, i)` or `substr(s, i, cnt)` + // If any of input argument is `NULL`, the result is `NULL` match args.len() { 1 => { - for (idx, (raw, start)) in string_view_array - .views() + for ((str_opt, raw_view), start_opt) in string_view_array .iter() + .zip(string_view_array.views().iter()) .zip(start_array.iter()) - .enumerate() { - if let Some(start) = start { - let start = (start - 1).max(0) as usize; - - // Safety: - // idx is always smaller or equal to string_view_array.views.len() - unsafe { - let str = string_view_array.value_unchecked(idx); - let (start, end) = get_true_start_end(str, start, -1); - let substr = &str[start..end]; + if let (Some(str), Some(start)) = (str_opt, start_opt) { + let (start, end) = get_true_start_end(str, start, None); + let substr = &str[start..end]; - make_and_append_view( - &mut views_buf, - &mut null_builder, - raw, - substr, - start as u32, - ); - } + make_and_append_view( + &mut views_buf, + &mut null_builder, + raw_view, + substr, + start as u32, + ); } else { null_builder.append_null(); views_buf.push(0); @@ -214,35 +225,31 @@ fn string_view_substr( } 2 => { let count_array = as_int64_array(&args[1])?; - for (idx, ((raw, start), count)) in string_view_array - .views() + for (((str_opt, raw_view), start_opt), count_opt) in string_view_array .iter() + .zip(string_view_array.views().iter()) .zip(start_array.iter()) .zip(count_array.iter()) - .enumerate() { - if let (Some(start), Some(count)) = (start, count) { - let start = (start - 1).max(0) as usize; + if let (Some(str), Some(start), Some(count)) = + (str_opt, start_opt, count_opt) + { if count < 0 { return exec_err!( "negative substring length not allowed: substr(, {start}, {count})" ); } else { - // Safety: - // idx is always smaller or equal to string_view_array.views.len() - unsafe { - let str = string_view_array.value_unchecked(idx); - let (start, end) = get_true_start_end(str, start, count); - let substr = &str[start..end]; - - make_and_append_view( - &mut views_buf, - &mut null_builder, - raw, - substr, - start as u32, - ); - } + let (start, end) = + get_true_start_end(str, start, Some(count as u64)); + let substr = &str[start..end]; + + make_and_append_view( + &mut views_buf, + &mut null_builder, + raw_view, + substr, + start as u32, + ); } } else { null_builder.append_null(); diff --git a/datafusion/sqllogictest/test_files/functions.slt b/datafusion/sqllogictest/test_files/functions.slt index b8519a463637..9bee9b8184ea 100644 --- a/datafusion/sqllogictest/test_files/functions.slt +++ b/datafusion/sqllogictest/test_files/functions.slt @@ -472,11 +472,6 @@ SELECT substr('alphabet', 30) ---- (empty) -query T -SELECT substr('alphabet', CAST(NULL AS int)) ----- -NULL - query T SELECT substr('alphabet', 3, 2) ---- @@ -487,15 +482,120 @@ SELECT substr('alphabet', 3, 20) ---- phabet +# test range ouside of string length +query TTTTTTTTTTTT +SELECT + substr('hi🌏', 1, 3), + substr('hi🌏', 1, 4), + substr('hi🌏', 1, 100), + substr('hi🌏', 0, 1), + substr('hi🌏', 0, 2), + substr('hi🌏', 0, 4), + substr('hi🌏', 0, 5), + substr('hi🌏', -10, 100), + substr('hi🌏', -10, 12), + substr('hi🌏', -10, 5), + substr('hi🌏', 10, 0), + substr('hi🌏', 10, 10); +---- +hi🌏 hi🌏 hi🌏 (empty) h hi🌏 hi🌏 hi🌏 h (empty) (empty) (empty) + +query TTTTTTTTTTTT +SELECT + substr('', 1, 3), + substr('', 1, 4), + substr('', 1, 100), + substr('', 0, 1), + substr('', 0, 2), + substr('', 0, 4), + substr('', 0, 5), + substr('', -10, 100), + substr('', -10, 12), + substr('', -10, 5), + substr('', 10, 0), + substr('', 10, 10); +---- +(empty) (empty) (empty) (empty) (empty) (empty) (empty) (empty) (empty) (empty) (empty) (empty) + +# Nulls +query TTTTTTTTTT +SELECT + substr('alphabet', NULL), + substr(NULL, 1), + substr(NULL, NULL), + substr('alphabet', CAST(NULL AS int), -20), + substr('alphabet', 3, CAST(NULL AS int)), + substr(NULL, 3, -4), + substr(NULL, NULL, 4), + substr(NULL, 1, NULL), + substr('', NULL, NULL), + substr(NULL, NULL, NULL); +---- +NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL + query T -SELECT substr('alphabet', CAST(NULL AS int), 20) +SELECT substr('Hello🌏世界', 5) ---- -NULL +o🌏世界 query T -SELECT substr('alphabet', 3, CAST(NULL AS int)) +SELECT substr('Hello🌏世界', 5, 3) ---- -NULL +o🌏世 + +statement ok +create table test_substr ( + c1 VARCHAR +) as values ('foo'), ('hello🌏世界'), ('💩'), ('ThisIsAVeryLongASCIIString'), (''), (NULL); + +statement ok +create table test_substr_stringview as +select c1 as c1, arrow_cast(c1, 'Utf8View') as c1_view from test_substr; + +# `substr()` on `StringViewArray`'s implementation operates directly on view's +# logical pointers, so check it's consistent with `StringArray` +query BBBBBBBBBBBBBB +select + substr(c1, 1) = substr(c1_view, 1), + substr(c1, 3) = substr(c1_view, 3), + substr(c1, 100) = substr(c1_view, 100), + substr(c1, -1) = substr(c1_view, -1), + substr(c1, 0, 0) = substr(c1_view, 0, 0), + substr(c1, -1, 2) = substr(c1_view, -1, 2), + substr(c1, -2, 10) = substr(c1_view, -2, 10), + substr(c1, -100, 200) = substr(c1_view, -100, 200), + substr(c1, -10, 10) = substr(c1_view, -10, 10), + substr(c1, -100, 10) = substr(c1_view, -100, 10), + substr(c1, 1, 100) = substr(c1_view, 1, 100), + substr(c1, 5, 3) = substr(c1_view, 5, 3), + substr(c1, 100, 200) = substr(c1_view, 100, 200), + substr(c1, 8, 0) = substr(c1_view, 8, 0) +from test_substr_stringview; +---- +true true true true true true true true true true true true true true +true true true true true true true true true true true true true true +true true true true true true true true true true true true true true +true true true true true true true true true true true true true true +true true true true true true true true true true true true true true +NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL + +# Check for non-ASCII strings +query TT +select substr(c1_view, 1), substr(c1_view, 5,3) from test_substr_stringview; +---- +foo (empty) +hello🌏世界 o🌏世 +💩 (empty) +ThisIsAVeryLongASCIIString IsA +(empty) (empty) +NULL NULL + +statement ok +drop table test_substr; + +statement ok +drop table test_substr_stringview; + statement error The SUBSTR function can only accept strings, but got Int64. SELECT substr(1, 3) @@ -503,6 +603,12 @@ SELECT substr(1, 3) statement error The SUBSTR function can only accept strings, but got Int64. SELECT substr(1, 3, 4) +statement error Execution error: negative substring length not allowed +select substr(arrow_cast('foo', 'Utf8View'), 1, -1); + +statement error Execution error: negative substring length not allowed +select substr('', 1, -1); + query T SELECT translate('12345', '143', 'ax') ---- From c71a9d7508e37e5d082e22d2953a12b61d290df5 Mon Sep 17 00:00:00 2001 From: Tai Le Manh <49281946+tlm365@users.noreply.github.com> Date: Tue, 10 Sep 2024 17:56:57 +0700 Subject: [PATCH 136/154] Implement native support StringView for `CONTAINS` function (#12168) * Implement native support StringView for contains function Signed-off-by: Tai Le Manh * Fix cargo fmt * Implement native support StringView for contains function Signed-off-by: Tai Le Manh * Fix cargo check * Fix unresolved doc link * Implement native support StringView for contains function Signed-off-by: Tai Le Manh * Update datafusion/functions/src/regexp_common.rs --------- Signed-off-by: Tai Le Manh Co-authored-by: Andrew Lamb --- datafusion/functions/Cargo.toml | 2 +- datafusion/functions/src/lib.rs | 3 + datafusion/functions/src/regex/mod.rs | 3 +- datafusion/functions/src/regexp_common.rs | 123 ++++++++++++ datafusion/functions/src/string/contains.rs | 190 +++++++++++++++--- .../sqllogictest/test_files/string_view.slt | 42 +++- .../source/user-guide/sql/scalar_functions.md | 2 +- 7 files changed, 329 insertions(+), 36 deletions(-) create mode 100644 datafusion/functions/src/regexp_common.rs diff --git a/datafusion/functions/Cargo.toml b/datafusion/functions/Cargo.toml index 3c95c03896e2..5b6dceaa420d 100644 --- a/datafusion/functions/Cargo.toml +++ b/datafusion/functions/Cargo.toml @@ -54,7 +54,7 @@ math_expressions = [] # enable regular expressions regex_expressions = ["regex"] # enable string functions -string_expressions = ["uuid"] +string_expressions = ["regex_expressions", "uuid"] # enable unicode functions unicode_expressions = ["hashbrown", "unicode-segmentation"] diff --git a/datafusion/functions/src/lib.rs b/datafusion/functions/src/lib.rs index 81be5552666d..bb680f3c67de 100644 --- a/datafusion/functions/src/lib.rs +++ b/datafusion/functions/src/lib.rs @@ -92,6 +92,9 @@ pub mod macros; pub mod string; make_stub_package!(string, "string_expressions"); +#[cfg(feature = "string_expressions")] +mod regexp_common; + /// Core datafusion expressions /// Enabled via feature flag `core_expressions` #[cfg(feature = "core_expressions")] diff --git a/datafusion/functions/src/regex/mod.rs b/datafusion/functions/src/regex/mod.rs index 4ac162290ddb..4afbe6cbbb89 100644 --- a/datafusion/functions/src/regex/mod.rs +++ b/datafusion/functions/src/regex/mod.rs @@ -15,11 +15,12 @@ // specific language governing permissions and limitations // under the License. -//! "regx" DataFusion functions +//! "regex" DataFusion functions pub mod regexplike; pub mod regexpmatch; pub mod regexpreplace; + // create UDFs make_udf_function!(regexpmatch::RegexpMatchFunc, REGEXP_MATCH, regexp_match); make_udf_function!(regexplike::RegexpLikeFunc, REGEXP_LIKE, regexp_like); diff --git a/datafusion/functions/src/regexp_common.rs b/datafusion/functions/src/regexp_common.rs new file mode 100644 index 000000000000..748c1a294f97 --- /dev/null +++ b/datafusion/functions/src/regexp_common.rs @@ -0,0 +1,123 @@ +// 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. + +//! Common utilities for implementing regex functions + +use crate::string::common::StringArrayType; + +use arrow::array::{Array, ArrayDataBuilder, BooleanArray}; +use arrow::datatypes::DataType; +use arrow_buffer::{BooleanBufferBuilder, NullBuffer}; +use datafusion_common::DataFusionError; +use regex::Regex; + +use std::collections::HashMap; + +#[cfg(doc)] +use arrow::array::{LargeStringArray, StringArray, StringViewArray}; +/// Perform SQL `array ~ regex_array` operation on +/// [`StringArray`] / [`LargeStringArray`] / [`StringViewArray`]. +/// +/// If `regex_array` element has an empty value, the corresponding result value is always true. +/// +/// `flags_array` are optional [`StringArray`] / [`LargeStringArray`] / [`StringViewArray`] flag, +/// which allow special search modes, such as case-insensitive and multi-line mode. +/// See the documentation [here](https://docs.rs/regex/1.5.4/regex/#grouping-and-flags) +/// for more information. +/// +/// It is inspired / copied from `regexp_is_match_utf8` [arrow-rs]. +/// +/// Can remove when is implemented upstream +/// +/// [arrow-rs]: https://github.com/apache/arrow-rs/blob/8c956a9f9ab26c14072740cce64c2b99cb039b13/arrow-string/src/regexp.rs#L31-L37 +pub fn regexp_is_match_utf8<'a, S1, S2, S3>( + array: &'a S1, + regex_array: &'a S2, + flags_array: Option<&'a S3>, +) -> datafusion_common::Result +where + &'a S1: StringArrayType<'a>, + &'a S2: StringArrayType<'a>, + &'a S3: StringArrayType<'a>, +{ + if array.len() != regex_array.len() { + return Err(DataFusionError::Execution( + "Cannot perform comparison operation on arrays of different length" + .to_string(), + )); + } + + let nulls = NullBuffer::union(array.nulls(), regex_array.nulls()); + + let mut patterns: HashMap = HashMap::new(); + let mut result = BooleanBufferBuilder::new(array.len()); + + let complete_pattern = match flags_array { + Some(flags) => Box::new(regex_array.iter().zip(flags.iter()).map( + |(pattern, flags)| { + pattern.map(|pattern| match flags { + Some(flag) => format!("(?{flag}){pattern}"), + None => pattern.to_string(), + }) + }, + )) as Box>>, + None => Box::new( + regex_array + .iter() + .map(|pattern| pattern.map(|pattern| pattern.to_string())), + ), + }; + + array + .iter() + .zip(complete_pattern) + .map(|(value, pattern)| { + match (value, pattern) { + (Some(_), Some(pattern)) if pattern == *"" => { + result.append(true); + } + (Some(value), Some(pattern)) => { + let existing_pattern = patterns.get(&pattern); + let re = match existing_pattern { + Some(re) => re, + None => { + let re = Regex::new(pattern.as_str()).map_err(|e| { + DataFusionError::Execution(format!( + "Regular expression did not compile: {e:?}" + )) + })?; + patterns.entry(pattern).or_insert(re) + } + }; + result.append(re.is_match(value)); + } + _ => result.append(false), + } + Ok(()) + }) + .collect::, DataFusionError>>()?; + + let data = unsafe { + ArrayDataBuilder::new(DataType::Boolean) + .len(array.len()) + .buffers(vec![result.into()]) + .nulls(nulls) + .build_unchecked() + }; + + Ok(BooleanArray::from(data)) +} diff --git a/datafusion/functions/src/string/contains.rs b/datafusion/functions/src/string/contains.rs index faf979f80614..c319f80661c3 100644 --- a/datafusion/functions/src/string/contains.rs +++ b/datafusion/functions/src/string/contains.rs @@ -15,19 +15,22 @@ // specific language governing permissions and limitations // under the License. +use crate::regexp_common::regexp_is_match_utf8; use crate::utils::make_scalar_function; -use arrow::array::{ArrayRef, OffsetSizeTrait}; + +use arrow::array::{Array, ArrayRef, AsArray, GenericStringArray, StringViewArray}; use arrow::datatypes::DataType; -use arrow::datatypes::DataType::Boolean; -use datafusion_common::cast::as_generic_string_array; +use arrow::datatypes::DataType::{Boolean, LargeUtf8, Utf8, Utf8View}; +use datafusion_common::exec_err; use datafusion_common::DataFusionError; use datafusion_common::Result; -use datafusion_common::{arrow_datafusion_err, exec_err}; use datafusion_expr::ScalarUDFImpl; use datafusion_expr::TypeSignature::Exact; use datafusion_expr::{ColumnarValue, Signature, Volatility}; + use std::any::Any; use std::sync::Arc; + #[derive(Debug)] pub struct ContainsFunc { signature: Signature, @@ -44,7 +47,17 @@ impl ContainsFunc { use DataType::*; Self { signature: Signature::one_of( - vec![Exact(vec![Utf8, Utf8]), Exact(vec![LargeUtf8, LargeUtf8])], + vec![ + Exact(vec![Utf8View, Utf8View]), + Exact(vec![Utf8View, Utf8]), + Exact(vec![Utf8View, LargeUtf8]), + Exact(vec![Utf8, Utf8View]), + Exact(vec![Utf8, Utf8]), + Exact(vec![Utf8, LargeUtf8]), + Exact(vec![LargeUtf8, Utf8View]), + Exact(vec![LargeUtf8, Utf8]), + Exact(vec![LargeUtf8, LargeUtf8]), + ], Volatility::Immutable, ), } @@ -69,28 +82,116 @@ impl ScalarUDFImpl for ContainsFunc { } fn invoke(&self, args: &[ColumnarValue]) -> Result { - match args[0].data_type() { - DataType::Utf8 => make_scalar_function(contains::, vec![])(args), - DataType::LargeUtf8 => make_scalar_function(contains::, vec![])(args), - other => { - exec_err!("unsupported data type {other:?} for function contains") - } - } + make_scalar_function(contains, vec![])(args) } } /// use regexp_is_match_utf8_scalar to do the calculation for contains -pub fn contains( - args: &[ArrayRef], -) -> Result { - let mod_str = as_generic_string_array::(&args[0])?; - let match_str = as_generic_string_array::(&args[1])?; - let res = arrow::compute::kernels::comparison::regexp_is_match_utf8( - mod_str, match_str, None, - ) - .map_err(|e| arrow_datafusion_err!(e))?; - - Ok(Arc::new(res) as ArrayRef) +pub fn contains(args: &[ArrayRef]) -> Result { + match (args[0].data_type(), args[1].data_type()) { + (Utf8View, Utf8View) => { + let mod_str = args[0].as_string_view(); + let match_str = args[1].as_string_view(); + let res = regexp_is_match_utf8::< + StringViewArray, + StringViewArray, + GenericStringArray, + >(mod_str, match_str, None)?; + + Ok(Arc::new(res) as ArrayRef) + } + (Utf8View, Utf8) => { + let mod_str = args[0].as_string_view(); + let match_str = args[1].as_string::(); + let res = regexp_is_match_utf8::< + StringViewArray, + GenericStringArray, + GenericStringArray, + >(mod_str, match_str, None)?; + + Ok(Arc::new(res) as ArrayRef) + } + (Utf8View, LargeUtf8) => { + let mod_str = args[0].as_string_view(); + let match_str = args[1].as_string::(); + let res = regexp_is_match_utf8::< + StringViewArray, + GenericStringArray, + GenericStringArray, + >(mod_str, match_str, None)?; + + Ok(Arc::new(res) as ArrayRef) + } + (Utf8, Utf8View) => { + let mod_str = args[0].as_string::(); + let match_str = args[1].as_string_view(); + let res = regexp_is_match_utf8::< + GenericStringArray, + StringViewArray, + GenericStringArray, + >(mod_str, match_str, None)?; + + Ok(Arc::new(res) as ArrayRef) + } + (Utf8, Utf8) => { + let mod_str = args[0].as_string::(); + let match_str = args[1].as_string::(); + let res = regexp_is_match_utf8::< + GenericStringArray, + GenericStringArray, + GenericStringArray, + >(mod_str, match_str, None)?; + + Ok(Arc::new(res) as ArrayRef) + } + (Utf8, LargeUtf8) => { + let mod_str = args[0].as_string::(); + let match_str = args[1].as_string::(); + let res = regexp_is_match_utf8::< + GenericStringArray, + GenericStringArray, + GenericStringArray, + >(mod_str, match_str, None)?; + + Ok(Arc::new(res) as ArrayRef) + } + (LargeUtf8, Utf8View) => { + let mod_str = args[0].as_string::(); + let match_str = args[1].as_string_view(); + let res = regexp_is_match_utf8::< + GenericStringArray, + StringViewArray, + GenericStringArray, + >(mod_str, match_str, None)?; + + Ok(Arc::new(res) as ArrayRef) + } + (LargeUtf8, Utf8) => { + let mod_str = args[0].as_string::(); + let match_str = args[1].as_string::(); + let res = regexp_is_match_utf8::< + GenericStringArray, + GenericStringArray, + GenericStringArray, + >(mod_str, match_str, None)?; + + Ok(Arc::new(res) as ArrayRef) + } + (LargeUtf8, LargeUtf8) => { + let mod_str = args[0].as_string::(); + let match_str = args[1].as_string::(); + let res = regexp_is_match_utf8::< + GenericStringArray, + GenericStringArray, + GenericStringArray, + >(mod_str, match_str, None)?; + + Ok(Arc::new(res) as ArrayRef) + } + other => { + exec_err!("Unsupported data type {other:?} for function `contains`.") + } + } } #[cfg(test)] @@ -138,6 +239,49 @@ mod tests { Boolean, BooleanArray ); + + test_function!( + ContainsFunc::new(), + &[ + ColumnarValue::Scalar(ScalarValue::Utf8View(Some(String::from( + "Apache" + )))), + ColumnarValue::Scalar(ScalarValue::Utf8View(Some(String::from("pac")))), + ], + Ok(Some(true)), + bool, + Boolean, + BooleanArray + ); + test_function!( + ContainsFunc::new(), + &[ + ColumnarValue::Scalar(ScalarValue::Utf8View(Some(String::from( + "Apache" + )))), + ColumnarValue::Scalar(ScalarValue::Utf8(Some(String::from("ap")))), + ], + Ok(Some(false)), + bool, + Boolean, + BooleanArray + ); + test_function!( + ContainsFunc::new(), + &[ + ColumnarValue::Scalar(ScalarValue::Utf8View(Some(String::from( + "Apache" + )))), + ColumnarValue::Scalar(ScalarValue::LargeUtf8(Some(String::from( + "DataFusion" + )))), + ], + Ok(Some(false)), + bool, + Boolean, + BooleanArray + ); + Ok(()) } } diff --git a/datafusion/sqllogictest/test_files/string_view.slt b/datafusion/sqllogictest/test_files/string_view.slt index f478ba47aa4c..171b8ec6c1d1 100644 --- a/datafusion/sqllogictest/test_files/string_view.slt +++ b/datafusion/sqllogictest/test_files/string_view.slt @@ -21,11 +21,10 @@ statement ok create table test_source as values - ('Andrew', 'X'), - ('Xiangpeng', 'Xiangpeng'), - ('Raphael', 'R'), - (NULL, 'R') -; + ('Andrew', 'X'), + ('Xiangpeng', 'Xiangpeng'), + ('Raphael', 'R'), + (NULL, 'R'); # Table with the different combination of column types statement ok @@ -800,17 +799,40 @@ logical_plan 02)--TableScan: test projection=[column1_utf8view, column2_utf8view] ## Ensure no casts for CONTAINS -## TODO https://github.com/apache/datafusion/issues/11838 query TT EXPLAIN SELECT CONTAINS(column1_utf8view, 'foo') as c1, - CONTAINS(column2_utf8view, column2_utf8view) as c2 + CONTAINS(column1_utf8view, column2_utf8view) as c2, + CONTAINS(column1_utf8view, column2_large_utf8) as c3, + CONTAINS(column1_utf8, column2_utf8view) as c4, + CONTAINS(column1_utf8, column2_utf8) as c5, + CONTAINS(column1_utf8, column2_large_utf8) as c6, + CONTAINS(column1_large_utf8, column1_utf8view) as c7, + CONTAINS(column1_large_utf8, column2_utf8) as c8, + CONTAINS(column1_large_utf8, column2_large_utf8) as c9 FROM test; ---- logical_plan -01)Projection: contains(CAST(test.column1_utf8view AS Utf8), Utf8("foo")) AS c1, contains(__common_expr_1, __common_expr_1) AS c2 -02)--Projection: CAST(test.column2_utf8view AS Utf8) AS __common_expr_1, test.column1_utf8view -03)----TableScan: test projection=[column1_utf8view, column2_utf8view] +01)Projection: contains(test.column1_utf8view, Utf8("foo")) AS c1, contains(test.column1_utf8view, test.column2_utf8view) AS c2, contains(test.column1_utf8view, test.column2_large_utf8) AS c3, contains(test.column1_utf8, test.column2_utf8view) AS c4, contains(test.column1_utf8, test.column2_utf8) AS c5, contains(test.column1_utf8, test.column2_large_utf8) AS c6, contains(test.column1_large_utf8, test.column1_utf8view) AS c7, contains(test.column1_large_utf8, test.column2_utf8) AS c8, contains(test.column1_large_utf8, test.column2_large_utf8) AS c9 +02)--TableScan: test projection=[column1_utf8, column2_utf8, column1_large_utf8, column2_large_utf8, column1_utf8view, column2_utf8view] + +query BBBBBBBBB +SELECT + CONTAINS(column1_utf8view, 'foo') as c1, + CONTAINS(column1_utf8view, column2_utf8view) as c2, + CONTAINS(column1_utf8view, column2_large_utf8) as c3, + CONTAINS(column1_utf8, column2_utf8view) as c4, + CONTAINS(column1_utf8, column2_utf8) as c5, + CONTAINS(column1_utf8, column2_large_utf8) as c6, + CONTAINS(column1_large_utf8, column1_utf8view) as c7, + CONTAINS(column1_large_utf8, column2_utf8) as c8, + CONTAINS(column1_large_utf8, column2_large_utf8) as c9 +FROM test; +---- +false false false false false false true false false +false true true true true true true true true +false true true true true true true true true +NULL NULL NULL NULL NULL NULL NULL NULL NULL ## Ensure no casts for ENDS_WITH query TT diff --git a/docs/source/user-guide/sql/scalar_functions.md b/docs/source/user-guide/sql/scalar_functions.md index c3d3ab7a64a7..e08524dcd3a7 100644 --- a/docs/source/user-guide/sql/scalar_functions.md +++ b/docs/source/user-guide/sql/scalar_functions.md @@ -1454,7 +1454,7 @@ position(substr in origstr) ### `contains` -Return true if search_string is found within string. +Return true if search_string is found within string (case-sensitive). ``` contains(string, search_string) From 41c5f4ed67fd48c27fb923c3c5cf8fd33f126ca4 Mon Sep 17 00:00:00 2001 From: Victor Barua Date: Tue, 10 Sep 2024 03:58:06 -0700 Subject: [PATCH 137/154] validate and adjust Substrait NamedTable schemas (#12223) (#12245) * fix: producer did not emit base_schema struct field for ReadRel Substrait plans are not valid without this, and it is generally useful for round trip testing * feat: include field_qualifier param for from_substrait_named_struct * feat: verify that Substrait and DataFusion agree on NamedScan schemas If the schema registered with DataFusion and the schema as given by the Substrait NamedScan do not have the same names and types, DataFusion should reject it * test: update existing substrait test + substrait validation test * added substrait_validation test * extracted useful test utilities The utils::test::TestSchemaCollector::generate_context_from_plan function can be used to dynamically generate a SessionContext from a Substrait plan, which will include the schemas for NamedTables as given in the Substrait plan. This helps us avoid the issue of DataFusion test schemas and Substrait plan schemas not being in sync. * feat: expose from_substrait_named_struct * refactor: remove unused imports * docs: add missing licenses * refactor: deal with unused code warnings * remove optional qualifier from from_substrait_named_struct * return DFSchema from from_substrait_named_struct * one must imagine clippy happy * accidental blah * loosen the validation for schemas allow cases where the Substrait schema is a subset of the DataFusion schema * minor doc tweaks * update test data to deal with case issues in tests * fix error message * improve readability of field compatability check * make TestSchemaCollector more flexible * fix doc typo Co-authored-by: Arttu * remove unecessary TODO * handle ReadRel projection on top of mismatched schema --------- Co-authored-by: Arttu --- .../substrait/src/logical_plan/consumer.rs | 131 +++++++++++- .../substrait/src/logical_plan/producer.rs | 16 +- .../tests/cases/consumer_integration.rs | 56 +++--- .../substrait/tests/cases/function_test.rs | 32 +-- .../substrait/tests/cases/logical_plans.rs | 59 ++---- datafusion/substrait/tests/cases/mod.rs | 1 + .../tests/cases/substrait_validations.rs | 151 ++++++++++++++ .../substrait/tests/substrait_integration.rs | 1 + .../test_plans/simple_select.substrait.json | 69 +++++++ .../simple_select_with_mask.substrait.json | 104 ++++++++++ .../substrait/tests/testdata/tpch/nation.csv | 2 +- .../substrait/tests/testdata/tpch/region.csv | 2 +- datafusion/substrait/tests/utils.rs | 186 ++++++++++++++++++ 13 files changed, 697 insertions(+), 113 deletions(-) create mode 100644 datafusion/substrait/tests/cases/substrait_validations.rs create mode 100644 datafusion/substrait/tests/testdata/test_plans/simple_select.substrait.json create mode 100644 datafusion/substrait/tests/testdata/test_plans/simple_select_with_mask.substrait.json create mode 100644 datafusion/substrait/tests/utils.rs diff --git a/datafusion/substrait/src/logical_plan/consumer.rs b/datafusion/substrait/src/logical_plan/consumer.rs index 01a854ffbdf2..c9d27237a49b 100644 --- a/datafusion/substrait/src/logical_plan/consumer.rs +++ b/datafusion/substrait/src/logical_plan/consumer.rs @@ -53,6 +53,8 @@ use crate::variation_const::{ }; use datafusion::arrow::array::{new_empty_array, AsArray}; use datafusion::common::scalar::ScalarStructBuilder; +use datafusion::dataframe::DataFrame; +use datafusion::logical_expr::builder::project; use datafusion::logical_expr::expr::InList; use datafusion::logical_expr::{ col, expr, Cast, Extension, GroupingSet, Like, LogicalPlanBuilder, Partitioning, @@ -277,6 +279,20 @@ pub fn extract_projection( ); Ok(LogicalPlan::TableScan(scan)) } + LogicalPlan::Projection(projection) => { + // create another Projection around the Projection to handle the field masking + let fields: Vec = column_indices + .into_iter() + .map(|i| { + let (qualifier, field) = + projection.schema.qualified_field(i); + let column = + Column::new(qualifier.cloned(), field.name()); + Expr::Column(column) + }) + .collect(); + project(LogicalPlan::Projection(projection), fields) + } _ => plan_err!("unexpected plan for table"), } } @@ -640,6 +656,10 @@ pub async fn from_substrait_rel( } Some(RelType::Read(read)) => match &read.as_ref().read_type { Some(ReadType::NamedTable(nt)) => { + let named_struct = read.base_schema.as_ref().ok_or_else(|| { + substrait_datafusion_err!("No base schema provided for Named Table") + })?; + let table_reference = match nt.names.len() { 0 => { return plan_err!("No table name found in NamedTable"); @@ -657,7 +677,13 @@ pub async fn from_substrait_rel( table: nt.names[2].clone().into(), }, }; - let t = ctx.table(table_reference).await?; + + let substrait_schema = + from_substrait_named_struct(named_struct, extensions)? + .replace_qualifier(table_reference.clone()); + + let t = ctx.table(table_reference.clone()).await?; + let t = ensure_schema_compatability(t, substrait_schema)?; let t = t.into_optimized_plan()?; extract_projection(t, &read.projection) } @@ -671,7 +697,7 @@ pub async fn from_substrait_rel( if vt.values.is_empty() { return Ok(LogicalPlan::EmptyRelation(EmptyRelation { produce_one_row: false, - schema, + schema: DFSchemaRef::new(schema), })); } @@ -704,7 +730,10 @@ pub async fn from_substrait_rel( }) .collect::>()?; - Ok(LogicalPlan::Values(Values { schema, values })) + Ok(LogicalPlan::Values(Values { + schema: DFSchemaRef::new(schema), + values, + })) } Some(ReadType::LocalFiles(lf)) => { fn extract_filename(name: &str) -> Option { @@ -850,6 +879,87 @@ pub async fn from_substrait_rel( } } +/// Ensures that the given Substrait schema is compatible with the schema as given by DataFusion +/// +/// This means: +/// 1. All fields present in the Substrait schema are present in the DataFusion schema. The +/// DataFusion schema may have MORE fields, but not the other way around. +/// 2. All fields are compatible. See [`ensure_field_compatability`] for details +/// +/// This function returns a DataFrame with fields adjusted if necessary in the event that the +/// Substrait schema is a subset of the DataFusion schema. +fn ensure_schema_compatability( + table: DataFrame, + substrait_schema: DFSchema, +) -> Result { + let df_schema = table.schema().to_owned().strip_qualifiers(); + if df_schema.logically_equivalent_names_and_types(&substrait_schema) { + return Ok(table); + } + let selected_columns = substrait_schema + .strip_qualifiers() + .fields() + .iter() + .map(|substrait_field| { + let df_field = + df_schema.field_with_unqualified_name(substrait_field.name())?; + ensure_field_compatability(df_field, substrait_field)?; + Ok(col(format!("\"{}\"", df_field.name()))) + }) + .collect::>()?; + + table.select(selected_columns) +} + +/// Ensures that the given Substrait field is compatible with the given DataFusion field +/// +/// A field is compatible between Substrait and DataFusion if: +/// 1. They have logically equivalent types. +/// 2. They have the same nullability OR the Substrait field is nullable and the DataFusion fields +/// is not nullable. +/// +/// If a Substrait field is not nullable, the Substrait plan may be built around assuming it is not +/// nullable. As such if DataFusion has that field as nullable the plan should be rejected. +fn ensure_field_compatability( + datafusion_field: &Field, + substrait_field: &Field, +) -> Result<()> { + if !DFSchema::datatype_is_logically_equal( + datafusion_field.data_type(), + substrait_field.data_type(), + ) { + return substrait_err!( + "Field '{}' in Substrait schema has a different type ({}) than the corresponding field in the table schema ({}).", + substrait_field.name(), + substrait_field.data_type(), + datafusion_field.data_type() + ); + } + + if !compatible_nullabilities( + datafusion_field.is_nullable(), + substrait_field.is_nullable(), + ) { + // TODO: from_substrait_struct_type needs to be updated to set the nullability correctly. It defaults to true for now. + return substrait_err!( + "Field '{}' is nullable in the DataFusion schema but not nullable in the Substrait schema.", + substrait_field.name() + ); + } + Ok(()) +} + +/// Returns true if the DataFusion and Substrait nullabilities are compatible, false otherwise +fn compatible_nullabilities( + datafusion_nullability: bool, + substrait_nullability: bool, +) -> bool { + // DataFusion and Substrait have the same nullability + (datafusion_nullability == substrait_nullability) + // DataFusion is not nullable and Substrait is nullable + || (!datafusion_nullability && substrait_nullability) +} + /// (Re)qualify the sides of a join if needed, i.e. if the columns from one side would otherwise /// conflict with the columns from the other. /// Substrait doesn't currently allow specifying aliases, neither for columns nor for tables. For @@ -1588,10 +1698,11 @@ fn next_struct_field_name( } } -fn from_substrait_named_struct( +/// Convert Substrait NamedStruct to DataFusion DFSchemaRef +pub fn from_substrait_named_struct( base_schema: &NamedStruct, extensions: &Extensions, -) -> Result { +) -> Result { let mut name_idx = 0; let fields = from_substrait_struct_type( base_schema.r#struct.as_ref().ok_or_else(|| { @@ -1603,12 +1714,12 @@ fn from_substrait_named_struct( ); if name_idx != base_schema.names.len() { return substrait_err!( - "Names list must match exactly to nested schema, but found {} uses for {} names", - name_idx, - base_schema.names.len() - ); + "Names list must match exactly to nested schema, but found {} uses for {} names", + name_idx, + base_schema.names.len() + ); } - Ok(DFSchemaRef::new(DFSchema::try_from(Schema::new(fields?))?)) + DFSchema::try_from(Schema::new(fields?)) } fn from_substrait_bound( diff --git a/datafusion/substrait/src/logical_plan/producer.rs b/datafusion/substrait/src/logical_plan/producer.rs index f323ae146600..a923aaf31abb 100644 --- a/datafusion/substrait/src/logical_plan/producer.rs +++ b/datafusion/substrait/src/logical_plan/producer.rs @@ -42,8 +42,8 @@ use crate::variation_const::{ use datafusion::arrow::array::{Array, GenericListArray, OffsetSizeTrait}; use datafusion::common::{ exec_err, internal_err, not_impl_err, plan_err, substrait_datafusion_err, + substrait_err, DFSchemaRef, ToDFSchema, }; -use datafusion::common::{substrait_err, DFSchemaRef}; #[allow(unused_imports)] use datafusion::logical_expr::expr::{ Alias, BinaryExpr, Case, Cast, GroupingSet, InList, InSubquery, Sort, WindowFunction, @@ -139,19 +139,13 @@ pub fn to_substrait_rel( maintain_singular_struct: false, }); + let table_schema = scan.source.schema().to_dfschema_ref()?; + let base_schema = to_substrait_named_struct(&table_schema, extensions)?; + Ok(Box::new(Rel { rel_type: Some(RelType::Read(Box::new(ReadRel { common: None, - base_schema: Some(NamedStruct { - names: scan - .source - .schema() - .fields() - .iter() - .map(|f| f.name().to_owned()) - .collect(), - r#struct: None, - }), + base_schema: Some(base_schema), filter: None, best_effort_filter: None, projection, diff --git a/datafusion/substrait/tests/cases/consumer_integration.rs b/datafusion/substrait/tests/cases/consumer_integration.rs index 0a86d27e013c..dad24559a06f 100644 --- a/datafusion/substrait/tests/cases/consumer_integration.rs +++ b/datafusion/substrait/tests/cases/consumer_integration.rs @@ -92,22 +92,22 @@ mod tests { let plan_str = format!("{}", plan); assert_eq!( plan_str, - "Projection: FILENAME_PLACEHOLDER_1.s_acctbal AS S_ACCTBAL, FILENAME_PLACEHOLDER_1.s_name AS S_NAME, FILENAME_PLACEHOLDER_3.n_name AS N_NAME, FILENAME_PLACEHOLDER_0.p_partkey AS P_PARTKEY, FILENAME_PLACEHOLDER_0.p_mfgr AS P_MFGR, FILENAME_PLACEHOLDER_1.s_address AS S_ADDRESS, FILENAME_PLACEHOLDER_1.s_phone AS S_PHONE, FILENAME_PLACEHOLDER_1.s_comment AS S_COMMENT\ + "Projection: FILENAME_PLACEHOLDER_1.s_acctbal AS S_ACCTBAL, FILENAME_PLACEHOLDER_1.s_name AS S_NAME, FILENAME_PLACEHOLDER_3.N_NAME, FILENAME_PLACEHOLDER_0.p_partkey AS P_PARTKEY, FILENAME_PLACEHOLDER_0.p_mfgr AS P_MFGR, FILENAME_PLACEHOLDER_1.s_address AS S_ADDRESS, FILENAME_PLACEHOLDER_1.s_phone AS S_PHONE, FILENAME_PLACEHOLDER_1.s_comment AS S_COMMENT\ \n Limit: skip=0, fetch=100\ - \n Sort: FILENAME_PLACEHOLDER_1.s_acctbal DESC NULLS FIRST, FILENAME_PLACEHOLDER_3.n_name ASC NULLS LAST, FILENAME_PLACEHOLDER_1.s_name ASC NULLS LAST, FILENAME_PLACEHOLDER_0.p_partkey ASC NULLS LAST\ - \n Projection: FILENAME_PLACEHOLDER_1.s_acctbal, FILENAME_PLACEHOLDER_1.s_name, FILENAME_PLACEHOLDER_3.n_name, FILENAME_PLACEHOLDER_0.p_partkey, FILENAME_PLACEHOLDER_0.p_mfgr, FILENAME_PLACEHOLDER_1.s_address, FILENAME_PLACEHOLDER_1.s_phone, FILENAME_PLACEHOLDER_1.s_comment\ - \n Filter: FILENAME_PLACEHOLDER_0.p_partkey = FILENAME_PLACEHOLDER_2.ps_partkey AND FILENAME_PLACEHOLDER_1.s_suppkey = FILENAME_PLACEHOLDER_2.ps_suppkey AND FILENAME_PLACEHOLDER_0.p_size = Int32(15) AND FILENAME_PLACEHOLDER_0.p_type LIKE CAST(Utf8(\"%BRASS\") AS Utf8) AND FILENAME_PLACEHOLDER_1.s_nationkey = FILENAME_PLACEHOLDER_3.n_nationkey AND FILENAME_PLACEHOLDER_3.n_regionkey = FILENAME_PLACEHOLDER_4.r_regionkey AND FILENAME_PLACEHOLDER_4.r_name = CAST(Utf8(\"EUROPE\") AS Utf8) AND FILENAME_PLACEHOLDER_2.ps_supplycost = ()\ + \n Sort: FILENAME_PLACEHOLDER_1.s_acctbal DESC NULLS FIRST, FILENAME_PLACEHOLDER_3.N_NAME ASC NULLS LAST, FILENAME_PLACEHOLDER_1.s_name ASC NULLS LAST, FILENAME_PLACEHOLDER_0.p_partkey ASC NULLS LAST\ + \n Projection: FILENAME_PLACEHOLDER_1.s_acctbal, FILENAME_PLACEHOLDER_1.s_name, FILENAME_PLACEHOLDER_3.N_NAME, FILENAME_PLACEHOLDER_0.p_partkey, FILENAME_PLACEHOLDER_0.p_mfgr, FILENAME_PLACEHOLDER_1.s_address, FILENAME_PLACEHOLDER_1.s_phone, FILENAME_PLACEHOLDER_1.s_comment\ + \n Filter: FILENAME_PLACEHOLDER_0.p_partkey = FILENAME_PLACEHOLDER_2.ps_partkey AND FILENAME_PLACEHOLDER_1.s_suppkey = FILENAME_PLACEHOLDER_2.ps_suppkey AND FILENAME_PLACEHOLDER_0.p_size = Int32(15) AND FILENAME_PLACEHOLDER_0.p_type LIKE CAST(Utf8(\"%BRASS\") AS Utf8) AND FILENAME_PLACEHOLDER_1.s_nationkey = FILENAME_PLACEHOLDER_3.N_NATIONKEY AND FILENAME_PLACEHOLDER_3.N_REGIONKEY = FILENAME_PLACEHOLDER_4.R_REGIONKEY AND FILENAME_PLACEHOLDER_4.R_NAME = CAST(Utf8(\"EUROPE\") AS Utf8) AND FILENAME_PLACEHOLDER_2.ps_supplycost = ()\ \n Subquery:\ \n Aggregate: groupBy=[[]], aggr=[[min(FILENAME_PLACEHOLDER_5.ps_supplycost)]]\ \n Projection: FILENAME_PLACEHOLDER_5.ps_supplycost\ - \n Filter: FILENAME_PLACEHOLDER_5.ps_partkey = FILENAME_PLACEHOLDER_5.ps_partkey AND FILENAME_PLACEHOLDER_6.s_suppkey = FILENAME_PLACEHOLDER_5.ps_suppkey AND FILENAME_PLACEHOLDER_6.s_nationkey = FILENAME_PLACEHOLDER_7.n_nationkey AND FILENAME_PLACEHOLDER_7.n_regionkey = FILENAME_PLACEHOLDER_8.r_regionkey AND FILENAME_PLACEHOLDER_8.r_name = CAST(Utf8(\"EUROPE\") AS Utf8)\ + \n Filter: FILENAME_PLACEHOLDER_5.ps_partkey = FILENAME_PLACEHOLDER_5.ps_partkey AND FILENAME_PLACEHOLDER_6.s_suppkey = FILENAME_PLACEHOLDER_5.ps_suppkey AND FILENAME_PLACEHOLDER_6.s_nationkey = FILENAME_PLACEHOLDER_7.N_NATIONKEY AND FILENAME_PLACEHOLDER_7.N_REGIONKEY = FILENAME_PLACEHOLDER_8.R_REGIONKEY AND FILENAME_PLACEHOLDER_8.R_NAME = CAST(Utf8(\"EUROPE\") AS Utf8)\ \n Inner Join: Filter: Boolean(true)\ \n Inner Join: Filter: Boolean(true)\ \n Inner Join: Filter: Boolean(true)\ \n TableScan: FILENAME_PLACEHOLDER_5 projection=[ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment]\ \n TableScan: FILENAME_PLACEHOLDER_6 projection=[s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment]\ - \n TableScan: FILENAME_PLACEHOLDER_7 projection=[n_nationkey, n_name, n_regionkey, n_comment]\ - \n TableScan: FILENAME_PLACEHOLDER_8 projection=[r_regionkey, r_name, r_comment]\ + \n TableScan: FILENAME_PLACEHOLDER_7 projection=[N_NATIONKEY, N_NAME, N_REGIONKEY, N_COMMENT]\ + \n TableScan: FILENAME_PLACEHOLDER_8 projection=[R_REGIONKEY, R_NAME, R_COMMENT]\ \n Inner Join: Filter: Boolean(true)\ \n Inner Join: Filter: Boolean(true)\ \n Inner Join: Filter: Boolean(true)\ @@ -115,8 +115,8 @@ mod tests { \n TableScan: FILENAME_PLACEHOLDER_0 projection=[p_partkey, p_name, p_mfgr, p_brand, p_type, p_size, p_container, p_retailprice, p_comment]\ \n TableScan: FILENAME_PLACEHOLDER_1 projection=[s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment]\ \n TableScan: FILENAME_PLACEHOLDER_2 projection=[ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment]\ - \n TableScan: FILENAME_PLACEHOLDER_3 projection=[n_nationkey, n_name, n_regionkey, n_comment]\ - \n TableScan: FILENAME_PLACEHOLDER_4 projection=[r_regionkey, r_name, r_comment]" + \n TableScan: FILENAME_PLACEHOLDER_3 projection=[N_NATIONKEY, N_NAME, N_REGIONKEY, N_COMMENT]\ + \n TableScan: FILENAME_PLACEHOLDER_4 projection=[R_REGIONKEY, R_NAME, R_COMMENT]" ); Ok(()) } @@ -196,11 +196,11 @@ mod tests { let plan = from_substrait_plan(&ctx, &proto).await?; let plan_str = format!("{}", plan); - assert_eq!(plan_str, "Projection: NATION.n_name AS N_NAME, sum(FILENAME_PLACEHOLDER_2.l_extendedprice * Int32(1) - FILENAME_PLACEHOLDER_2.l_discount) AS REVENUE\ + assert_eq!(plan_str, "Projection: NATION.N_NAME, sum(FILENAME_PLACEHOLDER_2.l_extendedprice * Int32(1) - FILENAME_PLACEHOLDER_2.l_discount) AS REVENUE\ \n Sort: sum(FILENAME_PLACEHOLDER_2.l_extendedprice * Int32(1) - FILENAME_PLACEHOLDER_2.l_discount) DESC NULLS FIRST\ - \n Aggregate: groupBy=[[NATION.n_name]], aggr=[[sum(FILENAME_PLACEHOLDER_2.l_extendedprice * Int32(1) - FILENAME_PLACEHOLDER_2.l_discount)]]\ - \n Projection: NATION.n_name, FILENAME_PLACEHOLDER_2.l_extendedprice * (CAST(Int32(1) AS Decimal128(19, 0)) - FILENAME_PLACEHOLDER_2.l_discount)\ - \n Filter: FILENAME_PLACEHOLDER_0.c_custkey = FILENAME_PLACEHOLDER_1.o_custkey AND FILENAME_PLACEHOLDER_2.l_orderkey = FILENAME_PLACEHOLDER_1.o_orderkey AND FILENAME_PLACEHOLDER_2.l_suppkey = FILENAME_PLACEHOLDER_3.s_suppkey AND FILENAME_PLACEHOLDER_0.c_nationkey = FILENAME_PLACEHOLDER_3.s_nationkey AND FILENAME_PLACEHOLDER_3.s_nationkey = NATION.n_nationkey AND NATION.n_regionkey = REGION.r_regionkey AND REGION.r_name = CAST(Utf8(\"ASIA\") AS Utf8) AND FILENAME_PLACEHOLDER_1.o_orderdate >= CAST(Utf8(\"1994-01-01\") AS Date32) AND FILENAME_PLACEHOLDER_1.o_orderdate < CAST(Utf8(\"1995-01-01\") AS Date32)\ + \n Aggregate: groupBy=[[NATION.N_NAME]], aggr=[[sum(FILENAME_PLACEHOLDER_2.l_extendedprice * Int32(1) - FILENAME_PLACEHOLDER_2.l_discount)]]\ + \n Projection: NATION.N_NAME, FILENAME_PLACEHOLDER_2.l_extendedprice * (CAST(Int32(1) AS Decimal128(19, 0)) - FILENAME_PLACEHOLDER_2.l_discount)\ + \n Filter: FILENAME_PLACEHOLDER_0.c_custkey = FILENAME_PLACEHOLDER_1.o_custkey AND FILENAME_PLACEHOLDER_2.l_orderkey = FILENAME_PLACEHOLDER_1.o_orderkey AND FILENAME_PLACEHOLDER_2.l_suppkey = FILENAME_PLACEHOLDER_3.s_suppkey AND FILENAME_PLACEHOLDER_0.c_nationkey = FILENAME_PLACEHOLDER_3.s_nationkey AND FILENAME_PLACEHOLDER_3.s_nationkey = NATION.N_NATIONKEY AND NATION.N_REGIONKEY = REGION.R_REGIONKEY AND REGION.R_NAME = CAST(Utf8(\"ASIA\") AS Utf8) AND FILENAME_PLACEHOLDER_1.o_orderdate >= CAST(Utf8(\"1994-01-01\") AS Date32) AND FILENAME_PLACEHOLDER_1.o_orderdate < CAST(Utf8(\"1995-01-01\") AS Date32)\ \n Inner Join: Filter: Boolean(true)\ \n Inner Join: Filter: Boolean(true)\ \n Inner Join: Filter: Boolean(true)\ @@ -210,8 +210,8 @@ mod tests { \n TableScan: FILENAME_PLACEHOLDER_1 projection=[o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment]\ \n TableScan: FILENAME_PLACEHOLDER_2 projection=[l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment]\ \n TableScan: FILENAME_PLACEHOLDER_3 projection=[s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment]\ - \n TableScan: NATION projection=[n_nationkey, n_name, n_regionkey, n_comment]\ - \n TableScan: REGION projection=[r_regionkey, r_name, r_comment]"); + \n TableScan: NATION projection=[N_NATIONKEY, N_NAME, N_REGIONKEY, N_COMMENT]\ + \n TableScan: REGION projection=[R_REGIONKEY, R_NAME, R_COMMENT]"); Ok(()) } @@ -255,19 +255,19 @@ mod tests { let plan = from_substrait_plan(&ctx, &proto).await?; let plan_str = format!("{}", plan); - assert_eq!(plan_str, "Projection: FILENAME_PLACEHOLDER_0.c_custkey AS C_CUSTKEY, FILENAME_PLACEHOLDER_0.c_name AS C_NAME, sum(FILENAME_PLACEHOLDER_2.l_extendedprice * Int32(1) - FILENAME_PLACEHOLDER_2.l_discount) AS REVENUE, FILENAME_PLACEHOLDER_0.c_acctbal AS C_ACCTBAL, FILENAME_PLACEHOLDER_3.n_name AS N_NAME, FILENAME_PLACEHOLDER_0.c_address AS C_ADDRESS, FILENAME_PLACEHOLDER_0.c_phone AS C_PHONE, FILENAME_PLACEHOLDER_0.c_comment AS C_COMMENT\ + assert_eq!(plan_str, "Projection: FILENAME_PLACEHOLDER_0.c_custkey AS C_CUSTKEY, FILENAME_PLACEHOLDER_0.c_name AS C_NAME, sum(FILENAME_PLACEHOLDER_2.l_extendedprice * Int32(1) - FILENAME_PLACEHOLDER_2.l_discount) AS REVENUE, FILENAME_PLACEHOLDER_0.c_acctbal AS C_ACCTBAL, FILENAME_PLACEHOLDER_3.N_NAME, FILENAME_PLACEHOLDER_0.c_address AS C_ADDRESS, FILENAME_PLACEHOLDER_0.c_phone AS C_PHONE, FILENAME_PLACEHOLDER_0.c_comment AS C_COMMENT\ \n Limit: skip=0, fetch=20\ \n Sort: sum(FILENAME_PLACEHOLDER_2.l_extendedprice * Int32(1) - FILENAME_PLACEHOLDER_2.l_discount) DESC NULLS FIRST\ - \n Projection: FILENAME_PLACEHOLDER_0.c_custkey, FILENAME_PLACEHOLDER_0.c_name, sum(FILENAME_PLACEHOLDER_2.l_extendedprice * Int32(1) - FILENAME_PLACEHOLDER_2.l_discount), FILENAME_PLACEHOLDER_0.c_acctbal, FILENAME_PLACEHOLDER_3.n_name, FILENAME_PLACEHOLDER_0.c_address, FILENAME_PLACEHOLDER_0.c_phone, FILENAME_PLACEHOLDER_0.c_comment\n Aggregate: groupBy=[[FILENAME_PLACEHOLDER_0.c_custkey, FILENAME_PLACEHOLDER_0.c_name, FILENAME_PLACEHOLDER_0.c_acctbal, FILENAME_PLACEHOLDER_0.c_phone, FILENAME_PLACEHOLDER_3.n_name, FILENAME_PLACEHOLDER_0.c_address, FILENAME_PLACEHOLDER_0.c_comment]], aggr=[[sum(FILENAME_PLACEHOLDER_2.l_extendedprice * Int32(1) - FILENAME_PLACEHOLDER_2.l_discount)]]\ - \n Projection: FILENAME_PLACEHOLDER_0.c_custkey, FILENAME_PLACEHOLDER_0.c_name, FILENAME_PLACEHOLDER_0.c_acctbal, FILENAME_PLACEHOLDER_0.c_phone, FILENAME_PLACEHOLDER_3.n_name, FILENAME_PLACEHOLDER_0.c_address, FILENAME_PLACEHOLDER_0.c_comment, FILENAME_PLACEHOLDER_2.l_extendedprice * (CAST(Int32(1) AS Decimal128(19, 0)) - FILENAME_PLACEHOLDER_2.l_discount)\ - \n Filter: FILENAME_PLACEHOLDER_0.c_custkey = FILENAME_PLACEHOLDER_1.o_custkey AND FILENAME_PLACEHOLDER_2.l_orderkey = FILENAME_PLACEHOLDER_1.o_orderkey AND FILENAME_PLACEHOLDER_1.o_orderdate >= CAST(Utf8(\"1993-10-01\") AS Date32) AND FILENAME_PLACEHOLDER_1.o_orderdate < CAST(Utf8(\"1994-01-01\") AS Date32) AND FILENAME_PLACEHOLDER_2.l_returnflag = Utf8(\"R\") AND FILENAME_PLACEHOLDER_0.c_nationkey = FILENAME_PLACEHOLDER_3.n_nationkey\ + \n Projection: FILENAME_PLACEHOLDER_0.c_custkey, FILENAME_PLACEHOLDER_0.c_name, sum(FILENAME_PLACEHOLDER_2.l_extendedprice * Int32(1) - FILENAME_PLACEHOLDER_2.l_discount), FILENAME_PLACEHOLDER_0.c_acctbal, FILENAME_PLACEHOLDER_3.N_NAME, FILENAME_PLACEHOLDER_0.c_address, FILENAME_PLACEHOLDER_0.c_phone, FILENAME_PLACEHOLDER_0.c_comment\n Aggregate: groupBy=[[FILENAME_PLACEHOLDER_0.c_custkey, FILENAME_PLACEHOLDER_0.c_name, FILENAME_PLACEHOLDER_0.c_acctbal, FILENAME_PLACEHOLDER_0.c_phone, FILENAME_PLACEHOLDER_3.N_NAME, FILENAME_PLACEHOLDER_0.c_address, FILENAME_PLACEHOLDER_0.c_comment]], aggr=[[sum(FILENAME_PLACEHOLDER_2.l_extendedprice * Int32(1) - FILENAME_PLACEHOLDER_2.l_discount)]]\ + \n Projection: FILENAME_PLACEHOLDER_0.c_custkey, FILENAME_PLACEHOLDER_0.c_name, FILENAME_PLACEHOLDER_0.c_acctbal, FILENAME_PLACEHOLDER_0.c_phone, FILENAME_PLACEHOLDER_3.N_NAME, FILENAME_PLACEHOLDER_0.c_address, FILENAME_PLACEHOLDER_0.c_comment, FILENAME_PLACEHOLDER_2.l_extendedprice * (CAST(Int32(1) AS Decimal128(19, 0)) - FILENAME_PLACEHOLDER_2.l_discount)\ + \n Filter: FILENAME_PLACEHOLDER_0.c_custkey = FILENAME_PLACEHOLDER_1.o_custkey AND FILENAME_PLACEHOLDER_2.l_orderkey = FILENAME_PLACEHOLDER_1.o_orderkey AND FILENAME_PLACEHOLDER_1.o_orderdate >= CAST(Utf8(\"1993-10-01\") AS Date32) AND FILENAME_PLACEHOLDER_1.o_orderdate < CAST(Utf8(\"1994-01-01\") AS Date32) AND FILENAME_PLACEHOLDER_2.l_returnflag = Utf8(\"R\") AND FILENAME_PLACEHOLDER_0.c_nationkey = FILENAME_PLACEHOLDER_3.N_NATIONKEY\ \n Inner Join: Filter: Boolean(true)\ \n Inner Join: Filter: Boolean(true)\ \n Inner Join: Filter: Boolean(true)\ \n TableScan: FILENAME_PLACEHOLDER_0 projection=[c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_mktsegment, c_comment]\ \n TableScan: FILENAME_PLACEHOLDER_1 projection=[o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment]\ \n TableScan: FILENAME_PLACEHOLDER_2 projection=[l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment]\ - \n TableScan: FILENAME_PLACEHOLDER_3 projection=[n_nationkey, n_name, n_regionkey, n_comment]"); + \n TableScan: FILENAME_PLACEHOLDER_3 projection=[N_NATIONKEY, N_NAME, N_REGIONKEY, N_COMMENT]"); Ok(()) } @@ -297,20 +297,20 @@ mod tests { \n Projection: sum(FILENAME_PLACEHOLDER_3.ps_supplycost * FILENAME_PLACEHOLDER_3.ps_availqty) * Decimal128(Some(1000000),11,10)\ \n Aggregate: groupBy=[[]], aggr=[[sum(FILENAME_PLACEHOLDER_3.ps_supplycost * FILENAME_PLACEHOLDER_3.ps_availqty)]]\ \n Projection: FILENAME_PLACEHOLDER_3.ps_supplycost * CAST(FILENAME_PLACEHOLDER_3.ps_availqty AS Decimal128(19, 0))\ - \n Filter: FILENAME_PLACEHOLDER_3.ps_suppkey = FILENAME_PLACEHOLDER_4.s_suppkey AND FILENAME_PLACEHOLDER_4.s_nationkey = FILENAME_PLACEHOLDER_5.n_nationkey AND FILENAME_PLACEHOLDER_5.n_name = CAST(Utf8(\"JAPAN\") AS Utf8)\ + \n Filter: FILENAME_PLACEHOLDER_3.ps_suppkey = FILENAME_PLACEHOLDER_4.s_suppkey AND FILENAME_PLACEHOLDER_4.s_nationkey = FILENAME_PLACEHOLDER_5.N_NATIONKEY AND FILENAME_PLACEHOLDER_5.N_NAME = CAST(Utf8(\"JAPAN\") AS Utf8)\ \n Inner Join: Filter: Boolean(true)\ \n Inner Join: Filter: Boolean(true)\ \n TableScan: FILENAME_PLACEHOLDER_3 projection=[ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment]\ \n TableScan: FILENAME_PLACEHOLDER_4 projection=[s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment]\ - \n TableScan: FILENAME_PLACEHOLDER_5 projection=[n_nationkey, n_name, n_regionkey, n_comment]\ + \n TableScan: FILENAME_PLACEHOLDER_5 projection=[N_NATIONKEY, N_NAME, N_REGIONKEY, N_COMMENT]\ \n Aggregate: groupBy=[[FILENAME_PLACEHOLDER_0.ps_partkey]], aggr=[[sum(FILENAME_PLACEHOLDER_0.ps_supplycost * FILENAME_PLACEHOLDER_0.ps_availqty)]]\ \n Projection: FILENAME_PLACEHOLDER_0.ps_partkey, FILENAME_PLACEHOLDER_0.ps_supplycost * CAST(FILENAME_PLACEHOLDER_0.ps_availqty AS Decimal128(19, 0))\ - \n Filter: FILENAME_PLACEHOLDER_0.ps_suppkey = FILENAME_PLACEHOLDER_1.s_suppkey AND FILENAME_PLACEHOLDER_1.s_nationkey = FILENAME_PLACEHOLDER_2.n_nationkey AND FILENAME_PLACEHOLDER_2.n_name = CAST(Utf8(\"JAPAN\") AS Utf8)\ + \n Filter: FILENAME_PLACEHOLDER_0.ps_suppkey = FILENAME_PLACEHOLDER_1.s_suppkey AND FILENAME_PLACEHOLDER_1.s_nationkey = FILENAME_PLACEHOLDER_2.N_NATIONKEY AND FILENAME_PLACEHOLDER_2.N_NAME = CAST(Utf8(\"JAPAN\") AS Utf8)\ \n Inner Join: Filter: Boolean(true)\ \n Inner Join: Filter: Boolean(true)\ \n TableScan: FILENAME_PLACEHOLDER_0 projection=[ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment]\ \n TableScan: FILENAME_PLACEHOLDER_1 projection=[s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment]\ - \n TableScan: FILENAME_PLACEHOLDER_2 projection=[n_nationkey, n_name, n_regionkey, n_comment]"); + \n TableScan: FILENAME_PLACEHOLDER_2 projection=[N_NATIONKEY, N_NAME, N_REGIONKEY, N_COMMENT]"); Ok(()) } @@ -498,7 +498,7 @@ mod tests { assert_eq!(plan_str, "Projection: FILENAME_PLACEHOLDER_0.s_name AS S_NAME, FILENAME_PLACEHOLDER_0.s_address AS S_ADDRESS\ \n Sort: FILENAME_PLACEHOLDER_0.s_name ASC NULLS LAST\ \n Projection: FILENAME_PLACEHOLDER_0.s_name, FILENAME_PLACEHOLDER_0.s_address\ - \n Filter: CAST(FILENAME_PLACEHOLDER_0.s_suppkey IN () AS Boolean) AND FILENAME_PLACEHOLDER_0.s_nationkey = FILENAME_PLACEHOLDER_1.n_nationkey AND FILENAME_PLACEHOLDER_1.n_name = CAST(Utf8(\"CANADA\") AS Utf8)\ + \n Filter: CAST(FILENAME_PLACEHOLDER_0.s_suppkey IN () AS Boolean) AND FILENAME_PLACEHOLDER_0.s_nationkey = FILENAME_PLACEHOLDER_1.N_NATIONKEY AND FILENAME_PLACEHOLDER_1.N_NAME = CAST(Utf8(\"CANADA\") AS Utf8)\ \n Subquery:\ \n Projection: FILENAME_PLACEHOLDER_2.ps_suppkey\ \n Filter: CAST(FILENAME_PLACEHOLDER_2.ps_partkey IN () AS Boolean) AND CAST(FILENAME_PLACEHOLDER_2.ps_availqty AS Decimal128(19, 1)) > ()\ @@ -515,7 +515,7 @@ mod tests { \n TableScan: FILENAME_PLACEHOLDER_2 projection=[ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment]\ \n Inner Join: Filter: Boolean(true)\ \n TableScan: FILENAME_PLACEHOLDER_0 projection=[s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment]\ - \n TableScan: FILENAME_PLACEHOLDER_1 projection=[n_nationkey, n_name, n_regionkey, n_comment]"); + \n TableScan: FILENAME_PLACEHOLDER_1 projection=[N_NATIONKEY, N_NAME, N_REGIONKEY, N_COMMENT]"); Ok(()) } @@ -543,7 +543,7 @@ mod tests { \n Sort: count(Int64(1)) DESC NULLS FIRST, FILENAME_PLACEHOLDER_0.s_name ASC NULLS LAST\ \n Aggregate: groupBy=[[FILENAME_PLACEHOLDER_0.s_name]], aggr=[[count(Int64(1))]]\ \n Projection: FILENAME_PLACEHOLDER_0.s_name\ - \n Filter: FILENAME_PLACEHOLDER_0.s_suppkey = FILENAME_PLACEHOLDER_1.l_suppkey AND FILENAME_PLACEHOLDER_2.o_orderkey = FILENAME_PLACEHOLDER_1.l_orderkey AND FILENAME_PLACEHOLDER_2.o_orderstatus = Utf8(\"F\") AND FILENAME_PLACEHOLDER_1.l_receiptdate > FILENAME_PLACEHOLDER_1.l_commitdate AND EXISTS () AND NOT EXISTS () AND FILENAME_PLACEHOLDER_0.s_nationkey = FILENAME_PLACEHOLDER_3.n_nationkey AND FILENAME_PLACEHOLDER_3.n_name = CAST(Utf8(\"SAUDI ARABIA\") AS Utf8)\ + \n Filter: FILENAME_PLACEHOLDER_0.s_suppkey = FILENAME_PLACEHOLDER_1.l_suppkey AND FILENAME_PLACEHOLDER_2.o_orderkey = FILENAME_PLACEHOLDER_1.l_orderkey AND FILENAME_PLACEHOLDER_2.o_orderstatus = Utf8(\"F\") AND FILENAME_PLACEHOLDER_1.l_receiptdate > FILENAME_PLACEHOLDER_1.l_commitdate AND EXISTS () AND NOT EXISTS () AND FILENAME_PLACEHOLDER_0.s_nationkey = FILENAME_PLACEHOLDER_3.N_NATIONKEY AND FILENAME_PLACEHOLDER_3.N_NAME = CAST(Utf8(\"SAUDI ARABIA\") AS Utf8)\ \n Subquery:\ \n Filter: FILENAME_PLACEHOLDER_4.l_orderkey = FILENAME_PLACEHOLDER_4.l_tax AND FILENAME_PLACEHOLDER_4.l_suppkey != FILENAME_PLACEHOLDER_4.l_linestatus\ \n TableScan: FILENAME_PLACEHOLDER_4 projection=[l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment]\ @@ -555,7 +555,7 @@ mod tests { \n Inner Join: Filter: Boolean(true)\ \n TableScan: FILENAME_PLACEHOLDER_0 projection=[s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment]\ \n TableScan: FILENAME_PLACEHOLDER_1 projection=[l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment]\n TableScan: FILENAME_PLACEHOLDER_2 projection=[o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment]\ - \n TableScan: FILENAME_PLACEHOLDER_3 projection=[n_nationkey, n_name, n_regionkey, n_comment]"); + \n TableScan: FILENAME_PLACEHOLDER_3 projection=[N_NATIONKEY, N_NAME, N_REGIONKEY, N_COMMENT]"); Ok(()) } diff --git a/datafusion/substrait/tests/cases/function_test.rs b/datafusion/substrait/tests/cases/function_test.rs index 610caf3a81df..85809da6f3e4 100644 --- a/datafusion/substrait/tests/cases/function_test.rs +++ b/datafusion/substrait/tests/cases/function_test.rs @@ -19,40 +19,26 @@ #[cfg(test)] mod tests { + use crate::utils::test::{add_plan_schemas_to_ctx, read_json}; + use datafusion::common::Result; - use datafusion::prelude::{CsvReadOptions, SessionContext}; + use datafusion::prelude::SessionContext; use datafusion_substrait::logical_plan::consumer::from_substrait_plan; - use std::fs::File; - use std::io::BufReader; - use substrait::proto::Plan; #[tokio::test] async fn contains_function_test() -> Result<()> { - let ctx = create_context().await?; - - let path = "tests/testdata/contains_plan.substrait.json"; - let proto = serde_json::from_reader::<_, Plan>(BufReader::new( - File::open(path).expect("file not found"), - )) - .expect("failed to parse json"); - - let plan = from_substrait_plan(&ctx, &proto).await?; + let proto_plan = read_json("tests/testdata/contains_plan.substrait.json"); + let ctx = add_plan_schemas_to_ctx(SessionContext::new(), &proto_plan); + let plan = from_substrait_plan(&ctx, &proto_plan).await?; let plan_str = format!("{}", plan); assert_eq!( plan_str, - "Projection: nation.b AS n_name\ - \n Filter: contains(nation.b, Utf8(\"IA\"))\ - \n TableScan: nation projection=[a, b, c, d, e, f]" + "Projection: nation.n_name\ + \n Filter: contains(nation.n_name, Utf8(\"IA\"))\ + \n TableScan: nation projection=[n_nationkey, n_name, n_regionkey, n_comment]" ); Ok(()) } - - async fn create_context() -> datafusion::common::Result { - let ctx = SessionContext::new(); - ctx.register_csv("nation", "tests/testdata/data.csv", CsvReadOptions::new()) - .await?; - Ok(ctx) - } } diff --git a/datafusion/substrait/tests/cases/logical_plans.rs b/datafusion/substrait/tests/cases/logical_plans.rs index f6a2b5036c80..8db2aa283d3c 100644 --- a/datafusion/substrait/tests/cases/logical_plans.rs +++ b/datafusion/substrait/tests/cases/logical_plans.rs @@ -19,13 +19,11 @@ #[cfg(test)] mod tests { + use crate::utils::test::{add_plan_schemas_to_ctx, read_json}; use datafusion::common::Result; use datafusion::dataframe::DataFrame; - use datafusion::prelude::{CsvReadOptions, SessionContext}; + use datafusion::prelude::SessionContext; use datafusion_substrait::logical_plan::consumer::from_substrait_plan; - use std::fs::File; - use std::io::BufReader; - use substrait::proto::Plan; #[tokio::test] async fn scalar_function_compound_signature() -> Result<()> { @@ -35,18 +33,17 @@ mod tests { // we don't yet produce such plans. // Once we start producing plans with compound signatures, this test can be replaced by the roundtrip tests. - let ctx = create_context().await?; - // File generated with substrait-java's Isthmus: - // ./isthmus-cli/build/graal/isthmus "select not d from data" -c "create table data (d boolean)" - let proto = read_json("tests/testdata/test_plans/select_not_bool.substrait.json"); - - let plan = from_substrait_plan(&ctx, &proto).await?; + // ./isthmus-cli/build/graal/isthmus --create "create table data (d boolean)" "select not d from data" + let proto_plan = + read_json("tests/testdata/test_plans/select_not_bool.substrait.json"); + let ctx = add_plan_schemas_to_ctx(SessionContext::new(), &proto_plan); + let plan = from_substrait_plan(&ctx, &proto_plan).await?; assert_eq!( format!("{}", plan), - "Projection: NOT DATA.a AS EXPR$0\ - \n TableScan: DATA projection=[a, b, c, d, e, f]" + "Projection: NOT DATA.D AS EXPR$0\ + \n TableScan: DATA projection=[D]" ); Ok(()) } @@ -61,19 +58,18 @@ mod tests { // we don't yet produce such plans. // Once we start producing plans with compound signatures, this test can be replaced by the roundtrip tests. - let ctx = create_context().await?; - // File generated with substrait-java's Isthmus: - // ./isthmus-cli/build/graal/isthmus "select sum(d) OVER (PARTITION BY part ORDER BY ord ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING) AS lead_expr from data" -c "create table data (d int, part int, ord int)" - let proto = read_json("tests/testdata/test_plans/select_window.substrait.json"); - - let plan = from_substrait_plan(&ctx, &proto).await?; + // ./isthmus-cli/build/graal/isthmus --create "create table data (d int, part int, ord int)" "select sum(d) OVER (PARTITION BY part ORDER BY ord ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING) AS lead_expr from data" + let proto_plan = + read_json("tests/testdata/test_plans/select_window.substrait.json"); + let ctx = add_plan_schemas_to_ctx(SessionContext::new(), &proto_plan); + let plan = from_substrait_plan(&ctx, &proto_plan).await?; assert_eq!( format!("{}", plan), - "Projection: sum(DATA.a) PARTITION BY [DATA.b] ORDER BY [DATA.c ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING AS LEAD_EXPR\ - \n WindowAggr: windowExpr=[[sum(DATA.a) PARTITION BY [DATA.b] ORDER BY [DATA.c ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING]]\ - \n TableScan: DATA projection=[a, b, c, d, e, f]" + "Projection: sum(DATA.D) PARTITION BY [DATA.PART] ORDER BY [DATA.ORD ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING AS LEAD_EXPR\ + \n WindowAggr: windowExpr=[[sum(DATA.D) PARTITION BY [DATA.PART] ORDER BY [DATA.ORD ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING]]\ + \n TableScan: DATA projection=[D, PART, ORD]" ); Ok(()) } @@ -83,11 +79,10 @@ mod tests { // DataFusion's Substrait consumer treats all lists as nullable, even if the Substrait plan specifies them as non-nullable. // That's because implementing the non-nullability consistently is non-trivial. // This test confirms that reading a plan with non-nullable lists works as expected. - let ctx = create_context().await?; - let proto = + let proto_plan = read_json("tests/testdata/test_plans/non_nullable_lists.substrait.json"); - - let plan = from_substrait_plan(&ctx, &proto).await?; + let ctx = add_plan_schemas_to_ctx(SessionContext::new(), &proto_plan); + let plan = from_substrait_plan(&ctx, &proto_plan).await?; assert_eq!(format!("{}", &plan), "Values: (List([1, 2]))"); @@ -96,18 +91,4 @@ mod tests { Ok(()) } - - fn read_json(path: &str) -> Plan { - serde_json::from_reader::<_, Plan>(BufReader::new( - File::open(path).expect("file not found"), - )) - .expect("failed to parse json") - } - - async fn create_context() -> datafusion::common::Result { - let ctx = SessionContext::new(); - ctx.register_csv("DATA", "tests/testdata/data.csv", CsvReadOptions::new()) - .await?; - Ok(ctx) - } } diff --git a/datafusion/substrait/tests/cases/mod.rs b/datafusion/substrait/tests/cases/mod.rs index d3ea7695e4b9..42aa23626106 100644 --- a/datafusion/substrait/tests/cases/mod.rs +++ b/datafusion/substrait/tests/cases/mod.rs @@ -21,3 +21,4 @@ mod logical_plans; mod roundtrip_logical_plan; mod roundtrip_physical_plan; mod serialize; +mod substrait_validations; diff --git a/datafusion/substrait/tests/cases/substrait_validations.rs b/datafusion/substrait/tests/cases/substrait_validations.rs new file mode 100644 index 000000000000..cb1fb67fc044 --- /dev/null +++ b/datafusion/substrait/tests/cases/substrait_validations.rs @@ -0,0 +1,151 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#[cfg(test)] +mod tests { + + // verify the schema compatability validations + mod schema_compatability { + use crate::utils::test::read_json; + use datafusion::arrow::datatypes::{DataType, Field}; + use datafusion::catalog_common::TableReference; + use datafusion::common::{DFSchema, Result}; + use datafusion::datasource::empty::EmptyTable; + use datafusion::prelude::SessionContext; + use datafusion_substrait::logical_plan::consumer::from_substrait_plan; + use std::collections::HashMap; + use std::sync::Arc; + + fn generate_context_with_table( + table_name: &str, + fields: Vec<(&str, DataType, bool)>, + ) -> Result { + let table_ref = TableReference::bare(table_name); + let fields: Vec<(Option, Arc)> = fields + .into_iter() + .map(|pair| { + let (field_name, data_type, nullable) = pair; + ( + Some(table_ref.clone()), + Arc::new(Field::new(field_name, data_type, nullable)), + ) + }) + .collect(); + + let df_schema = DFSchema::new_with_metadata(fields, HashMap::default())?; + + let ctx = SessionContext::new(); + ctx.register_table( + table_ref, + Arc::new(EmptyTable::new(df_schema.inner().clone())), + )?; + Ok(ctx) + } + + #[tokio::test] + async fn ensure_schema_match_exact() -> Result<()> { + let proto_plan = + read_json("tests/testdata/test_plans/simple_select.substrait.json"); + // this is the exact schema of the Substrait plan + let df_schema = + vec![("a", DataType::Int32, false), ("b", DataType::Int32, true)]; + + let ctx = generate_context_with_table("DATA", df_schema)?; + let plan = from_substrait_plan(&ctx, &proto_plan).await?; + + assert_eq!( + format!("{}", plan), + "Projection: DATA.a, DATA.b\ + \n TableScan: DATA projection=[a, b]" + ); + Ok(()) + } + + #[tokio::test] + async fn ensure_schema_match_subset() -> Result<()> { + let proto_plan = + read_json("tests/testdata/test_plans/simple_select.substrait.json"); + // the DataFusion schema { b, a, c } contains the Substrait schema { a, b } + let df_schema = vec![ + ("b", DataType::Int32, true), + ("a", DataType::Int32, false), + ("c", DataType::Int32, false), + ]; + let ctx = generate_context_with_table("DATA", df_schema)?; + let plan = from_substrait_plan(&ctx, &proto_plan).await?; + + assert_eq!( + format!("{}", plan), + "Projection: DATA.a, DATA.b\ + \n Projection: DATA.a, DATA.b\ + \n TableScan: DATA projection=[b, a]" + ); + Ok(()) + } + + #[tokio::test] + async fn ensure_schema_match_subset_with_mask() -> Result<()> { + let proto_plan = read_json( + "tests/testdata/test_plans/simple_select_with_mask.substrait.json", + ); + // the DataFusion schema { b, a, c, d } contains the Substrait schema { a, b, c } + let df_schema = vec![ + ("b", DataType::Int32, true), + ("a", DataType::Int32, false), + ("c", DataType::Int32, false), + ("d", DataType::Int32, false), + ]; + let ctx = generate_context_with_table("DATA", df_schema)?; + let plan = from_substrait_plan(&ctx, &proto_plan).await?; + + assert_eq!( + format!("{}", plan), + "Projection: DATA.a, DATA.b\ + \n Projection: DATA.a, DATA.b\ + \n Projection: DATA.a, DATA.b, DATA.c\ + \n TableScan: DATA projection=[b, a, c]" + ); + Ok(()) + } + + #[tokio::test] + async fn ensure_schema_match_not_subset() -> Result<()> { + let proto_plan = + read_json("tests/testdata/test_plans/simple_select.substrait.json"); + // the substrait plans contains a field b which is not in the schema + let df_schema = + vec![("a", DataType::Int32, false), ("c", DataType::Int32, true)]; + + let ctx = generate_context_with_table("DATA", df_schema)?; + let res = from_substrait_plan(&ctx, &proto_plan).await; + assert!(res.is_err()); + Ok(()) + } + + #[tokio::test] + async fn reject_plans_with_incompatible_field_types() -> Result<()> { + let proto_plan = + read_json("tests/testdata/test_plans/simple_select.substrait.json"); + + let ctx = + generate_context_with_table("DATA", vec![("a", DataType::Date32, true)])?; + let res = from_substrait_plan(&ctx, &proto_plan).await; + assert!(res.is_err()); + Ok(()) + } + } +} diff --git a/datafusion/substrait/tests/substrait_integration.rs b/datafusion/substrait/tests/substrait_integration.rs index 6ce41c9de71a..eedd4da373e0 100644 --- a/datafusion/substrait/tests/substrait_integration.rs +++ b/datafusion/substrait/tests/substrait_integration.rs @@ -17,3 +17,4 @@ /// Run all tests that are found in the `cases` directory mod cases; +mod utils; diff --git a/datafusion/substrait/tests/testdata/test_plans/simple_select.substrait.json b/datafusion/substrait/tests/testdata/test_plans/simple_select.substrait.json new file mode 100644 index 000000000000..aee27ef3b417 --- /dev/null +++ b/datafusion/substrait/tests/testdata/test_plans/simple_select.substrait.json @@ -0,0 +1,69 @@ +{ + "extensionUris": [], + "extensions": [], + "relations": [{ + "root": { + "input": { + "project": { + "common": { + "emit": { + "outputMapping": [2, 3] + } + }, + "input": { + "read": { + "common": { + "direct": { + } + }, + "baseSchema": { + "names": ["a", "b"], + "struct": { + "types": [{ + "i32": { + "typeVariationReference": 0, + "nullability": "NULLABILITY_NULLABLE" + } + }, { + "i32": { + "typeVariationReference": 0, + "nullability": "NULLABILITY_REQUIRED" + } + }], + "typeVariationReference": 0, + "nullability": "NULLABILITY_REQUIRED" + } + }, + "namedTable": { + "names": ["DATA"] + } + } + }, + "expressions": [{ + "selection": { + "directReference": { + "structField": { + "field": 0 + } + }, + "rootReference": { + } + } + }, { + "selection": { + "directReference": { + "structField": { + "field": 1 + } + }, + "rootReference": { + } + } + }] + } + }, + "names": ["a", "b"] + } + }], + "expectedTypeUrls": [] +} \ No newline at end of file diff --git a/datafusion/substrait/tests/testdata/test_plans/simple_select_with_mask.substrait.json b/datafusion/substrait/tests/testdata/test_plans/simple_select_with_mask.substrait.json new file mode 100644 index 000000000000..774126ca3836 --- /dev/null +++ b/datafusion/substrait/tests/testdata/test_plans/simple_select_with_mask.substrait.json @@ -0,0 +1,104 @@ +{ + "extensionUris": [], + "extensions": [], + "relations": [ + { + "root": { + "input": { + "project": { + "common": { + "emit": { + "outputMapping": [ + 2, + 3 + ] + } + }, + "input": { + "read": { + "common": { + "direct": {} + }, + "baseSchema": { + "names": [ + "a", + "b", + "c" + ], + "struct": { + "types": [ + { + "i32": { + "typeVariationReference": 0, + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "i32": { + "typeVariationReference": 0, + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "i32": { + "typeVariationReference": 0, + "nullability": "NULLABILITY_REQUIRED" + } + } + ], + "typeVariationReference": 0, + "nullability": "NULLABILITY_REQUIRED" + } + }, + "namedTable": { + "names": [ + "DATA" + ] + }, + "projection": { + "select": { + "struct_items": [ + { + "field": 0 + }, + { + "field": 1 + } + ] + } + } + } + }, + "expressions": [ + { + "selection": { + "directReference": { + "structField": { + "field": 0 + } + }, + "rootReference": {} + } + }, + { + "selection": { + "directReference": { + "structField": { + "field": 1 + } + }, + "rootReference": {} + } + } + ] + } + }, + "names": [ + "a", + "b" + ] + } + } + ], + "expectedTypeUrls": [] +} diff --git a/datafusion/substrait/tests/testdata/tpch/nation.csv b/datafusion/substrait/tests/testdata/tpch/nation.csv index fdf7421467d3..a88d1c0d31e7 100644 --- a/datafusion/substrait/tests/testdata/tpch/nation.csv +++ b/datafusion/substrait/tests/testdata/tpch/nation.csv @@ -1,2 +1,2 @@ -n_nationkey,n_name,n_regionkey,n_comment +N_NATIONKEY,N_NAME,N_REGIONKEY,N_COMMENT 0,ALGERIA,0, haggle. carefully final deposits detect slyly agai \ No newline at end of file diff --git a/datafusion/substrait/tests/testdata/tpch/region.csv b/datafusion/substrait/tests/testdata/tpch/region.csv index 6c3fb4524355..d29c39ab8543 100644 --- a/datafusion/substrait/tests/testdata/tpch/region.csv +++ b/datafusion/substrait/tests/testdata/tpch/region.csv @@ -1,2 +1,2 @@ -r_regionkey,r_name,r_comment +R_REGIONKEY,R_NAME,R_COMMENT 0,AFRICA,lar deposits. blithely final packages cajole. regular waters are final requests. regular accounts are according to \ No newline at end of file diff --git a/datafusion/substrait/tests/utils.rs b/datafusion/substrait/tests/utils.rs new file mode 100644 index 000000000000..685e3deec581 --- /dev/null +++ b/datafusion/substrait/tests/utils.rs @@ -0,0 +1,186 @@ +// 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)] +pub mod test { + use datafusion::catalog_common::TableReference; + use datafusion::datasource::empty::EmptyTable; + use datafusion::datasource::TableProvider; + use datafusion::prelude::SessionContext; + use datafusion_substrait::extensions::Extensions; + use datafusion_substrait::logical_plan::consumer::from_substrait_named_struct; + use std::fs::File; + use std::io::BufReader; + use std::sync::Arc; + use substrait::proto::read_rel::{NamedTable, ReadType}; + use substrait::proto::rel::RelType; + use substrait::proto::{Plan, ReadRel, Rel}; + + pub fn read_json(path: &str) -> Plan { + serde_json::from_reader::<_, Plan>(BufReader::new( + File::open(path).expect("file not found"), + )) + .expect("failed to parse json") + } + + pub fn add_plan_schemas_to_ctx(ctx: SessionContext, plan: &Plan) -> SessionContext { + let schemas = TestSchemaCollector::collect_schemas(plan); + for (table_reference, table) in schemas { + ctx.register_table(table_reference, table) + .expect("Failed to register table"); + } + ctx + } + + pub struct TestSchemaCollector { + schemas: Vec<(TableReference, Arc)>, + } + + impl TestSchemaCollector { + fn new() -> Self { + TestSchemaCollector { + schemas: Vec::new(), + } + } + + fn collect_schemas(plan: &Plan) -> Vec<(TableReference, Arc)> { + let mut schema_collector = Self::new(); + + for plan_rel in plan.relations.iter() { + match plan_rel + .rel_type + .as_ref() + .expect("PlanRel must set rel_type") + { + substrait::proto::plan_rel::RelType::Rel(r) => { + schema_collector.collect_schemas_from_rel(r) + } + substrait::proto::plan_rel::RelType::Root(r) => schema_collector + .collect_schemas_from_rel( + r.input.as_ref().expect("RelRoot must set input"), + ), + } + } + schema_collector.schemas + } + + fn collect_named_table(&mut self, read: &ReadRel, nt: &NamedTable) { + let table_reference = match nt.names.len() { + 0 => { + panic!("No table name found in NamedTable"); + } + 1 => TableReference::Bare { + table: nt.names[0].clone().into(), + }, + 2 => TableReference::Partial { + schema: nt.names[0].clone().into(), + table: nt.names[1].clone().into(), + }, + _ => TableReference::Full { + catalog: nt.names[0].clone().into(), + schema: nt.names[1].clone().into(), + table: nt.names[2].clone().into(), + }, + }; + + let substrait_schema = read + .base_schema + .as_ref() + .expect("No base schema found for NamedTable"); + let empty_extensions = Extensions { + functions: Default::default(), + types: Default::default(), + type_variations: Default::default(), + }; + + let df_schema = + from_substrait_named_struct(substrait_schema, &empty_extensions) + .expect( + "Unable to generate DataFusion schema from Substrait NamedStruct", + ) + .replace_qualifier(table_reference.clone()); + + let table = EmptyTable::new(df_schema.inner().clone()); + self.schemas.push((table_reference, Arc::new(table))); + } + + fn collect_schemas_from_rel(&mut self, rel: &Rel) { + match rel.rel_type.as_ref().unwrap() { + RelType::Read(r) => match r.read_type.as_ref().unwrap() { + // Virtual Tables do not contribute to the schema + ReadType::VirtualTable(_) => (), + ReadType::LocalFiles(_) => todo!(), + ReadType::NamedTable(nt) => self.collect_named_table(r, nt), + ReadType::ExtensionTable(_) => todo!(), + }, + RelType::Filter(f) => self.apply(f.input.as_ref().map(|b| b.as_ref())), + RelType::Fetch(f) => self.apply(f.input.as_ref().map(|b| b.as_ref())), + RelType::Aggregate(a) => self.apply(a.input.as_ref().map(|b| b.as_ref())), + RelType::Sort(s) => self.apply(s.input.as_ref().map(|b| b.as_ref())), + RelType::Join(j) => { + self.apply(j.left.as_ref().map(|b| b.as_ref())); + self.apply(j.right.as_ref().map(|b| b.as_ref())); + } + RelType::Project(p) => self.apply(p.input.as_ref().map(|b| b.as_ref())), + RelType::Set(s) => { + for input in s.inputs.iter() { + self.collect_schemas_from_rel(input); + } + } + RelType::ExtensionSingle(s) => { + self.apply(s.input.as_ref().map(|b| b.as_ref())) + } + RelType::ExtensionMulti(m) => { + for input in m.inputs.iter() { + self.collect_schemas_from_rel(input) + } + } + RelType::ExtensionLeaf(_) => {} + RelType::Cross(c) => { + self.apply(c.left.as_ref().map(|b| b.as_ref())); + self.apply(c.right.as_ref().map(|b| b.as_ref())); + } + // RelType::Reference(_) => {} + // RelType::Write(_) => {} + // RelType::Ddl(_) => {} + RelType::HashJoin(j) => { + self.apply(j.left.as_ref().map(|b| b.as_ref())); + self.apply(j.right.as_ref().map(|b| b.as_ref())); + } + RelType::MergeJoin(j) => { + self.apply(j.left.as_ref().map(|b| b.as_ref())); + self.apply(j.right.as_ref().map(|b| b.as_ref())); + } + RelType::NestedLoopJoin(j) => { + self.apply(j.left.as_ref().map(|b| b.as_ref())); + self.apply(j.right.as_ref().map(|b| b.as_ref())); + } + RelType::Window(w) => self.apply(w.input.as_ref().map(|b| b.as_ref())), + RelType::Exchange(e) => self.apply(e.input.as_ref().map(|b| b.as_ref())), + RelType::Expand(e) => self.apply(e.input.as_ref().map(|b| b.as_ref())), + _ => todo!(), + } + } + + fn apply(&mut self, input: Option<&Rel>) { + match input { + None => {} + Some(rel) => self.collect_schemas_from_rel(rel), + } + } + } +} From c3f95342adc1c33364db1075a8c532413a01bd0f Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 10 Sep 2024 06:58:35 -0400 Subject: [PATCH 138/154] Bump rstest from 0.17.0 to 0.22.0 in /datafusion-cli (#12413) Bumps [rstest](https://github.com/la10736/rstest) from 0.17.0 to 0.22.0. - [Release notes](https://github.com/la10736/rstest/releases) - [Changelog](https://github.com/la10736/rstest/blob/master/CHANGELOG.md) - [Commits](https://github.com/la10736/rstest/compare/0.17.0...v0.22.0) --- updated-dependencies: - dependency-name: rstest dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- datafusion-cli/Cargo.lock | 100 +++++++++++++++++++++++++------------- datafusion-cli/Cargo.toml | 2 +- 2 files changed, 68 insertions(+), 34 deletions(-) diff --git a/datafusion-cli/Cargo.lock b/datafusion-cli/Cargo.lock index d3af3d5596dc..c65bb2fd0880 100644 --- a/datafusion-cli/Cargo.lock +++ b/datafusion-cli/Cargo.lock @@ -436,7 +436,7 @@ checksum = "a27b8a3a6e1a44fa4c8baf1f653e4172e81486d4941f2237e20dc2d0cf4ddff1" dependencies = [ "proc-macro2", "quote", - "syn 2.0.77", + "syn", ] [[package]] @@ -1015,7 +1015,7 @@ dependencies = [ "heck 0.5.0", "proc-macro2", "quote", - "syn 2.0.77", + "syn", ] [[package]] @@ -1171,7 +1171,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "edb49164822f3ee45b17acd4a208cfc1251410cf0cad9a833234c9890774dd9f" dependencies = [ "quote", - "syn 2.0.77", + "syn", ] [[package]] @@ -1827,7 +1827,7 @@ checksum = "87750cf4b7a4c0625b1529e4c543c2182106e4dedc60a2a6455e00d212c489ac" dependencies = [ "proc-macro2", "quote", - "syn 2.0.77", + "syn", ] [[package]] @@ -2835,7 +2835,7 @@ checksum = "2f38a4412a78282e09a2cf38d195ea5420d15ba0602cb375210efbc877243965" dependencies = [ "proc-macro2", "quote", - "syn 2.0.77", + "syn", ] [[package]] @@ -2901,6 +2901,15 @@ dependencies = [ "termtree", ] +[[package]] +name = "proc-macro-crate" +version = "3.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8ecf48c7ca261d60b74ab1a7b20da18bede46776b2e55535cb958eb595c5fa7b" +dependencies = [ + "toml_edit", +] + [[package]] name = "proc-macro2" version = "1.0.86" @@ -3078,6 +3087,12 @@ version = "0.8.4" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "7a66a03ae7c801facd77a29370b4faec201768915ac14a721ba36f20bc9c209b" +[[package]] +name = "relative-path" +version = "1.9.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ba39f3699c378cd8970968dcbff9c43159ea4cfbd88d43c00b22f2ef10a435d2" + [[package]] name = "reqwest" version = "0.12.7" @@ -3146,9 +3161,9 @@ checksum = "3582f63211428f83597b51b2ddb88e2a91a9d52d12831f9d08f5e624e8977422" [[package]] name = "rstest" -version = "0.17.0" +version = "0.22.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "de1bb486a691878cd320c2f0d319ba91eeaa2e894066d8b5f8f117c000e9d962" +checksum = "7b423f0e62bdd61734b67cd21ff50871dfaeb9cc74f869dcd6af974fbcb19936" dependencies = [ "futures", "futures-timer", @@ -3158,15 +3173,19 @@ dependencies = [ [[package]] name = "rstest_macros" -version = "0.17.0" +version = "0.22.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "290ca1a1c8ca7edb7c3283bd44dc35dd54fdec6253a3912e201ba1072018fca8" +checksum = "c5e1711e7d14f74b12a58411c542185ef7fb7f2e7f8ee6e2940a883628522b42" dependencies = [ "cfg-if", + "glob", + "proc-macro-crate", "proc-macro2", "quote", + "regex", + "relative-path", "rustc_version", - "syn 1.0.109", + "syn", "unicode-ident", ] @@ -3435,7 +3454,7 @@ checksum = "a5831b979fd7b5439637af1752d535ff49f4860c0f341d1baeb6faf0f4242170" dependencies = [ "proc-macro2", "quote", - "syn 2.0.77", + "syn", ] [[package]] @@ -3527,7 +3546,7 @@ dependencies = [ "heck 0.5.0", "proc-macro2", "quote", - "syn 2.0.77", + "syn", ] [[package]] @@ -3570,7 +3589,7 @@ checksum = "01b2e185515564f15375f593fb966b5718bc624ba77fe49fa4616ad619690554" dependencies = [ "proc-macro2", "quote", - "syn 2.0.77", + "syn", ] [[package]] @@ -3616,7 +3635,7 @@ dependencies = [ "proc-macro2", "quote", "rustversion", - "syn 2.0.77", + "syn", ] [[package]] @@ -3629,7 +3648,7 @@ dependencies = [ "proc-macro2", "quote", "rustversion", - "syn 2.0.77", + "syn", ] [[package]] @@ -3638,17 +3657,6 @@ version = "2.6.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "13c2bddecc57b384dee18652358fb23172facb8a2c51ccc10d74c157bdea3292" -[[package]] -name = "syn" -version = "1.0.109" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "72b64191b275b66ffe2469e8af2c1cfe3bafa67b529ead792a6d0160888b4237" -dependencies = [ - "proc-macro2", - "quote", - "unicode-ident", -] - [[package]] name = "syn" version = "2.0.77" @@ -3714,7 +3722,7 @@ checksum = "a4558b58466b9ad7ca0f102865eccc95938dca1a74a856f2b57b6629050da261" dependencies = [ "proc-macro2", "quote", - "syn 2.0.77", + "syn", ] [[package]] @@ -3808,7 +3816,7 @@ checksum = "693d596312e88961bc67d7f1f97af8a70227d9f90c31bba5806eec004978d752" dependencies = [ "proc-macro2", "quote", - "syn 2.0.77", + "syn", ] [[package]] @@ -3845,6 +3853,23 @@ dependencies = [ "tokio", ] +[[package]] +name = "toml_datetime" +version = "0.6.8" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0dd7358ecb8fc2f8d014bf86f6f638ce72ba252a2c3a2572f2a795f1d23efb41" + +[[package]] +name = "toml_edit" +version = "0.22.20" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "583c44c02ad26b0c3f3066fe629275e50627026c51ac2e595cca4c230ce1ce1d" +dependencies = [ + "indexmap", + "toml_datetime", + "winnow", +] + [[package]] name = "tower" version = "0.4.13" @@ -3891,7 +3916,7 @@ checksum = "34704c8d6ebcbc939824180af020566b01a7c01f80641264eba0999f6c2b6be7" dependencies = [ "proc-macro2", "quote", - "syn 2.0.77", + "syn", ] [[package]] @@ -3936,7 +3961,7 @@ checksum = "f03ca4cb38206e2bef0700092660bb74d696f808514dae47fa1467cbfe26e96e" dependencies = [ "proc-macro2", "quote", - "syn 2.0.77", + "syn", ] [[package]] @@ -4085,7 +4110,7 @@ dependencies = [ "once_cell", "proc-macro2", "quote", - "syn 2.0.77", + "syn", "wasm-bindgen-shared", ] @@ -4119,7 +4144,7 @@ checksum = "afc340c74d9005395cf9dd098506f7f44e38f2b4a21c6aaacf9a105ea5e1e836" dependencies = [ "proc-macro2", "quote", - "syn 2.0.77", + "syn", "wasm-bindgen-backend", "wasm-bindgen-shared", ] @@ -4371,6 +4396,15 @@ version = "0.52.6" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "589f6da84c646204747d1270a2a5661ea66ed1cced2631d546fdfb155959f9ec" +[[package]] +name = "winnow" +version = "0.6.18" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "68a9bda4691f099d435ad181000724da8e5899daa10713c2d432552b9ccd3a6f" +dependencies = [ + "memchr", +] + [[package]] name = "xmlparser" version = "0.13.6" @@ -4404,7 +4438,7 @@ checksum = "fa4f8080344d4671fb4e831a13ad1e68092748387dfc4f55e356242fae12ce3e" dependencies = [ "proc-macro2", "quote", - "syn 2.0.77", + "syn", ] [[package]] diff --git a/datafusion-cli/Cargo.toml b/datafusion-cli/Cargo.toml index f2f52846ab54..ba0178924e41 100644 --- a/datafusion-cli/Cargo.toml +++ b/datafusion-cli/Cargo.toml @@ -68,4 +68,4 @@ url = "2.2" assert_cmd = "2.0" ctor = "0.2.0" predicates = "3.0" -rstest = "0.17" +rstest = "0.22" From 29babce95e69cab9ff2ce3beb4805196cae33316 Mon Sep 17 00:00:00 2001 From: Xiangpeng Hao Date: Tue, 10 Sep 2024 05:59:25 -0500 Subject: [PATCH 139/154] fix tpc-h parquet setting to respect global options (#12405) --- benchmarks/src/tpch/run.rs | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/benchmarks/src/tpch/run.rs b/benchmarks/src/tpch/run.rs index ebc5ac0dbd5a..d79eb65649bd 100644 --- a/benchmarks/src/tpch/run.rs +++ b/benchmarks/src/tpch/run.rs @@ -268,7 +268,8 @@ impl RunOpt { } "parquet" => { let path = format!("{path}/{table}"); - let format = ParquetFormat::default().with_enable_pruning(true); + let format = ParquetFormat::default() + .with_options(ctx.state().table_options().parquet.clone()); (Arc::new(format), path, DEFAULT_PARQUET_EXTENSION) } From c575bbf52ec8ce26ce9ac08058aeb7eca5581aaa Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 10 Sep 2024 07:00:33 -0400 Subject: [PATCH 140/154] Bump dirs from 4.0.0 to 5.0.1 in /datafusion-cli (#12411) Bumps [dirs](https://github.com/soc/dirs-rs) from 4.0.0 to 5.0.1. - [Commits](https://github.com/soc/dirs-rs/commits) --- updated-dependencies: - dependency-name: dirs dependency-type: direct:production update-type: version-update:semver-major ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- datafusion-cli/Cargo.lock | 17 ++++++++++++----- datafusion-cli/Cargo.toml | 2 +- 2 files changed, 13 insertions(+), 6 deletions(-) diff --git a/datafusion-cli/Cargo.lock b/datafusion-cli/Cargo.lock index c65bb2fd0880..5417f328d732 100644 --- a/datafusion-cli/Cargo.lock +++ b/datafusion-cli/Cargo.lock @@ -1608,9 +1608,9 @@ dependencies = [ [[package]] name = "dirs" -version = "4.0.0" +version = "5.0.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ca3aa72a6f96ea37bbc5aa912f6788242832f75369bdfdadcb0e38423f100059" +checksum = "44c45a9d03d6676652bcb5e724c7e988de1acad23a711b5217ab9cbecbec2225" dependencies = [ "dirs-sys", ] @@ -1627,13 +1627,14 @@ dependencies = [ [[package]] name = "dirs-sys" -version = "0.3.7" +version = "0.4.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1b1d1d91c932ef41c0f2663aa8b0ca0342d444d842c06914aa0a7e352d0bada6" +checksum = "520f05a5cbd335fae5a99ff7a6ab8627577660ee5cfd6a94a6a929b52ff0321c" dependencies = [ "libc", + "option-ext", "redox_users", - "winapi", + "windows-sys 0.48.0", ] [[package]] @@ -2675,6 +2676,12 @@ version = "0.1.5" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "ff011a302c396a5197692431fc1948019154afc178baf7d8e37367442a4601cf" +[[package]] +name = "option-ext" +version = "0.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "04744f49eae99ab78e0d5c0b603ab218f515ea8cfe5a456d7629ad883a3b6e7d" + [[package]] name = "ordered-float" version = "2.10.1" diff --git a/datafusion-cli/Cargo.toml b/datafusion-cli/Cargo.toml index ba0178924e41..dfd498c39805 100644 --- a/datafusion-cli/Cargo.toml +++ b/datafusion-cli/Cargo.toml @@ -52,7 +52,7 @@ datafusion = { path = "../datafusion/core", version = "41.0.0", features = [ "unicode_expressions", "compression", ] } -dirs = "4.0.0" +dirs = "5.0.1" env_logger = "0.9" futures = "0.3" mimalloc = { version = "0.1", default-features = false } From 8d2b240b7c21ac20475b42b37fbe3f7f5b8b7956 Mon Sep 17 00:00:00 2001 From: Adrian Garcia Badaracco <1755071+adriangb@users.noreply.github.com> Date: Tue, 10 Sep 2024 12:01:09 +0100 Subject: [PATCH 141/154] Allow using dictionary arrays as filters (#12382) * Allow using dictionaries as filters * revert, nested * fmt --- datafusion/core/tests/dataframe/mod.rs | 107 ++++++++++++++++++++++- datafusion/expr/src/logical_plan/plan.rs | 14 ++- 2 files changed, 118 insertions(+), 3 deletions(-) diff --git a/datafusion/core/tests/dataframe/mod.rs b/datafusion/core/tests/dataframe/mod.rs index 19ce9294cfad..171ef9561e55 100644 --- a/datafusion/core/tests/dataframe/mod.rs +++ b/datafusion/core/tests/dataframe/mod.rs @@ -29,7 +29,10 @@ use arrow::{ }, record_batch::RecordBatch, }; -use arrow_array::{Array, Float32Array, Float64Array, UnionArray}; +use arrow_array::{ + Array, BooleanArray, DictionaryArray, Float32Array, Float64Array, Int8Array, + UnionArray, +}; use arrow_buffer::ScalarBuffer; use arrow_schema::{ArrowError, UnionFields, UnionMode}; use datafusion_functions_aggregate::count::count_udaf; @@ -2363,3 +2366,105 @@ async fn dense_union_is_null() { ]; assert_batches_sorted_eq!(expected, &result_df.collect().await.unwrap()); } + +#[tokio::test] +async fn boolean_dictionary_as_filter() { + let values = vec![Some(true), Some(false), None, Some(true)]; + let keys = vec![0, 0, 1, 2, 1, 3, 1]; + let values_array = BooleanArray::from(values); + let keys_array = Int8Array::from(keys); + let array = + DictionaryArray::new(keys_array, Arc::new(values_array) as Arc); + let array = Arc::new(array); + + let field = Field::new( + "my_dict", + DataType::Dictionary(Box::new(DataType::Int8), Box::new(DataType::Boolean)), + true, + ); + let schema = Arc::new(Schema::new(vec![field])); + + let batch = RecordBatch::try_new(schema, vec![array.clone()]).unwrap(); + + let ctx = SessionContext::new(); + + ctx.register_batch("dict_batch", batch).unwrap(); + + let df = ctx.table("dict_batch").await.unwrap(); + + // view_all + let expected = [ + "+---------+", + "| my_dict |", + "+---------+", + "| true |", + "| true |", + "| false |", + "| |", + "| false |", + "| true |", + "| false |", + "+---------+", + ]; + assert_batches_eq!(expected, &df.clone().collect().await.unwrap()); + + let result_df = df.clone().filter(col("my_dict")).unwrap(); + let expected = [ + "+---------+", + "| my_dict |", + "+---------+", + "| true |", + "| true |", + "| true |", + "+---------+", + ]; + assert_batches_eq!(expected, &result_df.collect().await.unwrap()); + + // test nested dictionary + let keys = vec![0, 2]; // 0 -> true, 2 -> false + let keys_array = Int8Array::from(keys); + let nested_array = DictionaryArray::new(keys_array, array); + + let field = Field::new( + "my_nested_dict", + DataType::Dictionary( + Box::new(DataType::Int8), + Box::new(DataType::Dictionary( + Box::new(DataType::Int8), + Box::new(DataType::Boolean), + )), + ), + true, + ); + + let schema = Arc::new(Schema::new(vec![field])); + + let batch = RecordBatch::try_new(schema, vec![Arc::new(nested_array)]).unwrap(); + + ctx.register_batch("nested_dict_batch", batch).unwrap(); + + let df = ctx.table("nested_dict_batch").await.unwrap(); + + // view_all + let expected = [ + "+----------------+", + "| my_nested_dict |", + "+----------------+", + "| true |", + "| false |", + "+----------------+", + ]; + + assert_batches_eq!(expected, &df.clone().collect().await.unwrap()); + + let result_df = df.clone().filter(col("my_nested_dict")).unwrap(); + let expected = [ + "+----------------+", + "| my_nested_dict |", + "+----------------+", + "| true |", + "+----------------+", + ]; + + assert_batches_eq!(expected, &result_df.collect().await.unwrap()); +} diff --git a/datafusion/expr/src/logical_plan/plan.rs b/datafusion/expr/src/logical_plan/plan.rs index 975bfc9feebf..1c94c7f3afd3 100644 --- a/datafusion/expr/src/logical_plan/plan.rs +++ b/datafusion/expr/src/logical_plan/plan.rs @@ -2207,6 +2207,17 @@ impl Filter { Self::try_new_internal(predicate, input, true) } + fn is_allowed_filter_type(data_type: &DataType) -> bool { + match data_type { + // Interpret NULL as a missing boolean value. + DataType::Boolean | DataType::Null => true, + DataType::Dictionary(_, value_type) => { + Filter::is_allowed_filter_type(value_type.as_ref()) + } + _ => false, + } + } + fn try_new_internal( predicate: Expr, input: Arc, @@ -2217,8 +2228,7 @@ impl Filter { // construction (such as with correlated subqueries) so we make a best effort here and // ignore errors resolving the expression against the schema. if let Ok(predicate_type) = predicate.get_type(input.schema()) { - // Interpret NULL as a missing boolean value. - if predicate_type != DataType::Boolean && predicate_type != DataType::Null { + if !Filter::is_allowed_filter_type(&predicate_type) { return plan_err!( "Cannot create filter with non-boolean predicate '{predicate}' returning {predicate_type}" ); From 3ece7a736193a87941a00eb35f3001df282fd075 Mon Sep 17 00:00:00 2001 From: wiedld Date: Tue, 10 Sep 2024 01:32:04 -1000 Subject: [PATCH 142/154] Fix parquet statistics for ListingTable and Utf8View with `schema_force_string_view`, rename config option to `schema_force_view_types` (#12232) * chore: move schema_force_string_view upwards to be listed with other reading props * refactor(12123): have file schema be merged on view types with table schema * test(12123): test for with, and without schema_force_string_view * test(12123): demonstrate current upstream failure when reading page stats * chore(12123): update config.md * chore: cleanup * chore(12123): temporarily remove test until next arrow release * chore(12123): rename all variables to force_view_types * refactor(12123): make interface ParquetFormat::with_force_view_types public * chore(12123): rename helper method which coerces the schema (not merging fields) * chore(12123): add dosc to ParquetFormat to clarify exactly how the view types are used * test(12123): cleanup tests to be more explicit with ForceViews enum * test(12123): update tests to pass now that latest arrow-rs release is in * fix: use proper naming on benchmark --- benchmarks/src/clickbench.rs | 2 +- benchmarks/src/tpch/run.rs | 6 +- benchmarks/src/util/options.rs | 2 +- datafusion/common/src/config.rs | 8 +- .../common/src/file_options/parquet_writer.rs | 7 +- .../core/src/datasource/file_format/mod.rs | 45 ++++ .../src/datasource/file_format/parquet.rs | 236 +++++++++++++++--- .../datasource/physical_plan/parquet/mod.rs | 34 ++- .../physical_plan/parquet/opener.rs | 11 +- datafusion/functions/benches/pad.rs | 4 +- datafusion/functions/benches/repeat.rs | 4 +- datafusion/functions/benches/substr.rs | 8 +- .../proto/datafusion_common.proto | 2 +- datafusion/proto-common/src/from_proto/mod.rs | 2 +- .../proto-common/src/generated/pbjson.rs | 26 +- .../proto-common/src/generated/prost.rs | 2 +- datafusion/proto-common/src/to_proto/mod.rs | 2 +- .../src/generated/datafusion_proto_common.rs | 2 +- .../proto/src/logical_plan/file_formats.rs | 4 +- .../test_files/information_schema.slt | 4 +- docs/source/user-guide/configs.md | 2 +- 21 files changed, 318 insertions(+), 95 deletions(-) diff --git a/benchmarks/src/clickbench.rs b/benchmarks/src/clickbench.rs index a0f051d17623..207da4020b58 100644 --- a/benchmarks/src/clickbench.rs +++ b/benchmarks/src/clickbench.rs @@ -121,7 +121,7 @@ impl RunOpt { .options_mut() .execution .parquet - .schema_force_string_view = self.common.string_view; + .schema_force_view_types = self.common.force_view_types; let ctx = SessionContext::new_with_config(config); self.register_hits(&ctx).await?; diff --git a/benchmarks/src/tpch/run.rs b/benchmarks/src/tpch/run.rs index d79eb65649bd..1a1f51f70065 100644 --- a/benchmarks/src/tpch/run.rs +++ b/benchmarks/src/tpch/run.rs @@ -124,7 +124,7 @@ impl RunOpt { .options_mut() .execution .parquet - .schema_force_string_view = self.common.string_view; + .schema_force_view_types = self.common.force_view_types; let ctx = SessionContext::new_with_config(config); // register tables @@ -345,7 +345,7 @@ mod tests { partitions: Some(2), batch_size: 8192, debug: false, - string_view: false, + force_view_types: false, }; let opt = RunOpt { query: Some(query), @@ -379,7 +379,7 @@ mod tests { partitions: Some(2), batch_size: 8192, debug: false, - string_view: false, + force_view_types: false, }; let opt = RunOpt { query: Some(query), diff --git a/benchmarks/src/util/options.rs b/benchmarks/src/util/options.rs index 02591e293272..efdb074b2461 100644 --- a/benchmarks/src/util/options.rs +++ b/benchmarks/src/util/options.rs @@ -41,7 +41,7 @@ pub struct CommonOpt { /// If true, will use StringView/BinaryViewArray instead of String/BinaryArray /// when reading ParquetFiles #[structopt(long)] - pub string_view: bool, + pub force_view_types: bool, } impl CommonOpt { diff --git a/datafusion/common/src/config.rs b/datafusion/common/src/config.rs index d77655e8a77f..1e1c5d5424b0 100644 --- a/datafusion/common/src/config.rs +++ b/datafusion/common/src/config.rs @@ -380,6 +380,10 @@ config_namespace! { /// the filters are applied in the same order as written in the query pub reorder_filters: bool, default = false + /// (reading) If true, parquet reader will read columns of `Utf8/Utf8Large` with `Utf8View`, + /// and `Binary/BinaryLarge` with `BinaryView`. + pub schema_force_view_types: bool, default = false + // The following options affect writing to parquet files // and map to parquet::file::properties::WriterProperties @@ -483,10 +487,6 @@ config_namespace! { /// writing out already in-memory data, such as from a cached /// data frame. pub maximum_buffered_record_batches_per_stream: usize, default = 2 - - /// (reading) If true, parquet reader will read columns of `Utf8/Utf8Large` with `Utf8View`, - /// and `Binary/BinaryLarge` with `BinaryView`. - pub schema_force_string_view: bool, default = false } } diff --git a/datafusion/common/src/file_options/parquet_writer.rs b/datafusion/common/src/file_options/parquet_writer.rs index e42fb96ed6a5..5d553d59da4e 100644 --- a/datafusion/common/src/file_options/parquet_writer.rs +++ b/datafusion/common/src/file_options/parquet_writer.rs @@ -175,7 +175,7 @@ impl ParquetOptions { maximum_parallel_row_group_writers: _, maximum_buffered_record_batches_per_stream: _, bloom_filter_on_read: _, // reads not used for writer props - schema_force_string_view: _, + schema_force_view_types: _, } = self; let mut builder = WriterProperties::builder() @@ -441,7 +441,7 @@ mod tests { maximum_buffered_record_batches_per_stream: defaults .maximum_buffered_record_batches_per_stream, bloom_filter_on_read: defaults.bloom_filter_on_read, - schema_force_string_view: defaults.schema_force_string_view, + schema_force_view_types: defaults.schema_force_view_types, } } @@ -542,8 +542,7 @@ mod tests { maximum_buffered_record_batches_per_stream: global_options_defaults .maximum_buffered_record_batches_per_stream, bloom_filter_on_read: global_options_defaults.bloom_filter_on_read, - schema_force_string_view: global_options_defaults - .schema_force_string_view, + schema_force_view_types: global_options_defaults.schema_force_view_types, }, column_specific_options, key_value_metadata, diff --git a/datafusion/core/src/datasource/file_format/mod.rs b/datafusion/core/src/datasource/file_format/mod.rs index d21464b74b53..1dcf480cf4f2 100644 --- a/datafusion/core/src/datasource/file_format/mod.rs +++ b/datafusion/core/src/datasource/file_format/mod.rs @@ -229,6 +229,51 @@ pub fn transform_schema_to_view(schema: &Schema) -> Schema { Schema::new_with_metadata(transformed_fields, schema.metadata.clone()) } +/// Coerces the file schema if the table schema uses a view type. +pub(crate) fn coerce_file_schema_to_view_type( + table_schema: &Schema, + file_schema: &Schema, +) -> Option { + let mut transform = false; + let table_fields: HashMap<_, _> = table_schema + .fields + .iter() + .map(|f| { + let dt = f.data_type(); + if dt.equals_datatype(&DataType::Utf8View) { + transform = true; + } + (f.name(), dt) + }) + .collect(); + if !transform { + return None; + } + + let transformed_fields: Vec> = file_schema + .fields + .iter() + .map( + |field| match (table_fields.get(field.name()), field.data_type()) { + (Some(DataType::Utf8View), DataType::Utf8) + | (Some(DataType::Utf8View), DataType::LargeUtf8) => Arc::new( + Field::new(field.name(), DataType::Utf8View, field.is_nullable()), + ), + (Some(DataType::BinaryView), DataType::Binary) + | (Some(DataType::BinaryView), DataType::LargeBinary) => Arc::new( + Field::new(field.name(), DataType::BinaryView, field.is_nullable()), + ), + _ => field.clone(), + }, + ) + .collect(); + + Some(Schema::new_with_metadata( + transformed_fields, + file_schema.metadata.clone(), + )) +} + #[cfg(test)] pub(crate) mod test_util { use std::ops::Range; diff --git a/datafusion/core/src/datasource/file_format/parquet.rs b/datafusion/core/src/datasource/file_format/parquet.rs index 76e8ad9da559..2a862dd6dcb3 100644 --- a/datafusion/core/src/datasource/file_format/parquet.rs +++ b/datafusion/core/src/datasource/file_format/parquet.rs @@ -24,7 +24,10 @@ use std::sync::Arc; use super::write::demux::start_demuxer_task; use super::write::{create_writer, SharedBuffer}; -use super::{transform_schema_to_view, FileFormat, FileFormatFactory, FileScanConfig}; +use super::{ + coerce_file_schema_to_view_type, transform_schema_to_view, FileFormat, + FileFormatFactory, FileScanConfig, +}; use crate::arrow::array::RecordBatch; use crate::arrow::datatypes::{Fields, Schema, SchemaRef}; use crate::datasource::file_format::file_compression_type::FileCompressionType; @@ -227,6 +230,29 @@ impl ParquetFormat { pub fn options(&self) -> &TableParquetOptions { &self.options } + + /// Return `true` if should use view types. + /// + /// If this returns true, DataFusion will instruct the parquet reader + /// to read string / binary columns using view `StringView` or `BinaryView` + /// if the table schema specifies those types, regardless of any embedded metadata + /// that may specify an alternate Arrow type. The parquet reader is optimized + /// for reading `StringView` and `BinaryView` and such queries are significantly faster. + /// + /// If this returns false, the parquet reader will read the columns according to the + /// defaults or any embedded Arrow type information. This may result in reading + /// `StringArrays` and then casting to `StringViewArray` which is less efficient. + pub fn force_view_types(&self) -> bool { + self.options.global.schema_force_view_types + } + + /// If true, will use view types (StringView and BinaryView). + /// + /// Refer to [`Self::force_view_types`]. + pub fn with_force_view_types(mut self, use_views: bool) -> Self { + self.options.global.schema_force_view_types = use_views; + self + } } /// Clears all metadata (Schema level and field level) on an iterator @@ -317,12 +343,7 @@ impl FileFormat for ParquetFormat { Schema::try_merge(schemas) }?; - let schema = if state - .config_options() - .execution - .parquet - .schema_force_string_view - { + let schema = if self.force_view_types() { transform_schema_to_view(&schema) } else { schema @@ -515,10 +536,13 @@ pub fn statistics_from_parquet_meta_calc( statistics.total_byte_size = Precision::Exact(total_byte_size); let file_metadata = metadata.file_metadata(); - let file_schema = parquet_to_arrow_schema( + let mut file_schema = parquet_to_arrow_schema( file_metadata.schema_descr(), file_metadata.key_value_metadata(), )?; + if let Some(merged) = coerce_file_schema_to_view_type(&table_schema, &file_schema) { + file_schema = merged; + } statistics.column_statistics = if has_statistics { let (mut max_accs, mut min_accs) = create_max_min_accs(&table_schema); @@ -1249,8 +1273,8 @@ mod tests { use arrow_schema::{DataType, Field}; use async_trait::async_trait; use datafusion_common::cast::{ - as_binary_array, as_boolean_array, as_float32_array, as_float64_array, - as_int32_array, as_timestamp_nanosecond_array, + as_binary_array, as_binary_view_array, as_boolean_array, as_float32_array, + as_float64_array, as_int32_array, as_timestamp_nanosecond_array, }; use datafusion_common::config::ParquetOptions; use datafusion_common::ScalarValue; @@ -1271,8 +1295,12 @@ mod tests { use parquet::file::page_index::index::Index; use tokio::fs::File; - #[tokio::test] - async fn read_merged_batches() -> Result<()> { + enum ForceViews { + Yes, + No, + } + + async fn _run_read_merged_batches(force_views: ForceViews) -> Result<()> { let c1: ArrayRef = Arc::new(StringArray::from(vec![Some("Foo"), None, Some("bar")])); @@ -1286,7 +1314,11 @@ mod tests { let session = SessionContext::new(); let ctx = session.state(); - let format = ParquetFormat::default(); + let force_views = match force_views { + ForceViews::Yes => true, + ForceViews::No => false, + }; + let format = ParquetFormat::default().with_force_view_types(force_views); let schema = format.infer_schema(&ctx, &store, &meta).await.unwrap(); let stats = @@ -1316,6 +1348,14 @@ mod tests { Ok(()) } + #[tokio::test] + async fn read_merged_batches() -> Result<()> { + _run_read_merged_batches(ForceViews::No).await?; + _run_read_merged_batches(ForceViews::Yes).await?; + + Ok(()) + } + #[tokio::test] async fn is_schema_stable() -> Result<()> { let c1: ArrayRef = @@ -1446,8 +1486,7 @@ mod tests { } } - #[tokio::test] - async fn fetch_metadata_with_size_hint() -> Result<()> { + async fn _run_fetch_metadata_with_size_hint(force_views: ForceViews) -> Result<()> { let c1: ArrayRef = Arc::new(StringArray::from(vec![Some("Foo"), None, Some("bar")])); @@ -1471,7 +1510,13 @@ mod tests { let session = SessionContext::new(); let ctx = session.state(); - let format = ParquetFormat::default().with_metadata_size_hint(Some(9)); + let force_views = match force_views { + ForceViews::Yes => true, + ForceViews::No => false, + }; + let format = ParquetFormat::default() + .with_metadata_size_hint(Some(9)) + .with_force_view_types(force_views); let schema = format .infer_schema(&ctx, &store.upcast(), &meta) .await @@ -1501,7 +1546,9 @@ mod tests { // ensure the requests were coalesced into a single request assert_eq!(store.request_count(), 1); - let format = ParquetFormat::default().with_metadata_size_hint(Some(size_hint)); + let format = ParquetFormat::default() + .with_metadata_size_hint(Some(size_hint)) + .with_force_view_types(force_views); let schema = format .infer_schema(&ctx, &store.upcast(), &meta) .await @@ -1536,6 +1583,14 @@ mod tests { Ok(()) } + #[tokio::test] + async fn fetch_metadata_with_size_hint() -> Result<()> { + _run_fetch_metadata_with_size_hint(ForceViews::No).await?; + _run_fetch_metadata_with_size_hint(ForceViews::Yes).await?; + + Ok(()) + } + #[tokio::test] async fn test_statistics_from_parquet_metadata_dictionary() -> Result<()> { // Data for column c_dic: ["a", "b", "c", "d"] @@ -1578,8 +1633,9 @@ mod tests { Ok(()) } - #[tokio::test] - async fn test_statistics_from_parquet_metadata() -> Result<()> { + async fn _run_test_statistics_from_parquet_metadata( + force_views: ForceViews, + ) -> Result<()> { // Data for column c1: ["Foo", null, "bar"] let c1: ArrayRef = Arc::new(StringArray::from(vec![Some("Foo"), None, Some("bar")])); @@ -1597,28 +1653,42 @@ mod tests { let store = Arc::new(LocalFileSystem::new()) as _; let (files, _file_names) = store_parquet(vec![batch1, batch2], false).await?; - let state = SessionContext::new().state(); - let format = ParquetFormat::default(); + let force_views = match force_views { + ForceViews::Yes => true, + ForceViews::No => false, + }; + + let mut state = SessionContext::new().state(); + state = set_view_state(state, force_views); + let format = ParquetFormat::default().with_force_view_types(force_views); let schema = format.infer_schema(&state, &store, &files).await.unwrap(); let null_i64 = ScalarValue::Int64(None); - let null_utf8 = ScalarValue::Utf8(None); + let null_utf8 = if force_views { + ScalarValue::Utf8View(None) + } else { + ScalarValue::Utf8(None) + }; // Fetch statistics for first file let pq_meta = fetch_parquet_metadata(store.as_ref(), &files[0], None).await?; let stats = statistics_from_parquet_meta_calc(&pq_meta, schema.clone())?; - // assert_eq!(stats.num_rows, Precision::Exact(3)); // column c1 let c1_stats = &stats.column_statistics[0]; assert_eq!(c1_stats.null_count, Precision::Exact(1)); + let expected_type = if force_views { + ScalarValue::Utf8View + } else { + ScalarValue::Utf8 + }; assert_eq!( c1_stats.max_value, - Precision::Exact(ScalarValue::Utf8(Some("bar".to_string()))) + Precision::Exact(expected_type(Some("bar".to_string()))) ); assert_eq!( c1_stats.min_value, - Precision::Exact(ScalarValue::Utf8(Some("Foo".to_string()))) + Precision::Exact(expected_type(Some("Foo".to_string()))) ); // column c2: missing from the file so the table treats all 3 rows as null let c2_stats = &stats.column_statistics[1]; @@ -1644,6 +1714,15 @@ mod tests { Ok(()) } + #[tokio::test] + async fn test_statistics_from_parquet_metadata() -> Result<()> { + _run_test_statistics_from_parquet_metadata(ForceViews::No).await?; + + _run_test_statistics_from_parquet_metadata(ForceViews::Yes).await?; + + Ok(()) + } + #[tokio::test] async fn read_small_batches() -> Result<()> { let config = SessionConfig::new().with_batch_size(2); @@ -1718,10 +1797,31 @@ mod tests { Ok(()) } - #[tokio::test] - async fn read_alltypes_plain_parquet() -> Result<()> { + fn set_view_state(mut state: SessionState, use_views: bool) -> SessionState { + let mut options = TableParquetOptions::default(); + options.global.schema_force_view_types = use_views; + state + .register_file_format( + Arc::new(ParquetFormatFactory::new_with_options(options)), + true, + ) + .expect("ok"); + state + } + + async fn _run_read_alltypes_plain_parquet( + force_views: ForceViews, + expected: &str, + ) -> Result<()> { + let force_views = match force_views { + ForceViews::Yes => true, + ForceViews::No => false, + }; + let session_ctx = SessionContext::new(); - let state = session_ctx.state(); + let mut state = session_ctx.state(); + state = set_view_state(state, force_views); + let task_ctx = state.task_ctx(); let projection = None; let exec = get_exec(&state, "alltypes_plain.parquet", projection, None).await?; @@ -1733,8 +1833,20 @@ mod tests { .map(|f| format!("{}: {:?}", f.name(), f.data_type())) .collect(); let y = x.join("\n"); - assert_eq!( - "id: Int32\n\ + assert_eq!(expected, y); + + let batches = collect(exec, task_ctx).await?; + + assert_eq!(1, batches.len()); + assert_eq!(11, batches[0].num_columns()); + assert_eq!(8, batches[0].num_rows()); + + Ok(()) + } + + #[tokio::test] + async fn read_alltypes_plain_parquet() -> Result<()> { + let no_views = "id: Int32\n\ bool_col: Boolean\n\ tinyint_col: Int32\n\ smallint_col: Int32\n\ @@ -1744,15 +1856,21 @@ mod tests { double_col: Float64\n\ date_string_col: Binary\n\ string_col: Binary\n\ - timestamp_col: Timestamp(Nanosecond, None)", - y - ); + timestamp_col: Timestamp(Nanosecond, None)"; + _run_read_alltypes_plain_parquet(ForceViews::No, no_views).await?; - let batches = collect(exec, task_ctx).await?; - - assert_eq!(1, batches.len()); - assert_eq!(11, batches[0].num_columns()); - assert_eq!(8, batches[0].num_rows()); + let with_views = "id: Int32\n\ + bool_col: Boolean\n\ + tinyint_col: Int32\n\ + smallint_col: Int32\n\ + int_col: Int32\n\ + bigint_col: Int64\n\ + float_col: Float32\n\ + double_col: Float64\n\ + date_string_col: BinaryView\n\ + string_col: BinaryView\n\ + timestamp_col: Timestamp(Nanosecond, None)"; + _run_read_alltypes_plain_parquet(ForceViews::Yes, with_views).await?; Ok(()) } @@ -1889,7 +2007,9 @@ mod tests { #[tokio::test] async fn read_binary_alltypes_plain_parquet() -> Result<()> { let session_ctx = SessionContext::new(); - let state = session_ctx.state(); + let mut state = session_ctx.state(); + state = set_view_state(state, false); + let task_ctx = state.task_ctx(); let projection = Some(vec![9]); let exec = get_exec(&state, "alltypes_plain.parquet", projection, None).await?; @@ -1913,6 +2033,35 @@ mod tests { Ok(()) } + #[tokio::test] + async fn read_binaryview_alltypes_plain_parquet() -> Result<()> { + let session_ctx = SessionContext::new(); + let mut state = session_ctx.state(); + state = set_view_state(state, true); + + let task_ctx = state.task_ctx(); + let projection = Some(vec![9]); + let exec = get_exec(&state, "alltypes_plain.parquet", projection, None).await?; + + let batches = collect(exec, task_ctx).await?; + assert_eq!(1, batches.len()); + assert_eq!(1, batches[0].num_columns()); + assert_eq!(8, batches[0].num_rows()); + + let array = as_binary_view_array(batches[0].column(0))?; + let mut values: Vec<&str> = vec![]; + for i in 0..batches[0].num_rows() { + values.push(std::str::from_utf8(array.value(i)).unwrap()); + } + + assert_eq!( + "[\"0\", \"1\", \"0\", \"1\", \"0\", \"1\", \"0\", \"1\"]", + format!("{values:?}") + ); + + Ok(()) + } + #[tokio::test] async fn read_decimal_parquet() -> Result<()> { let session_ctx = SessionContext::new(); @@ -2047,8 +2196,13 @@ mod tests { limit: Option, ) -> Result> { let testdata = crate::test_util::parquet_test_data(); - let format = ParquetFormat::default(); - scan_format(state, &format, &testdata, file_name, projection, limit).await + + let format = state + .get_file_format_factory("parquet") + .map(|factory| factory.create(state, &Default::default()).unwrap()) + .unwrap_or(Arc::new(ParquetFormat::new())); + + scan_format(state, &*format, &testdata, file_name, projection, limit).await } fn build_ctx(store_url: &url::Url) -> Arc { diff --git a/datafusion/core/src/datasource/physical_plan/parquet/mod.rs b/datafusion/core/src/datasource/physical_plan/parquet/mod.rs index b2f86db742f2..54d4d7262a8e 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet/mod.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet/mod.rs @@ -726,10 +726,6 @@ impl ExecutionPlan for ParquetExec { enable_page_index: self.enable_page_index(), enable_bloom_filter: self.bloom_filter_on_read(), schema_adapter_factory, - schema_force_string_view: self - .table_parquet_options - .global - .schema_force_string_view, }; let stream = @@ -2095,6 +2091,36 @@ mod tests { Ok(()) } + #[tokio::test] + async fn test_struct_filter_parquet_with_view_types() -> Result<()> { + let tmp_dir = TempDir::new().unwrap(); + let path = tmp_dir.path().to_str().unwrap().to_string() + "/test.parquet"; + write_file(&path); + + let ctx = SessionContext::new(); + + let mut options = TableParquetOptions::default(); + options.global.schema_force_view_types = true; + let opt = + ListingOptions::new(Arc::new(ParquetFormat::default().with_options(options))); + + ctx.register_listing_table("base_table", path, opt, None, None) + .await + .unwrap(); + let sql = "select * from base_table where name='test02'"; + let batch = ctx.sql(sql).await.unwrap().collect().await.unwrap(); + assert_eq!(batch.len(), 1); + let expected = [ + "+---------------------+----+--------+", + "| struct | id | name |", + "+---------------------+----+--------+", + "| {id: 4, name: aaa2} | 2 | test02 |", + "+---------------------+----+--------+", + ]; + crate::assert_batches_eq!(expected, &batch); + Ok(()) + } + fn write_file(file: &String) { let struct_fields = Fields::from(vec![ Field::new("id", DataType::Int64, false), diff --git a/datafusion/core/src/datasource/physical_plan/parquet/opener.rs b/datafusion/core/src/datasource/physical_plan/parquet/opener.rs index 7e331293eb19..a12e60eb414a 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet/opener.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet/opener.rs @@ -17,7 +17,7 @@ //! [`ParquetOpener`] for opening Parquet files -use crate::datasource::file_format::transform_schema_to_view; +use crate::datasource::file_format::coerce_file_schema_to_view_type; use crate::datasource::physical_plan::parquet::page_filter::PagePruningAccessPlanFilter; use crate::datasource::physical_plan::parquet::row_group_filter::RowGroupAccessPlanFilter; use crate::datasource::physical_plan::parquet::{ @@ -57,7 +57,6 @@ pub(super) struct ParquetOpener { pub enable_page_index: bool, pub enable_bloom_filter: bool, pub schema_adapter_factory: Arc, - pub schema_force_string_view: bool, } impl FileOpener for ParquetOpener { @@ -93,7 +92,6 @@ impl FileOpener for ParquetOpener { ); let enable_bloom_filter = self.enable_bloom_filter; let limit = self.limit; - let schema_force_string_view = self.schema_force_string_view; Ok(Box::pin(async move { let options = ArrowReaderOptions::new().with_page_index(enable_page_index); @@ -101,9 +99,10 @@ impl FileOpener for ParquetOpener { let metadata = ArrowReaderMetadata::load_async(&mut reader, options.clone()).await?; let mut schema = metadata.schema().clone(); - - if schema_force_string_view { - schema = Arc::new(transform_schema_to_view(&schema)); + // read with view types + if let Some(merged) = coerce_file_schema_to_view_type(&table_schema, &schema) + { + schema = Arc::new(merged); } let options = ArrowReaderOptions::new() diff --git a/datafusion/functions/benches/pad.rs b/datafusion/functions/benches/pad.rs index 0c496bc63347..71fa68762c1e 100644 --- a/datafusion/functions/benches/pad.rs +++ b/datafusion/functions/benches/pad.rs @@ -67,11 +67,11 @@ where fn create_args( size: usize, str_len: usize, - use_string_view: bool, + force_view_types: bool, ) -> Vec { let length_array = Arc::new(create_primitive_array::(size, 0.0, str_len)); - if !use_string_view { + if !force_view_types { let string_array = Arc::new(create_string_array_with_len::(size, 0.1, str_len)); let fill_array = Arc::new(create_string_array_with_len::(size, 0.1, str_len)); diff --git a/datafusion/functions/benches/repeat.rs b/datafusion/functions/benches/repeat.rs index e45313660ea2..5643ccf07133 100644 --- a/datafusion/functions/benches/repeat.rs +++ b/datafusion/functions/benches/repeat.rs @@ -31,13 +31,13 @@ fn create_args( size: usize, str_len: usize, repeat_times: i64, - use_string_view: bool, + force_view_types: bool, ) -> Vec { let number_array = Arc::new(Int64Array::from( (0..size).map(|_| repeat_times).collect::>(), )); - if use_string_view { + if force_view_types { let string_array = Arc::new(create_string_view_array_with_len(size, 0.1, str_len, false)); vec![ diff --git a/datafusion/functions/benches/substr.rs b/datafusion/functions/benches/substr.rs index 1a696520c3ad..90ba75c1e8a5 100644 --- a/datafusion/functions/benches/substr.rs +++ b/datafusion/functions/benches/substr.rs @@ -30,7 +30,7 @@ fn create_args_without_count( size: usize, str_len: usize, start_half_way: bool, - use_string_view: bool, + force_view_types: bool, ) -> Vec { let start_array = Arc::new(Int64Array::from( (0..size) @@ -44,7 +44,7 @@ fn create_args_without_count( .collect::>(), )); - if use_string_view { + if force_view_types { let string_array = Arc::new(create_string_view_array_with_len(size, 0.1, str_len, false)); vec![ @@ -66,7 +66,7 @@ fn create_args_with_count( size: usize, str_len: usize, count_max: usize, - use_string_view: bool, + force_view_types: bool, ) -> Vec { let start_array = Arc::new(Int64Array::from((0..size).map(|_| 1).collect::>())); @@ -75,7 +75,7 @@ fn create_args_with_count( (0..size).map(|_| count).collect::>(), )); - if use_string_view { + if force_view_types { let string_array = Arc::new(create_string_view_array_with_len(size, 0.1, str_len, false)); vec![ diff --git a/datafusion/proto-common/proto/datafusion_common.proto b/datafusion/proto-common/proto/datafusion_common.proto index 51e94d2caaf4..d1506fcd64f0 100644 --- a/datafusion/proto-common/proto/datafusion_common.proto +++ b/datafusion/proto-common/proto/datafusion_common.proto @@ -493,7 +493,7 @@ message ParquetOptions { uint64 maximum_buffered_record_batches_per_stream = 25; // default = 2 bool bloom_filter_on_read = 26; // default = true bool bloom_filter_on_write = 27; // default = false - bool schema_force_string_view = 28; // default = false + bool schema_force_view_types = 28; // default = false oneof metadata_size_hint_opt { uint64 metadata_size_hint = 4; diff --git a/datafusion/proto-common/src/from_proto/mod.rs b/datafusion/proto-common/src/from_proto/mod.rs index 45d275fb488e..d1b4374fc0e7 100644 --- a/datafusion/proto-common/src/from_proto/mod.rs +++ b/datafusion/proto-common/src/from_proto/mod.rs @@ -957,7 +957,7 @@ impl TryFrom<&protobuf::ParquetOptions> for ParquetOptions { allow_single_file_parallelism: value.allow_single_file_parallelism, maximum_parallel_row_group_writers: value.maximum_parallel_row_group_writers as usize, maximum_buffered_record_batches_per_stream: value.maximum_buffered_record_batches_per_stream as usize, - schema_force_string_view: value.schema_force_string_view, + schema_force_view_types: value.schema_force_view_types, }) } } diff --git a/datafusion/proto-common/src/generated/pbjson.rs b/datafusion/proto-common/src/generated/pbjson.rs index 78ba829f8c50..fa5d1f442754 100644 --- a/datafusion/proto-common/src/generated/pbjson.rs +++ b/datafusion/proto-common/src/generated/pbjson.rs @@ -4891,7 +4891,7 @@ impl serde::Serialize for ParquetOptions { if self.bloom_filter_on_write { len += 1; } - if self.schema_force_string_view { + if self.schema_force_view_types { len += 1; } if self.dictionary_page_size_limit != 0 { @@ -4977,8 +4977,8 @@ impl serde::Serialize for ParquetOptions { if self.bloom_filter_on_write { struct_ser.serialize_field("bloomFilterOnWrite", &self.bloom_filter_on_write)?; } - if self.schema_force_string_view { - struct_ser.serialize_field("schemaForceStringView", &self.schema_force_string_view)?; + if self.schema_force_view_types { + struct_ser.serialize_field("schemaForceViewTypes", &self.schema_force_view_types)?; } if self.dictionary_page_size_limit != 0 { #[allow(clippy::needless_borrow)] @@ -5097,8 +5097,8 @@ impl<'de> serde::Deserialize<'de> for ParquetOptions { "bloomFilterOnRead", "bloom_filter_on_write", "bloomFilterOnWrite", - "schema_force_string_view", - "schemaForceStringView", + "schema_force_view_types", + "schemaForceViewTypes", "dictionary_page_size_limit", "dictionaryPageSizeLimit", "data_page_row_count_limit", @@ -5140,7 +5140,7 @@ impl<'de> serde::Deserialize<'de> for ParquetOptions { MaximumBufferedRecordBatchesPerStream, BloomFilterOnRead, BloomFilterOnWrite, - SchemaForceStringView, + schemaForceViewTypes, DictionaryPageSizeLimit, DataPageRowCountLimit, MaxRowGroupSize, @@ -5188,7 +5188,7 @@ impl<'de> serde::Deserialize<'de> for ParquetOptions { "maximumBufferedRecordBatchesPerStream" | "maximum_buffered_record_batches_per_stream" => Ok(GeneratedField::MaximumBufferedRecordBatchesPerStream), "bloomFilterOnRead" | "bloom_filter_on_read" => Ok(GeneratedField::BloomFilterOnRead), "bloomFilterOnWrite" | "bloom_filter_on_write" => Ok(GeneratedField::BloomFilterOnWrite), - "schemaForceStringView" | "schema_force_string_view" => Ok(GeneratedField::SchemaForceStringView), + "schemaForceViewTypes" | "schema_force_view_types" => Ok(GeneratedField::schemaForceViewTypes), "dictionaryPageSizeLimit" | "dictionary_page_size_limit" => Ok(GeneratedField::DictionaryPageSizeLimit), "dataPageRowCountLimit" | "data_page_row_count_limit" => Ok(GeneratedField::DataPageRowCountLimit), "maxRowGroupSize" | "max_row_group_size" => Ok(GeneratedField::MaxRowGroupSize), @@ -5234,7 +5234,7 @@ impl<'de> serde::Deserialize<'de> for ParquetOptions { let mut maximum_buffered_record_batches_per_stream__ = None; let mut bloom_filter_on_read__ = None; let mut bloom_filter_on_write__ = None; - let mut schema_force_string_view__ = None; + let mut schema_force_view_types__ = None; let mut dictionary_page_size_limit__ = None; let mut data_page_row_count_limit__ = None; let mut max_row_group_size__ = None; @@ -5336,11 +5336,11 @@ impl<'de> serde::Deserialize<'de> for ParquetOptions { } bloom_filter_on_write__ = Some(map_.next_value()?); } - GeneratedField::SchemaForceStringView => { - if schema_force_string_view__.is_some() { - return Err(serde::de::Error::duplicate_field("schemaForceStringView")); + GeneratedField::schemaForceViewTypes => { + if schema_force_view_types__.is_some() { + return Err(serde::de::Error::duplicate_field("schemaForceViewTypes")); } - schema_force_string_view__ = Some(map_.next_value()?); + schema_force_view_types__ = Some(map_.next_value()?); } GeneratedField::DictionaryPageSizeLimit => { if dictionary_page_size_limit__.is_some() { @@ -5442,7 +5442,7 @@ impl<'de> serde::Deserialize<'de> for ParquetOptions { maximum_buffered_record_batches_per_stream: maximum_buffered_record_batches_per_stream__.unwrap_or_default(), bloom_filter_on_read: bloom_filter_on_read__.unwrap_or_default(), bloom_filter_on_write: bloom_filter_on_write__.unwrap_or_default(), - schema_force_string_view: schema_force_string_view__.unwrap_or_default(), + schema_force_view_types: schema_force_view_types__.unwrap_or_default(), dictionary_page_size_limit: dictionary_page_size_limit__.unwrap_or_default(), data_page_row_count_limit: data_page_row_count_limit__.unwrap_or_default(), max_row_group_size: max_row_group_size__.unwrap_or_default(), diff --git a/datafusion/proto-common/src/generated/prost.rs b/datafusion/proto-common/src/generated/prost.rs index cb8f86a022a6..d6f982278d67 100644 --- a/datafusion/proto-common/src/generated/prost.rs +++ b/datafusion/proto-common/src/generated/prost.rs @@ -819,7 +819,7 @@ pub struct ParquetOptions { pub bloom_filter_on_write: bool, /// default = false #[prost(bool, tag = "28")] - pub schema_force_string_view: bool, + pub schema_force_view_types: bool, #[prost(uint64, tag = "12")] pub dictionary_page_size_limit: u64, #[prost(uint64, tag = "18")] diff --git a/datafusion/proto-common/src/to_proto/mod.rs b/datafusion/proto-common/src/to_proto/mod.rs index e608caf0ecf8..ebb53ae7577c 100644 --- a/datafusion/proto-common/src/to_proto/mod.rs +++ b/datafusion/proto-common/src/to_proto/mod.rs @@ -830,7 +830,7 @@ impl TryFrom<&ParquetOptions> for protobuf::ParquetOptions { allow_single_file_parallelism: value.allow_single_file_parallelism, maximum_parallel_row_group_writers: value.maximum_parallel_row_group_writers as u64, maximum_buffered_record_batches_per_stream: value.maximum_buffered_record_batches_per_stream as u64, - schema_force_string_view: value.schema_force_string_view, + schema_force_view_types: value.schema_force_view_types, }) } } diff --git a/datafusion/proto/src/generated/datafusion_proto_common.rs b/datafusion/proto/src/generated/datafusion_proto_common.rs index dc8d0017d3fd..be12d5b8e30e 100644 --- a/datafusion/proto/src/generated/datafusion_proto_common.rs +++ b/datafusion/proto/src/generated/datafusion_proto_common.rs @@ -819,7 +819,7 @@ pub struct ParquetOptions { pub bloom_filter_on_write: bool, /// default = false #[prost(bool, tag = "28")] - pub schema_force_string_view: bool, + pub schema_force_view_types: bool, #[prost(uint64, tag = "12")] pub dictionary_page_size_limit: u64, #[prost(uint64, tag = "18")] diff --git a/datafusion/proto/src/logical_plan/file_formats.rs b/datafusion/proto/src/logical_plan/file_formats.rs index 2e3476da6ac0..0f9f9d335afe 100644 --- a/datafusion/proto/src/logical_plan/file_formats.rs +++ b/datafusion/proto/src/logical_plan/file_formats.rs @@ -406,7 +406,7 @@ impl TableParquetOptionsProto { allow_single_file_parallelism: global_options.global.allow_single_file_parallelism, maximum_parallel_row_group_writers: global_options.global.maximum_parallel_row_group_writers as u64, maximum_buffered_record_batches_per_stream: global_options.global.maximum_buffered_record_batches_per_stream as u64, - schema_force_string_view: global_options.global.schema_force_string_view, + schema_force_view_types: global_options.global.schema_force_view_types, }), column_specific_options: column_specific_options.into_iter().map(|(column_name, options)| { ParquetColumnSpecificOptions { @@ -496,7 +496,7 @@ impl From<&ParquetOptionsProto> for ParquetOptions { allow_single_file_parallelism: proto.allow_single_file_parallelism, maximum_parallel_row_group_writers: proto.maximum_parallel_row_group_writers as usize, maximum_buffered_record_batches_per_stream: proto.maximum_buffered_record_batches_per_stream as usize, - schema_force_string_view: proto.schema_force_string_view, + schema_force_view_types: proto.schema_force_view_types, } } } diff --git a/datafusion/sqllogictest/test_files/information_schema.slt b/datafusion/sqllogictest/test_files/information_schema.slt index beefa24ba4c6..7acdf25b6596 100644 --- a/datafusion/sqllogictest/test_files/information_schema.slt +++ b/datafusion/sqllogictest/test_files/information_schema.slt @@ -200,7 +200,7 @@ datafusion.execution.parquet.metadata_size_hint NULL datafusion.execution.parquet.pruning true datafusion.execution.parquet.pushdown_filters false datafusion.execution.parquet.reorder_filters false -datafusion.execution.parquet.schema_force_string_view false +datafusion.execution.parquet.schema_force_view_types false datafusion.execution.parquet.skip_metadata true datafusion.execution.parquet.statistics_enabled page datafusion.execution.parquet.write_batch_size 1024 @@ -290,7 +290,7 @@ datafusion.execution.parquet.metadata_size_hint NULL (reading) If specified, the datafusion.execution.parquet.pruning true (reading) If true, the parquet reader attempts to skip entire row groups based on the predicate in the query and the metadata (min/max values) stored in the parquet file datafusion.execution.parquet.pushdown_filters false (reading) If true, filter expressions are be applied during the parquet decoding operation to reduce the number of rows decoded. This optimization is sometimes called "late materialization". datafusion.execution.parquet.reorder_filters false (reading) If true, filter expressions evaluated during the parquet decoding operation will be reordered heuristically to minimize the cost of evaluation. If false, the filters are applied in the same order as written in the query -datafusion.execution.parquet.schema_force_string_view false (reading) If true, parquet reader will read columns of `Utf8/Utf8Large` with `Utf8View`, and `Binary/BinaryLarge` with `BinaryView`. +datafusion.execution.parquet.schema_force_view_types false (reading) If true, parquet reader will read columns of `Utf8/Utf8Large` with `Utf8View`, and `Binary/BinaryLarge` with `BinaryView`. datafusion.execution.parquet.skip_metadata true (reading) If true, the parquet reader skip the optional embedded metadata that may be in the file Schema. This setting can help avoid schema conflicts when querying multiple parquet files with schemas containing compatible types but different metadata datafusion.execution.parquet.statistics_enabled page (writing) Sets if statistics are enabled for any column Valid values are: "none", "chunk", and "page" These values are not case sensitive. If NULL, uses default parquet writer setting datafusion.execution.parquet.write_batch_size 1024 (writing) Sets write_batch_size in bytes diff --git a/docs/source/user-guide/configs.md b/docs/source/user-guide/configs.md index f7e25bd55850..4cb43fb89016 100644 --- a/docs/source/user-guide/configs.md +++ b/docs/source/user-guide/configs.md @@ -56,6 +56,7 @@ Environment variables are read during `SessionConfig` initialisation so they mus | datafusion.execution.parquet.metadata_size_hint | NULL | (reading) If specified, the parquet reader will try and fetch the last `size_hint` bytes of the parquet file optimistically. If not specified, two reads are required: One read to fetch the 8-byte parquet footer and another to fetch the metadata length encoded in the footer | | datafusion.execution.parquet.pushdown_filters | false | (reading) If true, filter expressions are be applied during the parquet decoding operation to reduce the number of rows decoded. This optimization is sometimes called "late materialization". | | datafusion.execution.parquet.reorder_filters | false | (reading) If true, filter expressions evaluated during the parquet decoding operation will be reordered heuristically to minimize the cost of evaluation. If false, the filters are applied in the same order as written in the query | +| datafusion.execution.parquet.schema_force_view_types | false | (reading) If true, parquet reader will read columns of `Utf8/Utf8Large` with `Utf8View`, and `Binary/BinaryLarge` with `BinaryView`. | | datafusion.execution.parquet.data_pagesize_limit | 1048576 | (writing) Sets best effort maximum size of data page in bytes | | datafusion.execution.parquet.write_batch_size | 1024 | (writing) Sets write_batch_size in bytes | | datafusion.execution.parquet.writer_version | 1.0 | (writing) Sets parquet writer version valid values are "1.0" and "2.0" | @@ -76,7 +77,6 @@ Environment variables are read during `SessionConfig` initialisation so they mus | datafusion.execution.parquet.allow_single_file_parallelism | true | (writing) Controls whether DataFusion will attempt to speed up writing parquet files by serializing them in parallel. Each column in each row group in each output file are serialized in parallel leveraging a maximum possible core count of n_files*n_row_groups*n_columns. | | datafusion.execution.parquet.maximum_parallel_row_group_writers | 1 | (writing) By default parallel parquet writer is tuned for minimum memory usage in a streaming execution plan. You may see a performance benefit when writing large parquet files by increasing maximum_parallel_row_group_writers and maximum_buffered_record_batches_per_stream if your system has idle cores and can tolerate additional memory usage. Boosting these values is likely worthwhile when writing out already in-memory data, such as from a cached data frame. | | datafusion.execution.parquet.maximum_buffered_record_batches_per_stream | 2 | (writing) By default parallel parquet writer is tuned for minimum memory usage in a streaming execution plan. You may see a performance benefit when writing large parquet files by increasing maximum_parallel_row_group_writers and maximum_buffered_record_batches_per_stream if your system has idle cores and can tolerate additional memory usage. Boosting these values is likely worthwhile when writing out already in-memory data, such as from a cached data frame. | -| datafusion.execution.parquet.schema_force_string_view | false | (reading) If true, parquet reader will read columns of `Utf8/Utf8Large` with `Utf8View`, and `Binary/BinaryLarge` with `BinaryView`. | | datafusion.execution.planning_concurrency | 0 | Fan-out during initial physical planning. This is mostly use to plan `UNION` children in parallel. Defaults to the number of CPU cores on the system | | datafusion.execution.sort_spill_reservation_bytes | 10485760 | Specifies the reserved memory for each spillable sort operation to facilitate an in-memory merge. When a sort operation spills to disk, the in-memory data must be sorted and merged before being written to a file. This setting reserves a specific amount of memory for that in-memory sort/merge process. Note: This setting is irrelevant if the sort operation cannot spill (i.e., if there's no `DiskManager` configured). | | datafusion.execution.sort_in_place_threshold_bytes | 1048576 | When sorting, below what size should data be concatenated and sorted in a single RecordBatch rather than sorted in batches and merged. | From 6aae2ee7eb7b1fa9e352ec1899fdf640c848faf3 Mon Sep 17 00:00:00 2001 From: Bruce Ritchie Date: Wed, 11 Sep 2024 09:31:37 -0400 Subject: [PATCH 143/154] Add support for Utf8View, Boolean, Date32/64, int32/64 for writing hive style partitions (#12283) * Add support for Utf8View, Boolean, Date32/64, int32/64 for writing out hive style partitioning. * Swith to Cow vs String to reduce instances of string allocation. * Cargo fmt update. --- .../src/datasource/file_format/write/demux.rs | 71 +++++++++++++++++-- datafusion/sqllogictest/test_files/copy.slt | 22 ++++++ 2 files changed, 87 insertions(+), 6 deletions(-) diff --git a/datafusion/core/src/datasource/file_format/write/demux.rs b/datafusion/core/src/datasource/file_format/write/demux.rs index a58c77e31313..427b28db4030 100644 --- a/datafusion/core/src/datasource/file_format/write/demux.rs +++ b/datafusion/core/src/datasource/file_format/write/demux.rs @@ -18,6 +18,7 @@ //! Module containing helper methods/traits related to enabling //! dividing input stream into multiple output files at execution time +use std::borrow::Cow; use std::collections::HashMap; use std::sync::Arc; @@ -31,7 +32,11 @@ use arrow_array::builder::UInt64Builder; use arrow_array::cast::AsArray; use arrow_array::{downcast_dictionary_array, RecordBatch, StringArray, StructArray}; use arrow_schema::{DataType, Schema}; -use datafusion_common::cast::as_string_array; +use chrono::NaiveDate; +use datafusion_common::cast::{ + as_boolean_array, as_date32_array, as_date64_array, as_int32_array, as_int64_array, + as_string_array, as_string_view_array, +}; use datafusion_common::{exec_datafusion_err, DataFusionError}; use datafusion_common_runtime::SpawnedTask; use datafusion_execution::TaskContext; @@ -320,9 +325,11 @@ async fn hive_style_partitions_demuxer( fn compute_partition_keys_by_row<'a>( rb: &'a RecordBatch, partition_by: &'a [(String, DataType)], -) -> Result>> { +) -> Result>>> { let mut all_partition_values = vec![]; + const EPOCH_DAYS_FROM_CE: i32 = 719_163; + // For the purposes of writing partitioned data, we can rely on schema inference // to determine the type of the partition cols in order to provide a more ergonomic // UI which does not require specifying DataTypes manually. So, we ignore the @@ -342,7 +349,59 @@ fn compute_partition_keys_by_row<'a>( DataType::Utf8 => { let array = as_string_array(col_array)?; for i in 0..rb.num_rows() { - partition_values.push(array.value(i)); + partition_values.push(Cow::from(array.value(i))); + } + } + DataType::Utf8View => { + let array = as_string_view_array(col_array)?; + for i in 0..rb.num_rows() { + partition_values.push(Cow::from(array.value(i))); + } + } + DataType::Boolean => { + let array = as_boolean_array(col_array)?; + for i in 0..rb.num_rows() { + partition_values.push(Cow::from(array.value(i).to_string())); + } + } + DataType::Date32 => { + let array = as_date32_array(col_array)?; + // ISO-8601/RFC3339 format - yyyy-mm-dd + let format = "%Y-%m-%d"; + for i in 0..rb.num_rows() { + let date = NaiveDate::from_num_days_from_ce_opt( + EPOCH_DAYS_FROM_CE + array.value(i), + ) + .unwrap() + .format(format) + .to_string(); + partition_values.push(Cow::from(date)); + } + } + DataType::Date64 => { + let array = as_date64_array(col_array)?; + // ISO-8601/RFC3339 format - yyyy-mm-dd + let format = "%Y-%m-%d"; + for i in 0..rb.num_rows() { + let date = NaiveDate::from_num_days_from_ce_opt( + EPOCH_DAYS_FROM_CE + (array.value(i) / 86_400_000) as i32, + ) + .unwrap() + .format(format) + .to_string(); + partition_values.push(Cow::from(date)); + } + } + DataType::Int32 => { + let array = as_int32_array(col_array)?; + for i in 0..rb.num_rows() { + partition_values.push(Cow::from(array.value(i).to_string())); + } + } + DataType::Int64 => { + let array = as_int64_array(col_array)?; + for i in 0..rb.num_rows() { + partition_values.push(Cow::from(array.value(i).to_string())); } } DataType::Dictionary(_, _) => { @@ -354,7 +413,7 @@ fn compute_partition_keys_by_row<'a>( for val in array.values() { partition_values.push( - val.ok_or(exec_datafusion_err!("Cannot partition by null value for column {}", col))? + Cow::from(val.ok_or(exec_datafusion_err!("Cannot partition by null value for column {}", col))?), ); } }, @@ -377,13 +436,13 @@ fn compute_partition_keys_by_row<'a>( fn compute_take_arrays( rb: &RecordBatch, - all_partition_values: Vec>, + all_partition_values: Vec>>, ) -> HashMap, UInt64Builder> { let mut take_map = HashMap::new(); for i in 0..rb.num_rows() { let mut part_key = vec![]; for vals in all_partition_values.iter() { - part_key.push(vals[i].to_owned()); + part_key.push(vals[i].clone().into()); } let builder = take_map.entry(part_key).or_insert(UInt64Builder::new()); builder.append_value(i as u64); diff --git a/datafusion/sqllogictest/test_files/copy.slt b/datafusion/sqllogictest/test_files/copy.slt index d2a3a214d71e..40d229f9716c 100644 --- a/datafusion/sqllogictest/test_files/copy.slt +++ b/datafusion/sqllogictest/test_files/copy.slt @@ -108,6 +108,28 @@ select * from validate_partitioned_parquet_1_x order by column2; ---- a +# Copy to directory as partitioned files +query I +COPY (values (1::int, 2::bigint, 19968::date, arrow_cast(1725235200000, 'Date64'), false, 'x'), + (11::int, 22::bigint, 19969::date, arrow_cast(1725148800000, 'Date64'), true, 'y') +) +TO 'test_files/scratch/copy/partitioned_table5/' STORED AS parquet PARTITIONED BY (column1, column2, column3, column4, column5) +OPTIONS ('format.compression' 'zstd(10)'); +---- +2 + +# validate partitioning +statement ok +CREATE EXTERNAL TABLE validate_partitioned_parquet5 (column1 int, column2 bigint, column3 date, column4 date, column5 boolean, column6 varchar) STORED AS PARQUET +LOCATION 'test_files/scratch/copy/partitioned_table5/' PARTITIONED BY (column1, column2, column3, column4, column5); + +query IIDDBT +select column1, column2, column3, column4, column5, column6 from validate_partitioned_parquet5 order by column1,column2,column3,column4,column5; +---- +1 2 2024-09-02 2024-09-02 false x +11 22 2024-09-03 2024-09-01 true y + + statement ok create table test ("'test'" varchar, "'test2'" varchar, "'test3'" varchar); From f24f2cbeff92937f59c5b22f27d23ebf347c3013 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Wed, 11 Sep 2024 09:36:07 -0400 Subject: [PATCH 144/154] Minor: Fix project website links (#12419) --- README.md | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/README.md b/README.md index b1d38b61109f..8c76f0807b2d 100644 --- a/README.md +++ b/README.md @@ -33,12 +33,13 @@ [discord-badge]: https://img.shields.io/discord/885562378132000778.svg?logo=discord&style=flat-square [discord-url]: https://discord.com/invite/Qw5gKqHxUM -[Website](https://github.com/apache/datafusion) | -[Guides](https://github.com/apache/datafusion/tree/main/docs) | +[Website](https://datafusion.apache.org/) | [API Docs](https://docs.rs/datafusion/latest/datafusion/) | [Chat](https://discord.com/channels/885562378132000778/885562378132000781) -logo + + logo + Apache DataFusion is a very fast, extensible query engine for building high-quality data-centric systems in [Rust](http://rustlang.org), using the [Apache Arrow](https://arrow.apache.org) From 1ece2411eec6ff2df03f4e31ab39cc02d3844517 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Wed, 11 Sep 2024 10:43:00 -0400 Subject: [PATCH 145/154] Bump env_logger from 0.9.3 to 0.11.5 in /datafusion-cli (#12410) Bumps [env_logger](https://github.com/rust-cli/env_logger) from 0.9.3 to 0.11.5. - [Release notes](https://github.com/rust-cli/env_logger/releases) - [Changelog](https://github.com/rust-cli/env_logger/blob/main/CHANGELOG.md) - [Commits](https://github.com/rust-cli/env_logger/compare/v0.9.3...v0.11.5) --- updated-dependencies: - dependency-name: env_logger dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- datafusion-cli/Cargo.lock | 53 +++++++++++++-------------------------- datafusion-cli/Cargo.toml | 2 +- 2 files changed, 18 insertions(+), 37 deletions(-) diff --git a/datafusion-cli/Cargo.lock b/datafusion-cli/Cargo.lock index 5417f328d732..3e8d885560b6 100644 --- a/datafusion-cli/Cargo.lock +++ b/datafusion-cli/Cargo.lock @@ -454,17 +454,6 @@ version = "1.1.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "1505bd5d3d116872e7271a6d4e16d81d0c8570876c8de68093a09ac269d8aac0" -[[package]] -name = "atty" -version = "0.2.14" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d9b39be18770d11421cdb1b9947a45dd3f37e93092cbf377614828a319d5fee8" -dependencies = [ - "hermit-abi 0.1.19", - "libc", - "winapi", -] - [[package]] name = "autocfg" version = "1.3.0" @@ -1666,17 +1655,27 @@ version = "0.1.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "c34f04666d835ff5d62e058c3995147c06f42fe86ff053337632bca83e42702d" +[[package]] +name = "env_filter" +version = "0.1.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4f2c92ceda6ceec50f43169f9ee8424fe2db276791afde7b2cd8bc084cb376ab" +dependencies = [ + "log", + "regex", +] + [[package]] name = "env_logger" -version = "0.9.3" +version = "0.11.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a12e6657c4c97ebab115a42dcee77225f7f482cdd841cf7088c657a42e9e00e7" +checksum = "e13fa619b91fb2381732789fc5de83b45675e882f66623b7d8cb4f643017018d" dependencies = [ - "atty", + "anstream", + "anstyle", + "env_filter", "humantime", "log", - "regex", - "termcolor", ] [[package]] @@ -1971,15 +1970,6 @@ version = "0.5.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "2304e00983f87ffb38b55b444b5e3b60a884b5d30c0fca7d82fe33449bbe55ea" -[[package]] -name = "hermit-abi" -version = "0.1.19" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "62b467343b94ba476dcb2500d242dadbb39557df889310ac77c5d99100aaac33" -dependencies = [ - "libc", -] - [[package]] name = "hermit-abi" version = "0.3.9" @@ -2502,7 +2492,7 @@ version = "1.0.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "80e04d1dcff3aae0704555fe5fee3bcfaf3d1fdf8a7e521d5b9d2b42acb52cec" dependencies = [ - "hermit-abi 0.3.9", + "hermit-abi", "libc", "wasi", "windows-sys 0.52.0", @@ -2620,7 +2610,7 @@ version = "1.16.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "4161fcb6d602d4d2081af7c3a45852d875a03dd337a6bfdd6e06407b61342a43" dependencies = [ - "hermit-abi 0.3.9", + "hermit-abi", "libc", ] @@ -3697,15 +3687,6 @@ dependencies = [ "windows-sys 0.59.0", ] -[[package]] -name = "termcolor" -version = "1.4.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "06794f8f6c5c898b3275aebefa6b8a1cb24cd2c6c79397ab15774837a0bc5755" -dependencies = [ - "winapi-util", -] - [[package]] name = "termtree" version = "0.4.1" diff --git a/datafusion-cli/Cargo.toml b/datafusion-cli/Cargo.toml index dfd498c39805..143d5be8d05d 100644 --- a/datafusion-cli/Cargo.toml +++ b/datafusion-cli/Cargo.toml @@ -53,7 +53,7 @@ datafusion = { path = "../datafusion/core", version = "41.0.0", features = [ "compression", ] } dirs = "5.0.1" -env_logger = "0.9" +env_logger = "0.11" futures = "0.3" mimalloc = { version = "0.1", default-features = false } object_store = { version = "0.11.0", features = ["aws", "gcp", "http"] } From 13dc8a61ee9f82965cea1ee1e751455e77794f24 Mon Sep 17 00:00:00 2001 From: Tim Saucer Date: Wed, 11 Sep 2024 14:39:31 -0400 Subject: [PATCH 146/154] Check window functions by str for with_column (#12431) * Check for window functions already in schema by str match instead of if any are lacking a qualifier * cargo fmt * Update with single liner to make code easier to read * Update test documentation * Cargo fmt --- datafusion/core/src/dataframe/mod.rs | 45 +++++++++++++++++----------- 1 file changed, 27 insertions(+), 18 deletions(-) diff --git a/datafusion/core/src/dataframe/mod.rs b/datafusion/core/src/dataframe/mod.rs index 2138bd1294b4..e7aa1172a854 100644 --- a/datafusion/core/src/dataframe/mod.rs +++ b/datafusion/core/src/dataframe/mod.rs @@ -1452,28 +1452,31 @@ impl DataFrame { pub fn with_column(self, name: &str, expr: Expr) -> Result { let window_func_exprs = find_window_exprs(&[expr.clone()]); - let (plan, mut col_exists, window_func) = if window_func_exprs.is_empty() { - (self.plan, false, false) + let (window_fn_str, plan) = if window_func_exprs.is_empty() { + (None, self.plan) } else { ( + Some(window_func_exprs[0].to_string()), LogicalPlanBuilder::window_plan(self.plan, window_func_exprs)?, - true, - true, ) }; + let mut col_exists = false; let new_column = expr.alias(name); let mut fields: Vec = plan .schema() .iter() - .map(|(qualifier, field)| { + .filter_map(|(qualifier, field)| { if field.name() == name { col_exists = true; - new_column.clone() - } else if window_func && qualifier.is_none() { - col(Column::from((qualifier, field))).alias(name) + Some(new_column.clone()) } else { - col(Column::from((qualifier, field))) + let e = col(Column::from((qualifier, field))); + window_fn_str + .as_ref() + .filter(|s| *s == &e.to_string()) + .is_none() + .then_some(e) } }) .collect(); @@ -2975,7 +2978,8 @@ mod tests { Ok(()) } - // Test issue: https://github.com/apache/datafusion/issues/11982 + // Test issues: https://github.com/apache/datafusion/issues/11982 + // and https://github.com/apache/datafusion/issues/12425 // Window function was creating unwanted projection when using with_column() method. #[tokio::test] async fn test_window_function_with_column() -> Result<()> { @@ -2984,19 +2988,24 @@ mod tests { let df_impl = DataFrame::new(ctx.state(), df.plan.clone()); let func = row_number().alias("row_num"); - // Should create an additional column with alias 'r' that has window func results + // This first `with_column` results in a column without a `qualifier` + let df_impl = df_impl.with_column("s", col("c2") + col("c3"))?; + + // This second `with_column` should only alias `func` as `"r"` let df = df_impl.with_column("r", func)?.limit(0, Some(2))?; - assert_eq!(4, df.schema().fields().len()); + + df.clone().show().await?; + assert_eq!(5, df.schema().fields().len()); let df_results = df.clone().collect().await?; assert_batches_sorted_eq!( [ - "+----+----+-----+---+", - "| c1 | c2 | c3 | r |", - "+----+----+-----+---+", - "| c | 2 | 1 | 1 |", - "| d | 5 | -40 | 2 |", - "+----+----+-----+---+", + "+----+----+-----+-----+---+", + "| c1 | c2 | c3 | s | r |", + "+----+----+-----+-----+---+", + "| c | 2 | 1 | 3 | 1 |", + "| d | 5 | -40 | -35 | 2 |", + "+----+----+-----+-----+---+", ], &df_results ); From 9025c1ce2d349655836e44391cdffa53de9f3ff7 Mon Sep 17 00:00:00 2001 From: wiedld Date: Wed, 11 Sep 2024 09:56:01 -1000 Subject: [PATCH 147/154] Fix incorrect OFFSET during LIMIT pushdown. (#12399) * test: demonstrate offset not applied correctly with limit pushdown on multiple input streams * fix: do not pushdown when skip is applied * test: update tests after fix * chore: more doc cleanup * chore: move LIMIT+OFFSET tests to proper sqllogic test case * refactor: add global limit back (if there is a skip) during limit pushdown * Apply suggestions from code review * Add comment explaining why --------- Co-authored-by: Andrew Lamb --- .../physical-optimizer/src/limit_pushdown.rs | 18 ++- datafusion/sqllogictest/test_files/limit.slt | 146 ++++++++++++++++++ 2 files changed, 162 insertions(+), 2 deletions(-) diff --git a/datafusion/physical-optimizer/src/limit_pushdown.rs b/datafusion/physical-optimizer/src/limit_pushdown.rs index 7f45292f9e27..15d210e1b10b 100644 --- a/datafusion/physical-optimizer/src/limit_pushdown.rs +++ b/datafusion/physical-optimizer/src/limit_pushdown.rs @@ -126,10 +126,14 @@ impl From for Arc { /// The helper takes an `ExecutionPlan` and a global (algorithm) state which is /// an instance of `GlobalRequirements` and modifies these parameters while /// checking if the limits can be pushed down or not. +/// +/// If a limit is encountered, a [`TreeNodeRecursion::Stop`] is returned. Otherwise, +/// return a [`TreeNodeRecursion::Continue`]. pub fn pushdown_limit_helper( mut pushdown_plan: Arc, mut global_state: GlobalRequirements, ) -> Result<(Transformed>, GlobalRequirements)> { + // Extract limit, if exist, and return child inputs. if let Some(limit_exec) = extract_limit(&pushdown_plan) { // If we have fetch/skip info in the global state already, we need to // decide which one to continue with: @@ -199,10 +203,17 @@ pub fn pushdown_limit_helper( // This plan is combining input partitions, so we need to add the // fetch info to plan if possible. If not, we must add a `LimitExec` // with the information from the global state. + let mut new_plan = plan_with_fetch; + // Execution plans can't (yet) handle skip, so if we have one, + // we still need to add a global limit + if global_state.skip > 0 { + new_plan = + add_global_limit(new_plan, global_state.skip, global_state.fetch); + } global_state.fetch = skip_and_fetch; global_state.skip = 0; global_state.satisfied = true; - Ok((Transformed::yes(plan_with_fetch), global_state)) + Ok((Transformed::yes(new_plan), global_state)) } else if global_state.satisfied { // If the plan is already satisfied, do not add a limit: Ok((Transformed::no(pushdown_plan), global_state)) @@ -256,13 +267,17 @@ pub(crate) fn pushdown_limits( pushdown_plan: Arc, global_state: GlobalRequirements, ) -> Result> { + // Call pushdown_limit_helper. + // This will either extract the limit node (returning the child), or apply the limit pushdown. let (mut new_node, mut global_state) = pushdown_limit_helper(pushdown_plan, global_state)?; + // While limits exist, continue combining the global_state. while new_node.tnr == TreeNodeRecursion::Stop { (new_node, global_state) = pushdown_limit_helper(new_node.data, global_state)?; } + // Apply pushdown limits in children let children = new_node.data.children(); let new_children = children .into_iter() @@ -270,7 +285,6 @@ pub(crate) fn pushdown_limits( pushdown_limits(Arc::::clone(child), global_state.clone()) }) .collect::>()?; - new_node.data.with_new_children(new_children) } diff --git a/datafusion/sqllogictest/test_files/limit.slt b/datafusion/sqllogictest/test_files/limit.slt index 17bd398bd229..5b98392f1aa0 100644 --- a/datafusion/sqllogictest/test_files/limit.slt +++ b/datafusion/sqllogictest/test_files/limit.slt @@ -565,3 +565,149 @@ physical_plan statement ok drop table data; + + +#################### +# Test issue: limit pushdown with offsets +# Ensure the offset is not lost: https://github.com/apache/datafusion/issues/12423 +#################### + +statement ok +CREATE EXTERNAL TABLE ordered_table ( + a0 INT, + a INT, + b INT, + c INT UNSIGNED, + d INT +) +STORED AS CSV +WITH ORDER (c ASC) +LOCATION '../core/tests/data/window_2.csv' +OPTIONS ('format.has_header' 'true'); + +# all results +query II +SELECT b, sum(a) FROM ordered_table GROUP BY b order by b desc; +---- +3 25 +2 25 +1 0 +0 0 + +# limit only +query II +SELECT b, sum(a) FROM ordered_table GROUP BY b order by b desc LIMIT 3; +---- +3 25 +2 25 +1 0 + +# offset only +query II +SELECT b, sum(a) FROM ordered_table GROUP BY b order by b desc OFFSET 1; +---- +2 25 +1 0 +0 0 + +# offset + limit +query II +SELECT b, sum(a) FROM ordered_table GROUP BY b order by b desc OFFSET 1 LIMIT 2; +---- +2 25 +1 0 + +# Applying offset & limit when multiple streams from groupby +# the plan must still have a global limit to apply the offset +query TT +EXPLAIN SELECT b, sum(a) FROM ordered_table GROUP BY b order by b desc OFFSET 1 LIMIT 2; +---- +logical_plan +01)Limit: skip=1, fetch=2 +02)--Sort: ordered_table.b DESC NULLS FIRST, fetch=3 +03)----Aggregate: groupBy=[[ordered_table.b]], aggr=[[sum(CAST(ordered_table.a AS Int64))]] +04)------TableScan: ordered_table projection=[a, b] +physical_plan +01)GlobalLimitExec: skip=1, fetch=2 +02)--SortPreservingMergeExec: [b@0 DESC], fetch=3 +03)----SortExec: TopK(fetch=3), expr=[b@0 DESC], preserve_partitioning=[true] +04)------AggregateExec: mode=FinalPartitioned, gby=[b@0 as b], aggr=[sum(ordered_table.a)] +05)--------CoalesceBatchesExec: target_batch_size=8192 +06)----------RepartitionExec: partitioning=Hash([b@0], 4), input_partitions=4 +07)------------AggregateExec: mode=Partial, gby=[b@1 as b], aggr=[sum(ordered_table.a)] +08)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +09)----------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b], has_header=true + +# Applying offset & limit when multiple streams from union +# the plan must still have a global limit to apply the offset +query TT +explain select * FROM ( + select c FROM ordered_table + UNION ALL + select d FROM ordered_table +) order by 1 desc LIMIT 10 OFFSET 4; +---- +logical_plan +01)Limit: skip=4, fetch=10 +02)--Sort: ordered_table.c DESC NULLS FIRST, fetch=14 +03)----Union +04)------Projection: CAST(ordered_table.c AS Int64) AS c +05)--------TableScan: ordered_table projection=[c] +06)------Projection: CAST(ordered_table.d AS Int64) AS c +07)--------TableScan: ordered_table projection=[d] +physical_plan +01)GlobalLimitExec: skip=4, fetch=10 +02)--SortPreservingMergeExec: [c@0 DESC], fetch=14 +03)----UnionExec +04)------SortExec: TopK(fetch=14), expr=[c@0 DESC], preserve_partitioning=[true] +05)--------ProjectionExec: expr=[CAST(c@0 AS Int64) as c] +06)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +07)------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c], output_ordering=[c@0 ASC NULLS LAST], has_header=true +08)------SortExec: TopK(fetch=14), expr=[c@0 DESC], preserve_partitioning=[true] +09)--------ProjectionExec: expr=[CAST(d@0 AS Int64) as c] +10)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +11)------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[d], has_header=true + +# Applying LIMIT & OFFSET to subquery. +query III +select t1.b, c, c2 FROM ( + select b, c FROM ordered_table ORDER BY b desc, c desc OFFSET 1 LIMIT 4 +) as t1 INNER JOIN ( + select b, c as c2 FROM ordered_table ORDER BY b desc, d desc OFFSET 1 LIMIT 4 +) as t2 +ON t1.b = t2.b +ORDER BY t1.b desc, c desc, c2 desc; +---- +3 98 96 +3 98 89 +3 98 82 +3 98 79 +3 97 96 +3 97 89 +3 97 82 +3 97 79 +3 96 96 +3 96 89 +3 96 82 +3 96 79 +3 95 96 +3 95 89 +3 95 82 +3 95 79 + +# Apply OFFSET & LIMIT to both parent and child (subquery). +query III +select t1.b, c, c2 FROM ( + select b, c FROM ordered_table ORDER BY b desc, c desc OFFSET 1 LIMIT 4 +) as t1 INNER JOIN ( + select b, c as c2 FROM ordered_table ORDER BY b desc, d desc OFFSET 1 LIMIT 4 +) as t2 +ON t1.b = t2.b +ORDER BY t1.b desc, c desc, c2 desc +OFFSET 3 LIMIT 2; +---- +3 99 82 +3 99 79 + +statement ok +drop table ordered_table; From b9dabdbd5e122dbbf4af7a8b2264a3d457670615 Mon Sep 17 00:00:00 2001 From: Oleks V Date: Wed, 11 Sep 2024 16:31:56 -0700 Subject: [PATCH 148/154] doc: Update MSRV policy, shortening to max(4 months, 4 releases) (#12402) * doc: Updte MSRV policy to not depend on Rust compiler cadences --- README.md | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/README.md b/README.md index 8c76f0807b2d..816dc77714d2 100644 --- a/README.md +++ b/README.md @@ -98,9 +98,11 @@ Optional features: ## Rust Version Compatibility Policy -DataFusion's Minimum Required Stable Rust Version (MSRV) policy is to support -each stable Rust version for 6 months after it is -[released](https://github.com/rust-lang/rust/blob/master/RELEASES.md). This -generally translates to support for the most recent 3 to 4 stable Rust versions. +DataFusion's Minimum Required Stable Rust Version (MSRV) policy is to support stable [4 latest +Rust versions](https://releases.rs) OR the stable minor Rust version as of 4 months, whichever is lower. + +For example, given the releases `1.78.0`, `1.79.0`, `1.80.0`, `1.80.1` and `1.81.0` DataFusion will support 1.78.0, which is 3 minor versions prior to the most minor recent `1.81`. + +If a hotfix is released for the minimum supported Rust version (MSRV), the MSRV will be the minor version with all hotfixes, even if it surpasses the four-month window. We enforce this policy using a [MSRV CI Check](https://github.com/search?q=repo%3Aapache%2Fdatafusion+rust-version+language%3ATOML+path%3A%2F%5ECargo.toml%2F&type=code) From f7efd2d31adb51a67dc6bfb6d6eae6a525d60482 Mon Sep 17 00:00:00 2001 From: JasonLi Date: Thu, 12 Sep 2024 10:31:15 +0800 Subject: [PATCH 149/154] fix: upper case qualifier wildcard bug (#12426) --- datafusion/sql/src/expr/mod.rs | 4 +-- datafusion/sql/src/select.rs | 6 ++--- .../sqllogictest/test_files/wildcard.slt | 25 +++++++++++++++++++ 3 files changed, 29 insertions(+), 6 deletions(-) diff --git a/datafusion/sql/src/expr/mod.rs b/datafusion/sql/src/expr/mod.rs index 6e975d8557dc..c79c6358be36 100644 --- a/datafusion/sql/src/expr/mod.rs +++ b/datafusion/sql/src/expr/mod.rs @@ -27,7 +27,7 @@ use sqlparser::ast::{ use datafusion_common::{ internal_datafusion_err, internal_err, not_impl_err, plan_err, DFSchema, Result, - ScalarValue, TableReference, + ScalarValue, }; use datafusion_expr::expr::ScalarFunction; use datafusion_expr::expr::{InList, WildcardOptions}; @@ -562,7 +562,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { options: WildcardOptions::default(), }), SQLExpr::QualifiedWildcard(object_name) => Ok(Expr::Wildcard { - qualifier: Some(TableReference::from(object_name.to_string())), + qualifier: Some(self.object_name_to_table_reference(object_name)?), options: WildcardOptions::default(), }), SQLExpr::Tuple(values) => self.parse_tuple(schema, planner_context, values), diff --git a/datafusion/sql/src/select.rs b/datafusion/sql/src/select.rs index 8a26671fcb6c..500932b46809 100644 --- a/datafusion/sql/src/select.rs +++ b/datafusion/sql/src/select.rs @@ -18,9 +18,7 @@ use std::collections::HashSet; use std::sync::Arc; -use crate::planner::{ - idents_to_table_reference, ContextProvider, PlannerContext, SqlToRel, -}; +use crate::planner::{ContextProvider, PlannerContext, SqlToRel}; use crate::utils::{ check_columns_satisfy_exprs, extract_aliases, rebase_expr, resolve_aliases_to_exprs, resolve_columns, resolve_positions_to_exprs, transform_bottom_unnests, @@ -590,7 +588,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { } SelectItem::QualifiedWildcard(object_name, options) => { Self::check_wildcard_options(&options)?; - let qualifier = idents_to_table_reference(object_name.0, false)?; + let qualifier = self.object_name_to_table_reference(object_name)?; let planned_options = self.plan_wildcard_options( plan, empty_from, diff --git a/datafusion/sqllogictest/test_files/wildcard.slt b/datafusion/sqllogictest/test_files/wildcard.slt index 9285bdbf2306..7c076f040feb 100644 --- a/datafusion/sqllogictest/test_files/wildcard.slt +++ b/datafusion/sqllogictest/test_files/wildcard.slt @@ -108,6 +108,31 @@ SELECT t1.*, tb2.* FROM t1 JOIN t2 tb2 ON t2_id = t1_id ORDER BY t1_id statement error Error during planning: Invalid qualifier agg SELECT agg.* FROM aggregate_simple ORDER BY c1 +# select_upper_case_qualified_wildcard +query ITI +SELECT PUBLIC.t1.* FROM PUBLIC.t1 +---- +11 a 1 +22 b 2 +33 c 3 +44 d 4 + +query ITI +SELECT PUBLIC.t1.* FROM public.t1 +---- +11 a 1 +22 b 2 +33 c 3 +44 d 4 + +query ITI +SELECT public.t1.* FROM PUBLIC.t1 +---- +11 a 1 +22 b 2 +33 c 3 +44 d 4 + ######## # Clean up after the test ######## From 5b6b404b9fbccb8584e8c5d2f1c366f3de736f31 Mon Sep 17 00:00:00 2001 From: Samuel Colvin Date: Thu, 12 Sep 2024 11:29:16 +0100 Subject: [PATCH 150/154] Add a `version()` UDF (#12429) * Add a "version()" UDF * add "Apache" * fix rustdoc * add scalar_functions.md entry --- datafusion/functions/src/core/mod.rs | 3 + datafusion/functions/src/core/version.rs | 99 +++++++++++++++++++ .../source/user-guide/sql/scalar_functions.md | 20 ++++ 3 files changed, 122 insertions(+) create mode 100644 datafusion/functions/src/core/version.rs diff --git a/datafusion/functions/src/core/mod.rs b/datafusion/functions/src/core/mod.rs index af340930eabc..1c69f9c9b2f3 100644 --- a/datafusion/functions/src/core/mod.rs +++ b/datafusion/functions/src/core/mod.rs @@ -31,6 +31,7 @@ pub mod nvl; pub mod nvl2; pub mod planner; pub mod r#struct; +pub mod version; // create UDFs make_udf_function!(arrow_cast::ArrowCastFunc, ARROW_CAST, arrow_cast); @@ -42,6 +43,7 @@ make_udf_function!(r#struct::StructFunc, STRUCT, r#struct); make_udf_function!(named_struct::NamedStructFunc, NAMED_STRUCT, named_struct); make_udf_function!(getfield::GetFieldFunc, GET_FIELD, get_field); make_udf_function!(coalesce::CoalesceFunc, COALESCE, coalesce); +make_udf_function!(version::VersionFunc, VERSION, version); pub mod expr_fn { use datafusion_expr::{Expr, Literal}; @@ -104,5 +106,6 @@ pub fn functions() -> Vec> { // calls to `get_field` get_field(), coalesce(), + version(), ] } diff --git a/datafusion/functions/src/core/version.rs b/datafusion/functions/src/core/version.rs new file mode 100644 index 000000000000..212349e68981 --- /dev/null +++ b/datafusion/functions/src/core/version.rs @@ -0,0 +1,99 @@ +// 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. + +//! [`VersionFunc`]: Implementation of the `version` function. + +use std::any::Any; + +use arrow::datatypes::DataType; +use datafusion_common::{not_impl_err, plan_err, Result, ScalarValue}; +use datafusion_expr::{ColumnarValue, ScalarUDFImpl, Signature, Volatility}; + +#[derive(Debug)] +pub struct VersionFunc { + signature: Signature, +} + +impl Default for VersionFunc { + fn default() -> Self { + Self::new() + } +} + +impl VersionFunc { + pub fn new() -> Self { + Self { + signature: Signature::exact(vec![], Volatility::Immutable), + } + } +} + +impl ScalarUDFImpl for VersionFunc { + fn as_any(&self) -> &dyn Any { + self + } + + fn name(&self) -> &str { + "version" + } + + fn signature(&self) -> &Signature { + &self.signature + } + + fn return_type(&self, args: &[DataType]) -> Result { + if args.is_empty() { + Ok(DataType::Utf8) + } else { + plan_err!("version expects no arguments") + } + } + + fn invoke(&self, _: &[ColumnarValue]) -> Result { + not_impl_err!("version does not take any arguments") + } + + fn invoke_no_args(&self, _: usize) -> Result { + // TODO it would be great to add rust version and arrow version, + // but that requires a `build.rs` script and/or adding a version const to arrow-rs + let version = format!( + "Apache DataFusion {}, {} on {}", + env!("CARGO_PKG_VERSION"), + std::env::consts::ARCH, + std::env::consts::OS, + ); + Ok(ColumnarValue::Scalar(ScalarValue::Utf8(Some(version)))) + } +} + +#[cfg(test)] +mod test { + use super::*; + use datafusion_expr::ScalarUDF; + + #[tokio::test] + async fn test_version_udf() { + let version_udf = ScalarUDF::from(VersionFunc::new()); + let version = version_udf.invoke_no_args(0).unwrap(); + + if let ColumnarValue::Scalar(ScalarValue::Utf8(Some(version))) = version { + assert!(version.starts_with("Apache DataFusion")); + } else { + panic!("Expected version string"); + } + } +} diff --git a/docs/source/user-guide/sql/scalar_functions.md b/docs/source/user-guide/sql/scalar_functions.md index e08524dcd3a7..14852935dca4 100644 --- a/docs/source/user-guide/sql/scalar_functions.md +++ b/docs/source/user-guide/sql/scalar_functions.md @@ -3917,6 +3917,7 @@ sha512(expression) - [arrow_cast](#arrow_cast) - [arrow_typeof](#arrow_typeof) +- [version](#version) ### `arrow_cast` @@ -3975,3 +3976,22 @@ arrow_typeof(expression) +---------------------------+------------------------+ 1 row in set. Query took 0.001 seconds. ``` + +### `version` + +Returns the version of DataFusion. + +``` +version() +``` + +#### Example + +``` +> select version(); ++--------------------------------------------+ +| version() | ++--------------------------------------------+ +| Apache DataFusion 41.0.0, aarch64 on macos | ++--------------------------------------------+ +``` From 1f06308037426c46d1a9c4c2d3a8f335c4ad26d1 Mon Sep 17 00:00:00 2001 From: Bruce Ritchie Date: Thu, 12 Sep 2024 11:23:59 -0400 Subject: [PATCH 151/154] Support timestamps and steps of less than a day for range/generate_series (#12400) * Support timestamps and steps of less than a day for timestamps. * Updated docs for range and generate_series to add additional info wrt timestamp support. * Updates based on code review. * Cleanup error message Co-authored-by: Andrew Lamb --------- Co-authored-by: Andrew Lamb --- datafusion/functions-nested/src/range.rs | 179 ++++++++++++++++-- datafusion/sqllogictest/test_files/array.slt | 156 ++++++++++++++- .../source/user-guide/sql/scalar_functions.md | 17 +- 3 files changed, 322 insertions(+), 30 deletions(-) diff --git a/datafusion/functions-nested/src/range.rs b/datafusion/functions-nested/src/range.rs index 90cf8bcbd057..b3d8010cb668 100644 --- a/datafusion/functions-nested/src/range.rs +++ b/datafusion/functions-nested/src/range.rs @@ -18,19 +18,31 @@ //! [`ScalarUDFImpl`] definitions for range and gen_series functions. use crate::utils::make_scalar_function; -use arrow::array::{Array, ArrayRef, Date32Builder, Int64Array, ListArray, ListBuilder}; +use arrow::array::{Array, ArrayRef, Int64Array, ListArray, ListBuilder}; use arrow::datatypes::{DataType, Field}; -use arrow_array::types::{Date32Type, IntervalMonthDayNanoType}; -use arrow_array::NullArray; +use arrow_array::builder::{Date32Builder, TimestampNanosecondBuilder}; +use arrow_array::temporal_conversions::as_datetime_with_timezone; +use arrow_array::timezone::Tz; +use arrow_array::types::{ + Date32Type, IntervalMonthDayNanoType, TimestampNanosecondType as TSNT, +}; +use arrow_array::{NullArray, TimestampNanosecondArray}; use arrow_buffer::{BooleanBufferBuilder, NullBuffer, OffsetBuffer}; use arrow_schema::DataType::*; use arrow_schema::IntervalUnit::MonthDayNano; -use datafusion_common::cast::{as_date32_array, as_int64_array, as_interval_mdn_array}; -use datafusion_common::{exec_err, not_impl_datafusion_err, Result}; +use arrow_schema::TimeUnit::Nanosecond; +use datafusion_common::cast::{ + as_date32_array, as_int64_array, as_interval_mdn_array, as_timestamp_nanosecond_array, +}; +use datafusion_common::{ + exec_datafusion_err, exec_err, internal_err, not_impl_datafusion_err, Result, +}; use datafusion_expr::{ColumnarValue, ScalarUDFImpl, Signature, Volatility}; use itertools::Itertools; use std::any::Any; +use std::cmp::Ordering; use std::iter::from_fn; +use std::str::FromStr; use std::sync::Arc; make_udf_expr_and_func!( @@ -78,7 +90,7 @@ impl ScalarUDFImpl for Range { UInt16 => Ok(Int64), UInt32 => Ok(Int64), UInt64 => Ok(Int64), - Timestamp(_, _) => Ok(Date32), + Timestamp(_, tz) => Ok(Timestamp(Nanosecond, tz.clone())), Date32 => Ok(Date32), Date64 => Ok(Date32), Utf8 => Ok(Date32), @@ -109,8 +121,11 @@ impl ScalarUDFImpl for Range { match args[0].data_type() { Int64 => make_scalar_function(|args| gen_range_inner(args, false))(args), Date32 => make_scalar_function(|args| gen_range_date(args, false))(args), - _ => { - exec_err!("unsupported type for range") + Timestamp(_, _) => { + make_scalar_function(|args| gen_range_timestamp(args, false))(args) + } + dt => { + exec_err!("unsupported type for RANGE. Expected Int64, Date32 or Timestamp, got: {dt}") } } } @@ -152,8 +167,8 @@ impl ScalarUDFImpl for GenSeries { &self.signature } - fn coerce_types(&self, _arg_types: &[DataType]) -> Result> { - _arg_types + fn coerce_types(&self, arg_types: &[DataType]) -> Result> { + arg_types .iter() .map(|arg_type| match arg_type { Null => Ok(Null), @@ -165,7 +180,7 @@ impl ScalarUDFImpl for GenSeries { UInt16 => Ok(Int64), UInt32 => Ok(Int64), UInt64 => Ok(Int64), - Timestamp(_, _) => Ok(Date32), + Timestamp(_, tz) => Ok(Timestamp(Nanosecond, tz.clone())), Date32 => Ok(Date32), Date64 => Ok(Date32), Utf8 => Ok(Date32), @@ -196,9 +211,12 @@ impl ScalarUDFImpl for GenSeries { match args[0].data_type() { Int64 => make_scalar_function(|args| gen_range_inner(args, true))(args), Date32 => make_scalar_function(|args| gen_range_date(args, true))(args), + Timestamp(_, _) => { + make_scalar_function(|args| gen_range_timestamp(args, true))(args) + } dt => { exec_err!( - "unsupported type for gen_series. Expected Int64 or Date32, got: {}", + "unsupported type for GENERATE_SERIES. Expected Int64, Date32 or Timestamp, got: {}", dt ) } @@ -334,7 +352,7 @@ fn gen_range_iter( } } -fn gen_range_date(args: &[ArrayRef], include_upper: bool) -> Result { +fn gen_range_date(args: &[ArrayRef], include_upper_bound: bool) -> Result { if args.len() != 3 { return exec_err!("arguments length does not match"); } @@ -372,7 +390,7 @@ fn gen_range_date(args: &[ArrayRef], include_upper: bool) -> Result { } let neg = months < 0 || days < 0; - if !include_upper { + if !include_upper_bound { stop = Date32Type::subtract_month_day_nano(stop, step); } let mut new_date = start; @@ -394,3 +412,136 @@ fn gen_range_date(args: &[ArrayRef], include_upper: bool) -> Result { Ok(arr) } + +fn gen_range_timestamp(args: &[ArrayRef], include_upper_bound: bool) -> Result { + if args.len() != 3 { + return exec_err!( + "Arguments length must be 3 for {}", + if include_upper_bound { + "GENERATE_SERIES" + } else { + "RANGE" + } + ); + } + + // coerce_types fn should coerce all types to Timestamp(Nanosecond, tz) + let (start_arr, start_tz_opt) = cast_timestamp_arg(&args[0], include_upper_bound)?; + let (stop_arr, stop_tz_opt) = cast_timestamp_arg(&args[1], include_upper_bound)?; + let step_arr = as_interval_mdn_array(&args[2])?; + let start_tz = parse_tz(start_tz_opt)?; + let stop_tz = parse_tz(stop_tz_opt)?; + + // values are timestamps + let values_builder = start_tz_opt + .clone() + .map_or_else(TimestampNanosecondBuilder::new, |start_tz_str| { + TimestampNanosecondBuilder::new().with_timezone(start_tz_str) + }); + let mut list_builder = ListBuilder::new(values_builder); + + for idx in 0..start_arr.len() { + if start_arr.is_null(idx) || stop_arr.is_null(idx) || step_arr.is_null(idx) { + list_builder.append_null(); + continue; + } + + let start = start_arr.value(idx); + let stop = stop_arr.value(idx); + let step = step_arr.value(idx); + + let (months, days, ns) = IntervalMonthDayNanoType::to_parts(step); + if months == 0 && days == 0 && ns == 0 { + return exec_err!( + "Interval argument to {} must not be 0", + if include_upper_bound { + "GENERATE_SERIES" + } else { + "RANGE" + } + ); + } + + let neg = TSNT::add_month_day_nano(start, step, start_tz) + .ok_or(exec_datafusion_err!( + "Cannot generate timestamp range where start + step overflows" + ))? + .cmp(&start) + == Ordering::Less; + + let stop_dt = as_datetime_with_timezone::(stop, stop_tz).ok_or( + exec_datafusion_err!( + "Cannot generate timestamp for stop: {}: {:?}", + stop, + stop_tz + ), + )?; + + let mut current = start; + let mut current_dt = as_datetime_with_timezone::(current, start_tz).ok_or( + exec_datafusion_err!( + "Cannot generate timestamp for start: {}: {:?}", + current, + start_tz + ), + )?; + + let values = from_fn(|| { + if (include_upper_bound + && ((neg && current_dt < stop_dt) || (!neg && current_dt > stop_dt))) + || (!include_upper_bound + && ((neg && current_dt <= stop_dt) + || (!neg && current_dt >= stop_dt))) + { + return None; + } + + let prev_current = current; + + if let Some(ts) = TSNT::add_month_day_nano(current, step, start_tz) { + current = ts; + current_dt = as_datetime_with_timezone::(current, start_tz)?; + + Some(Some(prev_current)) + } else { + // we failed to parse the timestamp here so terminate the series + None + } + }); + + list_builder.append_value(values); + } + + let arr = Arc::new(list_builder.finish()); + + Ok(arr) +} + +fn cast_timestamp_arg( + arg: &ArrayRef, + include_upper: bool, +) -> Result<(&TimestampNanosecondArray, &Option>)> { + match arg.data_type() { + Timestamp(Nanosecond, tz_opt) => { + Ok((as_timestamp_nanosecond_array(arg)?, tz_opt)) + } + _ => { + internal_err!( + "Unexpected argument type for {} : {}", + if include_upper { + "GENERATE_SERIES" + } else { + "RANGE" + }, + arg.data_type() + ) + } + } +} + +fn parse_tz(tz: &Option>) -> Result { + let tz = tz.as_ref().map_or_else(|| "+00", |s| s); + + Tz::from_str(tz) + .map_err(|op| exec_datafusion_err!("failed to parse timezone {tz}: {:?}", op)) +} diff --git a/datafusion/sqllogictest/test_files/array.slt b/datafusion/sqllogictest/test_files/array.slt index b20b59339b64..f92bcfdf3421 100644 --- a/datafusion/sqllogictest/test_files/array.slt +++ b/datafusion/sqllogictest/test_files/array.slt @@ -5881,34 +5881,70 @@ select range(DATE '1992-09-01', NULL, INTERVAL '1' YEAR); ---- NULL +## should return NULL +query ? +select range(TIMESTAMP '1992-09-01', NULL, INTERVAL '1' YEAR); +---- +NULL + query ? select range(DATE '1992-09-01', DATE '1993-03-01', NULL); ---- NULL +query ? +select range(TIMESTAMP '1992-09-01', TIMESTAMP '1993-03-01', NULL); +---- +NULL + query ? select range(NULL, DATE '1993-03-01', INTERVAL '1' YEAR); ---- NULL +query ? +select range(NULL, TIMESTAMP '1993-03-01', INTERVAL '1' YEAR); +---- +NULL + query ? select range(NULL, NULL, NULL); ---- NULL +query ? +select range(NULL::timestamp, NULL::timestamp, NULL); +---- +NULL + query ? select range(DATE '1989-04-01', DATE '1993-03-01', INTERVAL '-1' YEAR) ---- [] +query ? +select range(TIMESTAMP '1989-04-01', TIMESTAMP '1993-03-01', INTERVAL '-1' YEAR) +---- +[] + query ? select range(DATE '1993-03-01', DATE '1989-04-01', INTERVAL '1' YEAR) ---- [] +query ? +select range(TIMESTAMP '1993-03-01', TIMESTAMP '1989-04-01', INTERVAL '1' YEAR) +---- +[] + query error DataFusion error: Execution error: Cannot generate date range less than 1 day\. select range(DATE '1993-03-01', DATE '1993-03-01', INTERVAL '1' HOUR) +query ? +select range(TIMESTAMP '1993-03-01', TIMESTAMP '1993-03-01', INTERVAL '1' HOUR) +---- +[] + query ????????? select generate_series(5), generate_series(2, 5), @@ -5923,8 +5959,31 @@ select generate_series(5), ---- [0, 1, 2, 3, 4, 5] [2, 3, 4, 5] [2, 5, 8] [1, 2, 3, 4, 5] [5, 4, 3, 2, 1] [10, 7, 4] [1992-09-01, 1992-10-01, 1992-11-01, 1992-12-01, 1993-01-01, 1993-02-01, 1993-03-01] [1993-02-01, 1993-01-31, 1993-01-30, 1993-01-29, 1993-01-28, 1993-01-27, 1993-01-26, 1993-01-25, 1993-01-24, 1993-01-23, 1993-01-22, 1993-01-21, 1993-01-20, 1993-01-19, 1993-01-18, 1993-01-17, 1993-01-16, 1993-01-15, 1993-01-14, 1993-01-13, 1993-01-12, 1993-01-11, 1993-01-10, 1993-01-09, 1993-01-08, 1993-01-07, 1993-01-06, 1993-01-05, 1993-01-04, 1993-01-03, 1993-01-02, 1993-01-01] [1989-04-01, 1990-04-01, 1991-04-01, 1992-04-01] -query error DataFusion error: Execution error: Cannot generate date range less than 1 day\. -select generate_series('2021-01-01'::timestamp, '2021-01-02'::timestamp, INTERVAL '1' HOUR); +query ? +select generate_series('2021-01-01'::timestamp, '2021-01-01T15:00:00'::timestamp, INTERVAL '1' HOUR); +---- +[2021-01-01T00:00:00, 2021-01-01T01:00:00, 2021-01-01T02:00:00, 2021-01-01T03:00:00, 2021-01-01T04:00:00, 2021-01-01T05:00:00, 2021-01-01T06:00:00, 2021-01-01T07:00:00, 2021-01-01T08:00:00, 2021-01-01T09:00:00, 2021-01-01T10:00:00, 2021-01-01T11:00:00, 2021-01-01T12:00:00, 2021-01-01T13:00:00, 2021-01-01T14:00:00, 2021-01-01T15:00:00] + +query ? +select generate_series('2021-01-01T00:00:00EST'::timestamp, '2021-01-01T15:00:00-12:00'::timestamp, INTERVAL '1' HOUR); +---- +[2021-01-01T05:00:00, 2021-01-01T06:00:00, 2021-01-01T07:00:00, 2021-01-01T08:00:00, 2021-01-01T09:00:00, 2021-01-01T10:00:00, 2021-01-01T11:00:00, 2021-01-01T12:00:00, 2021-01-01T13:00:00, 2021-01-01T14:00:00, 2021-01-01T15:00:00, 2021-01-01T16:00:00, 2021-01-01T17:00:00, 2021-01-01T18:00:00, 2021-01-01T19:00:00, 2021-01-01T20:00:00, 2021-01-01T21:00:00, 2021-01-01T22:00:00, 2021-01-01T23:00:00, 2021-01-02T00:00:00, 2021-01-02T01:00:00, 2021-01-02T02:00:00, 2021-01-02T03:00:00] + +query ? +select generate_series(arrow_cast('2021-01-01T00:00:00', 'Timestamp(Nanosecond, Some("-05:00"))'), arrow_cast('2021-01-01T15:00:00', 'Timestamp(Nanosecond, Some("+05:00"))'), INTERVAL '1' HOUR); +---- +[2021-01-01T00:00:00-05:00, 2021-01-01T01:00:00-05:00, 2021-01-01T02:00:00-05:00, 2021-01-01T03:00:00-05:00, 2021-01-01T04:00:00-05:00, 2021-01-01T05:00:00-05:00] + +## -5500000000 ns is -5.5 sec +query ? +select generate_series(arrow_cast('2021-01-01T00:00:00', 'Timestamp(Nanosecond, Some("-05:00"))'), arrow_cast('2021-01-01T06:00:00', 'Timestamp(Nanosecond, Some("-05:00"))'), INTERVAL '1 HOUR 30 MINUTE -5500000000 NANOSECOND'); +---- +[2021-01-01T00:00:00-05:00, 2021-01-01T01:29:54.500-05:00, 2021-01-01T02:59:49-05:00, 2021-01-01T04:29:43.500-05:00, 2021-01-01T05:59:38-05:00] + +## mixing types for timestamps is not supported +query error DataFusion error: Internal error: Unexpected argument type for GENERATE_SERIES : Date32 +select generate_series(arrow_cast('2021-01-01T00:00:00', 'Timestamp(Nanosecond, Some("-05:00"))'), DATE '2021-01-02', INTERVAL '1' HOUR); + ## should return NULL query ? @@ -5932,41 +5991,92 @@ select generate_series(DATE '1992-09-01', NULL, INTERVAL '1' YEAR); ---- NULL +## should return NULL +query ? +select generate_series(TIMESTAMP '1992-09-01', NULL, INTERVAL '1' YEAR); +---- +NULL + query ? select generate_series(DATE '1992-09-01', DATE '1993-03-01', NULL); ---- NULL +query ? +select generate_series(TIMESTAMP '1992-09-01', DATE '1993-03-01', NULL); +---- +NULL + query ? select generate_series(NULL, DATE '1993-03-01', INTERVAL '1' YEAR); ---- NULL +query ? +select generate_series(NULL, TIMESTAMP '1993-03-01', INTERVAL '1' YEAR); +---- +NULL + query ? select generate_series(NULL, NULL, NULL); ---- NULL +query ? +select generate_series(NULL::timestamp, NULL::timestamp, NULL); +---- +NULL + query ? select generate_series(DATE '1989-04-01', DATE '1993-03-01', INTERVAL '-1' YEAR) ---- [] +query ? +select generate_series(TIMESTAMP '1989-04-01', TIMESTAMP '1993-03-01', INTERVAL '-1' YEAR) +---- +[] + query ? select generate_series(DATE '1993-03-01', DATE '1989-04-01', INTERVAL '1' YEAR) ---- [] +query ? +select generate_series(TIMESTAMP '1993-03-01', TIMESTAMP '1989-04-01', INTERVAL '1' YEAR) +---- +[] + query error DataFusion error: Execution error: Cannot generate date range less than 1 day. select generate_series(DATE '2000-01-01', DATE '2000-01-03', INTERVAL '1' HOUR) query error DataFusion error: Execution error: Cannot generate date range less than 1 day. select generate_series(DATE '2000-01-01', DATE '2000-01-03', INTERVAL '-1' HOUR) +query ? +select generate_series(TIMESTAMP '2000-01-01', TIMESTAMP '2000-01-02', INTERVAL '1' HOUR) +---- +[2000-01-01T00:00:00, 2000-01-01T01:00:00, 2000-01-01T02:00:00, 2000-01-01T03:00:00, 2000-01-01T04:00:00, 2000-01-01T05:00:00, 2000-01-01T06:00:00, 2000-01-01T07:00:00, 2000-01-01T08:00:00, 2000-01-01T09:00:00, 2000-01-01T10:00:00, 2000-01-01T11:00:00, 2000-01-01T12:00:00, 2000-01-01T13:00:00, 2000-01-01T14:00:00, 2000-01-01T15:00:00, 2000-01-01T16:00:00, 2000-01-01T17:00:00, 2000-01-01T18:00:00, 2000-01-01T19:00:00, 2000-01-01T20:00:00, 2000-01-01T21:00:00, 2000-01-01T22:00:00, 2000-01-01T23:00:00, 2000-01-02T00:00:00] + +query ? +select generate_series(TIMESTAMP '2000-01-02', TIMESTAMP '2000-01-01', INTERVAL '-1' HOUR) +---- +[2000-01-02T00:00:00, 2000-01-01T23:00:00, 2000-01-01T22:00:00, 2000-01-01T21:00:00, 2000-01-01T20:00:00, 2000-01-01T19:00:00, 2000-01-01T18:00:00, 2000-01-01T17:00:00, 2000-01-01T16:00:00, 2000-01-01T15:00:00, 2000-01-01T14:00:00, 2000-01-01T13:00:00, 2000-01-01T12:00:00, 2000-01-01T11:00:00, 2000-01-01T10:00:00, 2000-01-01T09:00:00, 2000-01-01T08:00:00, 2000-01-01T07:00:00, 2000-01-01T06:00:00, 2000-01-01T05:00:00, 2000-01-01T04:00:00, 2000-01-01T03:00:00, 2000-01-01T02:00:00, 2000-01-01T01:00:00, 2000-01-01T00:00:00] + +# Test generate_series with small intervals +query ? +select generate_series('2000-01-01T00:00:00.000000001Z'::timestamp, '2000-01-01T00:00:00.00000001Z'::timestamp, INTERVAL '1' NANOSECONDS) +---- +[2000-01-01T00:00:00.000000001, 2000-01-01T00:00:00.000000002, 2000-01-01T00:00:00.000000003, 2000-01-01T00:00:00.000000004, 2000-01-01T00:00:00.000000005, 2000-01-01T00:00:00.000000006, 2000-01-01T00:00:00.000000007, 2000-01-01T00:00:00.000000008, 2000-01-01T00:00:00.000000009, 2000-01-01T00:00:00.000000010] + # Test generate_series with zero step query error DataFusion error: Execution error: step can't be 0 for function generate_series\(start \[, stop, step\]\) select generate_series(1, 1, 0); +# Test generate_series with zero step +query error DataFusion error: Execution error: Interval argument to GENERATE_SERIES must not be 0 +select generate_series(TIMESTAMP '2000-01-02', TIMESTAMP '2000-01-01', INTERVAL '0' MINUTE); + # Test generate_series with big steps query ???? select @@ -6021,7 +6131,7 @@ select generate_series(NULL) ---- NULL -# Test generate_series with a table of values +# Test generate_series with a table of date values statement ok CREATE TABLE date_table( start DATE, @@ -6053,15 +6163,45 @@ select generate_series(start, '1993-03-01'::date, INTERVAL '1 year') from date_t [1993-02-01] [1989-04-01, 1990-04-01, 1991-04-01, 1992-04-01] +# Test generate_series with a table of timestamp values +statement ok +CREATE TABLE timestamp_table( + start TIMESTAMP, + stop TIMESTAMP, + step INTERVAL +) AS VALUES + (TIMESTAMP '1992-01-01T00:00:00', TIMESTAMP '1993-01-02T00:00:00', INTERVAL '1' MONTH), + (TIMESTAMP '1993-02-01T00:00:00', TIMESTAMP '1993-01-01T00:00:00', INTERVAL '-1' DAY), + (TIMESTAMP '1989-04-01T00:00:00', TIMESTAMP '1993-03-01T00:00:00', INTERVAL '1' YEAR); -# https://github.com/apache/datafusion/issues/11922 query ? -select generate_series(start, '1993-03-01', INTERVAL '1 year') from date_table; +select generate_series(start, stop, step) from timestamp_table; ---- -[1992-01-01, 1993-01-01] -[1993-02-01] -[1989-04-01, 1990-04-01, 1991-04-01, 1992-04-01] +[1992-01-01T00:00:00, 1992-02-01T00:00:00, 1992-03-01T00:00:00, 1992-04-01T00:00:00, 1992-05-01T00:00:00, 1992-06-01T00:00:00, 1992-07-01T00:00:00, 1992-08-01T00:00:00, 1992-09-01T00:00:00, 1992-10-01T00:00:00, 1992-11-01T00:00:00, 1992-12-01T00:00:00, 1993-01-01T00:00:00] +[1993-02-01T00:00:00, 1993-01-31T00:00:00, 1993-01-30T00:00:00, 1993-01-29T00:00:00, 1993-01-28T00:00:00, 1993-01-27T00:00:00, 1993-01-26T00:00:00, 1993-01-25T00:00:00, 1993-01-24T00:00:00, 1993-01-23T00:00:00, 1993-01-22T00:00:00, 1993-01-21T00:00:00, 1993-01-20T00:00:00, 1993-01-19T00:00:00, 1993-01-18T00:00:00, 1993-01-17T00:00:00, 1993-01-16T00:00:00, 1993-01-15T00:00:00, 1993-01-14T00:00:00, 1993-01-13T00:00:00, 1993-01-12T00:00:00, 1993-01-11T00:00:00, 1993-01-10T00:00:00, 1993-01-09T00:00:00, 1993-01-08T00:00:00, 1993-01-07T00:00:00, 1993-01-06T00:00:00, 1993-01-05T00:00:00, 1993-01-04T00:00:00, 1993-01-03T00:00:00, 1993-01-02T00:00:00, 1993-01-01T00:00:00] +[1989-04-01T00:00:00, 1990-04-01T00:00:00, 1991-04-01T00:00:00, 1992-04-01T00:00:00] + +query ? +select generate_series(start, stop, INTERVAL '1 year') from timestamp_table; +---- +[1992-01-01T00:00:00, 1993-01-01T00:00:00] +[] +[1989-04-01T00:00:00, 1990-04-01T00:00:00, 1991-04-01T00:00:00, 1992-04-01T00:00:00] +query ? +select generate_series(start, '1993-03-01T00:00:00'::timestamp, INTERVAL '1 year') from timestamp_table; +---- +[1992-01-01T00:00:00, 1993-01-01T00:00:00] +[1993-02-01T00:00:00] +[1989-04-01T00:00:00, 1990-04-01T00:00:00, 1991-04-01T00:00:00, 1992-04-01T00:00:00] + +# https://github.com/apache/datafusion/issues/11922 +query ? +select generate_series(start, '1993-03-01T00:00:00'::timestamp, INTERVAL '1 year') from timestamp_table; +---- +[1992-01-01T00:00:00, 1993-01-01T00:00:00] +[1993-02-01T00:00:00] +[1989-04-01T00:00:00, 1990-04-01T00:00:00, 1991-04-01T00:00:00, 1992-04-01T00:00:00] ## array_except diff --git a/docs/source/user-guide/sql/scalar_functions.md b/docs/source/user-guide/sql/scalar_functions.md index 14852935dca4..6a2181b8db40 100644 --- a/docs/source/user-guide/sql/scalar_functions.md +++ b/docs/source/user-guide/sql/scalar_functions.md @@ -3223,9 +3223,9 @@ generate_series(start, stop, step) #### Arguments -- **start**: start of the range -- **end**: end of the range (included) -- **step**: increase by step (can not be 0) +- **start**: start of the series. Ints, timestamps, dates or string types that can be coerced to Date32 are supported. +- **end**: end of the series (included). Type must be the same as start. +- **step**: increase by step (can not be 0). Steps less than a day are supported only for timestamp ranges. #### Example @@ -3516,7 +3516,7 @@ The range start..end contains all values with start <= x < end. It is empty if s Step can not be 0 (then the range will be nonsense.). -Note that when the required range is a number, it accepts (stop), (start, stop), and (start, stop, step) as parameters, but when the required range is a date, it must be 3 non-NULL parameters. +Note that when the required range is a number, it accepts (stop), (start, stop), and (start, stop, step) as parameters, but when the required range is a date or timestamp, it must be 3 non-NULL parameters. For example, ``` @@ -3527,10 +3527,11 @@ SELECT range(1,5,1); are allowed in number ranges -but in date ranges, only +but in date and timestamp ranges, only ``` SELECT range(DATE '1992-09-01', DATE '1993-03-01', INTERVAL '1' MONTH); +SELECT range(TIMESTAMP '1992-09-01', TIMESTAMP '1993-03-01', INTERVAL '1' MONTH); ``` is allowed, and @@ -3545,9 +3546,9 @@ are not allowed #### Arguments -- **start**: start of the range -- **end**: end of the range (not included) -- **step**: increase by step (can not be 0) +- **start**: start of the range. Ints, timestamps, dates or string types that can be coerced to Date32 are supported. +- **end**: end of the range (not included). Type must be the same as start. +- **step**: increase by step (can not be 0). Steps less than a day are supported only for timestamp ranges. #### Aliases From 97ad0ad77d89a66e24a435667d2d43f19bb8794d Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Thu, 12 Sep 2024 11:54:02 -0400 Subject: [PATCH 152/154] Improve comments on target user and unify summaries (#12418) --- README.md | 25 ++++++++++++++++++++++--- datafusion/core/src/lib.rs | 24 ++++++++++++++---------- docs/source/index.rst | 25 +++++++++++++++++-------- 3 files changed, 53 insertions(+), 21 deletions(-) diff --git a/README.md b/README.md index 816dc77714d2..bb8526c24e2c 100644 --- a/README.md +++ b/README.md @@ -41,9 +41,28 @@ logo -Apache DataFusion is a very fast, extensible query engine for building high-quality data-centric systems in -[Rust](http://rustlang.org), using the [Apache Arrow](https://arrow.apache.org) -in-memory format. [Python Bindings](https://github.com/apache/datafusion-python) are also available. DataFusion offers SQL and Dataframe APIs, excellent [performance](https://benchmark.clickhouse.com/), built-in support for CSV, Parquet, JSON, and Avro, extensive customization, and a great community. +DataFusion is an extensible query engine written in [Rust] that +uses [Apache Arrow] as its in-memory format. DataFusion's target users are +developers building fast and feature rich database and analytic systems, +customized to particular workloads. See [use cases] for examples. + +"Out of the box," DataFusion offers [SQL] and [`Dataframe`] APIs, +excellent [performance], built-in support for CSV, Parquet, JSON, and Avro, +extensive customization, and a great community. +[Python Bindings] are also available. + +DataFusion features a full query planner, a columnar, streaming, multi-threaded, +vectorized execution engine, and partitioned data sources. You can +customize DataFusion at almost all points including additional data sources, +query languages, functions, custom operators and more. +See the [Architecture] section for more details. + +[rust]: http://rustlang.org +[apache arrow]: https://arrow.apache.org +[use cases]: https://datafusion.apache.org/user-guide/introduction.html#use-cases +[python bindings]: https://github.com/apache/datafusion-python +[performance]: https://benchmark.clickhouse.com/ +[architecture]: https://datafusion.apache.org/contributor-guide/architecture.html Here are links to some important information diff --git a/datafusion/core/src/lib.rs b/datafusion/core/src/lib.rs index 9c368415bb05..63d4fbc0bba5 100644 --- a/datafusion/core/src/lib.rs +++ b/datafusion/core/src/lib.rs @@ -17,24 +17,28 @@ #![warn(missing_docs, clippy::needless_borrow)] //! [DataFusion] is an extensible query engine written in Rust that -//! uses [Apache Arrow] as its in-memory format. DataFusion help developers -//! build fast and feature rich database and analytic systems, customized to -//! particular workloads. See [use cases] for examples +//! uses [Apache Arrow] as its in-memory format. DataFusion's target users are +//! developers building fast and feature rich database and analytic systems, +//! customized to particular workloads. See [use cases] for examples. //! -//! "Out of the box," DataFusion quickly runs complex [SQL] and -//! [`DataFrame`] queries using a full-featured query planner, a columnar, -//! streaming, multi-threaded, vectorized execution engine, and partitioned data -//! sources (Parquet, CSV, JSON, and Avro). +//! "Out of the box," DataFusion offers [SQL] and [`Dataframe`] APIs, +//! excellent [performance], built-in support for CSV, Parquet, JSON, and Avro, +//! extensive customization, and a great community. +//! [Python Bindings] are also available. //! -//! DataFusion is designed for easy customization such as -//! additional data sources, query languages, functions, custom -//! operators and more. See the [Architecture] section for more details. +//! DataFusion features a full query planner, a columnar, streaming, multi-threaded, +//! vectorized execution engine, and partitioned data sources. You can +//! customize DataFusion at almost all points including additional data sources, +//! query languages, functions, custom operators and more. +//! See the [Architecture] section below for more details. //! //! [DataFusion]: https://datafusion.apache.org/ //! [Apache Arrow]: https://arrow.apache.org //! [use cases]: https://datafusion.apache.org/user-guide/introduction.html#use-cases //! [SQL]: https://datafusion.apache.org/user-guide/sql/index.html //! [`DataFrame`]: dataframe::DataFrame +//! [performance]: https://benchmark.clickhouse.com/ +//! [Python Bindings]: https://github.com/apache/datafusion-python //! [Architecture]: #architecture //! //! # Examples diff --git a/docs/source/index.rst b/docs/source/index.rst index bb5ea430a321..4c67e808a4dd 100644 --- a/docs/source/index.rst +++ b/docs/source/index.rst @@ -32,14 +32,23 @@ Apache DataFusion Fork

-DataFusion is a very fast, extensible query engine for building high-quality data-centric systems in -`Rust `_, using the `Apache Arrow `_ -in-memory format. - -DataFusion offers SQL and Dataframe APIs, excellent -`performance `_, built-in support for -CSV, Parquet, JSON, and Avro, extensive customization, and a great -community. + +DataFusion is an extensible query engine written in `Rust `_ that +uses `Apache Arrow `_ as its in-memory format. DataFusion's target users are +developers building fast and feature rich database and analytic systems, +customized to particular workloads. See `use cases `_ for examples. + +"Out of the box," DataFusion offers `SQL `_ +and `Dataframe `_ APIs, +excellent `performance `_, built-in support for CSV, Parquet, JSON, and Avro, +extensive customization, and a great community. +`Python Bindings `_ are also available. + +DataFusion features a full query planner, a columnar, streaming, multi-threaded, +vectorized execution engine, and partitioned data sources. You can +customize DataFusion at almost all points including additional data sources, +query languages, functions, custom operators and more. +See the `Architecture `_ section for more details. To get started, see From 533fbc4f605fcc213a3fbdb1dcdc65968ac5e718 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Thu, 12 Sep 2024 11:59:32 -0400 Subject: [PATCH 153/154] Add 'Extensions List' page to the documentation (#12420) --- docs/source/contributor-guide/architecture.md | 18 ++---- docs/source/index.rst | 1 + docs/source/library-user-guide/extensions.md | 64 +++++++++++++++++++ 3 files changed, 69 insertions(+), 14 deletions(-) create mode 100644 docs/source/library-user-guide/extensions.md diff --git a/docs/source/contributor-guide/architecture.md b/docs/source/contributor-guide/architecture.md index 55c8a1d980df..1a094968a274 100644 --- a/docs/source/contributor-guide/architecture.md +++ b/docs/source/contributor-guide/architecture.md @@ -43,20 +43,10 @@ DataFusion. If you can not do what you want with the existing APIs, we would welcome you working with us to add new APIs to enable your use case, as described in the next section. -## `datafusion-contrib` - -While DataFusions comes with enough features "out of the box" to quickly start -with a working system, it can't include everything useful feature (e.g. -`TableProvider`s for all data formats). The [`datafusion-contrib`] project -contains a collection of community maintained extensions that are not part of -the core DataFusion project, and not under Apache Software Foundation governance -but may be useful to others in the community. If you are interested adding a -feature to DataFusion, a new extension in `datafusion-contrib` is likely a good -place to start. Please [contact] us via github issue, slack, or Discord and -we'll gladly set up a new repository for your extension. - -[`datafusion-contrib`]: https://github.com/datafusion-contrib -[contact]: ../contributor-guide/communication.md +Please see the [Extensions] section to find out more about existing DataFusion +extensions and how to contribute your extension to the community. + +[extensions]: ../library-user-guide/extensions.md ## Creating new Extension APIs diff --git a/docs/source/index.rst b/docs/source/index.rst index 4c67e808a4dd..32a5dce323f2 100644 --- a/docs/source/index.rst +++ b/docs/source/index.rst @@ -108,6 +108,7 @@ To get started, see :caption: Library User Guide library-user-guide/index + library-user-guide/extensions library-user-guide/using-the-sql-api library-user-guide/working-with-exprs library-user-guide/using-the-dataframe-api diff --git a/docs/source/library-user-guide/extensions.md b/docs/source/library-user-guide/extensions.md new file mode 100644 index 000000000000..0c7c891f4b1e --- /dev/null +++ b/docs/source/library-user-guide/extensions.md @@ -0,0 +1,64 @@ + + +# Extensions List + +DataFusion tries to provide a good set of features "out of the box" to quickly +start with a working system, but it can't include every useful feature (e.g. +`TableProvider`s for all data formats). + +Thankfully one of the core features of DataFusion is a flexible extension API +that allows users to extend its behavior at all points. This page lists some +community maintained extensions available for DataFusion. These extensions are +not part of the core DataFusion project, and not under Apache Software +Foundation governance but we list them here to be useful to others in the +community. + +If you know of an available extension that is not listed below, please open a PR +to add it to this page. If there is some feature you would like to see in +DataFusion, please consider creating a new extension in the `datafusion-contrib` +project (see [below](#datafusion-contrib)). Please [contact] us via github issue, slack, or Discord and +we'll gladly set up a new repository for your extension. + +| Name | Type | Description | +| ---------------------------- | ----------------- | --------------------------------------------------------------------------------- | +| [DataFusion Table Providers] | [`TableProvider`] | Support for `PostgreSQL`, `MySQL`, `SQLite`, `DuckDB`, and `Flight SQL` | +| [DataFusion Federation] | Framework | Allows DataFusion to execute (part of) a query plan by a remote execution engine. | +| [DataFusion ORC] | [`TableProvider`] | [Apache ORC] file format | +| [DataFusion JSON Functions] | Functions | Scalar functions for querying JSON strings | + +[`tableprovider`]: https://docs.rs/datafusion/latest/datafusion/catalog/trait.TableProvider.html +[datafusion table providers]: https://github.com/datafusion-contrib/datafusion-table-providers +[datafusion federation]: https://github.com/datafusion-contrib/datafusion-federation +[datafusion orc]: https://github.com/datafusion-contrib/datafusion-orc +[apache orc]: https://orc.apache.org/ +[datafusion json functions]: https://github.com/datafusion-contrib/datafusion-functions-json + +## `datafusion-contrib` + +The [`datafusion-contrib`] project contains a collection of community maintained +extensions that are not part of the core DataFusion project, and not under +Apache Software Foundation governance but may be useful to others in the +community. If you are interested adding a feature to DataFusion, a new extension +in `datafusion-contrib` is likely a good place to start. Please [contact] us via +github issue, slack, or Discord and we'll gladly set up a new repository for +your extension. + +[`datafusion-contrib`]: https://github.com/datafusion-contrib +[contact]: ../contributor-guide/communication.md From 04e8e5336670f785ac239245bcfad280df571cd3 Mon Sep 17 00:00:00 2001 From: WeblWabl Date: Thu, 12 Sep 2024 11:05:55 -0500 Subject: [PATCH 154/154] Fix: Internal error in regexp_replace() for some StringView input (#12203) * Fix: Internal error in regexp_replace() for some StringView input * fix regex bench * fmt * fix bench regx * clippy * fmt * adds tests for flags + includes type signature for utf8view with flag * fix: adding collect for string view type --- datafusion/functions/benches/regx.rs | 14 +- .../functions/src/regex/regexpreplace.rs | 258 ++++++++++++------ .../sqllogictest/test_files/string_view.slt | 90 ++++++ 3 files changed, 268 insertions(+), 94 deletions(-) diff --git a/datafusion/functions/benches/regx.rs b/datafusion/functions/benches/regx.rs index 23d57f38efae..45bfa2351128 100644 --- a/datafusion/functions/benches/regx.rs +++ b/datafusion/functions/benches/regx.rs @@ -18,7 +18,7 @@ extern crate criterion; use arrow::array::builder::StringBuilder; -use arrow::array::{ArrayRef, StringArray}; +use arrow::array::{ArrayRef, AsArray, StringArray}; use criterion::{black_box, criterion_group, criterion_main, Criterion}; use datafusion_functions::regex::regexplike::regexp_like; use datafusion_functions::regex::regexpmatch::regexp_match; @@ -122,12 +122,12 @@ fn criterion_benchmark(c: &mut Criterion) { b.iter(|| { black_box( - regexp_replace::(&[ - Arc::clone(&data), - Arc::clone(®ex), - Arc::clone(&replacement), - Arc::clone(&flags), - ]) + regexp_replace::( + data.as_string::(), + regex.as_string::(), + replacement.as_string::(), + Some(&flags), + ) .expect("regexp_replace should work on valid values"), ) }) diff --git a/datafusion/functions/src/regex/regexpreplace.rs b/datafusion/functions/src/regex/regexpreplace.rs index 0b0f7287e1ec..3eb72a1fb5f5 100644 --- a/datafusion/functions/src/regex/regexpreplace.rs +++ b/datafusion/functions/src/regex/regexpreplace.rs @@ -16,13 +16,13 @@ // under the License. //! Regx expressions -use arrow::array::new_null_array; -use arrow::array::ArrayAccessor; use arrow::array::ArrayDataBuilder; use arrow::array::BufferBuilder; use arrow::array::GenericStringArray; use arrow::array::StringViewBuilder; +use arrow::array::{new_null_array, ArrayIter, AsArray}; use arrow::array::{Array, ArrayRef, OffsetSizeTrait}; +use arrow::array::{ArrayAccessor, StringViewArray}; use arrow::datatypes::DataType; use datafusion_common::cast::as_string_view_array; use datafusion_common::exec_err; @@ -59,6 +59,7 @@ impl RegexpReplaceFunc { Exact(vec![Utf8, Utf8, Utf8]), Exact(vec![Utf8View, Utf8, Utf8]), Exact(vec![Utf8, Utf8, Utf8, Utf8]), + Exact(vec![Utf8View, Utf8, Utf8, Utf8]), ], Volatility::Immutable, ), @@ -187,104 +188,147 @@ fn regex_replace_posix_groups(replacement: &str) -> String { /// # Ok(()) /// # } /// ``` -pub fn regexp_replace(args: &[ArrayRef]) -> Result { +pub fn regexp_replace<'a, T: OffsetSizeTrait, V, B>( + string_array: V, + pattern_array: B, + replacement_array: B, + flags: Option<&ArrayRef>, +) -> Result +where + V: ArrayAccessor, + B: ArrayAccessor, +{ // Default implementation for regexp_replace, assumes all args are arrays // and args is a sequence of 3 or 4 elements. // creating Regex is expensive so create hashmap for memoization let mut patterns: HashMap = HashMap::new(); - match args.len() { - 3 => { - let string_array = as_generic_string_array::(&args[0])?; - let pattern_array = as_generic_string_array::(&args[1])?; - let replacement_array = as_generic_string_array::(&args[2])?; - - let result = string_array - .iter() - .zip(pattern_array.iter()) - .zip(replacement_array.iter()) - .map(|((string, pattern), replacement)| match (string, pattern, replacement) { - (Some(string), Some(pattern), Some(replacement)) => { - let replacement = regex_replace_posix_groups(replacement); - - // if patterns hashmap already has regexp then use else create and return - let re = match patterns.get(pattern) { - Some(re) => Ok(re), - None => { - match Regex::new(pattern) { - Ok(re) => { - patterns.insert(pattern.to_string(), re); - Ok(patterns.get(pattern).unwrap()) + let datatype = string_array.data_type().to_owned(); + + let string_array_iter = ArrayIter::new(string_array); + let pattern_array_iter = ArrayIter::new(pattern_array); + let replacement_array_iter = ArrayIter::new(replacement_array); + + match flags { + None => { + let result_iter = string_array_iter + .zip(pattern_array_iter) + .zip(replacement_array_iter) + .map(|((string, pattern), replacement)| { + match (string, pattern, replacement) { + (Some(string), Some(pattern), Some(replacement)) => { + let replacement = regex_replace_posix_groups(replacement); + // if patterns hashmap already has regexp then use else create and return + let re = match patterns.get(pattern) { + Some(re) => Ok(re), + None => match Regex::new(pattern) { + Ok(re) => { + patterns.insert(pattern.to_string(), re); + Ok(patterns.get(pattern).unwrap()) + } + Err(err) => { + Err(DataFusionError::External(Box::new(err))) + } }, - Err(err) => Err(DataFusionError::External(Box::new(err))), - } - } - }; + }; - Some(re.map(|re| re.replace(string, replacement.as_str()))).transpose() + Some(re.map(|re| re.replace(string, replacement.as_str()))) + .transpose() + } + _ => Ok(None), + } + }); + + match datatype { + DataType::Utf8 | DataType::LargeUtf8 => { + let result = + result_iter.collect::>>()?; + Ok(Arc::new(result) as ArrayRef) } - _ => Ok(None) - }) - .collect::>>()?; - - Ok(Arc::new(result) as ArrayRef) + DataType::Utf8View => { + let result = result_iter.collect::>()?; + Ok(Arc::new(result) as ArrayRef) + } + other => { + exec_err!( + "Unsupported data type {other:?} for function regex_replace" + ) + } + } } - 4 => { - let string_array = as_generic_string_array::(&args[0])?; - let pattern_array = as_generic_string_array::(&args[1])?; - let replacement_array = as_generic_string_array::(&args[2])?; - let flags_array = as_generic_string_array::(&args[3])?; - - let result = string_array - .iter() - .zip(pattern_array.iter()) - .zip(replacement_array.iter()) - .zip(flags_array.iter()) - .map(|(((string, pattern), replacement), flags)| match (string, pattern, replacement, flags) { - (Some(string), Some(pattern), Some(replacement), Some(flags)) => { - let replacement = regex_replace_posix_groups(replacement); - - // format flags into rust pattern - let (pattern, replace_all) = if flags == "g" { - (pattern.to_string(), true) - } else if flags.contains('g') { - (format!("(?{}){}", flags.to_string().replace('g', ""), pattern), true) - } else { - (format!("(?{flags}){pattern}"), false) - }; - - // if patterns hashmap already has regexp then use else create and return - let re = match patterns.get(&pattern) { - Some(re) => Ok(re), - None => { - match Regex::new(pattern.as_str()) { - Ok(re) => { - patterns.insert(pattern.clone(), re); - Ok(patterns.get(&pattern).unwrap()) + Some(flags) => { + let flags_array = as_generic_string_array::(flags)?; + + let result_iter = string_array_iter + .zip(pattern_array_iter) + .zip(replacement_array_iter) + .zip(flags_array.iter()) + .map(|(((string, pattern), replacement), flags)| { + match (string, pattern, replacement, flags) { + (Some(string), Some(pattern), Some(replacement), Some(flags)) => { + let replacement = regex_replace_posix_groups(replacement); + + // format flags into rust pattern + let (pattern, replace_all) = if flags == "g" { + (pattern.to_string(), true) + } else if flags.contains('g') { + ( + format!( + "(?{}){}", + flags.to_string().replace('g', ""), + pattern + ), + true, + ) + } else { + (format!("(?{flags}){pattern}"), false) + }; + + // if patterns hashmap already has regexp then use else create and return + let re = match patterns.get(&pattern) { + Some(re) => Ok(re), + None => match Regex::new(pattern.as_str()) { + Ok(re) => { + patterns.insert(pattern.clone(), re); + Ok(patterns.get(&pattern).unwrap()) + } + Err(err) => { + Err(DataFusionError::External(Box::new(err))) + } }, - Err(err) => Err(DataFusionError::External(Box::new(err))), - } + }; + + Some(re.map(|re| { + if replace_all { + re.replace_all(string, replacement.as_str()) + } else { + re.replace(string, replacement.as_str()) + } + })) + .transpose() } - }; - - Some(re.map(|re| { - if replace_all { - re.replace_all(string, replacement.as_str()) - } else { - re.replace(string, replacement.as_str()) - } - })).transpose() + _ => Ok(None), + } + }); + + match datatype { + DataType::Utf8 | DataType::LargeUtf8 => { + let result = + result_iter.collect::>>()?; + Ok(Arc::new(result) as ArrayRef) } - _ => Ok(None) - }) - .collect::>>()?; - - Ok(Arc::new(result) as ArrayRef) + DataType::Utf8View => { + let result = result_iter.collect::>()?; + Ok(Arc::new(result) as ArrayRef) + } + other => { + exec_err!( + "Unsupported data type {other:?} for function regex_replace" + ) + } + } } - other => exec_err!( - "regexp_replace was called with {other} arguments. It requires at least 3 and at most 4." - ), } } @@ -495,7 +539,47 @@ pub fn specialize_regexp_replace( .iter() .map(|arg| arg.clone().into_array(inferred_length)) .collect::>>()?; - regexp_replace::(&args) + + match args[0].data_type() { + DataType::Utf8View => { + let string_array = args[0].as_string_view(); + let pattern_array = args[1].as_string::(); + let replacement_array = args[2].as_string::(); + regexp_replace::( + string_array, + pattern_array, + replacement_array, + args.get(3), + ) + } + DataType::Utf8 => { + let string_array = args[0].as_string::(); + let pattern_array = args[1].as_string::(); + let replacement_array = args[2].as_string::(); + regexp_replace::( + string_array, + pattern_array, + replacement_array, + args.get(3), + ) + } + DataType::LargeUtf8 => { + let string_array = args[0].as_string::(); + let pattern_array = args[1].as_string::(); + let replacement_array = args[2].as_string::(); + regexp_replace::( + string_array, + pattern_array, + replacement_array, + args.get(3), + ) + } + other => { + exec_err!( + "Unsupported data type {other:?} for function regex_replace" + ) + } + } } } } diff --git a/datafusion/sqllogictest/test_files/string_view.slt b/datafusion/sqllogictest/test_files/string_view.slt index 171b8ec6c1d1..161e4e88dea1 100644 --- a/datafusion/sqllogictest/test_files/string_view.slt +++ b/datafusion/sqllogictest/test_files/string_view.slt @@ -459,6 +459,96 @@ Xiangpeng Raphael NULL +### Test REGEXP_REPLACE + +# Should run REGEXP_REPLACE with Scalar value for utf8view +query T +SELECT + REGEXP_REPLACE(column1_utf8view, 'e', 'f') AS k +FROM test; +---- +Andrfw +Xiangpfng +Raphafl +NULL + +# Should run REGEXP_REPLACE with Scalar value for utf8view with flag +query T +SELECT + REGEXP_REPLACE(column1_utf8view, 'e', 'f', 'i') AS k +FROM test; +---- +Andrfw +Xiangpfng +Raphafl +NULL + +# Should run REGEXP_REPLACE with Scalar value for utf8 +query T +SELECT + REGEXP_REPLACE(column1_utf8, 'e', 'f') AS k +FROM test; +---- +Andrfw +Xiangpfng +Raphafl +NULL + +# Should run REGEXP_REPLACE with Scalar value for utf8 with flag +query T +SELECT + REGEXP_REPLACE(column1_utf8, 'e', 'f', 'i') AS k +FROM test; +---- +Andrfw +Xiangpfng +Raphafl +NULL + +# Should run REGEXP_REPLACE with ScalarArray value for utf8view +query T +SELECT + REGEXP_REPLACE(column1_utf8view, lower(column1_utf8view), 'bar') AS k +FROM test; +---- +Andrew +Xiangpeng +Raphael +NULL + +# Should run REGEXP_REPLACE with ScalarArray value for utf8view with flag +query T +SELECT + REGEXP_REPLACE(column1_utf8view, lower(column1_utf8view), 'bar', 'g') AS k +FROM test; +---- +Andrew +Xiangpeng +Raphael +NULL + +# Should run REGEXP_REPLACE with ScalarArray value for utf8 +query T +SELECT + REGEXP_REPLACE(column1_utf8, lower(column1_utf8), 'bar') AS k +FROM test; +---- +Andrew +Xiangpeng +Raphael +NULL + +# Should run REGEXP_REPLACE with ScalarArray value for utf8 with flag +query T +SELECT + REGEXP_REPLACE(column1_utf8, lower(column1_utf8), 'bar', 'g') AS k +FROM test; +---- +Andrew +Xiangpeng +Raphael +NULL + ### Initcap query TT