diff --git a/Cargo.toml b/Cargo.toml index c1f8a604dd58..8d25478ab549 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -22,6 +22,7 @@ members = [ "datafusion/common", "datafusion/common-runtime", "datafusion/catalog", + "datafusion/catalog-listing", "datafusion/core", "datafusion/expr", "datafusion/expr-common", @@ -100,6 +101,7 @@ ctor = "0.2.9" dashmap = "6.0.1" datafusion = { path = "datafusion/core", version = "45.0.0", default-features = false } datafusion-catalog = { path = "datafusion/catalog", version = "45.0.0" } +datafusion-catalog-listing = { path = "datafusion/catalog-listing", version = "45.0.0" } datafusion-common = { path = "datafusion/common", version = "45.0.0", default-features = false } datafusion-common-runtime = { path = "datafusion/common-runtime", version = "45.0.0" } datafusion-doc = { path = "datafusion/doc", version = "45.0.0" } @@ -148,7 +150,7 @@ recursive = "0.1.1" regex = "1.8" rstest = "0.24.0" serde_json = "1" -sqlparser = { version = "0.53.0", features = ["visitor"] } +sqlparser = { version = "0.54.0", features = ["visitor"] } tempfile = "3" tokio = { version = "1.43", features = ["macros", "rt", "sync"] } url = "2.5.4" diff --git a/datafusion-cli/Cargo.lock b/datafusion-cli/Cargo.lock index bcbee29d6b5b..e320b2ffc835 100644 --- a/datafusion-cli/Cargo.lock +++ b/datafusion-cli/Cargo.lock @@ -1219,6 +1219,7 @@ dependencies = [ "bzip2 0.5.0", "chrono", "datafusion-catalog", + "datafusion-catalog-listing", "datafusion-common", "datafusion-common-runtime", "datafusion-execution", @@ -1274,6 +1275,28 @@ dependencies = [ "sqlparser", ] +[[package]] +name = "datafusion-catalog-listing" +version = "45.0.0" +dependencies = [ + "arrow", + "arrow-schema", + "chrono", + "datafusion-catalog", + "datafusion-common", + "datafusion-execution", + "datafusion-expr", + "datafusion-physical-expr", + "datafusion-physical-expr-common", + "datafusion-physical-plan", + "futures", + "glob", + "itertools 0.14.0", + "log", + "object_store", + "url", +] + [[package]] name = "datafusion-cli" version = "45.0.0" @@ -1309,7 +1332,6 @@ dependencies = [ "apache-avro", "arrow", "arrow-array", - "arrow-buffer", "arrow-ipc", "arrow-schema", "base64 0.22.1", @@ -1419,7 +1441,6 @@ version = "45.0.0" dependencies = [ "ahash", "arrow", - "arrow-buffer", "arrow-schema", "datafusion-common", "datafusion-doc", @@ -1451,7 +1472,6 @@ version = "45.0.0" dependencies = [ "arrow", "arrow-array", - "arrow-buffer", "arrow-ord", "arrow-schema", "datafusion-common", @@ -1537,7 +1557,6 @@ dependencies = [ "ahash", "arrow", "arrow-array", - "arrow-buffer", "arrow-schema", "datafusion-common", "datafusion-expr", @@ -1559,7 +1578,6 @@ version = "45.0.0" dependencies = [ "ahash", "arrow", - "arrow-buffer", "datafusion-common", "datafusion-expr-common", "hashbrown 0.14.5", @@ -1593,7 +1611,6 @@ dependencies = [ "ahash", "arrow", "arrow-array", - "arrow-buffer", "arrow-ord", "arrow-schema", "async-trait", @@ -3781,11 +3798,12 @@ checksum = "6980e8d7511241f8acf4aebddbb1ff938df5eebe98691418c4468d0b72a96a67" [[package]] name = "sqlparser" -version = "0.53.0" +version = "0.54.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "05a528114c392209b3264855ad491fcce534b94a38771b0a0b97a79379275ce8" +checksum = "c66e3b7374ad4a6af849b08b3e7a6eda0edbd82f0fd59b57e22671bf16979899" dependencies = [ "log", + "recursive", "sqlparser_derive", ] diff --git a/datafusion-examples/examples/advanced_parquet_index.rs b/datafusion-examples/examples/advanced_parquet_index.rs index 60466d1b1292..43dc592b997e 100644 --- a/datafusion-examples/examples/advanced_parquet_index.rs +++ b/datafusion-examples/examples/advanced_parquet_index.rs @@ -26,7 +26,6 @@ use std::sync::Arc; use arrow::array::{ArrayRef, Int32Array, RecordBatch, StringArray}; use arrow::datatypes::SchemaRef; use datafusion::catalog::Session; -use datafusion::common::config::TableParquetOptions; use datafusion::common::{ internal_datafusion_err, DFSchema, DataFusionError, Result, ScalarValue, }; @@ -492,16 +491,12 @@ impl TableProvider for IndexTableProvider { .with_file(indexed_file); let file_source = Arc::new( - ParquetSource::new( - Arc::clone(&schema), + ParquetSource::default() // provide the predicate so the DataSourceExec can try and prune // row groups internally - Some(predicate), - None, - TableParquetOptions::default(), - ) - // provide the factory to create parquet reader without re-reading metadata - .with_parquet_file_reader_factory(Arc::new(reader_factory)), + .with_predicate(Arc::clone(&schema), predicate) + // provide the factory to create parquet reader without re-reading metadata + .with_parquet_file_reader_factory(Arc::new(reader_factory)), ); let file_scan_config = FileScanConfig::new(object_store_url, schema, file_source) .with_limit(limit) diff --git a/datafusion-examples/examples/parquet_exec_visitor.rs b/datafusion-examples/examples/parquet_exec_visitor.rs index fac075358ec3..20809a1121c1 100644 --- a/datafusion-examples/examples/parquet_exec_visitor.rs +++ b/datafusion-examples/examples/parquet_exec_visitor.rs @@ -19,7 +19,7 @@ use std::sync::Arc; use datafusion::datasource::file_format::parquet::ParquetFormat; use datafusion::datasource::listing::{ListingOptions, PartitionedFile}; -use datafusion::datasource::physical_plan::FileScanConfig; +use datafusion::datasource::physical_plan::{FileScanConfig, ParquetSource}; use datafusion::error::DataFusionError; use datafusion::execution::context::SessionContext; use datafusion::physical_plan::metrics::MetricValue; @@ -100,7 +100,12 @@ impl ExecutionPlanVisitor for ParquetExecVisitor { if let Some(data_source) = plan.as_any().downcast_ref::() { let source = data_source.source(); if let Some(file_config) = source.as_any().downcast_ref::() { - if file_config.file_source().file_type().is_parquet() { + if file_config + .file_source() + .as_any() + .downcast_ref::() + .is_some() + { self.file_groups = Some(file_config.file_groups.clone()); let metrics = match data_source.metrics() { diff --git a/datafusion-examples/examples/parquet_index.rs b/datafusion-examples/examples/parquet_index.rs index 970284104ae7..f465699abed2 100644 --- a/datafusion-examples/examples/parquet_index.rs +++ b/datafusion-examples/examples/parquet_index.rs @@ -23,7 +23,6 @@ use arrow::datatypes::{Int32Type, SchemaRef}; use arrow::util::pretty::pretty_format_batches; use async_trait::async_trait; use datafusion::catalog::Session; -use datafusion::common::config::TableParquetOptions; use datafusion::common::{ internal_datafusion_err, DFSchema, DataFusionError, Result, ScalarValue, }; @@ -242,12 +241,8 @@ impl TableProvider for IndexTableProvider { let files = self.index.get_files(predicate.clone())?; let object_store_url = ObjectStoreUrl::parse("file://")?; - let source = Arc::new(ParquetSource::new( - self.schema(), - Some(predicate), - None, - TableParquetOptions::default(), - )); + let source = + Arc::new(ParquetSource::default().with_predicate(self.schema(), predicate)); let mut file_scan_config = FileScanConfig::new(object_store_url, self.schema(), source) .with_projection(projection.cloned()) diff --git a/datafusion/catalog-listing/Cargo.toml b/datafusion/catalog-listing/Cargo.toml new file mode 100644 index 000000000000..03132e7b7bb5 --- /dev/null +++ b/datafusion/catalog-listing/Cargo.toml @@ -0,0 +1,65 @@ +# 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-catalog-listing" +description = "datafusion-catalog-listing" +authors.workspace = true +edition.workspace = true +homepage.workspace = true +license.workspace = true +readme.workspace = true +repository.workspace = true +rust-version.workspace = true +version.workspace = true + +[dependencies] +arrow = { workspace = true } +arrow-schema = { workspace = true } +async-compression = { version = "0.4.0", features = [ + "bzip2", + "gzip", + "xz", + "zstd", + "tokio", +], optional = true } +chrono = { workspace = true } +datafusion-catalog = { workspace = true } +datafusion-common = { workspace = true, features = ["object_store"] } +datafusion-execution = { workspace = true } +datafusion-expr = { workspace = true } +datafusion-physical-expr = { workspace = true } +datafusion-physical-expr-common = { workspace = true } +datafusion-physical-plan = { workspace = true } +futures = { workspace = true } +glob = "0.3.0" +itertools = { workspace = true } +log = { workspace = true } +object_store = { workspace = true } +url = { workspace = true } + +[dev-dependencies] +async-trait = { workspace = true } +tempfile = { workspace = true } +tokio = { workspace = true } + +[lints] +workspace = true + +[lib] +name = "datafusion_catalog_listing" +path = "src/mod.rs" diff --git a/datafusion/catalog-listing/LICENSE.txt b/datafusion/catalog-listing/LICENSE.txt new file mode 120000 index 000000000000..1ef648f64b34 --- /dev/null +++ b/datafusion/catalog-listing/LICENSE.txt @@ -0,0 +1 @@ +../../LICENSE.txt \ No newline at end of file diff --git a/datafusion/catalog-listing/NOTICE.txt b/datafusion/catalog-listing/NOTICE.txt new file mode 120000 index 000000000000..fb051c92b10b --- /dev/null +++ b/datafusion/catalog-listing/NOTICE.txt @@ -0,0 +1 @@ +../../NOTICE.txt \ No newline at end of file diff --git a/datafusion/catalog-listing/README.md b/datafusion/catalog-listing/README.md new file mode 100644 index 000000000000..b4760c413d60 --- /dev/null +++ b/datafusion/catalog-listing/README.md @@ -0,0 +1,30 @@ + + +# DataFusion catalog-listing + +[DataFusion][df] is an extensible query execution framework, written in Rust, that uses Apache Arrow as its in-memory format. + +This crate is a submodule of DataFusion with [ListingTable], an implementation +of [TableProvider] based on files in a directory (either locally or on remote +object storage such as S3). + +[df]: https://crates.io/crates/datafusion +[listingtable]: https://docs.rs/datafusion/latest/datafusion/datasource/listing/struct.ListingTable.html +[tableprovider]: https://docs.rs/datafusion/latest/datafusion/datasource/trait.TableProvider.html diff --git a/datafusion/core/src/datasource/listing/helpers.rs b/datafusion/catalog-listing/src/helpers.rs similarity index 91% rename from datafusion/core/src/datasource/listing/helpers.rs rename to datafusion/catalog-listing/src/helpers.rs index 228b9a4e9f6b..6cb3f661e652 100644 --- a/datafusion/core/src/datasource/listing/helpers.rs +++ b/datafusion/catalog-listing/src/helpers.rs @@ -22,7 +22,7 @@ use std::sync::Arc; use super::ListingTableUrl; use super::PartitionedFile; -use crate::execution::context::SessionState; +use datafusion_catalog::Session; use datafusion_common::internal_err; use datafusion_common::{HashMap, Result, ScalarValue}; use datafusion_expr::{BinaryExpr, Operator}; @@ -154,7 +154,7 @@ pub fn split_files( chunks } -struct Partition { +pub struct Partition { /// The path to the partition, including the table prefix path: Path, /// How many path segments below the table prefix `path` contains @@ -183,7 +183,7 @@ impl Partition { } /// Returns a recursive list of the partitions in `table_path` up to `max_depth` -async fn list_partitions( +pub async fn list_partitions( store: &dyn ObjectStore, table_path: &ListingTableUrl, max_depth: usize, @@ -364,7 +364,7 @@ fn populate_partition_values<'a>( } } -fn evaluate_partition_prefix<'a>( +pub fn evaluate_partition_prefix<'a>( partition_cols: &'a [(String, DataType)], filters: &'a [Expr], ) -> Option { @@ -405,7 +405,7 @@ fn evaluate_partition_prefix<'a>( /// `filters` should only contain expressions that can be evaluated /// using only the partition columns. pub async fn pruned_partition_list<'a>( - ctx: &'a SessionState, + ctx: &'a dyn Session, store: &'a dyn ObjectStore, table_path: &'a ListingTableUrl, filters: &'a [Expr], @@ -489,7 +489,7 @@ pub async fn pruned_partition_list<'a>( /// Extract the partition values for the given `file_path` (in the given `table_path`) /// associated to the partitions defined by `table_partition_cols` -fn parse_partitions_for_path<'a, I>( +pub fn parse_partitions_for_path<'a, I>( table_path: &ListingTableUrl, file_path: &'a Path, table_partition_cols: I, @@ -517,17 +517,36 @@ where } Some(part_values) } +/// Describe a partition as a (path, depth, files) tuple for easier assertions +pub fn describe_partition(partition: &Partition) -> (&str, usize, Vec<&str>) { + ( + partition.path.as_ref(), + partition.depth, + partition + .files + .as_ref() + .map(|f| f.iter().map(|f| f.location.filename().unwrap()).collect()) + .unwrap_or_default(), + ) +} #[cfg(test)] mod tests { + use async_trait::async_trait; + use datafusion_execution::config::SessionConfig; + use datafusion_execution::runtime_env::RuntimeEnv; + use futures::FutureExt; + use object_store::memory::InMemory; + use std::any::Any; use std::ops::Not; - - use futures::StreamExt; - - use crate::test::object_store::make_test_store_and_state; - use datafusion_expr::{case, col, lit, Expr}; + // use futures::StreamExt; use super::*; + use datafusion_expr::{ + case, col, lit, AggregateUDF, Expr, LogicalPlan, ScalarUDF, WindowUDF, + }; + use datafusion_physical_expr_common::physical_expr::PhysicalExpr; + use datafusion_physical_plan::ExecutionPlan; #[test] fn test_split_files() { @@ -578,7 +597,7 @@ mod tests { ]); let filter = Expr::eq(col("mypartition"), lit("val1")); let pruned = pruned_partition_list( - &state, + state.as_ref(), store.as_ref(), &ListingTableUrl::parse("file:///tablepath/").unwrap(), &[filter], @@ -603,7 +622,7 @@ mod tests { ]); let filter = Expr::eq(col("mypartition"), lit("val1")); let pruned = pruned_partition_list( - &state, + state.as_ref(), store.as_ref(), &ListingTableUrl::parse("file:///tablepath/").unwrap(), &[filter], @@ -643,7 +662,7 @@ mod tests { let filter1 = Expr::eq(col("part1"), lit("p1v2")); let filter2 = Expr::eq(col("part2"), lit("p2v1")); let pruned = pruned_partition_list( - &state, + state.as_ref(), store.as_ref(), &ListingTableUrl::parse("file:///tablepath/").unwrap(), &[filter1, filter2], @@ -680,19 +699,6 @@ mod tests { ); } - /// Describe a partition as a (path, depth, files) tuple for easier assertions - fn describe_partition(partition: &Partition) -> (&str, usize, Vec<&str>) { - ( - partition.path.as_ref(), - partition.depth, - partition - .files - .as_ref() - .map(|f| f.iter().map(|f| f.location.filename().unwrap()).collect()) - .unwrap_or_default(), - ) - } - #[tokio::test] async fn test_list_partition() { let (store, _) = make_test_store_and_state(&[ @@ -994,4 +1000,74 @@ mod tests { Some(Path::from("a=1970-01-05")), ); } + + pub fn make_test_store_and_state( + files: &[(&str, u64)], + ) -> (Arc, Arc) { + let memory = InMemory::new(); + + for (name, size) in files { + memory + .put(&Path::from(*name), vec![0; *size as usize].into()) + .now_or_never() + .unwrap() + .unwrap(); + } + + (Arc::new(memory), Arc::new(MockSession {})) + } + + struct MockSession {} + + #[async_trait] + impl Session for MockSession { + fn session_id(&self) -> &str { + unimplemented!() + } + + fn config(&self) -> &SessionConfig { + unimplemented!() + } + + async fn create_physical_plan( + &self, + _logical_plan: &LogicalPlan, + ) -> Result> { + unimplemented!() + } + + fn create_physical_expr( + &self, + _expr: Expr, + _df_schema: &DFSchema, + ) -> Result> { + unimplemented!() + } + + fn scalar_functions(&self) -> &std::collections::HashMap> { + unimplemented!() + } + + fn aggregate_functions( + &self, + ) -> &std::collections::HashMap> { + unimplemented!() + } + + fn window_functions(&self) -> &std::collections::HashMap> { + unimplemented!() + } + + fn runtime_env(&self) -> &Arc { + unimplemented!() + } + + fn execution_props(&self) -> &ExecutionProps { + unimplemented!() + } + + fn as_any(&self) -> &dyn Any { + unimplemented!() + } + } } diff --git a/datafusion/catalog-listing/src/mod.rs b/datafusion/catalog-listing/src/mod.rs new file mode 100644 index 000000000000..e952e39fd479 --- /dev/null +++ b/datafusion/catalog-listing/src/mod.rs @@ -0,0 +1,278 @@ +// 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. + +//! A table that uses the `ObjectStore` listing capability +//! to get the list of files to process. + +pub mod helpers; +pub mod url; + +use chrono::TimeZone; +use datafusion_common::Result; +use datafusion_common::{ScalarValue, Statistics}; +use futures::Stream; +use object_store::{path::Path, ObjectMeta}; +use std::pin::Pin; +use std::sync::Arc; + +pub use self::url::ListingTableUrl; + +/// Stream of files get listed from object store +pub type PartitionedFileStream = + Pin> + Send + Sync + 'static>>; + +/// Only scan a subset of Row Groups from the Parquet file whose data "midpoint" +/// lies within the [start, end) byte offsets. This option can be used to scan non-overlapping +/// sections of a Parquet file in parallel. +#[derive(Debug, Clone, PartialEq, Hash, Eq, PartialOrd, Ord)] +pub struct FileRange { + /// Range start + pub start: i64, + /// Range end + pub end: i64, +} + +impl FileRange { + /// returns true if this file range contains the specified offset + pub fn contains(&self, offset: i64) -> bool { + offset >= self.start && offset < self.end + } +} + +#[derive(Debug, Clone)] +/// A single file or part of a file that should be read, along with its schema, statistics +/// and partition column values that need to be appended to each row. +pub struct PartitionedFile { + /// Path for the file (e.g. URL, filesystem path, etc) + pub object_meta: ObjectMeta, + /// Values of partition columns to be appended to each row. + /// + /// These MUST have the same count, order, and type than the [`table_partition_cols`]. + /// + /// You may use [`wrap_partition_value_in_dict`] to wrap them if you have used [`wrap_partition_type_in_dict`] to wrap the column type. + /// + /// + /// [`wrap_partition_type_in_dict`]: https://github.com/apache/datafusion/blob/main/datafusion/core/src/datasource/physical_plan/file_scan_config.rs#L55 + /// [`wrap_partition_value_in_dict`]: https://github.com/apache/datafusion/blob/main/datafusion/core/src/datasource/physical_plan/file_scan_config.rs#L62 + /// [`table_partition_cols`]: https://github.com/apache/datafusion/blob/main/datafusion/core/src/datasource/file_format/options.rs#L190 + pub partition_values: Vec, + /// An optional file range for a more fine-grained parallel execution + pub range: Option, + /// Optional statistics that describe the data in this file if known. + /// + /// DataFusion relies on these statistics for planning (in particular to sort file groups), + /// so if they are incorrect, incorrect answers may result. + pub statistics: Option, + /// An optional field for user defined per object metadata + pub extensions: Option>, + /// The estimated size of the parquet metadata, in bytes + pub metadata_size_hint: Option, +} + +impl PartitionedFile { + /// Create a simple file without metadata or partition + pub fn new(path: impl Into, size: u64) -> Self { + Self { + object_meta: ObjectMeta { + location: Path::from(path.into()), + last_modified: chrono::Utc.timestamp_nanos(0), + size: size as usize, + e_tag: None, + version: None, + }, + partition_values: vec![], + range: None, + statistics: None, + extensions: None, + metadata_size_hint: None, + } + } + + /// Create a file range without metadata or partition + pub fn new_with_range(path: String, size: u64, start: i64, end: i64) -> Self { + Self { + object_meta: ObjectMeta { + location: Path::from(path), + last_modified: chrono::Utc.timestamp_nanos(0), + size: size as usize, + e_tag: None, + version: None, + }, + partition_values: vec![], + range: Some(FileRange { start, end }), + statistics: None, + extensions: None, + metadata_size_hint: None, + } + .with_range(start, end) + } + + /// Provide a hint to the size of the file metadata. If a hint is provided + /// the reader will try and fetch the last `size_hint` bytes of the parquet file optimistically. + /// Without an appropriate hint, two read may be required to fetch the metadata. + pub fn with_metadata_size_hint(mut self, metadata_size_hint: usize) -> Self { + self.metadata_size_hint = Some(metadata_size_hint); + self + } + + /// Return a file reference from the given path + pub fn from_path(path: String) -> Result { + let size = std::fs::metadata(path.clone())?.len(); + Ok(Self::new(path, size)) + } + + /// Return the path of this partitioned file + pub fn path(&self) -> &Path { + &self.object_meta.location + } + + /// Update the file to only scan the specified range (in bytes) + pub fn with_range(mut self, start: i64, end: i64) -> Self { + self.range = Some(FileRange { start, end }); + self + } + + /// Update the user defined extensions for this file. + /// + /// This can be used to pass reader specific information. + pub fn with_extensions( + mut self, + extensions: Arc, + ) -> Self { + self.extensions = Some(extensions); + self + } +} + +impl From for PartitionedFile { + fn from(object_meta: ObjectMeta) -> Self { + PartitionedFile { + object_meta, + partition_values: vec![], + range: None, + statistics: None, + extensions: None, + metadata_size_hint: None, + } + } +} + +#[cfg(test)] +mod tests { + use super::ListingTableUrl; + use datafusion_execution::object_store::{ + DefaultObjectStoreRegistry, ObjectStoreRegistry, + }; + use object_store::{local::LocalFileSystem, path::Path}; + use std::{ops::Not, sync::Arc}; + use url::Url; + + #[test] + fn test_object_store_listing_url() { + let listing = ListingTableUrl::parse("file:///").unwrap(); + let store = listing.object_store(); + assert_eq!(store.as_str(), "file:///"); + + let listing = ListingTableUrl::parse("s3://bucket/").unwrap(); + let store = listing.object_store(); + assert_eq!(store.as_str(), "s3://bucket/"); + } + + #[test] + fn test_get_store_hdfs() { + let sut = DefaultObjectStoreRegistry::default(); + let url = Url::parse("hdfs://localhost:8020").unwrap(); + sut.register_store(&url, Arc::new(LocalFileSystem::new())); + let url = ListingTableUrl::parse("hdfs://localhost:8020/key").unwrap(); + sut.get_store(url.as_ref()).unwrap(); + } + + #[test] + fn test_get_store_s3() { + let sut = DefaultObjectStoreRegistry::default(); + let url = Url::parse("s3://bucket/key").unwrap(); + sut.register_store(&url, Arc::new(LocalFileSystem::new())); + let url = ListingTableUrl::parse("s3://bucket/key").unwrap(); + sut.get_store(url.as_ref()).unwrap(); + } + + #[test] + fn test_get_store_file() { + let sut = DefaultObjectStoreRegistry::default(); + let url = ListingTableUrl::parse("file:///bucket/key").unwrap(); + sut.get_store(url.as_ref()).unwrap(); + } + + #[test] + fn test_get_store_local() { + let sut = DefaultObjectStoreRegistry::default(); + let url = ListingTableUrl::parse("../").unwrap(); + sut.get_store(url.as_ref()).unwrap(); + } + + #[test] + fn test_url_contains() { + let url = ListingTableUrl::parse("file:///var/data/mytable/").unwrap(); + + // standard case with default config + assert!(url.contains( + &Path::parse("/var/data/mytable/data.parquet").unwrap(), + true + )); + + // standard case with `ignore_subdirectory` set to false + assert!(url.contains( + &Path::parse("/var/data/mytable/data.parquet").unwrap(), + false + )); + + // as per documentation, when `ignore_subdirectory` is true, we should ignore files that aren't + // a direct child of the `url` + assert!(url + .contains( + &Path::parse("/var/data/mytable/mysubfolder/data.parquet").unwrap(), + true + ) + .not()); + + // when we set `ignore_subdirectory` to false, we should not ignore the file + assert!(url.contains( + &Path::parse("/var/data/mytable/mysubfolder/data.parquet").unwrap(), + false + )); + + // as above, `ignore_subdirectory` is false, so we include the file + assert!(url.contains( + &Path::parse("/var/data/mytable/year=2024/data.parquet").unwrap(), + false + )); + + // in this case, we include the file even when `ignore_subdirectory` is true because the + // path segment is a hive partition which doesn't count as a subdirectory for the purposes + // of `Url::contains` + assert!(url.contains( + &Path::parse("/var/data/mytable/year=2024/data.parquet").unwrap(), + true + )); + + // testing an empty path with default config + assert!(url.contains(&Path::parse("/var/data/mytable/").unwrap(), true)); + + // testing an empty path with `ignore_subdirectory` set to false + assert!(url.contains(&Path::parse("/var/data/mytable/").unwrap(), false)); + } +} diff --git a/datafusion/core/src/datasource/listing/url.rs b/datafusion/catalog-listing/src/url.rs similarity index 98% rename from datafusion/core/src/datasource/listing/url.rs rename to datafusion/catalog-listing/src/url.rs index 6fb536ca2f05..2e6415ba3b2b 100644 --- a/datafusion/core/src/datasource/listing/url.rs +++ b/datafusion/catalog-listing/src/url.rs @@ -15,10 +15,9 @@ // specific language governing permissions and limitations // under the License. -use crate::execution::context::SessionState; +use datafusion_catalog::Session; use datafusion_common::{DataFusionError, Result}; use datafusion_execution::object_store::ObjectStoreUrl; -use datafusion_optimizer::OptimizerConfig; use futures::stream::BoxStream; use futures::{StreamExt, TryStreamExt}; use glob::Pattern; @@ -194,7 +193,7 @@ impl ListingTableUrl { /// /// Examples: /// ```rust - /// use datafusion::datasource::listing::ListingTableUrl; + /// use datafusion_catalog_listing::ListingTableUrl; /// let url = ListingTableUrl::parse("file:///foo/bar.csv").unwrap(); /// assert_eq!(url.file_extension(), Some("csv")); /// let url = ListingTableUrl::parse("file:///foo/bar").unwrap(); @@ -216,7 +215,7 @@ impl ListingTableUrl { /// Strips the prefix of this [`ListingTableUrl`] from the provided path, returning /// an iterator of the remaining path segments - pub(crate) fn strip_prefix<'a, 'b: 'a>( + pub fn strip_prefix<'a, 'b: 'a>( &'a self, path: &'b Path, ) -> Option + 'a> { @@ -230,11 +229,11 @@ impl ListingTableUrl { /// List all files identified by this [`ListingTableUrl`] for the provided `file_extension` pub async fn list_all_files<'a>( &'a self, - ctx: &'a SessionState, + ctx: &'a dyn Session, store: &'a dyn ObjectStore, file_extension: &'a str, ) -> Result>> { - let exec_options = &ctx.options().execution; + let exec_options = &ctx.config_options().execution; let ignore_subdirectory = exec_options.listing_table_ignore_subdirectory; // If the prefix is a file, use a head request, otherwise list let list = match self.is_collection() { @@ -325,6 +324,7 @@ impl std::fmt::Display for ListingTableUrl { } } +#[cfg(not(target_arch = "wasm32"))] const GLOB_START_CHARS: [char; 3] = ['?', '*', '[']; /// Splits `path` at the first path segment containing a glob expression, returning @@ -333,6 +333,7 @@ const GLOB_START_CHARS: [char; 3] = ['?', '*', '[']; /// Path delimiters are determined using [`std::path::is_separator`] which /// permits `/` as a path delimiter even on Windows platforms. /// +#[cfg(not(target_arch = "wasm32"))] fn split_glob_expression(path: &str) -> Option<(&str, &str)> { let mut last_separator = 0; diff --git a/datafusion/common/Cargo.toml b/datafusion/common/Cargo.toml index fe6d652be700..1050b376be8a 100644 --- a/datafusion/common/Cargo.toml +++ b/datafusion/common/Cargo.toml @@ -52,7 +52,6 @@ apache-avro = { version = "0.17", default-features = false, features = [ ], optional = true } arrow = { workspace = true } arrow-array = { workspace = true } -arrow-buffer = { workspace = true } arrow-ipc = { workspace = true } arrow-schema = { workspace = true } base64 = "0.22.1" diff --git a/datafusion/common/src/column.rs b/datafusion/common/src/column.rs index 9973a72e7bc0..05e2dff0bd43 100644 --- a/datafusion/common/src/column.rs +++ b/datafusion/common/src/column.rs @@ -84,7 +84,11 @@ impl Column { } } - fn from_idents(idents: &mut Vec) -> Option { + /// Create a Column from multiple normalized identifiers + /// + /// For example, `foo.bar` would be represented as a two element vector + /// `["foo", "bar"]` + fn from_idents(mut idents: Vec) -> Option { let (relation, name) = match idents.len() { 1 => (None, idents.remove(0)), 2 => ( @@ -126,7 +130,7 @@ impl Column { /// where `"foo.BAR"` would be parsed to a reference to column named `foo.BAR` pub fn from_qualified_name(flat_name: impl Into) -> Self { let flat_name = flat_name.into(); - Self::from_idents(&mut parse_identifiers_normalized(&flat_name, false)).unwrap_or( + Self::from_idents(parse_identifiers_normalized(&flat_name, false)).unwrap_or( Self { relation: None, name: flat_name, @@ -138,7 +142,7 @@ impl Column { /// Deserialize a fully qualified name string into a column preserving column text case pub fn from_qualified_name_ignore_case(flat_name: impl Into) -> Self { let flat_name = flat_name.into(); - Self::from_idents(&mut parse_identifiers_normalized(&flat_name, true)).unwrap_or( + Self::from_idents(parse_identifiers_normalized(&flat_name, true)).unwrap_or( Self { relation: None, name: flat_name, diff --git a/datafusion/common/src/config.rs b/datafusion/common/src/config.rs index 320417c35a79..3410cbff3b9d 100644 --- a/datafusion/common/src/config.rs +++ b/datafusion/common/src/config.rs @@ -310,7 +310,7 @@ config_namespace! { /// /// This is used to workaround bugs in the planner that are now caught by /// the new schema verification step. - pub skip_physical_aggregate_schema_check: bool, default = false + pub skip_physical_aggregate_schema_check: bool, default = true /// Specifies the reserved memory for each spillable sort operation to /// facilitate an in-memory merge. diff --git a/datafusion/common/src/hash_utils.rs b/datafusion/common/src/hash_utils.rs index 0d1d93acf1fc..e78d42257b9c 100644 --- a/datafusion/common/src/hash_utils.rs +++ b/datafusion/common/src/hash_utils.rs @@ -21,12 +21,11 @@ use std::sync::Arc; use ahash::RandomState; +use arrow::array::types::{IntervalDayTime, IntervalMonthDayNano}; use arrow::array::*; use arrow::datatypes::*; #[cfg(not(feature = "force_hash_collisions"))] use arrow::{downcast_dictionary_array, downcast_primitive_array}; -use arrow_buffer::IntervalDayTime; -use arrow_buffer::IntervalMonthDayNano; #[cfg(not(feature = "force_hash_collisions"))] use crate::cast::{ @@ -700,7 +699,7 @@ mod tests { // Tests actual values of hashes, which are different if forcing collisions #[cfg(not(feature = "force_hash_collisions"))] fn create_hashes_for_struct_arrays() { - use arrow_buffer::Buffer; + use arrow::buffer::Buffer; let boolarr = Arc::new(BooleanArray::from(vec![ false, false, true, true, true, true, diff --git a/datafusion/common/src/scalar/mod.rs b/datafusion/common/src/scalar/mod.rs index 9d3429b67796..6cd6a43941c8 100644 --- a/datafusion/common/src/scalar/mod.rs +++ b/datafusion/common/src/scalar/mod.rs @@ -40,6 +40,8 @@ use crate::cast::{ use crate::error::{DataFusionError, Result, _exec_err, _internal_err, _not_impl_err}; use crate::hash_utils::create_hashes; use crate::utils::SingleRowListArrayBuilder; +use arrow::array::types::{IntervalDayTime, IntervalMonthDayNano}; +use arrow::buffer::ScalarBuffer; use arrow::compute::kernels::numeric::*; use arrow::util::display::{array_value_to_string, ArrayFormatter, FormatOptions}; use arrow::{ @@ -54,7 +56,6 @@ use arrow::{ UInt16Type, UInt32Type, UInt64Type, UInt8Type, DECIMAL128_MAX_PRECISION, }, }; -use arrow_buffer::{IntervalDayTime, IntervalMonthDayNano, ScalarBuffer}; use arrow_schema::{UnionFields, UnionMode}; use crate::format::DEFAULT_CAST_OPTIONS; @@ -3958,12 +3959,11 @@ mod tests { }; use crate::assert_batches_eq; - use arrow::buffer::OffsetBuffer; + use arrow::array::{types::Float64Type, NullBufferBuilder}; + use arrow::buffer::{Buffer, OffsetBuffer}; use arrow::compute::{is_null, kernels}; use arrow::error::ArrowError; use arrow::util::pretty::pretty_format_columns; - use arrow_array::types::Float64Type; - use arrow_buffer::{Buffer, NullBufferBuilder}; use arrow_schema::Fields; use chrono::NaiveDate; use rand::Rng; diff --git a/datafusion/core/Cargo.toml b/datafusion/core/Cargo.toml index b708c18f5b75..c90e5ce0b5b6 100644 --- a/datafusion/core/Cargo.toml +++ b/datafusion/core/Cargo.toml @@ -100,6 +100,7 @@ bytes = { workspace = true } bzip2 = { version = "0.5.0", optional = true } chrono = { workspace = true } datafusion-catalog = { workspace = true } +datafusion-catalog-listing = { workspace = true } datafusion-common = { workspace = true, features = ["object_store"] } datafusion-common-runtime = { workspace = true } datafusion-execution = { workspace = true } @@ -136,7 +137,6 @@ xz2 = { version = "0.1", optional = true, features = ["static"] } zstd = { version = "0.13", optional = true, default-features = false } [dev-dependencies] -arrow-buffer = { workspace = true } async-trait = { workspace = true } criterion = { version = "0.5", features = ["async_tokio"] } ctor = { workspace = true } diff --git a/datafusion/core/src/datasource/data_source.rs b/datafusion/core/src/datasource/data_source.rs index 0219397afae0..3984ffa64c76 100644 --- a/datafusion/core/src/datasource/data_source.rs +++ b/datafusion/core/src/datasource/data_source.rs @@ -32,6 +32,8 @@ use datafusion_physical_plan::DisplayFormatType; use object_store::ObjectStore; /// Common behaviors that every file format needs to implement. +/// +/// See initialization examples on `ParquetSource`, `CsvSource` pub trait FileSource: Send + Sync { /// Creates a `dyn FileOpener` based on given parameters fn create_file_opener( @@ -54,51 +56,10 @@ pub trait FileSource: Send + Sync { fn metrics(&self) -> &ExecutionPlanMetricsSet; /// Return projected statistics fn statistics(&self) -> datafusion_common::Result; - /// Returns the file type such as Arrow, Avro, Parquet, ... - fn file_type(&self) -> FileType; + /// String representation of file source such as "csv", "json", "parquet" + fn file_type(&self) -> &str; /// Format FileType specific information fn fmt_extra(&self, _t: DisplayFormatType, _f: &mut Formatter) -> fmt::Result { Ok(()) } } - -/// Determines file types -pub enum FileType { - /// Arrow File - Arrow, - /// Avro File - Avro, - /// CSV File - Csv, - /// JSON File - Json, - /// Parquet File - Parquet, -} - -impl FileType { - pub(crate) fn to_str(&self) -> &str { - match self { - FileType::Arrow => "arrow", - FileType::Avro => "avro", - FileType::Csv => "csv", - FileType::Json => "json", - FileType::Parquet => "parquet", - } - } - - /// Is the file type avro? - pub fn is_avro(&self) -> bool { - matches!(self, FileType::Avro) - } - - /// Is the file type csv? - pub fn is_csv(&self) -> bool { - matches!(self, FileType::Csv) - } - - /// Is the file type parquet? - pub fn is_parquet(&self) -> bool { - matches!(self, FileType::Parquet) - } -} diff --git a/datafusion/core/src/datasource/file_format/mod.rs b/datafusion/core/src/datasource/file_format/mod.rs index 8c749f04a8f9..bb7e42d3f854 100644 --- a/datafusion/core/src/datasource/file_format/mod.rs +++ b/datafusion/core/src/datasource/file_format/mod.rs @@ -429,6 +429,7 @@ pub fn transform_schema_to_view(schema: &Schema) -> Schema { } /// Coerces the file schema if the table schema uses a view type. +#[cfg(not(target_arch = "wasm32"))] pub(crate) fn coerce_file_schema_to_view_type( table_schema: &Schema, file_schema: &Schema, @@ -493,6 +494,7 @@ pub fn transform_binary_to_string(schema: &Schema) -> Schema { /// If the table schema uses a string type, coerce the file schema to use a string type. /// /// See [parquet::ParquetFormat::binary_as_string] for details +#[cfg(not(target_arch = "wasm32"))] pub(crate) fn coerce_file_schema_to_string_type( table_schema: &Schema, file_schema: &Schema, diff --git a/datafusion/core/src/datasource/file_format/parquet.rs b/datafusion/core/src/datasource/file_format/parquet.rs index 4ed7946d7d4d..70121c96ae16 100644 --- a/datafusion/core/src/datasource/file_format/parquet.rs +++ b/datafusion/core/src/datasource/file_format/parquet.rs @@ -414,13 +414,15 @@ impl FileFormat for ParquetFormat { metadata_size_hint = Some(metadata); } - let source = Arc::new(ParquetSource::new( - Arc::clone(&conf.file_schema), - predicate, - metadata_size_hint, - self.options.clone(), - )); - conf = conf.with_source(source); + let mut source = ParquetSource::new(self.options.clone()); + + if let Some(predicate) = predicate { + source = source.with_predicate(Arc::clone(&conf.file_schema), predicate); + } + if let Some(metadata_size_hint) = metadata_size_hint { + source = source.with_metadata_size_hint(metadata_size_hint) + } + conf = conf.with_source(Arc::new(source)); Ok(conf.new_exec()) } diff --git a/datafusion/core/src/datasource/listing/mod.rs b/datafusion/core/src/datasource/listing/mod.rs index f11653ce1e52..39323b993d45 100644 --- a/datafusion/core/src/datasource/listing/mod.rs +++ b/datafusion/core/src/datasource/listing/mod.rs @@ -18,263 +18,6 @@ //! A table that uses the `ObjectStore` listing capability //! to get the list of files to process. -mod helpers; mod table; -mod url; - -use chrono::TimeZone; -use datafusion_common::Result; -use datafusion_common::{ScalarValue, Statistics}; -use futures::Stream; -use object_store::{path::Path, ObjectMeta}; -use std::pin::Pin; -use std::sync::Arc; - -pub use self::url::ListingTableUrl; +pub use datafusion_catalog_listing::*; pub use table::{ListingOptions, ListingTable, ListingTableConfig}; - -/// Stream of files get listed from object store -pub type PartitionedFileStream = - Pin> + Send + Sync + 'static>>; - -/// Only scan a subset of Row Groups from the Parquet file whose data "midpoint" -/// lies within the [start, end) byte offsets. This option can be used to scan non-overlapping -/// sections of a Parquet file in parallel. -#[derive(Debug, Clone, PartialEq, Hash, Eq, PartialOrd, Ord)] -pub struct FileRange { - /// Range start - pub start: i64, - /// Range end - pub end: i64, -} - -impl FileRange { - /// returns true if this file range contains the specified offset - pub fn contains(&self, offset: i64) -> bool { - offset >= self.start && offset < self.end - } -} - -#[derive(Debug, Clone)] -/// A single file or part of a file that should be read, along with its schema, statistics -/// and partition column values that need to be appended to each row. -pub struct PartitionedFile { - /// Path for the file (e.g. URL, filesystem path, etc) - pub object_meta: ObjectMeta, - /// Values of partition columns to be appended to each row. - /// - /// These MUST have the same count, order, and type than the [`table_partition_cols`]. - /// - /// You may use [`wrap_partition_value_in_dict`] to wrap them if you have used [`wrap_partition_type_in_dict`] to wrap the column type. - /// - /// - /// [`wrap_partition_type_in_dict`]: crate::datasource::physical_plan::wrap_partition_type_in_dict - /// [`wrap_partition_value_in_dict`]: crate::datasource::physical_plan::wrap_partition_value_in_dict - /// [`table_partition_cols`]: table::ListingOptions::table_partition_cols - pub partition_values: Vec, - /// An optional file range for a more fine-grained parallel execution - pub range: Option, - /// Optional statistics that describe the data in this file if known. - /// - /// DataFusion relies on these statistics for planning (in particular to sort file groups), - /// so if they are incorrect, incorrect answers may result. - pub statistics: Option, - /// An optional field for user defined per object metadata - pub extensions: Option>, - /// The estimated size of the parquet metadata, in bytes - pub metadata_size_hint: Option, -} - -impl PartitionedFile { - /// Create a simple file without metadata or partition - pub fn new(path: impl Into, size: u64) -> Self { - Self { - object_meta: ObjectMeta { - location: Path::from(path.into()), - last_modified: chrono::Utc.timestamp_nanos(0), - size: size as usize, - e_tag: None, - version: None, - }, - partition_values: vec![], - range: None, - statistics: None, - extensions: None, - metadata_size_hint: None, - } - } - - /// Create a file range without metadata or partition - pub fn new_with_range(path: String, size: u64, start: i64, end: i64) -> Self { - Self { - object_meta: ObjectMeta { - location: Path::from(path), - last_modified: chrono::Utc.timestamp_nanos(0), - size: size as usize, - e_tag: None, - version: None, - }, - partition_values: vec![], - range: Some(FileRange { start, end }), - statistics: None, - extensions: None, - metadata_size_hint: None, - } - .with_range(start, end) - } - - /// Provide a hint to the size of the file metadata. If a hint is provided - /// the reader will try and fetch the last `size_hint` bytes of the parquet file optimistically. - /// Without an appropriate hint, two read may be required to fetch the metadata. - pub fn with_metadata_size_hint(mut self, metadata_size_hint: usize) -> Self { - self.metadata_size_hint = Some(metadata_size_hint); - self - } - - /// Return a file reference from the given path - pub fn from_path(path: String) -> Result { - let size = std::fs::metadata(path.clone())?.len(); - Ok(Self::new(path, size)) - } - - /// Return the path of this partitioned file - pub fn path(&self) -> &Path { - &self.object_meta.location - } - - /// Update the file to only scan the specified range (in bytes) - pub fn with_range(mut self, start: i64, end: i64) -> Self { - self.range = Some(FileRange { start, end }); - self - } - - /// Update the user defined extensions for this file. - /// - /// This can be used to pass reader specific information. - pub fn with_extensions( - mut self, - extensions: Arc, - ) -> Self { - self.extensions = Some(extensions); - self - } -} - -impl From for PartitionedFile { - fn from(object_meta: ObjectMeta) -> Self { - PartitionedFile { - object_meta, - partition_values: vec![], - range: None, - statistics: None, - extensions: None, - metadata_size_hint: None, - } - } -} - -#[cfg(test)] -mod tests { - use super::ListingTableUrl; - use datafusion_execution::object_store::{ - DefaultObjectStoreRegistry, ObjectStoreRegistry, - }; - use object_store::{local::LocalFileSystem, path::Path}; - use std::{ops::Not, sync::Arc}; - use url::Url; - - #[test] - fn test_object_store_listing_url() { - let listing = ListingTableUrl::parse("file:///").unwrap(); - let store = listing.object_store(); - assert_eq!(store.as_str(), "file:///"); - - let listing = ListingTableUrl::parse("s3://bucket/").unwrap(); - let store = listing.object_store(); - assert_eq!(store.as_str(), "s3://bucket/"); - } - - #[test] - fn test_get_store_hdfs() { - let sut = DefaultObjectStoreRegistry::default(); - let url = Url::parse("hdfs://localhost:8020").unwrap(); - sut.register_store(&url, Arc::new(LocalFileSystem::new())); - let url = ListingTableUrl::parse("hdfs://localhost:8020/key").unwrap(); - sut.get_store(url.as_ref()).unwrap(); - } - - #[test] - fn test_get_store_s3() { - let sut = DefaultObjectStoreRegistry::default(); - let url = Url::parse("s3://bucket/key").unwrap(); - sut.register_store(&url, Arc::new(LocalFileSystem::new())); - let url = ListingTableUrl::parse("s3://bucket/key").unwrap(); - sut.get_store(url.as_ref()).unwrap(); - } - - #[test] - fn test_get_store_file() { - let sut = DefaultObjectStoreRegistry::default(); - let url = ListingTableUrl::parse("file:///bucket/key").unwrap(); - sut.get_store(url.as_ref()).unwrap(); - } - - #[test] - fn test_get_store_local() { - let sut = DefaultObjectStoreRegistry::default(); - let url = ListingTableUrl::parse("../").unwrap(); - sut.get_store(url.as_ref()).unwrap(); - } - - #[test] - fn test_url_contains() { - let url = ListingTableUrl::parse("file:///var/data/mytable/").unwrap(); - - // standard case with default config - assert!(url.contains( - &Path::parse("/var/data/mytable/data.parquet").unwrap(), - true - )); - - // standard case with `ignore_subdirectory` set to false - assert!(url.contains( - &Path::parse("/var/data/mytable/data.parquet").unwrap(), - false - )); - - // as per documentation, when `ignore_subdirectory` is true, we should ignore files that aren't - // a direct child of the `url` - assert!(url - .contains( - &Path::parse("/var/data/mytable/mysubfolder/data.parquet").unwrap(), - true - ) - .not()); - - // when we set `ignore_subdirectory` to false, we should not ignore the file - assert!(url.contains( - &Path::parse("/var/data/mytable/mysubfolder/data.parquet").unwrap(), - false - )); - - // as above, `ignore_subdirectory` is false, so we include the file - assert!(url.contains( - &Path::parse("/var/data/mytable/year=2024/data.parquet").unwrap(), - false - )); - - // in this case, we include the file even when `ignore_subdirectory` is true because the - // path segment is a hive partition which doesn't count as a subdirectory for the purposes - // of `Url::contains` - assert!(url.contains( - &Path::parse("/var/data/mytable/year=2024/data.parquet").unwrap(), - true - )); - - // testing an empty path with default config - assert!(url.contains(&Path::parse("/var/data/mytable/").unwrap(), true)); - - // testing an empty path with `ignore_subdirectory` set to false - assert!(url.contains(&Path::parse("/var/data/mytable/").unwrap(), false)); - } -} diff --git a/datafusion/core/src/datasource/physical_plan/arrow_file.rs b/datafusion/core/src/datasource/physical_plan/arrow_file.rs index 32a4ebceb14c..5c5dce46e936 100644 --- a/datafusion/core/src/datasource/physical_plan/arrow_file.rs +++ b/datafusion/core/src/datasource/physical_plan/arrow_file.rs @@ -20,7 +20,7 @@ use std::any::Any; use std::sync::Arc; -use crate::datasource::data_source::{FileSource, FileType}; +use crate::datasource::data_source::FileSource; use crate::datasource::listing::PartitionedFile; use crate::datasource::physical_plan::{ FileMeta, FileOpenFuture, FileOpener, FileScanConfig, JsonSource, @@ -253,8 +253,8 @@ impl FileSource for ArrowSource { .expect("projected_statistics must be set")) } - fn file_type(&self) -> FileType { - FileType::Arrow + fn file_type(&self) -> &str { + "arrow" } } diff --git a/datafusion/core/src/datasource/physical_plan/avro.rs b/datafusion/core/src/datasource/physical_plan/avro.rs index 720bde0e8e97..ee3f7071c11f 100644 --- a/datafusion/core/src/datasource/physical_plan/avro.rs +++ b/datafusion/core/src/datasource/physical_plan/avro.rs @@ -24,7 +24,7 @@ use std::sync::Arc; use super::{FileOpener, FileScanConfig}; #[cfg(feature = "avro")] use crate::datasource::avro_to_arrow::Reader as AvroReader; -use crate::datasource::data_source::{FileSource, FileType}; +use crate::datasource::data_source::FileSource; use crate::error::Result; use arrow::datatypes::SchemaRef; @@ -252,8 +252,8 @@ impl FileSource for AvroSource { .expect("projected_statistics must be set")) } - fn file_type(&self) -> FileType { - FileType::Avro + fn file_type(&self) -> &str { + "avro" } } diff --git a/datafusion/core/src/datasource/physical_plan/csv.rs b/datafusion/core/src/datasource/physical_plan/csv.rs index 404057b2515d..461ea94b9a23 100644 --- a/datafusion/core/src/datasource/physical_plan/csv.rs +++ b/datafusion/core/src/datasource/physical_plan/csv.rs @@ -24,7 +24,7 @@ use std::sync::Arc; use std::task::Poll; use super::{calculate_range, FileScanConfig, RangeCalculation}; -use crate::datasource::data_source::{FileSource, FileType}; +use crate::datasource::data_source::FileSource; use crate::datasource::file_format::file_compression_type::FileCompressionType; use crate::datasource::file_format::{deserialize_stream, DecoderDeserializer}; use crate::datasource::listing::{FileRange, ListingTableUrl, PartitionedFile}; @@ -609,8 +609,8 @@ impl FileSource for CsvSource { .clone() .expect("projected_statistics must be set")) } - fn file_type(&self) -> FileType { - FileType::Csv + fn file_type(&self) -> &str { + "csv" } fn fmt_extra(&self, _t: DisplayFormatType, f: &mut fmt::Formatter) -> fmt::Result { write!(f, ", has_header={}", self.has_header) 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 3c6fcf6df7cd..652632c31554 100644 --- a/datafusion/core/src/datasource/physical_plan/file_scan_config.rs +++ b/datafusion/core/src/datasource/physical_plan/file_scan_config.rs @@ -19,8 +19,8 @@ //! file sources. use super::{ - get_projected_output_ordering, statistics::MinMaxStatistics, FileGroupPartitioner, - FileGroupsDisplay, FileStream, + get_projected_output_ordering, statistics::MinMaxStatistics, AvroSource, + FileGroupPartitioner, FileGroupsDisplay, FileStream, }; use crate::datasource::file_format::file_compression_type::FileCompressionType; use crate::datasource::{listing::PartitionedFile, object_store::ObjectStoreUrl}; @@ -590,7 +590,7 @@ impl FileScanConfig { /// Write the data_type based on file_source fn fmt_file_source(&self, t: DisplayFormatType, f: &mut Formatter) -> fmt::Result { - write!(f, ", file_type={}", self.source.file_type().to_str())?; + write!(f, ", file_type={}", self.source.file_type())?; self.source.fmt_extra(t, f) } @@ -602,7 +602,7 @@ impl FileScanConfig { fn supports_repartition(&self) -> bool { !(self.file_compression_type.is_compressed() || self.new_lines_in_values - || self.source.file_type().is_avro()) + || self.source.as_any().downcast_ref::().is_some()) } } diff --git a/datafusion/core/src/datasource/physical_plan/json.rs b/datafusion/core/src/datasource/physical_plan/json.rs index 4c5bb3c1aede..f581f5cd0b4a 100644 --- a/datafusion/core/src/datasource/physical_plan/json.rs +++ b/datafusion/core/src/datasource/physical_plan/json.rs @@ -23,7 +23,7 @@ use std::sync::Arc; use std::task::Poll; use super::{calculate_range, FileScanConfig, RangeCalculation}; -use crate::datasource::data_source::{FileSource, FileType}; +use crate::datasource::data_source::FileSource; use crate::datasource::file_format::file_compression_type::FileCompressionType; use crate::datasource::file_format::{deserialize_stream, DecoderDeserializer}; use crate::datasource::listing::{ListingTableUrl, PartitionedFile}; @@ -310,8 +310,8 @@ impl FileSource for JsonSource { .expect("projected_statistics must be set to call")) } - fn file_type(&self) -> FileType { - FileType::Json + fn file_type(&self) -> &str { + "json" } } diff --git a/datafusion/core/src/datasource/physical_plan/parquet/mod.rs b/datafusion/core/src/datasource/physical_plan/parquet/mod.rs index 8fd9f94b27bf..6b1cb23ac1f7 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet/mod.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet/mod.rs @@ -87,9 +87,15 @@ impl From for ParquetExecBuilder { } } -/// [`ParquetExecBuilder`], builder for [`ParquetExec`]. +/// [`ParquetExecBuilder`], deprecated builder for [`ParquetExec`]. /// -/// See example on [`ParquetExec`]. +/// ParquetExec is replaced with `DataSourceExec` and it includes `ParquetSource` +/// +/// See example on [`ParquetSource`]. +#[deprecated( + since = "46.0.0", + note = "use DataSourceExec with ParquetSource instead" +)] #[allow(unused, deprecated)] pub struct ParquetExecBuilder { file_scan_config: FileScanConfig, @@ -211,12 +217,14 @@ impl ParquetExecBuilder { parquet_file_reader_factory, schema_adapter_factory, } = self; - let mut parquet = ParquetSource::new( - Arc::clone(&file_scan_config.file_schema), - predicate.clone(), - metadata_size_hint, - table_parquet_options, - ); + let mut parquet = ParquetSource::new(table_parquet_options); + if let Some(predicate) = predicate.clone() { + parquet = parquet + .with_predicate(Arc::clone(&file_scan_config.file_schema), predicate); + } + if let Some(metadata_size_hint) = metadata_size_hint { + parquet = parquet.with_metadata_size_hint(metadata_size_hint) + } if let Some(parquet_reader_factory) = parquet_file_reader_factory { parquet = parquet.with_parquet_file_reader_factory(parquet_reader_factory) } @@ -676,12 +684,10 @@ mod tests { // set up predicate (this is normally done by a layer higher up) let predicate = predicate.map(|p| logical2physical(&p, &file_schema)); - let mut source = ParquetSource::new( - Arc::clone(&file_schema), - predicate, - None, - TableParquetOptions::default(), - ); + let mut source = ParquetSource::default(); + if let Some(predicate) = predicate { + source = source.with_predicate(Arc::clone(&file_schema), predicate); + } if pushdown_predicate { source = source diff --git a/datafusion/core/src/datasource/physical_plan/parquet/source.rs b/datafusion/core/src/datasource/physical_plan/parquet/source.rs index 883b47121951..ab7c9f9e6ff9 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet/source.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet/source.rs @@ -20,7 +20,7 @@ use std::any::Any; use std::fmt::Formatter; use std::sync::Arc; -use crate::datasource::data_source::{FileSource, FileType}; +use crate::datasource::data_source::FileSource; use crate::datasource::physical_plan::parquet::opener::ParquetOpener; use crate::datasource::physical_plan::parquet::page_filter::PagePruningAccessPlanFilter; use crate::datasource::physical_plan::parquet::DefaultParquetFileReaderFactory; @@ -88,12 +88,8 @@ use object_store::ObjectStore; /// # let object_store_url = ObjectStoreUrl::local_filesystem(); /// # let predicate = lit(true); /// let source = Arc::new( -/// ParquetSource::new( -/// Arc::clone(&file_schema), -/// Some(predicate), -/// None, -/// TableParquetOptions::default() -/// ) +/// ParquetSource::default() +/// .with_predicate(Arc::clone(&file_schema), predicate) /// ); /// // Create a DataSourceExec for reading `file1.parquet` with a file size of 100MB /// let file_scan_config = FileScanConfig::new(object_store_url, file_schema, source) @@ -277,64 +273,10 @@ pub struct ParquetSource { } impl ParquetSource { - /// Initialize a ParquetSource, if default values are going to be used, - /// use `ParguetConfig::default()` instead - pub fn new( - file_schema: Arc, - predicate: Option>, - metadata_size_hint: Option, - table_parquet_options: TableParquetOptions, - ) -> Self { - debug!("Creating ParquetSource, schema: {:?}, predicate: {:?}, metadata_size_hint: {:?}", file_schema, predicate, metadata_size_hint); - - let metrics = ExecutionPlanMetricsSet::new(); - let predicate_creation_errors = - MetricBuilder::new(&metrics).global_counter("num_predicate_creation_errors"); - - let mut conf = ParquetSource::new_with_options(table_parquet_options); - conf.with_metrics(metrics); - if let Some(predicate) = predicate.clone() { - conf = conf.with_predicate(predicate); - } - - if let Some(metadata_size_hint) = metadata_size_hint { - conf = conf.with_metadata_size_hint(metadata_size_hint); - } - - let pruning_predicate = predicate - .clone() - .and_then(|predicate_expr| { - match PruningPredicate::try_new(predicate_expr, Arc::clone(&file_schema)) - { - Ok(pruning_predicate) => Some(Arc::new(pruning_predicate)), - Err(e) => { - debug!("Could not create pruning predicate for: {e}"); - predicate_creation_errors.add(1); - None - } - } - }) - .filter(|p| !p.always_true()); - if let Some(pruning_predicate) = pruning_predicate { - conf = conf.with_pruning_predicate(pruning_predicate); - } - - let page_pruning_predicate = predicate - .as_ref() - .map(|predicate_expr| { - PagePruningAccessPlanFilter::new(predicate_expr, Arc::clone(&file_schema)) - }) - .map(Arc::new); - if let Some(page_pruning_predicate) = page_pruning_predicate { - conf = conf.with_page_pruning_predicate(page_pruning_predicate); - } - - conf - } - - /// Create a new builder to read the data specified in the file scan + /// Create a new ParquetSource to read the data specified in the file scan /// configuration with the provided `TableParquetOptions`. - pub fn new_with_options(table_parquet_options: TableParquetOptions) -> Self { + /// if default values are going to be used, use `ParguetConfig::default()` instead + pub fn new(table_parquet_options: TableParquetOptions) -> Self { Self { table_parquet_options, ..Self::default() @@ -358,24 +300,40 @@ impl ParquetSource { conf } - fn with_predicate(&self, predicate: Arc) -> Self { - let mut conf = self.clone(); - conf.predicate = Some(predicate); - conf - } - - fn with_pruning_predicate(&self, pruning_predicate: Arc) -> Self { - let mut conf = self.clone(); - conf.pruning_predicate = Some(pruning_predicate); - conf - } - - fn with_page_pruning_predicate( + /// Set predicate information, also sets pruning_predicate and page_pruning_predicate attributes + pub fn with_predicate( &self, - page_pruning_predicate: Arc, + file_schema: Arc, + predicate: Arc, ) -> Self { let mut conf = self.clone(); + + let metrics = ExecutionPlanMetricsSet::new(); + let predicate_creation_errors = + MetricBuilder::new(&metrics).global_counter("num_predicate_creation_errors"); + + conf.with_metrics(metrics); + conf.predicate = Some(Arc::clone(&predicate)); + + match PruningPredicate::try_new(Arc::clone(&predicate), Arc::clone(&file_schema)) + { + Ok(pruning_predicate) => { + if !pruning_predicate.always_true() { + conf.pruning_predicate = Some(Arc::new(pruning_predicate)); + } + } + Err(e) => { + debug!("Could not create pruning predicate for: {e}"); + predicate_creation_errors.add(1); + } + }; + + let page_pruning_predicate = Arc::new(PagePruningAccessPlanFilter::new( + &predicate, + Arc::clone(&file_schema), + )); conf.page_pruning_predicate = Some(page_pruning_predicate); + conf } @@ -594,8 +552,8 @@ impl FileSource for ParquetSource { } } - fn file_type(&self) -> FileType { - FileType::Parquet + fn file_type(&self) -> &str { + "parquet" } fn fmt_extra(&self, t: DisplayFormatType, f: &mut Formatter) -> std::fmt::Result { diff --git a/datafusion/core/src/lib.rs b/datafusion/core/src/lib.rs index 2d3c37b10d33..9d42580178a4 100644 --- a/datafusion/core/src/lib.rs +++ b/datafusion/core/src/lib.rs @@ -809,8 +809,9 @@ pub mod variable { pub use datafusion_expr::var_provider::{VarProvider, VarType}; } -#[cfg(test)] +#[cfg(not(target_arch = "wasm32"))] pub mod test; + pub mod test_util; #[cfg(doctest)] diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index cee3acc08dae..04524967f481 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -692,6 +692,9 @@ impl DefaultPhysicalPlanner { differences.push(format!("field nullability at index {} [{}]: (physical) {} vs (logical) {}", i, physical_field.name(), physical_field.is_nullable(), logical_field.is_nullable())); } } + + log::warn!("Physical input schema should be the same as the one converted from logical input schema, but did not match for logical plan:\n{}", input.display_indent()); + return internal_err!("Physical input schema should be the same as the one converted from logical input schema. Differences: {}", differences .iter() .map(|s| format!("\n\t- {}", s)) diff --git a/datafusion/core/src/test/mod.rs b/datafusion/core/src/test/mod.rs index 383efec498c7..f2fef06c8f30 100644 --- a/datafusion/core/src/test/mod.rs +++ b/datafusion/core/src/test/mod.rs @@ -63,7 +63,7 @@ pub fn create_table_dual() -> Arc { Field::new("name", DataType::Utf8, false), ])); let batch = RecordBatch::try_new( - dual_schema.clone(), + Arc::::clone(&dual_schema), vec![ Arc::new(Int32Array::from(vec![1])), Arc::new(array::StringArray::from(vec!["a"])), @@ -242,7 +242,7 @@ pub fn table_with_sequence( let schema = Arc::new(Schema::new(vec![Field::new("i", DataType::Int32, true)])); let arr = Arc::new(Int32Array::from((seq_start..=seq_end).collect::>())); let partitions = vec![vec![RecordBatch::try_new( - schema.clone(), + Arc::::clone(&schema), vec![arr as ArrayRef], )?]]; Ok(Arc::new(MemTable::try_new(schema, partitions)?)) diff --git a/datafusion/core/src/test_util/parquet.rs b/datafusion/core/src/test_util/parquet.rs index cc7f00882c8d..67e0e1726917 100644 --- a/datafusion/core/src/test_util/parquet.rs +++ b/datafusion/core/src/test_util/parquet.rs @@ -155,7 +155,7 @@ impl TestParquetFile { maybe_filter: Option, ) -> Result> { let parquet_options = ctx.copied_table_options().parquet; - let source = Arc::new(ParquetSource::new_with_options(parquet_options.clone())); + let source = Arc::new(ParquetSource::new(parquet_options.clone())); let mut scan_config = FileScanConfig::new( self.object_store_url.clone(), Arc::clone(&self.schema), @@ -181,11 +181,9 @@ impl TestParquetFile { let physical_filter_expr = create_physical_expr(&filter, &df_schema, &ExecutionProps::default())?; - let source = Arc::new(ParquetSource::new( + let source = Arc::new(ParquetSource::new(parquet_options).with_predicate( Arc::clone(&scan_config.file_schema), - Some(Arc::clone(&physical_filter_expr)), - None, - parquet_options, + Arc::clone(&physical_filter_expr), )); scan_config = scan_config.with_source(source); let parquet_exec = scan_config.new_exec(); @@ -206,7 +204,12 @@ impl TestParquetFile { let source = maybe_file.source(); if let Some(maybe_parquet) = source.as_any().downcast_ref::() { - if maybe_parquet.file_source().file_type().is_parquet() { + if maybe_parquet + .file_source() + .as_any() + .downcast_ref::() + .is_some() + { return maybe_file.metrics(); } } diff --git a/datafusion/core/tests/dataframe/mod.rs b/datafusion/core/tests/dataframe/mod.rs index a3979c5d3a00..acd5ee6d5ef0 100644 --- a/datafusion/core/tests/dataframe/mod.rs +++ b/datafusion/core/tests/dataframe/mod.rs @@ -19,6 +19,7 @@ mod dataframe_functions; mod describe; +use arrow::buffer::ScalarBuffer; use arrow::datatypes::{DataType, Field, Float32Type, Int32Type, Schema, UInt64Type}; use arrow::util::pretty::pretty_format_batches; use arrow::{ @@ -33,7 +34,6 @@ use arrow_array::{ record_batch, Array, BooleanArray, DictionaryArray, Float32Array, Float64Array, Int8Array, UnionArray, }; -use arrow_buffer::ScalarBuffer; use arrow_schema::{ArrowError, SchemaRef, UnionFields, UnionMode}; use datafusion_functions_aggregate::count::count_udaf; use datafusion_functions_aggregate::expr_fn::{ diff --git a/datafusion/core/tests/expr_api/simplification.rs b/datafusion/core/tests/expr_api/simplification.rs index 1e6ff8088d0a..76df4a1f1105 100644 --- a/datafusion/core/tests/expr_api/simplification.rs +++ b/datafusion/core/tests/expr_api/simplification.rs @@ -17,9 +17,9 @@ //! This program demonstrates the DataFusion expression simplification API. +use arrow::array::types::IntervalDayTime; use arrow::datatypes::{DataType, Field, Schema}; use arrow_array::{ArrayRef, Int32Array}; -use arrow_buffer::IntervalDayTime; use chrono::{DateTime, TimeZone, Utc}; use datafusion::{error::Result, execution::context::ExecutionProps, prelude::*}; use datafusion_common::cast::as_int32_array; diff --git a/datafusion/core/tests/fuzz_cases/pruning.rs b/datafusion/core/tests/fuzz_cases/pruning.rs index 5f493a6cbf32..a43886b6df21 100644 --- a/datafusion/core/tests/fuzz_cases/pruning.rs +++ b/datafusion/core/tests/fuzz_cases/pruning.rs @@ -27,7 +27,6 @@ use datafusion::{ }, prelude::*, }; -use datafusion_common::config::TableParquetOptions; use datafusion_common::DFSchema; use datafusion_execution::object_store::ObjectStoreUrl; use datafusion_physical_expr::PhysicalExpr; @@ -305,19 +304,9 @@ async fn execute_with_predicate( ctx: &SessionContext, ) -> Vec { let parquet_source = if prune_stats { - ParquetSource::new( - Arc::clone(&schema), - Some(predicate.clone()), - None, - TableParquetOptions::default(), - ) + ParquetSource::default().with_predicate(Arc::clone(&schema), predicate.clone()) } else { - ParquetSource::new( - Arc::clone(&schema), - None, - None, - TableParquetOptions::default(), - ) + ParquetSource::default() }; let scan = FileScanConfig::new( ObjectStoreUrl::parse("memory://").unwrap(), diff --git a/datafusion/core/tests/parquet/external_access_plan.rs b/datafusion/core/tests/parquet/external_access_plan.rs index 7c4c191327d1..216f03aac746 100644 --- a/datafusion/core/tests/parquet/external_access_plan.rs +++ b/datafusion/core/tests/parquet/external_access_plan.rs @@ -30,7 +30,6 @@ use datafusion::datasource::listing::PartitionedFile; use datafusion::datasource::physical_plan::parquet::{ParquetAccessPlan, RowGroupAccess}; use datafusion::datasource::physical_plan::{FileScanConfig, ParquetSource}; use datafusion::prelude::SessionContext; -use datafusion_common::config::TableParquetOptions; use datafusion_common::{assert_contains, DFSchema}; use datafusion_execution::object_store::ObjectStoreUrl; use datafusion_expr::{col, lit, Expr}; @@ -345,12 +344,7 @@ impl TestFull { let source = if let Some(predicate) = predicate { let df_schema = DFSchema::try_from(schema.clone())?; let predicate = ctx.create_physical_expr(predicate, &df_schema)?; - Arc::new(ParquetSource::new( - Arc::clone(schema), - Some(predicate), - None, - TableParquetOptions::default(), - )) + Arc::new(ParquetSource::default().with_predicate(schema.clone(), predicate)) } else { Arc::new(ParquetSource::default()) }; diff --git a/datafusion/core/tests/parquet/page_pruning.rs b/datafusion/core/tests/parquet/page_pruning.rs index 075a698a3d2b..ea86bf3685bb 100644 --- a/datafusion/core/tests/parquet/page_pruning.rs +++ b/datafusion/core/tests/parquet/page_pruning.rs @@ -29,7 +29,6 @@ use datafusion::execution::context::SessionState; use datafusion::physical_plan::metrics::MetricValue; use datafusion::physical_plan::ExecutionPlan; use datafusion::prelude::SessionContext; -use datafusion_common::config::TableParquetOptions; use datafusion_common::{ScalarValue, ToDFSchema}; use datafusion_expr::execution_props::ExecutionProps; use datafusion_expr::{col, lit, Expr}; @@ -76,13 +75,9 @@ async fn get_parquet_exec(state: &SessionState, filter: Expr) -> DataSourceExec let predicate = create_physical_expr(&filter, &df_schema, &execution_props).unwrap(); let source = Arc::new( - ParquetSource::new( - Arc::clone(&schema), - Some(predicate), - None, - TableParquetOptions::default(), - ) - .with_enable_page_index(true), + ParquetSource::default() + .with_predicate(Arc::clone(&schema), predicate) + .with_enable_page_index(true), ); let base_config = FileScanConfig::new(object_store_url, schema, source).with_file(partitioned_file); diff --git a/datafusion/core/tests/parquet/utils.rs b/datafusion/core/tests/parquet/utils.rs index 28f5198496d7..dd5541461ff6 100644 --- a/datafusion/core/tests/parquet/utils.rs +++ b/datafusion/core/tests/parquet/utils.rs @@ -17,7 +17,7 @@ //! Utilities for parquet tests -use datafusion::datasource::physical_plan::FileScanConfig; +use datafusion::datasource::physical_plan::{FileScanConfig, ParquetSource}; use datafusion_physical_plan::metrics::MetricsSet; use datafusion_physical_plan::source::DataSourceExec; use datafusion_physical_plan::{accept, ExecutionPlan, ExecutionPlanVisitor}; @@ -50,7 +50,12 @@ impl ExecutionPlanVisitor for MetricsFinder { if let Some(exec) = plan.as_any().downcast_ref::() { let source = exec.source(); if let Some(file_config) = source.as_any().downcast_ref::() { - if file_config.file_source().file_type().is_parquet() { + if file_config + .file_source() + .as_any() + .downcast_ref::() + .is_some() + { self.metrics = exec.metrics(); } } diff --git a/datafusion/core/tests/physical_optimizer/limit_pushdown.rs b/datafusion/core/tests/physical_optimizer/limit_pushdown.rs index 49490b2a3d48..dd2c1960a658 100644 --- a/datafusion/core/tests/physical_optimizer/limit_pushdown.rs +++ b/datafusion/core/tests/physical_optimizer/limit_pushdown.rs @@ -233,12 +233,11 @@ fn transforms_coalesce_batches_exec_into_fetching_version_and_removes_local_limi LimitPushdown::new().optimize(global_limit, &ConfigOptions::new())?; let expected = [ - "GlobalLimitExec: skip=0, fetch=5", - " CoalescePartitionsExec", - " CoalesceBatchesExec: target_batch_size=8192, fetch=5", - " FilterExec: c3@2 > 0", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " StreamingTableExec: partition_sizes=1, projection=[c1, c2, c3], infinite_source=true" + "CoalescePartitionsExec: fetch=5", + " CoalesceBatchesExec: target_batch_size=8192, fetch=5", + " FilterExec: c3@2 > 0", + " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " StreamingTableExec: partition_sizes=1, projection=[c1, c2, c3], infinite_source=true" ]; assert_eq!(get_plan_string(&after_optimize), expected); @@ -378,11 +377,10 @@ fn keeps_pushed_local_limit_exec_when_there_are_multiple_input_partitions() -> R LimitPushdown::new().optimize(global_limit, &ConfigOptions::new())?; let expected = [ - "GlobalLimitExec: skip=0, fetch=5", - " CoalescePartitionsExec", - " FilterExec: c3@2 > 0", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " StreamingTableExec: partition_sizes=1, projection=[c1, c2, c3], infinite_source=true" + "CoalescePartitionsExec: fetch=5", + " FilterExec: c3@2 > 0", + " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " StreamingTableExec: partition_sizes=1, projection=[c1, c2, c3], infinite_source=true" ]; assert_eq!(get_plan_string(&after_optimize), expected); diff --git a/datafusion/core/tests/sql/explain_analyze.rs b/datafusion/core/tests/sql/explain_analyze.rs index 672abfa6b92a..d4b5ae8b2820 100644 --- a/datafusion/core/tests/sql/explain_analyze.rs +++ b/datafusion/core/tests/sql/explain_analyze.rs @@ -69,11 +69,6 @@ async fn explain_analyze_baseline_metrics() { "FilterExec: c13@1 != C2GT5KVyOPZpgKVl110TyZO0NcJ434", "metrics=[output_rows=99, elapsed_compute=" ); - assert_metrics!( - &formatted, - "GlobalLimitExec: skip=0, fetch=3, ", - "metrics=[output_rows=3, elapsed_compute=" - ); assert_metrics!( &formatted, "ProjectionExec: expr=[count(*)", @@ -101,9 +96,7 @@ async fn explain_analyze_baseline_metrics() { plan.as_any().downcast_ref::().is_some() || plan.as_any().downcast_ref::().is_some() - // CoalescePartitionsExec doesn't do any work so is not included || plan.as_any().downcast_ref::().is_some() - || plan.as_any().downcast_ref::().is_some() || plan.as_any().downcast_ref::().is_some() || plan.as_any().downcast_ref::().is_some() || plan.as_any().downcast_ref::().is_some() diff --git a/datafusion/expr/src/logical_plan/builder.rs b/datafusion/expr/src/logical_plan/builder.rs index ddd768c41b19..268025c09281 100644 --- a/datafusion/expr/src/logical_plan/builder.rs +++ b/datafusion/expr/src/logical_plan/builder.rs @@ -1602,7 +1602,7 @@ pub fn table_scan_with_filter_and_fetch( ) } -fn table_source(table_schema: &Schema) -> Arc { +pub fn table_source(table_schema: &Schema) -> Arc { let table_schema = Arc::new(table_schema.clone()); Arc::new(LogicalTableSource { table_schema }) } diff --git a/datafusion/expr/src/logical_plan/invariants.rs b/datafusion/expr/src/logical_plan/invariants.rs index a350977cb080..f4ac33b6afef 100644 --- a/datafusion/expr/src/logical_plan/invariants.rs +++ b/datafusion/expr/src/logical_plan/invariants.rs @@ -249,31 +249,26 @@ pub fn check_subquery_expr( // Recursively check the unsupported outer references in the sub query plan. fn check_correlations_in_subquery(inner_plan: &LogicalPlan) -> Result<()> { - check_inner_plan(inner_plan, true) + check_inner_plan(inner_plan) } // Recursively check the unsupported outer references in the sub query plan. #[cfg_attr(feature = "recursive_protection", recursive::recursive)] -fn check_inner_plan(inner_plan: &LogicalPlan, can_contain_outer_ref: bool) -> Result<()> { - if !can_contain_outer_ref && inner_plan.contains_outer_reference() { - return plan_err!("Accessing outer reference columns is not allowed in the plan"); - } +fn check_inner_plan(inner_plan: &LogicalPlan) -> Result<()> { // We want to support as many operators as possible inside the correlated subquery match inner_plan { LogicalPlan::Aggregate(_) => { inner_plan.apply_children(|plan| { - check_inner_plan(plan, can_contain_outer_ref)?; + check_inner_plan(plan)?; Ok(TreeNodeRecursion::Continue) })?; Ok(()) } - LogicalPlan::Filter(Filter { input, .. }) => { - check_inner_plan(input, can_contain_outer_ref) - } + LogicalPlan::Filter(Filter { input, .. }) => check_inner_plan(input), LogicalPlan::Window(window) => { check_mixed_out_refer_in_window(window)?; inner_plan.apply_children(|plan| { - check_inner_plan(plan, can_contain_outer_ref)?; + check_inner_plan(plan)?; Ok(TreeNodeRecursion::Continue) })?; Ok(()) @@ -290,7 +285,7 @@ fn check_inner_plan(inner_plan: &LogicalPlan, can_contain_outer_ref: bool) -> Re | LogicalPlan::SubqueryAlias(_) | LogicalPlan::Unnest(_) => { inner_plan.apply_children(|plan| { - check_inner_plan(plan, can_contain_outer_ref)?; + check_inner_plan(plan)?; Ok(TreeNodeRecursion::Continue) })?; Ok(()) @@ -303,7 +298,7 @@ fn check_inner_plan(inner_plan: &LogicalPlan, can_contain_outer_ref: bool) -> Re }) => match join_type { JoinType::Inner => { inner_plan.apply_children(|plan| { - check_inner_plan(plan, can_contain_outer_ref)?; + check_inner_plan(plan)?; Ok(TreeNodeRecursion::Continue) })?; Ok(()) @@ -312,26 +307,34 @@ fn check_inner_plan(inner_plan: &LogicalPlan, can_contain_outer_ref: bool) -> Re | JoinType::LeftSemi | JoinType::LeftAnti | JoinType::LeftMark => { - check_inner_plan(left, can_contain_outer_ref)?; - check_inner_plan(right, false) + check_inner_plan(left)?; + check_no_outer_references(right) } JoinType::Right | JoinType::RightSemi | JoinType::RightAnti => { - check_inner_plan(left, false)?; - check_inner_plan(right, can_contain_outer_ref) + check_no_outer_references(left)?; + check_inner_plan(right) } JoinType::Full => { inner_plan.apply_children(|plan| { - check_inner_plan(plan, false)?; + check_no_outer_references(plan)?; Ok(TreeNodeRecursion::Continue) })?; Ok(()) } }, LogicalPlan::Extension(_) => Ok(()), - _ => plan_err!( - "Unsupported operator in the subquery plan: {}", + plan => check_no_outer_references(plan), + } +} + +fn check_no_outer_references(inner_plan: &LogicalPlan) -> Result<()> { + if inner_plan.contains_outer_reference() { + plan_err!( + "Accessing outer reference columns is not allowed in the plan: {}", inner_plan.display() - ), + ) + } else { + Ok(()) } } @@ -473,6 +476,6 @@ mod test { }), }); - check_inner_plan(&plan, true).unwrap(); + check_inner_plan(&plan).unwrap(); } } diff --git a/datafusion/expr/src/logical_plan/statement.rs b/datafusion/expr/src/logical_plan/statement.rs index 93be04c27564..82acebee3de6 100644 --- a/datafusion/expr/src/logical_plan/statement.rs +++ b/datafusion/expr/src/logical_plan/statement.rs @@ -153,6 +153,7 @@ pub enum TransactionIsolationLevel { ReadCommitted, RepeatableRead, Serializable, + Snapshot, } /// Indicator that the following statements should be committed or rolled back atomically diff --git a/datafusion/functions-aggregate/Cargo.toml b/datafusion/functions-aggregate/Cargo.toml index 333f0d9cdd79..78e22011b61a 100644 --- a/datafusion/functions-aggregate/Cargo.toml +++ b/datafusion/functions-aggregate/Cargo.toml @@ -40,7 +40,6 @@ path = "src/lib.rs" [dependencies] ahash = { workspace = true } arrow = { workspace = true } -arrow-buffer = { workspace = true } arrow-schema = { workspace = true } datafusion-common = { workspace = true } datafusion-doc = { workspace = true } diff --git a/datafusion/functions-aggregate/benches/array_agg.rs b/datafusion/functions-aggregate/benches/array_agg.rs index c4599cdfc9b3..c9792d541a4f 100644 --- a/datafusion/functions-aggregate/benches/array_agg.rs +++ b/datafusion/functions-aggregate/benches/array_agg.rs @@ -17,7 +17,9 @@ use std::sync::Arc; -use arrow::array::{Array, ArrayRef, ArrowPrimitiveType, AsArray, ListArray}; +use arrow::array::{ + Array, ArrayRef, ArrowPrimitiveType, AsArray, ListArray, NullBufferBuilder, +}; use arrow::datatypes::Int64Type; use arrow::util::bench_util::create_primitive_array; use arrow_schema::Field; @@ -25,8 +27,8 @@ use criterion::{black_box, criterion_group, criterion_main, Criterion}; use datafusion_expr::Accumulator; use datafusion_functions_aggregate::array_agg::ArrayAggAccumulator; +use arrow::buffer::OffsetBuffer; use arrow::util::test_util::seedable_rng; -use arrow_buffer::{NullBufferBuilder, OffsetBuffer}; use rand::distributions::{Distribution, Standard}; use rand::Rng; diff --git a/datafusion/functions-aggregate/src/correlation.rs b/datafusion/functions-aggregate/src/correlation.rs index 2741fe4bfc00..ac57256ce882 100644 --- a/datafusion/functions-aggregate/src/correlation.rs +++ b/datafusion/functions-aggregate/src/correlation.rs @@ -23,7 +23,8 @@ use std::mem::size_of_val; use std::sync::Arc; use arrow::array::{ - downcast_array, Array, AsArray, BooleanArray, Float64Array, UInt64Array, + downcast_array, Array, AsArray, BooleanArray, Float64Array, NullBufferBuilder, + UInt64Array, }; use arrow::compute::{and, filter, is_not_null, kernels::cast}; use arrow::datatypes::{Float64Type, UInt64Type}; @@ -31,7 +32,6 @@ use arrow::{ array::ArrayRef, datatypes::{DataType, Field}, }; -use arrow_buffer::NullBufferBuilder; use datafusion_expr::{EmitTo, GroupsAccumulator}; use datafusion_functions_aggregate_common::aggregate::groups_accumulator::accumulate::accumulate_multiple; use log::debug; diff --git a/datafusion/functions-nested/Cargo.toml b/datafusion/functions-nested/Cargo.toml index e7254e4125cb..01fbc73cba12 100644 --- a/datafusion/functions-nested/Cargo.toml +++ b/datafusion/functions-nested/Cargo.toml @@ -42,7 +42,6 @@ path = "src/lib.rs" [dependencies] arrow = { workspace = true } arrow-array = { workspace = true } -arrow-buffer = { workspace = true } arrow-ord = { workspace = true } arrow-schema = { workspace = true } datafusion-common = { workspace = true } diff --git a/datafusion/functions-nested/benches/map.rs b/datafusion/functions-nested/benches/map.rs index f92bb6cecf9c..22bef99becef 100644 --- a/datafusion/functions-nested/benches/map.rs +++ b/datafusion/functions-nested/benches/map.rs @@ -17,8 +17,8 @@ extern crate criterion; +use arrow::buffer::{OffsetBuffer, ScalarBuffer}; use arrow_array::{Int32Array, ListArray, StringArray}; -use arrow_buffer::{OffsetBuffer, ScalarBuffer}; use arrow_schema::{DataType, Field}; use criterion::{black_box, criterion_group, criterion_main, Criterion}; use rand::prelude::ThreadRng; diff --git a/datafusion/functions-nested/src/array_has.rs b/datafusion/functions-nested/src/array_has.rs index df007b5cd60d..e56f5633b2a5 100644 --- a/datafusion/functions-nested/src/array_has.rs +++ b/datafusion/functions-nested/src/array_has.rs @@ -18,10 +18,10 @@ //! [`ScalarUDFImpl`] definitions for array_has, array_has_all and array_has_any functions. use arrow::array::{Array, ArrayRef, BooleanArray, OffsetSizeTrait}; +use arrow::buffer::BooleanBuffer; use arrow::datatypes::DataType; use arrow::row::{RowConverter, Rows, SortField}; use arrow_array::{Datum, GenericListArray, Scalar}; -use arrow_buffer::BooleanBuffer; 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}; diff --git a/datafusion/functions-nested/src/concat.rs b/datafusion/functions-nested/src/concat.rs index 93305faad56f..0e98c31ba663 100644 --- a/datafusion/functions-nested/src/concat.rs +++ b/datafusion/functions-nested/src/concat.rs @@ -20,9 +20,11 @@ use std::sync::Arc; use std::{any::Any, cmp::Ordering}; -use arrow::array::{Capacities, MutableArrayData}; -use arrow_array::{Array, ArrayRef, GenericListArray, OffsetSizeTrait}; -use arrow_buffer::{NullBufferBuilder, OffsetBuffer}; +use arrow::array::{ + Array, ArrayRef, Capacities, GenericListArray, MutableArrayData, NullBufferBuilder, + OffsetSizeTrait, +}; +use arrow::buffer::OffsetBuffer; use arrow_schema::{DataType, Field}; use datafusion_common::Result; use datafusion_common::{ diff --git a/datafusion/functions-nested/src/except.rs b/datafusion/functions-nested/src/except.rs index 356c92983ae2..8cb870dba058 100644 --- a/datafusion/functions-nested/src/except.rs +++ b/datafusion/functions-nested/src/except.rs @@ -18,10 +18,10 @@ //! [`ScalarUDFImpl`] definitions for array_except function. use crate::utils::{check_datatypes, make_scalar_function}; +use arrow::buffer::OffsetBuffer; use arrow::row::{RowConverter, SortField}; use arrow_array::cast::AsArray; use arrow_array::{Array, ArrayRef, GenericListArray, OffsetSizeTrait}; -use arrow_buffer::OffsetBuffer; use arrow_schema::{DataType, FieldRef}; use datafusion_common::{exec_err, internal_err, HashSet, Result}; use datafusion_expr::{ diff --git a/datafusion/functions-nested/src/extract.rs b/datafusion/functions-nested/src/extract.rs index c87a96dca7a4..8c120876c38a 100644 --- a/datafusion/functions-nested/src/extract.rs +++ b/datafusion/functions-nested/src/extract.rs @@ -17,17 +17,12 @@ //! [`ScalarUDFImpl`] definitions for array_element, array_slice, array_pop_front, array_pop_back, and array_any_value functions. -use arrow::array::Array; -use arrow::array::ArrayRef; -use arrow::array::ArrowNativeTypeOp; -use arrow::array::Capacities; -use arrow::array::GenericListArray; -use arrow::array::Int64Array; -use arrow::array::MutableArrayData; -use arrow::array::OffsetSizeTrait; +use arrow::array::{ + Array, ArrayRef, ArrowNativeTypeOp, Capacities, GenericListArray, Int64Array, + MutableArrayData, NullBufferBuilder, OffsetSizeTrait, +}; use arrow::buffer::OffsetBuffer; use arrow::datatypes::DataType; -use arrow_buffer::NullBufferBuilder; use arrow_schema::DataType::{FixedSizeList, LargeList, List}; use arrow_schema::Field; use datafusion_common::cast::as_int64_array; @@ -487,7 +482,17 @@ where // 0 ~ len - 1 let adjusted_zero_index = if index < 0 { if let Ok(index) = index.try_into() { - index + len + // When index < 0 and -index > length, index is clamped to the beginning of the list. + // Otherwise, when index < 0, the index is counted from the end of the list. + // + // Note, we actually test the contrapositive, index < -length, because negating a + // negative will panic if the negative is equal to the smallest representable value + // while negating a positive is always safe. + if index < (O::zero() - O::one()) * len { + O::zero() + } else { + index + len + } } else { return exec_err!("array_slice got invalid index: {}", index); } @@ -575,7 +580,7 @@ where "array_slice got invalid stride: {:?}, it cannot be 0", stride ); - } else if (from <= to && stride.is_negative()) + } else if (from < to && stride.is_negative()) || (from > to && stride.is_positive()) { // return empty array @@ -587,7 +592,7 @@ where internal_datafusion_err!("array_slice got invalid stride: {}", stride) })?; - if from <= to { + if from <= to && stride > O::zero() { assert!(start + to <= end); if stride.eq(&O::one()) { // stride is default to 1 diff --git a/datafusion/functions-nested/src/flatten.rs b/datafusion/functions-nested/src/flatten.rs index 30bf2fcbf624..b97b9e3c68a9 100644 --- a/datafusion/functions-nested/src/flatten.rs +++ b/datafusion/functions-nested/src/flatten.rs @@ -18,8 +18,8 @@ //! [`ScalarUDFImpl`] definitions for flatten function. use crate::utils::make_scalar_function; +use arrow::buffer::OffsetBuffer; use arrow_array::{ArrayRef, GenericListArray, OffsetSizeTrait}; -use arrow_buffer::OffsetBuffer; use arrow_schema::DataType; use arrow_schema::DataType::{FixedSizeList, LargeList, List, Null}; use datafusion_common::cast::{ diff --git a/datafusion/functions-nested/src/make_array.rs b/datafusion/functions-nested/src/make_array.rs index 0283cdd40275..6bc1f6f2c2d1 100644 --- a/datafusion/functions-nested/src/make_array.rs +++ b/datafusion/functions-nested/src/make_array.rs @@ -23,10 +23,10 @@ use std::vec; use crate::utils::make_scalar_function; use arrow::array::{ArrayData, Capacities, MutableArrayData}; +use arrow::buffer::OffsetBuffer; use arrow_array::{ new_null_array, Array, ArrayRef, GenericListArray, NullArray, OffsetSizeTrait, }; -use arrow_buffer::OffsetBuffer; use arrow_schema::DataType::{List, Null}; use arrow_schema::{DataType, Field}; use datafusion_common::utils::SingleRowListArrayBuilder; @@ -136,10 +136,7 @@ impl ScalarUDFImpl for MakeArray { } if let Some(new_type) = type_union_resolution(arg_types) { - // TODO: Move FixedSizeList to List in type_union_resolution - if let DataType::FixedSizeList(field, _) = new_type { - Ok(vec![List(field); arg_types.len()]) - } else if new_type.is_null() { + if new_type.is_null() { Ok(vec![DataType::Int64; arg_types.len()]) } else { Ok(vec![new_type; arg_types.len()]) diff --git a/datafusion/functions-nested/src/map.rs b/datafusion/functions-nested/src/map.rs index d484cc834262..8c78de68f86e 100644 --- a/datafusion/functions-nested/src/map.rs +++ b/datafusion/functions-nested/src/map.rs @@ -20,8 +20,9 @@ use std::collections::VecDeque; use std::sync::Arc; use arrow::array::ArrayData; +use arrow::buffer::Buffer; +use arrow::datatypes::ToByteSlice; use arrow_array::{Array, ArrayRef, MapArray, OffsetSizeTrait, StructArray}; -use arrow_buffer::{Buffer, ToByteSlice}; use arrow_schema::{DataType, Field, SchemaBuilder}; use datafusion_common::utils::{fixed_size_list_to_arrays, list_to_arrays}; diff --git a/datafusion/functions-nested/src/map_extract.rs b/datafusion/functions-nested/src/map_extract.rs index 8ccfae0ff93e..268c3235cc49 100644 --- a/datafusion/functions-nested/src/map_extract.rs +++ b/datafusion/functions-nested/src/map_extract.rs @@ -20,9 +20,9 @@ use arrow::array::{ArrayRef, Capacities, MutableArrayData}; use arrow_array::{make_array, ListArray}; +use arrow::buffer::OffsetBuffer; use arrow::datatypes::DataType; use arrow_array::{Array, MapArray}; -use arrow_buffer::OffsetBuffer; use arrow_schema::Field; use datafusion_common::{cast::as_map_array, exec_err, Result}; diff --git a/datafusion/functions-nested/src/range.rs b/datafusion/functions-nested/src/range.rs index ff148f04ac5f..c3f52cef3366 100644 --- a/datafusion/functions-nested/src/range.rs +++ b/datafusion/functions-nested/src/range.rs @@ -18,16 +18,16 @@ //! [`ScalarUDFImpl`] definitions for range and gen_series functions. use crate::utils::make_scalar_function; -use arrow::array::{Array, ArrayRef, Int64Array, ListArray, ListBuilder}; -use arrow::datatypes::{DataType, Field}; -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::{ + builder::{Date32Builder, TimestampNanosecondBuilder}, + temporal_conversions::as_datetime_with_timezone, + timezone::Tz, + types::{Date32Type, IntervalMonthDayNanoType, TimestampNanosecondType as TSNT}, + Array, ArrayRef, Int64Array, ListArray, ListBuilder, NullArray, NullBufferBuilder, + TimestampNanosecondArray, }; -use arrow_array::{NullArray, TimestampNanosecondArray}; -use arrow_buffer::{NullBufferBuilder, OffsetBuffer}; +use arrow::buffer::OffsetBuffer; +use arrow::datatypes::{DataType, Field}; use arrow_schema::DataType::*; use arrow_schema::IntervalUnit::MonthDayNano; use arrow_schema::TimeUnit::Nanosecond; diff --git a/datafusion/functions-nested/src/remove.rs b/datafusion/functions-nested/src/remove.rs index bf7f4746618f..64b6405176a3 100644 --- a/datafusion/functions-nested/src/remove.rs +++ b/datafusion/functions-nested/src/remove.rs @@ -19,11 +19,11 @@ use crate::utils; use crate::utils::make_scalar_function; +use arrow::buffer::OffsetBuffer; use arrow_array::cast::AsArray; use arrow_array::{ new_empty_array, Array, ArrayRef, BooleanArray, GenericListArray, OffsetSizeTrait, }; -use arrow_buffer::OffsetBuffer; use arrow_schema::{DataType, Field}; use datafusion_common::cast::as_int64_array; use datafusion_common::{exec_err, Result}; diff --git a/datafusion/functions-nested/src/repeat.rs b/datafusion/functions-nested/src/repeat.rs index 2bc4721f3cfa..455fb3dd3023 100644 --- a/datafusion/functions-nested/src/repeat.rs +++ b/datafusion/functions-nested/src/repeat.rs @@ -19,13 +19,13 @@ use crate::utils::make_scalar_function; use arrow::array::{Capacities, MutableArrayData}; +use arrow::buffer::OffsetBuffer; use arrow::compute; use arrow::compute::cast; use arrow_array::{ new_null_array, Array, ArrayRef, GenericListArray, ListArray, OffsetSizeTrait, UInt64Array, }; -use arrow_buffer::OffsetBuffer; use arrow_schema::DataType::{LargeList, List}; use arrow_schema::{DataType, Field}; use datafusion_common::cast::{as_large_list_array, as_list_array, as_uint64_array}; diff --git a/datafusion/functions-nested/src/replace.rs b/datafusion/functions-nested/src/replace.rs index 106887c51396..1f12625a52b8 100644 --- a/datafusion/functions-nested/src/replace.rs +++ b/datafusion/functions-nested/src/replace.rs @@ -18,12 +18,12 @@ //! [`ScalarUDFImpl`] definitions for array_replace, array_replace_n and array_replace_all functions. use arrow::array::{ - Array, ArrayRef, AsArray, Capacities, MutableArrayData, OffsetSizeTrait, + Array, ArrayRef, AsArray, Capacities, GenericListArray, MutableArrayData, + NullBufferBuilder, OffsetSizeTrait, }; use arrow::datatypes::DataType; -use arrow_array::GenericListArray; -use arrow_buffer::{NullBufferBuilder, OffsetBuffer}; +use arrow::buffer::OffsetBuffer; use arrow_schema::Field; use datafusion_common::cast::as_int64_array; use datafusion_common::{exec_err, Result}; diff --git a/datafusion/functions-nested/src/resize.rs b/datafusion/functions-nested/src/resize.rs index 441f44e47f6e..f167134f9b22 100644 --- a/datafusion/functions-nested/src/resize.rs +++ b/datafusion/functions-nested/src/resize.rs @@ -18,11 +18,12 @@ //! [`ScalarUDFImpl`] definitions for array_resize function. use crate::utils::make_scalar_function; -use arrow::array::{Capacities, MutableArrayData}; -use arrow_array::{ - new_null_array, Array, ArrayRef, GenericListArray, Int64Array, OffsetSizeTrait, +use arrow::array::{ + new_null_array, Array, ArrayRef, Capacities, GenericListArray, Int64Array, + MutableArrayData, NullBufferBuilder, OffsetSizeTrait, }; -use arrow_buffer::{ArrowNativeType, NullBufferBuilder, OffsetBuffer}; +use arrow::buffer::OffsetBuffer; +use arrow::datatypes::ArrowNativeType; use arrow_schema::DataType::{FixedSizeList, LargeList, List}; use arrow_schema::{DataType, FieldRef}; use datafusion_common::cast::{as_int64_array, as_large_list_array, as_list_array}; diff --git a/datafusion/functions-nested/src/reverse.rs b/datafusion/functions-nested/src/reverse.rs index 8538ba5cac12..9fd955094ae6 100644 --- a/datafusion/functions-nested/src/reverse.rs +++ b/datafusion/functions-nested/src/reverse.rs @@ -19,8 +19,8 @@ use crate::utils::make_scalar_function; use arrow::array::{Capacities, MutableArrayData}; +use arrow::buffer::OffsetBuffer; use arrow_array::{Array, ArrayRef, GenericListArray, OffsetSizeTrait}; -use arrow_buffer::OffsetBuffer; use arrow_schema::DataType::{LargeList, List, Null}; use arrow_schema::{DataType, FieldRef}; use datafusion_common::cast::{as_large_list_array, as_list_array}; diff --git a/datafusion/functions-nested/src/sort.rs b/datafusion/functions-nested/src/sort.rs index 8e45ccbf74b8..e4dcc02286f3 100644 --- a/datafusion/functions-nested/src/sort.rs +++ b/datafusion/functions-nested/src/sort.rs @@ -18,9 +18,9 @@ //! [`ScalarUDFImpl`] definitions for array_sort function. use crate::utils::make_scalar_function; +use arrow::array::{Array, ArrayRef, ListArray, NullBufferBuilder}; +use arrow::buffer::OffsetBuffer; use arrow::compute; -use arrow_array::{Array, ArrayRef, ListArray}; -use arrow_buffer::{NullBufferBuilder, OffsetBuffer}; use arrow_schema::DataType::{FixedSizeList, LargeList, List}; use arrow_schema::{DataType, Field, SortOptions}; use datafusion_common::cast::{as_list_array, as_string_array}; diff --git a/datafusion/functions-nested/src/utils.rs b/datafusion/functions-nested/src/utils.rs index c54d6d49cecc..e1961dccf54a 100644 --- a/datafusion/functions-nested/src/utils.rs +++ b/datafusion/functions-nested/src/utils.rs @@ -21,11 +21,11 @@ use std::sync::Arc; use arrow::{array::ArrayRef, datatypes::DataType}; +use arrow::buffer::OffsetBuffer; use arrow_array::{ Array, BooleanArray, GenericListArray, ListArray, OffsetSizeTrait, Scalar, UInt32Array, }; -use arrow_buffer::OffsetBuffer; use arrow_schema::{Field, Fields}; use datafusion_common::cast::{as_large_list_array, as_list_array}; use datafusion_common::{ diff --git a/datafusion/functions/src/core/arrow_cast.rs b/datafusion/functions/src/core/arrow_cast.rs index b0fba57460f8..0f9f11b4eff0 100644 --- a/datafusion/functions/src/core/arrow_cast.rs +++ b/datafusion/functions/src/core/arrow_cast.rs @@ -25,6 +25,7 @@ use datafusion_common::{ use datafusion_common::{exec_datafusion_err, DataFusionError}; use std::any::Any; +use crate::utils::take_function_args; use datafusion_expr::simplify::{ExprSimplifyResult, SimplifyInfo}; use datafusion_expr::{ ColumnarValue, Documentation, Expr, ReturnInfo, ReturnTypeArgs, ScalarUDFImpl, @@ -117,10 +118,9 @@ impl ScalarUDFImpl for ArrowCastFunc { fn return_type_from_args(&self, args: ReturnTypeArgs) -> Result { let nullable = args.nullables.iter().any(|&nullable| nullable); - // Length check handled in the signature - debug_assert_eq!(args.scalar_arguments.len(), 2); + let [_, type_arg] = take_function_args(self.name(), args.scalar_arguments)?; - args.scalar_arguments[1] + type_arg .and_then(|sv| sv.try_as_str().flatten().filter(|s| !s.is_empty())) .map_or_else( || { diff --git a/datafusion/functions/src/core/arrowtypeof.rs b/datafusion/functions/src/core/arrowtypeof.rs index 8a282beb1d7d..3c672384ffa1 100644 --- a/datafusion/functions/src/core/arrowtypeof.rs +++ b/datafusion/functions/src/core/arrowtypeof.rs @@ -15,8 +15,9 @@ // specific language governing permissions and limitations // under the License. +use crate::utils::take_function_args; use arrow::datatypes::DataType; -use datafusion_common::{exec_err, Result, ScalarValue}; +use datafusion_common::{Result, ScalarValue}; use datafusion_expr::{ColumnarValue, Documentation}; use datafusion_expr::{ScalarUDFImpl, Signature, Volatility}; use datafusion_macros::user_doc; @@ -80,14 +81,8 @@ impl ScalarUDFImpl for ArrowTypeOfFunc { args: &[ColumnarValue], _number_rows: usize, ) -> Result { - if args.len() != 1 { - return exec_err!( - "arrow_typeof function requires 1 arguments, got {}", - args.len() - ); - } - - let input_data_type = args[0].data_type(); + let [arg] = take_function_args(self.name(), args)?; + let input_data_type = arg.data_type(); Ok(ColumnarValue::Scalar(ScalarValue::from(format!( "{input_data_type}" )))) diff --git a/datafusion/functions/src/core/getfield.rs b/datafusion/functions/src/core/getfield.rs index 7c72d4594583..8533b3123d51 100644 --- a/datafusion/functions/src/core/getfield.rs +++ b/datafusion/functions/src/core/getfield.rs @@ -15,6 +15,7 @@ // specific language governing permissions and limitations // under the License. +use crate::utils::take_function_args; use arrow::array::{ make_array, Array, Capacities, MutableArrayData, Scalar, StringArray, }; @@ -99,14 +100,9 @@ impl ScalarUDFImpl for GetFieldFunc { } fn display_name(&self, args: &[Expr]) -> Result { - if args.len() != 2 { - return exec_err!( - "get_field function requires 2 arguments, got {}", - args.len() - ); - } + let [base, field_name] = take_function_args(self.name(), args)?; - let name = match &args[1] { + let name = match field_name { Expr::Literal(name) => name, _ => { return exec_err!( @@ -115,7 +111,7 @@ impl ScalarUDFImpl for GetFieldFunc { } }; - Ok(format!("{}[{}]", args[0], name)) + Ok(format!("{base}[{name}]")) } fn schema_name(&self, args: &[Expr]) -> Result { diff --git a/datafusion/functions/src/core/greatest.rs b/datafusion/functions/src/core/greatest.rs index 7ad8c7372896..6864da2d5c06 100644 --- a/datafusion/functions/src/core/greatest.rs +++ b/datafusion/functions/src/core/greatest.rs @@ -17,10 +17,10 @@ use crate::core::greatest_least_utils::GreatestLeastOperator; use arrow::array::{make_comparator, Array, BooleanArray}; +use arrow::buffer::BooleanBuffer; use arrow::compute::kernels::cmp; use arrow::compute::SortOptions; use arrow::datatypes::DataType; -use arrow_buffer::BooleanBuffer; use datafusion_common::{internal_err, Result, ScalarValue}; use datafusion_doc::Documentation; use datafusion_expr::ColumnarValue; diff --git a/datafusion/functions/src/core/least.rs b/datafusion/functions/src/core/least.rs index 02299feb9b74..a26b14babf2c 100644 --- a/datafusion/functions/src/core/least.rs +++ b/datafusion/functions/src/core/least.rs @@ -17,10 +17,10 @@ use crate::core::greatest_least_utils::GreatestLeastOperator; use arrow::array::{make_comparator, Array, BooleanArray}; +use arrow::buffer::BooleanBuffer; use arrow::compute::kernels::cmp; use arrow::compute::SortOptions; use arrow::datatypes::DataType; -use arrow_buffer::BooleanBuffer; use datafusion_common::{internal_err, Result, ScalarValue}; use datafusion_doc::Documentation; use datafusion_expr::ColumnarValue; diff --git a/datafusion/functions/src/core/nullif.rs b/datafusion/functions/src/core/nullif.rs index 9dd489826749..a0f3c8b8a452 100644 --- a/datafusion/functions/src/core/nullif.rs +++ b/datafusion/functions/src/core/nullif.rs @@ -16,15 +16,17 @@ // under the License. use arrow::datatypes::DataType; -use datafusion_common::{exec_err, Result}; +use datafusion_common::Result; use datafusion_expr::{ColumnarValue, Documentation}; +use crate::utils::take_function_args; use arrow::compute::kernels::cmp::eq; use arrow::compute::kernels::nullif::nullif; use datafusion_common::ScalarValue; use datafusion_expr::{ScalarUDFImpl, Signature, Volatility}; use datafusion_macros::user_doc; use std::any::Any; + #[user_doc( doc_section(label = "Conditional Functions"), description = "Returns _null_ if _expression1_ equals _expression2_; otherwise it returns _expression1_. @@ -119,14 +121,7 @@ impl ScalarUDFImpl for NullIfFunc { /// 1 - if the left is equal to this expr2, then the result is NULL, otherwise left value is passed. /// fn nullif_func(args: &[ColumnarValue]) -> Result { - if args.len() != 2 { - return exec_err!( - "{:?} args were supplied but NULLIF takes exactly two args", - args.len() - ); - } - - let (lhs, rhs) = (&args[0], &args[1]); + let [lhs, rhs] = take_function_args("nullif", args)?; match (lhs, rhs) { (ColumnarValue::Array(lhs), ColumnarValue::Scalar(rhs)) => { diff --git a/datafusion/functions/src/core/nvl.rs b/datafusion/functions/src/core/nvl.rs index bcd4b3908d2c..5b306c8093cb 100644 --- a/datafusion/functions/src/core/nvl.rs +++ b/datafusion/functions/src/core/nvl.rs @@ -15,16 +15,18 @@ // specific language governing permissions and limitations // under the License. +use crate::utils::take_function_args; use arrow::array::Array; use arrow::compute::is_not_null; use arrow::compute::kernels::zip::zip; use arrow::datatypes::DataType; -use datafusion_common::{internal_err, Result}; +use datafusion_common::Result; use datafusion_expr::{ ColumnarValue, Documentation, ScalarUDFImpl, Signature, Volatility, }; use datafusion_macros::user_doc; use std::sync::Arc; + #[user_doc( doc_section(label = "Conditional Functions"), description = "Returns _expression2_ if _expression1_ is NULL otherwise it returns _expression1_.", @@ -133,13 +135,8 @@ impl ScalarUDFImpl for NVLFunc { } fn nvl_func(args: &[ColumnarValue]) -> Result { - if args.len() != 2 { - return internal_err!( - "{:?} args were supplied but NVL/IFNULL takes exactly two args", - args.len() - ); - } - let (lhs_array, rhs_array) = match (&args[0], &args[1]) { + let [lhs, rhs] = take_function_args("nvl/ifnull", args)?; + let (lhs_array, rhs_array) = match (lhs, rhs) { (ColumnarValue::Array(lhs), ColumnarValue::Scalar(rhs)) => { (Arc::clone(lhs), rhs.to_array_of_size(lhs.len())?) } diff --git a/datafusion/functions/src/core/nvl2.rs b/datafusion/functions/src/core/nvl2.rs index 70b762a49ce0..b1f8e4e5c213 100644 --- a/datafusion/functions/src/core/nvl2.rs +++ b/datafusion/functions/src/core/nvl2.rs @@ -15,11 +15,12 @@ // specific language governing permissions and limitations // under the License. +use crate::utils::take_function_args; use arrow::array::Array; use arrow::compute::is_not_null; use arrow::compute::kernels::zip::zip; use arrow::datatypes::DataType; -use datafusion_common::{exec_err, internal_err, Result}; +use datafusion_common::{internal_err, Result}; use datafusion_expr::{ type_coercion::binary::comparison_coercion, ColumnarValue, Documentation, ScalarUDFImpl, Signature, Volatility, @@ -104,27 +105,22 @@ impl ScalarUDFImpl for NVL2Func { } fn coerce_types(&self, arg_types: &[DataType]) -> Result> { - if arg_types.len() != 3 { - return exec_err!( - "NVL2 takes exactly three arguments, but got {}", - arg_types.len() - ); - } - let new_type = arg_types.iter().skip(1).try_fold( - arg_types.first().unwrap().clone(), - |acc, x| { - // The coerced types found by `comparison_coercion` are not guaranteed to be - // coercible for the arguments. `comparison_coercion` returns more loose - // types that can be coerced to both `acc` and `x` for comparison purpose. - // See `maybe_data_types` for the actual coercion. - let coerced_type = comparison_coercion(&acc, x); - if let Some(coerced_type) = coerced_type { - Ok(coerced_type) - } else { - internal_err!("Coercion from {acc:?} to {x:?} failed.") - } - }, - )?; + let [tested, if_non_null, if_null] = take_function_args(self.name(), arg_types)?; + let new_type = + [if_non_null, if_null] + .iter() + .try_fold(tested.clone(), |acc, x| { + // The coerced types found by `comparison_coercion` are not guaranteed to be + // coercible for the arguments. `comparison_coercion` returns more loose + // types that can be coerced to both `acc` and `x` for comparison purpose. + // See `maybe_data_types` for the actual coercion. + let coerced_type = comparison_coercion(&acc, x); + if let Some(coerced_type) = coerced_type { + Ok(coerced_type) + } else { + internal_err!("Coercion from {acc:?} to {x:?} failed.") + } + })?; Ok(vec![new_type; arg_types.len()]) } @@ -134,12 +130,6 @@ impl ScalarUDFImpl for NVL2Func { } fn nvl2_func(args: &[ColumnarValue]) -> Result { - if args.len() != 3 { - return internal_err!( - "{:?} args were supplied but NVL2 takes exactly three args", - args.len() - ); - } let mut len = 1; let mut is_array = false; for arg in args { @@ -157,20 +147,22 @@ fn nvl2_func(args: &[ColumnarValue]) -> Result { ColumnarValue::Array(array) => Ok(Arc::clone(array)), }) .collect::>>()?; - let to_apply = is_not_null(&args[0])?; - let value = zip(&to_apply, &args[1], &args[2])?; + let [tested, if_non_null, if_null] = take_function_args("nvl2", args)?; + let to_apply = is_not_null(&tested)?; + let value = zip(&to_apply, &if_non_null, &if_null)?; Ok(ColumnarValue::Array(value)) } else { - let mut current_value = &args[1]; - match &args[0] { + let [tested, if_non_null, if_null] = take_function_args("nvl2", args)?; + match &tested { ColumnarValue::Array(_) => { internal_err!("except Scalar value, but got Array") } ColumnarValue::Scalar(scalar) => { if scalar.is_null() { - current_value = &args[2]; + Ok(if_null.clone()) + } else { + Ok(if_non_null.clone()) } - Ok(current_value.clone()) } } } diff --git a/datafusion/functions/src/core/version.rs b/datafusion/functions/src/core/version.rs index 908540731480..139763af7b38 100644 --- a/datafusion/functions/src/core/version.rs +++ b/datafusion/functions/src/core/version.rs @@ -17,13 +17,15 @@ //! [`VersionFunc`]: Implementation of the `version` function. +use crate::utils::take_function_args; use arrow::datatypes::DataType; -use datafusion_common::{internal_err, plan_err, Result, ScalarValue}; +use datafusion_common::{Result, ScalarValue}; use datafusion_expr::{ ColumnarValue, Documentation, ScalarUDFImpl, Signature, Volatility, }; use datafusion_macros::user_doc; use std::any::Any; + #[user_doc( doc_section(label = "Other Functions"), description = "Returns the version of DataFusion.", @@ -70,11 +72,8 @@ impl ScalarUDFImpl for VersionFunc { } fn return_type(&self, args: &[DataType]) -> Result { - if args.is_empty() { - Ok(DataType::Utf8) - } else { - plan_err!("version expects no arguments") - } + let [] = take_function_args(self.name(), args)?; + Ok(DataType::Utf8) } fn invoke_batch( @@ -82,9 +81,7 @@ impl ScalarUDFImpl for VersionFunc { args: &[ColumnarValue], _number_rows: usize, ) -> Result { - if !args.is_empty() { - return internal_err!("{} function does not accept arguments", self.name()); - } + let [] = take_function_args(self.name(), args)?; // 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!( diff --git a/datafusion/functions/src/crypto/basic.rs b/datafusion/functions/src/crypto/basic.rs index 860c68bc93f4..a15b9b57cff6 100644 --- a/datafusion/functions/src/crypto/basic.rs +++ b/datafusion/functions/src/crypto/basic.rs @@ -24,6 +24,7 @@ use blake2::{Blake2b512, Blake2s256, Digest}; use blake3::Hasher as Blake3; use datafusion_common::cast::as_binary_array; +use crate::utils::take_function_args; use arrow::compute::StringArrayType; use datafusion_common::plan_err; use datafusion_common::{ @@ -41,14 +42,8 @@ macro_rules! define_digest_function { ($NAME: ident, $METHOD: ident, $DOC: expr) => { #[doc = $DOC] pub fn $NAME(args: &[ColumnarValue]) -> Result { - if args.len() != 1 { - return exec_err!( - "{:?} args were supplied but {} takes exactly one argument", - args.len(), - DigestAlgorithm::$METHOD.to_string() - ); - } - digest_process(&args[0], DigestAlgorithm::$METHOD) + let [data] = take_function_args(&DigestAlgorithm::$METHOD.to_string(), args)?; + digest_process(data, DigestAlgorithm::$METHOD) } }; } @@ -114,13 +109,8 @@ pub enum DigestAlgorithm { /// Second argument is the algorithm to use. /// Standard algorithms are md5, sha1, sha224, sha256, sha384 and sha512. pub fn digest(args: &[ColumnarValue]) -> Result { - if args.len() != 2 { - return exec_err!( - "{:?} args were supplied but digest takes exactly two arguments", - args.len() - ); - } - let digest_algorithm = match &args[1] { + let [data, digest_algorithm] = take_function_args("digest", args)?; + let digest_algorithm = match digest_algorithm { ColumnarValue::Scalar(scalar) => match scalar.try_as_str() { Some(Some(method)) => method.parse::(), _ => exec_err!("Unsupported data type {scalar:?} for function digest"), @@ -129,7 +119,7 @@ pub fn digest(args: &[ColumnarValue]) -> Result { internal_err!("Digest using dynamically decided method is not yet supported") } }?; - digest_process(&args[0], digest_algorithm) + digest_process(data, digest_algorithm) } impl FromStr for DigestAlgorithm { @@ -175,15 +165,8 @@ impl fmt::Display for DigestAlgorithm { /// computes md5 hash digest of the given input pub fn md5(args: &[ColumnarValue]) -> Result { - if args.len() != 1 { - return exec_err!( - "{:?} args were supplied but {} takes exactly one argument", - args.len(), - DigestAlgorithm::Md5 - ); - } - - let value = digest_process(&args[0], DigestAlgorithm::Md5)?; + let [data] = take_function_args("md5", args)?; + let value = digest_process(data, DigestAlgorithm::Md5)?; // md5 requires special handling because of its unique utf8 return type Ok(match value { diff --git a/datafusion/functions/src/datetime/date_part.rs b/datafusion/functions/src/datetime/date_part.rs index bec378e137c0..c7dbf089e530 100644 --- a/datafusion/functions/src/datetime/date_part.rs +++ b/datafusion/functions/src/datetime/date_part.rs @@ -28,6 +28,7 @@ use arrow::datatypes::DataType::{ use arrow::datatypes::TimeUnit::{Microsecond, Millisecond, Nanosecond, Second}; use arrow::datatypes::{DataType, TimeUnit}; +use crate::utils::take_function_args; use datafusion_common::not_impl_err; use datafusion_common::{ cast::{ @@ -140,10 +141,9 @@ impl ScalarUDFImpl for DatePartFunc { } fn return_type_from_args(&self, args: ReturnTypeArgs) -> Result { - // Length check handled in the signature - debug_assert_eq!(args.scalar_arguments.len(), 2); + let [field, _] = take_function_args(self.name(), args.scalar_arguments)?; - args.scalar_arguments[0] + field .and_then(|sv| { sv.try_as_str() .flatten() diff --git a/datafusion/functions/src/datetime/make_date.rs b/datafusion/functions/src/datetime/make_date.rs index 3345ea2412b8..2d4db56cc788 100644 --- a/datafusion/functions/src/datetime/make_date.rs +++ b/datafusion/functions/src/datetime/make_date.rs @@ -26,6 +26,7 @@ use arrow::datatypes::DataType; use arrow::datatypes::DataType::{Date32, Int32, Int64, UInt32, UInt64, Utf8, Utf8View}; use chrono::prelude::*; +use crate::utils::take_function_args; use datafusion_common::{exec_err, Result, ScalarValue}; use datafusion_expr::{ ColumnarValue, Documentation, ScalarUDFImpl, Signature, Volatility, @@ -111,13 +112,6 @@ impl ScalarUDFImpl for MakeDateFunc { args: &[ColumnarValue], _number_rows: usize, ) -> Result { - if args.len() != 3 { - return exec_err!( - "make_date function requires 3 arguments, got {}", - args.len() - ); - } - // first, identify if any of the arguments is an Array. If yes, store its `len`, // as any scalar will need to be converted to an array of len `len`. let len = args @@ -127,9 +121,11 @@ impl ScalarUDFImpl for MakeDateFunc { ColumnarValue::Array(a) => Some(a.len()), }); - let years = args[0].cast_to(&Int32, None)?; - let months = args[1].cast_to(&Int32, None)?; - let days = args[2].cast_to(&Int32, None)?; + let [years, months, days] = take_function_args(self.name(), args)?; + + let years = years.cast_to(&Int32, None)?; + let months = months.cast_to(&Int32, None)?; + let days = days.cast_to(&Int32, None)?; let scalar_value_fn = |col: &ColumnarValue| -> Result { let ColumnarValue::Scalar(s) = col else { diff --git a/datafusion/functions/src/datetime/to_char.rs b/datafusion/functions/src/datetime/to_char.rs index a6d4c2b425ac..485fdc7a3384 100644 --- a/datafusion/functions/src/datetime/to_char.rs +++ b/datafusion/functions/src/datetime/to_char.rs @@ -28,6 +28,7 @@ use arrow::datatypes::TimeUnit::{Microsecond, Millisecond, Nanosecond, Second}; use arrow::error::ArrowError; use arrow::util::display::{ArrayFormatter, DurationFormat, FormatOptions}; +use crate::utils::take_function_args; use datafusion_common::{exec_err, Result, ScalarValue}; use datafusion_expr::TypeSignature::Exact; use datafusion_expr::{ @@ -140,28 +141,23 @@ impl ScalarUDFImpl for ToCharFunc { args: &[ColumnarValue], _number_rows: usize, ) -> Result { - if args.len() != 2 { - return exec_err!( - "to_char function requires 2 arguments, got {}", - args.len() - ); - } + let [date_time, format] = take_function_args(self.name(), args)?; - match &args[1] { + match format { ColumnarValue::Scalar(ScalarValue::Utf8(None)) | ColumnarValue::Scalar(ScalarValue::Null) => { - _to_char_scalar(args[0].clone(), None) + _to_char_scalar(date_time.clone(), None) } // constant format ColumnarValue::Scalar(ScalarValue::Utf8(Some(format))) => { // invoke to_char_scalar with the known string, without converting to array - _to_char_scalar(args[0].clone(), Some(format)) + _to_char_scalar(date_time.clone(), Some(format)) } ColumnarValue::Array(_) => _to_char_array(args), _ => { exec_err!( "Format for `to_char` must be non-null Utf8, received {:?}", - args[1].data_type() + format.data_type() ) } } diff --git a/datafusion/functions/src/lib.rs b/datafusion/functions/src/lib.rs index 7278fe3ec536..ffb0ab5f51ec 100644 --- a/datafusion/functions/src/lib.rs +++ b/datafusion/functions/src/lib.rs @@ -138,7 +138,7 @@ pub mod planner; pub mod strings; -mod utils; +pub mod utils; /// Fluent-style API for creating `Expr`s pub mod expr_fn { diff --git a/datafusion/functions/src/math/abs.rs b/datafusion/functions/src/math/abs.rs index 1af5e0dfaf37..a375af2ad29e 100644 --- a/datafusion/functions/src/math/abs.rs +++ b/datafusion/functions/src/math/abs.rs @@ -20,13 +20,14 @@ use std::any::Any; use std::sync::Arc; +use crate::utils::take_function_args; use arrow::array::{ ArrayRef, Decimal128Array, Decimal256Array, Float32Array, Float64Array, Int16Array, Int32Array, Int64Array, Int8Array, }; use arrow::datatypes::DataType; use arrow::error::ArrowError; -use datafusion_common::{exec_err, internal_datafusion_err, not_impl_err, Result}; +use datafusion_common::{internal_datafusion_err, not_impl_err, Result}; use datafusion_expr::interval_arithmetic::Interval; use datafusion_expr::sort_properties::{ExprProperties, SortProperties}; use datafusion_expr::{ @@ -34,12 +35,12 @@ use datafusion_expr::{ }; use datafusion_macros::user_doc; -type MathArrayFunction = fn(&Vec) -> Result; +type MathArrayFunction = fn(&ArrayRef) -> Result; macro_rules! make_abs_function { ($ARRAY_TYPE:ident) => {{ - |args: &Vec| { - let array = downcast_named_arg!(&args[0], "abs arg", $ARRAY_TYPE); + |input: &ArrayRef| { + let array = downcast_named_arg!(&input, "abs arg", $ARRAY_TYPE); let res: $ARRAY_TYPE = array.unary(|x| x.abs()); Ok(Arc::new(res) as ArrayRef) } @@ -48,8 +49,8 @@ macro_rules! make_abs_function { macro_rules! make_try_abs_function { ($ARRAY_TYPE:ident) => {{ - |args: &Vec| { - let array = downcast_named_arg!(&args[0], "abs arg", $ARRAY_TYPE); + |input: &ArrayRef| { + let array = downcast_named_arg!(&input, "abs arg", $ARRAY_TYPE); let res: $ARRAY_TYPE = array.try_unary(|x| { x.checked_abs().ok_or_else(|| { ArrowError::ComputeError(format!( @@ -66,11 +67,11 @@ macro_rules! make_try_abs_function { macro_rules! make_decimal_abs_function { ($ARRAY_TYPE:ident) => {{ - |args: &Vec| { - let array = downcast_named_arg!(&args[0], "abs arg", $ARRAY_TYPE); + |input: &ArrayRef| { + let array = downcast_named_arg!(&input, "abs arg", $ARRAY_TYPE); let res: $ARRAY_TYPE = array .unary(|x| x.wrapping_abs()) - .with_data_type(args[0].data_type().clone()); + .with_data_type(input.data_type().clone()); Ok(Arc::new(res) as ArrayRef) } }}; @@ -94,7 +95,7 @@ fn create_abs_function(input_data_type: &DataType) -> Result | DataType::UInt8 | DataType::UInt16 | DataType::UInt32 - | DataType::UInt64 => Ok(|args: &Vec| Ok(Arc::clone(&args[0]))), + | DataType::UInt64 => Ok(|input: &ArrayRef| Ok(Arc::clone(input))), // Decimal types DataType::Decimal128(_, _) => Ok(make_decimal_abs_function!(Decimal128Array)), @@ -172,15 +173,12 @@ impl ScalarUDFImpl for AbsFunc { _number_rows: usize, ) -> Result { let args = ColumnarValue::values_to_arrays(args)?; + let [input] = take_function_args(self.name(), args)?; - if args.len() != 1 { - return exec_err!("abs function requires 1 argument, got {}", args.len()); - } - - let input_data_type = args[0].data_type(); + let input_data_type = input.data_type(); let abs_fun = create_abs_function(input_data_type)?; - abs_fun(&args).map(ColumnarValue::Array) + abs_fun(&input).map(ColumnarValue::Array) } fn output_ordering(&self, input: &[ExprProperties]) -> Result { diff --git a/datafusion/functions/src/string/common.rs b/datafusion/functions/src/string/common.rs index 6e5f767013d4..5e0567eafea2 100644 --- a/datafusion/functions/src/string/common.rs +++ b/datafusion/functions/src/string/common.rs @@ -23,11 +23,10 @@ use std::sync::Arc; use crate::strings::make_and_append_view; use arrow::array::{ new_null_array, Array, ArrayRef, GenericStringArray, GenericStringBuilder, - OffsetSizeTrait, StringBuilder, StringViewArray, + NullBufferBuilder, OffsetSizeTrait, StringBuilder, StringViewArray, }; -use arrow::buffer::Buffer; +use arrow::buffer::{Buffer, ScalarBuffer}; use arrow::datatypes::DataType; -use arrow_buffer::{NullBufferBuilder, ScalarBuffer}; use datafusion_common::cast::{as_generic_string_array, as_string_view_array}; use datafusion_common::Result; use datafusion_common::{exec_err, ScalarValue}; diff --git a/datafusion/functions/src/strings.rs b/datafusion/functions/src/strings.rs index bb991c28fe4d..6299b353d57a 100644 --- a/datafusion/functions/src/strings.rs +++ b/datafusion/functions/src/strings.rs @@ -19,11 +19,11 @@ use std::mem::size_of; use arrow::array::{ make_view, Array, ArrayAccessor, ArrayDataBuilder, ArrayIter, ByteView, - GenericStringArray, LargeStringArray, OffsetSizeTrait, StringArray, StringViewArray, - StringViewBuilder, + GenericStringArray, LargeStringArray, NullBufferBuilder, OffsetSizeTrait, + StringArray, StringViewArray, StringViewBuilder, }; +use arrow::buffer::{MutableBuffer, NullBuffer}; use arrow::datatypes::DataType; -use arrow_buffer::{MutableBuffer, NullBuffer, NullBufferBuilder}; /// Abstracts iteration over different types of string arrays. #[deprecated(since = "45.0.0", note = "Use arrow::array::StringArrayType instead")] diff --git a/datafusion/functions/src/unicode/substr.rs b/datafusion/functions/src/unicode/substr.rs index 00737a2fe814..3767166cab51 100644 --- a/datafusion/functions/src/unicode/substr.rs +++ b/datafusion/functions/src/unicode/substr.rs @@ -22,10 +22,10 @@ use crate::strings::make_and_append_view; use crate::utils::{make_scalar_function, utf8_to_str_type}; use arrow::array::{ Array, ArrayIter, ArrayRef, AsArray, GenericStringBuilder, Int64Array, - OffsetSizeTrait, StringArrayType, StringViewArray, + NullBufferBuilder, OffsetSizeTrait, StringArrayType, StringViewArray, }; +use arrow::buffer::ScalarBuffer; use arrow::datatypes::DataType; -use arrow_buffer::{NullBufferBuilder, ScalarBuffer}; use datafusion_common::cast::as_int64_array; use datafusion_common::{exec_err, plan_err, Result}; use datafusion_expr::{ diff --git a/datafusion/functions/src/utils.rs b/datafusion/functions/src/utils.rs index 39d8aeeda460..966fd8209a04 100644 --- a/datafusion/functions/src/utils.rs +++ b/datafusion/functions/src/utils.rs @@ -18,10 +18,51 @@ use arrow::array::ArrayRef; use arrow::datatypes::DataType; -use datafusion_common::{Result, ScalarValue}; +use datafusion_common::{exec_datafusion_err, Result, ScalarValue}; use datafusion_expr::function::Hint; use datafusion_expr::ColumnarValue; +/// Converts a collection of function arguments into an fixed-size array of length N +/// producing a reasonable error message in case of unexpected number of arguments. +/// +/// # Example +/// ``` +/// # use datafusion_common::ScalarValue; +/// # use datafusion_common::Result; +/// # use datafusion_expr_common::columnar_value::ColumnarValue; +/// # use datafusion_functions::utils::take_function_args; +/// fn my_function(args: &[ColumnarValue]) -> Result<()> { +/// // function expects 2 args, so create a 2-element array +/// let [arg1, arg2] = take_function_args("my_function", args)?; +/// // ... do stuff.. +/// Ok(()) +/// } +/// +/// // Calling the function with 1 argument produces an error: +/// let ten = ColumnarValue::from(ScalarValue::from(10i32)); +/// let twenty = ColumnarValue::from(ScalarValue::from(20i32)); +/// let args = vec![ten.clone()]; +/// let err = my_function(&args).unwrap_err(); +/// assert_eq!(err.to_string(), "Execution error: my_function function requires 2 arguments, got 1"); +/// // Calling the function with 2 arguments works great +/// let args = vec![ten, twenty]; +/// my_function(&args).unwrap(); +/// ``` +pub fn take_function_args( + function_name: &str, + args: impl IntoIterator, +) -> Result<[T; N]> { + let args = args.into_iter().collect::>(); + args.try_into().map_err(|v: Vec| { + exec_datafusion_err!( + "{} function requires {} arguments, got {}", + function_name, + N, + v.len() + ) + }) +} + /// Creates a function to identify the optimal return type of a string function given /// the type of its first argument. /// diff --git a/datafusion/optimizer/src/decorrelate_predicate_subquery.rs b/datafusion/optimizer/src/decorrelate_predicate_subquery.rs index a87688c1a317..c18c48251daa 100644 --- a/datafusion/optimizer/src/decorrelate_predicate_subquery.rs +++ b/datafusion/optimizer/src/decorrelate_predicate_subquery.rs @@ -33,7 +33,7 @@ use datafusion_expr::expr_rewriter::create_col_from_scalar_expr; use datafusion_expr::logical_plan::{JoinType, Subquery}; use datafusion_expr::utils::{conjunction, split_conjunction_owned}; use datafusion_expr::{ - exists, in_subquery, not, not_exists, not_in_subquery, BinaryExpr, Expr, Filter, + exists, in_subquery, lit, not, not_exists, not_in_subquery, BinaryExpr, Expr, Filter, LogicalPlan, LogicalPlanBuilder, Operator, }; @@ -342,7 +342,7 @@ fn build_join( replace_qualified_name(filter, &all_correlated_cols, &alias).map(Some) })?; - if let Some(join_filter) = match (join_filter_opt, in_predicate_opt) { + let join_filter = match (join_filter_opt, in_predicate_opt) { ( Some(join_filter), Some(Expr::BinaryExpr(BinaryExpr { @@ -353,9 +353,9 @@ fn build_join( ) => { let right_col = create_col_from_scalar_expr(right.deref(), alias)?; let in_predicate = Expr::eq(left.deref().clone(), Expr::Column(right_col)); - Some(in_predicate.and(join_filter)) + in_predicate.and(join_filter) } - (Some(join_filter), _) => Some(join_filter), + (Some(join_filter), _) => join_filter, ( _, Some(Expr::BinaryExpr(BinaryExpr { @@ -366,24 +366,23 @@ fn build_join( ) => { let right_col = create_col_from_scalar_expr(right.deref(), alias)?; let in_predicate = Expr::eq(left.deref().clone(), Expr::Column(right_col)); - Some(in_predicate) + in_predicate } - _ => None, - } { - // join our sub query into the main plan - let new_plan = LogicalPlanBuilder::from(left.clone()) - .join_on(sub_query_alias, join_type, Some(join_filter))? - .build()?; - debug!( - "predicate subquery optimized:\n{}", - new_plan.display_indent() - ); - Ok(Some(new_plan)) - } else { - Ok(None) - } + (None, None) => lit(true), + _ => return Ok(None), + }; + // join our sub query into the main plan + let new_plan = LogicalPlanBuilder::from(left.clone()) + .join_on(sub_query_alias, join_type, Some(join_filter))? + .build()?; + debug!( + "predicate subquery optimized:\n{}", + new_plan.display_indent() + ); + Ok(Some(new_plan)) } +#[derive(Debug)] struct SubqueryInfo { query: Subquery, where_in_expr: Option, @@ -429,6 +428,7 @@ mod tests { use crate::test::*; use arrow::datatypes::{DataType, Field, Schema}; + use datafusion_expr::builder::table_source; use datafusion_expr::{and, binary_expr, col, lit, not, out_ref_col, table_scan}; fn assert_optimized_plan_equal(plan: LogicalPlan, expected: &str) -> Result<()> { @@ -1423,7 +1423,14 @@ mod tests { .project(vec![col("customer.c_custkey")])? .build()?; - assert_optimization_skipped(Arc::new(DecorrelatePredicateSubquery::new()), plan) + let expected = "Projection: customer.c_custkey [c_custkey:Int64]\ + \n LeftSemi Join: Filter: Boolean(true) [c_custkey:Int64, c_name:Utf8]\ + \n TableScan: customer [c_custkey:Int64, c_name:Utf8]\ + \n SubqueryAlias: __correlated_sq_1 [o_custkey:Int64]\ + \n Projection: orders.o_custkey [o_custkey:Int64]\ + \n Filter: orders.o_custkey = orders.o_custkey [o_orderkey:Int64, o_custkey:Int64, o_orderstatus:Utf8, o_totalprice:Float64;N]\ + \n TableScan: orders [o_orderkey:Int64, o_custkey:Int64, o_orderstatus:Utf8, o_totalprice:Float64;N]"; + assert_optimized_plan_equal(plan, expected) } /// Test for correlated exists subquery not equal @@ -1608,14 +1615,14 @@ mod tests { .project(vec![col("customer.c_custkey")])? .build()?; - // not optimized - let expected = r#"Projection: customer.c_custkey [c_custkey:Int64] - Filter: EXISTS () OR customer.c_custkey = Int32(1) [c_custkey:Int64, c_name:Utf8] - Subquery: [o_custkey:Int64] - Projection: orders.o_custkey [o_custkey:Int64] - Filter: customer.c_custkey = orders.o_custkey [o_orderkey:Int64, o_custkey:Int64, o_orderstatus:Utf8, o_totalprice:Float64;N] - TableScan: orders [o_orderkey:Int64, o_custkey:Int64, o_orderstatus:Utf8, o_totalprice:Float64;N] - TableScan: customer [c_custkey:Int64, c_name:Utf8]"#; + let expected = "Projection: customer.c_custkey [c_custkey:Int64]\ + \n Filter: __correlated_sq_1.mark OR customer.c_custkey = Int32(1) [c_custkey:Int64, c_name:Utf8, mark:Boolean]\ + \n LeftMark Join: Filter: Boolean(true) [c_custkey:Int64, c_name:Utf8, mark:Boolean]\ + \n TableScan: customer [c_custkey:Int64, c_name:Utf8]\ + \n SubqueryAlias: __correlated_sq_1 [o_custkey:Int64]\ + \n Projection: orders.o_custkey [o_custkey:Int64]\ + \n Filter: customer.c_custkey = orders.o_custkey [o_orderkey:Int64, o_custkey:Int64, o_orderstatus:Utf8, o_totalprice:Float64;N]\ + \n TableScan: orders [o_orderkey:Int64, o_custkey:Int64, o_orderstatus:Utf8, o_totalprice:Float64;N]"; assert_optimized_plan_equal(plan, expected) } @@ -1654,7 +1661,13 @@ mod tests { .project(vec![col("test.b")])? .build()?; - assert_optimization_skipped(Arc::new(DecorrelatePredicateSubquery::new()), plan) + let expected = "Projection: test.b [b:UInt32]\ + \n LeftSemi Join: Filter: Boolean(true) [a:UInt32, b:UInt32, c:UInt32]\ + \n TableScan: test [a:UInt32, b:UInt32, c:UInt32]\ + \n SubqueryAlias: __correlated_sq_1 [c:UInt32]\ + \n Projection: sq.c [c:UInt32]\ + \n TableScan: sq [a:UInt32, b:UInt32, c:UInt32]"; + assert_optimized_plan_equal(plan, expected) } /// Test for single NOT exists subquery filter @@ -1666,7 +1679,13 @@ mod tests { .project(vec![col("test.b")])? .build()?; - assert_optimization_skipped(Arc::new(DecorrelatePredicateSubquery::new()), plan) + let expected = "Projection: test.b [b:UInt32]\ + \n LeftAnti Join: Filter: Boolean(true) [a:UInt32, b:UInt32, c:UInt32]\ + \n TableScan: test [a:UInt32, b:UInt32, c:UInt32]\ + \n SubqueryAlias: __correlated_sq_1 [c:UInt32]\ + \n Projection: sq.c [c:UInt32]\ + \n TableScan: sq [a:UInt32, b:UInt32, c:UInt32]"; + assert_optimized_plan_equal(plan, expected) } #[test] @@ -1750,12 +1769,12 @@ mod tests { // Subquery and outer query refer to the same table. let expected = "Projection: test.b [b:UInt32]\ - \n Filter: EXISTS () [a:UInt32, b:UInt32, c:UInt32]\ - \n Subquery: [c:UInt32]\ + \n LeftSemi Join: Filter: Boolean(true) [a:UInt32, b:UInt32, c:UInt32]\ + \n TableScan: test [a:UInt32, b:UInt32, c:UInt32]\ + \n SubqueryAlias: __correlated_sq_1 [c:UInt32]\ \n Projection: test.c [c:UInt32]\ \n Filter: test.a > test.b [a:UInt32, b:UInt32, c:UInt32]\ - \n TableScan: test [a:UInt32, b:UInt32, c:UInt32]\ - \n TableScan: test [a:UInt32, b:UInt32, c:UInt32]"; + \n TableScan: test [a:UInt32, b:UInt32, c:UInt32]"; assert_optimized_plan_equal(plan, expected) } @@ -1844,6 +1863,68 @@ mod tests { assert_optimized_plan_equal(plan, expected) } + #[test] + fn exists_uncorrelated_unnest() -> Result<()> { + let subquery_table_source = table_source(&Schema::new(vec![Field::new( + "arr", + DataType::List(Arc::new(Field::new_list_field(DataType::Int32, true))), + true, + )])); + let subquery = LogicalPlanBuilder::scan_with_filters( + "sq", + subquery_table_source, + None, + vec![], + )? + .unnest_column("arr")? + .build()?; + let table_scan = test_table_scan()?; + let plan = LogicalPlanBuilder::from(table_scan) + .filter(exists(Arc::new(subquery)))? + .project(vec![col("test.b")])? + .build()?; + + let expected = "Projection: test.b [b:UInt32]\ + \n LeftSemi Join: Filter: Boolean(true) [a:UInt32, b:UInt32, c:UInt32]\ + \n TableScan: test [a:UInt32, b:UInt32, c:UInt32]\ + \n SubqueryAlias: __correlated_sq_1 [arr:Int32;N]\ + \n Unnest: lists[sq.arr|depth=1] structs[] [arr:Int32;N]\ + \n TableScan: sq [arr:List(Field { name: \"item\", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} });N]"; + assert_optimized_plan_equal(plan, expected) + } + + #[test] + fn exists_correlated_unnest() -> Result<()> { + let table_scan = test_table_scan()?; + let subquery_table_source = table_source(&Schema::new(vec![Field::new( + "a", + DataType::List(Arc::new(Field::new_list_field(DataType::UInt32, true))), + true, + )])); + let subquery = LogicalPlanBuilder::scan_with_filters( + "sq", + subquery_table_source, + None, + vec![], + )? + .unnest_column("a")? + .filter(col("a").eq(out_ref_col(DataType::UInt32, "test.b")))? + .build()?; + let plan = LogicalPlanBuilder::from(table_scan) + .filter(exists(Arc::new(subquery)))? + .project(vec![col("test.b")])? + .build()?; + + let expected = "Projection: test.b [b:UInt32]\ + \n LeftSemi Join: Filter: __correlated_sq_1.a = test.b [a:UInt32, b:UInt32, c:UInt32]\ + \n TableScan: test [a:UInt32, b:UInt32, c:UInt32]\ + \n SubqueryAlias: __correlated_sq_1 [a:UInt32;N]\ + \n Unnest: lists[sq.a|depth=1] structs[] [a:UInt32;N]\ + \n TableScan: sq [a:List(Field { name: \"item\", data_type: UInt32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} });N]"; + + assert_optimized_plan_equal(plan, expected) + } + #[test] fn upper_case_ident() -> Result<()> { let fields = vec![ diff --git a/datafusion/optimizer/src/unwrap_cast_in_comparison.rs b/datafusion/optimizer/src/unwrap_cast_in_comparison.rs index e2b8a966cb92..4964ebd55517 100644 --- a/datafusion/optimizer/src/unwrap_cast_in_comparison.rs +++ b/datafusion/optimizer/src/unwrap_cast_in_comparison.rs @@ -25,6 +25,7 @@ use crate::optimizer::ApplyOrder; use crate::{OptimizerConfig, OptimizerRule}; use crate::utils::NamePreserver; +#[allow(deprecated)] use arrow::datatypes::{ DataType, TimeUnit, MAX_DECIMAL128_FOR_EACH_PRECISION, MIN_DECIMAL128_FOR_EACH_PRECISION, diff --git a/datafusion/physical-expr-common/Cargo.toml b/datafusion/physical-expr-common/Cargo.toml index 00ddb11cc42d..14d6ca64d15e 100644 --- a/datafusion/physical-expr-common/Cargo.toml +++ b/datafusion/physical-expr-common/Cargo.toml @@ -38,7 +38,6 @@ path = "src/lib.rs" [dependencies] ahash = { workspace = true } arrow = { workspace = true } -arrow-buffer = { workspace = true } datafusion-common = { workspace = true, default-features = true } datafusion-expr-common = { workspace = true } hashbrown = { workspace = true } diff --git a/datafusion/physical-expr-common/src/binary_map.rs b/datafusion/physical-expr-common/src/binary_map.rs index bdff494518da..809c619e9845 100644 --- a/datafusion/physical-expr-common/src/binary_map.rs +++ b/datafusion/physical-expr-common/src/binary_map.rs @@ -19,14 +19,14 @@ //! StringArray / LargeStringArray / BinaryArray / LargeBinaryArray. use ahash::RandomState; -use arrow::array::cast::AsArray; -use arrow::array::types::{ByteArrayType, GenericBinaryType, GenericStringType}; use arrow::array::{ + cast::AsArray, + types::{ByteArrayType, GenericBinaryType, GenericStringType}, Array, ArrayRef, BufferBuilder, GenericBinaryArray, GenericStringArray, - OffsetSizeTrait, + NullBufferBuilder, OffsetSizeTrait, }; +use arrow::buffer::{NullBuffer, OffsetBuffer, ScalarBuffer}; use arrow::datatypes::DataType; -use arrow_buffer::{NullBuffer, NullBufferBuilder, OffsetBuffer, ScalarBuffer}; use datafusion_common::hash_utils::create_hashes; use datafusion_common::utils::proxy::{HashTableAllocExt, VecAllocExt}; use std::any::type_name; diff --git a/datafusion/physical-expr/Cargo.toml b/datafusion/physical-expr/Cargo.toml index 5e0832673697..d93a402db318 100644 --- a/datafusion/physical-expr/Cargo.toml +++ b/datafusion/physical-expr/Cargo.toml @@ -39,7 +39,6 @@ path = "src/lib.rs" ahash = { workspace = true } arrow = { workspace = true } arrow-array = { workspace = true } -arrow-buffer = { workspace = true } arrow-schema = { workspace = true } datafusion-common = { workspace = true, default-features = true } datafusion-expr = { workspace = true } diff --git a/datafusion/physical-expr/src/equivalence/properties.rs b/datafusion/physical-expr/src/equivalence/properties.rs index a6417044a061..e0edf5686035 100755 --- a/datafusion/physical-expr/src/equivalence/properties.rs +++ b/datafusion/physical-expr/src/equivalence/properties.rs @@ -2148,17 +2148,38 @@ fn calculate_union_binary( }) .collect::>(); + // TEMP HACK WORKAROUND + // Revert code from https://github.com/apache/datafusion/pull/12562 + // Context: https://github.com/apache/datafusion/issues/13748 + // Context: https://github.com/influxdata/influxdb_iox/issues/13038 + // Next, calculate valid orderings for the union by searching for prefixes // in both sides. - let mut orderings = UnionEquivalentOrderingBuilder::new(); - orderings.add_satisfied_orderings(lhs.normalized_oeq_class(), lhs.constants(), &rhs); - orderings.add_satisfied_orderings(rhs.normalized_oeq_class(), rhs.constants(), &lhs); - let orderings = orderings.build(); - - let mut eq_properties = - EquivalenceProperties::new(lhs.schema).with_constants(constants); - + let mut orderings = vec![]; + for mut ordering in lhs.normalized_oeq_class().into_iter() { + // Progressively shorten the ordering to search for a satisfied prefix: + while !rhs.ordering_satisfy(&ordering) { + ordering.pop(); + } + // There is a non-trivial satisfied prefix, add it as a valid ordering: + if !ordering.is_empty() { + orderings.push(ordering); + } + } + for mut ordering in rhs.normalized_oeq_class().into_iter() { + // Progressively shorten the ordering to search for a satisfied prefix: + while !lhs.ordering_satisfy(&ordering) { + ordering.pop(); + } + // There is a non-trivial satisfied prefix, add it as a valid ordering: + if !ordering.is_empty() { + orderings.push(ordering); + } + } + let mut eq_properties = EquivalenceProperties::new(lhs.schema); + eq_properties.constants = constants; eq_properties.add_new_orderings(orderings); + Ok(eq_properties) } @@ -2204,6 +2225,7 @@ struct UnionEquivalentOrderingBuilder { orderings: Vec, } +#[expect(unused)] impl UnionEquivalentOrderingBuilder { fn new() -> Self { Self { orderings: vec![] } @@ -3552,134 +3574,6 @@ mod tests { .run() } - #[test] - fn test_union_equivalence_properties_constants_fill_gaps() { - let schema = create_test_schema().unwrap(); - UnionEquivalenceTest::new(&schema) - .with_child_sort_and_const_exprs( - // First child orderings: [a ASC, c ASC], const [b] - vec![vec!["a", "c"]], - vec!["b"], - &schema, - ) - .with_child_sort_and_const_exprs( - // Second child orderings: [b ASC, c ASC], const [a] - vec![vec!["b", "c"]], - vec!["a"], - &schema, - ) - .with_expected_sort_and_const_exprs( - // Union orderings: [ - // [a ASC, b ASC, c ASC], - // [b ASC, a ASC, c ASC] - // ], const [] - vec![vec!["a", "b", "c"], vec!["b", "a", "c"]], - vec![], - ) - .run() - } - - #[test] - fn test_union_equivalence_properties_constants_no_fill_gaps() { - let schema = create_test_schema().unwrap(); - UnionEquivalenceTest::new(&schema) - .with_child_sort_and_const_exprs( - // First child orderings: [a ASC, c ASC], const [d] // some other constant - vec![vec!["a", "c"]], - vec!["d"], - &schema, - ) - .with_child_sort_and_const_exprs( - // Second child orderings: [b ASC, c ASC], const [a] - vec![vec!["b", "c"]], - vec!["a"], - &schema, - ) - .with_expected_sort_and_const_exprs( - // Union orderings: [[a]] (only a is constant) - vec![vec!["a"]], - vec![], - ) - .run() - } - - #[test] - fn test_union_equivalence_properties_constants_fill_some_gaps() { - let schema = create_test_schema().unwrap(); - UnionEquivalenceTest::new(&schema) - .with_child_sort_and_const_exprs( - // First child orderings: [c ASC], const [a, b] // some other constant - vec![vec!["c"]], - vec!["a", "b"], - &schema, - ) - .with_child_sort_and_const_exprs( - // Second child orderings: [a DESC, b], const [] - vec![vec!["a DESC", "b"]], - vec![], - &schema, - ) - .with_expected_sort_and_const_exprs( - // Union orderings: [[a, b]] (can fill in the a/b with constants) - vec![vec!["a DESC", "b"]], - vec![], - ) - .run() - } - - #[test] - fn test_union_equivalence_properties_constants_fill_gaps_non_symmetric() { - let schema = create_test_schema().unwrap(); - UnionEquivalenceTest::new(&schema) - .with_child_sort_and_const_exprs( - // First child orderings: [a ASC, c ASC], const [b] - vec![vec!["a", "c"]], - vec!["b"], - &schema, - ) - .with_child_sort_and_const_exprs( - // Second child orderings: [b ASC, c ASC], const [a] - vec![vec!["b DESC", "c"]], - vec!["a"], - &schema, - ) - .with_expected_sort_and_const_exprs( - // Union orderings: [ - // [a ASC, b ASC, c ASC], - // [b ASC, a ASC, c ASC] - // ], const [] - vec![vec!["a", "b DESC", "c"], vec!["b DESC", "a", "c"]], - vec![], - ) - .run() - } - - #[test] - fn test_union_equivalence_properties_constants_gap_fill_symmetric() { - let schema = create_test_schema().unwrap(); - UnionEquivalenceTest::new(&schema) - .with_child_sort_and_const_exprs( - // First child: [a ASC, b ASC, d ASC], const [c] - vec![vec!["a", "b", "d"]], - vec!["c"], - &schema, - ) - .with_child_sort_and_const_exprs( - // Second child: [a ASC, c ASC, d ASC], const [b] - vec![vec!["a", "c", "d"]], - vec!["b"], - &schema, - ) - .with_expected_sort_and_const_exprs( - // Union orderings: - // [a, b, c, d] - // [a, c, b, d] - vec![vec!["a", "c", "b", "d"], vec!["a", "b", "c", "d"]], - vec![], - ) - .run() - } - #[test] fn test_union_equivalence_properties_constants_gap_fill_and_common() { let schema = create_test_schema().unwrap(); @@ -3705,34 +3599,6 @@ mod tests { .run() } - #[test] - fn test_union_equivalence_properties_constants_middle_desc() { - let schema = create_test_schema().unwrap(); - UnionEquivalenceTest::new(&schema) - .with_child_sort_and_const_exprs( - // NB `b DESC` in the first child - // - // First child: [a ASC, b DESC, d ASC], const [c] - vec![vec!["a", "b DESC", "d"]], - vec!["c"], - &schema, - ) - .with_child_sort_and_const_exprs( - // Second child: [a ASC, c ASC, d ASC], const [b] - vec![vec!["a", "c", "d"]], - vec!["b"], - &schema, - ) - .with_expected_sort_and_const_exprs( - // Union orderings: - // [a, b, d] (c constant) - // [a, c, d] (b constant) - vec![vec!["a", "c", "b DESC", "d"], vec!["a", "b DESC", "c", "d"]], - vec![], - ) - .run() - } - // TODO tests with multiple constants #[derive(Debug)] diff --git a/datafusion/physical-expr/src/expressions/in_list.rs b/datafusion/physical-expr/src/expressions/in_list.rs index 29577740aab4..dfe9a905dfea 100644 --- a/datafusion/physical-expr/src/expressions/in_list.rs +++ b/datafusion/physical-expr/src/expressions/in_list.rs @@ -25,6 +25,7 @@ use std::sync::Arc; use crate::physical_expr::physical_exprs_bag_equal; use crate::PhysicalExpr; +use arrow::array::types::{IntervalDayTime, IntervalMonthDayNano}; use arrow::array::*; use arrow::buffer::BooleanBuffer; use arrow::compute::kernels::boolean::{not, or_kleene}; @@ -32,7 +33,6 @@ use arrow::compute::take; use arrow::datatypes::*; use arrow::util::bit_iterator::BitIndexIterator; use arrow::{downcast_dictionary_array, downcast_primitive_array}; -use arrow_buffer::{IntervalDayTime, IntervalMonthDayNano}; use datafusion_common::cast::{ as_boolean_array, as_generic_binary_array, as_string_array, }; diff --git a/datafusion/physical-expr/src/expressions/is_not_null.rs b/datafusion/physical-expr/src/expressions/is_not_null.rs index 4930865f4c98..8e3544622b80 100644 --- a/datafusion/physical-expr/src/expressions/is_not_null.rs +++ b/datafusion/physical-expr/src/expressions/is_not_null.rs @@ -115,12 +115,12 @@ pub fn is_not_null(arg: Arc) -> Result> mod tests { use super::*; use crate::expressions::col; + use arrow::buffer::ScalarBuffer; use arrow::{ array::{BooleanArray, StringArray}, datatypes::*, }; use arrow_array::{Array, Float64Array, Int32Array, UnionArray}; - use arrow_buffer::ScalarBuffer; use datafusion_common::cast::as_boolean_array; #[test] diff --git a/datafusion/physical-expr/src/expressions/is_null.rs b/datafusion/physical-expr/src/expressions/is_null.rs index 6a02d5ecc1f2..ca8d67230557 100644 --- a/datafusion/physical-expr/src/expressions/is_null.rs +++ b/datafusion/physical-expr/src/expressions/is_null.rs @@ -114,12 +114,12 @@ pub fn is_null(arg: Arc) -> Result> { mod tests { use super::*; use crate::expressions::col; + use arrow::buffer::ScalarBuffer; use arrow::{ array::{BooleanArray, StringArray}, datatypes::*, }; use arrow_array::{Array, Float64Array, Int32Array, UnionArray}; - use arrow_buffer::ScalarBuffer; use datafusion_common::cast::as_boolean_array; #[test] diff --git a/datafusion/physical-expr/src/intervals/cp_solver.rs b/datafusion/physical-expr/src/intervals/cp_solver.rs index f5a83c58deec..166d2564fdf3 100644 --- a/datafusion/physical-expr/src/intervals/cp_solver.rs +++ b/datafusion/physical-expr/src/intervals/cp_solver.rs @@ -722,8 +722,8 @@ mod tests { use crate::expressions::{BinaryExpr, Column}; use crate::intervals::test_utils::gen_conjunctive_numerical_expr; + use arrow::array::types::{IntervalDayTime, IntervalMonthDayNano}; use arrow::datatypes::TimeUnit; - use arrow_buffer::{IntervalDayTime, IntervalMonthDayNano}; use arrow_schema::Field; use datafusion_common::ScalarValue; diff --git a/datafusion/physical-expr/src/intervals/utils.rs b/datafusion/physical-expr/src/intervals/utils.rs index 496db7b454df..56af8238c04e 100644 --- a/datafusion/physical-expr/src/intervals/utils.rs +++ b/datafusion/physical-expr/src/intervals/utils.rs @@ -24,7 +24,7 @@ use crate::{ PhysicalExpr, }; -use arrow_buffer::{IntervalDayTime, IntervalMonthDayNano}; +use arrow::array::types::{IntervalDayTime, IntervalMonthDayNano}; use arrow_schema::{DataType, SchemaRef}; use datafusion_common::{internal_err, Result, ScalarValue}; use datafusion_expr::interval_arithmetic::Interval; diff --git a/datafusion/physical-optimizer/src/limit_pushdown.rs b/datafusion/physical-optimizer/src/limit_pushdown.rs index bc0b64cdd7e4..5887cb51a727 100644 --- a/datafusion/physical-optimizer/src/limit_pushdown.rs +++ b/datafusion/physical-optimizer/src/limit_pushdown.rs @@ -146,15 +146,6 @@ pub fn pushdown_limit_helper( global_state.skip = skip; global_state.fetch = fetch; - if limit_exec.input().as_any().is::() { - // If the child is a `CoalescePartitionsExec`, we should not remove the limit - // the push_down through the `CoalescePartitionsExec` to each partition will not guarantee the limit. - // TODO: we may have a better solution if we can support with_fetch for limit inside CoalescePartitionsExec. - // Follow-up issue: https://github.com/apache/datafusion/issues/14446 - global_state.satisfied = true; - return Ok((Transformed::no(pushdown_plan), global_state)); - } - // Now the global state has the most recent information, we can remove // the `LimitExec` plan. We will decide later if we should add it again // or not. diff --git a/datafusion/physical-optimizer/src/sanity_checker.rs b/datafusion/physical-optimizer/src/sanity_checker.rs index 8edbb0f09114..ad01a0047eaa 100644 --- a/datafusion/physical-optimizer/src/sanity_checker.rs +++ b/datafusion/physical-optimizer/src/sanity_checker.rs @@ -32,6 +32,8 @@ use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; use datafusion_physical_expr::intervals::utils::{check_support, is_datatype_supported}; use datafusion_physical_plan::execution_plan::{Boundedness, EmissionType}; use datafusion_physical_plan::joins::SymmetricHashJoinExec; +use datafusion_physical_plan::sorts::sort::SortExec; +use datafusion_physical_plan::union::UnionExec; use datafusion_physical_plan::{get_plan_string, ExecutionPlanProperties}; use crate::PhysicalOptimizerRule; @@ -135,6 +137,14 @@ pub fn check_plan_sanity( plan.required_input_ordering(), plan.required_input_distribution(), ) { + // TEMP HACK WORKAROUND https://github.com/apache/datafusion/issues/11492 + if child.as_any().downcast_ref::().is_some() { + continue; + } + if child.as_any().downcast_ref::().is_some() { + continue; + } + let child_eq_props = child.equivalence_properties(); if let Some(sort_req) = sort_req { if !child_eq_props.ordering_satisfy_requirement(&sort_req) { diff --git a/datafusion/physical-plan/Cargo.toml b/datafusion/physical-plan/Cargo.toml index a72d19cda3b1..a002e3861f11 100644 --- a/datafusion/physical-plan/Cargo.toml +++ b/datafusion/physical-plan/Cargo.toml @@ -42,7 +42,6 @@ path = "src/lib.rs" ahash = { workspace = true } arrow = { workspace = true } arrow-array = { workspace = true } -arrow-buffer = { workspace = true } arrow-ord = { workspace = true } arrow-schema = { workspace = true } async-trait = { workspace = true } diff --git a/datafusion/physical-plan/src/aggregates/group_values/multi_group_by/bytes.rs b/datafusion/physical-plan/src/aggregates/group_values/multi_group_by/bytes.rs index e75c75a235b7..c4525256dbae 100644 --- a/datafusion/physical-plan/src/aggregates/group_values/multi_group_by/bytes.rs +++ b/datafusion/physical-plan/src/aggregates/group_values/multi_group_by/bytes.rs @@ -17,11 +17,12 @@ use crate::aggregates::group_values::multi_group_by::{nulls_equal_to, GroupColumn}; use crate::aggregates::group_values::null_builder::MaybeNullBufferBuilder; -use arrow::array::{AsArray, BufferBuilder, GenericBinaryArray, GenericStringArray}; +use arrow::array::{ + types::GenericStringType, Array, ArrayRef, AsArray, BufferBuilder, + GenericBinaryArray, GenericByteArray, GenericStringArray, OffsetSizeTrait, +}; use arrow::buffer::{OffsetBuffer, ScalarBuffer}; use arrow::datatypes::{ByteArrayType, DataType, GenericBinaryType}; -use arrow_array::types::GenericStringType; -use arrow_array::{Array, ArrayRef, GenericByteArray, OffsetSizeTrait}; use datafusion_common::utils::proxy::VecAllocExt; use datafusion_physical_expr_common::binary_map::{OutputType, INITIAL_BUFFER_CAPACITY}; use itertools::izip; @@ -404,8 +405,7 @@ mod tests { use std::sync::Arc; use crate::aggregates::group_values::multi_group_by::bytes::ByteGroupValueBuilder; - use arrow_array::{ArrayRef, StringArray}; - use arrow_buffer::NullBufferBuilder; + use arrow::array::{ArrayRef, NullBufferBuilder, StringArray}; use datafusion_physical_expr::binary_map::OutputType; use super::GroupColumn; diff --git a/datafusion/physical-plan/src/aggregates/group_values/multi_group_by/bytes_view.rs b/datafusion/physical-plan/src/aggregates/group_values/multi_group_by/bytes_view.rs index c3d88b894999..d170411b833c 100644 --- a/datafusion/physical-plan/src/aggregates/group_values/multi_group_by/bytes_view.rs +++ b/datafusion/physical-plan/src/aggregates/group_values/multi_group_by/bytes_view.rs @@ -18,10 +18,9 @@ use crate::aggregates::group_values::multi_group_by::{nulls_equal_to, GroupColumn}; use crate::aggregates::group_values::null_builder::MaybeNullBufferBuilder; use arrow::array::{make_view, AsArray, ByteView}; -use arrow::buffer::ScalarBuffer; +use arrow::buffer::{Buffer, ScalarBuffer}; use arrow::datatypes::ByteViewType; use arrow_array::{Array, ArrayRef, GenericByteViewArray}; -use arrow_buffer::Buffer; use itertools::izip; use std::marker::PhantomData; use std::mem::{replace, size_of}; @@ -545,10 +544,8 @@ mod tests { use std::sync::Arc; use crate::aggregates::group_values::multi_group_by::bytes_view::ByteViewGroupValueBuilder; - use arrow::array::AsArray; + use arrow::array::{ArrayRef, AsArray, NullBufferBuilder, StringViewArray}; use arrow::datatypes::StringViewType; - use arrow_array::{ArrayRef, StringViewArray}; - use arrow_buffer::NullBufferBuilder; use super::GroupColumn; diff --git a/datafusion/physical-plan/src/aggregates/group_values/multi_group_by/primitive.rs b/datafusion/physical-plan/src/aggregates/group_values/multi_group_by/primitive.rs index cd5dfae86ee9..c85245d05592 100644 --- a/datafusion/physical-plan/src/aggregates/group_values/multi_group_by/primitive.rs +++ b/datafusion/physical-plan/src/aggregates/group_values/multi_group_by/primitive.rs @@ -17,9 +17,8 @@ use crate::aggregates::group_values::multi_group_by::{nulls_equal_to, GroupColumn}; use crate::aggregates::group_values::null_builder::MaybeNullBufferBuilder; +use arrow::array::{cast::AsArray, Array, ArrayRef, ArrowPrimitiveType, PrimitiveArray}; use arrow::buffer::ScalarBuffer; -use arrow_array::cast::AsArray; -use arrow_array::{Array, ArrayRef, ArrowPrimitiveType, PrimitiveArray}; use arrow_schema::DataType; use datafusion_execution::memory_pool::proxy::VecAllocExt; use itertools::izip; @@ -212,9 +211,8 @@ mod tests { use std::sync::Arc; use crate::aggregates::group_values::multi_group_by::primitive::PrimitiveGroupValueBuilder; + use arrow::array::{ArrayRef, Int64Array, NullBufferBuilder}; use arrow::datatypes::Int64Type; - use arrow_array::{ArrayRef, Int64Array}; - use arrow_buffer::NullBufferBuilder; use arrow_schema::DataType; use super::GroupColumn; diff --git a/datafusion/physical-plan/src/aggregates/group_values/null_builder.rs b/datafusion/physical-plan/src/aggregates/group_values/null_builder.rs index a584cf58e50a..369d921d2fc8 100644 --- a/datafusion/physical-plan/src/aggregates/group_values/null_builder.rs +++ b/datafusion/physical-plan/src/aggregates/group_values/null_builder.rs @@ -15,7 +15,8 @@ // specific language governing permissions and limitations // under the License. -use arrow_buffer::{BooleanBufferBuilder, NullBuffer}; +use arrow::array::BooleanBufferBuilder; +use arrow::buffer::NullBuffer; /// Builder for an (optional) null mask /// diff --git a/datafusion/physical-plan/src/aggregates/group_values/single_group_by/primitive.rs b/datafusion/physical-plan/src/aggregates/group_values/single_group_by/primitive.rs index 78a5f619fcd6..5a6235edb25a 100644 --- a/datafusion/physical-plan/src/aggregates/group_values/single_group_by/primitive.rs +++ b/datafusion/physical-plan/src/aggregates/group_values/single_group_by/primitive.rs @@ -17,12 +17,13 @@ use crate::aggregates::group_values::GroupValues; use ahash::RandomState; +use arrow::array::types::{IntervalDayTime, IntervalMonthDayNano}; +use arrow::array::{ + cast::AsArray, ArrayRef, ArrowNativeTypeOp, ArrowPrimitiveType, NullBufferBuilder, + PrimitiveArray, +}; use arrow::datatypes::i256; use arrow::record_batch::RecordBatch; -use arrow_array::cast::AsArray; -use arrow_array::{ArrayRef, ArrowNativeTypeOp, ArrowPrimitiveType, PrimitiveArray}; -use arrow_buffer::NullBufferBuilder; -use arrow_buffer::{IntervalDayTime, IntervalMonthDayNano}; use arrow_schema::DataType; use datafusion_common::Result; use datafusion_execution::memory_pool::proxy::VecAllocExt; diff --git a/datafusion/physical-plan/src/aggregates/topk/hash_table.rs b/datafusion/physical-plan/src/aggregates/topk/hash_table.rs index 23a07ebec305..514214858fa1 100644 --- a/datafusion/physical-plan/src/aggregates/topk/hash_table.rs +++ b/datafusion/physical-plan/src/aggregates/topk/hash_table.rs @@ -20,13 +20,13 @@ use crate::aggregates::group_values::HashValue; use crate::aggregates::topk::heap::Comparable; use ahash::RandomState; +use arrow::array::types::{IntervalDayTime, IntervalMonthDayNano}; use arrow::datatypes::i256; use arrow_array::builder::PrimitiveBuilder; use arrow_array::cast::AsArray; use arrow_array::{ downcast_primitive, Array, ArrayRef, ArrowPrimitiveType, PrimitiveArray, StringArray, }; -use arrow_buffer::{IntervalDayTime, IntervalMonthDayNano}; use arrow_schema::DataType; use datafusion_common::DataFusionError; use datafusion_common::Result; diff --git a/datafusion/physical-plan/src/aggregates/topk/heap.rs b/datafusion/physical-plan/src/aggregates/topk/heap.rs index ec1277f8fd55..fc68df9b82ed 100644 --- a/datafusion/physical-plan/src/aggregates/topk/heap.rs +++ b/datafusion/physical-plan/src/aggregates/topk/heap.rs @@ -17,10 +17,11 @@ //! A custom binary heap implementation for performant top K aggregation +use arrow::array::types::{IntervalDayTime, IntervalMonthDayNano}; +use arrow::buffer::ScalarBuffer; use arrow::datatypes::i256; use arrow_array::cast::AsArray; use arrow_array::{downcast_primitive, ArrayRef, ArrowPrimitiveType, PrimitiveArray}; -use arrow_buffer::{IntervalDayTime, IntervalMonthDayNano, ScalarBuffer}; use arrow_schema::DataType; use datafusion_common::DataFusionError; use datafusion_common::Result; diff --git a/datafusion/physical-plan/src/coalesce_partitions.rs b/datafusion/physical-plan/src/coalesce_partitions.rs index 3900bd1ddca9..9a955155c01e 100644 --- a/datafusion/physical-plan/src/coalesce_partitions.rs +++ b/datafusion/physical-plan/src/coalesce_partitions.rs @@ -43,6 +43,8 @@ pub struct CoalescePartitionsExec { /// Execution metrics metrics: ExecutionPlanMetricsSet, cache: PlanProperties, + /// Optional number of rows to fetch. Stops producing rows after this fetch + pub(crate) fetch: Option, } impl CoalescePartitionsExec { @@ -53,6 +55,7 @@ impl CoalescePartitionsExec { input, metrics: ExecutionPlanMetricsSet::new(), cache, + fetch: None, } } @@ -83,9 +86,12 @@ impl DisplayAs for CoalescePartitionsExec { f: &mut std::fmt::Formatter, ) -> std::fmt::Result { match t { - DisplayFormatType::Default | DisplayFormatType::Verbose => { - write!(f, "CoalescePartitionsExec") - } + DisplayFormatType::Default | DisplayFormatType::Verbose => match self.fetch { + Some(fetch) => { + write!(f, "CoalescePartitionsExec: fetch={fetch}") + } + None => write!(f, "CoalescePartitionsExec"), + }, } } } @@ -116,9 +122,9 @@ impl ExecutionPlan for CoalescePartitionsExec { self: Arc, children: Vec>, ) -> Result> { - Ok(Arc::new(CoalescePartitionsExec::new(Arc::clone( - &children[0], - )))) + let mut plan = CoalescePartitionsExec::new(Arc::clone(&children[0])); + plan.fetch = self.fetch; + Ok(Arc::new(plan)) } fn execute( @@ -164,7 +170,11 @@ impl ExecutionPlan for CoalescePartitionsExec { } let stream = builder.build(); - Ok(Box::pin(ObservedStream::new(stream, baseline_metrics))) + Ok(Box::pin(ObservedStream::new( + stream, + baseline_metrics, + self.fetch, + ))) } } } @@ -174,7 +184,7 @@ impl ExecutionPlan for CoalescePartitionsExec { } fn statistics(&self) -> Result { - self.input.statistics() + Statistics::with_fetch(self.input.statistics()?, self.schema(), self.fetch, 0, 1) } fn supports_limit_pushdown(&self) -> bool { @@ -197,8 +207,28 @@ impl ExecutionPlan for CoalescePartitionsExec { return Ok(None); } // CoalescePartitionsExec always has a single child, so zero indexing is safe. - make_with_child(projection, projection.input().children()[0]) - .map(|e| Some(Arc::new(CoalescePartitionsExec::new(e)) as _)) + make_with_child(projection, projection.input().children()[0]).map(|e| { + if self.fetch.is_some() { + let mut plan = CoalescePartitionsExec::new(e); + plan.fetch = self.fetch; + Some(Arc::new(plan) as _) + } else { + Some(Arc::new(CoalescePartitionsExec::new(e)) as _) + } + }) + } + + fn fetch(&self) -> Option { + self.fetch + } + + fn with_fetch(&self, limit: Option) -> Option> { + Some(Arc::new(CoalescePartitionsExec { + input: Arc::clone(&self.input), + fetch: limit, + metrics: self.metrics.clone(), + cache: self.cache.clone(), + })) } } diff --git a/datafusion/physical-plan/src/joins/hash_join.rs b/datafusion/physical-plan/src/joins/hash_join.rs index 1bec67ba2e19..76e535d93b7e 100644 --- a/datafusion/physical-plan/src/joins/hash_join.rs +++ b/datafusion/physical-plan/src/joins/hash_join.rs @@ -1645,9 +1645,9 @@ mod tests { }; use arrow::array::{Date32Array, Int32Array}; + use arrow::buffer::NullBuffer; use arrow::datatypes::{DataType, Field}; use arrow_array::StructArray; - use arrow_buffer::NullBuffer; use datafusion_common::{ assert_batches_eq, assert_batches_sorted_eq, assert_contains, exec_err, ScalarValue, diff --git a/datafusion/physical-plan/src/joins/mod.rs b/datafusion/physical-plan/src/joins/mod.rs index bfdeb2fd6e27..fdb5cdeb5136 100644 --- a/datafusion/physical-plan/src/joins/mod.rs +++ b/datafusion/physical-plan/src/joins/mod.rs @@ -17,7 +17,7 @@ //! DataFusion Join implementations -use arrow_buffer::BooleanBufferBuilder; +use arrow::array::BooleanBufferBuilder; pub use cross_join::CrossJoinExec; pub use hash_join::HashJoinExec; pub use nested_loop_join::NestedLoopJoinExec; diff --git a/datafusion/physical-plan/src/joins/stream_join_utils.rs b/datafusion/physical-plan/src/joins/stream_join_utils.rs index 6d4f06b3aef2..a3b3a37aa7ef 100644 --- a/datafusion/physical-plan/src/joins/stream_join_utils.rs +++ b/datafusion/physical-plan/src/joins/stream_join_utils.rs @@ -26,9 +26,11 @@ use crate::joins::utils::{JoinFilter, JoinHashMapType}; use crate::metrics::{ExecutionPlanMetricsSet, MetricBuilder}; use crate::{metrics, ExecutionPlan}; +use arrow::array::{ + ArrowPrimitiveType, BooleanBufferBuilder, NativeAdapter, PrimitiveArray, RecordBatch, +}; use arrow::compute::concat_batches; -use arrow_array::{ArrowPrimitiveType, NativeAdapter, PrimitiveArray, RecordBatch}; -use arrow_buffer::{ArrowNativeType, BooleanBufferBuilder}; +use arrow::datatypes::ArrowNativeType; use arrow_schema::{Schema, SchemaRef}; use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; use datafusion_common::{ diff --git a/datafusion/physical-plan/src/joins/symmetric_hash_join.rs b/datafusion/physical-plan/src/joins/symmetric_hash_join.rs index 59aab3395ea2..47af4ab9a765 100644 --- a/datafusion/physical-plan/src/joins/symmetric_hash_join.rs +++ b/datafusion/physical-plan/src/joins/symmetric_hash_join.rs @@ -63,9 +63,8 @@ use arrow::array::{ UInt64Array, }; use arrow::compute::concat_batches; -use arrow::datatypes::{Schema, SchemaRef}; +use arrow::datatypes::{ArrowNativeType, Schema, SchemaRef}; use arrow::record_batch::RecordBatch; -use arrow_buffer::ArrowNativeType; use datafusion_common::hash_utils::create_hashes; use datafusion_common::utils::bisect; use datafusion_common::{internal_err, plan_err, HashSet, JoinSide, JoinType, Result}; diff --git a/datafusion/physical-plan/src/joins/test_utils.rs b/datafusion/physical-plan/src/joins/test_utils.rs index df46cb7c96ed..a866f7291bbd 100644 --- a/datafusion/physical-plan/src/joins/test_utils.rs +++ b/datafusion/physical-plan/src/joins/test_utils.rs @@ -28,12 +28,12 @@ use crate::repartition::RepartitionExec; use crate::source::DataSourceExec; use crate::{common, ExecutionPlan, ExecutionPlanProperties, Partitioning}; +use arrow::array::types::IntervalDayTime; use arrow::util::pretty::pretty_format_batches; use arrow_array::{ ArrayRef, Float64Array, Int32Array, IntervalDayTimeArray, RecordBatch, TimestampMillisecondArray, }; -use arrow_buffer::IntervalDayTime; use arrow_schema::{DataType, Schema}; use datafusion_common::{Result, ScalarValue}; use datafusion_execution::TaskContext; diff --git a/datafusion/physical-plan/src/joins/utils.rs b/datafusion/physical-plan/src/joins/utils.rs index 5327793d01e2..dbe90077bc8c 100644 --- a/datafusion/physical-plan/src/joins/utils.rs +++ b/datafusion/physical-plan/src/joins/utils.rs @@ -37,11 +37,12 @@ use arrow::array::{ UInt32Builder, UInt64Array, }; use arrow::compute; -use arrow::datatypes::{Field, Schema, SchemaBuilder, UInt32Type, UInt64Type}; +use arrow::datatypes::{ + ArrowNativeType, Field, Schema, SchemaBuilder, UInt32Type, UInt64Type, +}; use arrow::record_batch::{RecordBatch, RecordBatchOptions}; use arrow_array::builder::UInt64Builder; use arrow_array::{ArrowPrimitiveType, NativeAdapter, PrimitiveArray}; -use arrow_buffer::ArrowNativeType; use datafusion_common::cast::as_boolean_array; use datafusion_common::stats::Precision; use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; diff --git a/datafusion/physical-plan/src/sorts/cursor.rs b/datafusion/physical-plan/src/sorts/cursor.rs index 5cd24b89f5c1..2c298c9327f4 100644 --- a/datafusion/physical-plan/src/sorts/cursor.rs +++ b/datafusion/physical-plan/src/sorts/cursor.rs @@ -18,6 +18,7 @@ use std::cmp::Ordering; use arrow::buffer::ScalarBuffer; +use arrow::buffer::{Buffer, OffsetBuffer}; use arrow::compute::SortOptions; use arrow::datatypes::ArrowNativeTypeOp; use arrow::row::Rows; @@ -25,7 +26,6 @@ use arrow_array::types::ByteArrayType; use arrow_array::{ Array, ArrowPrimitiveType, GenericByteArray, OffsetSizeTrait, PrimitiveArray, }; -use arrow_buffer::{Buffer, OffsetBuffer}; use datafusion_execution::memory_pool::MemoryReservation; /// A comparable collection of values for use with [`Cursor`] diff --git a/datafusion/physical-plan/src/stream.rs b/datafusion/physical-plan/src/stream.rs index 331cded165a8..5c941c76ae47 100644 --- a/datafusion/physical-plan/src/stream.rs +++ b/datafusion/physical-plan/src/stream.rs @@ -444,18 +444,44 @@ impl Stream for EmptyRecordBatchStream { pub(crate) struct ObservedStream { inner: SendableRecordBatchStream, baseline_metrics: BaselineMetrics, + fetch: Option, + produced: usize, } impl ObservedStream { pub fn new( inner: SendableRecordBatchStream, baseline_metrics: BaselineMetrics, + fetch: Option, ) -> Self { Self { inner, baseline_metrics, + fetch, + produced: 0, } } + + fn limit_reached( + &mut self, + poll: Poll>>, + ) -> Poll>> { + let Some(fetch) = self.fetch else { return poll }; + + if self.produced >= fetch { + return Poll::Ready(None); + } + + if let Poll::Ready(Some(Ok(batch))) = &poll { + if self.produced + batch.num_rows() > fetch { + let batch = batch.slice(0, fetch.saturating_sub(self.produced)); + self.produced += batch.num_rows(); + return Poll::Ready(Some(Ok(batch))); + }; + self.produced += batch.num_rows() + } + poll + } } impl RecordBatchStream for ObservedStream { @@ -471,7 +497,10 @@ impl Stream for ObservedStream { mut self: Pin<&mut Self>, cx: &mut Context<'_>, ) -> Poll> { - let poll = self.inner.poll_next_unpin(cx); + let mut poll = self.inner.poll_next_unpin(cx); + if self.fetch.is_some() { + poll = self.limit_reached(poll); + } self.baseline_metrics.record_poll(poll) } } diff --git a/datafusion/physical-plan/src/union.rs b/datafusion/physical-plan/src/union.rs index 5edb631124ca..9bf55c818a43 100644 --- a/datafusion/physical-plan/src/union.rs +++ b/datafusion/physical-plan/src/union.rs @@ -237,7 +237,11 @@ impl ExecutionPlan for UnionExec { if partition < input.output_partitioning().partition_count() { let stream = input.execute(partition, context)?; debug!("Found a Union partition to execute"); - return Ok(Box::pin(ObservedStream::new(stream, baseline_metrics))); + return Ok(Box::pin(ObservedStream::new( + stream, + baseline_metrics, + None, + ))); } else { partition -= input.output_partitioning().partition_count(); } @@ -448,7 +452,11 @@ impl ExecutionPlan for InterleaveExec { self.schema(), input_stream_vec, )); - return Ok(Box::pin(ObservedStream::new(stream, baseline_metrics))); + return Ok(Box::pin(ObservedStream::new( + stream, + baseline_metrics, + None, + ))); } warn!("Error in InterleaveExec: Partition {} not found", partition); @@ -499,31 +507,42 @@ pub fn can_interleave>>( } fn union_schema(inputs: &[Arc]) -> SchemaRef { - let first_schema = inputs[0].schema(); + // needs to handle n children, including child which have an empty projection or different number of fields + let num_fields = inputs.iter().fold(0, |acc, input| { + std::cmp::max(acc, input.schema().fields().len()) + }); - let fields = (0..first_schema.fields().len()) + let fields: Vec = (0..num_fields) .map(|i| { - inputs - .iter() - .enumerate() - .map(|(input_idx, input)| { - let field = input.schema().field(i).clone(); - let mut metadata = field.metadata().clone(); + // collect fields for i + let field_options_for_i = + inputs.iter().enumerate().filter_map(|(input_idx, input)| { + let field = if input.schema().fields().len() <= i { + return None; + } else { + input.schema().field(i).clone() + }; + // merge field metadata + let mut metadata = field.metadata().clone(); let other_metadatas = inputs .iter() .enumerate() - .filter(|(other_idx, _)| *other_idx != input_idx) + .filter(|(other_idx, other_input)| { + *other_idx != input_idx + && other_input.schema().fields().len() > i + }) .flat_map(|(_, other_input)| { other_input.schema().field(i).metadata().clone().into_iter() }); - metadata.extend(other_metadatas); - field.with_metadata(metadata) - }) + Some(field.with_metadata(metadata)) + }); + + // pick first nullable field (if exists) + field_options_for_i .find_or_first(Field::is_nullable) - // We can unwrap this because if inputs was empty, this would've already panic'ed when we - // indexed into inputs[0]. + // We can unwrap this because if inputs was empty, we would never had iterated with (0..num_fields) .unwrap() }) .collect::>(); diff --git a/datafusion/physical-plan/src/unnest.rs b/datafusion/physical-plan/src/unnest.rs index ef6797c9b10d..942dd7881052 100644 --- a/datafusion/physical-plan/src/unnest.rs +++ b/datafusion/physical-plan/src/unnest.rs @@ -951,9 +951,11 @@ fn repeat_arrs_from_indices( #[cfg(test)] mod tests { use super::*; + use arrow::array::{ + GenericListArray, NullBufferBuilder, OffsetSizeTrait, StringArray, + }; + use arrow::buffer::{NullBuffer, OffsetBuffer}; use arrow::datatypes::{Field, Int32Type}; - use arrow_array::{GenericListArray, OffsetSizeTrait, StringArray}; - use arrow_buffer::{NullBuffer, NullBufferBuilder, OffsetBuffer}; use datafusion_common::assert_batches_eq; // Create a GenericListArray with the following list values: diff --git a/datafusion/proto/src/physical_plan/mod.rs b/datafusion/proto/src/physical_plan/mod.rs index 0033b2d5b4b1..6fa28e882ed6 100644 --- a/datafusion/proto/src/physical_plan/mod.rs +++ b/datafusion/proto/src/physical_plan/mod.rs @@ -269,17 +269,16 @@ impl AsExecutionPlan for protobuf::PhysicalPlanNode { if let Some(table_options) = scan.parquet_options.as_ref() { options = table_options.try_into()?; } - let source = Arc::new(ParquetSource::new( - Arc::clone(&schema), - predicate, - None, - options, - )); + let mut source = ParquetSource::new(options); + + if let Some(predicate) = predicate { + source = source.with_predicate(Arc::clone(&schema), predicate); + } let base_config = parse_protobuf_file_scan_config( scan.base_conf.as_ref().unwrap(), registry, extension_codec, - source, + Arc::new(source), )?; Ok(base_config.new_exec()) } diff --git a/datafusion/proto/tests/cases/roundtrip_physical_plan.rs b/datafusion/proto/tests/cases/roundtrip_physical_plan.rs index 0755dbdc720e..fdd529cfd1b9 100644 --- a/datafusion/proto/tests/cases/roundtrip_physical_plan.rs +++ b/datafusion/proto/tests/cases/roundtrip_physical_plan.rs @@ -713,12 +713,9 @@ fn roundtrip_parquet_exec_with_pruning_predicate() -> Result<()> { let mut options = TableParquetOptions::new(); options.global.pushdown_filters = true; - let source = Arc::new(ParquetSource::new( - Arc::clone(&file_schema), - Some(predicate), - None, - options, - )); + let source = Arc::new( + ParquetSource::new(options).with_predicate(Arc::clone(&file_schema), predicate), + ); let scan_config = FileScanConfig { object_store_url: ObjectStoreUrl::local_filesystem(), @@ -787,12 +784,10 @@ fn roundtrip_parquet_exec_with_custom_predicate_expr() -> Result<()> { inner: Arc::new(Column::new("col", 1)), }); - let source = Arc::new(ParquetSource::new( - Arc::clone(&file_schema), - Some(custom_predicate_expr), - None, - TableParquetOptions::default(), - )); + let source = Arc::new( + ParquetSource::default() + .with_predicate(Arc::clone(&file_schema), custom_predicate_expr), + ); let scan_config = FileScanConfig { object_store_url: ObjectStoreUrl::local_filesystem(), diff --git a/datafusion/sql/src/expr/identifier.rs b/datafusion/sql/src/expr/identifier.rs index 130cdf083da3..ab5c550691bd 100644 --- a/datafusion/sql/src/expr/identifier.rs +++ b/datafusion/sql/src/expr/identifier.rs @@ -92,7 +92,7 @@ impl SqlToRel<'_, S> { } } - pub(super) fn sql_compound_identifier_to_expr( + pub(crate) fn sql_compound_identifier_to_expr( &self, ids: Vec, schema: &DFSchema, diff --git a/datafusion/sql/src/expr/mod.rs b/datafusion/sql/src/expr/mod.rs index 951e81c1fdee..de753da895d3 100644 --- a/datafusion/sql/src/expr/mod.rs +++ b/datafusion/sql/src/expr/mod.rs @@ -21,14 +21,14 @@ use datafusion_expr::planner::{ PlannerResult, RawBinaryExpr, RawDictionaryExpr, RawFieldAccessExpr, }; use sqlparser::ast::{ - BinaryOperator, CastFormat, CastKind, DataType as SQLDataType, DictionaryField, - Expr as SQLExpr, ExprWithAlias as SQLExprWithAlias, MapEntry, StructField, Subscript, - TrimWhereField, Value, + AccessExpr, BinaryOperator, CastFormat, CastKind, DataType as SQLDataType, + DictionaryField, Expr as SQLExpr, ExprWithAlias as SQLExprWithAlias, MapEntry, + StructField, Subscript, TrimWhereField, Value, }; use datafusion_common::{ - internal_datafusion_err, internal_err, not_impl_err, plan_err, DFSchema, Result, - ScalarValue, + internal_datafusion_err, internal_err, not_impl_err, plan_err, Column, DFSchema, + Result, ScalarValue, }; use datafusion_expr::expr::ScalarFunction; @@ -238,14 +238,14 @@ impl SqlToRel<'_, S> { self.sql_identifier_to_expr(id, schema, planner_context) } - SQLExpr::MapAccess { .. } => { - not_impl_err!("Map Access") - } - // ["foo"], [4] or [4:5] - SQLExpr::Subscript { expr, subscript } => { - self.sql_subscript_to_expr(*expr, subscript, schema, planner_context) - } + SQLExpr::CompoundFieldAccess { root, access_chain } => self + .sql_compound_field_access_to_expr( + *root, + access_chain, + schema, + planner_context, + ), SQLExpr::CompoundIdentifier(ids) => { self.sql_compound_identifier_to_expr(ids, schema, planner_context) @@ -982,84 +982,146 @@ impl SqlToRel<'_, S> { Ok(Expr::Cast(Cast::new(Box::new(expr), dt))) } - fn sql_subscript_to_expr( + fn sql_compound_field_access_to_expr( &self, - expr: SQLExpr, - subscript: Box, + root: SQLExpr, + access_chain: Vec, 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"); + let mut root = self.sql_expr_to_logical_expr(root, schema, planner_context)?; + let fields = access_chain + .into_iter() + .map(|field| match field { + AccessExpr::Subscript(subscript) => { + 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), + ) => Ok(Some(GetFieldAccess::NamedStructField { + name: ScalarValue::from(s), + })), + SQLExpr::JsonAccess { .. } => { + not_impl_err!("JsonAccess") + } + // otherwise treat like a list index + _ => Ok(Some(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) + }; + + Ok(Some(GetFieldAccess::ListRange { + start: Box::new(lower_bound), + stop: Box::new(upper_bound), + stride: Box::new(stride), + })) + } } - // 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), + AccessExpr::Dot(expr) => { + let expr = + self.sql_expr_to_logical_expr(expr, schema, planner_context)?; + match expr { + Expr::Column(Column { + name, + relation, + spans, + }) => { + if let Some(relation) = &relation { + // If the first part of the dot access is a column reference, we should + // check if the column is from the same table as the root expression. + // If it is, we should replace the root expression with the column reference. + // Otherwise, we should treat the dot access as a named field access. + if relation.table() == root.schema_name().to_string() { + root = Expr::Column(Column { + name, + relation: Some(relation.clone()), + spans, + }); + Ok(None) + } else { + plan_err!( + "table name mismatch: {} != {}", + relation.table(), + root.schema_name() + ) + } + } else { + Ok(Some(GetFieldAccess::NamedStructField { + name: ScalarValue::from(name), + })) + } + } + _ => not_impl_err!( + "Dot access not supported for non-column expr: {expr:?}" + ), + } } - } - }; + }) + .collect::>>()?; - 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; + fields + .into_iter() + .flatten() + .try_fold(root, |expr, field_access| { + 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:?}" - ) + not_impl_err!( + "GetFieldAccess not supported by ExprPlanner: {field_access_expr:?}" + ) + }) } } diff --git a/datafusion/sql/src/parser.rs b/datafusion/sql/src/parser.rs index cc42ec1bf311..9725166b8ae0 100644 --- a/datafusion/sql/src/parser.rs +++ b/datafusion/sql/src/parser.rs @@ -571,7 +571,7 @@ impl<'a> DFParser<'a> { loop { if let Token::Word(_) = self.parser.peek_token().token { - let identifier = self.parser.parse_identifier(false)?; + let identifier = self.parser.parse_identifier()?; partitions.push(identifier.to_string()); } else { return self.expected("partition name", self.parser.peek_token()); @@ -674,7 +674,7 @@ impl<'a> DFParser<'a> { } fn parse_column_def(&mut self) -> Result { - let name = self.parser.parse_identifier(false)?; + let name = self.parser.parse_identifier()?; let data_type = self.parser.parse_data_type()?; let collation = if self.parser.parse_keyword(Keyword::COLLATE) { Some(self.parser.parse_object_name(false)?) @@ -684,7 +684,7 @@ impl<'a> DFParser<'a> { let mut options = vec![]; loop { if self.parser.parse_keyword(Keyword::CONSTRAINT) { - let name = Some(self.parser.parse_identifier(false)?); + let name = Some(self.parser.parse_identifier()?); if let Some(option) = self.parser.parse_optional_column_option()? { options.push(ColumnOptionDef { name, option }); } else { diff --git a/datafusion/sql/src/planner.rs b/datafusion/sql/src/planner.rs index f22ff6d94fc2..85d428cae84f 100644 --- a/datafusion/sql/src/planner.rs +++ b/datafusion/sql/src/planner.rs @@ -451,7 +451,10 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { SQLDataType::UnsignedBigInt(_) | SQLDataType::UnsignedInt8(_) => Ok(DataType::UInt64), SQLDataType::Float(_) => Ok(DataType::Float32), SQLDataType::Real | SQLDataType::Float4 => Ok(DataType::Float32), - SQLDataType::Double | SQLDataType::DoublePrecision | SQLDataType::Float8 => Ok(DataType::Float64), + SQLDataType::Double(ExactNumberInfo::None) | SQLDataType::DoublePrecision | SQLDataType::Float8 => Ok(DataType::Float64), + SQLDataType::Double(ExactNumberInfo::Precision(_)|ExactNumberInfo::PrecisionAndScale(_, _)) => { + not_impl_err!("Unsupported SQL type (precision/scale not supported) {sql_type}") + } SQLDataType::Char(_) | SQLDataType::Text | SQLDataType::String(_) => Ok(DataType::Utf8), @@ -587,7 +590,9 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { | SQLDataType::MediumText | SQLDataType::LongText | SQLDataType::Bit(_) - |SQLDataType::BitVarying(_) + | SQLDataType::BitVarying(_) + // BigQuery UDFs + | SQLDataType::AnyType => not_impl_err!( "Unsupported SQL type {sql_type:?}" ), diff --git a/datafusion/sql/src/relation/join.rs b/datafusion/sql/src/relation/join.rs index 75f39792bce1..88665401dc31 100644 --- a/datafusion/sql/src/relation/join.rs +++ b/datafusion/sql/src/relation/join.rs @@ -18,7 +18,9 @@ use crate::planner::{ContextProvider, PlannerContext, SqlToRel}; use datafusion_common::{not_impl_err, Column, Result}; use datafusion_expr::{JoinType, LogicalPlan, LogicalPlanBuilder}; -use sqlparser::ast::{Join, JoinConstraint, JoinOperator, TableFactor, TableWithJoins}; +use sqlparser::ast::{ + Join, JoinConstraint, JoinOperator, ObjectName, TableFactor, TableWithJoins, +}; use std::collections::HashSet; impl SqlToRel<'_, S> { @@ -123,11 +125,22 @@ impl SqlToRel<'_, S> { .join_on(right, join_type, Some(expr))? .build() } - JoinConstraint::Using(idents) => { - let keys: Vec = idents + JoinConstraint::Using(object_names) => { + let keys = object_names .into_iter() - .map(|x| Column::from_name(self.ident_normalizer.normalize(x))) - .collect(); + .map(|object_name| { + let ObjectName(mut object_names) = object_name; + if object_names.len() != 1 { + not_impl_err!( + "Invalid identifier in USING clause. Expected single identifier, got {}", ObjectName(object_names) + ) + } else { + let id = object_names.swap_remove(0); + Ok(self.ident_normalizer.normalize(id)) + } + }) + .collect::>>()?; + LogicalPlanBuilder::from(left) .join_using(right, join_type, keys)? .build() diff --git a/datafusion/sql/src/set_expr.rs b/datafusion/sql/src/set_expr.rs index 290c0174784a..3ddbe373ecd3 100644 --- a/datafusion/sql/src/set_expr.rs +++ b/datafusion/sql/src/set_expr.rs @@ -133,6 +133,9 @@ impl SqlToRel<'_, S> { (SetOperator::Except, false) => { LogicalPlanBuilder::except(left_plan, right_plan, false) } + (SetOperator::Minus, _) => { + not_impl_err!("MINUS Set Operator not implemented") + } } } } diff --git a/datafusion/sql/src/statement.rs b/datafusion/sql/src/statement.rs index dfd3a4fd76a2..83c91ecde69a 100644 --- a/datafusion/sql/src/statement.rs +++ b/datafusion/sql/src/statement.rs @@ -56,7 +56,7 @@ use datafusion_expr::{ }; use sqlparser::ast::{ self, BeginTransactionKind, NullsDistinctOption, ShowStatementIn, - ShowStatementOptions, SqliteOnConflict, + ShowStatementOptions, SqliteOnConflict, TableObject, UpdateTableFromKind, }; use sqlparser::ast::{ Assignment, AssignmentTarget, ColumnDef, CreateIndex, CreateTable, @@ -497,6 +497,7 @@ impl SqlToRel<'_, S> { if_not_exists, temporary, to, + params, } => { if materialized { return not_impl_err!("Materialized views not supported")?; @@ -532,6 +533,7 @@ impl SqlToRel<'_, S> { if_not_exists, temporary, to, + params, }; let sql = stmt.to_string(); let Statement::CreateView { @@ -818,7 +820,6 @@ impl SqlToRel<'_, S> { Statement::Insert(Insert { or, into, - table_name, columns, overwrite, source, @@ -832,7 +833,17 @@ impl SqlToRel<'_, S> { mut replace_into, priority, insert_alias, + assignments, + has_table_keyword, + settings, + format_clause, }) => { + let table_name = match table { + TableObject::TableName(table_name) => table_name, + TableObject::TableFunction(_) => { + return not_impl_err!("INSERT INTO Table functions not supported") + } + }; if let Some(or) = or { match or { SqliteOnConflict::Replace => replace_into = true, @@ -845,9 +856,6 @@ impl SqlToRel<'_, S> { if !after_columns.is_empty() { plan_err!("After-columns clause not supported")?; } - if table { - plan_err!("Table clause not supported")?; - } if on.is_some() { plan_err!("Insert-on clause not supported")?; } @@ -873,7 +881,18 @@ impl SqlToRel<'_, S> { if insert_alias.is_some() { plan_err!("Inserts with an alias not supported")?; } - let _ = into; // optional keyword doesn't change behavior + if !assignments.is_empty() { + plan_err!("Inserts with assignments not supported")?; + } + if settings.is_some() { + plan_err!("Inserts with settings not supported")?; + } + if format_clause.is_some() { + plan_err!("Inserts with format clause not supported")?; + } + // optional keywords don't change behavior + let _ = into; + let _ = has_table_keyword; self.insert_to_plan(table_name, columns, source, overwrite, replace_into) } Statement::Update { @@ -884,6 +903,11 @@ impl SqlToRel<'_, S> { returning, or, } => { + let from = + from.map(|update_table_from_kind| match update_table_from_kind { + UpdateTableFromKind::BeforeSet(from) => from, + UpdateTableFromKind::AfterSet(from) => from, + }); if returning.is_some() { plan_err!("Update-returning clause not yet supported")?; } @@ -969,6 +993,9 @@ impl SqlToRel<'_, S> { ast::TransactionIsolationLevel::Serializable => { TransactionIsolationLevel::Serializable } + ast::TransactionIsolationLevel::Snapshot => { + TransactionIsolationLevel::Snapshot + } }; let access_mode = match access_mode { ast::TransactionAccessMode::ReadOnly => { @@ -984,7 +1011,17 @@ impl SqlToRel<'_, S> { }); Ok(LogicalPlan::Statement(statement)) } - Statement::Commit { chain } => { + Statement::Commit { + chain, + end, + modifier, + } => { + if end { + return not_impl_err!("COMMIT AND END not supported"); + }; + if let Some(modifier) = modifier { + return not_impl_err!("COMMIT {modifier} not supported"); + }; let statement = PlanStatement::TransactionEnd(TransactionEnd { conclusion: TransactionConclusion::Commit, chain, diff --git a/datafusion/sql/src/unparser/ast.rs b/datafusion/sql/src/unparser/ast.rs index e320a4510e46..6d77c01ea888 100644 --- a/datafusion/sql/src/unparser/ast.rs +++ b/datafusion/sql/src/unparser/ast.rs @@ -466,6 +466,7 @@ impl TableRelationBuilder { partitions: self.partitions.clone(), with_ordinality: false, json_path: None, + sample: None, }) } fn create_empty() -> Self { diff --git a/datafusion/sql/src/unparser/dialect.rs b/datafusion/sql/src/unparser/dialect.rs index 830435fd013c..adfb7a0d0cd2 100644 --- a/datafusion/sql/src/unparser/dialect.rs +++ b/datafusion/sql/src/unparser/dialect.rs @@ -17,6 +17,7 @@ use std::{collections::HashMap, sync::Arc}; +use super::{utils::character_length_to_sql, utils::date_part_to_sql, Unparser}; use arrow_schema::TimeUnit; use datafusion_common::Result; use datafusion_expr::Expr; @@ -29,8 +30,6 @@ use sqlparser::{ keywords::ALL_KEYWORDS, }; -use super::{utils::character_length_to_sql, utils::date_part_to_sql, Unparser}; - pub type ScalarFnToSqlHandler = Box Result> + Send + Sync>; @@ -65,7 +64,7 @@ pub trait Dialect: Send + Sync { /// Does the dialect use DOUBLE PRECISION to represent Float64 rather than DOUBLE? /// E.g. Postgres uses DOUBLE PRECISION instead of DOUBLE fn float64_ast_dtype(&self) -> ast::DataType { - ast::DataType::Double + ast::DataType::Double(ast::ExactNumberInfo::None) } /// The SQL type to use for Arrow Utf8 unparsing @@ -526,7 +525,7 @@ impl Default for CustomDialect { supports_nulls_first_in_sort: true, use_timestamp_for_date64: false, interval_style: IntervalStyle::SQLStandard, - float64_ast_dtype: ast::DataType::Double, + float64_ast_dtype: ast::DataType::Double(ast::ExactNumberInfo::None), utf8_cast_dtype: ast::DataType::Varchar(None), large_utf8_cast_dtype: ast::DataType::Text, date_field_extract_style: DateFieldExtractStyle::DatePart, @@ -718,7 +717,7 @@ impl CustomDialectBuilder { supports_nulls_first_in_sort: true, use_timestamp_for_date64: false, interval_style: IntervalStyle::PostgresVerbose, - float64_ast_dtype: ast::DataType::Double, + float64_ast_dtype: ast::DataType::Double(ast::ExactNumberInfo::None), utf8_cast_dtype: ast::DataType::Varchar(None), large_utf8_cast_dtype: ast::DataType::Text, date_field_extract_style: DateFieldExtractStyle::DatePart, diff --git a/datafusion/sql/src/unparser/expr.rs b/datafusion/sql/src/unparser/expr.rs index 5fad68cf4638..51ae73e03e29 100644 --- a/datafusion/sql/src/unparser/expr.rs +++ b/datafusion/sql/src/unparser/expr.rs @@ -544,9 +544,9 @@ impl Unparser<'_> { } let array = self.expr_to_sql(&args[0])?; let index = self.expr_to_sql(&args[1])?; - Ok(ast::Expr::Subscript { - expr: Box::new(array), - subscript: Box::new(Subscript::Index { index }), + Ok(ast::Expr::CompoundFieldAccess { + root: Box::new(array), + access_chain: vec![ast::AccessExpr::Subscript(Subscript::Index { index })], }) } @@ -1667,6 +1667,7 @@ mod tests { use datafusion_functions_nested::map::map; use datafusion_functions_window::rank::rank_udwf; use datafusion_functions_window::row_number::row_number_udwf; + use sqlparser::ast::ExactNumberInfo; use crate::unparser::dialect::{ CharacterLengthStyle, CustomDialect, CustomDialectBuilder, DateFieldExtractStyle, @@ -2184,7 +2185,7 @@ mod tests { #[test] fn custom_dialect_float64_ast_dtype() -> Result<()> { for (float64_ast_dtype, identifier) in [ - (ast::DataType::Double, "DOUBLE"), + (ast::DataType::Double(ExactNumberInfo::None), "DOUBLE"), (ast::DataType::DoublePrecision, "DOUBLE PRECISION"), ] { let dialect = CustomDialectBuilder::new() diff --git a/datafusion/sql/src/unparser/plan.rs b/datafusion/sql/src/unparser/plan.rs index 368131751e91..0fa203c60b7b 100644 --- a/datafusion/sql/src/unparser/plan.rs +++ b/datafusion/sql/src/unparser/plan.rs @@ -1089,7 +1089,7 @@ impl Unparser<'_> { &self, join_conditions: &[(Expr, Expr)], ) -> Option { - let mut idents = Vec::with_capacity(join_conditions.len()); + let mut object_names = Vec::with_capacity(join_conditions.len()); for (left, right) in join_conditions { match (left, right) { ( @@ -1104,14 +1104,18 @@ impl Unparser<'_> { spans: _, }), ) if left_name == right_name => { - idents.push(self.new_ident_quoted_if_needs(left_name.to_string())); + // For example, if the join condition `t1.id = t2.id` + // this is represented as two columns like `[t1.id, t2.id]` + // This code forms `id` (without relation name) + let ident = self.new_ident_quoted_if_needs(left_name.to_string()); + object_names.push(ast::ObjectName(vec![ident])); } // USING is only valid with matching column names; arbitrary expressions // are not allowed _ => return None, } } - Some(ast::JoinConstraint::Using(idents)) + Some(ast::JoinConstraint::Using(object_names)) } /// Convert a join constraint and associated conditions and filter to a SQL AST node diff --git a/datafusion/sql/tests/sql_integration.rs b/datafusion/sql/tests/sql_integration.rs index b9502c152004..6a0db3888f83 100644 --- a/datafusion/sql/tests/sql_integration.rs +++ b/datafusion/sql/tests/sql_integration.rs @@ -3673,7 +3673,7 @@ fn test_non_prepare_statement_should_infer_types() { #[test] #[should_panic( - expected = "Expected: [NOT] NULL or TRUE|FALSE or [NOT] DISTINCT FROM after IS, found: $1" + expected = "Expected: [NOT] NULL | TRUE | FALSE | DISTINCT | [form] NORMALIZED FROM after IS, found: $1" )] fn test_prepare_statement_to_plan_panic_is_param() { let sql = "PREPARE my_plan(INT) AS SELECT id, age FROM person WHERE age is $1"; diff --git a/datafusion/sqllogictest/test_files/aggregate.slt b/datafusion/sqllogictest/test_files/aggregate.slt index 002ccb08fe1d..b447b5da0e45 100644 --- a/datafusion/sqllogictest/test_files/aggregate.slt +++ b/datafusion/sqllogictest/test_files/aggregate.slt @@ -5032,18 +5032,17 @@ logical_plan 03)----Aggregate: groupBy=[[aggregate_test_100.c3]], aggr=[[min(aggregate_test_100.c1)]] 04)------TableScan: aggregate_test_100 projection=[c1, c3] physical_plan -01)GlobalLimitExec: skip=0, fetch=5 -02)--CoalescePartitionsExec -03)----AggregateExec: mode=FinalPartitioned, gby=[c3@0 as c3, min(aggregate_test_100.c1)@1 as min(aggregate_test_100.c1)], aggr=[], lim=[5] -04)------CoalesceBatchesExec: target_batch_size=8192 -05)--------RepartitionExec: partitioning=Hash([c3@0, min(aggregate_test_100.c1)@1], 4), input_partitions=4 -06)----------AggregateExec: mode=Partial, gby=[c3@0 as c3, min(aggregate_test_100.c1)@1 as min(aggregate_test_100.c1)], aggr=[], lim=[5] -07)------------AggregateExec: mode=FinalPartitioned, gby=[c3@0 as c3], aggr=[min(aggregate_test_100.c1)] -08)--------------CoalesceBatchesExec: target_batch_size=8192 -09)----------------RepartitionExec: partitioning=Hash([c3@0], 4), input_partitions=4 -10)------------------AggregateExec: mode=Partial, gby=[c3@1 as c3], aggr=[min(aggregate_test_100.c1)] -11)--------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -12)----------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c3], file_type=csv, has_header=true +01)CoalescePartitionsExec: fetch=5 +02)--AggregateExec: mode=FinalPartitioned, gby=[c3@0 as c3, min(aggregate_test_100.c1)@1 as min(aggregate_test_100.c1)], aggr=[], lim=[5] +03)----CoalesceBatchesExec: target_batch_size=8192 +04)------RepartitionExec: partitioning=Hash([c3@0, min(aggregate_test_100.c1)@1], 4), input_partitions=4 +05)--------AggregateExec: mode=Partial, gby=[c3@0 as c3, min(aggregate_test_100.c1)@1 as min(aggregate_test_100.c1)], aggr=[], lim=[5] +06)----------AggregateExec: mode=FinalPartitioned, gby=[c3@0 as c3], aggr=[min(aggregate_test_100.c1)] +07)------------CoalesceBatchesExec: target_batch_size=8192 +08)--------------RepartitionExec: partitioning=Hash([c3@0], 4), input_partitions=4 +09)----------------AggregateExec: mode=Partial, gby=[c3@1 as c3], aggr=[min(aggregate_test_100.c1)] +10)------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +11)--------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c3], file_type=csv, has_header=true # diff --git a/datafusion/sqllogictest/test_files/array.slt b/datafusion/sqllogictest/test_files/array.slt index ff701b55407c..d804bb424fb1 100644 --- a/datafusion/sqllogictest/test_files/array.slt +++ b/datafusion/sqllogictest/test_files/array.slt @@ -1941,12 +1941,12 @@ select array_slice(arrow_cast(make_array(1, 2, 3, 4, 5), 'LargeList(Int64)'), -4 query ?? select array_slice(make_array(1, 2, 3, 4, 5), -7, -2), array_slice(make_array('h', 'e', 'l', 'l', 'o'), -7, -3); ---- -[] [] +[1, 2, 3, 4] [h, e, l] query ?? select array_slice(arrow_cast(make_array(1, 2, 3, 4, 5), 'LargeList(Int64)'), -7, -2), array_slice(arrow_cast(make_array('h', 'e', 'l', 'l', 'o'), 'LargeList(Utf8)'), -7, -3); ---- -[] [] +[1, 2, 3, 4] [h, e, l] # array_slice scalar function #20 (with negative indexes; nested array) query ?? @@ -1993,6 +1993,28 @@ select array_slice(arrow_cast(make_array(1, 2, 3, 4, 5), 'LargeList(Int64)'), 2, ---- [2, 3, 4] [h, e] +# array_slice scalar function #24 (with first negative index larger than len) +query ?? +select array_slice(make_array(1, 2, 3, 4, 5), -2147483648, 1), list_slice(make_array('h', 'e', 'l', 'l', 'o'), -2147483648, 1); +---- +[1] [h] + +query ?? +select array_slice(arrow_cast(make_array(1, 2, 3, 4, 5), 'LargeList(Int64)'), -9223372036854775808, 1), list_slice(arrow_cast(make_array('h', 'e', 'l', 'l', 'o'), 'LargeList(Utf8)'), -9223372036854775808, 1); +---- +[1] [h] + +# array_slice scalar function #25 (with negative step and equal indexes) +query ?? +select array_slice(make_array(1, 2, 3, 4, 5), 2, 2, -1), list_slice(make_array('h', 'e', 'l', 'l', 'o'), 2, 2, -1); +---- +[2] [e] + +query ?? +select array_slice(arrow_cast(make_array(1, 2, 3, 4, 5), 'LargeList(Int64)'), 2, 2, -1), list_slice(arrow_cast(make_array('h', 'e', 'l', 'l', 'o'), 'LargeList(Utf8)'), 2, 2, -1); +---- +[2] [e] + # array_slice with columns query ? select array_slice(column1, column2, column3) from slices; @@ -6862,9 +6884,8 @@ select make_array(f0) from fixed_size_list_array query T select arrow_typeof(make_array(f0)) from fixed_size_list_array ---- -List(Field { name: "item", data_type: List(Field { name: "item", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }) -List(Field { name: "item", data_type: List(Field { name: "item", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }) - +List(Field { name: "item", data_type: FixedSizeList(Field { name: "item", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }, 2), nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }) +List(Field { name: "item", data_type: FixedSizeList(Field { name: "item", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }, 2), nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }) query ? select array_concat(column1, [7]) from arrays_values_v2; diff --git a/datafusion/sqllogictest/test_files/explain.slt b/datafusion/sqllogictest/test_files/explain.slt index 0f0ecf1d6eec..037565ce05f9 100644 --- a/datafusion/sqllogictest/test_files/explain.slt +++ b/datafusion/sqllogictest/test_files/explain.slt @@ -423,13 +423,17 @@ query TT explain select a from t1 where exists (select count(*) from t2); ---- logical_plan -01)Filter: EXISTS () -02)--Subquery: -03)----Projection: count(*) -04)------Aggregate: groupBy=[[]], aggr=[[count(Int64(1)) AS count(*)]] -05)--------TableScan: t2 -06)--TableScan: t1 projection=[a] -physical_plan_error This feature is not implemented: Physical plan does not support logical expression Exists(Exists { subquery: , negated: false }) +01)LeftSemi Join: +02)--TableScan: t1 projection=[a] +03)--SubqueryAlias: __correlated_sq_1 +04)----Projection: +05)------Aggregate: groupBy=[[]], aggr=[[count(Int64(1)) AS count(*)]] +06)--------TableScan: t2 projection=[] +physical_plan +01)NestedLoopJoinExec: join_type=LeftSemi +02)--DataSourceExec: partitions=1, partition_sizes=[0] +03)--ProjectionExec: expr=[] +04)----PlaceholderRowExec statement ok drop table t1; diff --git a/datafusion/sqllogictest/test_files/information_schema.slt b/datafusion/sqllogictest/test_files/information_schema.slt index 5a1caad46732..2da28bf34fee 100644 --- a/datafusion/sqllogictest/test_files/information_schema.slt +++ b/datafusion/sqllogictest/test_files/information_schema.slt @@ -223,7 +223,7 @@ datafusion.execution.parquet.writer_version 1.0 datafusion.execution.planning_concurrency 13 datafusion.execution.skip_partial_aggregation_probe_ratio_threshold 0.8 datafusion.execution.skip_partial_aggregation_probe_rows_threshold 100000 -datafusion.execution.skip_physical_aggregate_schema_check false +datafusion.execution.skip_physical_aggregate_schema_check true datafusion.execution.soft_max_rows_per_output_file 50000000 datafusion.execution.sort_in_place_threshold_bytes 1048576 datafusion.execution.sort_spill_reservation_bytes 10485760 @@ -318,7 +318,7 @@ datafusion.execution.parquet.writer_version 1.0 (writing) Sets parquet writer ve datafusion.execution.planning_concurrency 13 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.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.skip_physical_aggregate_schema_check false When set to true, skips verifying that the schema produced by planning the input of `LogicalPlan::Aggregate` exactly matches the schema of the input plan. When set to false, if the schema does not match exactly (including nullability and metadata), a planning error will be raised. This is used to workaround bugs in the planner that are now caught by the new schema verification step. +datafusion.execution.skip_physical_aggregate_schema_check true When set to true, skips verifying that the schema produced by planning the input of `LogicalPlan::Aggregate` exactly matches the schema of the input plan. When set to false, if the schema does not match exactly (including nullability and metadata), a planning error will be raised. This is used to workaround bugs in the planner that are now caught by the new schema verification step. 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.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.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). diff --git a/datafusion/sqllogictest/test_files/interval.slt b/datafusion/sqllogictest/test_files/interval.slt index db453adf12ba..1ef3048ddc66 100644 --- a/datafusion/sqllogictest/test_files/interval.slt +++ b/datafusion/sqllogictest/test_files/interval.slt @@ -25,27 +25,10 @@ select Interval(MonthDayNano) Interval(MonthDayNano) -## This is incredibly confusing but document it in tests: -# -# years is parsed as a column name -# year is parsed as part of the interval type. -# -# postgres=# select interval '5' year; -# interval -# ---------- -# 5 years -# (1 row) -# -# postgres=# select interval '5' years; -# years -# ---------- -# 00:00:05 -# (1 row) query ? select interval '5' years ---- -5.000000000 secs - +60 mons # check all different kinds of intervals query ? @@ -61,7 +44,7 @@ select interval '5' month query ? select interval '5' months ---- -5.000000000 secs +5 mons query ? select interval '5' week @@ -83,7 +66,7 @@ select interval '5' hour query ? select interval '5' hours ---- -5.000000000 secs +5 hours query ? select interval '5' minute diff --git a/datafusion/sqllogictest/test_files/joins.slt b/datafusion/sqllogictest/test_files/joins.slt index 04a52e7e5d8a..5d311bc43293 100644 --- a/datafusion/sqllogictest/test_files/joins.slt +++ b/datafusion/sqllogictest/test_files/joins.slt @@ -2623,6 +2623,10 @@ SELECT t1.c1, t2.c2 FROM test_partition_table t1 JOIN test_partition_table t2 US 0 9 0 10 +# left_join_using_qualified (snowflake syntax) +query error DataFusion error: This feature is not implemented: Invalid identifier in USING clause\. Expected single identifier, got t2\.c2 +SELECT t1.c1, t2.c2 FROM test_partition_table t1 JOIN test_partition_table t2 USING (t2.c2) ORDER BY t2.c2; + # left_join_using_join_key_projection query III SELECT t1.c1, t1.c2, t2.c2 FROM test_partition_table t1 JOIN test_partition_table t2 USING (c2) ORDER BY t2.c2 diff --git a/datafusion/sqllogictest/test_files/limit.slt b/datafusion/sqllogictest/test_files/limit.slt index baab1d9ff30b..4e74b27b875f 100644 --- a/datafusion/sqllogictest/test_files/limit.slt +++ b/datafusion/sqllogictest/test_files/limit.slt @@ -848,9 +848,8 @@ physical_plan 01)ProjectionExec: expr=[foo@0 as foo] 02)--SortExec: TopK(fetch=1000), expr=[part_key@1 ASC NULLS LAST], preserve_partitioning=[false] 03)----ProjectionExec: expr=[1 as foo, part_key@0 as part_key] -04)------GlobalLimitExec: skip=0, fetch=1 -05)--------CoalescePartitionsExec -06)----------DataSourceExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/test_limit_with_partitions/part-0.parquet:0..794], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/test_limit_with_partitions/part-1.parquet:0..794], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/test_limit_with_partitions/part-2.parquet:0..794]]}, projection=[part_key], limit=1, file_type=parquet +04)------CoalescePartitionsExec: fetch=1 +05)--------DataSourceExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/test_limit_with_partitions/part-0.parquet:0..794], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/test_limit_with_partitions/part-1.parquet:0..794], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/test_limit_with_partitions/part-2.parquet:0..794]]}, projection=[part_key], limit=1, file_type=parquet query I with selection as ( diff --git a/datafusion/sqllogictest/test_files/repartition.slt b/datafusion/sqllogictest/test_files/repartition.slt index 663dee9b1e56..70666346e2ca 100644 --- a/datafusion/sqllogictest/test_files/repartition.slt +++ b/datafusion/sqllogictest/test_files/repartition.slt @@ -121,12 +121,11 @@ logical_plan 02)--Filter: sink_table.c3 > Int16(0) 03)----TableScan: sink_table projection=[c1, c2, c3] physical_plan -01)GlobalLimitExec: skip=0, fetch=5 -02)--CoalescePartitionsExec -03)----CoalesceBatchesExec: target_batch_size=8192, fetch=5 -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 +01)CoalescePartitionsExec: fetch=5 +02)--CoalesceBatchesExec: target_batch_size=8192, fetch=5 +03)----FilterExec: c3@2 > 0 +04)------RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1 +05)--------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 diff --git a/datafusion/sqllogictest/test_files/subquery.slt b/datafusion/sqllogictest/test_files/subquery.slt index 56f2bec63616..b0c9ad93e155 100644 --- a/datafusion/sqllogictest/test_files/subquery.slt +++ b/datafusion/sqllogictest/test_files/subquery.slt @@ -389,19 +389,24 @@ where o_orderstatus in ( 2 3 +# uncorrelated exists +query I +SELECT 1 WHERE EXISTS (SELECT 1) +---- +1 + #exists_subquery_with_same_table #Subquery and outer query refer to the same table. -#It will not be rewritten to join because it is not a correlated subquery. query TT explain SELECT t1_id, t1_name, t1_int FROM t1 WHERE EXISTS(SELECT t1_int FROM t1 WHERE t1.t1_id > t1.t1_int) ---- logical_plan -01)Filter: EXISTS () -02)--Subquery: -03)----Projection: t1.t1_int -04)------Filter: t1.t1_int < t1.t1_id -05)--------TableScan: t1 -06)--TableScan: t1 projection=[t1_id, t1_name, t1_int] +01)LeftSemi Join: +02)--TableScan: t1 projection=[t1_id, t1_name, t1_int] +03)--SubqueryAlias: __correlated_sq_1 +04)----Projection: +05)------Filter: t1.t1_int < t1.t1_id +06)--------TableScan: t1 projection=[t1_id, t1_int] #in_subquery_with_same_table @@ -615,11 +620,10 @@ query TT explain SELECT t1_id, t1_name FROM t1 WHERE EXISTS (SELECT NULL) ---- logical_plan -01)Filter: EXISTS () -02)--Subquery: -03)----Projection: NULL -04)------EmptyRelation -05)--TableScan: t1 projection=[t1_id, t1_name] +01)LeftSemi Join: +02)--TableScan: t1 projection=[t1_id, t1_name] +03)--SubqueryAlias: __correlated_sq_1 +04)----EmptyRelation #exists_subquery_with_limit #de-correlated, limit is removed diff --git a/datafusion/sqllogictest/test_files/union.slt b/datafusion/sqllogictest/test_files/union.slt index 4ea3ef27d139..935e7b1648f9 100644 --- a/datafusion/sqllogictest/test_files/union.slt +++ b/datafusion/sqllogictest/test_files/union.slt @@ -510,34 +510,36 @@ logical_plan 19)------------Projection: Int64(1) AS c1 20)--------------EmptyRelation physical_plan -01)GlobalLimitExec: skip=0, fetch=3 -02)--CoalescePartitionsExec -03)----UnionExec -04)------ProjectionExec: expr=[count(*)@0 as cnt] -05)--------AggregateExec: mode=Final, gby=[], aggr=[count(*)] -06)----------CoalescePartitionsExec -07)------------AggregateExec: mode=Partial, gby=[], aggr=[count(*)] -08)--------------ProjectionExec: expr=[] -09)----------------AggregateExec: mode=FinalPartitioned, gby=[c1@0 as c1], aggr=[] -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)------------------------CoalesceBatchesExec: target_batch_size=2 -14)--------------------------FilterExec: c13@1 != C2GT5KVyOPZpgKVl110TyZO0NcJ434, projection=[c1@0] -15)----------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -16)------------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c13], file_type=csv, 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 +01)CoalescePartitionsExec: fetch=3 +02)--UnionExec +03)----ProjectionExec: expr=[count(*)@0 as cnt] +04)------AggregateExec: mode=Final, gby=[], aggr=[count(*)] +05)--------CoalescePartitionsExec +06)----------AggregateExec: mode=Partial, gby=[], aggr=[count(*)] +07)------------ProjectionExec: expr=[] +08)--------------AggregateExec: mode=FinalPartitioned, gby=[c1@0 as c1], aggr=[] +09)----------------CoalesceBatchesExec: target_batch_size=2 +10)------------------RepartitionExec: partitioning=Hash([c1@0], 4), input_partitions=4 +11)--------------------AggregateExec: mode=Partial, gby=[c1@0 as c1], aggr=[] +12)----------------------CoalesceBatchesExec: target_batch_size=2 +13)------------------------FilterExec: c13@1 != C2GT5KVyOPZpgKVl110TyZO0NcJ434, projection=[c1@0] +14)--------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +15)----------------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c13], file_type=csv, has_header=true +16)----ProjectionExec: expr=[1 as cnt] +17)------PlaceholderRowExec +18)----ProjectionExec: expr=[lead(b.c1,Int64(1)) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@1 as cnt] +19)------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] +20)--------ProjectionExec: expr=[1 as c1] +21)----------PlaceholderRowExec ######## # Clean up after the test ######## +statement ok +drop table t + statement ok drop table t1; @@ -778,76 +780,36 @@ select make_array(make_array(1)) x UNION ALL SELECT make_array(arrow_cast(make_a [[-1]] [[1]] +### +# Test for https://github.com/apache/datafusion/issues/11492 +### + +# Input data is +# a,b,c +# 1,2,3 + statement ok -CREATE EXTERNAL TABLE aggregate_test_100 ( - c1 VARCHAR NOT NULL, - c2 TINYINT NOT NULL, - c3 SMALLINT NOT NULL, - c4 SMALLINT, - c5 INT, - c6 BIGINT NOT NULL, - c7 SMALLINT NOT NULL, - c8 INT NOT NULL, - c9 BIGINT UNSIGNED NOT NULL, - c10 VARCHAR NOT NULL, - c11 FLOAT NOT NULL, - c12 DOUBLE NOT NULL, - c13 VARCHAR NOT NULL +CREATE EXTERNAL TABLE t ( + a INT, + b INT, + c INT ) STORED AS CSV -LOCATION '../../testing/data/csv/aggregate_test_100.csv' +LOCATION '../core/tests/data/example.csv' +WITH ORDER (a ASC) OPTIONS ('format.has_header' 'true'); -statement ok -set datafusion.execution.batch_size = 2; +query T +SELECT (SELECT a from t ORDER BY a) UNION ALL (SELECT 'bar' as a from t) ORDER BY a; +---- +1 +bar -# Constant value tracking across union -query TT -explain -SELECT * FROM( -( - SELECT * FROM aggregate_test_100 WHERE c1='a' -) -UNION ALL -( - SELECT * FROM aggregate_test_100 WHERE c1='a' -)) -ORDER BY c1 +query I +SELECT (SELECT a from t ORDER BY a) UNION ALL (SELECT NULL as a from t) ORDER BY a; ---- -logical_plan -01)Sort: aggregate_test_100.c1 ASC NULLS LAST -02)--Union -03)----Filter: aggregate_test_100.c1 = Utf8("a") -04)------TableScan: aggregate_test_100 projection=[c1, c2, c3, c4, c5, c6, c7, c8, c9, c10, c11, c12, c13], partial_filters=[aggregate_test_100.c1 = Utf8("a")] -05)----Filter: aggregate_test_100.c1 = Utf8("a") -06)------TableScan: aggregate_test_100 projection=[c1, c2, c3, c4, c5, c6, c7, c8, c9, c10, c11, c12, c13], partial_filters=[aggregate_test_100.c1 = Utf8("a")] -physical_plan -01)CoalescePartitionsExec -02)--UnionExec -03)----CoalesceBatchesExec: target_batch_size=2 -04)------FilterExec: c1@0 = a -05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c3, c4, c5, c6, c7, c8, c9, c10, c11, c12, c13], file_type=csv, has_header=true -07)----CoalesceBatchesExec: target_batch_size=2 -08)------FilterExec: c1@0 = a -09)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -10)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c3, c4, c5, c6, c7, c8, c9, c10, c11, c12, c13], file_type=csv, has_header=true +1 +NULL -# Clean up after the test statement ok -drop table aggregate_test_100; - -# test for https://github.com/apache/datafusion/issues/14352 -query TB rowsort -SELECT - a, - a IS NOT NULL -FROM ( - -- second column, even though it's not selected, was necessary to reproduce the bug linked above - SELECT 'foo' AS a, 3 AS b - UNION ALL - SELECT NULL AS a, 4 AS b -) ----- -NULL false -foo true +drop table t diff --git a/datafusion/sqllogictest/test_files/unnest.slt b/datafusion/sqllogictest/test_files/unnest.slt index 6840857bfc44..9c46410c4909 100644 --- a/datafusion/sqllogictest/test_files/unnest.slt +++ b/datafusion/sqllogictest/test_files/unnest.slt @@ -901,6 +901,20 @@ logical_plan 09)------------EmptyRelation physical_plan_error This feature is not implemented: Physical plan does not support logical expression OuterReferenceColumn(List(Field { name: "item", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), Column { relation: Some(Bare { table: "u" }), name: "column1" }) +# uncorrelated EXISTS with unnest +query I +SELECT 1 +WHERE EXISTS (SELECT unnest([2])) +---- +1 + +# EXISTS with unnest below correlation +query I +SELECT i +FROM (VALUES (1), (3), (7)) AS t(i) +WHERE EXISTS (SELECT 1 FROM unnest([2,3,4]) AS u(j) WHERE i + j = 6) +---- +3 ## Unnest in subquery query IIII diff --git a/datafusion/substrait/Cargo.toml b/datafusion/substrait/Cargo.toml index 226f65b983dd..f13d2b77a787 100644 --- a/datafusion/substrait/Cargo.toml +++ b/datafusion/substrait/Cargo.toml @@ -31,7 +31,6 @@ rust-version = { workspace = true } workspace = true [dependencies] -arrow-buffer = { workspace = true } async-recursion = "1.0" async-trait = { workspace = true } chrono = { workspace = true } diff --git a/datafusion/substrait/src/logical_plan/consumer.rs b/datafusion/substrait/src/logical_plan/consumer.rs index b17a8967e5bb..89112e3fe84e 100644 --- a/datafusion/substrait/src/logical_plan/consumer.rs +++ b/datafusion/substrait/src/logical_plan/consumer.rs @@ -15,7 +15,8 @@ // specific language governing permissions and limitations // under the License. -use arrow_buffer::{IntervalDayTime, IntervalMonthDayNano, OffsetBuffer}; +use arrow::array::types::{IntervalDayTime, IntervalMonthDayNano}; +use arrow::buffer::OffsetBuffer; use async_recursion::async_recursion; use datafusion::arrow::array::MapArray; use datafusion::arrow::datatypes::{ @@ -67,7 +68,7 @@ use datafusion::logical_expr::{ }; use datafusion::prelude::{lit, JoinType}; use datafusion::{ - error::Result, logical_expr::utils::split_conjunction, prelude::Column, + arrow, error::Result, logical_expr::utils::split_conjunction, prelude::Column, scalar::ScalarValue, }; use std::collections::HashSet; @@ -3278,7 +3279,8 @@ mod test { from_substrait_literal_without_names, from_substrait_rex, DefaultSubstraitConsumer, }; - use arrow_buffer::IntervalMonthDayNano; + use arrow::array::types::IntervalMonthDayNano; + use datafusion::arrow; use datafusion::common::DFSchema; use datafusion::error::Result; use datafusion::execution::SessionState; diff --git a/datafusion/substrait/src/logical_plan/producer.rs b/datafusion/substrait/src/logical_plan/producer.rs index e501ddf5c698..42c226174932 100644 --- a/datafusion/substrait/src/logical_plan/producer.rs +++ b/datafusion/substrait/src/logical_plan/producer.rs @@ -2535,7 +2535,8 @@ mod test { from_substrait_named_struct, from_substrait_type_without_names, DefaultSubstraitConsumer, }; - use arrow_buffer::{IntervalDayTime, IntervalMonthDayNano}; + use arrow::array::types::{IntervalDayTime, IntervalMonthDayNano}; + use datafusion::arrow; use datafusion::arrow::array::{ GenericListArray, Int64Builder, MapBuilder, StringBuilder, }; diff --git a/datafusion/substrait/src/physical_plan/producer.rs b/datafusion/substrait/src/physical_plan/producer.rs index fa7aeb9b1a59..3fc94a33442b 100644 --- a/datafusion/substrait/src/physical_plan/producer.rs +++ b/datafusion/substrait/src/physical_plan/producer.rs @@ -27,7 +27,7 @@ use datafusion::error::{DataFusionError, Result}; use datafusion::physical_plan::source::DataSourceExec; use datafusion::physical_plan::{displayable, ExecutionPlan}; -use datafusion::datasource::physical_plan::FileScanConfig; +use datafusion::datasource::physical_plan::{FileScanConfig, ParquetSource}; use substrait::proto::expression::mask_expression::{StructItem, StructSelect}; use substrait::proto::expression::MaskExpression; use substrait::proto::r#type::{ @@ -54,7 +54,11 @@ pub fn to_substrait_rel( if let Some(data_source) = plan.as_any().downcast_ref::() { let source = data_source.source(); if let Some(file_config) = source.as_any().downcast_ref::() { - let is_parquet = file_config.file_source().file_type().is_parquet(); + let is_parquet = file_config + .file_source() + .as_any() + .downcast_ref::() + .is_some(); if is_parquet { let mut substrait_files = vec![]; for (partition_index, files) in file_config.file_groups.iter().enumerate() diff --git a/dev/changelog/45.0.0.md b/dev/changelog/45.0.0.md index 2303eee92a1d..ca905c0a1a56 100644 --- a/dev/changelog/45.0.0.md +++ b/dev/changelog/45.0.0.md @@ -19,7 +19,7 @@ under the License. # Apache DataFusion 45.0.0 Changelog -This release consists of 252 commits from 83 contributors. See credits at the end of this changelog for more information. +This release consists of 258 commits from 83 contributors. See credits at the end of this changelog for more information. **Breaking changes:** @@ -94,6 +94,7 @@ This release consists of 252 commits from 83 contributors. See credits at the en - Support arrays_overlap function (alias of `array_has_any`) [#14217](https://github.com/apache/datafusion/pull/14217) (erenavsarogullari) - chore: Adding commit activity badge [#14386](https://github.com/apache/datafusion/pull/14386) (comphead) - docs: Clarify join behavior in `DataFrame::join` [#14393](https://github.com/apache/datafusion/pull/14393) (rkrishn7) +- Prepare for `45.0.0` release: Version and Changelog [#14397](https://github.com/apache/datafusion/pull/14397) (alamb) **Other:** @@ -290,13 +291,18 @@ This release consists of 252 commits from 83 contributors. See credits at the en - FFI support for versions and alternate tokio runtimes [#13937](https://github.com/apache/datafusion/pull/13937) (timsaucer) - Do not rename struct fields when coercing types in `CASE` [#14384](https://github.com/apache/datafusion/pull/14384) (alamb) - Add `TableProvider::insert_into` into FFI Bindings [#14391](https://github.com/apache/datafusion/pull/14391) (davisp) +- [branch-45]: Backport chore: Upgrade to `arrow`/`parquet` `54.1.0` and fix clippy/ci (#14415) [#14453](https://github.com/apache/datafusion/pull/14453) (alamb) +- [release-45] Fix join type coercion (#14387) [#14454](https://github.com/apache/datafusion/pull/14454) (alamb) +- [branch-45] Support `Utf8View` to `numeric` coercion (#14377) [#14455](https://github.com/apache/datafusion/pull/14455) (alamb) +- [branch-45] Update REGEXP_MATCH scalar function to support Utf8View (#14449) [#14457](https://github.com/apache/datafusion/pull/14457) (alamb) +- [branch-45] Fix regression list Type Coercion List with inner type struct which has large/view types (#14385) [#14456](https://github.com/apache/datafusion/pull/14456) (alamb) ## Credits Thank you to everyone who contributed to this release. Here is a breakdown of commits (PRs merged) per contributor. ``` - 46 Andrew Lamb + 52 Andrew Lamb 22 Ian Lai 20 dependabot[bot] 8 Bruce Ritchie diff --git a/docs/source/user-guide/configs.md b/docs/source/user-guide/configs.md index 999735f4c059..20029f765876 100644 --- a/docs/source/user-guide/configs.md +++ b/docs/source/user-guide/configs.md @@ -80,7 +80,7 @@ Environment variables are read during `SessionConfig` initialisation so they mus | 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.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.skip_physical_aggregate_schema_check | false | When set to true, skips verifying that the schema produced by planning the input of `LogicalPlan::Aggregate` exactly matches the schema of the input plan. When set to false, if the schema does not match exactly (including nullability and metadata), a planning error will be raised. This is used to workaround bugs in the planner that are now caught by the new schema verification step. | +| datafusion.execution.skip_physical_aggregate_schema_check | true | When set to true, skips verifying that the schema produced by planning the input of `LogicalPlan::Aggregate` exactly matches the schema of the input plan. When set to false, if the schema does not match exactly (including nullability and metadata), a planning error will be raised. This is used to workaround bugs in the planner that are now caught by the new schema verification step. | | 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 |