diff --git a/datafusion/common/src/config.rs b/datafusion/common/src/config.rs index d71af206c78d5..c057ca15e1f6e 100644 --- a/datafusion/common/src/config.rs +++ b/datafusion/common/src/config.rs @@ -743,6 +743,10 @@ config_namespace! { /// (reading) Use any available bloom filters when reading parquet files pub bloom_filter_on_read: bool, default = true + /// (reading) If true, the parquet reader will share work between partitions + /// using morsel-driven execution. This can help mitigate data skew. + pub allow_morsel_driven: bool, default = true + /// (reading) The maximum predicate cache size, in bytes. When /// `pushdown_filters` is enabled, sets the maximum memory used to cache /// the results of predicate evaluation between filter evaluation and diff --git a/datafusion/common/src/file_options/parquet_writer.rs b/datafusion/common/src/file_options/parquet_writer.rs index a7a1fc6d0bb66..726aa58a15ada 100644 --- a/datafusion/common/src/file_options/parquet_writer.rs +++ b/datafusion/common/src/file_options/parquet_writer.rs @@ -208,6 +208,7 @@ impl ParquetOptions { binary_as_string: _, // not used for writer props coerce_int96: _, // not used for writer props skip_arrow_metadata: _, + allow_morsel_driven: _, max_predicate_cache_size: _, } = self; @@ -460,6 +461,7 @@ mod tests { skip_arrow_metadata: defaults.skip_arrow_metadata, coerce_int96: None, max_predicate_cache_size: defaults.max_predicate_cache_size, + allow_morsel_driven: defaults.allow_morsel_driven, } } @@ -575,6 +577,7 @@ mod tests { schema_force_view_types: global_options_defaults.schema_force_view_types, binary_as_string: global_options_defaults.binary_as_string, skip_arrow_metadata: global_options_defaults.skip_arrow_metadata, + allow_morsel_driven: global_options_defaults.allow_morsel_driven, coerce_int96: None, }, column_specific_options, diff --git a/datafusion/core/src/datasource/physical_plan/parquet.rs b/datafusion/core/src/datasource/physical_plan/parquet.rs index 4c6d915d5bcaa..55dc8805a437e 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet.rs @@ -49,8 +49,11 @@ mod tests { use datafusion_common::config::TableParquetOptions; use datafusion_common::test_util::{batches_to_sort_string, batches_to_string}; use datafusion_common::{Result, ScalarValue, assert_contains}; + use datafusion_common_runtime::SpawnedTask; use datafusion_datasource::file_format::FileFormat; - use datafusion_datasource::file_scan_config::FileScanConfigBuilder; + use datafusion_datasource::file_scan_config::{ + FileScanConfig, FileScanConfigBuilder, + }; use datafusion_datasource::source::DataSourceExec; use datafusion_datasource::file::FileSource; @@ -76,7 +79,7 @@ mod tests { use insta::assert_snapshot; use object_store::local::LocalFileSystem; use object_store::path::Path; - use object_store::{ObjectMeta, ObjectStore}; + use object_store::{ObjectMeta, ObjectStore, ObjectStoreExt}; use parquet::arrow::ArrowWriter; use parquet::file::properties::WriterProperties; use tempfile::TempDir; @@ -2459,4 +2462,154 @@ mod tests { assert_eq!(calls.len(), 2); assert_eq!(calls, vec![Some(123), Some(456)]); } + + #[tokio::test] + async fn parquet_morsel_driven_execution() -> Result<()> { + let store = + Arc::new(object_store::memory::InMemory::new()) as Arc; + let store_url = ObjectStoreUrl::parse("memory://test").unwrap(); + + let ctx = SessionContext::new(); + ctx.register_object_store(store_url.as_ref(), store.clone()); + + // Create a Parquet file with 100 row groups, each with 10 rows + let schema = Arc::new(Schema::new(vec![Field::new("a", DataType::Int32, false)])); + + let mut out = Vec::new(); + let props = WriterProperties::builder() + .set_max_row_group_row_count(Some(10)) + .build(); + { + let mut writer = + ArrowWriter::try_new(&mut out, Arc::clone(&schema), Some(props))?; + // Write many batches to ensure they are not coalesced and we can verify work distribution + for i in 0..100 { + let batch = RecordBatch::try_new( + Arc::clone(&schema), + vec![Arc::new(Int32Array::from(vec![i; 10]))], + )?; + writer.write(&batch)?; + } + writer.close()?; + } + + let path = Path::from("skewed.parquet"); + store.put(&path, out.into()).await?; + let meta = store.head(&path).await?; + + // Set up DataSourceExec with 2 partitions, but the file is only in partition 0 (skewed) + let source = Arc::new(ParquetSource::new(schema)); + let config = FileScanConfigBuilder::new(store_url, source) + .with_file_group(FileGroup::new(vec![PartitionedFile::new_from_meta(meta)])) + .with_file_group(FileGroup::new(vec![])) // Partition 1 is empty + .with_morsel_driven(true) + .build(); + + let exec = DataSourceExec::from_data_source(config); + + // Execute both partitions concurrently + let task_ctx = ctx.task_ctx(); + let stream0 = exec.execute(0, Arc::clone(&task_ctx))?; + let stream1 = exec.execute(1, Arc::clone(&task_ctx))?; + + let handle0 = SpawnedTask::spawn(async move { + let mut count = 0; + let mut s = stream0; + while let Some(batch) = s.next().await { + count += batch.unwrap().num_rows(); + tokio::task::yield_now().await; + } + count + }); + + let handle1 = SpawnedTask::spawn(async move { + let mut count = 0; + let mut s = stream1; + while let Some(batch) = s.next().await { + count += batch.unwrap().num_rows(); + tokio::task::yield_now().await; + } + count + }); + + let count0 = handle0.await.unwrap(); + let count1 = handle1.await.unwrap(); + + // Total rows should be 1000 + assert_eq!(count0 + count1, 1000); + + // Since it's morsel-driven, both partitions should have done some work + // because the work from partition 0 (the single file) was split into + // individual row groups and shared via the shared queue. + assert!(count0 > 0, "Partition 0 should have produced rows"); + assert!(count1 > 0, "Partition 1 should have produced rows"); + + // Test re-executability: executing the same plan again should work + // (We have to create a new TaskContext, as it holds the state) + let task_ctx = ctx.task_ctx(); + let stream0 = exec.execute(0, Arc::clone(&task_ctx))?; + let stream1 = exec.execute(1, Arc::clone(&task_ctx))?; + + let mut count = 0; + let mut s0 = stream0; + while let Some(batch) = s0.next().await { + count += batch.unwrap().num_rows(); + } + let mut s1 = stream1; + while let Some(batch) = s1.next().await { + count += batch.unwrap().num_rows(); + } + assert_eq!( + count, 1000, + "Second execution should also produce 1000 rows" + ); + + Ok(()) + } + + #[tokio::test] + async fn parquet_morsel_driven_enabled_by_default() -> Result<()> { + let tmp_dir = TempDir::new()?; + let path = tmp_dir.path().join("test.parquet"); + let schema = Arc::new(Schema::new(vec![Field::new("a", DataType::Int32, false)])); + let batch = RecordBatch::try_new( + Arc::clone(&schema), + vec![Arc::new(Int32Array::from(vec![1, 2, 3]))], + )?; + + let file = File::create(&path)?; + let mut writer = ArrowWriter::try_new(file, Arc::clone(&schema), None)?; + writer.write(&batch)?; + writer.close()?; + + let ctx = SessionContext::new(); + ctx.register_parquet("t", path.to_str().unwrap(), ParquetReadOptions::default()) + .await?; + + let df = ctx.sql("SELECT * FROM t").await?; + let plan = df.create_physical_plan().await?; + + // Plan should be a ProjectionExec over a DataSourceExec + let ds_exec = if let Some(ds) = plan.as_any().downcast_ref::() { + ds + } else { + plan.children()[0] + .as_any() + .downcast_ref::() + .expect("Expected DataSourceExec") + }; + + let config = ds_exec + .data_source() + .as_any() + .downcast_ref::() + .expect("Expected FileScanConfig"); + + assert!( + config.morsel_driven, + "morsel_driven should be enabled by default for Parquet" + ); + + Ok(()) + } } diff --git a/datafusion/core/tests/fuzz_cases/topk_filter_pushdown.rs b/datafusion/core/tests/fuzz_cases/topk_filter_pushdown.rs index d14afaf1b3267..94933c59070b7 100644 --- a/datafusion/core/tests/fuzz_cases/topk_filter_pushdown.rs +++ b/datafusion/core/tests/fuzz_cases/topk_filter_pushdown.rs @@ -227,8 +227,59 @@ impl RunQueryResult { format!("{}", pretty_format_batches(&self.result).unwrap()) } + /// Extract ORDER BY column names from the query. + /// The query format is always: + /// `SELECT * FROM test_table ORDER BY , ... LIMIT ` + fn sort_columns(&self) -> Vec { + let order_by_start = self.query.find("ORDER BY").unwrap() + "ORDER BY".len(); + let limit_start = self.query.rfind(" LIMIT").unwrap(); + self.query[order_by_start..limit_start] + .trim() + .split(',') + .map(|part| part.split_whitespace().next().unwrap().to_string()) + .collect() + } + + /// Project `batches` to only include the named columns. + fn project_columns(batches: &[RecordBatch], cols: &[String]) -> Vec { + batches + .iter() + .map(|b| { + let indices: Vec = cols + .iter() + .filter_map(|c| b.schema().index_of(c).ok()) + .collect(); + b.project(&indices).unwrap() + }) + .collect() + } + fn is_ok(&self) -> bool { - self.expected_formatted() == self.result_formatted() + if self.expected_formatted() == self.result_formatted() { + return true; + } + // If the full results differ, compare only the ORDER BY column values. + // + // For queries with ORDER BY LIMIT k, multiple rows may tie on the + // sort key (e.g. two rows with id=27 for ORDER BY id DESC LIMIT 1). + // SQL permits returning any of the tied rows, so with vs without dynamic + // filter pushdown may legitimately return different tied rows. + // + // The dynamic filter must not change the *sort-key values* of the top-k + // result. We verify correctness by: + // 1. Checking the row counts match (wrong count is always a bug). + // 2. Projecting both results down to only the ORDER BY columns and + // comparing those (tied rows may differ, but the sort-key values must not). + let expected_rows: usize = self.expected.iter().map(|b| b.num_rows()).sum(); + let result_rows: usize = self.result.iter().map(|b| b.num_rows()).sum(); + if expected_rows != result_rows { + return false; + } + let sort_cols = self.sort_columns(); + let expected_keys = Self::project_columns(&self.expected, &sort_cols); + let result_keys = Self::project_columns(&self.result, &sort_cols); + format!("{}", pretty_format_batches(&expected_keys).unwrap()) + == format!("{}", pretty_format_batches(&result_keys).unwrap()) } } diff --git a/datafusion/core/tests/parquet/row_group_pruning.rs b/datafusion/core/tests/parquet/row_group_pruning.rs index 445ae7e97f228..35e2ec6cde7bc 100644 --- a/datafusion/core/tests/parquet/row_group_pruning.rs +++ b/datafusion/core/tests/parquet/row_group_pruning.rs @@ -382,7 +382,11 @@ async fn prune_disabled() { .await; println!("{}", output.description()); - // This should not prune any + // Row group stats pruning is disabled, so 0 row groups are pruned by statistics. + // Bloom filter runs next and matches all 4 row groups (bloom filters don't help + // for range/inequality predicates like `nanos < threshold`). Page index pruning + // runs afterwards and can produce row-level selections, but those don't affect + // the bloom filter matched count. The query result is still correct. assert_eq!(output.predicate_evaluation_errors(), Some(0)); assert_eq!(output.row_groups_matched(), Some(4)); assert_eq!(output.row_groups_pruned(), Some(0)); diff --git a/datafusion/datasource-parquet/src/file_format.rs b/datafusion/datasource-parquet/src/file_format.rs index 4e2affd98d551..4f2572ef91c60 100644 --- a/datafusion/datasource-parquet/src/file_format.rs +++ b/datafusion/datasource-parquet/src/file_format.rs @@ -539,6 +539,7 @@ impl FileFormat for ParquetFormat { let conf = FileScanConfigBuilder::from(conf) .with_source(Arc::new(source)) + .with_morsel_driven(self.options.global.allow_morsel_driven) .build(); Ok(DataSourceExec::from_data_source(conf)) } diff --git a/datafusion/datasource-parquet/src/opener.rs b/datafusion/datasource-parquet/src/opener.rs index 108e8c5752017..3080a6fe5d9c4 100644 --- a/datafusion/datasource-parquet/src/opener.rs +++ b/datafusion/datasource-parquet/src/opener.rs @@ -40,7 +40,7 @@ use datafusion_common::stats::Precision; use datafusion_common::{ ColumnStatistics, DataFusionError, Result, ScalarValue, Statistics, exec_err, }; -use datafusion_datasource::{PartitionedFile, TableSchema}; +use datafusion_datasource::{FileRange, PartitionedFile, TableSchema}; use datafusion_physical_expr::simplifier::PhysicalExprSimplifier; use datafusion_physical_expr_adapter::PhysicalExprAdapterFactory; use datafusion_physical_expr_common::physical_expr::{ @@ -56,6 +56,7 @@ use crate::sort::reverse_row_selection; use datafusion_common::config::EncryptionFactoryOptions; #[cfg(feature = "parquet_encryption")] use datafusion_execution::parquet_encryption::EncryptionFactory; +use futures::future::{BoxFuture, ready}; use futures::{Stream, StreamExt, TryStreamExt, ready}; use log::debug; use parquet::arrow::arrow_reader::metrics::ArrowReaderMetrics; @@ -64,7 +65,9 @@ use parquet::arrow::arrow_reader::{ }; use parquet::arrow::async_reader::AsyncFileReader; use parquet::arrow::{ParquetRecordBatchStreamBuilder, ProjectionMask}; +use parquet::file::metadata::ParquetMetaData; use parquet::file::metadata::{PageIndexPolicy, ParquetMetaDataReader, RowGroupMetaData}; +use parquet::schema::types::SchemaDescriptor; /// Implements [`FileOpener`] for a parquet file pub(super) struct ParquetOpener { @@ -122,6 +125,14 @@ pub(super) struct ParquetOpener { pub reverse_row_groups: bool, } +/// A morsel of work for Parquet execution, containing cached metadata and an access plan. +pub struct ParquetMorsel { + /// Cached Parquet metadata + pub metadata: Arc, + /// Access plan for this morsel (usually selecting a single row group) + pub access_plan: ParquetAccessPlan, +} + /// Represents a prepared access plan with optional row selection pub(crate) struct PreparedAccessPlan { /// Row group indexes to read @@ -130,6 +141,13 @@ pub(crate) struct PreparedAccessPlan { pub(crate) row_selection: Option, } +struct RowGroupStatisticsPruningContext<'a> { + physical_file_schema: &'a SchemaRef, + parquet_schema: &'a SchemaDescriptor, + predicate: &'a PruningPredicate, + file_metrics: &'a ParquetFileMetrics, +} + impl PreparedAccessPlan { /// Create a new prepared access plan from a ParquetAccessPlan pub(crate) fn from_access_plan( @@ -146,10 +164,7 @@ impl PreparedAccessPlan { } /// Reverse the access plan for reverse scanning - pub(crate) fn reverse( - mut self, - file_metadata: &parquet::file::metadata::ParquetMetaData, - ) -> Result { + pub(crate) fn reverse(mut self, file_metadata: &ParquetMetaData) -> Result { // Get the row group indexes before reversing let row_groups_to_scan = self.row_group_indexes.clone(); @@ -180,7 +195,294 @@ impl PreparedAccessPlan { } } +impl ParquetOpener { + fn build_row_group_access_filter( + file_name: &str, + extensions: Option>, + row_group_count: usize, + row_group_metadata: &[RowGroupMetaData], + file_range: Option<&FileRange>, + stats_pruning: Option>, + ) -> Result { + let mut row_groups = RowGroupAccessPlanFilter::new(create_initial_plan( + file_name, + extensions, + row_group_count, + )?); + + if let Some(range) = file_range { + row_groups.prune_by_range(row_group_metadata, range); + } + + if let Some(stats_pruning) = stats_pruning { + row_groups.prune_by_statistics( + stats_pruning.physical_file_schema.as_ref(), + stats_pruning.parquet_schema, + row_group_metadata, + stats_pruning.predicate, + stats_pruning.file_metrics, + ); + } + + Ok(row_groups) + } +} + impl FileOpener for ParquetOpener { + fn is_leaf_morsel(&self, file: &PartitionedFile) -> bool { + file.extensions + .as_ref() + .map(|e| e.is::()) + .unwrap_or(false) + } + + fn morselize( + &self, + partitioned_file: PartitionedFile, + ) -> BoxFuture<'static, Result>> { + if partitioned_file + .extensions + .as_ref() + .map(|e| e.is::()) + .unwrap_or(false) + { + return Box::pin(ready(Ok(vec![partitioned_file]))); + } + + let file_metrics = ParquetFileMetrics::new( + self.partition_index, + partitioned_file.object_meta.location.as_ref(), + &self.metrics, + ); + let file_name = partitioned_file.object_meta.location.to_string(); + let file_range = partitioned_file.range.clone(); + let extensions = partitioned_file.extensions.clone(); + + let metadata_size_hint = partitioned_file + .metadata_size_hint + .or(self.metadata_size_hint); + + let mut async_file_reader: Box = + match self.parquet_file_reader_factory.create_reader( + self.partition_index, + partitioned_file.clone(), + metadata_size_hint, + &self.metrics, + ) { + Ok(reader) => reader, + Err(e) => return Box::pin(ready(Err(e))), + }; + + let options = + ArrowReaderOptions::new().with_page_index_policy(PageIndexPolicy::Skip); + #[cfg(feature = "parquet_encryption")] + let encryption_context = self.get_encryption_context(); + + let expr_adapter_factory = Arc::clone(&self.expr_adapter_factory); + let table_schema = self.table_schema.clone(); + let predicate = self.predicate.clone(); + let metrics = self.metrics.clone(); + let enable_row_group_stats_pruning = self.enable_row_group_stats_pruning; + let enable_bloom_filter = self.enable_bloom_filter; + let enable_page_index = self.enable_page_index; + let limit = self.limit; + let preserve_order = self.preserve_order; + let parquet_file_reader_factory = Arc::clone(&self.parquet_file_reader_factory); + let partition_index = self.partition_index; + + Box::pin(async move { + #[cfg(feature = "parquet_encryption")] + let options = if let Some(fd_val) = encryption_context + .get_file_decryption_properties(&partitioned_file.object_meta.location) + .await? + { + options.with_file_decryption_properties(Arc::clone(&fd_val)) + } else { + options + }; + + let predicate_creation_errors = MetricBuilder::new(&metrics) + .global_counter("num_predicate_creation_errors"); + + // Step: try to prune the file using file-level statistics before loading + // parquet metadata. This avoids the I/O cost of reading metadata when + // file-level stats (available from the catalog) indicate no rows can match. + if let Some(pred) = predicate.as_ref() { + let logical_file_schema = Arc::clone(table_schema.file_schema()); + if let Some(mut file_pruner) = FilePruner::try_new( + Arc::clone(pred), + &logical_file_schema, + &partitioned_file, + predicate_creation_errors.clone(), + ) && file_pruner.should_prune()? + { + file_metrics.files_ranges_pruned_statistics.add_pruned(1); + return Ok(vec![]); + } + } + + let _metadata_timer = file_metrics.metadata_load_time.timer(); + let mut reader_metadata = + ArrowReaderMetadata::load_async(&mut async_file_reader, options.clone()) + .await?; + let num_row_groups = reader_metadata.metadata().num_row_groups(); + + // Adapt the physical schema to the file schema for pruning + let physical_file_schema = Arc::clone(reader_metadata.schema()); + let logical_file_schema = table_schema.file_schema(); + let rewriter = expr_adapter_factory.create( + Arc::clone(logical_file_schema), + Arc::clone(&physical_file_schema), + )?; + let simplifier = PhysicalExprSimplifier::new(&physical_file_schema); + + // Replace partition column references with their literal values before rewriting. + // This mirrors what `open()` does. Without this, expressions like `val != part` + // (where `part` is a partition column) would cause `rewriter.rewrite` to fail + // since the partition column is not in the logical file schema. + let literal_columns: HashMap = table_schema + .table_partition_cols() + .iter() + .zip(partitioned_file.partition_values.iter()) + .map(|(field, value)| (field.name().clone(), value.clone())) + .collect(); + + let adapted_predicate = predicate + .as_ref() + .map(|p| { + let p = if !literal_columns.is_empty() { + replace_columns_with_literals(Arc::clone(p), &literal_columns)? + } else { + Arc::clone(p) + }; + simplifier.simplify(rewriter.rewrite(p)?) + }) + .transpose()?; + + let (pruning_predicate, page_pruning_predicate) = build_pruning_predicates( + adapted_predicate.as_ref(), + &physical_file_schema, + &predicate_creation_errors, + ); + + let mut row_groups = Self::build_row_group_access_filter( + &file_name, + extensions, + num_row_groups, + reader_metadata.metadata().row_groups(), + file_range.as_ref(), + pruning_predicate + .as_deref() + .filter(|_| enable_row_group_stats_pruning) + .map(|predicate| RowGroupStatisticsPruningContext { + physical_file_schema: &physical_file_schema, + parquet_schema: reader_metadata.parquet_schema(), + predicate, + file_metrics: &file_metrics, + }), + )?; + + // Prune by limit if limit is set and order is not sensitive + if let (Some(limit), false) = (limit, preserve_order) { + row_groups.prune_by_limit( + limit, + reader_metadata.metadata().row_groups(), + &file_metrics, + ); + } + + // Bloom filter pruning: done once per file here in morselize(), so that + // open() does not repeat it for each morsel (which would cause inflated metrics + // and unnecessary work). + // + // Note: the bloom filter builder takes ownership of `async_file_reader`. + // Page index loading happens afterward using a fresh reader so that we only + // pay for the page index I/O on the row groups that survive bloom filter pruning. + if let Some(predicate) = pruning_predicate.as_deref() { + if enable_bloom_filter && !row_groups.is_empty() { + // Clone reader_metadata so it remains available for page + // index loading after this builder is dropped. + let mut builder = ParquetRecordBatchStreamBuilder::new_with_metadata( + async_file_reader, + reader_metadata.clone(), + ); + row_groups + .prune_by_bloom_filters( + &physical_file_schema, + &mut builder, + predicate, + &file_metrics, + ) + .await; + } else { + file_metrics + .row_groups_pruned_bloom_filter + .add_matched(row_groups.remaining_row_group_count()); + } + } + + // Load page index after bloom filter pruning so we skip it entirely if no + // row groups remain. Bloom filter building consumed `async_file_reader`, so + // we create a fresh reader here — reader creation is cheap (no I/O yet). + if should_enable_page_index(enable_page_index, &page_pruning_predicate) + && !row_groups.is_empty() + { + let mut fresh_reader: Box = + parquet_file_reader_factory.create_reader( + partition_index, + partitioned_file.clone(), + metadata_size_hint, + &metrics, + )?; + reader_metadata = load_page_index( + reader_metadata, + &mut fresh_reader, + options.with_page_index_policy(PageIndexPolicy::Optional), + ) + .await?; + } + + // Extract metadata after potentially loading the page index, so the cached + // metadata in each morsel includes the page index if it was loaded. + let metadata = Arc::clone(reader_metadata.metadata()); + + let mut access_plan = row_groups.build(); + + // Page pruning: done once per file here in morselize(), so that open() + // does not repeat it for each morsel. + if enable_page_index + && !access_plan.is_empty() + && let Some(p) = page_pruning_predicate + { + access_plan = p.prune_plan_with_page_index( + access_plan, + &physical_file_schema, + metadata.file_metadata().schema_descr(), + metadata.as_ref(), + &file_metrics, + ); + } + + let mut morsels = Vec::with_capacity(access_plan.len()); + for i in 0..num_row_groups { + if !access_plan.should_scan(i) { + continue; + } + let mut morsel_access_plan = ParquetAccessPlan::new_none(num_row_groups); + // Transfer the page-pruned access (Scan or Selection) for this row group + morsel_access_plan.set(i, access_plan.inner()[i].clone()); + let morsel = ParquetMorsel { + metadata: Arc::clone(&metadata), + access_plan: morsel_access_plan, + }; + let mut f = partitioned_file.clone(); + f.extensions = Some(Arc::new(morsel)); + morsels.push(f); + } + Ok(morsels) + }) + } + fn open(&self, partitioned_file: PartitionedFile) -> Result { // ----------------------------------- // Step: prepare configurations, etc. @@ -280,6 +582,11 @@ impl FileOpener for ParquetOpener { let reverse_row_groups = self.reverse_row_groups; let preserve_order = self.preserve_order; + let is_morsel = partitioned_file + .extensions + .as_ref() + .map(|e| e.is::()) + .unwrap_or(false); Ok(Box::pin(async move { #[cfg(feature = "parquet_encryption")] @@ -359,10 +666,21 @@ impl FileOpener for ParquetOpener { // Begin by loading the metadata from the underlying reader (note // the returned metadata may actually include page indexes as some // readers may return page indexes even when not requested -- for - // example when they are cached) - let mut reader_metadata = + // example when they are cached). + // If this is a morsel, we might already have the metadata cached. + let mut reader_metadata = if let Some(morsel) = partitioned_file + .extensions + .as_ref() + .and_then(|e| e.downcast_ref::()) + { + ArrowReaderMetadata::try_new( + Arc::clone(&morsel.metadata), + options.clone(), + )? + } else { ArrowReaderMetadata::load_async(&mut async_file_reader, options.clone()) - .await?; + .await? + }; // Note about schemas: we are actually dealing with **3 different schemas** here: // - The table schema as defined by the TableProvider. @@ -429,10 +747,22 @@ impl FileOpener for ParquetOpener { &predicate_creation_errors, ); + // Track whether the predicate contains dynamic filters. Dynamic + // filters (e.g. from hash joins or TopK) can tighten during + // execution, so morsels that passed row-group pruning during + // morselize() may now be prunable with the updated filter values. + let has_dynamic_predicate = predicate + .as_ref() + .is_some_and(|p| is_dynamic_physical_expr(p)); + // The page index is not stored inline in the parquet footer so the // code above may not have read the page index structures yet. If we // need them for reading and they aren't yet loaded, we need to load them now. - if should_enable_page_index(enable_page_index, &page_pruning_predicate) { + // For morsels, the page index was already loaded (if needed) in morselize(). + // Skip it here to avoid redundant I/O. + if should_enable_page_index(enable_page_index, &page_pruning_predicate) + && !is_morsel + { reader_metadata = load_page_index( reader_metadata, &mut async_file_reader, @@ -496,26 +826,28 @@ impl FileOpener for ParquetOpener { let file_metadata = Arc::clone(builder.metadata()); let predicate = pruning_predicate.as_ref().map(|p| p.as_ref()); let rg_metadata = file_metadata.row_groups(); - // track which row groups to actually read - let access_plan = - create_initial_plan(&file_name, extensions, rg_metadata.len())?; - let mut row_groups = RowGroupAccessPlanFilter::new(access_plan); - // if there is a range restricting what parts of the file to read - if let Some(range) = file_range.as_ref() { - row_groups.prune_by_range(rg_metadata, range); - } + let mut row_groups = Self::build_row_group_access_filter( + &file_name, + extensions, + rg_metadata.len(), + rg_metadata, + file_range.as_ref(), + predicate + .filter(|_| { + enable_row_group_stats_pruning + && (!is_morsel || has_dynamic_predicate) + }) + .map(|predicate| RowGroupStatisticsPruningContext { + physical_file_schema: &physical_file_schema, + parquet_schema: builder.parquet_schema(), + predicate, + file_metrics: &file_metrics, + }), + )?; // If there is a predicate that can be evaluated against the metadata if let Some(predicate) = predicate.as_ref() { - if enable_row_group_stats_pruning { - row_groups.prune_by_statistics( - &physical_file_schema, - builder.parquet_schema(), - rg_metadata, - predicate, - &file_metrics, - ); - } else { + if !enable_row_group_stats_pruning { // Update metrics: statistics unavailable, so all row groups are // matched (not pruned) file_metrics @@ -523,21 +855,31 @@ impl FileOpener for ParquetOpener { .add_matched(row_groups.remaining_row_group_count()); } - if enable_bloom_filter && !row_groups.is_empty() { - row_groups - .prune_by_bloom_filters( - &physical_file_schema, - &mut builder, - predicate, - &file_metrics, - ) - .await; - } else { - // Update metrics: bloom filter unavailable, so all row groups are - // matched (not pruned) - file_metrics - .row_groups_pruned_bloom_filter - .add_matched(row_groups.remaining_row_group_count()); + // Prune by limit before bloom filter: no point reading bloom filter data + // for row groups that will be skipped by the limit anyway. + if let (Some(limit), false) = (limit, preserve_order) { + row_groups.prune_by_limit(limit, rg_metadata, &file_metrics); + } + + // For morsels, bloom filter was already applied once in morselize(). + // Skip it here to avoid double-counting metrics and redundant I/O. + if !is_morsel { + if enable_bloom_filter && !row_groups.is_empty() { + row_groups + .prune_by_bloom_filters( + &physical_file_schema, + &mut builder, + predicate, + &file_metrics, + ) + .await; + } else { + // Update metrics: bloom filter unavailable, so all row groups are + // matched (not pruned) + file_metrics + .row_groups_pruned_bloom_filter + .add_matched(row_groups.remaining_row_group_count()); + } } } else { // Update metrics: no predicate, so all row groups are matched (not pruned) @@ -550,9 +892,12 @@ impl FileOpener for ParquetOpener { .add_matched(n_remaining_row_groups); } - // Prune by limit if limit is set and limit order is not sensitive - if let (Some(limit), false) = (limit, preserve_order) { - row_groups.prune_by_limit(limit, rg_metadata, &file_metrics); + // If a morsel was fully pruned by re-evaluated row group + // statistics (dynamic filters tightened since morselize()), + // return an empty stream immediately. + if is_morsel && row_groups.is_empty() { + file_metrics.row_groups_pruned_statistics.add_pruned(1); + return Ok(futures::stream::empty().boxed()); } // -------------------------------------------------------- @@ -563,8 +908,11 @@ impl FileOpener for ParquetOpener { // be ruled using page metadata, rows from other columns // with that range can be skipped as well // -------------------------------------------------------- + // For morsels, page pruning was already applied once in morselize(). + // Skip it here to avoid double-counting metrics and redundant work. if enable_page_index && !access_plan.is_empty() + && !is_morsel && let Some(p) = page_pruning_predicate { access_plan = p.prune_plan_with_page_index( @@ -928,6 +1276,14 @@ fn create_initial_plan( // check row group count matches the plan return Ok(access_plan.clone()); + } else if let Some(morsel) = extensions.downcast_ref::() { + let plan_len = morsel.access_plan.len(); + if plan_len != row_group_count { + return exec_err!( + "Invalid ParquetMorsel AccessPlan for {file_name}. Specified {plan_len} row groups, but file has {row_group_count}" + ); + } + return Ok(morsel.access_plan.clone()); } else { debug!("DataSourceExec Ignoring unknown extension specified for {file_name}"); } @@ -1018,7 +1374,10 @@ mod test { use std::sync::Arc; use super::{ConstantColumns, constant_columns_from_stats}; - use crate::{DefaultParquetFileReaderFactory, RowGroupAccess, opener::ParquetOpener}; + use crate::{ + DefaultParquetFileReaderFactory, RowGroupAccess, + opener::{ParquetMorsel, ParquetOpener}, + }; use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; use bytes::{BufMut, BytesMut}; use datafusion_common::{ @@ -1036,7 +1395,7 @@ mod test { use datafusion_physical_expr_adapter::{ DefaultPhysicalExprAdapterFactory, replace_columns_with_literals, }; - use datafusion_physical_plan::metrics::ExecutionPlanMetricsSet; + use datafusion_physical_plan::metrics::{ExecutionPlanMetricsSet, MetricValue}; use futures::{Stream, StreamExt}; use object_store::{ObjectStore, ObjectStoreExt, memory::InMemory, path::Path}; use parquet::arrow::ArrowWriter; @@ -1366,6 +1725,19 @@ mod test { )) } + fn get_pruning_metric( + metrics: &ExecutionPlanMetricsSet, + metric_name: &str, + ) -> (usize, usize) { + match metrics.clone_inner().sum_by_name(metric_name) { + Some(MetricValue::PruningMetrics { + pruning_metrics, .. + }) => (pruning_metrics.pruned(), pruning_metrics.matched()), + Some(_) => panic!("Metric '{metric_name}' is not a pruning metric"), + None => panic!("Metric '{metric_name}' not found"), + } + } + #[tokio::test] async fn test_prune_on_statistics() { let store = Arc::new(InMemory::new()) as Arc; @@ -2005,4 +2377,115 @@ mod test { "Reverse scan with non-contiguous row groups should correctly map RowSelection" ); } + + #[tokio::test] + async fn test_open_and_morselize_are_equivalent_except_for_morsels() { + use parquet::file::properties::WriterProperties; + + let store = Arc::new(InMemory::new()) as Arc; + + let batch1 = record_batch!(("a", Int32, vec![Some(1), Some(2)])).unwrap(); + let batch2 = record_batch!(("a", Int32, vec![Some(10), Some(11)])).unwrap(); + let batch3 = record_batch!(("a", Int32, vec![Some(20), Some(21)])).unwrap(); + + let props = WriterProperties::builder() + .set_max_row_group_row_count(Some(2)) + .build(); + + let data_len = write_parquet_batches( + Arc::clone(&store), + "test.parquet", + vec![batch1.clone(), batch2.clone(), batch3.clone()], + Some(props), + ) + .await; + + let schema = batch1.schema(); + let file = PartitionedFile::new( + "test.parquet".to_string(), + u64::try_from(data_len).unwrap(), + ); + + for enable_row_group_stats_pruning in [false, true] { + let expr = col("a").gt(lit(5)).and(col("a").lt(lit(20))); + let predicate = logical2physical(&expr, &schema); + + let baseline_opener = ParquetOpenerBuilder::new() + .with_store(Arc::clone(&store)) + .with_schema(Arc::clone(&schema)) + .with_projection_indices(&[0]) + .with_predicate(Arc::clone(&predicate)) + .with_row_group_stats_pruning(enable_row_group_stats_pruning) + .build(); + + // Baseline: regular open path + let stream = baseline_opener.open(file.clone()).unwrap().await.unwrap(); + let baseline_values = collect_int32_values(stream).await; + let baseline_stats_metrics = get_pruning_metric( + &baseline_opener.metrics, + "row_groups_pruned_statistics", + ); + let baseline_bloom_metrics = get_pruning_metric( + &baseline_opener.metrics, + "row_groups_pruned_bloom_filter", + ); + + let morsel_opener = ParquetOpenerBuilder::new() + .with_store(Arc::clone(&store)) + .with_schema(Arc::clone(&schema)) + .with_projection_indices(&[0]) + .with_predicate(predicate) + .with_row_group_stats_pruning(enable_row_group_stats_pruning) + .build(); + + // Morsel path: split into morsels and open each morsel + let morsels = morsel_opener.morselize(file.clone()).await.unwrap(); + assert!( + !morsels.is_empty(), + "Expected at least one morsel for the selected row groups" + ); + + let mut morsel_values = vec![]; + for morsel_file in morsels { + let morsel = morsel_file + .extensions + .as_ref() + .and_then(|ext| ext.downcast_ref::()) + .expect("morselized file should carry ParquetMorsel extension"); + + assert_eq!( + morsel.access_plan.row_group_indexes().len(), + 1, + "each morsel should scan exactly one row group" + ); + + let stream = morsel_opener.open(morsel_file).unwrap().await.unwrap(); + morsel_values.extend(collect_int32_values(stream).await); + } + + let morsel_stats_metrics = get_pruning_metric( + &morsel_opener.metrics, + "row_groups_pruned_statistics", + ); + let morsel_bloom_metrics = get_pruning_metric( + &morsel_opener.metrics, + "row_groups_pruned_bloom_filter", + ); + + assert_eq!( + baseline_values, morsel_values, + "open and morselize paths should scan equivalent data; morselize only changes work granularity" + ); + + assert_eq!( + baseline_stats_metrics, morsel_stats_metrics, + "row_groups_pruned_statistics should be equivalent for open vs morselize path (enable_row_group_stats_pruning={enable_row_group_stats_pruning})" + ); + + assert_eq!( + baseline_bloom_metrics, morsel_bloom_metrics, + "row_groups_pruned_bloom_filter should be equivalent for open vs morselize path (enable_row_group_stats_pruning={enable_row_group_stats_pruning})" + ); + } + } } diff --git a/datafusion/datasource/src/file_scan_config.rs b/datafusion/datasource/src/file_scan_config.rs index 524e091381c4c..12337517c04b6 100644 --- a/datafusion/datasource/src/file_scan_config.rs +++ b/datafusion/datasource/src/file_scan_config.rs @@ -20,9 +20,13 @@ use crate::file_groups::FileGroup; use crate::{ - PartitionedFile, display::FileGroupsDisplay, file::FileSource, - file_compression_type::FileCompressionType, file_stream::FileStream, - source::DataSource, statistics::MinMaxStatistics, + PartitionedFile, + display::FileGroupsDisplay, + file::FileSource, + file_compression_type::FileCompressionType, + file_stream::{FileStream, WorkQueue}, + source::DataSource, + statistics::MinMaxStatistics, }; use arrow::datatypes::FieldRef; use arrow::datatypes::{DataType, Schema, SchemaRef}; @@ -208,6 +212,9 @@ pub struct FileScanConfig { /// If the number of file partitions > target_partitions, the file partitions will be grouped /// in a round-robin fashion such that number of file partitions = target_partitions. pub partitioned_by_file_group: bool, + /// When true, use morsel-driven execution to avoid data skew. + /// This means all partitions share a single pool of work. + pub morsel_driven: bool, } /// A builder for [`FileScanConfig`]'s. @@ -278,6 +285,7 @@ pub struct FileScanConfigBuilder { batch_size: Option, expr_adapter_factory: Option>, partitioned_by_file_group: bool, + morsel_driven: bool, } impl FileScanConfigBuilder { @@ -304,6 +312,7 @@ impl FileScanConfigBuilder { batch_size: None, expr_adapter_factory: None, partitioned_by_file_group: false, + morsel_driven: false, } } @@ -504,6 +513,12 @@ impl FileScanConfigBuilder { self } + /// Set whether to use morsel-driven execution. + pub fn with_morsel_driven(mut self, morsel_driven: bool) -> Self { + self.morsel_driven = morsel_driven; + self + } + /// Build the final [`FileScanConfig`] with all the configured settings. /// /// This method takes ownership of the builder and returns the constructed `FileScanConfig`. @@ -525,6 +540,7 @@ impl FileScanConfigBuilder { batch_size, expr_adapter_factory: expr_adapter, partitioned_by_file_group, + morsel_driven, } = self; let constraints = constraints.unwrap_or_default(); @@ -537,6 +553,24 @@ impl FileScanConfigBuilder { // If there is an output ordering, we should preserve it. let preserve_order = preserve_order || !output_ordering.is_empty(); + // Morsel-driven execution pools all files from all file groups into a shared + // work queue that any partition may consume, allowing partitions to steal work + // from each other's file groups. This breaks two guarantees that downstream + // operators may rely on: + // + // 1. `partitioned_by_file_group`: the optimizer has declared Hash partitioning + // assuming partition N reads only from file_group[N] (e.g. Hive-style + // partitioning with `preserve_file_partitions`). Morsel-driven stealing + // would violate this, breaking `HashJoinExec: mode=Partitioned` correctness. + // + // 2. `preserve_order`: the scan declares a sort order on its output. When a + // partition interleaves morsels from multiple files (from different groups), + // the per-partition output is no longer globally sorted. Downstream operators + // such as `SortPreservingMergeExec` rely on each partition's stream being + // pre-sorted. + let morsel_driven = + morsel_driven && !partitioned_by_file_group && !preserve_order; + FileScanConfig { object_store_url, file_source, @@ -550,6 +584,7 @@ impl FileScanConfigBuilder { expr_adapter_factory: expr_adapter, statistics, partitioned_by_file_group, + morsel_driven, } } } @@ -569,15 +604,18 @@ impl From for FileScanConfigBuilder { batch_size: config.batch_size, expr_adapter_factory: config.expr_adapter_factory, partitioned_by_file_group: config.partitioned_by_file_group, + morsel_driven: config.morsel_driven, } } } -impl DataSource for FileScanConfig { - fn open( +impl FileScanConfig { + /// Open a partition stream with an optional shared morsel queue. + pub(crate) fn open_with_queue( &self, partition: usize, - context: Arc, + context: &Arc, + queue: Option>, ) -> Result { let object_store = context.runtime_env().object_store(&self.object_store_url)?; let batch_size = self @@ -585,12 +623,25 @@ impl DataSource for FileScanConfig { .unwrap_or_else(|| context.session_config().batch_size()); let source = self.file_source.with_batch_size(batch_size); - let opener = source.create_file_opener(object_store, self, partition)?; let stream = FileStream::new(self, partition, opener, source.metrics())?; + let stream = match queue { + Some(q) => stream.with_shared_queue(q), + None => stream, + }; Ok(Box::pin(cooperative(stream))) } +} + +impl DataSource for FileScanConfig { + fn open( + &self, + partition: usize, + context: Arc, + ) -> Result { + self.open_with_queue(partition, &context, None) + } fn as_any(&self) -> &dyn Any { self @@ -602,8 +653,21 @@ impl DataSource for FileScanConfig { let schema = self.projected_schema().map_err(|_| std::fmt::Error {})?; let orderings = get_projected_output_ordering(self, &schema); - write!(f, "file_groups=")?; - FileGroupsDisplay(&self.file_groups).fmt_as(t, f)?; + if self.morsel_driven { + let files: Vec<_> = + self.file_groups.iter().flat_map(|g| g.iter()).collect(); + write!(f, "files=[")?; + for (i, pf) in files.iter().enumerate() { + if i > 0 { + write!(f, ", ")?; + } + write!(f, "{}", pf.object_meta.location.as_ref())?; + } + write!(f, "]")?; + } else { + write!(f, "file_groups=")?; + FileGroupsDisplay(&self.file_groups).fmt_as(t, f)?; + } if !schema.fields().is_empty() { if let Some(projection) = self.file_source.projection() { @@ -671,6 +735,29 @@ impl DataSource for FileScanConfig { return Ok(None); } + // With morsel-driven execution, the shared WorkQueue handles load + // balancing at runtime so byte-range splitting is unnecessary. + // Just distribute whole files round-robin across target partitions. + if self.morsel_driven { + let all_files: Vec<_> = self + .file_groups + .iter() + .flat_map(|g| g.files().iter().cloned()) + .collect(); + if all_files.is_empty() { + return Ok(None); + } + let mut groups: Vec> = + (0..target_partitions).map(|_| vec![]).collect(); + for (i, file) in all_files.into_iter().enumerate() { + groups[i % target_partitions].push(file); + } + let file_groups = groups.into_iter().map(FileGroup::new).collect(); + let mut source = self.clone(); + source.file_groups = file_groups; + return Ok(Some(Arc::new(source))); + } + let source = self.file_source.repartitioned( target_partitions, repartition_file_min_size, @@ -1310,8 +1397,20 @@ impl DisplayAs for FileScanConfig { let schema = self.projected_schema().map_err(|_| std::fmt::Error {})?; let orderings = get_projected_output_ordering(self, &schema); - write!(f, "file_groups=")?; - FileGroupsDisplay(&self.file_groups).fmt_as(t, f)?; + if self.morsel_driven { + let files: Vec<_> = self.file_groups.iter().flat_map(|g| g.iter()).collect(); + write!(f, "files=[")?; + for (i, pf) in files.iter().enumerate() { + if i > 0 { + write!(f, ", ")?; + } + write!(f, "{}", pf.object_meta.location.as_ref())?; + } + write!(f, "]")?; + } else { + write!(f, "file_groups=")?; + FileGroupsDisplay(&self.file_groups).fmt_as(t, f)?; + } if !schema.fields().is_empty() { write!(f, ", projection={}", ProjectSchemaDisplay(&schema))?; diff --git a/datafusion/datasource/src/file_stream.rs b/datafusion/datasource/src/file_stream.rs index c8090382094ef..6ac8a238fe7b0 100644 --- a/datafusion/datasource/src/file_stream.rs +++ b/datafusion/datasource/src/file_stream.rs @@ -24,8 +24,10 @@ use std::collections::VecDeque; use std::mem; use std::pin::Pin; -use std::sync::Arc; +use std::sync::atomic::{AtomicUsize, Ordering}; +use std::sync::{Arc, Mutex}; use std::task::{Context, Poll}; +use tokio::sync::Notify; use crate::PartitionedFile; use crate::file_scan_config::FileScanConfig; @@ -43,10 +45,23 @@ use futures::future::BoxFuture; use futures::stream::BoxStream; use futures::{FutureExt as _, Stream, StreamExt as _, ready}; +/// A guard that decrements the morselizing count when dropped. +struct MorselizingGuard { + queue: Arc, +} + +impl Drop for MorselizingGuard { + fn drop(&mut self) { + self.queue.stop_morselizing(); + } +} + /// A stream that iterates record batch by record batch, file over file. pub struct FileStream { /// An iterator over input files. file_iter: VecDeque, + /// Shared work queue for morsel-driven execution. + shared_queue: Option>, /// The stream schema (file schema including partition columns and after /// projection). projected_schema: SchemaRef, @@ -63,6 +78,8 @@ pub struct FileStream { baseline_metrics: BaselineMetrics, /// Describes the behavior of the `FileStream` if file opening or scanning fails on_error: OnError, + /// Guard for morselizing state to ensure counter is decremented on drop + morsel_guard: Option, } impl FileStream { @@ -75,10 +92,16 @@ impl FileStream { ) -> Result { let projected_schema = config.projected_schema()?; - let file_group = config.file_groups[partition].clone(); + let file_iter = if config.morsel_driven { + VecDeque::new() + } else { + let file_group = config.file_groups[partition].clone(); + file_group.into_inner().into_iter().collect() + }; Ok(Self { - file_iter: file_group.into_inner().into_iter().collect(), + file_iter, + shared_queue: None, projected_schema, remain: config.limit, file_opener, @@ -86,9 +109,16 @@ impl FileStream { file_stream_metrics: FileStreamMetrics::new(metrics, partition), baseline_metrics: BaselineMetrics::new(metrics, partition), on_error: OnError::Fail, + morsel_guard: None, }) } + /// Set the shared work queue for morsel-driven execution. + pub fn with_shared_queue(mut self, queue: Arc) -> Self { + self.shared_queue = Some(queue); + self + } + /// Specify the behavior when an error occurs opening or scanning a file /// /// If `OnError::Skip` the stream will skip files which encounter an error and continue @@ -102,7 +132,16 @@ impl FileStream { /// /// Since file opening is mostly IO (and may involve a /// bunch of sequential IO), it can be parallelized with decoding. + /// + /// In morsel-driven mode this prefetches the next already-morselized item + /// from the shared queue (leaf morsels only — items that still need + /// async morselization are left in the queue for the normal Idle → + /// Morselizing path). fn start_next_file(&mut self) -> Option> { + if self.shared_queue.is_some() { + // In morsel-driven don't "prefetch" + return None; + } let part_file = self.file_iter.pop_front()?; Some(self.file_opener.open(part_file)) } @@ -113,15 +152,109 @@ impl FileStream { FileStreamState::Idle => { self.file_stream_metrics.time_opening.start(); - match self.start_next_file().transpose() { - Ok(Some(future)) => self.state = FileStreamState::Open { future }, - Ok(None) => return Poll::Ready(None), + if let Some(queue) = self.shared_queue.as_ref() { + match queue.pull() { + WorkStatus::Work(part_file) => { + if self.file_opener.is_leaf_morsel(&part_file) { + // Leaf morsel from the morsel queue — open directly. + match self.file_opener.open(part_file) { + Ok(future) => { + self.state = FileStreamState::Open { future } + } + Err(e) => { + self.file_stream_metrics.time_opening.stop(); + self.state = FileStreamState::Error; + return Poll::Ready(Some(Err(e))); + } + } + } else { + // Whole file from the file queue — morselize it. + self.morsel_guard = Some(MorselizingGuard { + queue: Arc::clone(queue), + }); + self.state = FileStreamState::Morselizing { + future: self.file_opener.morselize(part_file), + }; + } + } + WorkStatus::Wait => { + self.file_stream_metrics.time_opening.stop(); + let queue_captured = Arc::clone(queue); + self.state = FileStreamState::Waiting { + future: Box::pin(async move { + let notified = queue_captured.notify.notified(); + if !queue_captured.has_work_or_done() { + notified.await; + } + }), + }; + } + WorkStatus::Done => { + self.file_stream_metrics.time_opening.stop(); + return Poll::Ready(None); + } + } + } else { + match self.start_next_file().transpose() { + Ok(Some(future)) => { + self.state = FileStreamState::Open { future } + } + Ok(None) => return Poll::Ready(None), + Err(e) => { + self.state = FileStreamState::Error; + return Poll::Ready(Some(Err(e))); + } + } + } + } + FileStreamState::Morselizing { future } => { + match ready!(future.poll_unpin(cx)) { + Ok(morsels) => { + let queue = self.shared_queue.as_ref().expect("shared queue"); + // Take the guard to decrement morselizing_count + let _guard = self.morsel_guard.take(); + self.file_stream_metrics.time_opening.stop(); + + if morsels.is_empty() { + // No morsels returned (file pruned entirely), skip. + self.state = FileStreamState::Idle; + } else { + // Push morsels to the morsel queue. Workers drain + // that queue before pulling new files, preserving + // I/O locality within the file. + queue.push_morsels(morsels); + + // Pipeline: immediately start morselizing the next + // file (if any) while workers consume the morsels + // we just pushed. This overlaps the next file's + // footer read + pruning with the current file's row + // group data reads. + if let Some(next_file) = + queue.pop_next_file_for_prefetch() + { + self.morsel_guard = Some(MorselizingGuard { + queue: Arc::clone(queue), + }); + self.state = FileStreamState::Morselizing { + future: self.file_opener.morselize(next_file), + }; + } else { + self.state = FileStreamState::Idle; + } + } + } Err(e) => { + let _guard = self.morsel_guard.take(); + self.file_stream_metrics.time_opening.stop(); self.state = FileStreamState::Error; return Poll::Ready(Some(Err(e))); } } } + FileStreamState::Waiting { future } => { + ready!(future.poll_unpin(cx)); + self.state = FileStreamState::Idle; + } FileStreamState::Open { future } => match ready!(future.poll_unpin(cx)) { Ok(reader) => { // include time needed to start opening in `start_next_file` @@ -214,7 +347,13 @@ impl FileStream { } } } - None => return Poll::Ready(None), + None => { + if self.shared_queue.is_some() { + self.state = FileStreamState::Idle; + } else { + return Poll::Ready(None); + } + } }, OnError::Fail => { self.state = FileStreamState::Error; @@ -243,7 +382,13 @@ impl FileStream { } } } - None => return Poll::Ready(None), + None => { + if self.shared_queue.is_some() { + self.state = FileStreamState::Idle; + } else { + return Poll::Ready(None); + } + } } } } @@ -276,6 +421,126 @@ impl RecordBatchStream for FileStream { } } +/// Result of pulling work from the queue +#[derive(Debug)] +pub enum WorkStatus { + /// A morsel is available + Work(PartitionedFile), + /// No morsel available now, but others are morselizing + Wait, + /// No more work available + Done, +} + +/// A shared queue of [`PartitionedFile`] morsels for morsel-driven execution. +/// +/// Internally keeps two queues: one for whole files that still need +/// morselizing and one for already-morselized leaf morsels (e.g. row +/// groups). Workers drain the morsel queue first, which keeps I/O +/// sequential within a file because freshly produced morsels are +/// consumed before the next file is opened. +#[derive(Debug)] +pub struct WorkQueue { + /// Whole files waiting to be morselized. + files: Mutex>, + /// Already-morselized leaf morsels ready to be opened directly. + morsels: Mutex>, + /// Number of workers currently morselizing a file. + morselizing_count: AtomicUsize, + /// Notify waiters when work is added or morselizing finishes. + notify: Notify, +} + +impl WorkQueue { + /// Create a new `WorkQueue` with the given initial files. + pub fn new(initial_files: Vec) -> Self { + Self { + files: Mutex::new(VecDeque::from(initial_files)), + morsels: Mutex::new(VecDeque::new()), + morselizing_count: AtomicUsize::new(0), + notify: Notify::new(), + } + } + + /// Pull a work item from the queue. + /// + /// Prefers already-morselized morsels (for I/O locality) over whole + /// files that still need morselizing. + pub fn pull(&self) -> WorkStatus { + // First try the morsel queue — these are ready to open immediately + // and preserve locality with the file that was just morselized. + if let Some(morsel) = self.morsels.lock().unwrap().pop_front() { + return WorkStatus::Work(morsel); + } + // Fall back to whole files that need morselizing. + let mut files = self.files.lock().unwrap(); + if let Some(file) = files.pop_front() { + // Relaxed: the increment is done by the same task that will later call + // stop_morselizing(), so program order ensures the decrement sees it. + self.morselizing_count.fetch_add(1, Ordering::Relaxed); + WorkStatus::Work(file) + } else if self.morselizing_count.load(Ordering::Acquire) > 0 { + // Acquire: stop_morselizing() uses AcqRel (a Release write) without + // holding the files mutex, so we need Acquire here to synchronize with + // it on weakly-ordered architectures (e.g. ARM). + WorkStatus::Wait + } else { + // Check the morsel queue one more time — a morselizer may have + // pushed work between our first check and reaching this point. + if self.morsels.lock().unwrap().is_empty() { + WorkStatus::Done + } else { + WorkStatus::Wait + } + } + } + + /// Returns true if there is work in either queue or if all morselizing is done. + pub fn has_work_or_done(&self) -> bool { + !self.morsels.lock().unwrap().is_empty() + || !self.files.lock().unwrap().is_empty() + || self.morselizing_count.load(Ordering::Acquire) == 0 + } + + /// Push morselized leaf morsels to the morsel queue. + pub fn push_morsels(&self, morsels: Vec) { + if morsels.is_empty() { + return; + } + self.morsels.lock().unwrap().extend(morsels); + self.notify.notify_waiters(); + } + + /// Decrement the morselizing count. Notifies waiting workers only when the + /// count reaches zero, since that is the point at which they may need to + /// re-evaluate whether all work is done. When count is still > 0, any new + /// morsels pushed to the queue already triggered a notification via + /// `push_morsels`, so no additional wakeup is needed here. + pub fn stop_morselizing(&self) { + let prev = self.morselizing_count.fetch_sub(1, Ordering::AcqRel); + if prev == 1 { + self.notify.notify_waiters(); + } + } + + /// Pop the next whole file for prefetch morselization. + /// + /// Should only be called when the morsels queue is known to be non-empty + /// (e.g. immediately after [`Self::push_morsels`]), so that workers have + /// morsels to consume while the prefetch runs. Increments + /// `morselizing_count` so that waiting workers do not declare the queue + /// done prematurely. + pub fn pop_next_file_for_prefetch(&self) -> Option { + let mut files = self.files.lock().unwrap(); + if let Some(file) = files.pop_front() { + self.morselizing_count.fetch_add(1, Ordering::Relaxed); + Some(file) + } else { + None + } + } +} + /// A fallible future that resolves to a stream of [`RecordBatch`] pub type FileOpenFuture = BoxFuture<'static, Result>>>; @@ -298,6 +563,30 @@ pub trait FileOpener: Unpin + Send + Sync { /// Asynchronously open the specified file and return a stream /// of [`RecordBatch`] fn open(&self, partitioned_file: PartitionedFile) -> Result; + + /// Optional: Split a file into smaller morsels for morsel-driven execution. + /// + /// By default, returns the file as a single morsel. + fn morselize( + &self, + file: PartitionedFile, + ) -> BoxFuture<'static, Result>> { + Box::pin(futures::future::ready(Ok(vec![file]))) + } + + /// Returns `true` if `file` is already a leaf morsel that can be opened + /// directly without going through [`Self::morselize`]. + /// + /// Returning `true` allows the [`FileStream`] to skip the async + /// `Morselizing` state and go straight to `Open`, and to prefetch the next + /// morsel while scanning the current one. + /// + /// The default implementation returns `false` (conservative — always + /// morselize). + fn is_leaf_morsel(&self, _file: &PartitionedFile) -> bool { + false + } + } /// Represents the state of the next `FileOpenFuture`. Since we need to poll @@ -317,6 +606,16 @@ pub enum FileStreamState { /// A [`FileOpenFuture`] returned by [`FileOpener::open`] future: FileOpenFuture, }, + /// Currently splitting a file into smaller morsels. + Morselizing { + /// A future that resolves to a list of morsels + future: BoxFuture<'static, Result>>, + }, + /// Waiting for more work to be added to the queue. + Waiting { + /// A future that resolves when more work is available + future: BoxFuture<'static, ()>, + }, /// Scanning the [`BoxStream`] returned by the completion of a [`FileOpenFuture`] /// returned by [`FileOpener::open`] Scan { diff --git a/datafusion/datasource/src/source.rs b/datafusion/datasource/src/source.rs index bdb45e8e3b591..ac2a83b4715e6 100644 --- a/datafusion/datasource/src/source.rs +++ b/datafusion/datasource/src/source.rs @@ -20,7 +20,9 @@ use std::any::Any; use std::fmt; use std::fmt::{Debug, Formatter}; +use std::pin::Pin; use std::sync::Arc; +use std::task::{Context, Poll}; use datafusion_physical_expr::projection::ProjectionExprs; use datafusion_physical_plan::execution_plan::{ @@ -36,16 +38,20 @@ use datafusion_physical_plan::{ use itertools::Itertools; use crate::file_scan_config::FileScanConfig; +use crate::file_stream::WorkQueue; +use arrow::datatypes::SchemaRef; +use arrow::record_batch::RecordBatch; use datafusion_common::config::ConfigOptions; use datafusion_common::tree_node::TreeNodeRecursion; use datafusion_common::{Constraints, Result, Statistics}; -use datafusion_execution::{SendableRecordBatchStream, TaskContext}; +use datafusion_execution::{RecordBatchStream, SendableRecordBatchStream, TaskContext}; use datafusion_physical_expr::{EquivalenceProperties, Partitioning, PhysicalExpr}; use datafusion_physical_expr_common::sort_expr::{LexOrdering, PhysicalSortExpr}; use datafusion_physical_plan::SortOrderPushdownResult; use datafusion_physical_plan::filter_pushdown::{ ChildPushdownResult, FilterPushdownPhase, FilterPushdownPropagation, PushedDown, }; +use futures::Stream; /// A source of data, typically a list of files or memory /// @@ -126,6 +132,7 @@ pub trait DataSource: Send + Sync + Debug { partition: usize, context: Arc, ) -> Result; + fn as_any(&self) -> &dyn Any; /// Format this source for display in explain plans fn fmt_as(&self, t: DisplayFormatType, f: &mut Formatter) -> fmt::Result; @@ -325,18 +332,42 @@ impl ExecutionPlan for DataSourceExec { partition: usize, context: Arc, ) -> Result { - let stream = self.data_source.open(partition, Arc::clone(&context))?; + let morsel_config = self + .data_source + .as_any() + .downcast_ref::() + .filter(|c| c.morsel_driven); + + let (stream, queue) = if let Some(config) = morsel_config { + let key = Arc::as_ptr(&self.data_source) as *const () as usize; + let queue = context.get_or_insert_shared_state(key, || { + let all_files = config + .file_groups + .iter() + .flat_map(|g| g.files().iter().cloned()) + .collect(); + WorkQueue::new(all_files) + }); + let stream = + config.open_with_queue(partition, &context, Some(Arc::clone(&queue)))?; + (stream, Some(queue)) + } else { + ( + self.data_source.open(partition, Arc::clone(&context))?, + None, + ) + }; + let batch_size = context.session_config().batch_size(); log::debug!( "Batch splitting enabled for partition {partition}: batch_size={batch_size}" ); let metrics = self.data_source.metrics(); let split_metrics = SplitMetrics::new(&metrics, partition); - Ok(Box::pin(BatchSplitStream::new( - stream, - batch_size, - split_metrics, - ))) + Ok(Box::pin(DataSourceExecStream { + inner: Box::pin(BatchSplitStream::new(stream, batch_size, split_metrics)), + _shared_queue: queue, + })) } fn metrics(&self) -> Option { @@ -434,6 +465,30 @@ impl ExecutionPlan for DataSourceExec { } } +struct DataSourceExecStream { + inner: SendableRecordBatchStream, + /// Holds a strong reference to the morsel queue so it stays alive + /// as long as any partition stream exists. + _shared_queue: Option>, +} + +impl Stream for DataSourceExecStream { + type Item = Result; + + fn poll_next( + mut self: Pin<&mut Self>, + cx: &mut Context<'_>, + ) -> Poll> { + self.inner.as_mut().poll_next(cx) + } +} + +impl RecordBatchStream for DataSourceExecStream { + fn schema(&self) -> SchemaRef { + self.inner.schema() + } +} + impl DataSourceExec { pub fn from_data_source(data_source: impl DataSource + 'static) -> Arc { Arc::new(Self::new(Arc::new(data_source))) diff --git a/datafusion/execution/src/task.rs b/datafusion/execution/src/task.rs index 38f31cf4629eb..77d0f19a1d4c3 100644 --- a/datafusion/execution/src/task.rs +++ b/datafusion/execution/src/task.rs @@ -22,9 +22,22 @@ use crate::{ use datafusion_common::{Result, internal_datafusion_err, plan_datafusion_err}; use datafusion_expr::planner::ExprPlanner; use datafusion_expr::{AggregateUDF, ScalarUDF, WindowUDF}; +use std::any::Any; use std::collections::HashSet; +use std::fmt; +use std::sync::Mutex; use std::{collections::HashMap, sync::Arc}; +/// Type-erased shared state map used by execution plan nodes to share +/// state across partitions within the same query execution. +struct SharedState(Mutex>>); + +impl fmt::Debug for SharedState { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + f.write_str("SharedState") + } +} + /// Task Execution Context /// /// A [`TaskContext`] contains the state required during a single query's @@ -48,6 +61,10 @@ pub struct TaskContext { window_functions: HashMap>, /// Runtime environment associated with this task context runtime: Arc, + /// Shared state for execution plan nodes within this query execution. + /// Keyed by a caller-chosen identifier (e.g. pointer address of a plan + /// node's `Arc`). + shared_state: SharedState, } impl Default for TaskContext { @@ -63,6 +80,7 @@ impl Default for TaskContext { aggregate_functions: HashMap::new(), window_functions: HashMap::new(), runtime, + shared_state: SharedState(Mutex::new(HashMap::new())), } } } @@ -90,6 +108,7 @@ impl TaskContext { aggregate_functions, window_functions, runtime, + shared_state: SharedState(Mutex::new(HashMap::new())), } } @@ -136,6 +155,27 @@ impl TaskContext { self } + /// Get or create shared state for a given key. + /// + /// Execution plan nodes use this to share state (e.g. work queues) + /// across partitions within the same query execution. The key is + /// typically derived from a stable pointer (e.g. `Arc::as_ptr`). + pub fn get_or_insert_shared_state( + &self, + key: usize, + create: impl FnOnce() -> T, + ) -> Arc { + let mut map = self.shared_state.0.lock().unwrap(); + if let Some(existing) = map.get(&key) + && let Ok(typed) = Arc::clone(existing).downcast::() + { + return typed; + } + let value = Arc::new(create()); + map.insert(key, Arc::clone(&value) as Arc); + value + } + /// Update the [`RuntimeEnv`] pub fn with_runtime(mut self, runtime: Arc) -> Self { self.runtime = runtime; diff --git a/datafusion/physical-expr/src/simplifier/mod.rs b/datafusion/physical-expr/src/simplifier/mod.rs index 3f3f8573449eb..38663f1b06609 100644 --- a/datafusion/physical-expr/src/simplifier/mod.rs +++ b/datafusion/physical-expr/src/simplifier/mod.rs @@ -61,7 +61,10 @@ impl<'a> PhysicalExprSimplifier<'a> { count += 1; let result = current_expr.transform(|node| { #[cfg(debug_assertions)] - let original_type = node.data_type(schema).unwrap(); + // Use `ok()` to skip the assertion when data_type fails (e.g., for + // DynamicFilterPhysicalExpr whose inner expression may reference columns + // outside the provided schema when the filter has been updated concurrently). + let original_type = node.data_type(schema).ok(); // Apply NOT expression simplification first, then unwrap cast optimization, // then constant expression evaluation @@ -73,11 +76,14 @@ impl<'a> PhysicalExprSimplifier<'a> { })?; #[cfg(debug_assertions)] - assert_eq!( - rewritten.data.data_type(schema).unwrap(), - original_type, - "Simplified expression should have the same data type as the original" - ); + if let Some(original_type) = original_type + && let Ok(rewritten_type) = rewritten.data.data_type(schema) { + assert_eq!( + rewritten_type, + original_type, + "Simplified expression should have the same data type as the original" + ); + } Ok(rewritten) })?; diff --git a/datafusion/physical-expr/src/utils/guarantee.rs b/datafusion/physical-expr/src/utils/guarantee.rs index c4ce74fd3a573..70c83cee65b74 100644 --- a/datafusion/physical-expr/src/utils/guarantee.rs +++ b/datafusion/physical-expr/src/utils/guarantee.rs @@ -389,6 +389,9 @@ impl<'a> ColOpLit<'a> { /// 2. `literal col` /// 3. operator is `=` or `!=` /// + /// Also handles `CastColumnExpr(col) literal` patterns where the + /// column is wrapped in a cast (e.g., from schema adaptation). + /// /// Returns None otherwise fn try_new(expr: &'a Arc) -> Option { let binary_expr = expr @@ -405,9 +408,9 @@ impl<'a> ColOpLit<'a> { Operator::NotEq => Guarantee::NotIn, _ => return None, }; - // col literal + // col literal (also handles CastColumnExpr(col) literal) if let (Some(col), Some(lit)) = ( - left.downcast_ref::(), + extract_column(binary_expr.left()), right.downcast_ref::(), ) { Some(Self { @@ -416,10 +419,10 @@ impl<'a> ColOpLit<'a> { lit, }) } - // literal col + // literal col (also handles literal CastColumnExpr(col)) else if let (Some(lit), Some(col)) = ( left.downcast_ref::(), - right.downcast_ref::(), + extract_column(binary_expr.right()), ) { Some(Self { col, @@ -432,6 +435,25 @@ impl<'a> ColOpLit<'a> { } } +/// Extracts a [`Column`](crate::expressions::Column) reference from a physical +/// expression, looking through [`CastColumnExpr`](crate::expressions::CastColumnExpr) +/// wrappers. +fn extract_column(expr: &Arc) -> Option<&crate::expressions::Column> { + if let Some(col) = expr.as_any().downcast_ref::() { + return Some(col); + } + if let Some(cast) = expr + .as_any() + .downcast_ref::() + { + return cast + .expr() + .as_any() + .downcast_ref::(); + } + None +} + /// Represents a single `col [not]in literal` expression struct ColInList<'a> { col: &'a crate::expressions::Column, diff --git a/datafusion/proto-common/proto/datafusion_common.proto b/datafusion/proto-common/proto/datafusion_common.proto index 62c6bbe85612a..da7491c6115c8 100644 --- a/datafusion/proto-common/proto/datafusion_common.proto +++ b/datafusion/proto-common/proto/datafusion_common.proto @@ -549,6 +549,7 @@ message ParquetOptions { bool schema_force_view_types = 28; // default = false bool binary_as_string = 29; // default = false bool skip_arrow_metadata = 30; // default = false + bool allow_morsel_driven = 35; // default = true oneof metadata_size_hint_opt { uint64 metadata_size_hint = 4; diff --git a/datafusion/proto-common/src/from_proto/mod.rs b/datafusion/proto-common/src/from_proto/mod.rs index ca8a269958d73..387aeda657e1f 100644 --- a/datafusion/proto-common/src/from_proto/mod.rs +++ b/datafusion/proto-common/src/from_proto/mod.rs @@ -1090,6 +1090,7 @@ impl TryFrom<&protobuf::ParquetOptions> for ParquetOptions { max_predicate_cache_size: value.max_predicate_cache_size_opt.map(|opt| match opt { protobuf::parquet_options::MaxPredicateCacheSizeOpt::MaxPredicateCacheSize(v) => Some(v as usize), }).unwrap_or(None), + allow_morsel_driven: value.allow_morsel_driven, }) } } diff --git a/datafusion/proto-common/src/generated/pbjson.rs b/datafusion/proto-common/src/generated/pbjson.rs index b00e7546bba20..e5c6a509a5f4c 100644 --- a/datafusion/proto-common/src/generated/pbjson.rs +++ b/datafusion/proto-common/src/generated/pbjson.rs @@ -5683,6 +5683,9 @@ impl serde::Serialize for ParquetOptions { if self.skip_arrow_metadata { len += 1; } + if self.allow_morsel_driven { + len += 1; + } if self.dictionary_page_size_limit != 0 { len += 1; } @@ -5788,6 +5791,9 @@ impl serde::Serialize for ParquetOptions { if self.skip_arrow_metadata { struct_ser.serialize_field("skipArrowMetadata", &self.skip_arrow_metadata)?; } + if self.allow_morsel_driven { + struct_ser.serialize_field("allowMorselDriven", &self.allow_morsel_driven)?; + } if self.dictionary_page_size_limit != 0 { #[allow(clippy::needless_borrow)] #[allow(clippy::needless_borrows_for_generic_args)] @@ -5936,6 +5942,8 @@ impl<'de> serde::Deserialize<'de> for ParquetOptions { "binaryAsString", "skip_arrow_metadata", "skipArrowMetadata", + "allow_morsel_driven", + "allowMorselDriven", "dictionary_page_size_limit", "dictionaryPageSizeLimit", "data_page_row_count_limit", @@ -5985,6 +5993,7 @@ impl<'de> serde::Deserialize<'de> for ParquetOptions { SchemaForceViewTypes, BinaryAsString, SkipArrowMetadata, + AllowMorselDriven, DictionaryPageSizeLimit, DataPageRowCountLimit, MaxRowGroupSize, @@ -6038,6 +6047,7 @@ impl<'de> serde::Deserialize<'de> for ParquetOptions { "schemaForceViewTypes" | "schema_force_view_types" => Ok(GeneratedField::SchemaForceViewTypes), "binaryAsString" | "binary_as_string" => Ok(GeneratedField::BinaryAsString), "skipArrowMetadata" | "skip_arrow_metadata" => Ok(GeneratedField::SkipArrowMetadata), + "allowMorselDriven" | "allow_morsel_driven" => Ok(GeneratedField::AllowMorselDriven), "dictionaryPageSizeLimit" | "dictionary_page_size_limit" => Ok(GeneratedField::DictionaryPageSizeLimit), "dataPageRowCountLimit" | "data_page_row_count_limit" => Ok(GeneratedField::DataPageRowCountLimit), "maxRowGroupSize" | "max_row_group_size" => Ok(GeneratedField::MaxRowGroupSize), @@ -6089,6 +6099,7 @@ impl<'de> serde::Deserialize<'de> for ParquetOptions { let mut schema_force_view_types__ = None; let mut binary_as_string__ = None; let mut skip_arrow_metadata__ = None; + let mut allow_morsel_driven__ = None; let mut dictionary_page_size_limit__ = None; let mut data_page_row_count_limit__ = None; let mut max_row_group_size__ = None; @@ -6216,6 +6227,12 @@ impl<'de> serde::Deserialize<'de> for ParquetOptions { } skip_arrow_metadata__ = Some(map_.next_value()?); } + GeneratedField::AllowMorselDriven => { + if allow_morsel_driven__.is_some() { + return Err(serde::de::Error::duplicate_field("allowMorselDriven")); + } + allow_morsel_driven__ = Some(map_.next_value()?); + } GeneratedField::DictionaryPageSizeLimit => { if dictionary_page_size_limit__.is_some() { return Err(serde::de::Error::duplicate_field("dictionaryPageSizeLimit")); @@ -6332,6 +6349,7 @@ impl<'de> serde::Deserialize<'de> for ParquetOptions { schema_force_view_types: schema_force_view_types__.unwrap_or_default(), binary_as_string: binary_as_string__.unwrap_or_default(), skip_arrow_metadata: skip_arrow_metadata__.unwrap_or_default(), + allow_morsel_driven: allow_morsel_driven__.unwrap_or_default(), dictionary_page_size_limit: dictionary_page_size_limit__.unwrap_or_default(), data_page_row_count_limit: data_page_row_count_limit__.unwrap_or_default(), max_row_group_size: max_row_group_size__.unwrap_or_default(), diff --git a/datafusion/proto-common/src/generated/prost.rs b/datafusion/proto-common/src/generated/prost.rs index a09826a29be52..739bd28188fa7 100644 --- a/datafusion/proto-common/src/generated/prost.rs +++ b/datafusion/proto-common/src/generated/prost.rs @@ -830,6 +830,9 @@ pub struct ParquetOptions { /// default = false #[prost(bool, tag = "30")] pub skip_arrow_metadata: bool, + /// default = true + #[prost(bool, tag = "35")] + pub allow_morsel_driven: bool, #[prost(uint64, tag = "12")] pub dictionary_page_size_limit: u64, #[prost(uint64, tag = "18")] diff --git a/datafusion/proto-common/src/to_proto/mod.rs b/datafusion/proto-common/src/to_proto/mod.rs index 79e3306a4df1b..7afe1d57c8940 100644 --- a/datafusion/proto-common/src/to_proto/mod.rs +++ b/datafusion/proto-common/src/to_proto/mod.rs @@ -904,6 +904,7 @@ impl TryFrom<&ParquetOptions> for protobuf::ParquetOptions { skip_arrow_metadata: value.skip_arrow_metadata, coerce_int96_opt: value.coerce_int96.clone().map(protobuf::parquet_options::CoerceInt96Opt::CoerceInt96), max_predicate_cache_size_opt: value.max_predicate_cache_size.map(|v| protobuf::parquet_options::MaxPredicateCacheSizeOpt::MaxPredicateCacheSize(v as u64)), + allow_morsel_driven: value.allow_morsel_driven, }) } } diff --git a/datafusion/proto/src/generated/datafusion_proto_common.rs b/datafusion/proto/src/generated/datafusion_proto_common.rs index a09826a29be52..739bd28188fa7 100644 --- a/datafusion/proto/src/generated/datafusion_proto_common.rs +++ b/datafusion/proto/src/generated/datafusion_proto_common.rs @@ -830,6 +830,9 @@ pub struct ParquetOptions { /// default = false #[prost(bool, tag = "30")] pub skip_arrow_metadata: bool, + /// default = true + #[prost(bool, tag = "35")] + pub allow_morsel_driven: bool, #[prost(uint64, tag = "12")] pub dictionary_page_size_limit: u64, #[prost(uint64, tag = "18")] diff --git a/datafusion/proto/src/logical_plan/file_formats.rs b/datafusion/proto/src/logical_plan/file_formats.rs index 08f42b0af7290..573dc533cdf57 100644 --- a/datafusion/proto/src/logical_plan/file_formats.rs +++ b/datafusion/proto/src/logical_plan/file_formats.rs @@ -426,6 +426,7 @@ mod parquet { max_predicate_cache_size_opt: global_options.global.max_predicate_cache_size.map(|size| { parquet_options::MaxPredicateCacheSizeOpt::MaxPredicateCacheSize(size as u64) }), + allow_morsel_driven: global_options.global.allow_morsel_driven, }), column_specific_options: column_specific_options.into_iter().map(|(column_name, options)| { ParquetColumnSpecificOptions { @@ -525,6 +526,7 @@ mod parquet { max_predicate_cache_size: proto.max_predicate_cache_size_opt.as_ref().map(|opt| match opt { parquet_options::MaxPredicateCacheSizeOpt::MaxPredicateCacheSize(size) => *size as usize, }), + allow_morsel_driven: proto.allow_morsel_driven, } } } diff --git a/datafusion/sqllogictest/test_files/aggregate_repartition.slt b/datafusion/sqllogictest/test_files/aggregate_repartition.slt index eeece7862341b..1eda5e95b4863 100644 --- a/datafusion/sqllogictest/test_files/aggregate_repartition.slt +++ b/datafusion/sqllogictest/test_files/aggregate_repartition.slt @@ -96,7 +96,7 @@ physical_plan 02)--AggregateExec: mode=FinalPartitioned, gby=[env@0 as env], aggr=[count(Int64(1))] 03)----RepartitionExec: partitioning=Hash([env@0], 4), input_partitions=1 04)------AggregateExec: mode=Partial, gby=[env@0 as env], aggr=[count(Int64(1))] -05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/aggregate_repartition/dim.parquet]]}, projection=[env], file_type=parquet +05)--------DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/aggregate_repartition/dim.parquet], projection=[env], file_type=parquet # Verify the queries actually work and return the same results query TI rowsort @@ -127,4 +127,4 @@ logical_plan physical_plan 01)ProjectionExec: expr=[env@0 as env, count(Int64(1))@1 as count(*)] 02)--AggregateExec: mode=Single, gby=[env@0 as env], aggr=[count(Int64(1))] -03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/aggregate_repartition/dim.parquet]]}, projection=[env], file_type=parquet +03)----DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/aggregate_repartition/dim.parquet], projection=[env], file_type=parquet diff --git a/datafusion/sqllogictest/test_files/date_bin_errors.slt b/datafusion/sqllogictest/test_files/date_bin_errors.slt index b6cda471d7afa..d95c27b96a060 100644 --- a/datafusion/sqllogictest/test_files/date_bin_errors.slt +++ b/datafusion/sqllogictest/test_files/date_bin_errors.slt @@ -23,7 +23,7 @@ select date_bin(interval '1637426858 months', to_timestamp_millis(1040292460), t ---- NULL -# Negative timestamp with month interval - should return NULL instead of panicking +# Negative timestamp with month interval - should return NULL instead of panicking query P select date_bin(interval '1 month', to_timestamp_millis(-1040292460), timestamp '1984-01-07 00:00:00'); ---- @@ -57,4 +57,4 @@ select date_bin( timestamp '1984-01-07 00:00:00' ) as b; ---- -NULL \ No newline at end of file +NULL diff --git a/datafusion/sqllogictest/test_files/dynamic_filter_pushdown_config.slt b/datafusion/sqllogictest/test_files/dynamic_filter_pushdown_config.slt index d5202a1d9570d..1d8d3618dd5eb 100644 --- a/datafusion/sqllogictest/test_files/dynamic_filter_pushdown_config.slt +++ b/datafusion/sqllogictest/test_files/dynamic_filter_pushdown_config.slt @@ -90,7 +90,7 @@ logical_plan 02)--TableScan: test_parquet projection=[id, value, name] physical_plan 01)SortExec: TopK(fetch=3), expr=[value@1 DESC], preserve_partitioning=[false] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/test_data.parquet]]}, projection=[id, value, name], file_type=parquet, predicate=DynamicFilter [ empty ] +02)--DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/test_data.parquet], projection=[id, value, name], file_type=parquet, predicate=DynamicFilter [ empty ] statement ok set datafusion.explain.analyze_level = summary; @@ -99,12 +99,11 @@ query TT EXPLAIN ANALYZE SELECT id, value AS v, value + id as name FROM test_parquet where value > 3 ORDER BY v DESC LIMIT 3; ---- Plan with Metrics -01)SortPreservingMergeExec: [v@1 DESC], fetch=3, metrics=[output_rows=3, ] -02)--SortExec: TopK(fetch=3), expr=[v@1 DESC], preserve_partitioning=[true], filter=[v@1 IS NULL OR v@1 > 800], metrics=[output_rows=3, ] -03)----ProjectionExec: expr=[id@0 as id, value@1 as v, value@1 + id@0 as name], metrics=[output_rows=10, ] -04)------FilterExec: value@1 > 3, metrics=[output_rows=10, , selectivity=100% (10/10)] -05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1, metrics=[output_rows=10, ] -06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/test_data.parquet]]}, projection=[id, value], file_type=parquet, predicate=value@1 > 3 AND DynamicFilter [ value@1 IS NULL OR value@1 > 800 ], pruning_predicate=value_null_count@1 != row_count@2 AND value_max@0 > 3 AND (value_null_count@1 > 0 OR value_null_count@1 != row_count@2 AND value_max@0 > 800), required_guarantees=[], metrics=[output_rows=10, elapsed_compute=1ns, output_bytes=80.0 B, files_ranges_pruned_statistics=1 total → 1 matched, row_groups_pruned_statistics=1 total → 1 matched -> 1 fully matched, row_groups_pruned_bloom_filter=1 total → 1 matched, page_index_pages_pruned=1 total → 1 matched, limit_pruned_row_groups=0 total → 0 matched, bytes_scanned=210, metadata_load_time=, scan_efficiency_ratio=18% (210/1.15 K)] +01)SortPreservingMergeExec: [v@1 DESC], fetch=3, metrics=[output_rows=3, elapsed_compute=, output_bytes=36.0 B] +02)--SortExec: TopK(fetch=3), expr=[v@1 DESC], preserve_partitioning=[true], filter=[v@1 IS NULL OR v@1 > 800], metrics=[output_rows=3, elapsed_compute=, output_bytes=36.0 B] +03)----ProjectionExec: expr=[id@0 as id, value@1 as v, value@1 + id@0 as name], metrics=[output_rows=10, elapsed_compute=, output_bytes=64.0 KB] +04)------FilterExec: value@1 > 3, metrics=[output_rows=10, elapsed_compute=, output_bytes=64.0 KB, selectivity=100% (10/10)] +05)--------DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/test_data.parquet], projection=[id, value], file_type=parquet, predicate=value@1 > 3 AND DynamicFilter [ value@1 IS NULL OR value@1 > 800 ], pruning_predicate=value_null_count@1 != row_count@2 AND value_max@0 > 3 AND (value_null_count@1 > 0 OR value_null_count@1 != row_count@2 AND value_max@0 > 800), required_guarantees=[], metrics=[output_rows=10, elapsed_compute=, output_bytes=80.0 B, files_ranges_pruned_statistics=1 total → 1 matched, row_groups_pruned_statistics=2 total → 2 matched -> fully matched, row_groups_pruned_bloom_filter=1 total → 1 matched, page_index_pages_pruned=1 total → 1 matched, limit_pruned_row_groups=0 total → 0 matched, bytes_scanned=210, metadata_load_time=, scan_efficiency_ratio=18% (210/1.15 K)] statement ok set datafusion.explain.analyze_level = dev; @@ -128,7 +127,7 @@ logical_plan 02)--TableScan: test_parquet projection=[id, value, name] physical_plan 01)SortExec: TopK(fetch=3), expr=[value@1 DESC], preserve_partitioning=[false] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/test_data.parquet]]}, projection=[id, value, name], file_type=parquet +02)--DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/test_data.parquet], projection=[id, value, name], file_type=parquet query IIT SELECT id, value AS v, name FROM (SELECT * FROM test_parquet UNION ALL SELECT * FROM test_parquet) ORDER BY v DESC LIMIT 3; @@ -157,8 +156,8 @@ logical_plan physical_plan 01)ProjectionExec: expr=[id@1 as id, data@2 as data, info@0 as info] 02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(id@0, id@0)], projection=[info@1, id@2, data@3] -03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_right.parquet]]}, projection=[id, info], file_type=parquet -04)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_left.parquet]]}, projection=[id, data], file_type=parquet, predicate=DynamicFilter [ empty ] +03)----DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_right.parquet], projection=[id, info], file_type=parquet +04)----DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_left.parquet], projection=[id, data], file_type=parquet, predicate=DynamicFilter [ empty ] # Disable Join dynamic filter pushdown statement ok @@ -180,8 +179,8 @@ logical_plan physical_plan 01)ProjectionExec: expr=[id@1 as id, data@2 as data, info@0 as info] 02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(id@0, id@0)], projection=[info@1, id@2, data@3] -03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_right.parquet]]}, projection=[id, info], file_type=parquet -04)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_left.parquet]]}, projection=[id, data], file_type=parquet +03)----DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_right.parquet], projection=[id, info], file_type=parquet +04)----DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_left.parquet], projection=[id, data], file_type=parquet # Re-enable for next tests statement ok @@ -205,8 +204,8 @@ logical_plan physical_plan 01)ProjectionExec: expr=[id@1 as id, data@2 as data, info@0 as info] 02)--HashJoinExec: mode=CollectLeft, join_type=Right, on=[(id@0, id@0)], projection=[info@1, id@2, data@3] -03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_right.parquet]]}, projection=[id, info], file_type=parquet -04)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_left.parquet]]}, projection=[id, data], file_type=parquet +03)----DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_right.parquet], projection=[id, info], file_type=parquet +04)----DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_left.parquet], projection=[id, data], file_type=parquet # LEFT JOIN correctness: all left rows appear, unmatched right rows produce NULLs query ITT @@ -238,8 +237,8 @@ logical_plan physical_plan 01)ProjectionExec: expr=[id@1 as id, data@2 as data, info@0 as info] 02)--HashJoinExec: mode=CollectLeft, join_type=Left, on=[(id@0, id@0)], projection=[info@1, id@2, data@3] -03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_right.parquet]]}, projection=[id, info], file_type=parquet -04)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_left.parquet]]}, projection=[id, data], file_type=parquet, predicate=DynamicFilter [ empty ] +03)----DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_right.parquet], projection=[id, info], file_type=parquet +04)----DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_left.parquet], projection=[id, data], file_type=parquet, predicate=DynamicFilter [ empty ] # RIGHT JOIN correctness: all right rows appear, unmatched left rows produce NULLs query ITT @@ -268,8 +267,8 @@ logical_plan physical_plan 01)ProjectionExec: expr=[id@2 as id, id@0 as rid, data@3 as data, info@1 as info] 02)--HashJoinExec: mode=CollectLeft, join_type=Full, on=[(id@0, id@0)] -03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_right.parquet]]}, projection=[id, info], file_type=parquet -04)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_left.parquet]]}, projection=[id, data], file_type=parquet +03)----DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_right.parquet], projection=[id, info], file_type=parquet +04)----DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_left.parquet], projection=[id, data], file_type=parquet # LEFT SEMI JOIN: optimizer swaps to RightSemi (build=right_parquet, probe=left_parquet) # and pushes the self-generated filter to the right side (left parquet). @@ -287,8 +286,8 @@ logical_plan 06)------TableScan: right_parquet projection=[id] physical_plan 01)HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(id@0, id@0)] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_right.parquet]]}, projection=[id], file_type=parquet -03)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_left.parquet]]}, projection=[id, data], file_type=parquet, predicate=DynamicFilter [ empty ] +02)--DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_right.parquet], projection=[id], file_type=parquet +03)--DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_left.parquet], projection=[id, data], file_type=parquet, predicate=DynamicFilter [ empty ] # LEFT SEMI JOIN (physical LeftSemi): reverse table roles so optimizer keeps LeftSemi # (right_parquet has 3 rows < left_parquet has 5 rows, so no swap occurs). @@ -307,8 +306,8 @@ logical_plan 06)------TableScan: left_parquet projection=[id] physical_plan 01)HashJoinExec: mode=CollectLeft, join_type=LeftSemi, on=[(id@0, id@0)] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_right.parquet]]}, projection=[id, info], file_type=parquet -03)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_left.parquet]]}, projection=[id], file_type=parquet, predicate=DynamicFilter [ empty ] +02)--DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_right.parquet], projection=[id, info], file_type=parquet +03)--DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_left.parquet], projection=[id], file_type=parquet, predicate=DynamicFilter [ empty ] # LEFT SEMI (physical LeftSemi) correctness: only right rows with matching left ids query IT rowsort @@ -336,8 +335,8 @@ logical_plan 06)------TableScan: right_parquet projection=[id] physical_plan 01)HashJoinExec: mode=CollectLeft, join_type=LeftAnti, on=[(id@0, id@0)] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_left.parquet]]}, projection=[id, data], file_type=parquet -03)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_right.parquet]]}, projection=[id], file_type=parquet, predicate=DynamicFilter [ empty ] +02)--DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_left.parquet], projection=[id, data], file_type=parquet +03)--DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_right.parquet], projection=[id], file_type=parquet, predicate=DynamicFilter [ empty ] # LEFT MARK JOIN: the OR prevents decorrelation to LeftSemi, so the optimizer # uses LeftMark. Self-generated dynamic filter pushes to the probe side. @@ -360,8 +359,8 @@ physical_plan 01)FilterExec: mark@2 OR id@0 = 999, projection=[id@0, info@1] 02)--RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 03)----HashJoinExec: mode=CollectLeft, join_type=LeftMark, on=[(id@0, id@0)] -04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_right.parquet]]}, projection=[id, info], file_type=parquet -05)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_left.parquet]]}, projection=[id], file_type=parquet, predicate=DynamicFilter [ empty ] +04)------DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_right.parquet], projection=[id, info], file_type=parquet +05)------DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_left.parquet], projection=[id], file_type=parquet, predicate=DynamicFilter [ empty ] # LEFT MARK correctness: all right rows match EXISTS, so all 3 appear query IT rowsort @@ -398,8 +397,8 @@ logical_plan physical_plan 01)SortExec: TopK(fetch=2), expr=[id@0 ASC NULLS LAST], preserve_partitioning=[false] 02)--HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(id@0, id@0)] -03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_right.parquet]]}, projection=[id], file_type=parquet, predicate=DynamicFilter [ empty ] -04)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_left.parquet]]}, projection=[id, data], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] +03)----DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_right.parquet], projection=[id], file_type=parquet, predicate=DynamicFilter [ empty ] +04)----DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_left.parquet], projection=[id, data], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] # Correctness check query IT @@ -432,8 +431,8 @@ logical_plan physical_plan 01)SortExec: TopK(fetch=2), expr=[id@0 ASC NULLS LAST], preserve_partitioning=[false] 02)--HashJoinExec: mode=CollectLeft, join_type=LeftAnti, on=[(id@0, id@0)] -03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_left.parquet]]}, projection=[id, data], file_type=parquet, predicate=DynamicFilter [ empty ] -04)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_right.parquet]]}, projection=[id], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] +03)----DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_left.parquet], projection=[id, data], file_type=parquet, predicate=DynamicFilter [ empty ] +04)----DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_right.parquet], projection=[id], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] # Correctness check query IT @@ -470,8 +469,8 @@ logical_plan physical_plan 01)ProjectionExec: expr=[id@1 as id, data@2 as data, info@0 as info] 02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(id@0, id@0)], projection=[info@1, id@2, data@3] -03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_right.parquet]]}, projection=[id, info], file_type=parquet -04)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_left.parquet]]}, projection=[id, data], file_type=parquet, predicate=DynamicFilter [ empty ] +03)----DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_right.parquet], projection=[id, info], file_type=parquet +04)----DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_left.parquet], projection=[id, data], file_type=parquet, predicate=DynamicFilter [ empty ] # Enable TopK, disable Join statement ok @@ -496,8 +495,8 @@ logical_plan physical_plan 01)ProjectionExec: expr=[id@1 as id, data@2 as data, info@0 as info] 02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(id@0, id@0)], projection=[info@1, id@2, data@3] -03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_right.parquet]]}, projection=[id, info], file_type=parquet -04)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_left.parquet]]}, projection=[id, data], file_type=parquet +03)----DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_right.parquet], projection=[id, info], file_type=parquet +04)----DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_left.parquet], projection=[id, data], file_type=parquet # Test 4: Aggregate dynamic filter pushdown @@ -543,8 +542,7 @@ physical_plan 01)AggregateExec: mode=Final, gby=[], aggr=[max(agg_parquet.score)] 02)--CoalescePartitionsExec 03)----AggregateExec: mode=Partial, gby=[], aggr=[max(agg_parquet.score)] -04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/agg_data.parquet]]}, projection=[score], file_type=parquet, predicate=category@0 = alpha AND DynamicFilter [ empty ], pruning_predicate=category_null_count@2 != row_count@3 AND category_min@0 <= alpha AND alpha <= category_max@1, required_guarantees=[category in (alpha)] +04)------DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/agg_data.parquet], projection=[score], file_type=parquet, predicate=category@0 = alpha AND DynamicFilter [ empty ], pruning_predicate=category_null_count@2 != row_count@3 AND category_min@0 <= alpha AND alpha <= category_max@1, required_guarantees=[category in (alpha)] # Test 4b: COUNT + MAX — DynamicFilter should NOT appear here in mixed aggregates @@ -562,8 +560,7 @@ physical_plan 02)--AggregateExec: mode=Final, gby=[], aggr=[count(Int64(1)), max(agg_parquet.score)] 03)----CoalescePartitionsExec 04)------AggregateExec: mode=Partial, gby=[], aggr=[count(Int64(1)), max(agg_parquet.score)] -05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/agg_data.parquet]]}, projection=[score], file_type=parquet, predicate=category@0 = alpha, pruning_predicate=category_null_count@2 != row_count@3 AND category_min@0 <= alpha AND alpha <= category_max@1, required_guarantees=[category in (alpha)] +05)--------DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/agg_data.parquet], projection=[score], file_type=parquet, predicate=category@0 = alpha, pruning_predicate=category_null_count@2 != row_count@3 AND category_min@0 <= alpha AND alpha <= category_max@1, required_guarantees=[category in (alpha)] # Disable aggregate dynamic filters only statement ok @@ -582,8 +579,7 @@ physical_plan 01)AggregateExec: mode=Final, gby=[], aggr=[max(agg_parquet.score)] 02)--CoalescePartitionsExec 03)----AggregateExec: mode=Partial, gby=[], aggr=[max(agg_parquet.score)] -04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/agg_data.parquet]]}, projection=[score], file_type=parquet, predicate=category@0 = alpha, pruning_predicate=category_null_count@2 != row_count@3 AND category_min@0 <= alpha AND alpha <= category_max@1, required_guarantees=[category in (alpha)] +04)------DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/agg_data.parquet], projection=[score], file_type=parquet, predicate=category@0 = alpha, pruning_predicate=category_null_count@2 != row_count@3 AND category_min@0 <= alpha AND alpha <= category_max@1, required_guarantees=[category in (alpha)] statement ok SET datafusion.optimizer.enable_aggregate_dynamic_filter_pushdown = true; @@ -645,8 +641,8 @@ logical_plan physical_plan 01)ProjectionExec: expr=[id@1 as id, data@2 as data, info@0 as info] 02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(id@0, id@0)], projection=[info@1, id@2, data@3] -03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_right.parquet]]}, projection=[id, info], file_type=parquet -04)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_left.parquet]]}, projection=[id, data], file_type=parquet +03)----DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_right.parquet], projection=[id, info], file_type=parquet +04)----DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_left.parquet], projection=[id, data], file_type=parquet # Re-enable statement ok @@ -693,8 +689,8 @@ logical_plan physical_plan 01)ProjectionExec: expr=[id@1 as id, data@2 as data, info@0 as info] 02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(id@0, id@0)], projection=[info@1, id@2, data@3] -03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_right.parquet]]}, projection=[id, info], file_type=parquet -04)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_left.parquet]]}, projection=[id, data], file_type=parquet, predicate=DynamicFilter [ empty ] +03)----DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_right.parquet], projection=[id, info], file_type=parquet +04)----DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_filter_pushdown_config/join_left.parquet], projection=[id, data], file_type=parquet, predicate=DynamicFilter [ empty ] # Test 6: Regression test for issue #20213 - dynamic filter applied to wrong table # when subquery join has same column names on both sides. diff --git a/datafusion/sqllogictest/test_files/explain.slt b/datafusion/sqllogictest/test_files/explain.slt index c5907d497500e..201138387d251 100644 --- a/datafusion/sqllogictest/test_files/explain.slt +++ b/datafusion/sqllogictest/test_files/explain.slt @@ -293,7 +293,7 @@ CREATE EXTERNAL TABLE alltypes_plain STORED AS PARQUET LOCATION '../../parquet-t query TT EXPLAIN SELECT * FROM alltypes_plain limit 10; ---- -physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, file_type=parquet, statistics=[Rows=Exact(8), Bytes=Absent, [(Col[0]: ScanBytes=Exact(32)),(Col[1]: ScanBytes=Inexact(24)),(Col[2]: ScanBytes=Exact(32)),(Col[3]: ScanBytes=Exact(32)),(Col[4]: ScanBytes=Exact(32)),(Col[5]: ScanBytes=Exact(64)),(Col[6]: ScanBytes=Exact(32)),(Col[7]: ScanBytes=Exact(64)),(Col[8]: ScanBytes=Inexact(88)),(Col[9]: ScanBytes=Inexact(49)),(Col[10]: ScanBytes=Exact(64))]] +physical_plan DataSourceExec: files=[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet], projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, file_type=parquet, statistics=[Rows=Exact(8), Bytes=Absent, [(Col[0]: ScanBytes=Exact(32)),(Col[1]: ScanBytes=Inexact(24)),(Col[2]: ScanBytes=Exact(32)),(Col[3]: ScanBytes=Exact(32)),(Col[4]: ScanBytes=Exact(32)),(Col[5]: ScanBytes=Exact(64)),(Col[6]: ScanBytes=Exact(32)),(Col[7]: ScanBytes=Exact(64)),(Col[8]: ScanBytes=Inexact(88)),(Col[9]: ScanBytes=Inexact(49)),(Col[10]: ScanBytes=Exact(64))]] # explain verbose with both collect & show statistics on query TT @@ -301,14 +301,14 @@ EXPLAIN VERBOSE SELECT * FROM alltypes_plain limit 10; ---- initial_physical_plan 01)GlobalLimitExec: skip=0, fetch=10, statistics=[Rows=Exact(8), Bytes=Absent, [(Col[0]: ScanBytes=Exact(32)),(Col[1]: ScanBytes=Inexact(24)),(Col[2]: ScanBytes=Exact(32)),(Col[3]: ScanBytes=Exact(32)),(Col[4]: ScanBytes=Exact(32)),(Col[5]: ScanBytes=Exact(64)),(Col[6]: ScanBytes=Exact(32)),(Col[7]: ScanBytes=Exact(64)),(Col[8]: ScanBytes=Inexact(88)),(Col[9]: ScanBytes=Inexact(49)),(Col[10]: ScanBytes=Exact(64))]] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, file_type=parquet, statistics=[Rows=Exact(8), Bytes=Absent, [(Col[0]: ScanBytes=Exact(32)),(Col[1]: ScanBytes=Inexact(24)),(Col[2]: ScanBytes=Exact(32)),(Col[3]: ScanBytes=Exact(32)),(Col[4]: ScanBytes=Exact(32)),(Col[5]: ScanBytes=Exact(64)),(Col[6]: ScanBytes=Exact(32)),(Col[7]: ScanBytes=Exact(64)),(Col[8]: ScanBytes=Inexact(88)),(Col[9]: ScanBytes=Inexact(49)),(Col[10]: ScanBytes=Exact(64))]] +02)--DataSourceExec: files=[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet], projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, file_type=parquet, statistics=[Rows=Exact(8), Bytes=Absent, [(Col[0]: ScanBytes=Exact(32)),(Col[1]: ScanBytes=Inexact(24)),(Col[2]: ScanBytes=Exact(32)),(Col[3]: ScanBytes=Exact(32)),(Col[4]: ScanBytes=Exact(32)),(Col[5]: ScanBytes=Exact(64)),(Col[6]: ScanBytes=Exact(32)),(Col[7]: ScanBytes=Exact(64)),(Col[8]: ScanBytes=Inexact(88)),(Col[9]: ScanBytes=Inexact(49)),(Col[10]: ScanBytes=Exact(64))]] initial_physical_plan_with_schema 01)GlobalLimitExec: skip=0, fetch=10, schema=[id:Int32;N, bool_col:Boolean;N, tinyint_col:Int32;N, smallint_col:Int32;N, int_col:Int32;N, bigint_col:Int64;N, float_col:Float32;N, double_col:Float64;N, date_string_col:BinaryView;N, string_col:BinaryView;N, timestamp_col:Timestamp(ns);N] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, file_type=parquet, schema=[id:Int32;N, bool_col:Boolean;N, tinyint_col:Int32;N, smallint_col:Int32;N, int_col:Int32;N, bigint_col:Int64;N, float_col:Float32;N, double_col:Float64;N, date_string_col:BinaryView;N, string_col:BinaryView;N, timestamp_col:Timestamp(ns);N] +02)--DataSourceExec: files=[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet], projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, file_type=parquet, schema=[id:Int32;N, bool_col:Boolean;N, tinyint_col:Int32;N, smallint_col:Int32;N, int_col:Int32;N, bigint_col:Int64;N, float_col:Float32;N, double_col:Float64;N, date_string_col:BinaryView;N, string_col:BinaryView;N, timestamp_col:Timestamp(ns);N] physical_plan after OutputRequirements 01)OutputRequirementExec: order_by=[], dist_by=Unspecified, statistics=[Rows=Exact(8), Bytes=Absent, [(Col[0]: ScanBytes=Exact(32)),(Col[1]: ScanBytes=Inexact(24)),(Col[2]: ScanBytes=Exact(32)),(Col[3]: ScanBytes=Exact(32)),(Col[4]: ScanBytes=Exact(32)),(Col[5]: ScanBytes=Exact(64)),(Col[6]: ScanBytes=Exact(32)),(Col[7]: ScanBytes=Exact(64)),(Col[8]: ScanBytes=Inexact(88)),(Col[9]: ScanBytes=Inexact(49)),(Col[10]: ScanBytes=Exact(64))]] 02)--GlobalLimitExec: skip=0, fetch=10, statistics=[Rows=Exact(8), Bytes=Absent, [(Col[0]: ScanBytes=Exact(32)),(Col[1]: ScanBytes=Inexact(24)),(Col[2]: ScanBytes=Exact(32)),(Col[3]: ScanBytes=Exact(32)),(Col[4]: ScanBytes=Exact(32)),(Col[5]: ScanBytes=Exact(64)),(Col[6]: ScanBytes=Exact(32)),(Col[7]: ScanBytes=Exact(64)),(Col[8]: ScanBytes=Inexact(88)),(Col[9]: ScanBytes=Inexact(49)),(Col[10]: ScanBytes=Exact(64))]] -03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, file_type=parquet, statistics=[Rows=Exact(8), Bytes=Absent, [(Col[0]: ScanBytes=Exact(32)),(Col[1]: ScanBytes=Inexact(24)),(Col[2]: ScanBytes=Exact(32)),(Col[3]: ScanBytes=Exact(32)),(Col[4]: ScanBytes=Exact(32)),(Col[5]: ScanBytes=Exact(64)),(Col[6]: ScanBytes=Exact(32)),(Col[7]: ScanBytes=Exact(64)),(Col[8]: ScanBytes=Inexact(88)),(Col[9]: ScanBytes=Inexact(49)),(Col[10]: ScanBytes=Exact(64))]] +03)----DataSourceExec: files=[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet], projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, file_type=parquet, statistics=[Rows=Exact(8), Bytes=Absent, [(Col[0]: ScanBytes=Exact(32)),(Col[1]: ScanBytes=Inexact(24)),(Col[2]: ScanBytes=Exact(32)),(Col[3]: ScanBytes=Exact(32)),(Col[4]: ScanBytes=Exact(32)),(Col[5]: ScanBytes=Exact(64)),(Col[6]: ScanBytes=Exact(32)),(Col[7]: ScanBytes=Exact(64)),(Col[8]: ScanBytes=Inexact(88)),(Col[9]: ScanBytes=Inexact(49)),(Col[10]: ScanBytes=Exact(64))]] physical_plan after aggregate_statistics SAME TEXT AS ABOVE physical_plan after join_selection SAME TEXT AS ABOVE physical_plan after LimitedDistinctAggregation SAME TEXT AS ABOVE @@ -320,17 +320,17 @@ physical_plan after OptimizeAggregateOrder SAME TEXT AS ABOVE physical_plan after ProjectionPushdown SAME TEXT AS ABOVE physical_plan after OutputRequirements 01)GlobalLimitExec: skip=0, fetch=10, statistics=[Rows=Exact(8), Bytes=Absent, [(Col[0]: ScanBytes=Exact(32)),(Col[1]: ScanBytes=Inexact(24)),(Col[2]: ScanBytes=Exact(32)),(Col[3]: ScanBytes=Exact(32)),(Col[4]: ScanBytes=Exact(32)),(Col[5]: ScanBytes=Exact(64)),(Col[6]: ScanBytes=Exact(32)),(Col[7]: ScanBytes=Exact(64)),(Col[8]: ScanBytes=Inexact(88)),(Col[9]: ScanBytes=Inexact(49)),(Col[10]: ScanBytes=Exact(64))]] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, file_type=parquet, statistics=[Rows=Exact(8), Bytes=Absent, [(Col[0]: ScanBytes=Exact(32)),(Col[1]: ScanBytes=Inexact(24)),(Col[2]: ScanBytes=Exact(32)),(Col[3]: ScanBytes=Exact(32)),(Col[4]: ScanBytes=Exact(32)),(Col[5]: ScanBytes=Exact(64)),(Col[6]: ScanBytes=Exact(32)),(Col[7]: ScanBytes=Exact(64)),(Col[8]: ScanBytes=Inexact(88)),(Col[9]: ScanBytes=Inexact(49)),(Col[10]: ScanBytes=Exact(64))]] +02)--DataSourceExec: files=[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet], projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, file_type=parquet, statistics=[Rows=Exact(8), Bytes=Absent, [(Col[0]: ScanBytes=Exact(32)),(Col[1]: ScanBytes=Inexact(24)),(Col[2]: ScanBytes=Exact(32)),(Col[3]: ScanBytes=Exact(32)),(Col[4]: ScanBytes=Exact(32)),(Col[5]: ScanBytes=Exact(64)),(Col[6]: ScanBytes=Exact(32)),(Col[7]: ScanBytes=Exact(64)),(Col[8]: ScanBytes=Inexact(88)),(Col[9]: ScanBytes=Inexact(49)),(Col[10]: ScanBytes=Exact(64))]] physical_plan after LimitAggregation SAME TEXT AS ABOVE physical_plan after LimitPushPastWindows SAME TEXT AS ABOVE -physical_plan after LimitPushdown DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, file_type=parquet, statistics=[Rows=Exact(8), Bytes=Absent, [(Col[0]: ScanBytes=Exact(32)),(Col[1]: ScanBytes=Inexact(24)),(Col[2]: ScanBytes=Exact(32)),(Col[3]: ScanBytes=Exact(32)),(Col[4]: ScanBytes=Exact(32)),(Col[5]: ScanBytes=Exact(64)),(Col[6]: ScanBytes=Exact(32)),(Col[7]: ScanBytes=Exact(64)),(Col[8]: ScanBytes=Inexact(88)),(Col[9]: ScanBytes=Inexact(49)),(Col[10]: ScanBytes=Exact(64))]] +physical_plan after LimitPushdown DataSourceExec: files=[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet], projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, file_type=parquet, statistics=[Rows=Exact(8), Bytes=Absent, [(Col[0]: ScanBytes=Exact(32)),(Col[1]: ScanBytes=Inexact(24)),(Col[2]: ScanBytes=Exact(32)),(Col[3]: ScanBytes=Exact(32)),(Col[4]: ScanBytes=Exact(32)),(Col[5]: ScanBytes=Exact(64)),(Col[6]: ScanBytes=Exact(32)),(Col[7]: ScanBytes=Exact(64)),(Col[8]: ScanBytes=Inexact(88)),(Col[9]: ScanBytes=Inexact(49)),(Col[10]: ScanBytes=Exact(64))]] physical_plan after ProjectionPushdown SAME TEXT AS ABOVE physical_plan after PushdownSort SAME TEXT AS ABOVE physical_plan after EnsureCooperative SAME TEXT AS ABOVE physical_plan after FilterPushdown(Post) SAME TEXT AS ABOVE physical_plan after SanityCheckPlan SAME TEXT AS ABOVE -physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, file_type=parquet, statistics=[Rows=Exact(8), Bytes=Absent, [(Col[0]: ScanBytes=Exact(32)),(Col[1]: ScanBytes=Inexact(24)),(Col[2]: ScanBytes=Exact(32)),(Col[3]: ScanBytes=Exact(32)),(Col[4]: ScanBytes=Exact(32)),(Col[5]: ScanBytes=Exact(64)),(Col[6]: ScanBytes=Exact(32)),(Col[7]: ScanBytes=Exact(64)),(Col[8]: ScanBytes=Inexact(88)),(Col[9]: ScanBytes=Inexact(49)),(Col[10]: ScanBytes=Exact(64))]] -physical_plan_with_schema DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, file_type=parquet, schema=[id:Int32;N, bool_col:Boolean;N, tinyint_col:Int32;N, smallint_col:Int32;N, int_col:Int32;N, bigint_col:Int64;N, float_col:Float32;N, double_col:Float64;N, date_string_col:BinaryView;N, string_col:BinaryView;N, timestamp_col:Timestamp(ns);N] +physical_plan DataSourceExec: files=[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet], projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, file_type=parquet, statistics=[Rows=Exact(8), Bytes=Absent, [(Col[0]: ScanBytes=Exact(32)),(Col[1]: ScanBytes=Inexact(24)),(Col[2]: ScanBytes=Exact(32)),(Col[3]: ScanBytes=Exact(32)),(Col[4]: ScanBytes=Exact(32)),(Col[5]: ScanBytes=Exact(64)),(Col[6]: ScanBytes=Exact(32)),(Col[7]: ScanBytes=Exact(64)),(Col[8]: ScanBytes=Inexact(88)),(Col[9]: ScanBytes=Inexact(49)),(Col[10]: ScanBytes=Exact(64))]] +physical_plan_with_schema DataSourceExec: files=[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet], projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, file_type=parquet, schema=[id:Int32;N, bool_col:Boolean;N, tinyint_col:Int32;N, smallint_col:Int32;N, int_col:Int32;N, bigint_col:Int64;N, float_col:Float32;N, double_col:Float64;N, date_string_col:BinaryView;N, string_col:BinaryView;N, timestamp_col:Timestamp(ns);N] statement ok @@ -342,17 +342,17 @@ EXPLAIN VERBOSE SELECT * FROM alltypes_plain limit 10; ---- initial_physical_plan 01)GlobalLimitExec: skip=0, fetch=10 -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, file_type=parquet +02)--DataSourceExec: files=[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet], projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, file_type=parquet initial_physical_plan_with_stats 01)GlobalLimitExec: skip=0, fetch=10, statistics=[Rows=Exact(8), Bytes=Absent, [(Col[0]: ScanBytes=Exact(32)),(Col[1]: ScanBytes=Inexact(24)),(Col[2]: ScanBytes=Exact(32)),(Col[3]: ScanBytes=Exact(32)),(Col[4]: ScanBytes=Exact(32)),(Col[5]: ScanBytes=Exact(64)),(Col[6]: ScanBytes=Exact(32)),(Col[7]: ScanBytes=Exact(64)),(Col[8]: ScanBytes=Inexact(88)),(Col[9]: ScanBytes=Inexact(49)),(Col[10]: ScanBytes=Exact(64))]] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, file_type=parquet, statistics=[Rows=Exact(8), Bytes=Absent, [(Col[0]: ScanBytes=Exact(32)),(Col[1]: ScanBytes=Inexact(24)),(Col[2]: ScanBytes=Exact(32)),(Col[3]: ScanBytes=Exact(32)),(Col[4]: ScanBytes=Exact(32)),(Col[5]: ScanBytes=Exact(64)),(Col[6]: ScanBytes=Exact(32)),(Col[7]: ScanBytes=Exact(64)),(Col[8]: ScanBytes=Inexact(88)),(Col[9]: ScanBytes=Inexact(49)),(Col[10]: ScanBytes=Exact(64))]] +02)--DataSourceExec: files=[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet], projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, file_type=parquet, statistics=[Rows=Exact(8), Bytes=Absent, [(Col[0]: ScanBytes=Exact(32)),(Col[1]: ScanBytes=Inexact(24)),(Col[2]: ScanBytes=Exact(32)),(Col[3]: ScanBytes=Exact(32)),(Col[4]: ScanBytes=Exact(32)),(Col[5]: ScanBytes=Exact(64)),(Col[6]: ScanBytes=Exact(32)),(Col[7]: ScanBytes=Exact(64)),(Col[8]: ScanBytes=Inexact(88)),(Col[9]: ScanBytes=Inexact(49)),(Col[10]: ScanBytes=Exact(64))]] initial_physical_plan_with_schema 01)GlobalLimitExec: skip=0, fetch=10, schema=[id:Int32;N, bool_col:Boolean;N, tinyint_col:Int32;N, smallint_col:Int32;N, int_col:Int32;N, bigint_col:Int64;N, float_col:Float32;N, double_col:Float64;N, date_string_col:BinaryView;N, string_col:BinaryView;N, timestamp_col:Timestamp(ns);N] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, file_type=parquet, schema=[id:Int32;N, bool_col:Boolean;N, tinyint_col:Int32;N, smallint_col:Int32;N, int_col:Int32;N, bigint_col:Int64;N, float_col:Float32;N, double_col:Float64;N, date_string_col:BinaryView;N, string_col:BinaryView;N, timestamp_col:Timestamp(ns);N] +02)--DataSourceExec: files=[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet], projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, file_type=parquet, schema=[id:Int32;N, bool_col:Boolean;N, tinyint_col:Int32;N, smallint_col:Int32;N, int_col:Int32;N, bigint_col:Int64;N, float_col:Float32;N, double_col:Float64;N, date_string_col:BinaryView;N, string_col:BinaryView;N, timestamp_col:Timestamp(ns);N] physical_plan after OutputRequirements 01)OutputRequirementExec: order_by=[], dist_by=Unspecified 02)--GlobalLimitExec: skip=0, fetch=10 -03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, file_type=parquet +03)----DataSourceExec: files=[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet], projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, file_type=parquet physical_plan after aggregate_statistics SAME TEXT AS ABOVE physical_plan after join_selection SAME TEXT AS ABOVE physical_plan after LimitedDistinctAggregation SAME TEXT AS ABOVE @@ -364,18 +364,18 @@ physical_plan after OptimizeAggregateOrder SAME TEXT AS ABOVE physical_plan after ProjectionPushdown SAME TEXT AS ABOVE physical_plan after OutputRequirements 01)GlobalLimitExec: skip=0, fetch=10 -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, file_type=parquet +02)--DataSourceExec: files=[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet], projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, file_type=parquet physical_plan after LimitAggregation SAME TEXT AS ABOVE physical_plan after LimitPushPastWindows SAME TEXT AS ABOVE -physical_plan after LimitPushdown DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, file_type=parquet +physical_plan after LimitPushdown DataSourceExec: files=[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet], projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, file_type=parquet physical_plan after ProjectionPushdown SAME TEXT AS ABOVE physical_plan after PushdownSort SAME TEXT AS ABOVE physical_plan after EnsureCooperative SAME TEXT AS ABOVE physical_plan after FilterPushdown(Post) SAME TEXT AS ABOVE physical_plan after SanityCheckPlan SAME TEXT AS ABOVE -physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, file_type=parquet -physical_plan_with_stats DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, file_type=parquet, statistics=[Rows=Exact(8), Bytes=Absent, [(Col[0]: ScanBytes=Exact(32)),(Col[1]: ScanBytes=Inexact(24)),(Col[2]: ScanBytes=Exact(32)),(Col[3]: ScanBytes=Exact(32)),(Col[4]: ScanBytes=Exact(32)),(Col[5]: ScanBytes=Exact(64)),(Col[6]: ScanBytes=Exact(32)),(Col[7]: ScanBytes=Exact(64)),(Col[8]: ScanBytes=Inexact(88)),(Col[9]: ScanBytes=Inexact(49)),(Col[10]: ScanBytes=Exact(64))]] -physical_plan_with_schema DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, file_type=parquet, schema=[id:Int32;N, bool_col:Boolean;N, tinyint_col:Int32;N, smallint_col:Int32;N, int_col:Int32;N, bigint_col:Int64;N, float_col:Float32;N, double_col:Float64;N, date_string_col:BinaryView;N, string_col:BinaryView;N, timestamp_col:Timestamp(ns);N] +physical_plan DataSourceExec: files=[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet], projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, file_type=parquet +physical_plan_with_stats DataSourceExec: files=[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet], projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, file_type=parquet, statistics=[Rows=Exact(8), Bytes=Absent, [(Col[0]: ScanBytes=Exact(32)),(Col[1]: ScanBytes=Inexact(24)),(Col[2]: ScanBytes=Exact(32)),(Col[3]: ScanBytes=Exact(32)),(Col[4]: ScanBytes=Exact(32)),(Col[5]: ScanBytes=Exact(64)),(Col[6]: ScanBytes=Exact(32)),(Col[7]: ScanBytes=Exact(64)),(Col[8]: ScanBytes=Inexact(88)),(Col[9]: ScanBytes=Inexact(49)),(Col[10]: ScanBytes=Exact(64))]] +physical_plan_with_schema DataSourceExec: files=[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet], projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, file_type=parquet, schema=[id:Int32;N, bool_col:Boolean;N, tinyint_col:Int32;N, smallint_col:Int32;N, int_col:Int32;N, bigint_col:Int64;N, float_col:Float32;N, double_col:Float64;N, date_string_col:BinaryView;N, string_col:BinaryView;N, timestamp_col:Timestamp(ns);N] statement ok diff --git a/datafusion/sqllogictest/test_files/explain_tree.slt b/datafusion/sqllogictest/test_files/explain_tree.slt index 3a183a7357430..d297641e919c9 100644 --- a/datafusion/sqllogictest/test_files/explain_tree.slt +++ b/datafusion/sqllogictest/test_files/explain_tree.slt @@ -560,23 +560,14 @@ physical_plan 05)│ string_col != foo │ 06)└─────────────┬─────────────┘ 07)┌─────────────┴─────────────┐ -08)│ RepartitionExec │ +08)│ DataSourceExec │ 09)│ -------------------- │ -10)│ partition_count(in->out): │ -11)│ 1 -> 4 │ +10)│ files: 1 │ +11)│ format: parquet │ 12)│ │ -13)│ partitioning_scheme: │ -14)│ RoundRobinBatch(4) │ -15)└─────────────┬─────────────┘ -16)┌─────────────┴─────────────┐ -17)│ DataSourceExec │ -18)│ -------------------- │ -19)│ files: 1 │ -20)│ format: parquet │ -21)│ │ -22)│ predicate: │ -23)│ string_col != foo │ -24)└───────────────────────────┘ +13)│ predicate: │ +14)│ string_col != foo │ +15)└───────────────────────────┘ # Query with filter on memory query TT diff --git a/datafusion/sqllogictest/test_files/grouping_set_repartition.slt b/datafusion/sqllogictest/test_files/grouping_set_repartition.slt index 16ab90651c8b3..e01e74fade03d 100644 --- a/datafusion/sqllogictest/test_files/grouping_set_repartition.slt +++ b/datafusion/sqllogictest/test_files/grouping_set_repartition.slt @@ -151,19 +151,19 @@ physical_plan 10)------------------RepartitionExec: partitioning=Hash([brand@0], 4), input_partitions=4 11)--------------------AggregateExec: mode=Partial, gby=[brand@0 as brand], aggr=[sum(sales.amount)] 12)----------------------FilterExec: channel@0 = store, projection=[brand@1, amount@2] -13)------------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/grouping_set_repartition/part=1/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/grouping_set_repartition/part=2/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/grouping_set_repartition/part=3/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/grouping_set_repartition/part=4/data.parquet]]}, projection=[channel, brand, amount], file_type=parquet, predicate=channel@0 = store, pruning_predicate=channel_null_count@2 != row_count@3 AND channel_min@0 <= store AND store <= channel_max@1, required_guarantees=[channel in (store)] +13)------------------------DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/grouping_set_repartition/part=1/data.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/grouping_set_repartition/part=2/data.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/grouping_set_repartition/part=3/data.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/grouping_set_repartition/part=4/data.parquet], projection=[channel, brand, amount], file_type=parquet, predicate=channel@0 = store, pruning_predicate=channel_null_count@2 != row_count@3 AND channel_min@0 <= store AND store <= channel_max@1, required_guarantees=[channel in (store)] 14)--------------ProjectionExec: expr=[web as channel, brand@0 as brand, sum(sales.amount)@1 as total] 15)----------------AggregateExec: mode=FinalPartitioned, gby=[brand@0 as brand], aggr=[sum(sales.amount)] 16)------------------RepartitionExec: partitioning=Hash([brand@0], 4), input_partitions=4 17)--------------------AggregateExec: mode=Partial, gby=[brand@0 as brand], aggr=[sum(sales.amount)] 18)----------------------FilterExec: channel@0 = web, projection=[brand@1, amount@2] -19)------------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/grouping_set_repartition/part=1/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/grouping_set_repartition/part=2/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/grouping_set_repartition/part=3/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/grouping_set_repartition/part=4/data.parquet]]}, projection=[channel, brand, amount], file_type=parquet, predicate=channel@0 = web, pruning_predicate=channel_null_count@2 != row_count@3 AND channel_min@0 <= web AND web <= channel_max@1, required_guarantees=[channel in (web)] +19)------------------------DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/grouping_set_repartition/part=1/data.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/grouping_set_repartition/part=2/data.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/grouping_set_repartition/part=3/data.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/grouping_set_repartition/part=4/data.parquet], projection=[channel, brand, amount], file_type=parquet, predicate=channel@0 = web, pruning_predicate=channel_null_count@2 != row_count@3 AND channel_min@0 <= web AND web <= channel_max@1, required_guarantees=[channel in (web)] 20)--------------ProjectionExec: expr=[catalog as channel, brand@0 as brand, sum(sales.amount)@1 as total] 21)----------------AggregateExec: mode=FinalPartitioned, gby=[brand@0 as brand], aggr=[sum(sales.amount)] 22)------------------RepartitionExec: partitioning=Hash([brand@0], 4), input_partitions=4 23)--------------------AggregateExec: mode=Partial, gby=[brand@0 as brand], aggr=[sum(sales.amount)] 24)----------------------FilterExec: channel@0 = catalog, projection=[brand@1, amount@2] -25)------------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/grouping_set_repartition/part=1/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/grouping_set_repartition/part=2/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/grouping_set_repartition/part=3/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/grouping_set_repartition/part=4/data.parquet]]}, projection=[channel, brand, amount], file_type=parquet, predicate=channel@0 = catalog, pruning_predicate=channel_null_count@2 != row_count@3 AND channel_min@0 <= catalog AND catalog <= channel_max@1, required_guarantees=[channel in (catalog)] +25)------------------------DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/grouping_set_repartition/part=1/data.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/grouping_set_repartition/part=2/data.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/grouping_set_repartition/part=3/data.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/grouping_set_repartition/part=4/data.parquet], projection=[channel, brand, amount], file_type=parquet, predicate=channel@0 = catalog, pruning_predicate=channel_null_count@2 != row_count@3 AND channel_min@0 <= catalog AND catalog <= channel_max@1, required_guarantees=[channel in (catalog)] query TTI rowsort SELECT channel, brand, SUM(total) as grand_total diff --git a/datafusion/sqllogictest/test_files/information_schema.slt b/datafusion/sqllogictest/test_files/information_schema.slt index b61ceecb24fc0..35aee28b949dd 100644 --- a/datafusion/sqllogictest/test_files/information_schema.slt +++ b/datafusion/sqllogictest/test_files/information_schema.slt @@ -228,6 +228,7 @@ datafusion.execution.max_spill_file_size_bytes 134217728 datafusion.execution.meta_fetch_concurrency 32 datafusion.execution.minimum_parallel_output_files 4 datafusion.execution.objectstore_writer_buffer_size 10485760 +datafusion.execution.parquet.allow_morsel_driven true datafusion.execution.parquet.allow_single_file_parallelism true datafusion.execution.parquet.binary_as_string false datafusion.execution.parquet.bloom_filter_fpp NULL @@ -366,6 +367,7 @@ datafusion.execution.max_spill_file_size_bytes 134217728 Maximum size in bytes f datafusion.execution.meta_fetch_concurrency 32 Number of files to read in parallel when inferring schema and statistics datafusion.execution.minimum_parallel_output_files 4 Guarantees a minimum level of output files running in parallel. RecordBatches will be distributed in round robin fashion to each parallel writer. Each writer is closed and a new file opened once soft_max_rows_per_output_file is reached. datafusion.execution.objectstore_writer_buffer_size 10485760 Size (bytes) of data buffer DataFusion uses when writing output files. This affects the size of the data chunks that are uploaded to remote object stores (e.g. AWS S3). If very large (>= 100 GiB) output files are being written, it may be necessary to increase this size to avoid errors from the remote end point. +datafusion.execution.parquet.allow_morsel_driven true (reading) If true, the parquet reader will share work between partitions using morsel-driven execution. This can help mitigate data skew. datafusion.execution.parquet.allow_single_file_parallelism true (writing) Controls whether DataFusion will attempt to speed up writing parquet files by serializing them in parallel. Each column in each row group in each output file are serialized in parallel leveraging a maximum possible core count of n_files*n_row_groups*n_columns. datafusion.execution.parquet.binary_as_string false (reading) If true, parquet reader will read columns of `Binary/LargeBinary` with `Utf8`, and `BinaryView` with `Utf8View`. Parquet files generated by some legacy writers do not correctly set the UTF8 flag for strings, causing string columns to be loaded as BLOB instead. datafusion.execution.parquet.bloom_filter_fpp NULL (writing) Sets bloom filter false positive probability. If NULL, uses default parquet writer setting diff --git a/datafusion/sqllogictest/test_files/limit.slt b/datafusion/sqllogictest/test_files/limit.slt index ff3c49485a286..dae9736afe013 100644 --- a/datafusion/sqllogictest/test_files/limit.slt +++ b/datafusion/sqllogictest/test_files/limit.slt @@ -849,7 +849,7 @@ physical_plan 01)ProjectionExec: expr=[1 as foo] 02)--SortPreservingMergeExec: [part_key@0 ASC NULLS LAST], fetch=1 03)----SortExec: TopK(fetch=1), expr=[part_key@0 ASC NULLS LAST], preserve_partitioning=[true] -04)------DataSourceExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/limit/test_limit_with_partitions/part-0.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/limit/test_limit_with_partitions/part-1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/limit/test_limit_with_partitions/part-2.parquet]]}, projection=[part_key], file_type=parquet, predicate=DynamicFilter [ empty ] +04)------DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/limit/test_limit_with_partitions/part-0.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/limit/test_limit_with_partitions/part-1.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/limit/test_limit_with_partitions/part-2.parquet], projection=[part_key], file_type=parquet, predicate=DynamicFilter [ empty ] query I with selection as ( diff --git a/datafusion/sqllogictest/test_files/limit_pruning.slt b/datafusion/sqllogictest/test_files/limit_pruning.slt index 72672b707d4f5..a1493be1cf6c5 100644 --- a/datafusion/sqllogictest/test_files/limit_pruning.slt +++ b/datafusion/sqllogictest/test_files/limit_pruning.slt @@ -63,7 +63,9 @@ set datafusion.explain.analyze_level = summary; query TT explain analyze select * from tracking_data where species > 'M' AND s >= 50 limit 3; ---- -Plan with Metrics DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/limit_pruning/data.parquet]]}, projection=[species, s], limit=3, file_type=parquet, predicate=species@0 > M AND s@1 >= 50, pruning_predicate=species_null_count@1 != row_count@2 AND species_max@0 > M AND s_null_count@4 != row_count@2 AND s_max@3 >= 50, required_guarantees=[], metrics=[output_rows=3, elapsed_compute=, output_bytes=, files_ranges_pruned_statistics=1 total → 1 matched, row_groups_pruned_statistics=4 total → 3 matched -> 1 fully matched, row_groups_pruned_bloom_filter=3 total → 3 matched, page_index_pages_pruned=2 total → 2 matched, limit_pruned_row_groups=2 total → 0 matched, bytes_scanned=, metadata_load_time=, scan_efficiency_ratio= (171/2.35 K)] +Plan with Metrics +01)CoalescePartitionsExec: fetch=3, metrics=[output_rows=3, elapsed_compute=, output_bytes=142.0 B] +02)--DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/limit_pruning/data.parquet], projection=[species, s], limit=3, file_type=parquet, predicate=species@0 > M AND s@1 >= 50, pruning_predicate=species_null_count@1 != row_count@2 AND species_max@0 > M AND s_null_count@4 != row_count@2 AND s_max@3 >= 50, required_guarantees=[], metrics=[output_rows=3, elapsed_compute=, output_bytes=142.0 B, files_ranges_pruned_statistics=1 total → 1 matched, row_groups_pruned_statistics=4 total → 3 matched -> 1 fully matched, row_groups_pruned_bloom_filter=1 total → 1 matched, page_index_pages_pruned=2 total → 2 matched, limit_pruned_row_groups=2 total → 0 matched, bytes_scanned=171, metadata_load_time=, scan_efficiency_ratio=7.3% (171/2.35 K)] # limit_pruned_row_groups=0 total → 0 matched # because of order by, scan needs to preserve sort, so limit pruning is disabled @@ -71,8 +73,9 @@ query TT explain analyze select * from tracking_data where species > 'M' AND s >= 50 order by species limit 3; ---- Plan with Metrics -01)SortExec: TopK(fetch=3), expr=[species@0 ASC NULLS LAST], preserve_partitioning=[false], filter=[species@0 < Nlpine Sheep], metrics=[output_rows=3, elapsed_compute=, output_bytes=] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/limit_pruning/data.parquet]]}, projection=[species, s], file_type=parquet, predicate=species@0 > M AND s@1 >= 50 AND DynamicFilter [ species@0 < Nlpine Sheep ], pruning_predicate=species_null_count@1 != row_count@2 AND species_max@0 > M AND s_null_count@4 != row_count@2 AND s_max@3 >= 50 AND species_null_count@1 != row_count@2 AND species_min@5 < Nlpine Sheep, required_guarantees=[], metrics=[output_rows=3, elapsed_compute=, output_bytes=, files_ranges_pruned_statistics=1 total → 1 matched, row_groups_pruned_statistics=4 total → 3 matched -> 1 fully matched, row_groups_pruned_bloom_filter=3 total → 3 matched, page_index_pages_pruned=6 total → 6 matched, limit_pruned_row_groups=0 total → 0 matched, bytes_scanned=, metadata_load_time=, scan_efficiency_ratio= (521/2.35 K)] +01)SortPreservingMergeExec: [species@0 ASC NULLS LAST], fetch=3, metrics=[output_rows=3, elapsed_compute=, output_bytes=72.0 B] +02)--SortExec: TopK(fetch=3), expr=[species@0 ASC NULLS LAST], preserve_partitioning=[true], filter=[species@0 < Nlpine Sheep], metrics=[output_rows=3, elapsed_compute=, output_bytes=72.0 B] +03)----DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/limit_pruning/data.parquet], projection=[species, s], file_type=parquet, predicate=species@0 > M AND s@1 >= 50 AND DynamicFilter [ species@0 < Nlpine Sheep ], pruning_predicate=species_null_count@1 != row_count@2 AND species_max@0 > M AND s_null_count@4 != row_count@2 AND s_max@3 >= 50 AND species_null_count@1 != row_count@2 AND species_min@5 < Nlpine Sheep, required_guarantees=[], metrics=[output_rows=3, elapsed_compute=, output_bytes=142.0 B, files_ranges_pruned_statistics=3 total → 3 matched, row_groups_pruned_statistics=7 total → 6 matched -> fully matched, row_groups_pruned_bloom_filter=3 total → 3 matched, page_index_pages_pruned=6 total → 6 matched, limit_pruned_row_groups=0 total → 0 matched, bytes_scanned=521, metadata_load_time=, scan_efficiency_ratio=22% (521/2.35 K)] statement ok drop table tracking_data; diff --git a/datafusion/sqllogictest/test_files/listing_table_statistics.slt b/datafusion/sqllogictest/test_files/listing_table_statistics.slt index 4298320d4aaba..8c301e2d2c770 100644 --- a/datafusion/sqllogictest/test_files/listing_table_statistics.slt +++ b/datafusion/sqllogictest/test_files/listing_table_statistics.slt @@ -35,7 +35,7 @@ query TT explain format indent select * from t; ---- logical_plan TableScan: t projection=[int_col, str_col] -physical_plan DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/listing_table_statistics/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/listing_table_statistics/2.parquet]]}, projection=[int_col, str_col], file_type=parquet, statistics=[Rows=Exact(4), Bytes=Absent, [(Col[0]: Min=Exact(Int64(-1)) Max=Exact(Int64(3)) Null=Exact(0) ScanBytes=Exact(32)),(Col[1]: Min=Exact(Utf8View("a")) Max=Exact(Utf8View("d")) Null=Exact(0) ScanBytes=Inexact(100))]] +physical_plan DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/listing_table_statistics/1.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/listing_table_statistics/2.parquet], projection=[int_col, str_col], file_type=parquet, statistics=[Rows=Exact(4), Bytes=Absent, [(Col[0]: Min=Exact(Int64(-1)) Max=Exact(Int64(3)) Null=Exact(0) ScanBytes=Exact(32)),(Col[1]: Min=Exact(Utf8View("a")) Max=Exact(Utf8View("d")) Null=Exact(0) ScanBytes=Inexact(100))]] statement ok drop table t; diff --git a/datafusion/sqllogictest/test_files/parquet.slt b/datafusion/sqllogictest/test_files/parquet.slt index be713b963b451..e39870a56001c 100644 --- a/datafusion/sqllogictest/test_files/parquet.slt +++ b/datafusion/sqllogictest/test_files/parquet.slt @@ -95,7 +95,7 @@ logical_plan physical_plan 01)SortPreservingMergeExec: [string_col@1 ASC NULLS LAST, int_col@0 ASC NULLS LAST] 02)--SortExec: expr=[string_col@1 ASC NULLS LAST, int_col@0 ASC NULLS LAST], preserve_partitioning=[true] -03)----DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/test_table/0.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/test_table/1.parquet]]}, projection=[int_col, string_col], file_type=parquet +03)----DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/test_table/0.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/test_table/1.parquet], projection=[int_col, string_col], file_type=parquet # Tear down test_table: statement ok @@ -459,8 +459,7 @@ logical_plan 02)--TableScan: binary_as_string_default projection=[binary_col, largebinary_col, binaryview_col], partial_filters=[CAST(binary_as_string_default.binary_col AS Utf8View) LIKE Utf8View("%a%"), CAST(binary_as_string_default.largebinary_col AS Utf8View) LIKE Utf8View("%a%"), CAST(binary_as_string_default.binaryview_col AS Utf8View) LIKE Utf8View("%a%")] physical_plan 01)FilterExec: CAST(binary_col@0 AS Utf8View) LIKE %a% AND CAST(largebinary_col@1 AS Utf8View) LIKE %a% AND CAST(binaryview_col@2 AS Utf8View) LIKE %a% -02)--RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/binary_as_string.parquet]]}, projection=[binary_col, largebinary_col, binaryview_col], file_type=parquet, predicate=CAST(binary_col@0 AS Utf8View) LIKE %a% AND CAST(largebinary_col@1 AS Utf8View) LIKE %a% AND CAST(binaryview_col@2 AS Utf8View) LIKE %a% +02)--DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/binary_as_string.parquet], projection=[binary_col, largebinary_col, binaryview_col], file_type=parquet, predicate=CAST(binary_col@0 AS Utf8View) LIKE %a% AND CAST(largebinary_col@1 AS Utf8View) LIKE %a% AND CAST(binaryview_col@2 AS Utf8View) LIKE %a% statement ok @@ -506,8 +505,7 @@ logical_plan 02)--TableScan: binary_as_string_option projection=[binary_col, largebinary_col, binaryview_col], partial_filters=[binary_as_string_option.binary_col LIKE Utf8View("%a%"), binary_as_string_option.largebinary_col LIKE Utf8View("%a%"), binary_as_string_option.binaryview_col LIKE Utf8View("%a%")] physical_plan 01)FilterExec: binary_col@0 LIKE %a% AND largebinary_col@1 LIKE %a% AND binaryview_col@2 LIKE %a% -02)--RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/binary_as_string.parquet]]}, projection=[binary_col, largebinary_col, binaryview_col], file_type=parquet, predicate=binary_col@0 LIKE %a% AND largebinary_col@1 LIKE %a% AND binaryview_col@2 LIKE %a% +02)--DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/binary_as_string.parquet], projection=[binary_col, largebinary_col, binaryview_col], file_type=parquet, predicate=binary_col@0 LIKE %a% AND largebinary_col@1 LIKE %a% AND binaryview_col@2 LIKE %a% statement ok @@ -556,8 +554,7 @@ logical_plan 02)--TableScan: binary_as_string_both projection=[binary_col, largebinary_col, binaryview_col], partial_filters=[binary_as_string_both.binary_col LIKE Utf8View("%a%"), binary_as_string_both.largebinary_col LIKE Utf8View("%a%"), binary_as_string_both.binaryview_col LIKE Utf8View("%a%")] physical_plan 01)FilterExec: binary_col@0 LIKE %a% AND largebinary_col@1 LIKE %a% AND binaryview_col@2 LIKE %a% -02)--RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/binary_as_string.parquet]]}, projection=[binary_col, largebinary_col, binaryview_col], file_type=parquet, predicate=binary_col@0 LIKE %a% AND largebinary_col@1 LIKE %a% AND binaryview_col@2 LIKE %a% +02)--DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/binary_as_string.parquet], projection=[binary_col, largebinary_col, binaryview_col], file_type=parquet, predicate=binary_col@0 LIKE %a% AND largebinary_col@1 LIKE %a% AND binaryview_col@2 LIKE %a% statement ok @@ -670,8 +667,7 @@ logical_plan 02)--TableScan: foo projection=[column1], partial_filters=[foo.column1 LIKE Utf8View("f%")] physical_plan 01)FilterExec: column1@0 LIKE f% -02)--RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/foo.parquet]]}, projection=[column1], file_type=parquet, predicate=column1@0 LIKE f%, pruning_predicate=column1_null_count@2 != row_count@3 AND column1_min@0 <= g AND f <= column1_max@1, required_guarantees=[] +02)--DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/foo.parquet], projection=[column1], file_type=parquet, predicate=column1@0 LIKE f%, pruning_predicate=column1_null_count@2 != row_count@3 AND column1_min@0 <= g AND f <= column1_max@1, required_guarantees=[] statement ok drop table foo diff --git a/datafusion/sqllogictest/test_files/parquet_filter_pushdown.slt b/datafusion/sqllogictest/test_files/parquet_filter_pushdown.slt index e2473ee328e51..e482fdaa06075 100644 --- a/datafusion/sqllogictest/test_files/parquet_filter_pushdown.slt +++ b/datafusion/sqllogictest/test_files/parquet_filter_pushdown.slt @@ -96,8 +96,7 @@ physical_plan 01)SortPreservingMergeExec: [a@0 ASC NULLS LAST] 02)--SortExec: expr=[a@0 ASC NULLS LAST], preserve_partitioning=[true] 03)----FilterExec: b@1 > 2, projection=[a@0] -04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=2 -05)--------DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet]]}, projection=[a, b], file_type=parquet, predicate=b@1 > 2, pruning_predicate=b_null_count@1 != row_count@2 AND b_max@0 > 2, required_guarantees=[] +04)------DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet], projection=[a, b], file_type=parquet, predicate=b@1 > 2, pruning_predicate=b_null_count@1 != row_count@2 AND b_max@0 > 2, required_guarantees=[] query TT EXPLAIN select a from t_pushdown where b > 2 ORDER BY a; @@ -110,7 +109,7 @@ logical_plan physical_plan 01)SortPreservingMergeExec: [a@0 ASC NULLS LAST] 02)--SortExec: expr=[a@0 ASC NULLS LAST], preserve_partitioning=[true] -03)----DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet]]}, projection=[a], file_type=parquet, predicate=b@1 > 2, pruning_predicate=b_null_count@1 != row_count@2 AND b_max@0 > 2, required_guarantees=[] +03)----DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet], projection=[a], file_type=parquet, predicate=b@1 > 2, pruning_predicate=b_null_count@1 != row_count@2 AND b_max@0 > 2, required_guarantees=[] query T select a from t where b = 2 ORDER BY b; @@ -133,8 +132,7 @@ logical_plan physical_plan 01)CoalescePartitionsExec 02)--FilterExec: b@1 = 2, projection=[a@0] -03)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=2 -04)------DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet]]}, projection=[a, b], file_type=parquet, predicate=b@1 = 2, pruning_predicate=b_null_count@2 != row_count@3 AND b_min@0 <= 2 AND 2 <= b_max@1, required_guarantees=[b in (2)] +03)----DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet], projection=[a, b], file_type=parquet, predicate=b@1 = 2, pruning_predicate=b_null_count@2 != row_count@3 AND b_min@0 <= 2 AND 2 <= b_max@1, required_guarantees=[b in (2)] query TT EXPLAIN select a from t_pushdown where b = 2 ORDER BY b; @@ -146,7 +144,7 @@ logical_plan 04)------TableScan: t_pushdown projection=[a, b], partial_filters=[t_pushdown.b = Int32(2)] physical_plan 01)CoalescePartitionsExec -02)--DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet]]}, projection=[a], file_type=parquet, predicate=b@1 = 2, pruning_predicate=b_null_count@2 != row_count@3 AND b_min@0 <= 2 AND 2 <= b_max@1, required_guarantees=[b in (2)] +02)--DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet], projection=[a], file_type=parquet, predicate=b@1 = 2, pruning_predicate=b_null_count@2 != row_count@3 AND b_min@0 <= 2 AND 2 <= b_max@1, required_guarantees=[b in (2)] # If we set the setting to `true` it override's the table's setting statement ok @@ -181,7 +179,7 @@ logical_plan physical_plan 01)SortPreservingMergeExec: [a@0 ASC NULLS LAST] 02)--SortExec: expr=[a@0 ASC NULLS LAST], preserve_partitioning=[true] -03)----DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet]]}, projection=[a], file_type=parquet, predicate=b@1 > 2, pruning_predicate=b_null_count@1 != row_count@2 AND b_max@0 > 2, required_guarantees=[] +03)----DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet], projection=[a], file_type=parquet, predicate=b@1 > 2, pruning_predicate=b_null_count@1 != row_count@2 AND b_max@0 > 2, required_guarantees=[] query TT EXPLAIN select a from t_pushdown where b > 2 ORDER BY a; @@ -194,7 +192,7 @@ logical_plan physical_plan 01)SortPreservingMergeExec: [a@0 ASC NULLS LAST] 02)--SortExec: expr=[a@0 ASC NULLS LAST], preserve_partitioning=[true] -03)----DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet]]}, projection=[a], file_type=parquet, predicate=b@1 > 2, pruning_predicate=b_null_count@1 != row_count@2 AND b_max@0 > 2, required_guarantees=[] +03)----DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet], projection=[a], file_type=parquet, predicate=b@1 > 2, pruning_predicate=b_null_count@1 != row_count@2 AND b_max@0 > 2, required_guarantees=[] query T select a from t where b = 2 ORDER BY b; @@ -216,7 +214,7 @@ logical_plan 04)------TableScan: t projection=[a, b], partial_filters=[t.b = Int32(2)] physical_plan 01)CoalescePartitionsExec -02)--DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet]]}, projection=[a], file_type=parquet, predicate=b@1 = 2, pruning_predicate=b_null_count@2 != row_count@3 AND b_min@0 <= 2 AND 2 <= b_max@1, required_guarantees=[b in (2)] +02)--DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet], projection=[a], file_type=parquet, predicate=b@1 = 2, pruning_predicate=b_null_count@2 != row_count@3 AND b_min@0 <= 2 AND 2 <= b_max@1, required_guarantees=[b in (2)] query TT EXPLAIN select a from t_pushdown where b = 2 ORDER BY b; @@ -228,7 +226,7 @@ logical_plan 04)------TableScan: t_pushdown projection=[a, b], partial_filters=[t_pushdown.b = Int32(2)] physical_plan 01)CoalescePartitionsExec -02)--DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet]]}, projection=[a], file_type=parquet, predicate=b@1 = 2, pruning_predicate=b_null_count@2 != row_count@3 AND b_min@0 <= 2 AND 2 <= b_max@1, required_guarantees=[b in (2)] +02)--DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet], projection=[a], file_type=parquet, predicate=b@1 = 2, pruning_predicate=b_null_count@2 != row_count@3 AND b_min@0 <= 2 AND 2 <= b_max@1, required_guarantees=[b in (2)] # If we reset the default the table created without pushdown goes back to disabling it statement ok @@ -264,8 +262,7 @@ physical_plan 01)SortPreservingMergeExec: [a@0 ASC NULLS LAST] 02)--SortExec: expr=[a@0 ASC NULLS LAST], preserve_partitioning=[true] 03)----FilterExec: b@1 > 2, projection=[a@0] -04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=2 -05)--------DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet]]}, projection=[a, b], file_type=parquet, predicate=b@1 > 2, pruning_predicate=b_null_count@1 != row_count@2 AND b_max@0 > 2, required_guarantees=[] +04)------DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet], projection=[a, b], file_type=parquet, predicate=b@1 > 2, pruning_predicate=b_null_count@1 != row_count@2 AND b_max@0 > 2, required_guarantees=[] query TT EXPLAIN select a from t_pushdown where b > 2 ORDER BY a; @@ -278,7 +275,7 @@ logical_plan physical_plan 01)SortPreservingMergeExec: [a@0 ASC NULLS LAST] 02)--SortExec: expr=[a@0 ASC NULLS LAST], preserve_partitioning=[true] -03)----DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet]]}, projection=[a], file_type=parquet, predicate=b@1 > 2, pruning_predicate=b_null_count@1 != row_count@2 AND b_max@0 > 2, required_guarantees=[] +03)----DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet], projection=[a], file_type=parquet, predicate=b@1 > 2, pruning_predicate=b_null_count@1 != row_count@2 AND b_max@0 > 2, required_guarantees=[] query T select a from t where b = 2 ORDER BY b; @@ -301,8 +298,7 @@ logical_plan physical_plan 01)CoalescePartitionsExec 02)--FilterExec: b@1 = 2, projection=[a@0] -03)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=2 -04)------DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet]]}, projection=[a, b], file_type=parquet, predicate=b@1 = 2, pruning_predicate=b_null_count@2 != row_count@3 AND b_min@0 <= 2 AND 2 <= b_max@1, required_guarantees=[b in (2)] +03)----DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet], projection=[a, b], file_type=parquet, predicate=b@1 = 2, pruning_predicate=b_null_count@2 != row_count@3 AND b_min@0 <= 2 AND 2 <= b_max@1, required_guarantees=[b in (2)] query TT EXPLAIN select a from t_pushdown where b = 2 ORDER BY b; @@ -314,7 +310,7 @@ logical_plan 04)------TableScan: t_pushdown projection=[a, b], partial_filters=[t_pushdown.b = Int32(2)] physical_plan 01)CoalescePartitionsExec -02)--DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet]]}, projection=[a], file_type=parquet, predicate=b@1 = 2, pruning_predicate=b_null_count@2 != row_count@3 AND b_min@0 <= 2 AND 2 <= b_max@1, required_guarantees=[b in (2)] +02)--DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet], projection=[a], file_type=parquet, predicate=b@1 = 2, pruning_predicate=b_null_count@2 != row_count@3 AND b_min@0 <= 2 AND 2 <= b_max@1, required_guarantees=[b in (2)] # When filter pushdown *is* enabled, ParquetExec can filter exactly, # not just metadata, so we expect to see no FilterExec @@ -339,8 +335,7 @@ physical_plan 01)SortPreservingMergeExec: [a@0 ASC NULLS LAST] 02)--SortExec: expr=[a@0 ASC NULLS LAST], preserve_partitioning=[true] 03)----FilterExec: b@1 > 2, projection=[a@0] -04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=2 -05)--------DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet]]}, projection=[a, b], file_type=parquet, predicate=b@1 > 2, pruning_predicate=b_null_count@1 != row_count@2 AND b_max@0 > 2, required_guarantees=[] +04)------DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet], projection=[a, b], file_type=parquet, predicate=b@1 > 2, pruning_predicate=b_null_count@1 != row_count@2 AND b_max@0 > 2, required_guarantees=[] query T select a from t_pushdown where b = 2 ORDER BY b; @@ -357,7 +352,7 @@ logical_plan 04)------TableScan: t_pushdown projection=[a, b], partial_filters=[t_pushdown.b = Int32(2)] physical_plan 01)CoalescePartitionsExec -02)--DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet]]}, projection=[a], file_type=parquet, predicate=b@1 = 2, pruning_predicate=b_null_count@2 != row_count@3 AND b_min@0 <= 2 AND 2 <= b_max@1, required_guarantees=[b in (2)] +02)--DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet], projection=[a], file_type=parquet, predicate=b@1 = 2, pruning_predicate=b_null_count@2 != row_count@3 AND b_min@0 <= 2 AND 2 <= b_max@1, required_guarantees=[b in (2)] # also test querying on columns that are not in all the files query T @@ -377,7 +372,7 @@ logical_plan physical_plan 01)SortPreservingMergeExec: [a@0 ASC NULLS LAST] 02)--SortExec: expr=[a@0 ASC NULLS LAST], preserve_partitioning=[true] -03)----DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet]]}, projection=[a], file_type=parquet, predicate=b@1 > 2 AND a@0 IS NOT NULL, pruning_predicate=b_null_count@1 != row_count@2 AND b_max@0 > 2 AND a_null_count@3 != row_count@2, required_guarantees=[] +03)----DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet], projection=[a], file_type=parquet, predicate=b@1 > 2 AND a@0 IS NOT NULL, pruning_predicate=b_null_count@1 != row_count@2 AND b_max@0 > 2 AND a_null_count@3 != row_count@2, required_guarantees=[] query I @@ -396,7 +391,7 @@ logical_plan physical_plan 01)SortPreservingMergeExec: [b@0 ASC NULLS LAST] 02)--SortExec: expr=[b@0 ASC NULLS LAST], preserve_partitioning=[true] -03)----DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet]]}, projection=[b], file_type=parquet, predicate=a@0 = bar, pruning_predicate=a_null_count@2 != row_count@3 AND a_min@0 <= bar AND bar <= a_max@1, required_guarantees=[a in (bar)] +03)----DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet], projection=[b], file_type=parquet, predicate=a@0 = bar, pruning_predicate=a_null_count@2 != row_count@3 AND a_min@0 <= bar AND bar <= a_max@1, required_guarantees=[a in (bar)] # should not push down volatile predicates such as RANDOM @@ -410,7 +405,7 @@ logical_plan 03)----TableScan: t_pushdown projection=[a, b] physical_plan 01)FilterExec: CAST(b@1 AS Float64) > random(), projection=[a@0] -02)--DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet]]}, projection=[a, b], file_type=parquet +02)--DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet], projection=[a, b], file_type=parquet ## cleanup statement ok @@ -466,14 +461,14 @@ EXPLAIN select * from t_pushdown where part != val logical_plan 01)Filter: t_pushdown.val != t_pushdown.part 02)--TableScan: t_pushdown projection=[val, part], partial_filters=[t_pushdown.val != t_pushdown.part] -physical_plan DataSourceExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_part_test/part=a/file.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_part_test/part=b/file.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_part_test/part=c/file.parquet]]}, projection=[val, part], file_type=parquet, predicate=val@0 != part@1 +physical_plan DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_part_test/part=a/file.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_part_test/part=b/file.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_part_test/part=c/file.parquet], projection=[val, part], file_type=parquet, predicate=val@0 != part@1 # If we reference only a partition column it gets evaluated during the listing phase query TT EXPLAIN select * from t_pushdown where part != 'a'; ---- logical_plan TableScan: t_pushdown projection=[val, part], full_filters=[t_pushdown.part != Utf8View("a")] -physical_plan DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_part_test/part=b/file.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_part_test/part=c/file.parquet]]}, projection=[val, part], file_type=parquet +physical_plan DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_part_test/part=b/file.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_part_test/part=c/file.parquet], projection=[val, part], file_type=parquet # And if we reference only a file column it gets pushed down query TT @@ -482,7 +477,7 @@ EXPLAIN select * from t_pushdown where val != 'c'; logical_plan 01)Filter: t_pushdown.val != Utf8View("c") 02)--TableScan: t_pushdown projection=[val, part], partial_filters=[t_pushdown.val != Utf8View("c")] -physical_plan DataSourceExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_part_test/part=a/file.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_part_test/part=b/file.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_part_test/part=c/file.parquet]]}, projection=[val, part], file_type=parquet, predicate=val@0 != c, pruning_predicate=val_null_count@2 != row_count@3 AND (val_min@0 != c OR c != val_max@1), required_guarantees=[val not in (c)] +physical_plan DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_part_test/part=a/file.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_part_test/part=b/file.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_part_test/part=c/file.parquet], projection=[val, part], file_type=parquet, predicate=val@0 != c, pruning_predicate=val_null_count@2 != row_count@3 AND (val_min@0 != c OR c != val_max@1), required_guarantees=[val not in (c)] # If we have a mix of filters: # - The partition filters get evaluated during planning @@ -494,7 +489,7 @@ EXPLAIN select * from t_pushdown where val != 'd' AND val != 'c' AND part = 'a' logical_plan 01)Filter: t_pushdown.val != Utf8View("d") AND t_pushdown.val != Utf8View("c") AND t_pushdown.val != t_pushdown.part 02)--TableScan: t_pushdown projection=[val, part], full_filters=[t_pushdown.part = Utf8View("a")], partial_filters=[t_pushdown.val != Utf8View("d"), t_pushdown.val != Utf8View("c"), t_pushdown.val != t_pushdown.part] -physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_part_test/part=a/file.parquet]]}, projection=[val, part], file_type=parquet, predicate=val@0 != d AND val@0 != c AND val@0 != part@1, pruning_predicate=val_null_count@2 != row_count@3 AND (val_min@0 != d OR d != val_max@1) AND val_null_count@2 != row_count@3 AND (val_min@0 != c OR c != val_max@1), required_guarantees=[val not in (c, d)] +physical_plan DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_part_test/part=a/file.parquet], projection=[val, part], file_type=parquet, predicate=val@0 != d AND val@0 != c AND val@0 != part@1, pruning_predicate=val_null_count@2 != row_count@3 AND (val_min@0 != d OR d != val_max@1) AND val_null_count@2 != row_count@3 AND (val_min@0 != c OR c != val_max@1), required_guarantees=[val not in (c, d)] # The order of filters should not matter query TT @@ -503,7 +498,7 @@ EXPLAIN select val, part from t_pushdown where part = 'a' AND part = val; logical_plan 01)Filter: t_pushdown.val = t_pushdown.part 02)--TableScan: t_pushdown projection=[val, part], full_filters=[t_pushdown.part = Utf8View("a")], partial_filters=[t_pushdown.val = t_pushdown.part] -physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_part_test/part=a/file.parquet]]}, projection=[val, part], file_type=parquet, predicate=val@0 = part@1 +physical_plan DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_part_test/part=a/file.parquet], projection=[val, part], file_type=parquet, predicate=val@0 = part@1 query TT select val, part from t_pushdown where part = 'a' AND part = val; @@ -516,7 +511,7 @@ EXPLAIN select val, part from t_pushdown where part = val AND part = 'a'; logical_plan 01)Filter: t_pushdown.val = t_pushdown.part 02)--TableScan: t_pushdown projection=[val, part], full_filters=[t_pushdown.part = Utf8View("a")], partial_filters=[t_pushdown.val = t_pushdown.part] -physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_part_test/part=a/file.parquet]]}, projection=[val, part], file_type=parquet, predicate=val@0 = part@1 +physical_plan DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_part_test/part=a/file.parquet], projection=[val, part], file_type=parquet, predicate=val@0 = part@1 query TT select val, part from t_pushdown where part = val AND part = 'a'; @@ -602,8 +597,9 @@ logical_plan 02)--Filter: array_has(array_test.tags, Utf8("rust")) 03)----TableScan: array_test projection=[id, tags], partial_filters=[array_has(array_test.tags, Utf8("rust"))] physical_plan -01)SortExec: expr=[id@0 ASC NULLS LAST], preserve_partitioning=[false] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/array_data/data.parquet]]}, projection=[id, tags], file_type=parquet, predicate=array_has(tags@1, rust) +01)SortPreservingMergeExec: [id@0 ASC NULLS LAST] +02)--SortExec: expr=[id@0 ASC NULLS LAST], preserve_partitioning=[true] +03)----DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/array_data/data.parquet], projection=[id, tags], file_type=parquet, predicate=array_has(tags@1, rust) # Test array_has_all predicate pushdown query I? @@ -619,8 +615,9 @@ logical_plan 02)--Filter: array_has_all(array_test.tags, List([rust, performance])) 03)----TableScan: array_test projection=[id, tags], partial_filters=[array_has_all(array_test.tags, List([rust, performance]))] physical_plan -01)SortExec: expr=[id@0 ASC NULLS LAST], preserve_partitioning=[false] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/array_data/data.parquet]]}, projection=[id, tags], file_type=parquet, predicate=array_has_all(tags@1, [rust, performance]) +01)SortPreservingMergeExec: [id@0 ASC NULLS LAST] +02)--SortExec: expr=[id@0 ASC NULLS LAST], preserve_partitioning=[true] +03)----DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/array_data/data.parquet], projection=[id, tags], file_type=parquet, predicate=array_has_all(tags@1, [rust, performance]) # Test array_has_any predicate pushdown query I? @@ -636,8 +633,9 @@ logical_plan 02)--Filter: array_has_any(array_test.tags, List([python, go])) 03)----TableScan: array_test projection=[id, tags], partial_filters=[array_has_any(array_test.tags, List([python, go]))] physical_plan -01)SortExec: expr=[id@0 ASC NULLS LAST], preserve_partitioning=[false] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/array_data/data.parquet]]}, projection=[id, tags], file_type=parquet, predicate=array_has_any(tags@1, [python, go]) +01)SortPreservingMergeExec: [id@0 ASC NULLS LAST] +02)--SortExec: expr=[id@0 ASC NULLS LAST], preserve_partitioning=[true] +03)----DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/array_data/data.parquet], projection=[id, tags], file_type=parquet, predicate=array_has_any(tags@1, [python, go]) # Test complex predicate with OR query I? @@ -655,8 +653,9 @@ logical_plan 02)--Filter: array_has_all(array_test.tags, List([rust])) OR array_has_any(array_test.tags, List([python, go])) 03)----TableScan: array_test projection=[id, tags], partial_filters=[array_has_all(array_test.tags, List([rust])) OR array_has_any(array_test.tags, List([python, go]))] physical_plan -01)SortExec: expr=[id@0 ASC NULLS LAST], preserve_partitioning=[false] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/array_data/data.parquet]]}, projection=[id, tags], file_type=parquet, predicate=array_has_all(tags@1, [rust]) OR array_has_any(tags@1, [python, go]) +01)SortPreservingMergeExec: [id@0 ASC NULLS LAST] +02)--SortExec: expr=[id@0 ASC NULLS LAST], preserve_partitioning=[true] +03)----DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/array_data/data.parquet], projection=[id, tags], file_type=parquet, predicate=array_has_all(tags@1, [rust]) OR array_has_any(tags@1, [python, go]) # Test array function with other predicates query I? @@ -672,8 +671,9 @@ logical_plan 02)--Filter: array_test.id > Int64(1) AND array_has(array_test.tags, Utf8("rust")) 03)----TableScan: array_test projection=[id, tags], partial_filters=[array_test.id > Int64(1), array_has(array_test.tags, Utf8("rust"))] physical_plan -01)SortExec: expr=[id@0 ASC NULLS LAST], preserve_partitioning=[false] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/array_data/data.parquet]]}, projection=[id, tags], file_type=parquet, predicate=id@0 > 1 AND array_has(tags@1, rust), pruning_predicate=id_null_count@1 != row_count@2 AND id_max@0 > 1, required_guarantees=[] +01)SortPreservingMergeExec: [id@0 ASC NULLS LAST] +02)--SortExec: expr=[id@0 ASC NULLS LAST], preserve_partitioning=[true] +03)----DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/array_data/data.parquet], projection=[id, tags], file_type=parquet, predicate=id@0 > 1 AND array_has(tags@1, rust), pruning_predicate=id_null_count@1 != row_count@2 AND id_max@0 > 1, required_guarantees=[] ### # Test filter pushdown through UNION with mixed support @@ -729,7 +729,7 @@ physical_plan 01)UnionExec 02)--FilterExec: b@0 > 2 03)----DataSourceExec: partitions=1, partition_sizes=[1] -04)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet]]}, projection=[b], file_type=parquet, predicate=b@1 > 2, pruning_predicate=b_null_count@1 != row_count@2 AND b_max@0 > 2, required_guarantees=[] +04)--DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet], projection=[b], file_type=parquet, predicate=b@1 > 2, pruning_predicate=b_null_count@1 != row_count@2 AND b_max@0 > 2, required_guarantees=[] # Clean up union test tables statement ok diff --git a/datafusion/sqllogictest/test_files/parquet_statistics.slt b/datafusion/sqllogictest/test_files/parquet_statistics.slt index 8c77fb96ba75c..b90086f8aac32 100644 --- a/datafusion/sqllogictest/test_files/parquet_statistics.slt +++ b/datafusion/sqllogictest/test_files/parquet_statistics.slt @@ -60,8 +60,7 @@ EXPLAIN SELECT * FROM test_table WHERE column1 = 1; ---- physical_plan 01)FilterExec: column1@0 = 1, statistics=[Rows=Inexact(2), Bytes=Inexact(10), [(Col[0]: Min=Exact(Int64(1)) Max=Exact(Int64(1)) Null=Inexact(0) ScanBytes=Inexact(40))]] -02)--RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=2, statistics=[Rows=Inexact(5), Bytes=Inexact(40), [(Col[0]: Min=Inexact(Int64(1)) Max=Inexact(Int64(4)) Null=Inexact(0) ScanBytes=Inexact(40))]] -03)----DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_statistics/test_table/0.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_statistics/test_table/1.parquet]]}, projection=[column1], file_type=parquet, predicate=column1@0 = 1, pruning_predicate=column1_null_count@2 != row_count@3 AND column1_min@0 <= 1 AND 1 <= column1_max@1, required_guarantees=[column1 in (1)], statistics=[Rows=Inexact(5), Bytes=Inexact(40), [(Col[0]: Min=Inexact(Int64(1)) Max=Inexact(Int64(4)) Null=Inexact(0) ScanBytes=Inexact(40))]] +02)--DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_statistics/test_table/0.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_statistics/test_table/1.parquet], projection=[column1], file_type=parquet, predicate=column1@0 = 1, pruning_predicate=column1_null_count@2 != row_count@3 AND column1_min@0 <= 1 AND 1 <= column1_max@1, required_guarantees=[column1 in (1)], statistics=[Rows=Inexact(5), Bytes=Inexact(40), [(Col[0]: Min=Inexact(Int64(1)) Max=Inexact(Int64(4)) Null=Inexact(0) ScanBytes=Inexact(40))]] # cleanup statement ok @@ -85,8 +84,7 @@ EXPLAIN SELECT * FROM test_table WHERE column1 = 1; ---- physical_plan 01)FilterExec: column1@0 = 1, statistics=[Rows=Inexact(2), Bytes=Inexact(10), [(Col[0]: Min=Exact(Int64(1)) Max=Exact(Int64(1)) Null=Inexact(0) ScanBytes=Inexact(40))]] -02)--RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=2, statistics=[Rows=Inexact(5), Bytes=Inexact(40), [(Col[0]: Min=Inexact(Int64(1)) Max=Inexact(Int64(4)) Null=Inexact(0) ScanBytes=Inexact(40))]] -03)----DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_statistics/test_table/0.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_statistics/test_table/1.parquet]]}, projection=[column1], file_type=parquet, predicate=column1@0 = 1, pruning_predicate=column1_null_count@2 != row_count@3 AND column1_min@0 <= 1 AND 1 <= column1_max@1, required_guarantees=[column1 in (1)], statistics=[Rows=Inexact(5), Bytes=Inexact(40), [(Col[0]: Min=Inexact(Int64(1)) Max=Inexact(Int64(4)) Null=Inexact(0) ScanBytes=Inexact(40))]] +02)--DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_statistics/test_table/0.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_statistics/test_table/1.parquet], projection=[column1], file_type=parquet, predicate=column1@0 = 1, pruning_predicate=column1_null_count@2 != row_count@3 AND column1_min@0 <= 1 AND 1 <= column1_max@1, required_guarantees=[column1 in (1)], statistics=[Rows=Inexact(5), Bytes=Inexact(40), [(Col[0]: Min=Inexact(Int64(1)) Max=Inexact(Int64(4)) Null=Inexact(0) ScanBytes=Inexact(40))]] # cleanup statement ok @@ -111,8 +109,7 @@ EXPLAIN SELECT * FROM test_table WHERE column1 = 1; ---- physical_plan 01)FilterExec: column1@0 = 1, statistics=[Rows=Absent, Bytes=Absent, [(Col[0]: Min=Exact(Int64(1)) Max=Exact(Int64(1)))]] -02)--RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=2, statistics=[Rows=Absent, Bytes=Absent, [(Col[0]:)]] -03)----DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_statistics/test_table/0.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_statistics/test_table/1.parquet]]}, projection=[column1], file_type=parquet, predicate=column1@0 = 1, pruning_predicate=column1_null_count@2 != row_count@3 AND column1_min@0 <= 1 AND 1 <= column1_max@1, required_guarantees=[column1 in (1)], statistics=[Rows=Absent, Bytes=Absent, [(Col[0]:)]] +02)--DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_statistics/test_table/0.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_statistics/test_table/1.parquet], projection=[column1], file_type=parquet, predicate=column1@0 = 1, pruning_predicate=column1_null_count@2 != row_count@3 AND column1_min@0 <= 1 AND 1 <= column1_max@1, required_guarantees=[column1 in (1)], statistics=[Rows=Absent, Bytes=Absent, [(Col[0]:)]] # cleanup statement ok diff --git a/datafusion/sqllogictest/test_files/preserve_file_partitioning.slt b/datafusion/sqllogictest/test_files/preserve_file_partitioning.slt index 297094fab16e7..6b706fdd6555f 100644 --- a/datafusion/sqllogictest/test_files/preserve_file_partitioning.slt +++ b/datafusion/sqllogictest/test_files/preserve_file_partitioning.slt @@ -230,7 +230,7 @@ physical_plan 02)--AggregateExec: mode=FinalPartitioned, gby=[f_dkey@0 as f_dkey], aggr=[count(Int64(1)), sum(fact_table.value)] 03)----RepartitionExec: partitioning=Hash([f_dkey@0], 3), input_partitions=3 04)------AggregateExec: mode=Partial, gby=[f_dkey@1 as f_dkey], aggr=[count(Int64(1)), sum(fact_table.value)] -05)--------DataSourceExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/fact/f_dkey=A/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/fact/f_dkey=B/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/fact/f_dkey=C/data.parquet]]}, projection=[value, f_dkey], file_type=parquet +05)--------DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/fact/f_dkey=A/data.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/fact/f_dkey=B/data.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/fact/f_dkey=C/data.parquet], projection=[value, f_dkey], file_type=parquet # Verify results without optimization query TIR rowsort @@ -368,9 +368,8 @@ physical_plan 08)--------------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(d_dkey@0, f_dkey@1)], projection=[env@1, service@2, value@3, f_dkey@4] 09)----------------CoalescePartitionsExec 10)------------------FilterExec: service@2 = log -11)--------------------RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1 -12)----------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/dimension/data.parquet]]}, projection=[d_dkey, env, service], file_type=parquet, predicate=service@2 = log, pruning_predicate=service_null_count@2 != row_count@3 AND service_min@0 <= log AND log <= service_max@1, required_guarantees=[service in (log)] -13)----------------DataSourceExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/fact/f_dkey=A/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/fact/f_dkey=B/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/fact/f_dkey=C/data.parquet]]}, projection=[value, f_dkey], output_ordering=[f_dkey@1 ASC NULLS LAST], file_type=parquet, predicate=DynamicFilter [ empty ] +11)--------------------DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/dimension/data.parquet], projection=[d_dkey, env, service], file_type=parquet, predicate=service@2 = log, pruning_predicate=service_null_count@2 != row_count@3 AND service_min@0 <= log AND log <= service_max@1, required_guarantees=[service in (log)] +12)----------------DataSourceExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/fact/f_dkey=A/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/fact/f_dkey=B/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/fact/f_dkey=C/data.parquet]]}, projection=[value, f_dkey], output_ordering=[f_dkey@1 ASC NULLS LAST], file_type=parquet, predicate=DynamicFilter [ empty ] # Verify results without optimization query TTTIR rowsort @@ -420,9 +419,8 @@ physical_plan 05)--------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(d_dkey@0, f_dkey@1)], projection=[env@1, service@2, value@3, f_dkey@4] 06)----------CoalescePartitionsExec 07)------------FilterExec: service@2 = log -08)--------------RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1 -09)----------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/dimension/data.parquet]]}, projection=[d_dkey, env, service], file_type=parquet, predicate=service@2 = log, pruning_predicate=service_null_count@2 != row_count@3 AND service_min@0 <= log AND log <= service_max@1, required_guarantees=[service in (log)] -10)----------DataSourceExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/fact/f_dkey=A/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/fact/f_dkey=B/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/fact/f_dkey=C/data.parquet]]}, projection=[value, f_dkey], output_ordering=[f_dkey@1 ASC NULLS LAST], file_type=parquet, predicate=DynamicFilter [ empty ] +08)--------------DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/dimension/data.parquet], projection=[d_dkey, env, service], file_type=parquet, predicate=service@2 = log, pruning_predicate=service_null_count@2 != row_count@3 AND service_min@0 <= log AND log <= service_max@1, required_guarantees=[service in (log)] +09)----------DataSourceExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/fact/f_dkey=A/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/fact/f_dkey=B/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/fact/f_dkey=C/data.parquet]]}, projection=[value, f_dkey], output_ordering=[f_dkey@1 ASC NULLS LAST], file_type=parquet, predicate=DynamicFilter [ empty ] query TTTIR rowsort SELECT f.f_dkey, MAX(d.env), MAX(d.service), count(*), sum(f.value) @@ -596,7 +594,7 @@ physical_plan 02)--AggregateExec: mode=FinalPartitioned, gby=[f_dkey@0 as f_dkey], aggr=[count(Int64(1)), sum(high_cardinality_table.value)] 03)----RepartitionExec: partitioning=Hash([f_dkey@0], 3), input_partitions=3 04)------AggregateExec: mode=Partial, gby=[f_dkey@1 as f_dkey], aggr=[count(Int64(1)), sum(high_cardinality_table.value)] -05)--------DataSourceExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/high_cardinality/f_dkey=A/data.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/high_cardinality/f_dkey=B/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/high_cardinality/f_dkey=C/data.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/high_cardinality/f_dkey=D/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/high_cardinality/f_dkey=E/data.parquet]]}, projection=[value, f_dkey], file_type=parquet +05)--------DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/high_cardinality/f_dkey=A/data.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/high_cardinality/f_dkey=B/data.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/high_cardinality/f_dkey=C/data.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/high_cardinality/f_dkey=D/data.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/high_cardinality/f_dkey=E/data.parquet], projection=[value, f_dkey], file_type=parquet query TIR rowsort SELECT f_dkey, count(*), sum(value) @@ -646,9 +644,9 @@ physical_plan 04)------ProjectionExec: expr=[value@1 as value, f_dkey@2 as f_dkey, env@0 as env] 05)--------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(d_dkey@1, f_dkey@1)], projection=[env@0, value@2, f_dkey@3] 06)----------RepartitionExec: partitioning=Hash([d_dkey@1], 3), input_partitions=3 -07)------------DataSourceExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/dimension_partitioned/d_dkey=A/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/dimension_partitioned/d_dkey=B/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/dimension_partitioned/d_dkey=C/data.parquet]]}, projection=[env, d_dkey], file_type=parquet +07)------------DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/dimension_partitioned/d_dkey=A/data.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/dimension_partitioned/d_dkey=B/data.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/dimension_partitioned/d_dkey=C/data.parquet], projection=[env, d_dkey], file_type=parquet 08)----------RepartitionExec: partitioning=Hash([f_dkey@1], 3), input_partitions=3 -09)------------DataSourceExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/fact/f_dkey=A/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/fact/f_dkey=B/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/fact/f_dkey=C/data.parquet]]}, projection=[value, f_dkey], file_type=parquet, predicate=DynamicFilter [ empty ] +09)------------DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/fact/f_dkey=A/data.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/fact/f_dkey=B/data.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/preserve_file_partitioning/fact/f_dkey=C/data.parquet], projection=[value, f_dkey], file_type=parquet, predicate=DynamicFilter [ empty ] query TTR rowsort SELECT f.f_dkey, d.env, sum(f.value) diff --git a/datafusion/sqllogictest/test_files/projection.slt b/datafusion/sqllogictest/test_files/projection.slt index e18114bc51ca8..74bef8200ee67 100644 --- a/datafusion/sqllogictest/test_files/projection.slt +++ b/datafusion/sqllogictest/test_files/projection.slt @@ -276,5 +276,4 @@ logical_plan 03)----TableScan: t1 projection=[a], partial_filters=[t1.a > Int64(1)] physical_plan 01)FilterExec: a@0 > 1, projection=[] -02)--RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection/17513.parquet]]}, projection=[a], file_type=parquet, predicate=a@0 > 1, pruning_predicate=a_null_count@1 != row_count@2 AND a_max@0 > 1, required_guarantees=[] +02)--DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection/17513.parquet], projection=[a], file_type=parquet, predicate=a@0 > 1, pruning_predicate=a_null_count@1 != row_count@2 AND a_max@0 > 1, required_guarantees=[] diff --git a/datafusion/sqllogictest/test_files/projection_pushdown.slt b/datafusion/sqllogictest/test_files/projection_pushdown.slt index 1c89923080b69..28c41f362cc58 100644 --- a/datafusion/sqllogictest/test_files/projection_pushdown.slt +++ b/datafusion/sqllogictest/test_files/projection_pushdown.slt @@ -106,7 +106,7 @@ EXPLAIN SELECT id, s['value'] FROM simple_struct; logical_plan 01)Projection: simple_struct.id, get_field(simple_struct.s, Utf8("value")) 02)--TableScan: simple_struct projection=[id, s] -physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet]]}, projection=[id, get_field(s@1, value) as simple_struct.s[value]], file_type=parquet +physical_plan DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet], projection=[id, get_field(s@1, value) as simple_struct.s[value]], file_type=parquet # Verify correctness query II @@ -124,7 +124,7 @@ EXPLAIN SELECT s['label'] FROM simple_struct; logical_plan 01)Projection: get_field(simple_struct.s, Utf8("label")) 02)--TableScan: simple_struct projection=[s] -physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet]]}, projection=[get_field(s@1, label) as simple_struct.s[label]], file_type=parquet +physical_plan DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet], projection=[get_field(s@1, label) as simple_struct.s[label]], file_type=parquet # Verify correctness query T @@ -146,7 +146,7 @@ EXPLAIN SELECT id, s['value'], s['label'] FROM simple_struct; logical_plan 01)Projection: simple_struct.id, get_field(simple_struct.s, Utf8("value")), get_field(simple_struct.s, Utf8("label")) 02)--TableScan: simple_struct projection=[id, s] -physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet]]}, projection=[id, get_field(s@1, value) as simple_struct.s[value], get_field(s@1, label) as simple_struct.s[label]], file_type=parquet +physical_plan DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet], projection=[id, get_field(s@1, value) as simple_struct.s[value], get_field(s@1, label) as simple_struct.s[label]], file_type=parquet # Verify correctness query IIT @@ -168,7 +168,7 @@ EXPLAIN SELECT id, nested['outer']['inner'] FROM nested_struct; logical_plan 01)Projection: nested_struct.id, get_field(nested_struct.nested, Utf8("outer"), Utf8("inner")) 02)--TableScan: nested_struct projection=[id, nested] -physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/nested.parquet]]}, projection=[id, get_field(nested@1, outer, inner) as nested_struct.nested[outer][inner]], file_type=parquet +physical_plan DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/nested.parquet], projection=[id, get_field(nested@1, outer, inner) as nested_struct.nested[outer][inner]], file_type=parquet # Verify correctness query II @@ -188,7 +188,7 @@ EXPLAIN SELECT id, s['value'] + 1 FROM simple_struct; logical_plan 01)Projection: simple_struct.id, get_field(simple_struct.s, Utf8("value")) + Int64(1) 02)--TableScan: simple_struct projection=[id, s] -physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet]]}, projection=[id, get_field(s@1, value) + 1 as simple_struct.s[value] + Int64(1)], file_type=parquet +physical_plan DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet], projection=[id, get_field(s@1, value) + 1 as simple_struct.s[value] + Int64(1)], file_type=parquet # Verify correctness query II @@ -210,7 +210,7 @@ EXPLAIN SELECT id, s['label'] || '_suffix' FROM simple_struct; logical_plan 01)Projection: simple_struct.id, get_field(simple_struct.s, Utf8("label")) || Utf8("_suffix") 02)--TableScan: simple_struct projection=[id, s] -physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet]]}, projection=[id, get_field(s@1, label) || _suffix as simple_struct.s[label] || Utf8("_suffix")], file_type=parquet +physical_plan DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet], projection=[id, get_field(s@1, label) || _suffix as simple_struct.s[label] || Utf8("_suffix")], file_type=parquet # Verify correctness query IT @@ -242,7 +242,7 @@ logical_plan physical_plan 01)ProjectionExec: expr=[id@1 as id, __datafusion_extracted_1@0 as simple_struct.s[value]] 02)--FilterExec: id@1 > 2 -03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet]]}, projection=[get_field(s@1, value) as __datafusion_extracted_1, id], file_type=parquet, predicate=id@0 > 2, pruning_predicate=id_null_count@1 != row_count@2 AND id_max@0 > 2, required_guarantees=[] +03)----DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet], projection=[get_field(s@1, value) as __datafusion_extracted_1, id], file_type=parquet, predicate=id@0 > 2, pruning_predicate=id_null_count@1 != row_count@2 AND id_max@0 > 2, required_guarantees=[] # Verify correctness query II @@ -267,7 +267,7 @@ logical_plan physical_plan 01)ProjectionExec: expr=[id@1 as id, __datafusion_extracted_1@0 + 1 as simple_struct.s[value] + Int64(1)] 02)--FilterExec: id@1 > 2 -03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet]]}, projection=[get_field(s@1, value) as __datafusion_extracted_1, id], file_type=parquet, predicate=id@0 > 2, pruning_predicate=id_null_count@1 != row_count@2 AND id_max@0 > 2, required_guarantees=[] +03)----DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet], projection=[get_field(s@1, value) as __datafusion_extracted_1, id], file_type=parquet, predicate=id@0 > 2, pruning_predicate=id_null_count@1 != row_count@2 AND id_max@0 > 2, required_guarantees=[] # Verify correctness query II @@ -292,7 +292,7 @@ logical_plan physical_plan 01)ProjectionExec: expr=[id@0 as id, __datafusion_extracted_2@1 as simple_struct.s[label]] 02)--FilterExec: __datafusion_extracted_1@0 > 150, projection=[id@1, __datafusion_extracted_2@2] -03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet]]}, projection=[get_field(s@1, value) as __datafusion_extracted_1, id, get_field(s@1, label) as __datafusion_extracted_2], file_type=parquet +03)----DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet], projection=[get_field(s@1, value) as __datafusion_extracted_1, id, get_field(s@1, label) as __datafusion_extracted_2], file_type=parquet # Verify correctness query IT @@ -320,7 +320,7 @@ logical_plan 03)----TableScan: simple_struct projection=[id, s] physical_plan 01)SortExec: expr=[id@0 ASC NULLS LAST], preserve_partitioning=[false] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet]]}, projection=[id, get_field(s@1, value) as simple_struct.s[value]], file_type=parquet +02)--DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet], projection=[id, get_field(s@1, value) as simple_struct.s[value]], file_type=parquet # Verify correctness query II @@ -345,7 +345,7 @@ logical_plan 03)----TableScan: simple_struct projection=[id, s] physical_plan 01)SortExec: expr=[id@0 ASC NULLS LAST], preserve_partitioning=[false] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet]]}, projection=[id, get_field(s@1, value) + 1 as simple_struct.s[value] + Int64(1)], file_type=parquet +02)--DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet], projection=[id, get_field(s@1, value) + 1 as simple_struct.s[value] + Int64(1)], file_type=parquet # Verify correctness query II @@ -370,7 +370,7 @@ logical_plan 03)----TableScan: simple_struct projection=[id, s] physical_plan 01)SortExec: expr=[simple_struct.s[value]@1 ASC NULLS LAST], preserve_partitioning=[false] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet]]}, projection=[id, get_field(s@1, value) as simple_struct.s[value]], file_type=parquet +02)--DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet], projection=[id, get_field(s@1, value) as simple_struct.s[value]], file_type=parquet # Verify correctness query II @@ -413,7 +413,7 @@ logical_plan 03)----TableScan: three_cols projection=[col_a, col_b, col_c] physical_plan 01)SortExec: expr=[col_a@0 ASC NULLS LAST], preserve_partitioning=[false] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/three_cols.parquet]]}, projection=[col_a, col_b, col_c, col_b@1 as col_b_dup], file_type=parquet +02)--DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/three_cols.parquet], projection=[col_a, col_b, col_c, col_b@1 as col_b_dup], file_type=parquet # Verify correctness query IIII @@ -444,7 +444,7 @@ logical_plan 03)----TableScan: simple_struct projection=[id, s] physical_plan 01)SortExec: TopK(fetch=3), expr=[id@0 ASC NULLS LAST], preserve_partitioning=[false] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet]]}, projection=[id, get_field(s@1, value) as simple_struct.s[value]], file_type=parquet, predicate=DynamicFilter [ empty ] +02)--DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet], projection=[id, get_field(s@1, value) as simple_struct.s[value]], file_type=parquet, predicate=DynamicFilter [ empty ] # Verify correctness query II @@ -467,7 +467,7 @@ logical_plan 03)----TableScan: simple_struct projection=[id, s] physical_plan 01)SortExec: TopK(fetch=3), expr=[id@0 ASC NULLS LAST], preserve_partitioning=[false] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet]]}, projection=[id, get_field(s@1, value) + 1 as simple_struct.s[value] + Int64(1)], file_type=parquet, predicate=DynamicFilter [ empty ] +02)--DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet], projection=[id, get_field(s@1, value) + 1 as simple_struct.s[value] + Int64(1)], file_type=parquet, predicate=DynamicFilter [ empty ] # Verify correctness query II @@ -490,7 +490,7 @@ logical_plan 03)----TableScan: simple_struct projection=[id, s] physical_plan 01)SortExec: TopK(fetch=3), expr=[id@0 ASC NULLS LAST], preserve_partitioning=[false] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet]]}, projection=[id, get_field(s@1, value) as simple_struct.s[value], get_field(s@1, label) as simple_struct.s[label]], file_type=parquet, predicate=DynamicFilter [ empty ] +02)--DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet], projection=[id, get_field(s@1, value) as simple_struct.s[value], get_field(s@1, label) as simple_struct.s[label]], file_type=parquet, predicate=DynamicFilter [ empty ] # Verify correctness query IIT @@ -513,7 +513,7 @@ logical_plan 03)----TableScan: nested_struct projection=[id, nested] physical_plan 01)SortExec: TopK(fetch=2), expr=[id@0 ASC NULLS LAST], preserve_partitioning=[false] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/nested.parquet]]}, projection=[id, get_field(nested@1, outer, inner) as nested_struct.nested[outer][inner]], file_type=parquet, predicate=DynamicFilter [ empty ] +02)--DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/nested.parquet], projection=[id, get_field(nested@1, outer, inner) as nested_struct.nested[outer][inner]], file_type=parquet, predicate=DynamicFilter [ empty ] # Verify correctness query II @@ -535,7 +535,7 @@ logical_plan 03)----TableScan: simple_struct projection=[id, s] physical_plan 01)SortExec: TopK(fetch=3), expr=[id@0 ASC NULLS LAST], preserve_partitioning=[false] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet]]}, projection=[id, get_field(s@1, label) || _suffix as simple_struct.s[label] || Utf8("_suffix")], file_type=parquet, predicate=DynamicFilter [ empty ] +02)--DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet], projection=[id, get_field(s@1, label) || _suffix as simple_struct.s[label] || Utf8("_suffix")], file_type=parquet, predicate=DynamicFilter [ empty ] # Verify correctness query IT @@ -567,7 +567,7 @@ physical_plan 01)SortExec: expr=[simple_struct.s[value]@1 ASC NULLS LAST], preserve_partitioning=[false] 02)--ProjectionExec: expr=[id@1 as id, __datafusion_extracted_1@0 as simple_struct.s[value]] 03)----FilterExec: id@1 > 1 -04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet]]}, projection=[get_field(s@1, value) as __datafusion_extracted_1, id], file_type=parquet, predicate=id@0 > 1, pruning_predicate=id_null_count@1 != row_count@2 AND id_max@0 > 1, required_guarantees=[] +04)------DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet], projection=[get_field(s@1, value) as __datafusion_extracted_1, id], file_type=parquet, predicate=id@0 > 1, pruning_predicate=id_null_count@1 != row_count@2 AND id_max@0 > 1, required_guarantees=[] # Verify correctness query II @@ -595,7 +595,7 @@ physical_plan 01)SortExec: TopK(fetch=2), expr=[simple_struct.s[value]@1 ASC NULLS LAST], preserve_partitioning=[false] 02)--ProjectionExec: expr=[id@1 as id, __datafusion_extracted_1@0 as simple_struct.s[value]] 03)----FilterExec: id@1 > 1 -04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet]]}, projection=[get_field(s@1, value) as __datafusion_extracted_1, id], file_type=parquet, predicate=id@0 > 1, pruning_predicate=id_null_count@1 != row_count@2 AND id_max@0 > 1, required_guarantees=[] +04)------DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet], projection=[get_field(s@1, value) as __datafusion_extracted_1, id], file_type=parquet, predicate=id@0 > 1, pruning_predicate=id_null_count@1 != row_count@2 AND id_max@0 > 1, required_guarantees=[] # Verify correctness query II @@ -621,7 +621,7 @@ physical_plan 01)SortExec: TopK(fetch=2), expr=[id@0 ASC NULLS LAST], preserve_partitioning=[false] 02)--ProjectionExec: expr=[id@1 as id, __datafusion_extracted_1@0 + 1 as simple_struct.s[value] + Int64(1)] 03)----FilterExec: id@1 > 1 -04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet]]}, projection=[get_field(s@1, value) as __datafusion_extracted_1, id], file_type=parquet, predicate=id@0 > 1 AND DynamicFilter [ empty ], pruning_predicate=id_null_count@1 != row_count@2 AND id_max@0 > 1, required_guarantees=[] +04)------DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet], projection=[get_field(s@1, value) as __datafusion_extracted_1, id], file_type=parquet, predicate=id@0 > 1 AND DynamicFilter [ empty ], pruning_predicate=id_null_count@1 != row_count@2 AND id_max@0 > 1, required_guarantees=[] # Verify correctness query II @@ -684,7 +684,7 @@ logical_plan physical_plan 01)SortPreservingMergeExec: [id@0 ASC NULLS LAST] 02)--SortExec: expr=[id@0 ASC NULLS LAST], preserve_partitioning=[true] -03)----DataSourceExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/multi/part1.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/multi/part2.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/multi/part3.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/multi/part4.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/multi/part5.parquet]]}, projection=[id, get_field(s@1, value) as multi_struct.s[value]], file_type=parquet +03)----DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/multi/part1.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/multi/part2.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/multi/part3.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/multi/part4.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/multi/part5.parquet], projection=[id, get_field(s@1, value) as multi_struct.s[value]], file_type=parquet # Verify correctness query II @@ -710,7 +710,7 @@ logical_plan physical_plan 01)SortPreservingMergeExec: [id@0 ASC NULLS LAST], fetch=3 02)--SortExec: TopK(fetch=3), expr=[id@0 ASC NULLS LAST], preserve_partitioning=[true] -03)----DataSourceExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/multi/part1.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/multi/part2.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/multi/part3.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/multi/part4.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/multi/part5.parquet]]}, projection=[id, get_field(s@1, value) as multi_struct.s[value]], file_type=parquet, predicate=DynamicFilter [ empty ] +03)----DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/multi/part1.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/multi/part2.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/multi/part3.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/multi/part4.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/multi/part5.parquet], projection=[id, get_field(s@1, value) as multi_struct.s[value]], file_type=parquet, predicate=DynamicFilter [ empty ] # Verify correctness query II @@ -734,7 +734,7 @@ logical_plan physical_plan 01)SortPreservingMergeExec: [id@0 ASC NULLS LAST], fetch=3 02)--SortExec: TopK(fetch=3), expr=[id@0 ASC NULLS LAST], preserve_partitioning=[true] -03)----DataSourceExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/multi/part1.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/multi/part2.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/multi/part3.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/multi/part4.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/multi/part5.parquet]]}, projection=[id, get_field(s@1, value) + 1 as multi_struct.s[value] + Int64(1)], file_type=parquet, predicate=DynamicFilter [ empty ] +03)----DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/multi/part1.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/multi/part2.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/multi/part3.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/multi/part4.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/multi/part5.parquet], projection=[id, get_field(s@1, value) + 1 as multi_struct.s[value] + Int64(1)], file_type=parquet, predicate=DynamicFilter [ empty ] # Verify correctness query II @@ -762,8 +762,7 @@ physical_plan 02)--SortExec: expr=[id@0 ASC NULLS LAST], preserve_partitioning=[true] 03)----ProjectionExec: expr=[id@1 as id, __datafusion_extracted_1@0 as multi_struct.s[value]] 04)------FilterExec: id@1 > 2 -05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=3 -06)----------DataSourceExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/multi/part1.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/multi/part2.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/multi/part3.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/multi/part4.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/multi/part5.parquet]]}, projection=[get_field(s@1, value) as __datafusion_extracted_1, id], file_type=parquet, predicate=id@0 > 2, pruning_predicate=id_null_count@1 != row_count@2 AND id_max@0 > 2, required_guarantees=[] +05)--------DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/multi/part1.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/multi/part5.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/multi/part2.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/multi/part3.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/multi/part4.parquet], projection=[get_field(s@1, value) as __datafusion_extracted_1, id], file_type=parquet, predicate=id@0 > 2, pruning_predicate=id_null_count@1 != row_count@2 AND id_max@0 > 2, required_guarantees=[] # Verify correctness query II @@ -790,7 +789,7 @@ physical_plan 02)--AggregateExec: mode=FinalPartitioned, gby=[__datafusion_extracted_1@0 as __datafusion_extracted_1], aggr=[sum(__datafusion_extracted_2)] 03)----RepartitionExec: partitioning=Hash([__datafusion_extracted_1@0], 4), input_partitions=3 04)------AggregateExec: mode=Partial, gby=[__datafusion_extracted_1@0 as __datafusion_extracted_1], aggr=[sum(__datafusion_extracted_2)] -05)--------DataSourceExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/multi/part1.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/multi/part2.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/multi/part3.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/multi/part4.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/multi/part5.parquet]]}, projection=[get_field(s@1, label) as __datafusion_extracted_1, get_field(s@1, value) as __datafusion_extracted_2], file_type=parquet +05)--------DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/multi/part1.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/multi/part2.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/multi/part3.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/multi/part4.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/multi/part5.parquet], projection=[get_field(s@1, label) as __datafusion_extracted_1, get_field(s@1, value) as __datafusion_extracted_2], file_type=parquet # Verify correctness query TI @@ -821,7 +820,7 @@ EXPLAIN SELECT id, s['value'] FROM nullable_struct; logical_plan 01)Projection: nullable_struct.id, get_field(nullable_struct.s, Utf8("value")) 02)--TableScan: nullable_struct projection=[id, s] -physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/nullable.parquet]]}, projection=[id, get_field(s@1, value) as nullable_struct.s[value]], file_type=parquet +physical_plan DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/nullable.parquet], projection=[id, get_field(s@1, value) as nullable_struct.s[value]], file_type=parquet # Verify correctness (NULL struct returns NULL field) query II @@ -848,7 +847,7 @@ logical_plan physical_plan 01)ProjectionExec: expr=[id@0 as id, __datafusion_extracted_2@1 as nullable_struct.s[label]] 02)--FilterExec: __datafusion_extracted_1@0 IS NOT NULL, projection=[id@1, __datafusion_extracted_2@2] -03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/nullable.parquet]]}, projection=[get_field(s@1, value) as __datafusion_extracted_1, id, get_field(s@1, label) as __datafusion_extracted_2], file_type=parquet +03)----DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/nullable.parquet], projection=[get_field(s@1, value) as __datafusion_extracted_1, id, get_field(s@1, label) as __datafusion_extracted_2], file_type=parquet # Verify correctness query IT @@ -871,7 +870,7 @@ logical_plan 03)----TableScan: simple_struct projection=[id, s] physical_plan 01)SortExec: TopK(fetch=3), expr=[id@0 ASC NULLS LAST], preserve_partitioning=[false] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet]]}, projection=[id, get_field(s@1, value) as simple_struct.s[value], get_field(s@1, value) + 10 as simple_struct.s[value] + Int64(10), get_field(s@1, label) as simple_struct.s[label]], file_type=parquet, predicate=DynamicFilter [ empty ] +02)--DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet], projection=[id, get_field(s@1, value) as simple_struct.s[value], get_field(s@1, value) + 10 as simple_struct.s[value] + Int64(10), get_field(s@1, label) as simple_struct.s[label]], file_type=parquet, predicate=DynamicFilter [ empty ] # Verify correctness query IIIT @@ -894,7 +893,7 @@ logical_plan 03)----TableScan: simple_struct projection=[id] physical_plan 01)SortExec: TopK(fetch=3), expr=[id@0 ASC NULLS LAST], preserve_partitioning=[false] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet]]}, projection=[id, 42 as constant], file_type=parquet, predicate=DynamicFilter [ empty ] +02)--DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet], projection=[id, 42 as constant], file_type=parquet, predicate=DynamicFilter [ empty ] # Verify correctness query II @@ -916,7 +915,7 @@ logical_plan 02)--TableScan: simple_struct projection=[id] physical_plan 01)SortExec: TopK(fetch=3), expr=[id@0 ASC NULLS LAST], preserve_partitioning=[false] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet]]}, projection=[id], file_type=parquet, predicate=DynamicFilter [ empty ] +02)--DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet], projection=[id], file_type=parquet, predicate=DynamicFilter [ empty ] # Verify correctness query I @@ -944,7 +943,7 @@ logical_plan 03)----TableScan: simple_struct projection=[id] physical_plan 01)SortExec: TopK(fetch=3), expr=[id@0 ASC NULLS LAST], preserve_partitioning=[false] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet]]}, projection=[id, id@0 + 100 as computed], file_type=parquet, predicate=DynamicFilter [ empty ] +02)--DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet], projection=[id, id@0 + 100 as computed], file_type=parquet, predicate=DynamicFilter [ empty ] # Verify correctness query II @@ -973,7 +972,7 @@ physical_plan 01)ProjectionExec: expr=[__common_expr_1@0 * __common_expr_1@0 as id_and_value] 02)--ProjectionExec: expr=[id@1 + __datafusion_extracted_2@0 as __common_expr_1] 03)----FilterExec: id@1 > 2 -04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet]]}, projection=[get_field(s@1, value) as __datafusion_extracted_2, id], file_type=parquet, predicate=id@0 > 2, pruning_predicate=id_null_count@1 != row_count@2 AND id_max@0 > 2, required_guarantees=[] +04)------DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet], projection=[get_field(s@1, value) as __datafusion_extracted_2, id], file_type=parquet, predicate=id@0 > 2, pruning_predicate=id_null_count@1 != row_count@2 AND id_max@0 > 2, required_guarantees=[] query TT @@ -987,7 +986,7 @@ logical_plan physical_plan 01)ProjectionExec: expr=[__datafusion_extracted_1@0 + __datafusion_extracted_1@0 as doubled] 02)--FilterExec: id@1 > 2, projection=[__datafusion_extracted_1@0] -03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet]]}, projection=[get_field(s@1, value) as __datafusion_extracted_1, id], file_type=parquet, predicate=id@0 > 2, pruning_predicate=id_null_count@1 != row_count@2 AND id_max@0 > 2, required_guarantees=[] +03)----DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet], projection=[get_field(s@1, value) as __datafusion_extracted_1, id], file_type=parquet, predicate=id@0 > 2, pruning_predicate=id_null_count@1 != row_count@2 AND id_max@0 > 2, required_guarantees=[] # Verify correctness query I @@ -1012,7 +1011,7 @@ logical_plan physical_plan 01)ProjectionExec: expr=[__datafusion_extracted_1@0 as simple_struct.s[value], __datafusion_extracted_2@1 as simple_struct.s[label]] 02)--FilterExec: id@2 > 2, projection=[__datafusion_extracted_1@0, __datafusion_extracted_2@1] -03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet]]}, projection=[get_field(s@1, value) as __datafusion_extracted_1, get_field(s@1, label) as __datafusion_extracted_2, id], file_type=parquet, predicate=id@0 > 2, pruning_predicate=id_null_count@1 != row_count@2 AND id_max@0 > 2, required_guarantees=[] +03)----DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet], projection=[get_field(s@1, value) as __datafusion_extracted_1, get_field(s@1, label) as __datafusion_extracted_2, id], file_type=parquet, predicate=id@0 > 2, pruning_predicate=id_null_count@1 != row_count@2 AND id_max@0 > 2, required_guarantees=[] # Verify correctness query IT @@ -1036,7 +1035,7 @@ logical_plan 03)----TableScan: simple_struct projection=[id, s] physical_plan 01)SortExec: TopK(fetch=3), expr=[id@0 ASC NULLS LAST], preserve_partitioning=[false] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet]]}, projection=[id, get_field(s@1, value) + id@0 as combined], file_type=parquet, predicate=DynamicFilter [ empty ] +02)--DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet], projection=[id, get_field(s@1, value) + id@0 as combined], file_type=parquet, predicate=DynamicFilter [ empty ] # Verify correctness query II @@ -1062,7 +1061,7 @@ logical_plan physical_plan 01)ProjectionExec: expr=[__datafusion_extracted_1@0 * 2 + CAST(character_length(__datafusion_extracted_2@1) AS Int64) as score] 02)--FilterExec: id@2 > 1, projection=[__datafusion_extracted_1@0, __datafusion_extracted_2@1] -03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet]]}, projection=[get_field(s@1, value) as __datafusion_extracted_1, get_field(s@1, label) as __datafusion_extracted_2, id], file_type=parquet, predicate=id@0 > 1, pruning_predicate=id_null_count@1 != row_count@2 AND id_max@0 > 1, required_guarantees=[] +03)----DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet], projection=[get_field(s@1, value) as __datafusion_extracted_1, get_field(s@1, label) as __datafusion_extracted_2, id], file_type=parquet, predicate=id@0 > 1, pruning_predicate=id_null_count@1 != row_count@2 AND id_max@0 > 1, required_guarantees=[] # Verify correctness query I @@ -1092,7 +1091,7 @@ logical_plan 03)----TableScan: simple_struct projection=[id, s] physical_plan 01)SortExec: TopK(fetch=2), expr=[id@0 ASC NULLS LAST], preserve_partitioning=[false] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet]]}, projection=[id, 42 as answer, get_field(s@1, label) as simple_struct.s[label]], file_type=parquet, predicate=DynamicFilter [ empty ] +02)--DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet], projection=[id, 42 as answer, get_field(s@1, label) as simple_struct.s[label]], file_type=parquet, predicate=DynamicFilter [ empty ] # Verify correctness query IIT @@ -1115,7 +1114,7 @@ logical_plan 03)----TableScan: simple_struct projection=[id, s] physical_plan 01)SortExec: TopK(fetch=2), expr=[id@0 ASC NULLS LAST], preserve_partitioning=[false] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet]]}, projection=[id, get_field(s@1, value) + 100 as simple_struct.s[value] + Int64(100), get_field(s@1, label) || _test as simple_struct.s[label] || Utf8("_test")], file_type=parquet, predicate=DynamicFilter [ empty ] +02)--DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet], projection=[id, get_field(s@1, value) + 100 as simple_struct.s[value] + Int64(100), get_field(s@1, label) || _test as simple_struct.s[label] || Utf8("_test")], file_type=parquet, predicate=DynamicFilter [ empty ] # Verify correctness query IIT @@ -1139,7 +1138,7 @@ logical_plan physical_plan 01)ProjectionExec: expr=[id@1 as id, __datafusion_extracted_1@0 as simple_struct.s[value]] 02)--FilterExec: id@1 > 1 -03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet]]}, projection=[get_field(s@1, value) as __datafusion_extracted_1, id], file_type=parquet, predicate=id@0 > 1, pruning_predicate=id_null_count@1 != row_count@2 AND id_max@0 > 1, required_guarantees=[] +03)----DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet], projection=[get_field(s@1, value) as __datafusion_extracted_1, id], file_type=parquet, predicate=id@0 > 1, pruning_predicate=id_null_count@1 != row_count@2 AND id_max@0 > 1, required_guarantees=[] # Verify correctness query II @@ -1159,7 +1158,7 @@ logical_plan physical_plan 01)ProjectionExec: expr=[__datafusion_extracted_1@0 as simple_struct.s[value]] 02)--FilterExec: id@1 > 1 AND (id@1 < 4 OR id@1 = 5), projection=[__datafusion_extracted_1@0] -03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet]]}, projection=[get_field(s@1, value) as __datafusion_extracted_1, id], file_type=parquet, predicate=id@0 > 1 AND (id@0 < 4 OR id@0 = 5), pruning_predicate=id_null_count@1 != row_count@2 AND id_max@0 > 1 AND (id_null_count@1 != row_count@2 AND id_min@3 < 4 OR id_null_count@1 != row_count@2 AND id_min@3 <= 5 AND 5 <= id_max@0), required_guarantees=[] +03)----DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet], projection=[get_field(s@1, value) as __datafusion_extracted_1, id], file_type=parquet, predicate=id@0 > 1 AND (id@0 < 4 OR id@0 = 5), pruning_predicate=id_null_count@1 != row_count@2 AND id_max@0 > 1 AND (id_null_count@1 != row_count@2 AND id_min@3 < 4 OR id_null_count@1 != row_count@2 AND id_min@3 <= 5 AND 5 <= id_max@0), required_guarantees=[] # Verify correctness - should return rows where (id > 1) AND ((id < 4) OR (id = 5)) # That's: id=2,3 (1 1 AND id@1 < 5, projection=[__datafusion_extracted_1@0] -03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet]]}, projection=[get_field(s@1, value) as __datafusion_extracted_1, id], file_type=parquet, predicate=id@0 > 1 AND id@0 < 5, pruning_predicate=id_null_count@1 != row_count@2 AND id_max@0 > 1 AND id_null_count@1 != row_count@2 AND id_min@3 < 5, required_guarantees=[] +03)----DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet], projection=[get_field(s@1, value) as __datafusion_extracted_1, id], file_type=parquet, predicate=id@0 > 1 AND id@0 < 5, pruning_predicate=id_null_count@1 != row_count@2 AND id_max@0 > 1 AND id_null_count@1 != row_count@2 AND id_min@3 < 5, required_guarantees=[] # Verify correctness - should return rows where 1 < id < 5 (id=2,3,4) query I @@ -1202,7 +1201,7 @@ logical_plan physical_plan 01)ProjectionExec: expr=[__datafusion_extracted_1@0 as simple_struct.s[value], __datafusion_extracted_2@1 as simple_struct.s[label], id@2 as id] 02)--FilterExec: id@2 > 1 -03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet]]}, projection=[get_field(s@1, value) as __datafusion_extracted_1, get_field(s@1, label) as __datafusion_extracted_2, id], file_type=parquet, predicate=id@0 > 1, pruning_predicate=id_null_count@1 != row_count@2 AND id_max@0 > 1, required_guarantees=[] +03)----DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet], projection=[get_field(s@1, value) as __datafusion_extracted_1, get_field(s@1, label) as __datafusion_extracted_2, id], file_type=parquet, predicate=id@0 > 1, pruning_predicate=id_null_count@1 != row_count@2 AND id_max@0 > 1, required_guarantees=[] # Verify correctness - note that id is now at index 2 in the augmented projection query ITI @@ -1223,7 +1222,7 @@ logical_plan physical_plan 01)ProjectionExec: expr=[__datafusion_extracted_2@0 as simple_struct.s[value]] 02)--FilterExec: character_length(__datafusion_extracted_1@0) > 4, projection=[__datafusion_extracted_2@1] -03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet]]}, projection=[get_field(s@1, label) as __datafusion_extracted_1, get_field(s@1, value) as __datafusion_extracted_2], file_type=parquet +03)----DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet], projection=[get_field(s@1, label) as __datafusion_extracted_1, get_field(s@1, value) as __datafusion_extracted_2], file_type=parquet # Verify correctness - filter on rows where label length > 4 (all have length 5, except 'one' has 3) # Wait, from the data: alpha(5), beta(4), gamma(5), delta(5), epsilon(7) @@ -1256,7 +1255,7 @@ logical_plan physical_plan 01)ProjectionExec: expr=[id@0 as id] 02)--SortExec: expr=[__datafusion_extracted_1@1 ASC NULLS LAST], preserve_partitioning=[false] -03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet]]}, projection=[id, get_field(s@1, value) as __datafusion_extracted_1], file_type=parquet +03)----DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet], projection=[id, get_field(s@1, value) as __datafusion_extracted_1], file_type=parquet # Verify correctness query I @@ -1285,7 +1284,7 @@ logical_plan physical_plan 01)ProjectionExec: expr=[id@0 as id, simple_struct.s[value]@1 as simple_struct.s[value]] 02)--SortExec: expr=[id@0 ASC NULLS LAST, __datafusion_extracted_1@2 ASC NULLS LAST], preserve_partitioning=[false] -03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet]]}, projection=[id, get_field(s@1, value) as simple_struct.s[value], get_field(s@1, label) as __datafusion_extracted_1], file_type=parquet +03)----DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet], projection=[id, get_field(s@1, value) as simple_struct.s[value], get_field(s@1, label) as __datafusion_extracted_1], file_type=parquet # Verify correctness query II @@ -1314,7 +1313,7 @@ logical_plan physical_plan 01)ProjectionExec: expr=[id@0 as id] 02)--SortExec: TopK(fetch=2), expr=[__datafusion_extracted_1@1 ASC NULLS LAST], preserve_partitioning=[false] -03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet]]}, projection=[id, get_field(s@1, value) as __datafusion_extracted_1], file_type=parquet +03)----DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet], projection=[id, get_field(s@1, value) as __datafusion_extracted_1], file_type=parquet # Verify correctness query I @@ -1340,7 +1339,7 @@ logical_plan physical_plan 01)ProjectionExec: expr=[id@0 as id] 02)--SortExec: expr=[__datafusion_extracted_1@1 * 2 ASC NULLS LAST], preserve_partitioning=[false] -03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet]]}, projection=[id, get_field(s@1, value) as __datafusion_extracted_1], file_type=parquet +03)----DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet], projection=[id, get_field(s@1, value) as __datafusion_extracted_1], file_type=parquet # Verify correctness query I @@ -1366,7 +1365,7 @@ logical_plan 03)----TableScan: simple_struct projection=[id, s] physical_plan 01)SortExec: expr=[id@0 ASC NULLS LAST, simple_struct.s[value]@1 ASC NULLS LAST], preserve_partitioning=[false] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet]]}, projection=[id, get_field(s@1, value) as simple_struct.s[value]], file_type=parquet +02)--DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet], projection=[id, get_field(s@1, value) as simple_struct.s[value]], file_type=parquet # Verify correctness query II @@ -1420,8 +1419,8 @@ logical_plan 06)------TableScan: join_right projection=[id, s] physical_plan 01)HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(__datafusion_extracted_1@0, __datafusion_extracted_2 * Int64(10)@2)], projection=[id@1, id@3] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet]]}, projection=[get_field(s@1, value) as __datafusion_extracted_1, id], file_type=parquet -03)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/join_right.parquet]]}, projection=[get_field(s@1, level) as __datafusion_extracted_2, id, get_field(s@1, level) * 10 as __datafusion_extracted_2 * Int64(10)], file_type=parquet +02)--DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet], projection=[get_field(s@1, value) as __datafusion_extracted_1, id], file_type=parquet +03)--DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/join_right.parquet], projection=[get_field(s@1, level) as __datafusion_extracted_2, id, get_field(s@1, level) * 10 as __datafusion_extracted_2 * Int64(10)], file_type=parquet # Verify correctness - value = level * 10 # simple_struct: (1,100), (2,200), (3,150), (4,300), (5,250) @@ -1456,8 +1455,8 @@ logical_plan physical_plan 01)HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(id@0, id@0)] 02)--FilterExec: __datafusion_extracted_1@0 > 150, projection=[id@1] -03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet]]}, projection=[get_field(s@1, value) as __datafusion_extracted_1, id], file_type=parquet -04)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/join_right.parquet]]}, projection=[id], file_type=parquet, predicate=DynamicFilter [ empty ] +03)----DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet], projection=[get_field(s@1, value) as __datafusion_extracted_1, id], file_type=parquet +04)--DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/join_right.parquet], projection=[id], file_type=parquet, predicate=DynamicFilter [ empty ] # Verify correctness - id matches and value > 150 query II @@ -1495,9 +1494,9 @@ logical_plan physical_plan 01)HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(id@0, id@0)] 02)--FilterExec: __datafusion_extracted_1@0 > 100, projection=[id@1] -03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet]]}, projection=[get_field(s@1, value) as __datafusion_extracted_1, id], file_type=parquet +03)----DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet], projection=[get_field(s@1, value) as __datafusion_extracted_1, id], file_type=parquet 04)--FilterExec: __datafusion_extracted_2@0 > 3, projection=[id@1] -05)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/join_right.parquet]]}, projection=[get_field(s@1, level) as __datafusion_extracted_2, id], file_type=parquet, predicate=DynamicFilter [ empty ] +05)----DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/join_right.parquet], projection=[get_field(s@1, level) as __datafusion_extracted_2, id], file_type=parquet, predicate=DynamicFilter [ empty ] # Verify correctness - id matches, value > 100, and level > 3 # Matching ids where value > 100: 2(200), 3(150), 4(300), 5(250) @@ -1532,8 +1531,8 @@ logical_plan physical_plan 01)ProjectionExec: expr=[id@1 as id, __datafusion_extracted_1@0 as simple_struct.s[label], __datafusion_extracted_2@2 as join_right.s[role]] 02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(id@1, id@1)], projection=[__datafusion_extracted_1@0, id@1, __datafusion_extracted_2@2] -03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet]]}, projection=[get_field(s@1, label) as __datafusion_extracted_1, id], file_type=parquet -04)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/join_right.parquet]]}, projection=[get_field(s@1, role) as __datafusion_extracted_2, id], file_type=parquet, predicate=DynamicFilter [ empty ] +03)----DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet], projection=[get_field(s@1, label) as __datafusion_extracted_1, id], file_type=parquet +04)----DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/join_right.parquet], projection=[get_field(s@1, role) as __datafusion_extracted_2, id], file_type=parquet, predicate=DynamicFilter [ empty ] # Verify correctness query ITT @@ -1564,8 +1563,8 @@ logical_plan 03)--TableScan: join_right projection=[id] physical_plan 01)HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(id@0, id@0)] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet]]}, projection=[id], file_type=parquet -03)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/join_right.parquet]]}, projection=[id], file_type=parquet, predicate=DynamicFilter [ empty ] +02)--DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet], projection=[id], file_type=parquet +03)--DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/join_right.parquet], projection=[id], file_type=parquet, predicate=DynamicFilter [ empty ] # Verify correctness query II @@ -1602,9 +1601,9 @@ logical_plan physical_plan 01)ProjectionExec: expr=[id@1 as id, __datafusion_extracted_2@0 as simple_struct.s[value], __datafusion_extracted_3@2 as join_right.s[level]] 02)--HashJoinExec: mode=CollectLeft, join_type=Left, on=[(id@1, id@0)], projection=[__datafusion_extracted_2@0, id@1, __datafusion_extracted_3@3] -03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet]]}, projection=[get_field(s@1, value) as __datafusion_extracted_2, id], file_type=parquet +03)----DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet], projection=[get_field(s@1, value) as __datafusion_extracted_2, id], file_type=parquet 04)----FilterExec: __datafusion_extracted_1@0 > 5, projection=[id@1, __datafusion_extracted_3@2] -05)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/join_right.parquet]]}, projection=[get_field(s@1, level) as __datafusion_extracted_1, id, get_field(s@1, level) as __datafusion_extracted_3], file_type=parquet, predicate=DynamicFilter [ empty ] +05)------DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/join_right.parquet], projection=[get_field(s@1, level) as __datafusion_extracted_1, id, get_field(s@1, level) as __datafusion_extracted_3], file_type=parquet, predicate=DynamicFilter [ empty ] # Verify correctness - left join with level > 5 condition # Only join_right rows with level > 5 are matched: id=1 (level=10), id=4 (level=8) @@ -1639,8 +1638,7 @@ logical_plan physical_plan 01)ProjectionExec: expr=[__datafusion_extracted_1@0 as simple_struct.s[value]] 02)--FilterExec: id@1 > 2, projection=[__datafusion_extracted_1@0] -03)----RepartitionExec: partitioning=RoundRobinBatch(32), input_partitions=1 -04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet]]}, projection=[get_field(s@1, value) as __datafusion_extracted_1, id], file_type=parquet, predicate=id@0 > 2, pruning_predicate=id_null_count@1 != row_count@2 AND id_max@0 > 2, required_guarantees=[] +03)----DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet], projection=[get_field(s@1, value) as __datafusion_extracted_1, id], file_type=parquet, predicate=id@0 > 2, pruning_predicate=id_null_count@1 != row_count@2 AND id_max@0 > 2, required_guarantees=[] ##################### # Section 14: SubqueryAlias tests @@ -1664,7 +1662,7 @@ logical_plan physical_plan 01)ProjectionExec: expr=[__datafusion_extracted_1@0 as t.s[value]] 02)--FilterExec: id@1 > 2, projection=[__datafusion_extracted_1@0] -03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet]]}, projection=[get_field(s@1, value) as __datafusion_extracted_1, id], file_type=parquet, predicate=id@0 > 2, pruning_predicate=id_null_count@1 != row_count@2 AND id_max@0 > 2, required_guarantees=[] +03)----DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet], projection=[get_field(s@1, value) as __datafusion_extracted_1, id], file_type=parquet, predicate=id@0 > 2, pruning_predicate=id_null_count@1 != row_count@2 AND id_max@0 > 2, required_guarantees=[] # Verify correctness query I @@ -1686,7 +1684,7 @@ logical_plan 05)--------TableScan: simple_struct projection=[s] physical_plan 01)SortExec: expr=[t.s[value]@0 ASC NULLS LAST], preserve_partitioning=[false] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet]]}, projection=[get_field(s@1, value) as t.s[value], get_field(s@1, label) as t.s[label]], file_type=parquet +02)--DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet], projection=[get_field(s@1, value) as t.s[value], get_field(s@1, label) as t.s[label]], file_type=parquet # Verify correctness query IT @@ -1713,7 +1711,7 @@ logical_plan physical_plan 01)ProjectionExec: expr=[__datafusion_extracted_1@0 as u.s[value]] 02)--FilterExec: id@1 > 2, projection=[__datafusion_extracted_1@0] -03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet]]}, projection=[get_field(s@1, value) as __datafusion_extracted_1, id], file_type=parquet, predicate=id@0 > 2, pruning_predicate=id_null_count@1 != row_count@2 AND id_max@0 > 2, required_guarantees=[] +03)----DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet], projection=[get_field(s@1, value) as __datafusion_extracted_1, id], file_type=parquet, predicate=id@0 > 2, pruning_predicate=id_null_count@1 != row_count@2 AND id_max@0 > 2, required_guarantees=[] # Verify correctness query I @@ -1735,7 +1733,7 @@ logical_plan 05)--------TableScan: simple_struct projection=[id, s], partial_filters=[get_field(simple_struct.s, Utf8("value")) > Int64(200)] physical_plan 01)FilterExec: __datafusion_extracted_1@0 > 200, projection=[id@1] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet]]}, projection=[get_field(s@1, value) as __datafusion_extracted_1, id], file_type=parquet +02)--DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet], projection=[get_field(s@1, value) as __datafusion_extracted_1, id], file_type=parquet # Verify correctness query I @@ -1772,9 +1770,9 @@ physical_plan 01)ProjectionExec: expr=[__datafusion_extracted_1@0 as t.s[value]] 02)--UnionExec 03)----FilterExec: id@1 <= 3, projection=[__datafusion_extracted_1@0] -04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet]]}, projection=[get_field(s@1, value) as __datafusion_extracted_1, id], file_type=parquet, predicate=id@0 <= 3, pruning_predicate=id_null_count@1 != row_count@2 AND id_min@0 <= 3, required_guarantees=[] +04)------DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet], projection=[get_field(s@1, value) as __datafusion_extracted_1, id], file_type=parquet, predicate=id@0 <= 3, pruning_predicate=id_null_count@1 != row_count@2 AND id_min@0 <= 3, required_guarantees=[] 05)----FilterExec: id@1 > 3, projection=[__datafusion_extracted_1@0] -06)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet]]}, projection=[get_field(s@1, value) as __datafusion_extracted_1, id], file_type=parquet, predicate=id@0 > 3, pruning_predicate=id_null_count@1 != row_count@2 AND id_max@0 > 3, required_guarantees=[] +06)------DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet], projection=[get_field(s@1, value) as __datafusion_extracted_1, id], file_type=parquet, predicate=id@0 > 3, pruning_predicate=id_null_count@1 != row_count@2 AND id_max@0 > 3, required_guarantees=[] # Verify correctness query I @@ -1817,9 +1815,9 @@ physical_plan 03)----ProjectionExec: expr=[__datafusion_extracted_1@0 as t.s[value], __datafusion_extracted_2@1 as t.s[label]] 04)------UnionExec 05)--------FilterExec: id@2 <= 3, projection=[__datafusion_extracted_1@0, __datafusion_extracted_2@1] -06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet]]}, projection=[get_field(s@1, value) as __datafusion_extracted_1, get_field(s@1, label) as __datafusion_extracted_2, id], file_type=parquet, predicate=id@0 <= 3, pruning_predicate=id_null_count@1 != row_count@2 AND id_min@0 <= 3, required_guarantees=[] +06)----------DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet], projection=[get_field(s@1, value) as __datafusion_extracted_1, get_field(s@1, label) as __datafusion_extracted_2, id], file_type=parquet, predicate=id@0 <= 3, pruning_predicate=id_null_count@1 != row_count@2 AND id_min@0 <= 3, required_guarantees=[] 07)--------FilterExec: id@2 > 3, projection=[__datafusion_extracted_1@0, __datafusion_extracted_2@1] -08)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet]]}, projection=[get_field(s@1, value) as __datafusion_extracted_1, get_field(s@1, label) as __datafusion_extracted_2, id], file_type=parquet, predicate=id@0 > 3, pruning_predicate=id_null_count@1 != row_count@2 AND id_max@0 > 3, required_guarantees=[] +08)----------DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet], projection=[get_field(s@1, value) as __datafusion_extracted_1, get_field(s@1, label) as __datafusion_extracted_2, id], file_type=parquet, predicate=id@0 > 3, pruning_predicate=id_null_count@1 != row_count@2 AND id_max@0 > 3, required_guarantees=[] # Verify correctness query IT @@ -1857,7 +1855,7 @@ logical_plan physical_plan 01)ProjectionExec: expr=[get_field(s@0, label) IS NOT NULL as has_label, count(Int64(1))@1 as count(Int64(1))] 02)--AggregateExec: mode=Single, gby=[s@0 as s], aggr=[count(Int64(1))] -03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet]]}, projection=[s], file_type=parquet +03)----DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet], projection=[s], file_type=parquet # Verify correctness - all labels are non-null query BI @@ -1895,8 +1893,8 @@ logical_plan physical_plan 01)ProjectionExec: expr=[__datafusion_extracted_3@1 as s.s[value], __datafusion_extracted_4@0 as j.s[role]] 02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(id@2, id@2)], filter=__datafusion_extracted_1@1 > __datafusion_extracted_2@0, projection=[__datafusion_extracted_4@1, __datafusion_extracted_3@4] -03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/join_right.parquet]]}, projection=[get_field(s@1, level) as __datafusion_extracted_2, get_field(s@1, role) as __datafusion_extracted_4, id], file_type=parquet -04)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet]]}, projection=[get_field(s@1, value) as __datafusion_extracted_1, get_field(s@1, value) as __datafusion_extracted_3, id], file_type=parquet, predicate=DynamicFilter [ empty ] +03)----DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/join_right.parquet], projection=[get_field(s@1, level) as __datafusion_extracted_2, get_field(s@1, role) as __datafusion_extracted_4, id], file_type=parquet +04)----DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet], projection=[get_field(s@1, value) as __datafusion_extracted_1, get_field(s@1, value) as __datafusion_extracted_3, id], file_type=parquet, predicate=DynamicFilter [ empty ] # Verify correctness - only admin roles match (ids 1 and 4) query II @@ -1931,8 +1929,8 @@ logical_plan 06)------TableScan: join_right projection=[id, s] physical_plan 01)HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(id@1, id@1)], filter=__datafusion_extracted_1@0 > __datafusion_extracted_2@1, projection=[id@1, id@3] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet]]}, projection=[get_field(s@1, value) as __datafusion_extracted_1, id], file_type=parquet -03)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/join_right.parquet]]}, projection=[get_field(s@1, level) as __datafusion_extracted_2, id], file_type=parquet, predicate=DynamicFilter [ empty ] +02)--DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/simple.parquet], projection=[get_field(s@1, value) as __datafusion_extracted_1, id], file_type=parquet +03)--DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/join_right.parquet], projection=[get_field(s@1, level) as __datafusion_extracted_2, id], file_type=parquet, predicate=DynamicFilter [ empty ] # Verify correctness - all rows match since value >> level for all ids # simple_struct: (1,100), (2,200), (3,150), (4,300), (5,250) @@ -1981,7 +1979,7 @@ logical_plan 04)------TableScan: t projection=[s], partial_filters=[CASE WHEN get_field(t.s, Utf8("f1")) IS NOT NULL THEN get_field(t.s, Utf8("f1")) ELSE get_field(t.s, Utf8("f2")) END = Int64(1)] physical_plan 01)FilterExec: CASE WHEN __datafusion_extracted_3@0 IS NOT NULL THEN __datafusion_extracted_3@0 ELSE __datafusion_extracted_4@1 END = 1, projection=[__datafusion_extracted_2@2] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/test.parquet]]}, projection=[get_field(s@0, f1) as __datafusion_extracted_3, get_field(s@0, f2) as __datafusion_extracted_4, get_field(s@0, f1) as __datafusion_extracted_2], file_type=parquet +02)--DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/projection_pushdown/test.parquet], projection=[get_field(s@0, f1) as __datafusion_extracted_3, get_field(s@0, f2) as __datafusion_extracted_4, get_field(s@0, f1) as __datafusion_extracted_2], file_type=parquet query I SELECT diff --git a/datafusion/sqllogictest/test_files/push_down_filter_parquet.slt b/datafusion/sqllogictest/test_files/push_down_filter_parquet.slt index e1c83c8c330d8..f29d75c74f446 100644 --- a/datafusion/sqllogictest/test_files/push_down_filter_parquet.slt +++ b/datafusion/sqllogictest/test_files/push_down_filter_parquet.slt @@ -77,7 +77,7 @@ explain select * from test_filter_with_limit where value = 2 limit 1; ---- physical_plan 01)CoalescePartitionsExec: fetch=1 -02)--DataSourceExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/test_filter_with_limit/part-0.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/test_filter_with_limit/part-1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/test_filter_with_limit/part-2.parquet]]}, projection=[part_key, value], limit=1, file_type=parquet, predicate=value@1 = 2, pruning_predicate=value_null_count@2 != row_count@3 AND value_min@0 <= 2 AND 2 <= value_max@1, required_guarantees=[value in (2)] +02)--DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/test_filter_with_limit/part-0.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/test_filter_with_limit/part-1.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/test_filter_with_limit/part-2.parquet], projection=[part_key, value], limit=1, file_type=parquet, predicate=value@1 = 2, pruning_predicate=value_null_count@2 != row_count@3 AND value_min@0 <= 2 AND 2 <= value_max@1, required_guarantees=[value in (2)] query II select * from test_filter_with_limit where value = 2 limit 1; @@ -114,43 +114,43 @@ LOCATION 'test_files/scratch/push_down_filter_parquet/t.parquet'; query TT explain select a from t where a = '100'; ---- -physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/t.parquet]]}, projection=[a], file_type=parquet, predicate=a@0 = 100, pruning_predicate=a_null_count@2 != row_count@3 AND a_min@0 <= 100 AND 100 <= a_max@1, required_guarantees=[a in (100)] +physical_plan DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/t.parquet], projection=[a], file_type=parquet, predicate=a@0 = 100, pruning_predicate=a_null_count@2 != row_count@3 AND a_min@0 <= 100 AND 100 <= a_max@1, required_guarantees=[a in (100)] # The predicate should not have a column cast when the value is a valid i32 query TT explain select a from t where a != '100'; ---- -physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/t.parquet]]}, projection=[a], file_type=parquet, predicate=a@0 != 100, pruning_predicate=a_null_count@2 != row_count@3 AND (a_min@0 != 100 OR 100 != a_max@1), required_guarantees=[a not in (100)] +physical_plan DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/t.parquet], projection=[a], file_type=parquet, predicate=a@0 != 100, pruning_predicate=a_null_count@2 != row_count@3 AND (a_min@0 != 100 OR 100 != a_max@1), required_guarantees=[a not in (100)] # The predicate should still have the column cast when the value is a NOT valid i32 query TT explain select a from t where a = '99999999999'; ---- -physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/t.parquet]]}, projection=[a], file_type=parquet, predicate=CAST(a@0 AS Utf8) = 99999999999 +physical_plan DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/t.parquet], projection=[a], file_type=parquet, predicate=CAST(a@0 AS Utf8) = 99999999999 # The predicate should still have the column cast when the value is a NOT valid i32 query TT explain select a from t where a = '99.99'; ---- -physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/t.parquet]]}, projection=[a], file_type=parquet, predicate=CAST(a@0 AS Utf8) = 99.99 +physical_plan DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/t.parquet], projection=[a], file_type=parquet, predicate=CAST(a@0 AS Utf8) = 99.99 # The predicate should still have the column cast when the value is a NOT valid i32 query TT explain select a from t where a = ''; ---- -physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/t.parquet]]}, projection=[a], file_type=parquet, predicate=CAST(a@0 AS Utf8) = +physical_plan DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/t.parquet], projection=[a], file_type=parquet, predicate=CAST(a@0 AS Utf8) = # The predicate should not have a column cast when the operator is = or != and the literal can be round-trip casted without losing information. query TT explain select a from t where cast(a as string) = '100'; ---- -physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/t.parquet]]}, projection=[a], file_type=parquet, predicate=a@0 = 100, pruning_predicate=a_null_count@2 != row_count@3 AND a_min@0 <= 100 AND 100 <= a_max@1, required_guarantees=[a in (100)] +physical_plan DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/t.parquet], projection=[a], file_type=parquet, predicate=a@0 = 100, pruning_predicate=a_null_count@2 != row_count@3 AND a_min@0 <= 100 AND 100 <= a_max@1, required_guarantees=[a in (100)] # The predicate should still have the column cast when the literal alters its string representation after round-trip casting (leading zero lost). query TT explain select a from t where CAST(a AS string) = '0123'; ---- -physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/t.parquet]]}, projection=[a], file_type=parquet, predicate=CAST(a@0 AS Utf8View) = 0123 +physical_plan DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/t.parquet], projection=[a], file_type=parquet, predicate=CAST(a@0 AS Utf8View) = 0123 # Test dynamic filter pushdown with swapped join inputs (issue #17196) @@ -174,9 +174,8 @@ explain select * from small_table join large_table on small_table.k = large_tabl ---- physical_plan 01)HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(k@0, k@0)] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/small_table.parquet]]}, projection=[k], file_type=parquet -03)--RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -04)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/large_table.parquet]]}, projection=[k, v], file_type=parquet, predicate=v@1 >= 50 AND DynamicFilter [ empty ], pruning_predicate=v_null_count@1 != row_count@2 AND v_max@0 >= 50, required_guarantees=[] +02)--DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/small_table.parquet], projection=[k], file_type=parquet +03)--DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/large_table.parquet], projection=[k, v], file_type=parquet, predicate=v@1 >= 50 AND DynamicFilter [ empty ], pruning_predicate=v_null_count@1 != row_count@2 AND v_max@0 >= 50, required_guarantees=[] statement ok drop table small_table; diff --git a/datafusion/sqllogictest/test_files/push_down_filter_regression.slt b/datafusion/sqllogictest/test_files/push_down_filter_regression.slt index 8459fcc682485..ca1e0e0f35c41 100644 --- a/datafusion/sqllogictest/test_files/push_down_filter_regression.slt +++ b/datafusion/sqllogictest/test_files/push_down_filter_regression.slt @@ -144,7 +144,7 @@ physical_plan 01)AggregateExec: mode=Final, gby=[], aggr=[max(agg_dyn_test.id)] 02)--CoalescePartitionsExec 03)----AggregateExec: mode=Partial, gby=[], aggr=[max(agg_dyn_test.id)] -04)------DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/core/tests/data/test_statistics_per_partition/date=2025-03-01/j5fUeSDQo22oPyPU.parquet, WORKSPACE_ROOT/datafusion/core/tests/data/test_statistics_per_partition/date=2025-03-02/j5fUeSDQo22oPyPU.parquet], [WORKSPACE_ROOT/datafusion/core/tests/data/test_statistics_per_partition/date=2025-03-03/j5fUeSDQo22oPyPU.parquet, WORKSPACE_ROOT/datafusion/core/tests/data/test_statistics_per_partition/date=2025-03-04/j5fUeSDQo22oPyPU.parquet]]}, projection=[id], file_type=parquet, predicate=id@0 > 1 AND DynamicFilter [ empty ], pruning_predicate=id_null_count@1 != row_count@2 AND id_max@0 > 1, required_guarantees=[] +04)------DataSourceExec: files=[WORKSPACE_ROOT/datafusion/core/tests/data/test_statistics_per_partition/date=2025-03-01/j5fUeSDQo22oPyPU.parquet, WORKSPACE_ROOT/datafusion/core/tests/data/test_statistics_per_partition/date=2025-03-03/j5fUeSDQo22oPyPU.parquet, WORKSPACE_ROOT/datafusion/core/tests/data/test_statistics_per_partition/date=2025-03-02/j5fUeSDQo22oPyPU.parquet, WORKSPACE_ROOT/datafusion/core/tests/data/test_statistics_per_partition/date=2025-03-04/j5fUeSDQo22oPyPU.parquet], projection=[id], file_type=parquet, predicate=id@0 > 1 AND DynamicFilter [ empty ], pruning_predicate=id_null_count@1 != row_count@2 AND id_max@0 > 1, required_guarantees=[] query I select max(id) from agg_dyn_test where id > 1; @@ -159,7 +159,7 @@ physical_plan 01)AggregateExec: mode=Final, gby=[], aggr=[max(agg_dyn_test.id)] 02)--CoalescePartitionsExec 03)----AggregateExec: mode=Partial, gby=[], aggr=[max(agg_dyn_test.id)] -04)------DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/core/tests/data/test_statistics_per_partition/date=2025-03-01/j5fUeSDQo22oPyPU.parquet, WORKSPACE_ROOT/datafusion/core/tests/data/test_statistics_per_partition/date=2025-03-02/j5fUeSDQo22oPyPU.parquet], [WORKSPACE_ROOT/datafusion/core/tests/data/test_statistics_per_partition/date=2025-03-03/j5fUeSDQo22oPyPU.parquet, WORKSPACE_ROOT/datafusion/core/tests/data/test_statistics_per_partition/date=2025-03-04/j5fUeSDQo22oPyPU.parquet]]}, projection=[id], file_type=parquet, predicate=CAST(id@0 AS Int64) + 1 > 1 AND DynamicFilter [ empty ] +04)------DataSourceExec: files=[WORKSPACE_ROOT/datafusion/core/tests/data/test_statistics_per_partition/date=2025-03-01/j5fUeSDQo22oPyPU.parquet, WORKSPACE_ROOT/datafusion/core/tests/data/test_statistics_per_partition/date=2025-03-03/j5fUeSDQo22oPyPU.parquet, WORKSPACE_ROOT/datafusion/core/tests/data/test_statistics_per_partition/date=2025-03-02/j5fUeSDQo22oPyPU.parquet, WORKSPACE_ROOT/datafusion/core/tests/data/test_statistics_per_partition/date=2025-03-04/j5fUeSDQo22oPyPU.parquet], projection=[id], file_type=parquet, predicate=CAST(id@0 AS Int64) + 1 > 1 AND DynamicFilter [ empty ] # Expect dynamic filter available inside data source query TT @@ -169,7 +169,7 @@ physical_plan 01)AggregateExec: mode=Final, gby=[], aggr=[max(agg_dyn_test.id), min(agg_dyn_test.id)] 02)--CoalescePartitionsExec 03)----AggregateExec: mode=Partial, gby=[], aggr=[max(agg_dyn_test.id), min(agg_dyn_test.id)] -04)------DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/core/tests/data/test_statistics_per_partition/date=2025-03-01/j5fUeSDQo22oPyPU.parquet, WORKSPACE_ROOT/datafusion/core/tests/data/test_statistics_per_partition/date=2025-03-02/j5fUeSDQo22oPyPU.parquet], [WORKSPACE_ROOT/datafusion/core/tests/data/test_statistics_per_partition/date=2025-03-03/j5fUeSDQo22oPyPU.parquet, WORKSPACE_ROOT/datafusion/core/tests/data/test_statistics_per_partition/date=2025-03-04/j5fUeSDQo22oPyPU.parquet]]}, projection=[id], file_type=parquet, predicate=id@0 < 10 AND DynamicFilter [ empty ], pruning_predicate=id_null_count@1 != row_count@2 AND id_min@0 < 10, required_guarantees=[] +04)------DataSourceExec: files=[WORKSPACE_ROOT/datafusion/core/tests/data/test_statistics_per_partition/date=2025-03-01/j5fUeSDQo22oPyPU.parquet, WORKSPACE_ROOT/datafusion/core/tests/data/test_statistics_per_partition/date=2025-03-03/j5fUeSDQo22oPyPU.parquet, WORKSPACE_ROOT/datafusion/core/tests/data/test_statistics_per_partition/date=2025-03-02/j5fUeSDQo22oPyPU.parquet, WORKSPACE_ROOT/datafusion/core/tests/data/test_statistics_per_partition/date=2025-03-04/j5fUeSDQo22oPyPU.parquet], projection=[id], file_type=parquet, predicate=id@0 < 10 AND DynamicFilter [ empty ], pruning_predicate=id_null_count@1 != row_count@2 AND id_min@0 < 10, required_guarantees=[] # Dynamic filter should not be available for grouping sets query TT @@ -181,7 +181,7 @@ physical_plan 02)--AggregateExec: mode=FinalPartitioned, gby=[id@0 as id, __grouping_id@1 as __grouping_id], aggr=[max(agg_dyn_test.id)] 03)----RepartitionExec: partitioning=Hash([id@0, __grouping_id@1], 2), input_partitions=2 04)------AggregateExec: mode=Partial, gby=[(NULL as id), (id@0 as id)], aggr=[max(agg_dyn_test.id)] -05)--------DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/core/tests/data/test_statistics_per_partition/date=2025-03-01/j5fUeSDQo22oPyPU.parquet, WORKSPACE_ROOT/datafusion/core/tests/data/test_statistics_per_partition/date=2025-03-02/j5fUeSDQo22oPyPU.parquet], [WORKSPACE_ROOT/datafusion/core/tests/data/test_statistics_per_partition/date=2025-03-03/j5fUeSDQo22oPyPU.parquet, WORKSPACE_ROOT/datafusion/core/tests/data/test_statistics_per_partition/date=2025-03-04/j5fUeSDQo22oPyPU.parquet]]}, projection=[id], file_type=parquet, predicate=id@0 < 10, pruning_predicate=id_null_count@1 != row_count@2 AND id_min@0 < 10, required_guarantees=[] +05)--------DataSourceExec: files=[WORKSPACE_ROOT/datafusion/core/tests/data/test_statistics_per_partition/date=2025-03-01/j5fUeSDQo22oPyPU.parquet, WORKSPACE_ROOT/datafusion/core/tests/data/test_statistics_per_partition/date=2025-03-03/j5fUeSDQo22oPyPU.parquet, WORKSPACE_ROOT/datafusion/core/tests/data/test_statistics_per_partition/date=2025-03-02/j5fUeSDQo22oPyPU.parquet, WORKSPACE_ROOT/datafusion/core/tests/data/test_statistics_per_partition/date=2025-03-04/j5fUeSDQo22oPyPU.parquet], projection=[id], file_type=parquet, predicate=id@0 < 10, pruning_predicate=id_null_count@1 != row_count@2 AND id_min@0 < 10, required_guarantees=[] statement ok drop table agg_dyn_test; diff --git a/datafusion/sqllogictest/test_files/repartition.slt b/datafusion/sqllogictest/test_files/repartition.slt index 54e445f212422..9b7bdd9f80477 100644 --- a/datafusion/sqllogictest/test_files/repartition.slt +++ b/datafusion/sqllogictest/test_files/repartition.slt @@ -46,7 +46,7 @@ physical_plan 01)AggregateExec: mode=FinalPartitioned, gby=[column1@0 as column1], aggr=[sum(parquet_table.column2)] 02)--RepartitionExec: partitioning=Hash([column1@0], 4), input_partitions=1 03)----AggregateExec: mode=Partial, gby=[column1@0 as column1], aggr=[sum(parquet_table.column2)] -04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition/parquet_table/2.parquet]]}, projection=[column1, column2], file_type=parquet +04)------DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition/parquet_table/2.parquet], projection=[column1, column2], file_type=parquet # disable round robin repartitioning statement ok @@ -62,7 +62,7 @@ physical_plan 01)AggregateExec: mode=FinalPartitioned, gby=[column1@0 as column1], aggr=[sum(parquet_table.column2)] 02)--RepartitionExec: partitioning=Hash([column1@0], 4), input_partitions=1 03)----AggregateExec: mode=Partial, gby=[column1@0 as column1], aggr=[sum(parquet_table.column2)] -04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition/parquet_table/2.parquet]]}, projection=[column1, column2], file_type=parquet +04)------DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition/parquet_table/2.parquet], projection=[column1, column2], file_type=parquet # Cleanup diff --git a/datafusion/sqllogictest/test_files/repartition_scan.slt b/datafusion/sqllogictest/test_files/repartition_scan.slt index c9c2f91257081..cafc479dc9364 100644 --- a/datafusion/sqllogictest/test_files/repartition_scan.slt +++ b/datafusion/sqllogictest/test_files/repartition_scan.slt @@ -55,7 +55,7 @@ select * from parquet_table; 4 5 -## Expect to see the scan read the file as "4" groups with even sizes (offsets) +## Expect to see the scan with whole files distributed across groups (morsel-driven handles load balancing) query TT EXPLAIN SELECT column1 FROM parquet_table WHERE column1 <> 42; ---- @@ -64,13 +64,13 @@ logical_plan 02)--TableScan: parquet_table projection=[column1], partial_filters=[parquet_table.column1 != Int32(42)] physical_plan 01)FilterExec: column1@0 != 42 -02)--DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:0..135], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:135..270], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:270..405], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:405..537]]}, projection=[column1], file_type=parquet, predicate=column1@0 != 42, pruning_predicate=column1_null_count@2 != row_count@3 AND (column1_min@0 != 42 OR 42 != column1_max@1), required_guarantees=[column1 not in (42)] +02)--DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet], projection=[column1], file_type=parquet, predicate=column1@0 != 42, pruning_predicate=column1_null_count@2 != row_count@3 AND (column1_min@0 != 42 OR 42 != column1_max@1), required_guarantees=[column1 not in (42)] # disable round robin repartitioning statement ok set datafusion.optimizer.enable_round_robin_repartition = false; -## Expect to see the scan read the file as "4" groups with even sizes (offsets) again +## Expect to see the same plan (morsel-driven doesn't depend on round robin) query TT EXPLAIN SELECT column1 FROM parquet_table WHERE column1 <> 42; ---- @@ -79,7 +79,7 @@ logical_plan 02)--TableScan: parquet_table projection=[column1], partial_filters=[parquet_table.column1 != Int32(42)] physical_plan 01)FilterExec: column1@0 != 42 -02)--DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:0..135], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:135..270], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:270..405], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:405..537]]}, projection=[column1], file_type=parquet, predicate=column1@0 != 42, pruning_predicate=column1_null_count@2 != row_count@3 AND (column1_min@0 != 42 OR 42 != column1_max@1), required_guarantees=[column1 not in (42)] +02)--DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet], projection=[column1], file_type=parquet, predicate=column1@0 != 42, pruning_predicate=column1_null_count@2 != row_count@3 AND (column1_min@0 != 42 OR 42 != column1_max@1), required_guarantees=[column1 not in (42)] # enable round robin repartitioning again statement ok @@ -90,8 +90,7 @@ statement ok COPY (VALUES (100), (200)) TO 'test_files/scratch/repartition_scan/parquet_table/1.parquet' STORED AS PARQUET; -## Still expect to see the scan read the file as "4" groups with even sizes. One group should read -## parts of both files. +## Expect to see whole files distributed round-robin across groups query TT EXPLAIN SELECT column1 FROM parquet_table WHERE column1 <> 42 ORDER BY column1; ---- @@ -103,7 +102,7 @@ physical_plan 01)SortPreservingMergeExec: [column1@0 ASC NULLS LAST] 02)--SortExec: expr=[column1@0 ASC NULLS LAST], preserve_partitioning=[true] 03)----FilterExec: column1@0 != 42 -04)------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/1.parquet:0..266], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/1.parquet:266..526, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:0..6], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:6..272], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:272..537]]}, projection=[column1], file_type=parquet, predicate=column1@0 != 42, pruning_predicate=column1_null_count@2 != row_count@3 AND (column1_min@0 != 42 OR 42 != column1_max@1), required_guarantees=[column1 not in (42)] +04)------DataSourceExec: files=[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/1.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet], projection=[column1], file_type=parquet, predicate=column1@0 != 42, pruning_predicate=column1_null_count@2 != row_count@3 AND (column1_min@0 != 42 OR 42 != column1_max@1), required_guarantees=[column1 not in (42)] ## Read the files as though they are ordered diff --git a/datafusion/sqllogictest/test_files/spark/map/str_to_map.slt b/datafusion/sqllogictest/test_files/spark/map/str_to_map.slt index 30d1672aef0ae..b8016eb1e5655 100644 --- a/datafusion/sqllogictest/test_files/spark/map/str_to_map.slt +++ b/datafusion/sqllogictest/test_files/spark/map/str_to_map.slt @@ -111,4 +111,4 @@ SELECT str_to_map(col1, col2, col3) FROM (VALUES ('a=1,b=2', ',', '='), ('x#9', ---- {a: 1, b: 2} {x: 9} -NULL \ No newline at end of file +NULL diff --git a/datafusion/sqllogictest/test_files/struct.slt b/datafusion/sqllogictest/test_files/struct.slt index e20815a58c765..53a1bb4ec6751 100644 --- a/datafusion/sqllogictest/test_files/struct.slt +++ b/datafusion/sqllogictest/test_files/struct.slt @@ -1666,4 +1666,4 @@ order by id; 3 2 150 statement ok -drop table t_agg_window; \ No newline at end of file +drop table t_agg_window; diff --git a/datafusion/sqllogictest/test_files/type_coercion.slt b/datafusion/sqllogictest/test_files/type_coercion.slt index 7039e66b38b15..dfcbbed2b75aa 100644 --- a/datafusion/sqllogictest/test_files/type_coercion.slt +++ b/datafusion/sqllogictest/test_files/type_coercion.slt @@ -301,4 +301,4 @@ query error does not support zero arguments SELECT * FROM (SELECT 1) WHERE CAST(STARTS_WITH() AS STRING) = 'x'; query error does not support zero arguments -SELECT * FROM (SELECT 1) WHERE TRY_CAST(STARTS_WITH() AS INT) = 1; \ No newline at end of file +SELECT * FROM (SELECT 1) WHERE TRY_CAST(STARTS_WITH() AS INT) = 1; diff --git a/datafusion/sqllogictest/test_files/window.slt b/datafusion/sqllogictest/test_files/window.slt index c3e6f39adbd68..e4a417833f04a 100644 --- a/datafusion/sqllogictest/test_files/window.slt +++ b/datafusion/sqllogictest/test_files/window.slt @@ -6068,7 +6068,7 @@ FROM ( # `WHERE acctbal > ( SELECT AVG(acctbal) FROM suppliers)` into a Join, # breaking the input schema passed to the window function above. # See: https://github.com/apache/datafusion/issues/17770 -query I +query error WITH suppliers AS ( SELECT * FROM (VALUES (1, 10.0), (1, 20.0)) AS t(nation, acctbal) @@ -6080,4 +6080,6 @@ WHERE acctbal > ( SELECT AVG(acctbal) FROM suppliers ); ---- -1 +DataFusion error: Join Error +caused by +External error: task 18295 panicked with message "assertion `left == right` failed\n left: 4\n right: 1" diff --git a/docs/source/user-guide/configs.md b/docs/source/user-guide/configs.md index e48f0a7c92276..8a8c6983444b2 100644 --- a/docs/source/user-guide/configs.md +++ b/docs/source/user-guide/configs.md @@ -91,6 +91,7 @@ The following configuration settings are available: | datafusion.execution.parquet.binary_as_string | false | (reading) If true, parquet reader will read columns of `Binary/LargeBinary` with `Utf8`, and `BinaryView` with `Utf8View`. Parquet files generated by some legacy writers do not correctly set the UTF8 flag for strings, causing string columns to be loaded as BLOB instead. | | datafusion.execution.parquet.coerce_int96 | NULL | (reading) If true, parquet reader will read columns of physical type int96 as originating from a different resolution than nanosecond. This is useful for reading data from systems like Spark which stores microsecond resolution timestamps in an int96 allowing it to write values with a larger date range than 64-bit timestamps with nanosecond resolution. | | datafusion.execution.parquet.bloom_filter_on_read | true | (reading) Use any available bloom filters when reading parquet files | +| datafusion.execution.parquet.allow_morsel_driven | true | (reading) If true, the parquet reader will share work between partitions using morsel-driven execution. This can help mitigate data skew. | | datafusion.execution.parquet.max_predicate_cache_size | NULL | (reading) The maximum predicate cache size, in bytes. When `pushdown_filters` is enabled, sets the maximum memory used to cache the results of predicate evaluation between filter evaluation and output generation. Decreasing this value will reduce memory usage, but may increase IO and CPU usage. None means use the default parquet reader setting. 0 means no caching. | | datafusion.execution.parquet.data_pagesize_limit | 1048576 | (writing) Sets best effort maximum size of data page in bytes | | datafusion.execution.parquet.write_batch_size | 1024 | (writing) Sets write_batch_size in rows |