diff --git a/datafusion/core/tests/parquet/dynamic_row_group_pruning.rs b/datafusion/core/tests/parquet/dynamic_row_group_pruning.rs new file mode 100644 index 0000000000000..d578f915ac09b --- /dev/null +++ b/datafusion/core/tests/parquet/dynamic_row_group_pruning.rs @@ -0,0 +1,208 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +//! End-to-end test for **runtime row-group pruning** driven by a TopK +//! `SortExec`'s `DynamicFilterPhysicalExpr`. +//! +//! A 5-row-group parquet file is constructed with disjoint statistics on +//! the sort column (`v`): row group `i` contains values +//! `[i*100, (i+1)*100)`. The query `ORDER BY v DESC LIMIT 5` fills the +//! TopK heap from the row group with the largest values; the threshold +//! then proves the remaining row groups cannot contribute. The runtime +//! `RowGroupPruner` in the parquet scan must observe the tightened +//! threshold and increment `row_groups_pruned_dynamic_filter`. +//! +//! We assert a property (`pruned >= 1`) rather than an exact count +//! because batch-arrival timing affects how soon the TopK heap fills, +//! and we don't want this test to become flaky. + +use std::sync::Arc; + +use arrow::array::{ArrayRef, Int64Array, RecordBatch}; +use arrow_schema::{DataType, Field, Schema}; + +use crate::parquet::Unit::RowGroup; +use crate::parquet::{ContextWithParquet, Scenario}; + +/// Build five `RecordBatch`es whose `v` column ranges are disjoint: +/// batch `i` carries `v` values `[i*100, (i+1)*100)`. When written with +/// `max_row_group_row_count = 100` each batch lands in its own row group. +fn build_five_disjoint_batches(schema: &Arc) -> Vec { + (0..5i64) + .map(|rg| { + let base = rg * 100; + let values: Vec = (base..base + 100).collect(); + let col: ArrayRef = Arc::new(Int64Array::from(values)); + RecordBatch::try_new(Arc::clone(schema), vec![col]).unwrap() + }) + .collect() +} + +/// Build five `RecordBatch`es in *descending* value order: batch 0 holds +/// `v ∈ [400, 500)`, batch 4 holds `v ∈ [0, 100)`. The physical row-group +/// order on disk therefore does **not** match the order a `ORDER BY v ASC` +/// query wants — sort-pushdown's `reorder_by_statistics` must rearrange +/// the access plan so the scan reads RG 4 first, then RG 3, etc. +fn build_five_disjoint_batches_desc(schema: &Arc) -> Vec { + (0..5i64) + .map(|rg| { + let base = (4 - rg) * 100; + let values: Vec = (base..base + 100).collect(); + let col: ArrayRef = Arc::new(Int64Array::from(values)); + RecordBatch::try_new(Arc::clone(schema), vec![col]).unwrap() + }) + .collect() +} + +/// `ORDER BY v DESC LIMIT 5` against a 5-RG file with disjoint per-RG +/// stats must trigger runtime RG pruning: the first RG read fills the +/// heap, and the tightened threshold proves every other RG unreachable. +#[tokio::test] +async fn dynamic_rg_pruning_metric_fires_for_topk_descending_limit() { + let schema = Arc::new(Schema::new(vec![Field::new("v", DataType::Int64, false)])); + let batches = build_five_disjoint_batches(&schema); + + // `with_custom_data` honors the custom schema + batches and ignores + // `Scenario`. `Unit::RowGroup(100)` enables `pushdown_filters`, which + // is required for the TopK dynamic filter to reach the parquet scan. + let mut ctx = ContextWithParquet::with_custom_data( + Scenario::Int, + RowGroup(100), + Arc::clone(&schema), + batches, + ) + .await; + + let output = ctx.query("SELECT v FROM t ORDER BY v DESC LIMIT 5").await; + + assert_eq!(output.result_rows, 5, "query must return LIMIT rows",); + + let pruned = output + .row_groups_pruned_dynamic_filter() + .expect("`row_groups_pruned_dynamic_filter` metric must be registered"); + assert!( + pruned >= 1, + "dynamic RG pruner must skip at least one row group; \ + pruned={pruned}\n{}", + output.description(), + ); +} + +/// Regression for the rg_plan / `reorder_by_statistics` ordering bug. +/// +/// When `sort_order_for_reorder` is set on the parquet scan, +/// `prepare_access_plan` calls +/// [`PreparedAccessPlan::reorder_by_statistics`], which rearranges +/// `row_group_indexes` so the decoder reads row groups in stats-optimal +/// order (smallest-min first for ASC, etc.). The stream's per-RG plan +/// (`rg_plan`) — which the runtime pruner walks one entry at a time — +/// **must use this reordered list**, not the access plan's natural +/// (index-ascending) order. Otherwise the pruner would consult the +/// metadata of RG K while the decoder is actually about to yield RG K', +/// silently producing wrong results. +/// +/// This test makes the failure visible: +/// +/// - File is written with RGs in *descending* `v` order (RG 0 has the +/// largest values, RG 4 has the smallest). +/// - Query is `ORDER BY v ASC LIMIT 5`, so sort-pushdown reorders the +/// access plan to read RG 4 first, then RG 3, etc. +/// - The smallest five values (which form the entire correct LIMIT +/// answer) live in RG 4 alone. After they are emitted, the TopK +/// threshold tightens enough that the per-RG pruner skips every other +/// RG. +/// +/// Without the fix, `rg_plan` would be `[0, 1, 2, 3, 4]` while the +/// decoder reads `[4, 3, 2, 1, 0]`. The first yielded reader (for RG 4 +/// in the decoder) would be tracked as if it were RG 0, the pruner +/// would check RG 1's stats (id range 300..400) against a threshold +/// already tightened to `v < 5`, prune RG 1 (because nothing in +/// 300..400 can satisfy `v < 5`), and then the rebuild via +/// `into_builder` would scan a row group whose data does not match its +/// expected metadata. The query would return fewer than five rows or +/// the wrong rows. +#[tokio::test] +async fn dynamic_rg_pruning_handles_sort_pushdown_reorder() { + let schema = Arc::new(Schema::new(vec![Field::new("v", DataType::Int64, false)])); + let batches = build_five_disjoint_batches_desc(&schema); + + let mut ctx = ContextWithParquet::with_custom_data( + Scenario::Int, + RowGroup(100), + Arc::clone(&schema), + batches, + ) + .await; + + let output = ctx.query("SELECT v FROM t ORDER BY v ASC LIMIT 5").await; + + // Correctness — the five smallest values in the file are 0..=4. + // If `rg_plan` is misaligned with the decoder's read order, the + // pruner consults the wrong RG's stats and the result row count or + // values would drift. + assert_eq!(output.result_rows, 5, "query must return LIMIT rows"); + let formatted = output.pretty_results(); + for v in 0..=4i64 { + assert!( + formatted.contains(&format!("| {v} ")), + "output must contain the smallest value {v}; got:\n{formatted}", + ); + } + + // Behavior — the per-RG pruner must engage. We don't pin the exact + // count (batch-arrival timing affects how soon the heap fills); we + // only require that at least one row group is skipped at runtime. + let pruned = output + .row_groups_pruned_dynamic_filter() + .expect("`row_groups_pruned_dynamic_filter` metric must be registered"); + assert!( + pruned >= 1, + "with `sort_order_for_reorder` active and a tight TopK, the \ + runtime pruner must skip at least one row group; pruned={pruned}\n{}", + output.description(), + ); +} + +/// A query without ORDER BY does not produce a TopK and therefore no +/// `DynamicFilterPhysicalExpr` reaches the scan. The runtime pruner must +/// stay quiet — the metric should be 0. +#[tokio::test] +async fn dynamic_rg_pruning_metric_quiet_without_topk() { + let schema = Arc::new(Schema::new(vec![Field::new("v", DataType::Int64, false)])); + let batches = build_five_disjoint_batches(&schema); + + let mut ctx = ContextWithParquet::with_custom_data( + Scenario::Int, + RowGroup(100), + Arc::clone(&schema), + batches, + ) + .await; + + // Plain `SELECT *` — no sort, no limit, no dynamic filter. + let output = ctx.query("SELECT v FROM t").await; + assert_eq!(output.result_rows, 500); + + let pruned = output.row_groups_pruned_dynamic_filter().unwrap_or(0); + assert_eq!( + pruned, + 0, + "without TopK there is no dynamic filter, so the runtime pruner \ + must not fire; pruned={pruned}\n{}", + output.description(), + ); +} diff --git a/datafusion/core/tests/parquet/mod.rs b/datafusion/core/tests/parquet/mod.rs index 12296f8498d9f..41225296434a9 100644 --- a/datafusion/core/tests/parquet/mod.rs +++ b/datafusion/core/tests/parquet/mod.rs @@ -46,6 +46,7 @@ use tempfile::NamedTempFile; mod content_defined_chunking; mod custom_reader; +mod dynamic_row_group_pruning; #[cfg(feature = "parquet_encryption")] mod encryption; mod expr_adapter; @@ -147,6 +148,12 @@ struct TestOutput { } impl TestOutput { + /// Pretty-printed result batches, useful for asserting concrete row + /// values in regression tests. + fn pretty_results(&self) -> &str { + &self.pretty_results + } + /// retrieve the value of the named metric, if any fn metric_value(&self, metric_name: &str) -> Option { if let Some(pm) = self.pruning_metric(metric_name) { @@ -259,6 +266,13 @@ impl TestOutput { .map(|pm| pm.total_pruned()) } + /// The number of row groups pruned at runtime by the dynamic + /// row-group pruner (e.g. driven by a TopK `SortExec` threshold + /// pushed down via `DynamicFilterPhysicalExpr`). + fn row_groups_pruned_dynamic_filter(&self) -> Option { + self.metric_value("row_groups_pruned_dynamic_filter") + } + fn description(&self) -> String { format!( "Input:\n{}\nQuery:\n{}\nOutput:\n{}\nMetrics:\n{}", diff --git a/datafusion/datasource-parquet/src/access_plan.rs b/datafusion/datasource-parquet/src/access_plan.rs index edbea39948f09..109ecf18eabc8 100644 --- a/datafusion/datasource-parquet/src/access_plan.rs +++ b/datafusion/datasource-parquet/src/access_plan.rs @@ -115,24 +115,6 @@ pub enum RowGroupAccess { Selection(RowSelection), } -/// A consecutive set of row groups that share the same row filter requirement. -#[derive(Debug, Clone, PartialEq)] -pub(crate) struct RowGroupRun { - /// True if this run needs row filter evaluation. - pub(crate) needs_filter: bool, - /// The access plan for this run. - pub(crate) access_plan: ParquetAccessPlan, -} - -impl RowGroupRun { - fn new(needs_filter: bool, access_plan: ParquetAccessPlan) -> Self { - Self { - needs_filter, - access_plan, - } - } -} - impl RowGroupAccess { /// Return true if this row group should be scanned pub fn should_scan(&self) -> bool { @@ -213,12 +195,6 @@ impl ParquetAccessPlan { &self.fully_matched } - /// Return true if any scanned row group is fully matched. - fn has_fully_matched(&self) -> bool { - self.row_group_index_iter() - .any(|idx| self.is_fully_matched(idx)) - } - /// Set to scan only the [`RowSelection`] in the specified row group. /// /// Behavior is different depending on the existing access @@ -404,54 +380,6 @@ impl ParquetAccessPlan { self.row_groups } - /// Split this plan into consecutive row group runs that share the same row - /// filter requirement. - pub(crate) fn split_runs(self, needs_filter: bool) -> Vec { - if !needs_filter || !self.has_fully_matched() { - return vec![RowGroupRun::new(needs_filter, self)]; - } - - let num_row_groups = self.row_groups.len(); - let row_groups = self.row_groups; - let fully_matched = self.fully_matched; - let mut runs: Vec = Vec::new(); - - for (idx, (access, fully_matched)) in - row_groups.into_iter().zip(fully_matched).enumerate() - { - if !access.should_scan() { - continue; - } - - let row_group_needs_filter = !fully_matched; - if let Some(run) = runs - .last_mut() - .filter(|run| run.needs_filter == row_group_needs_filter) - { - run.access_plan.set(idx, access); - if fully_matched { - run.access_plan.mark_fully_matched(idx); - } - } else { - let mut run_plan = ParquetAccessPlan::new_none(num_row_groups); - run_plan.set(idx, access); - if fully_matched { - run_plan.mark_fully_matched(idx); - } - runs.push(RowGroupRun::new(row_group_needs_filter, run_plan)); - } - } - - if runs.is_empty() { - vec![RowGroupRun::new( - needs_filter, - ParquetAccessPlan::new_none(num_row_groups), - )] - } else { - runs - } - } - /// Prepare this plan and resolve to the final `PreparedAccessPlan` pub(crate) fn prepare( self, diff --git a/datafusion/datasource-parquet/src/metrics.rs b/datafusion/datasource-parquet/src/metrics.rs index 4bf009afd6d63..2a7be3db8a0ea 100644 --- a/datafusion/datasource-parquet/src/metrics.rs +++ b/datafusion/datasource-parquet/src/metrics.rs @@ -53,6 +53,14 @@ pub struct ParquetFileMetrics { pub limit_pruned_row_groups: PruningMetrics, /// Number of row groups pruned by statistics pub row_groups_pruned_statistics: PruningMetrics, + /// Number of row groups pruned at runtime by a dynamic predicate + /// (e.g. the threshold expression a TopK `SortExec` pushes down). + /// + /// Unlike [`Self::row_groups_pruned_statistics`], which is decided once + /// at access-plan time, this counter reflects row groups that survived + /// the initial pruning but were proved unreachable mid-scan after the + /// dynamic filter tightened. + pub row_groups_pruned_dynamic_filter: Count, /// Total number of bytes scanned pub bytes_scanned: Count, /// Total rows filtered out by predicates pushed into parquet scan @@ -198,6 +206,11 @@ impl ParquetFileMetrics { .with_category(MetricCategory::Rows) .gauge("predicate_cache_records", partition); + let row_groups_pruned_dynamic_filter = MetricBuilder::new(metrics) + .with_new_label("filename", filename.to_string()) + .with_type(MetricType::Summary) + .counter("row_groups_pruned_dynamic_filter", partition); + Self { files_ranges_pruned_statistics, predicate_evaluation_errors, @@ -217,6 +230,7 @@ impl ParquetFileMetrics { scan_efficiency_ratio, predicate_cache_inner_records, predicate_cache_records, + row_groups_pruned_dynamic_filter, } } diff --git a/datafusion/datasource-parquet/src/opener/mod.rs b/datafusion/datasource-parquet/src/opener/mod.rs index 5b517663f9c03..c2250a27a2d76 100644 --- a/datafusion/datasource-parquet/src/opener/mod.rs +++ b/datafusion/datasource-parquet/src/opener/mod.rs @@ -26,7 +26,9 @@ use self::encryption::EncryptionContext; use crate::access_plan::PreparedAccessPlan; use crate::decoder_projection::DecoderProjection; use crate::page_filter::PagePruningAccessPlanFilter; -use crate::push_decoder::{DecoderBuilderConfig, PushDecoderStreamState}; +use crate::push_decoder::{ + DecoderBuilderConfig, PushDecoderStreamState, RgPlanEntry, RowGroupPruner, +}; use crate::row_filter::RowFilterGenerator; use crate::row_group_filter::{BloomFilterStatistics, RowGroupAccessPlanFilter}; use crate::{ @@ -1353,7 +1355,7 @@ impl RowGroupsPrunedParquetOpen { prepared.virtual_state.as_deref(), )?; - let (decoder, pending_decoders, remaining_limit) = { + let (decoder, rg_plan) = { let pushdown_predicate = prepared .pushdown_filters .then_some(prepared.predicate.as_ref()) @@ -1366,50 +1368,41 @@ impl RowGroupsPrunedParquetOpen { &prepared.file_metrics, ); - // Split into consecutive runs of row groups that share the same filter - // requirement. Fully matched row groups skip the RowFilter; others need it. - // Reverse the run order for reverse scans so the combined decoder stream - // preserves the requested global row group order. - let mut runs = access_plan.split_runs(row_filter_generator.has_row_filter()); - if prepared.reverse_row_groups { - runs.reverse(); - } - let run_count = runs.len(); - let decoder_limit = prepared.limit.filter(|_| run_count == 1); - let remaining_limit = prepared.limit.filter(|_| run_count > 1); - + // Build the prepared access plan first — `prepare_access_plan` may + // call `reorder_by_statistics` (for `sort_order_for_reorder`) and + // `reverse` (for `reverse_row_groups`), both of which mutate + // `row_group_indexes` to the physical scan order the decoder will + // actually read. We MUST build our `rg_plan` from this reordered + // list, otherwise our per-RG pruner check would consult the + // metadata of a different RG than the decoder is about to yield. let decoder_config = DecoderBuilderConfig { projection_mask: decoder_projection.projection_mask(), batch_size: prepared.batch_size, arrow_reader_metrics: &arrow_reader_metrics, force_filter_selections: prepared.force_filter_selections, - decoder_limit, + decoder_limit: prepared.limit, }; - // Build a decoder per run. - let mut decoders = VecDeque::with_capacity(runs.len()); - for run in runs { - let prepared_access_plan = prepare_access_plan(run.access_plan)?; - let mut builder = - decoder_config.build(prepared_access_plan, reader_metadata.clone()); - if run.needs_filter { - if let Some(row_filter) = row_filter_generator.next_filter() { - builder = builder.with_row_filter(row_filter); - } - if let Some(max_predicate_cache_size) = - prepared.max_predicate_cache_size - { - builder = builder - .with_max_predicate_cache_size(max_predicate_cache_size); - } + let prepared_access_plan = prepare_access_plan(access_plan)?; + let rg_plan: VecDeque = prepared_access_plan + .row_group_indexes + .iter() + .copied() + .map(|rg_index| RgPlanEntry { rg_index }) + .collect(); + + let mut builder = + decoder_config.build(prepared_access_plan, reader_metadata.clone()); + if let Some(row_filter) = row_filter_generator.next_filter() { + builder = builder.with_row_filter(row_filter); + if let Some(max_predicate_cache_size) = prepared.max_predicate_cache_size + { + builder = + builder.with_max_predicate_cache_size(max_predicate_cache_size); } - decoders.push_back(builder.build()?); } - let decoder = decoders - .pop_front() - .expect("at least one decoder must be created"); - (decoder, decoders, remaining_limit) + (builder.build()?, rg_plan) }; let predicate_cache_inner_records = @@ -1419,16 +1412,40 @@ impl RowGroupsPrunedParquetOpen { let files_ranges_pruned_statistics = prepared.file_metrics.files_ranges_pruned_statistics.clone(); + + // Build a dynamic row-group pruner only when both halves are useful: + // 1) the scan has a predicate (so there is something to evaluate), + // 2) there is at least one pending run that could be skipped. + // The pruner consults the predicate's `snapshot_generation` so the + // cost is one rebuild per dynamic-filter update, not per RG check. + let row_group_pruner = match (&prepared.predicate, rg_plan.len() > 1) { + (Some(predicate), true) => Some(RowGroupPruner::new( + Arc::clone(predicate), + Arc::clone(&prepared.physical_file_schema), + Arc::clone(reader_metadata.metadata()), + prepared.predicate_creation_errors.clone(), + prepared.file_metrics.predicate_evaluation_errors.clone(), + )), + _ => None, + }; + let row_groups_pruned_dynamic = prepared + .file_metrics + .row_groups_pruned_dynamic_filter + .clone(); + let stream = PushDecoderStreamState { - decoder, - pending_decoders, - remaining_limit, + decoder: Some(decoder), + active_reader: None, + rg_plan, + remaining_limit: None, reader: prepared.async_file_reader, decoder_projection, arrow_reader_metrics, predicate_cache_inner_records, predicate_cache_records, baseline_metrics: prepared.baseline_metrics, + row_group_pruner, + row_groups_pruned_dynamic, } .into_stream(); @@ -2880,92 +2897,6 @@ mod test { assert_eq!(values, vec![7, 4, 5, 6, 3]); } - #[test] - fn test_split_decoder_runs_no_fully_matched() { - // All row groups need filtering: single run. - let plan = ParquetAccessPlan::new(vec![ - RowGroupAccess::Scan, - RowGroupAccess::Scan, - RowGroupAccess::Scan, - ]); - let runs = plan.split_runs(true); - assert_eq!(runs.len(), 1); - assert!(runs[0].needs_filter); - assert_eq!(runs[0].access_plan.row_group_indexes(), vec![0, 1, 2]); - } - - #[test] - fn test_split_decoder_runs_all_fully_matched() { - // All row groups are fully matched: single run, no filter. - let mut plan = ParquetAccessPlan::new(vec![ - RowGroupAccess::Scan, - RowGroupAccess::Scan, - RowGroupAccess::Scan, - ]); - plan.mark_fully_matched(0); - plan.mark_fully_matched(1); - plan.mark_fully_matched(2); - - let runs = plan.split_runs(true); - assert_eq!(runs.len(), 1); - assert!(!runs[0].needs_filter); - assert_eq!(runs[0].access_plan.row_group_indexes(), vec![0, 1, 2]); - } - - #[test] - fn test_split_decoder_runs_mixed() { - // [F, M, M, F, M] creates 4 runs preserving order. - let mut plan = ParquetAccessPlan::new(vec![ - RowGroupAccess::Scan, // 0: filtered - RowGroupAccess::Scan, // 1: matched - RowGroupAccess::Scan, // 2: matched - RowGroupAccess::Scan, // 3: filtered - RowGroupAccess::Scan, // 4: matched - ]); - plan.mark_fully_matched(1); - plan.mark_fully_matched(2); - plan.mark_fully_matched(4); - - let runs = plan.split_runs(true); - assert_eq!(runs.len(), 4); - - assert!(runs[0].needs_filter); - assert_eq!(runs[0].access_plan.row_group_indexes(), vec![0]); - - assert!(!runs[1].needs_filter); - assert_eq!(runs[1].access_plan.row_group_indexes(), vec![1, 2]); - - assert!(runs[2].needs_filter); - assert_eq!(runs[2].access_plan.row_group_indexes(), vec![3]); - - assert!(!runs[3].needs_filter); - assert_eq!(runs[3].access_plan.row_group_indexes(), vec![4]); - } - - #[test] - fn test_split_decoder_runs_with_skipped_groups() { - // Skipped row groups are excluded from all runs. - let mut plan = ParquetAccessPlan::new(vec![ - RowGroupAccess::Scan, // 0: filtered - RowGroupAccess::Skip, // 1: pruned - RowGroupAccess::Scan, // 2: matched - RowGroupAccess::Scan, // 3: filtered - ]); - plan.mark_fully_matched(2); - - let runs = plan.split_runs(true); - assert_eq!(runs.len(), 3); - - assert!(runs[0].needs_filter); - assert_eq!(runs[0].access_plan.row_group_indexes(), vec![0]); - - assert!(!runs[1].needs_filter); - assert_eq!(runs[1].access_plan.row_group_indexes(), vec![2]); - - assert!(runs[2].needs_filter); - assert_eq!(runs[2].access_plan.row_group_indexes(), vec![3]); - } - /// Helpers for tests that exercise parquet virtual columns /// (e.g. `row_number`) plumbed through `TableSchema`/`ParquetOpener`. mod virtual_columns { diff --git a/datafusion/datasource-parquet/src/push_decoder.rs b/datafusion/datasource-parquet/src/push_decoder.rs index 3156b9e35fe24..95301044f4b24 100644 --- a/datafusion/datasource-parquet/src/push_decoder.rs +++ b/datafusion/datasource-parquet/src/push_decoder.rs @@ -32,22 +32,32 @@ //! [`PushDecoderStreamState::into_stream`] for consumption. use std::collections::VecDeque; +use std::sync::Arc; use arrow::array::RecordBatch; +use arrow::datatypes::SchemaRef; use futures::StreamExt; use futures::stream::BoxStream; +use log::debug; use parquet::DecodeResult; use parquet::arrow::ProjectionMask; use parquet::arrow::arrow_reader::metrics::ArrowReaderMetrics; -use parquet::arrow::arrow_reader::{ArrowReaderMetadata, RowSelectionPolicy}; +use parquet::arrow::arrow_reader::{ + ArrowReaderMetadata, ParquetRecordBatchReader, RowSelectionPolicy, +}; use parquet::arrow::async_reader::AsyncFileReader; use parquet::arrow::push_decoder::{ParquetPushDecoder, ParquetPushDecoderBuilder}; +use parquet::file::metadata::ParquetMetaData; use datafusion_common::{DataFusionError, Result}; -use datafusion_physical_plan::metrics::{BaselineMetrics, Gauge}; +use datafusion_physical_expr::expressions::DynamicFilterTracking; +use datafusion_physical_expr_common::physical_expr::PhysicalExpr; +use datafusion_physical_plan::metrics::{BaselineMetrics, Count, Gauge}; +use datafusion_pruning::{PruningPredicate, build_pruning_predicate}; use crate::access_plan::PreparedAccessPlan; use crate::decoder_projection::DecoderProjection; +use crate::row_group_filter::RowGroupPruningStatistics; /// Shared options applied to every [`ParquetPushDecoderBuilder`] in a file scan. /// @@ -94,6 +104,133 @@ impl DecoderBuilderConfig<'_> { } } +#[derive(Debug, Clone)] +pub(crate) struct RgPlanEntry { + pub(crate) rg_index: usize, +} + +/// Runtime row-group pruner driven by a dynamic predicate (e.g. the +/// threshold expression a `TopK` operator pushes down). +/// +/// Mirrors the [`FilePruner`](datafusion_pruning::FilePruner) pattern at +/// the row-group level: subscribes once to every still-incomplete dynamic +/// filter inside the predicate via +/// [`DynamicFilterTracker`](datafusion_physical_expr::expressions::DynamicFilterTracker) +/// and only rebuilds the [`PruningPredicate`] when one of those +/// subscriptions reports an update, then evaluates the cached predicate +/// against the statistics of the requested row groups. +pub(crate) struct RowGroupPruner { + predicate: Arc, + arrow_schema: SchemaRef, + parquet_metadata: Arc, + /// Classifies the predicate's dynamic-filter content. The `Watching` + /// variant carries a tracker that subscribes to every not-yet-complete + /// dynamic filter; for `Static` / `AllComplete` the predicate cannot + /// change so a single up-front `pruning_predicate` build suffices. + tracking: DynamicFilterTracking, + /// First-call sentinel: forces an initial `pruning_predicate` build + /// even when `tracking` is `Static` / `AllComplete`. + needs_initial_build: bool, + /// Cached pruning predicate. `None` means we couldn't build one for the + /// current generation (e.g. the predicate has no analyzable bounds); + /// in that case we conservatively don't prune. + pruning_predicate: Option>, + /// Metric for `build_pruning_predicate` failures (predicate creation). + predicate_creation_errors: Count, + /// Metric for `PruningPredicate::prune` failures (evaluating an + /// already-built predicate against row-group statistics). + predicate_evaluation_errors: Count, +} + +impl RowGroupPruner { + pub(crate) fn new( + predicate: Arc, + arrow_schema: SchemaRef, + parquet_metadata: Arc, + predicate_creation_errors: Count, + predicate_evaluation_errors: Count, + ) -> Self { + let tracking = DynamicFilterTracking::classify(&predicate); + Self { + predicate, + arrow_schema, + parquet_metadata, + tracking, + needs_initial_build: true, + pruning_predicate: None, + predicate_creation_errors, + predicate_evaluation_errors, + } + } + + /// Returns `true` when the statistics for `row_group_indices` prove that + /// every requested row group can be skipped under the current value of + /// the dynamic predicate. + /// + /// On any error (predicate construction, statistics evaluation) the + /// pruner conservatively returns `false` and logs the failure, so a + /// flaky pruning path never silently drops data. + pub(crate) fn should_prune(&mut self, row_group_indices: &[usize]) -> bool { + if row_group_indices.is_empty() { + return false; + } + + // Refresh the cached `PruningPredicate` on the first call and + // whenever a watched dynamic filter has advanced since we last + // looked. `changed()` is a single atomic load per still-incomplete + // filter — no tree walk on every check. + let dynamic_changed = self + .tracking + .watcher() + .is_some_and(|tracker| tracker.changed()); + if self.needs_initial_build || dynamic_changed { + self.pruning_predicate = build_pruning_predicate( + Arc::clone(&self.predicate), + &self.arrow_schema, + &self.predicate_creation_errors, + ); + self.needs_initial_build = false; + } + + let Some(pp) = self.pruning_predicate.as_ref() else { + return false; + }; + + let row_group_metadatas = row_group_indices + .iter() + .map(|&i| self.parquet_metadata.row_group(i)) + .collect::>(); + let stats = RowGroupPruningStatistics { + parquet_schema: self.parquet_metadata.file_metadata().schema_descr(), + row_group_metadatas, + arrow_schema: self.arrow_schema.as_ref(), + // Match the existing static row-group pruning behavior: when a + // statistic's null count is missing, treat it as zero. This is + // sound for runtime pruning because the predicate only needs to + // prove a row group *cannot* contain matching rows. + missing_null_counts_as_zero: true, + }; + + match pp.prune(&stats) { + // `prune` returns `false` per container that the predicate proves + // cannot contain matching rows. We can skip the run only when + // every requested row group is in that state. + Ok(values) => values.iter().all(|&keep| !keep), + Err(e) => { + // The predicate was already built successfully (we hold `pp`); + // this failure is in *evaluating* it against the row-group + // stats, so it belongs in the evaluation-errors counter, not + // creation-errors. + debug!( + "Ignoring error evaluating runtime row-group pruning predicate: {e}" + ); + self.predicate_evaluation_errors.add(1); + false + } + } + } +} + /// State for a stream that decodes a single Parquet file using a push-based decoder. /// /// The [`transition`](Self::transition) method drives the decoder in a loop: it requests @@ -101,11 +238,9 @@ impl DecoderBuilderConfig<'_> { /// [`ParquetPushDecoder`], and yields projected [`RecordBatch`]es until the file is /// fully consumed. pub(crate) struct PushDecoderStreamState { - pub(crate) decoder: ParquetPushDecoder, - /// Additional decoders to process after the current one finishes. - /// Used when fully matched row groups split the scan into consecutive - /// runs with different filter configurations, maintaining original order. - pub(crate) pending_decoders: VecDeque, + pub(crate) decoder: Option, + pub(crate) active_reader: Option, + pub(crate) rg_plan: VecDeque, /// Global remaining row limit across all decoder runs. /// /// Decoder-local limits are only safe for single-run scans. When the scan @@ -120,6 +255,17 @@ pub(crate) struct PushDecoderStreamState { pub(crate) predicate_cache_inner_records: Gauge, pub(crate) predicate_cache_records: Gauge, pub(crate) baseline_metrics: BaselineMetrics, + /// Dynamic row-group pruner consulted between decoder runs. + /// + /// When the file scan was opened with a dynamic predicate (typically the + /// threshold expression a `TopK` `SortExec` pushed down), we re-evaluate + /// that predicate against each pending run's row-group statistics at run + /// boundaries and skip runs the current threshold proves cannot + /// contribute to the result. `None` when the scan has no dynamic + /// predicate or only one decoder run. + pub(crate) row_group_pruner: Option, + /// Count of row groups skipped at runtime by [`Self::row_group_pruner`]. + pub(crate) row_groups_pruned_dynamic: Count, } impl PushDecoderStreamState { @@ -151,7 +297,72 @@ impl PushDecoderStreamState { if self.remaining_limit == Some(0) { return None; } - match self.decoder.try_decode() { + + // Step 1: drain a batch from the active reader if any. + if let Some(reader) = self.active_reader.as_mut() { + match reader.next() { + Some(Ok(batch)) => { + let mut timer = self.baseline_metrics.elapsed_compute().timer(); + self.copy_arrow_reader_metrics(); + let result = self.project_batch(&batch); + timer.stop(); + drop(timer); + return Some((result, self)); + } + Some(Err(e)) => { + return Some((Err(DataFusionError::from(e)), self)); + } + None => { + // Reader exhausted: drop and fall through to per-RG + // boundary handling, then try_next_reader. + self.active_reader = None; + } + } + } + + // Step 2: at RG boundary, drop pruned RGs from the head of the + // plan. Rebuild the decoder via `into_builder` so it skips the + // pruned RGs entirely. Buffered bytes for already-fetched RGs + // carry across the rebuild. + if !self.rg_plan.is_empty() { + let mut pruned_head = 0usize; + while let Some(next) = self.rg_plan.front() { + let pruned = self + .row_group_pruner + .as_mut() + .map(|p| p.should_prune(&[next.rg_index])) + .unwrap_or(false); + if pruned { + pruned_head += 1; + self.rg_plan.pop_front(); + self.row_groups_pruned_dynamic.add(1); + } else { + break; + } + } + if pruned_head > 0 { + if self.rg_plan.is_empty() { + return None; + } + let decoder = self.decoder.take().expect("decoder present"); + let new_indices: Vec = + self.rg_plan.iter().map(|e| e.rg_index).collect(); + let rebuilt = match decoder.into_builder() { + Ok(b) => b.with_row_groups(new_indices).build(), + Err(e) => Err(e), + }; + match rebuilt { + Ok(d) => self.decoder = Some(d), + Err(e) => { + return Some((Err(DataFusionError::from(e)), self)); + } + } + } + } + + // Step 3: drive the decoder. + let decoder = self.decoder.as_mut().expect("decoder present"); + match decoder.try_next_reader() { Ok(DecodeResult::NeedsData(ranges)) => { let data = self .reader @@ -160,43 +371,26 @@ impl PushDecoderStreamState { .map_err(DataFusionError::from); match data { Ok(data) => { - if let Err(e) = self.decoder.push_ranges(ranges, data) { + if let Err(e) = self + .decoder + .as_mut() + .expect("decoder present") + .push_ranges(ranges, data) + { return Some((Err(DataFusionError::from(e)), self)); } } Err(e) => return Some((Err(e), self)), } } - Ok(DecodeResult::Data(batch)) => { - let batch = if let Some(remaining_limit) = self.remaining_limit { - if batch.num_rows() > remaining_limit { - self.remaining_limit = Some(0); - batch.slice(0, remaining_limit) - } else { - self.remaining_limit = - Some(remaining_limit - batch.num_rows()); - batch - } - } else { - batch - }; - let mut timer = self.baseline_metrics.elapsed_compute().timer(); - self.copy_arrow_reader_metrics(); - let result = self.project_batch(&batch); - timer.stop(); - // Release the borrow on baseline_metrics before moving self - drop(timer); - return Some((result, self)); - } - Ok(DecodeResult::Finished) => { - // If there are pending decoders (e.g. for consecutive runs - // with different filter configurations), switch to the next. - if let Some(next) = self.pending_decoders.pop_front() { - self.decoder = next; - continue; - } - return None; + Ok(DecodeResult::Data(reader)) => { + // Pop the RG this reader is for (we already filtered + // pruned ones in step 2, so `rg_plan.front()` is the RG + // the decoder is about to read). + self.rg_plan.pop_front(); + self.active_reader = Some(reader); } + Ok(DecodeResult::Finished) => return None, Err(e) => { return Some((Err(DataFusionError::from(e)), self)); } @@ -219,3 +413,178 @@ impl PushDecoderStreamState { self.decoder_projection.map(batch) } } + +#[cfg(test)] +mod tests { + use super::*; + + use arrow::array::{Int64Array, RecordBatch}; + use arrow::datatypes::{DataType, Field, Schema}; + use bytes::Bytes; + use datafusion_common::ScalarValue; + use datafusion_expr::Operator; + use datafusion_physical_expr::expressions::{ + BinaryExpr, Column, DynamicFilterPhysicalExpr, lit, + }; + use datafusion_physical_plan::metrics::{ExecutionPlanMetricsSet, MetricBuilder}; + use parquet::arrow::ArrowWriter; + use parquet::file::metadata::ParquetMetaDataPushDecoder; + use parquet::file::properties::WriterProperties; + + /// Build a tiny in-memory Parquet file with three row groups whose `v` + /// column statistics are disjoint: RG0 → 0..1000, RG1 → 1000..2000, + /// RG2 → 2000..3000. Returns (metadata, schema). + fn build_three_rg_file() -> (Arc, SchemaRef) { + let schema = Arc::new(Schema::new(vec![Field::new("v", DataType::Int64, false)])); + let mut buf = Vec::new(); + let props = WriterProperties::builder() + .set_max_row_group_row_count(Some(1000)) + .build(); + let mut writer = + ArrowWriter::try_new(&mut buf, Arc::clone(&schema), Some(props)).unwrap(); + for rg in 0..3i64 { + let base = rg * 1000; + let vals: Vec = (base..base + 1000).collect(); + let batch = RecordBatch::try_new( + Arc::clone(&schema), + vec![Arc::new(Int64Array::from(vals))], + ) + .unwrap(); + writer.write(&batch).unwrap(); + writer.flush().unwrap(); + } + writer.close().unwrap(); + + let file = Bytes::from(buf); + let len = file.len() as u64; + let mut md = ParquetMetaDataPushDecoder::try_new(len).unwrap(); + // One range covering the whole file. Using `expect` rather than + // `allow` per this crate's `clippy::allow-attributes` lint. + #[expect( + clippy::single_range_in_vec_init, + reason = "we want a single range covering the whole file" + )] + let ranges = vec![0..len]; + md.push_ranges(ranges, vec![file]).unwrap(); + let DecodeResult::Data(meta) = md.try_decode().unwrap() else { + panic!("decoding metadata"); + }; + assert_eq!(meta.num_row_groups(), 3, "test fixture must have 3 RGs"); + (Arc::new(meta), schema) + } + + /// Create a fresh `(creation_errors, evaluation_errors)` counter pair + /// for tests. The names mirror the two metrics + /// [`RowGroupPruner::new`] consumes — predicate construction is + /// accounted separately from per-row-group evaluation. + fn pruner_error_counters() -> (Count, Count) { + let metrics = ExecutionPlanMetricsSet::new(); + let creation = + MetricBuilder::new(&metrics).counter("num_predicate_creation_errors", 0); + let evaluation = + MetricBuilder::new(&metrics).counter("predicate_evaluation_errors", 0); + (creation, evaluation) + } + + /// `v > literal` predicate on a single-column schema. + fn gt_predicate(threshold: i64) -> Arc { + Arc::new(BinaryExpr::new( + Arc::new(Column::new("v", 0)), + Operator::Gt, + lit(ScalarValue::Int64(Some(threshold))), + )) + } + + #[test] + fn row_group_pruner_skips_only_disqualified_row_groups() { + let (meta, schema) = build_three_rg_file(); + let (creation, evaluation) = pruner_error_counters(); + let mut pruner = RowGroupPruner::new( + gt_predicate(1500), + Arc::clone(&schema), + Arc::clone(&meta), + creation, + evaluation, + ); + + // RG0 (0..1000) is entirely below threshold → fully prunable. + assert!(pruner.should_prune(&[0]), "RG0 should be pruned"); + // RG1 (1000..2000) straddles the threshold → not safe to prune. + assert!(!pruner.should_prune(&[1]), "RG1 must NOT be pruned"); + // RG2 (2000..3000) is entirely above threshold → keep. + assert!(!pruner.should_prune(&[2]), "RG2 must NOT be pruned"); + // Run covering both RG0 and RG1 cannot be skipped — RG1 is alive. + assert!( + !pruner.should_prune(&[0, 1]), + "mixed run with a live RG must NOT be pruned" + ); + // Empty input is a no-op (defensive guard). + assert!(!pruner.should_prune(&[])); + } + + #[test] + fn row_group_pruner_tracks_dynamic_filter_updates() { + let (meta, schema) = build_three_rg_file(); + let dynamic = Arc::new(DynamicFilterPhysicalExpr::new( + vec![Arc::new(Column::new("v", 0))], + gt_predicate(500), + )); + let (creation, evaluation) = pruner_error_counters(); + let mut pruner = RowGroupPruner::new( + Arc::clone(&dynamic) as Arc, + Arc::clone(&schema), + Arc::clone(&meta), + creation, + evaluation, + ); + + // Initial threshold 500 → only the lower half of RG0 fails, so RG0 + // (0..1000) straddles the threshold and stays alive. + assert!(!pruner.should_prune(&[0])); + assert!(!pruner.should_prune(&[1])); + + // Tighten the threshold via the dynamic filter — TopK fills its + // heap and updates the threshold to 2500. + dynamic + .update(gt_predicate(2500)) + .expect("update threshold"); + + // After the update the pruner must rebuild its `PruningPredicate` + // (driven by the `DynamicFilterTracker`'s change notification) and + // re-evaluate. RG0 and RG1 are both entirely below 2500 now. + assert!( + pruner.should_prune(&[0]), + "RG0 must be pruned after threshold tightens to 2500" + ); + assert!( + pruner.should_prune(&[1]), + "RG1 must be pruned after threshold tightens to 2500" + ); + assert!( + !pruner.should_prune(&[2]), + "RG2 (2000..3000) still straddles 2500" + ); + } + + #[test] + fn row_group_pruner_falls_back_to_conservative_when_predicate_has_no_bounds() { + // A predicate the pruning analyzer can't decompose (e.g. a bare + // column reference of bool type would normally be valid, but a + // non-binary expression on a non-bool column doesn't yield bounds). + // We use `lit(true)` which produces no column references, so + // `build_pruning_predicate` will return None. + let (meta, schema) = build_three_rg_file(); + let (creation, evaluation) = pruner_error_counters(); + let mut pruner = RowGroupPruner::new( + lit(true) as Arc, + Arc::clone(&schema), + Arc::clone(&meta), + creation, + evaluation, + ); + // No pruning predicate could be built → conservatively keep RGs. + assert!(!pruner.should_prune(&[0])); + assert!(!pruner.should_prune(&[1])); + assert!(!pruner.should_prune(&[2])); + } +} diff --git a/datafusion/datasource-parquet/src/row_filter.rs b/datafusion/datasource-parquet/src/row_filter.rs index f19dbd6c6fa63..14aaa9d083182 100644 --- a/datafusion/datasource-parquet/src/row_filter.rs +++ b/datafusion/datasource-parquet/src/row_filter.rs @@ -1082,12 +1082,11 @@ pub fn build_row_filter( .map(|filters| Some(RowFilter::new(filters))) } -/// Builds row filters for decoder runs. +/// Builds row filters for a parquet decoder. /// -/// A [`RowFilter`] must be owned by a decoder, so scans split across multiple -/// decoder runs need a fresh filter for each run that evaluates row predicates. -/// The first filter is built eagerly during construction so callers can cheaply -/// query [`has_row_filter`](Self::has_row_filter) before splitting the scan. +/// A [`RowFilter`] is owned by a decoder. The first filter is built eagerly +/// during construction so the caller can attach it to the decoder via +/// [`next_filter`](Self::next_filter) without a redundant build call. pub(crate) struct RowFilterGenerator<'a> { predicate: Option<&'a Arc>, physical_file_schema: &'a SchemaRef, @@ -1117,10 +1116,6 @@ impl<'a> RowFilterGenerator<'a> { generator } - pub(crate) fn has_row_filter(&self) -> bool { - self.first_row_filter.is_some() - } - pub(crate) fn next_filter(&mut self) -> Option { self.first_row_filter.take().or_else(|| self.build()) } diff --git a/datafusion/datasource-parquet/src/row_group_filter.rs b/datafusion/datasource-parquet/src/row_group_filter.rs index 1e9b0636e59e9..82a36d66b32a3 100644 --- a/datafusion/datasource-parquet/src/row_group_filter.rs +++ b/datafusion/datasource-parquet/src/row_group_filter.rs @@ -453,12 +453,16 @@ impl RowGroupAccessPlanFilter { } } -/// Wraps a slice of [`RowGroupMetaData`] in a way that implements [`PruningStatistics`] -struct RowGroupPruningStatistics<'a> { - parquet_schema: &'a SchemaDescriptor, - row_group_metadatas: Vec<&'a RowGroupMetaData>, - arrow_schema: &'a Schema, - missing_null_counts_as_zero: bool, +/// Wraps a slice of [`RowGroupMetaData`] in a way that implements [`PruningStatistics`]. +/// +/// Visible to sibling modules so runtime row-group pruners (e.g. the dynamic +/// TopK pruner in `push_decoder.rs`) can reuse this adapter without +/// duplicating the statistics-to-`PruningStatistics` plumbing. +pub(crate) struct RowGroupPruningStatistics<'a> { + pub(crate) parquet_schema: &'a SchemaDescriptor, + pub(crate) row_group_metadatas: Vec<&'a RowGroupMetaData>, + pub(crate) arrow_schema: &'a Schema, + pub(crate) missing_null_counts_as_zero: bool, } impl<'a> RowGroupPruningStatistics<'a> { diff --git a/datafusion/datasource-parquet/src/source.rs b/datafusion/datasource-parquet/src/source.rs index 8228cd273eae6..cc8828143211d 100644 --- a/datafusion/datasource-parquet/src/source.rs +++ b/datafusion/datasource-parquet/src/source.rs @@ -40,6 +40,7 @@ use datafusion_common::config::TableParquetOptions; use datafusion_datasource::TableSchema; use datafusion_datasource::file::FileSource; use datafusion_datasource::file_scan_config::FileScanConfig; +use datafusion_physical_expr::expressions::DynamicFilterTracking; use datafusion_physical_expr::projection::ProjectionExprs; use datafusion_physical_expr::{EquivalenceProperties, conjunction}; use datafusion_physical_expr_adapter::DefaultPhysicalExprAdapterFactory; @@ -704,6 +705,19 @@ impl FileSource for ParquetSource { write!(f, ", reverse_row_groups=true")?; } + // Plan-time marker for dynamic RG-level pruning: if the + // predicate is dynamic (e.g. a TopK threshold expression), + // the parquet opener will split decoder runs per row group + // and consult `RowGroupPruner` between each. The actual + // pruning count appears as + // `row_groups_pruned_dynamic_filter` in EXPLAIN ANALYZE. + if let Some(predicate) = self.filter() + && DynamicFilterTracking::classify(&predicate) + .contains_dynamic_filter() + { + write!(f, ", dynamic_rg_pruning=eligible")?; + } + // Try to build the pruning predicates. // These are only generated here because it's useful to have *some* // idea of what pushdown is happening when viewing plans. @@ -1088,6 +1102,88 @@ mod tests { assert!(source.filter().is_some()); } + /// Render a `ParquetSource`'s `fmt_extra` output as a `String` for + /// inspection in tests. + fn render_fmt_extra(source: &ParquetSource, t: DisplayFormatType) -> String { + use std::fmt::Display; + + struct Wrap<'a> { + source: &'a ParquetSource, + t: DisplayFormatType, + } + impl Display for Wrap<'_> { + fn fmt(&self, f: &mut Formatter) -> std::fmt::Result { + self.source.fmt_extra(self.t, f) + } + } + Wrap { source, t }.to_string() + } + + /// EXPLAIN must surface a `dynamic_rg_pruning=eligible` marker when the + /// predicate carries a `DynamicFilterPhysicalExpr`. This is the + /// plan-time signal that the runtime row-group pruner will fire at + /// every RG boundary. + #[test] + fn fmt_extra_marks_dynamic_predicate_as_pruning_eligible() { + use arrow::datatypes::{DataType, Field, Schema}; + use datafusion_physical_expr::expressions::{Column, DynamicFilterPhysicalExpr}; + + let schema = Arc::new(Schema::new(vec![Field::new("v", DataType::Int64, false)])); + let dynamic = Arc::new(DynamicFilterPhysicalExpr::new( + vec![Arc::new(Column::new("v", 0))], + lit(true), + )) as Arc; + + let source = + ParquetSource::new(Arc::clone(&schema)).with_predicate(Arc::clone(&dynamic)); + + let rendered = render_fmt_extra(&source, DisplayFormatType::Default); + assert!( + rendered.contains("dynamic_rg_pruning=eligible"), + "expected marker in Default fmt_extra, got: {rendered}" + ); + + let rendered_verbose = render_fmt_extra(&source, DisplayFormatType::Verbose); + assert!( + rendered_verbose.contains("dynamic_rg_pruning=eligible"), + "expected marker in Verbose fmt_extra, got: {rendered_verbose}" + ); + } + + /// EXPLAIN must NOT show the dynamic-RG-pruning marker when the + /// predicate is purely static — the optimization will not fire, so + /// surfacing it would mislead the reader. + #[test] + fn fmt_extra_omits_marker_for_static_predicate() { + use arrow::datatypes::Schema; + + let schema = Arc::new(Schema::empty()); + let predicate = lit(true); + let source = ParquetSource::new(schema).with_predicate(predicate); + + let rendered = render_fmt_extra(&source, DisplayFormatType::Default); + assert!( + !rendered.contains("dynamic_rg_pruning"), + "did not expect marker for static predicate, got: {rendered}" + ); + } + + /// EXPLAIN must NOT show the marker when there is no predicate at all + /// (e.g. unfiltered table scan). + #[test] + fn fmt_extra_omits_marker_when_no_predicate() { + use arrow::datatypes::Schema; + + let schema = Arc::new(Schema::empty()); + let source = ParquetSource::new(schema); + + let rendered = render_fmt_extra(&source, DisplayFormatType::Default); + assert!( + !rendered.contains("dynamic_rg_pruning"), + "did not expect marker for predicate-less scan, got: {rendered}" + ); + } + /// Helpers for the `try_pushdown_sort` regression tests below. mod pushdown_sort_helpers { use super::*; diff --git a/datafusion/sqllogictest/test_files/clickbench.slt b/datafusion/sqllogictest/test_files/clickbench.slt index 60f7aadb8cfb1..7c2a8bcaa15f5 100644 --- a/datafusion/sqllogictest/test_files/clickbench.slt +++ b/datafusion/sqllogictest/test_files/clickbench.slt @@ -652,7 +652,7 @@ physical_plan 03)----ProjectionExec: expr=[WatchID@0 as WatchID, JavaEnable@1 as JavaEnable, Title@2 as Title, GoodEvent@3 as GoodEvent, EventTime@4 as EventTime, CounterID@6 as CounterID, ClientIP@7 as ClientIP, RegionID@8 as RegionID, UserID@9 as UserID, CounterClass@10 as CounterClass, OS@11 as OS, UserAgent@12 as UserAgent, URL@13 as URL, Referer@14 as Referer, IsRefresh@15 as IsRefresh, RefererCategoryID@16 as RefererCategoryID, RefererRegionID@17 as RefererRegionID, URLCategoryID@18 as URLCategoryID, URLRegionID@19 as URLRegionID, ResolutionWidth@20 as ResolutionWidth, ResolutionHeight@21 as ResolutionHeight, ResolutionDepth@22 as ResolutionDepth, FlashMajor@23 as FlashMajor, FlashMinor@24 as FlashMinor, FlashMinor2@25 as FlashMinor2, NetMajor@26 as NetMajor, NetMinor@27 as NetMinor, UserAgentMajor@28 as UserAgentMajor, UserAgentMinor@29 as UserAgentMinor, CookieEnable@30 as CookieEnable, JavascriptEnable@31 as JavascriptEnable, IsMobile@32 as IsMobile, MobilePhone@33 as MobilePhone, MobilePhoneModel@34 as MobilePhoneModel, Params@35 as Params, IPNetworkID@36 as IPNetworkID, TraficSourceID@37 as TraficSourceID, SearchEngineID@38 as SearchEngineID, SearchPhrase@39 as SearchPhrase, AdvEngineID@40 as AdvEngineID, IsArtifical@41 as IsArtifical, WindowClientWidth@42 as WindowClientWidth, WindowClientHeight@43 as WindowClientHeight, ClientTimeZone@44 as ClientTimeZone, ClientEventTime@45 as ClientEventTime, SilverlightVersion1@46 as SilverlightVersion1, SilverlightVersion2@47 as SilverlightVersion2, SilverlightVersion3@48 as SilverlightVersion3, SilverlightVersion4@49 as SilverlightVersion4, PageCharset@50 as PageCharset, CodeVersion@51 as CodeVersion, IsLink@52 as IsLink, IsDownload@53 as IsDownload, IsNotBounce@54 as IsNotBounce, FUniqID@55 as FUniqID, OriginalURL@56 as OriginalURL, HID@57 as HID, IsOldCounter@58 as IsOldCounter, IsEvent@59 as IsEvent, IsParameter@60 as IsParameter, DontCountHits@61 as DontCountHits, WithHash@62 as WithHash, HitColor@63 as HitColor, LocalEventTime@64 as LocalEventTime, Age@65 as Age, Sex@66 as Sex, Income@67 as Income, Interests@68 as Interests, Robotness@69 as Robotness, RemoteIP@70 as RemoteIP, WindowName@71 as WindowName, OpenerName@72 as OpenerName, HistoryLength@73 as HistoryLength, BrowserLanguage@74 as BrowserLanguage, BrowserCountry@75 as BrowserCountry, SocialNetwork@76 as SocialNetwork, SocialAction@77 as SocialAction, HTTPError@78 as HTTPError, SendTiming@79 as SendTiming, DNSTiming@80 as DNSTiming, ConnectTiming@81 as ConnectTiming, ResponseStartTiming@82 as ResponseStartTiming, ResponseEndTiming@83 as ResponseEndTiming, FetchTiming@84 as FetchTiming, SocialSourceNetworkID@85 as SocialSourceNetworkID, SocialSourcePage@86 as SocialSourcePage, ParamPrice@87 as ParamPrice, ParamOrderID@88 as ParamOrderID, ParamCurrency@89 as ParamCurrency, ParamCurrencyID@90 as ParamCurrencyID, OpenstatServiceName@91 as OpenstatServiceName, OpenstatCampaignID@92 as OpenstatCampaignID, OpenstatAdID@93 as OpenstatAdID, OpenstatSourceID@94 as OpenstatSourceID, UTMSource@95 as UTMSource, UTMMedium@96 as UTMMedium, UTMCampaign@97 as UTMCampaign, UTMContent@98 as UTMContent, UTMTerm@99 as UTMTerm, FromTag@100 as FromTag, HasGCLID@101 as HasGCLID, RefererHash@102 as RefererHash, URLHash@103 as URLHash, CLID@104 as CLID, CAST(CAST(EventDate@5 AS Int32) AS Date32) as EventDate] 04)------FilterExec: URL@13 LIKE %google% 05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet]]}, projection=[WatchID, JavaEnable, Title, GoodEvent, EventTime, EventDate, CounterID, ClientIP, RegionID, UserID, CounterClass, OS, UserAgent, URL, Referer, IsRefresh, RefererCategoryID, RefererRegionID, URLCategoryID, URLRegionID, ResolutionWidth, ResolutionHeight, ResolutionDepth, FlashMajor, FlashMinor, FlashMinor2, NetMajor, NetMinor, UserAgentMajor, UserAgentMinor, CookieEnable, JavascriptEnable, IsMobile, MobilePhone, MobilePhoneModel, Params, IPNetworkID, TraficSourceID, SearchEngineID, SearchPhrase, AdvEngineID, IsArtifical, WindowClientWidth, WindowClientHeight, ClientTimeZone, ClientEventTime, SilverlightVersion1, SilverlightVersion2, SilverlightVersion3, SilverlightVersion4, PageCharset, CodeVersion, IsLink, IsDownload, IsNotBounce, FUniqID, OriginalURL, HID, IsOldCounter, IsEvent, IsParameter, DontCountHits, WithHash, HitColor, LocalEventTime, Age, Sex, Income, Interests, Robotness, RemoteIP, WindowName, OpenerName, HistoryLength, BrowserLanguage, BrowserCountry, SocialNetwork, SocialAction, HTTPError, SendTiming, DNSTiming, ConnectTiming, ResponseStartTiming, ResponseEndTiming, FetchTiming, SocialSourceNetworkID, SocialSourcePage, ParamPrice, ParamOrderID, ParamCurrency, ParamCurrencyID, OpenstatServiceName, OpenstatCampaignID, OpenstatAdID, OpenstatSourceID, UTMSource, UTMMedium, UTMCampaign, UTMContent, UTMTerm, FromTag, HasGCLID, RefererHash, URLHash, CLID], file_type=parquet, predicate=URL@13 LIKE %google% AND DynamicFilter [ empty ] +06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet]]}, projection=[WatchID, JavaEnable, Title, GoodEvent, EventTime, EventDate, CounterID, ClientIP, RegionID, UserID, CounterClass, OS, UserAgent, URL, Referer, IsRefresh, RefererCategoryID, RefererRegionID, URLCategoryID, URLRegionID, ResolutionWidth, ResolutionHeight, ResolutionDepth, FlashMajor, FlashMinor, FlashMinor2, NetMajor, NetMinor, UserAgentMajor, UserAgentMinor, CookieEnable, JavascriptEnable, IsMobile, MobilePhone, MobilePhoneModel, Params, IPNetworkID, TraficSourceID, SearchEngineID, SearchPhrase, AdvEngineID, IsArtifical, WindowClientWidth, WindowClientHeight, ClientTimeZone, ClientEventTime, SilverlightVersion1, SilverlightVersion2, SilverlightVersion3, SilverlightVersion4, PageCharset, CodeVersion, IsLink, IsDownload, IsNotBounce, FUniqID, OriginalURL, HID, IsOldCounter, IsEvent, IsParameter, DontCountHits, WithHash, HitColor, LocalEventTime, Age, Sex, Income, Interests, Robotness, RemoteIP, WindowName, OpenerName, HistoryLength, BrowserLanguage, BrowserCountry, SocialNetwork, SocialAction, HTTPError, SendTiming, DNSTiming, ConnectTiming, ResponseStartTiming, ResponseEndTiming, FetchTiming, SocialSourceNetworkID, SocialSourcePage, ParamPrice, ParamOrderID, ParamCurrency, ParamCurrencyID, OpenstatServiceName, OpenstatCampaignID, OpenstatAdID, OpenstatSourceID, UTMSource, UTMMedium, UTMCampaign, UTMContent, UTMTerm, FromTag, HasGCLID, RefererHash, URLHash, CLID], file_type=parquet, predicate=URL@13 LIKE %google% AND DynamicFilter [ empty ], dynamic_rg_pruning=eligible query IITIIIIIIIIITTIIIIIIIIIITIIITIIIITTIIITIIIIIIIIIITIIIIITIIIIIITIIIIIIIIIITTTTIIIIIIIITITTITTTTTTTTTTIIIID SELECT * FROM hits WHERE "URL" LIKE '%google%' ORDER BY "EventTime" LIMIT 10; @@ -675,7 +675,7 @@ physical_plan 03)----SortExec: TopK(fetch=10), expr=[EventTime@1 ASC NULLS LAST], preserve_partitioning=[true] 04)------FilterExec: SearchPhrase@1 != , projection=[SearchPhrase@1, EventTime@0] 05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet]]}, projection=[EventTime, SearchPhrase], file_type=parquet, predicate=SearchPhrase@39 != AND DynamicFilter [ empty ], pruning_predicate=SearchPhrase_null_count@2 != row_count@3 AND (SearchPhrase_min@0 != OR != SearchPhrase_max@1), required_guarantees=[SearchPhrase not in ()] +06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet]]}, projection=[EventTime, SearchPhrase], file_type=parquet, predicate=SearchPhrase@39 != AND DynamicFilter [ empty ], dynamic_rg_pruning=eligible, pruning_predicate=SearchPhrase_null_count@2 != row_count@3 AND (SearchPhrase_min@0 != OR != SearchPhrase_max@1), required_guarantees=[SearchPhrase not in ()] query T SELECT "SearchPhrase" FROM hits WHERE "SearchPhrase" <> '' ORDER BY "EventTime" LIMIT 10; @@ -695,7 +695,7 @@ physical_plan 02)--SortExec: TopK(fetch=10), expr=[SearchPhrase@0 ASC NULLS LAST], preserve_partitioning=[true] 03)----FilterExec: SearchPhrase@0 != 04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet]]}, projection=[SearchPhrase], file_type=parquet, predicate=SearchPhrase@39 != AND DynamicFilter [ empty ], pruning_predicate=SearchPhrase_null_count@2 != row_count@3 AND (SearchPhrase_min@0 != OR != SearchPhrase_max@1), required_guarantees=[SearchPhrase not in ()] +05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet]]}, projection=[SearchPhrase], file_type=parquet, predicate=SearchPhrase@39 != AND DynamicFilter [ empty ], dynamic_rg_pruning=eligible, pruning_predicate=SearchPhrase_null_count@2 != row_count@3 AND (SearchPhrase_min@0 != OR != SearchPhrase_max@1), required_guarantees=[SearchPhrase not in ()] query T SELECT "SearchPhrase" FROM hits WHERE "SearchPhrase" <> '' ORDER BY "SearchPhrase" LIMIT 10; @@ -718,7 +718,7 @@ physical_plan 03)----SortExec: TopK(fetch=10), expr=[EventTime@1 ASC NULLS LAST, SearchPhrase@0 ASC NULLS LAST], preserve_partitioning=[true] 04)------FilterExec: SearchPhrase@1 != , projection=[SearchPhrase@1, EventTime@0] 05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet]]}, projection=[EventTime, SearchPhrase], file_type=parquet, predicate=SearchPhrase@39 != AND DynamicFilter [ empty ], pruning_predicate=SearchPhrase_null_count@2 != row_count@3 AND (SearchPhrase_min@0 != OR != SearchPhrase_max@1), required_guarantees=[SearchPhrase not in ()] +06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/clickbench_hits_10.parquet]]}, projection=[EventTime, SearchPhrase], file_type=parquet, predicate=SearchPhrase@39 != AND DynamicFilter [ empty ], dynamic_rg_pruning=eligible, pruning_predicate=SearchPhrase_null_count@2 != row_count@3 AND (SearchPhrase_min@0 != OR != SearchPhrase_max@1), required_guarantees=[SearchPhrase not in ()] query T SELECT "SearchPhrase" FROM hits WHERE "SearchPhrase" <> '' ORDER BY "EventTime", "SearchPhrase" LIMIT 10; diff --git a/datafusion/sqllogictest/test_files/dynamic_filter_pushdown_config.slt b/datafusion/sqllogictest/test_files/dynamic_filter_pushdown_config.slt index e779ce2cbffb0..7b4d397b095cc 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 ], sort_order_for_reorder=[value@1 DESC], reverse_row_groups=true +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 ], sort_order_for_reorder=[value@1 DESC], reverse_row_groups=true, dynamic_rg_pruning=eligible statement ok set datafusion.explain.analyze_level = summary; @@ -104,7 +104,7 @@ Plan with Metrics 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=, 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=0 total → 0 matched, page_index_pages_skipped_by_fully_matched=1, limit_pruned_row_groups=0 total → 0 matched, bytes_scanned=210, metadata_load_time=, scan_efficiency_ratio=18.31% (210/1.15 K)] +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 ], dynamic_rg_pruning=eligible, 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=1 total → 1 matched -> 1 fully matched, row_groups_pruned_bloom_filter=1 total → 1 matched, page_index_pages_pruned=0 total → 0 matched, page_index_pages_skipped_by_fully_matched=1, limit_pruned_row_groups=0 total → 0 matched, bytes_scanned=210, row_groups_pruned_dynamic_filter=0, metadata_load_time=, scan_efficiency_ratio=18.31% (210/1.15 K)] statement ok set datafusion.explain.analyze_level = dev; @@ -157,7 +157,7 @@ logical_plan physical_plan 01)HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(id@0, id@0)], projection=[id@2, data@3, info@1] 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, data], file_type=parquet, predicate=DynamicFilter [ empty ] +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 ], dynamic_rg_pruning=eligible # Disable Join dynamic filter pushdown statement ok @@ -235,7 +235,7 @@ logical_plan physical_plan 01)HashJoinExec: mode=CollectLeft, join_type=Left, on=[(id@0, id@0)], projection=[id@2, data@3, info@1] 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, data], file_type=parquet, predicate=DynamicFilter [ empty ] +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 ], dynamic_rg_pruning=eligible # RIGHT JOIN correctness: all right rows appear, unmatched left rows produce NULLs query ITT @@ -284,7 +284,7 @@ logical_plan 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 ] +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 ], dynamic_rg_pruning=eligible # 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). @@ -304,7 +304,7 @@ logical_plan 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 ] +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 ], dynamic_rg_pruning=eligible # LEFT SEMI (physical LeftSemi) correctness: only right rows with matching left ids query IT rowsort @@ -338,7 +338,7 @@ physical_plan 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)--SortExec: expr=[data@1 DESC], preserve_partitioning=[false] 04)----FilterExec: DynamicFilter [ empty ] -05)------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 ], sort_order_for_reorder=[data@1 DESC], reverse_row_groups=true +05)------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 ], sort_order_for_reorder=[data@1 DESC], reverse_row_groups=true, dynamic_rg_pruning=eligible statement count 0 SET datafusion.execution.parquet.pushdown_filters = true; @@ -361,7 +361,7 @@ 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)--SortExec: expr=[data@1 DESC], preserve_partitioning=[false] -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 ], sort_order_for_reorder=[data@1 DESC], reverse_row_groups=true +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 ], sort_order_for_reorder=[data@1 DESC], reverse_row_groups=true, dynamic_rg_pruning=eligible statement count 0 RESET datafusion.execution.parquet.pushdown_filters; @@ -383,7 +383,7 @@ logical_plan 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 ] +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 ], dynamic_rg_pruning=eligible # LEFT MARK JOIN: the OR prevents decorrelation to LeftSemi, so the optimizer # uses LeftMark. Self-generated dynamic filter pushes to the probe side. @@ -407,7 +407,7 @@ physical_plan 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 ] +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 ], dynamic_rg_pruning=eligible # LEFT MARK correctness: all right rows match EXISTS, so all 3 appear query IT rowsort @@ -444,8 +444,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: 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 ], dynamic_rg_pruning=eligible +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 ], dynamic_rg_pruning=eligible # Correctness check query IT @@ -478,8 +478,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: 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 ], dynamic_rg_pruning=eligible +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 ], dynamic_rg_pruning=eligible # Correctness check query IT @@ -516,7 +516,7 @@ logical_plan physical_plan 01)HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(id@0, id@0)], projection=[id@2, data@3, info@1] 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, data], file_type=parquet, predicate=DynamicFilter [ empty ] +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 ], dynamic_rg_pruning=eligible # Enable TopK, disable Join statement ok @@ -588,7 +588,7 @@ physical_plan 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)] +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 ], dynamic_rg_pruning=eligible, 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 @@ -736,7 +736,7 @@ logical_plan physical_plan 01)HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(id@0, id@0)], projection=[id@2, data@3, info@1] 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, data], file_type=parquet, predicate=DynamicFilter [ empty ] +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 ], dynamic_rg_pruning=eligible # 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/dynamic_row_group_pruning.slt b/datafusion/sqllogictest/test_files/dynamic_row_group_pruning.slt new file mode 100644 index 0000000000000..acf50ab8b01cd --- /dev/null +++ b/datafusion/sqllogictest/test_files/dynamic_row_group_pruning.slt @@ -0,0 +1,111 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at + +# http://www.apache.org/licenses/LICENSE-2.0 + +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +# End-to-end SLT for **dynamic row-group pruning** driven by a TopK +# `SortExec`'s `DynamicFilterPhysicalExpr`. +# +# Builds a 5-row-group parquet file with disjoint per-RG ranges of `v`: +# RG 0: 0..3, RG 1: 3..6, RG 2: 6..9, RG 3: 9..12, RG 4: 12..15 +# `ORDER BY v DESC LIMIT 3` fills the TopK heap from the row group with +# the largest values; the tightened threshold then proves every other +# row group unreachable. The opener splits decoder runs per row group +# (because the predicate carries a `DynamicFilterPhysicalExpr`) and the +# runtime `RowGroupPruner` increments +# `row_groups_pruned_dynamic_filter`. + +statement ok +set datafusion.execution.parquet.pushdown_filters = true; + +statement ok +set datafusion.explain.analyze_level = summary; + +statement ok +CREATE TABLE source_data AS VALUES +-- RG 0 + (0), (1), (2), +-- RG 1 + (3), (4), (5), +-- RG 2 + (6), (7), (8), +-- RG 3 + (9), (10), (11), +-- RG 4 + (12), (13), (14); + +statement ok +COPY (SELECT column1 as v FROM source_data) +TO 'test_files/scratch/dynamic_row_group_pruning/data.parquet' +STORED AS PARQUET +OPTIONS ( + 'format.max_row_group_size' '3' +); + +statement ok +drop table source_data; + +statement ok +CREATE EXTERNAL TABLE t +STORED AS PARQUET +LOCATION 'test_files/scratch/dynamic_row_group_pruning/data.parquet'; + +# Sanity: query returns the right rows. +query I +SELECT v FROM t ORDER BY v DESC LIMIT 3; +---- +14 +13 +12 + +# Plain `EXPLAIN` must surface the plan-time eligibility marker +# `dynamic_rg_pruning=eligible` on the `DataSourceExec` line: the +# predicate is dynamic, so the runtime row-group pruner will be +# consulted at each decoder-run boundary. This is the only knob users +# have for spotting the optimization without running the query. +query TT +explain select v from t order by v desc limit 3; +---- +logical_plan +01)Sort: t.v DESC NULLS FIRST, fetch=3 +02)--TableScan: t projection=[v] +physical_plan +01)SortExec: TopK(fetch=3), expr=[v@0 DESC], preserve_partitioning=[false] +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_row_group_pruning/data.parquet]]}, projection=[v], file_type=parquet, predicate=DynamicFilter [ empty ], sort_order_for_reorder=[v@0 DESC], reverse_row_groups=true, dynamic_rg_pruning=eligible + +# `EXPLAIN ANALYZE` must surface the runtime metric +# `row_groups_pruned_dynamic_filter` with a non-zero value. Five +# disjoint row groups, `LIMIT 3` fits inside the highest RG, so the +# pruner skips the other four. Note the exact `=4`: the data is small +# enough that the TopK heap fills in a single batch, and execution is +# single-threaded, so the count is deterministic. Time- and size-keyed +# fields are masked with ``. +query TT +explain analyze select v from t order by v desc limit 3; +---- +Plan with Metrics +01)SortExec: TopK(fetch=3), expr=[v@0 DESC], preserve_partitioning=[false], filter=[v@0 IS NULL OR v@0 > 12], metrics=[output_rows=3, elapsed_compute=, output_bytes=] +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/dynamic_row_group_pruning/data.parquet]]}, projection=[v], file_type=parquet, predicate=DynamicFilter [ v@0 IS NULL OR v@0 > 12 ], sort_order_for_reorder=[v@0 DESC], reverse_row_groups=true, dynamic_rg_pruning=eligible, pruning_predicate=v_null_count@0 > 0 OR v_null_count@0 != row_count@2 AND v_max@1 > 12, required_guarantees=[], metrics=[output_rows=3, elapsed_compute=, output_bytes=, files_ranges_pruned_statistics=1 total → 1 matched, row_groups_pruned_statistics=5 total → 5 matched, row_groups_pruned_bloom_filter=5 total → 5 matched, page_index_pages_pruned=0 total → 0 matched, limit_pruned_row_groups=0 total → 0 matched, bytes_scanned=, row_groups_pruned_dynamic_filter=4, metadata_load_time=, scan_efficiency_ratio=] + +statement ok +drop table t; + +# Config reset — without these the SLT runner flags the file for +# leaking session state into subsequent files. +statement ok +RESET datafusion.execution.parquet.pushdown_filters; + +statement ok +RESET datafusion.explain.analyze_level; diff --git a/datafusion/sqllogictest/test_files/explain_analyze.slt b/datafusion/sqllogictest/test_files/explain_analyze.slt index b1856e0adda16..623580fce94e3 100644 --- a/datafusion/sqllogictest/test_files/explain_analyze.slt +++ b/datafusion/sqllogictest/test_files/explain_analyze.slt @@ -231,7 +231,7 @@ explain analyze select * from cat_tracking where species > 'M' AND s >= 50 order ---- Plan with Metrics 01)SortExec: TopK(fetch=3), expr=[species@0 ASC NULLS LAST], preserve_partitioning=[false], filter=[species@0 < Nlpine Sheep], metrics=[] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/explain_analyze/data.parquet]]}, projection=[species, s], file_type=parquet, predicate=species@0 > M AND s@1 >= 50 AND DynamicFilter [ species@0 < Nlpine Sheep ], sort_order_for_reorder=[species@0 ASC NULLS LAST], 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=[] +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/explain_analyze/data.parquet]]}, projection=[species, s], file_type=parquet, predicate=species@0 > M AND s@1 >= 50 AND DynamicFilter [ species@0 < Nlpine Sheep ], sort_order_for_reorder=[species@0 ASC NULLS LAST], dynamic_rg_pruning=eligible, 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=[] statement ok reset datafusion.explain.analyze_categories; @@ -247,7 +247,7 @@ explain analyze select * from cat_tracking where species > 'M' AND s >= 50 order ---- 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] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/explain_analyze/data.parquet]]}, projection=[species, s], file_type=parquet, predicate=species@0 > M AND s@1 >= 50 AND DynamicFilter [ species@0 < Nlpine Sheep ], sort_order_for_reorder=[species@0 ASC NULLS LAST], 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, 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, page_index_pages_skipped_by_fully_matched=1, limit_pruned_row_groups=0 total → 0 matched, scan_efficiency_ratio=21.75% (485/2.23 K)] +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/explain_analyze/data.parquet]]}, projection=[species, s], file_type=parquet, predicate=species@0 > M AND s@1 >= 50 AND DynamicFilter [ species@0 < Nlpine Sheep ], sort_order_for_reorder=[species@0 ASC NULLS LAST], dynamic_rg_pruning=eligible, 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, 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, page_index_pages_skipped_by_fully_matched=1, limit_pruned_row_groups=0 total → 0 matched, scan_efficiency_ratio=21.75% (485/2.23 K)] statement ok reset datafusion.explain.analyze_categories; @@ -262,7 +262,7 @@ explain analyze select * from cat_tracking where species > 'M' AND s >= 50 order ---- 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, output_bytes=] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/explain_analyze/data.parquet]]}, projection=[species, s], file_type=parquet, predicate=species@0 > M AND s@1 >= 50 AND DynamicFilter [ species@0 < Nlpine Sheep ], sort_order_for_reorder=[species@0 ASC NULLS LAST], 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, 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, page_index_pages_skipped_by_fully_matched=1, limit_pruned_row_groups=0 total → 0 matched, bytes_scanned=, scan_efficiency_ratio=] +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/explain_analyze/data.parquet]]}, projection=[species, s], file_type=parquet, predicate=species@0 > M AND s@1 >= 50 AND DynamicFilter [ species@0 < Nlpine Sheep ], sort_order_for_reorder=[species@0 ASC NULLS LAST], dynamic_rg_pruning=eligible, 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, 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, page_index_pages_skipped_by_fully_matched=1, limit_pruned_row_groups=0 total → 0 matched, bytes_scanned=, scan_efficiency_ratio=] statement ok reset datafusion.explain.analyze_categories; @@ -277,7 +277,7 @@ explain analyze select * from cat_tracking where species > 'M' AND s >= 50 order ---- 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, output_bytes=] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/explain_analyze/data.parquet]]}, projection=[species, s], file_type=parquet, predicate=species@0 > M AND s@1 >= 50 AND DynamicFilter [ species@0 < Nlpine Sheep ], sort_order_for_reorder=[species@0 ASC NULLS LAST], 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, 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, page_index_pages_skipped_by_fully_matched=1, limit_pruned_row_groups=0 total → 0 matched, bytes_scanned=, scan_efficiency_ratio=] +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/explain_analyze/data.parquet]]}, projection=[species, s], file_type=parquet, predicate=species@0 > M AND s@1 >= 50 AND DynamicFilter [ species@0 < Nlpine Sheep ], sort_order_for_reorder=[species@0 ASC NULLS LAST], dynamic_rg_pruning=eligible, 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, 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, page_index_pages_skipped_by_fully_matched=1, limit_pruned_row_groups=0 total → 0 matched, bytes_scanned=, scan_efficiency_ratio=] statement ok reset datafusion.explain.analyze_categories; @@ -292,7 +292,7 @@ explain analyze select * from cat_tracking where species > 'M' AND s >= 50 order ---- Plan with Metrics 01)SortExec: TopK(fetch=3), expr=[species@0 ASC NULLS LAST], preserve_partitioning=[false], filter=[species@0 < Nlpine Sheep], metrics=[elapsed_compute=] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/explain_analyze/data.parquet]]}, projection=[species, s], file_type=parquet, predicate=species@0 > M AND s@1 >= 50 AND DynamicFilter [ species@0 < Nlpine Sheep ], sort_order_for_reorder=[species@0 ASC NULLS LAST], 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=[elapsed_compute=, metadata_load_time=] +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/explain_analyze/data.parquet]]}, projection=[species, s], file_type=parquet, predicate=species@0 > M AND s@1 >= 50 AND DynamicFilter [ species@0 < Nlpine Sheep ], sort_order_for_reorder=[species@0 ASC NULLS LAST], dynamic_rg_pruning=eligible, 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=[elapsed_compute=, metadata_load_time=] statement ok reset datafusion.explain.analyze_categories; @@ -319,7 +319,7 @@ EXPLAIN (ANALYZE, METRICS 'none', LEVEL summary) select * from cat_tracking wher ---- Plan with Metrics 01)SortExec: TopK(fetch=3), expr=[species@0 ASC NULLS LAST], preserve_partitioning=[false], filter=[species@0 < Nlpine Sheep], metrics=[] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/explain_analyze/data.parquet]]}, projection=[species, s], file_type=parquet, predicate=species@0 > M AND s@1 >= 50 AND DynamicFilter [ species@0 < Nlpine Sheep ], sort_order_for_reorder=[species@0 ASC NULLS LAST], 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=[] +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/explain_analyze/data.parquet]]}, projection=[species, s], file_type=parquet, predicate=species@0 > M AND s@1 >= 50 AND DynamicFilter [ species@0 < Nlpine Sheep ], sort_order_for_reorder=[species@0 ASC NULLS LAST], dynamic_rg_pruning=eligible, 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=[] # ---- (METRICS 'rows', LEVEL summary) — row-count metrics only ---- @@ -328,7 +328,7 @@ EXPLAIN (ANALYZE, METRICS 'rows', LEVEL summary) select * from cat_tracking wher ---- 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] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/explain_analyze/data.parquet]]}, projection=[species, s], file_type=parquet, predicate=species@0 > M AND s@1 >= 50 AND DynamicFilter [ species@0 < Nlpine Sheep ], sort_order_for_reorder=[species@0 ASC NULLS LAST], 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, 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, page_index_pages_skipped_by_fully_matched=1, limit_pruned_row_groups=0 total → 0 matched, scan_efficiency_ratio=21.75% (485/2.23 K)] +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/explain_analyze/data.parquet]]}, projection=[species, s], file_type=parquet, predicate=species@0 > M AND s@1 >= 50 AND DynamicFilter [ species@0 < Nlpine Sheep ], sort_order_for_reorder=[species@0 ASC NULLS LAST], dynamic_rg_pruning=eligible, 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, 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, page_index_pages_skipped_by_fully_matched=1, limit_pruned_row_groups=0 total → 0 matched, scan_efficiency_ratio=21.75% (485/2.23 K)] # ---- Quoted-string METRICS with multiple categories ---- @@ -337,7 +337,7 @@ EXPLAIN (ANALYZE, METRICS 'rows,bytes', LEVEL summary) select * from cat_trackin ---- 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, output_bytes=] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/explain_analyze/data.parquet]]}, projection=[species, s], file_type=parquet, predicate=species@0 > M AND s@1 >= 50 AND DynamicFilter [ species@0 < Nlpine Sheep ], sort_order_for_reorder=[species@0 ASC NULLS LAST], 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, 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, page_index_pages_skipped_by_fully_matched=1, limit_pruned_row_groups=0 total → 0 matched, bytes_scanned=, scan_efficiency_ratio=] +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/explain_analyze/data.parquet]]}, projection=[species, s], file_type=parquet, predicate=species@0 > M AND s@1 >= 50 AND DynamicFilter [ species@0 < Nlpine Sheep ], sort_order_for_reorder=[species@0 ASC NULLS LAST], dynamic_rg_pruning=eligible, 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, 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, page_index_pages_skipped_by_fully_matched=1, limit_pruned_row_groups=0 total → 0 matched, bytes_scanned=, scan_efficiency_ratio=] # ---- (METRICS 'timing', LEVEL summary) — timing metrics only ---- @@ -346,7 +346,7 @@ EXPLAIN (ANALYZE, METRICS 'timing', LEVEL summary) select * from cat_tracking wh ---- Plan with Metrics 01)SortExec: TopK(fetch=3), expr=[species@0 ASC NULLS LAST], preserve_partitioning=[false], filter=[species@0 < Nlpine Sheep], metrics=[elapsed_compute=] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/explain_analyze/data.parquet]]}, projection=[species, s], file_type=parquet, predicate=species@0 > M AND s@1 >= 50 AND DynamicFilter [ species@0 < Nlpine Sheep ], sort_order_for_reorder=[species@0 ASC NULLS LAST], 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=[elapsed_compute=, metadata_load_time=] +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/explain_analyze/data.parquet]]}, projection=[species, s], file_type=parquet, predicate=species@0 > M AND s@1 >= 50 AND DynamicFilter [ species@0 < Nlpine Sheep ], sort_order_for_reorder=[species@0 ASC NULLS LAST], dynamic_rg_pruning=eligible, 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=[elapsed_compute=, metadata_load_time=] # ---- TIMING sugar: `METRICS 'rows,bytes', TIMING off` ↔ rows+bytes only ---- # Equivalent to METRICS 'rows,bytes' since the sugar removes the timing @@ -357,7 +357,7 @@ EXPLAIN (ANALYZE, METRICS 'rows,bytes', TIMING off, LEVEL summary) select * from ---- 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, output_bytes=] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/explain_analyze/data.parquet]]}, projection=[species, s], file_type=parquet, predicate=species@0 > M AND s@1 >= 50 AND DynamicFilter [ species@0 < Nlpine Sheep ], sort_order_for_reorder=[species@0 ASC NULLS LAST], 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, 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, page_index_pages_skipped_by_fully_matched=1, limit_pruned_row_groups=0 total → 0 matched, bytes_scanned=, scan_efficiency_ratio=] +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/explain_analyze/data.parquet]]}, projection=[species, s], file_type=parquet, predicate=species@0 > M AND s@1 >= 50 AND DynamicFilter [ species@0 < Nlpine Sheep ], sort_order_for_reorder=[species@0 ASC NULLS LAST], dynamic_rg_pruning=eligible, 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, 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, page_index_pages_skipped_by_fully_matched=1, limit_pruned_row_groups=0 total → 0 matched, bytes_scanned=, scan_efficiency_ratio=] # ---- TIMING sugar: `METRICS 'rows', TIMING on` ↔ rows + timing ---- @@ -366,7 +366,7 @@ EXPLAIN (ANALYZE, METRICS 'rows', TIMING on, LEVEL summary) select * from cat_tr ---- 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=] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/explain_analyze/data.parquet]]}, projection=[species, s], file_type=parquet, predicate=species@0 > M AND s@1 >= 50 AND DynamicFilter [ species@0 < Nlpine Sheep ], sort_order_for_reorder=[species@0 ASC NULLS LAST], 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=, 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, page_index_pages_skipped_by_fully_matched=1, limit_pruned_row_groups=0 total → 0 matched, metadata_load_time=, scan_efficiency_ratio=21.75% (485/2.23 K)] +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/explain_analyze/data.parquet]]}, projection=[species, s], file_type=parquet, predicate=species@0 > M AND s@1 >= 50 AND DynamicFilter [ species@0 < Nlpine Sheep ], sort_order_for_reorder=[species@0 ASC NULLS LAST], dynamic_rg_pruning=eligible, 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=, 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, page_index_pages_skipped_by_fully_matched=1, limit_pruned_row_groups=0 total → 0 matched, metadata_load_time=, scan_efficiency_ratio=21.75% (485/2.23 K)] # ---- SUMMARY sugar: `SUMMARY on` ↔ `LEVEL summary` ---- # Equivalent to METRICS 'rows', LEVEL summary above. @@ -376,7 +376,7 @@ EXPLAIN (ANALYZE, METRICS 'rows', SUMMARY on) select * from cat_tracking where s ---- 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] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/explain_analyze/data.parquet]]}, projection=[species, s], file_type=parquet, predicate=species@0 > M AND s@1 >= 50 AND DynamicFilter [ species@0 < Nlpine Sheep ], sort_order_for_reorder=[species@0 ASC NULLS LAST], 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, 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, page_index_pages_skipped_by_fully_matched=1, limit_pruned_row_groups=0 total → 0 matched, scan_efficiency_ratio=21.75% (485/2.23 K)] +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/explain_analyze/data.parquet]]}, projection=[species, s], file_type=parquet, predicate=species@0 > M AND s@1 >= 50 AND DynamicFilter [ species@0 < Nlpine Sheep ], sort_order_for_reorder=[species@0 ASC NULLS LAST], dynamic_rg_pruning=eligible, 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, 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, page_index_pages_skipped_by_fully_matched=1, limit_pruned_row_groups=0 total → 0 matched, scan_efficiency_ratio=21.75% (485/2.23 K)] # ---- Statement option overrides session config ---- # Session says 'timing' but statement-level `METRICS 'rows'` wins. @@ -389,7 +389,7 @@ EXPLAIN (ANALYZE, METRICS 'rows', LEVEL summary) select * from cat_tracking wher ---- 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] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/explain_analyze/data.parquet]]}, projection=[species, s], file_type=parquet, predicate=species@0 > M AND s@1 >= 50 AND DynamicFilter [ species@0 < Nlpine Sheep ], sort_order_for_reorder=[species@0 ASC NULLS LAST], 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, 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, page_index_pages_skipped_by_fully_matched=1, limit_pruned_row_groups=0 total → 0 matched, scan_efficiency_ratio=21.75% (485/2.23 K)] +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/explain_analyze/data.parquet]]}, projection=[species, s], file_type=parquet, predicate=species@0 > M AND s@1 >= 50 AND DynamicFilter [ species@0 < Nlpine Sheep ], sort_order_for_reorder=[species@0 ASC NULLS LAST], dynamic_rg_pruning=eligible, 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, 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, page_index_pages_skipped_by_fully_matched=1, limit_pruned_row_groups=0 total → 0 matched, scan_efficiency_ratio=21.75% (485/2.23 K)] # ---- pgjson format: structural golden with no metrics ---- @@ -451,7 +451,7 @@ EXPLAIN (ANALYZE, METRICS rows, LEVEL summary) select * from cat_tracking where ---- 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] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/explain_analyze/data.parquet]]}, projection=[species, s], file_type=parquet, predicate=species@0 > M AND s@1 >= 50 AND DynamicFilter [ species@0 < Nlpine Sheep ], sort_order_for_reorder=[species@0 ASC NULLS LAST], 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, 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, page_index_pages_skipped_by_fully_matched=1, limit_pruned_row_groups=0 total → 0 matched, scan_efficiency_ratio=21.75% (485/2.23 K)] +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/explain_analyze/data.parquet]]}, projection=[species, s], file_type=parquet, predicate=species@0 > M AND s@1 >= 50 AND DynamicFilter [ species@0 < Nlpine Sheep ], sort_order_for_reorder=[species@0 ASC NULLS LAST], dynamic_rg_pruning=eligible, 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, 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, page_index_pages_skipped_by_fully_matched=1, limit_pruned_row_groups=0 total → 0 matched, scan_efficiency_ratio=21.75% (485/2.23 K)] statement ok reset datafusion.sql_parser.dialect; diff --git a/datafusion/sqllogictest/test_files/limit.slt b/datafusion/sqllogictest/test_files/limit.slt index ca2b36727d627..b9847059089ef 100644 --- a/datafusion/sqllogictest/test_files/limit.slt +++ b/datafusion/sqllogictest/test_files/limit.slt @@ -868,7 +868,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 ], sort_order_for_reorder=[part_key@0 ASC NULLS LAST] +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 ], sort_order_for_reorder=[part_key@0 ASC NULLS LAST], dynamic_rg_pruning=eligible query I with selection as ( diff --git a/datafusion/sqllogictest/test_files/limit_pruning.slt b/datafusion/sqllogictest/test_files/limit_pruning.slt index 3c3f0222f3736..4ef0b5c74f3e7 100644 --- a/datafusion/sqllogictest/test_files/limit_pruning.slt +++ b/datafusion/sqllogictest/test_files/limit_pruning.slt @@ -120,7 +120,7 @@ explain analyze select * from tracking_data where species > 'M' AND s >= 50 orde ---- 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 ], sort_order_for_reorder=[species@0 ASC NULLS LAST], 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=2 total → 2 matched, page_index_pages_skipped_by_fully_matched=1, limit_pruned_row_groups=0 total → 0 matched, bytes_scanned=, metadata_load_time=, scan_efficiency_ratio= (485/2.23 K)] +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 ], sort_order_for_reorder=[species@0 ASC NULLS LAST], dynamic_rg_pruning=eligible, 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=2 total → 2 matched, page_index_pages_skipped_by_fully_matched=1, limit_pruned_row_groups=0 total → 0 matched, bytes_scanned=, row_groups_pruned_dynamic_filter=0, metadata_load_time=, scan_efficiency_ratio= (/)] statement ok drop table tracking_data; diff --git a/datafusion/sqllogictest/test_files/preserve_file_partitioning.slt b/datafusion/sqllogictest/test_files/preserve_file_partitioning.slt index 175d7d90cd8ed..412d606df903f 100644 --- a/datafusion/sqllogictest/test_files/preserve_file_partitioning.slt +++ b/datafusion/sqllogictest/test_files/preserve_file_partitioning.slt @@ -367,7 +367,7 @@ physical_plan 08)--------------FilterExec: service@2 = log 09)----------------RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1 10)------------------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)] -11)------------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: 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 ], dynamic_rg_pruning=eligible # Verify results without optimization query TTTIR rowsort @@ -418,7 +418,7 @@ physical_plan 06)----------FilterExec: service@2 = log 07)------------RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1 08)--------------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)] -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 ] +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 ], dynamic_rg_pruning=eligible query TTTIR rowsort SELECT f.f_dkey, MAX(d.env), MAX(d.service), count(*), sum(f.value) @@ -643,7 +643,7 @@ physical_plan 05)--------RepartitionExec: partitioning=Hash([d_dkey@1], 3), input_partitions=3 06)----------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)--------RepartitionExec: partitioning=Hash([f_dkey@1], 3), input_partitions=3 -08)----------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 ] +08)----------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 ], dynamic_rg_pruning=eligible query TTR rowsort SELECT f.f_dkey, d.env, sum(f.value) diff --git a/datafusion/sqllogictest/test_files/projection_pushdown.slt b/datafusion/sqllogictest/test_files/projection_pushdown.slt index 344aef1f92cf9..c1cb8ed561e96 100644 --- a/datafusion/sqllogictest/test_files/projection_pushdown.slt +++ b/datafusion/sqllogictest/test_files/projection_pushdown.slt @@ -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 ], sort_order_for_reorder=[id@0 ASC NULLS LAST] +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 ], sort_order_for_reorder=[id@0 ASC NULLS LAST], dynamic_rg_pruning=eligible # 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 ], sort_order_for_reorder=[id@0 ASC NULLS LAST] +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 ], sort_order_for_reorder=[id@0 ASC NULLS LAST], dynamic_rg_pruning=eligible # 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 ], sort_order_for_reorder=[id@0 ASC NULLS LAST] +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 ], sort_order_for_reorder=[id@0 ASC NULLS LAST], dynamic_rg_pruning=eligible # 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 ], sort_order_for_reorder=[id@0 ASC NULLS LAST] +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 ], sort_order_for_reorder=[id@0 ASC NULLS LAST], dynamic_rg_pruning=eligible # 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 ], sort_order_for_reorder=[id@0 ASC NULLS LAST] +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 ], sort_order_for_reorder=[id@0 ASC NULLS LAST], dynamic_rg_pruning=eligible # Verify correctness query IT @@ -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 AND DynamicFilter [ empty ], pruning_predicate=id_null_count@1 != row_count@2 AND id_max@0 > 1, required_guarantees=[] +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 ], dynamic_rg_pruning=eligible, 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: 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 ], dynamic_rg_pruning=eligible, pruning_predicate=id_null_count@1 != row_count@2 AND id_max@0 > 1, required_guarantees=[] # Verify correctness query II @@ -713,7 +713,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 ], sort_order_for_reorder=[id@0 ASC NULLS LAST] +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 ], sort_order_for_reorder=[id@0 ASC NULLS LAST], dynamic_rg_pruning=eligible # Verify correctness query II @@ -737,7 +737,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 ], sort_order_for_reorder=[id@0 ASC NULLS LAST] +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 ], sort_order_for_reorder=[id@0 ASC NULLS LAST], dynamic_rg_pruning=eligible # Verify correctness query II @@ -874,7 +874,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 ], sort_order_for_reorder=[id@0 ASC NULLS LAST] +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 ], sort_order_for_reorder=[id@0 ASC NULLS LAST], dynamic_rg_pruning=eligible # Verify correctness query IIIT @@ -897,7 +897,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 ], sort_order_for_reorder=[id@0 ASC NULLS LAST] +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 ], sort_order_for_reorder=[id@0 ASC NULLS LAST], dynamic_rg_pruning=eligible # Verify correctness query II @@ -919,7 +919,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 ], sort_order_for_reorder=[id@0 ASC NULLS LAST] +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 ], sort_order_for_reorder=[id@0 ASC NULLS LAST], dynamic_rg_pruning=eligible # Verify correctness query I @@ -947,7 +947,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 ], sort_order_for_reorder=[id@0 ASC NULLS LAST] +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 ], sort_order_for_reorder=[id@0 ASC NULLS LAST], dynamic_rg_pruning=eligible # Verify correctness query II @@ -1039,7 +1039,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 ], sort_order_for_reorder=[id@0 ASC NULLS LAST] +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 ], sort_order_for_reorder=[id@0 ASC NULLS LAST], dynamic_rg_pruning=eligible # Verify correctness query II @@ -1095,7 +1095,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 ], sort_order_for_reorder=[id@0 ASC NULLS LAST] +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 ], sort_order_for_reorder=[id@0 ASC NULLS LAST], dynamic_rg_pruning=eligible # Verify correctness query IIT @@ -1118,7 +1118,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 ], sort_order_for_reorder=[id@0 ASC NULLS LAST] +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 ], sort_order_for_reorder=[id@0 ASC NULLS LAST], dynamic_rg_pruning=eligible # Verify correctness query IIT @@ -1317,7 +1317,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, predicate=DynamicFilter [ empty ] +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, predicate=DynamicFilter [ empty ], dynamic_rg_pruning=eligible # Verify correctness query I @@ -1424,7 +1424,7 @@ logical_plan 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, predicate=DynamicFilter [ empty ] +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, predicate=DynamicFilter [ empty ], dynamic_rg_pruning=eligible # Verify correctness - value = level * 10 # simple_struct: (1,100), (2,200), (3,150), (4,300), (5,250) @@ -1460,7 +1460,7 @@ 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, predicate=get_field(s@1, value) > 150 -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 ] +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 ], dynamic_rg_pruning=eligible # Verify correctness - id matches and value > 150 query II @@ -1500,7 +1500,7 @@ physical_plan 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, predicate=get_field(s@1, value) > 100 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=get_field(s@1, level) > 3 AND DynamicFilter [ empty ] +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=get_field(s@1, level) > 3 AND DynamicFilter [ empty ], dynamic_rg_pruning=eligible # Verify correctness - id matches, value > 100, and level > 3 # Matching ids where value > 100: 2(200), 3(150), 4(300), 5(250) @@ -1536,7 +1536,7 @@ physical_plan 01)ProjectionExec: expr=[id@0 as id, __datafusion_extracted_1@1 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=[id@1, __datafusion_extracted_1@0, __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 ] +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 ], dynamic_rg_pruning=eligible # Verify correctness query ITT @@ -1568,7 +1568,7 @@ logical_plan 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 ] +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 ], dynamic_rg_pruning=eligible # Verify correctness query II @@ -1607,7 +1607,7 @@ physical_plan 02)--HashJoinExec: mode=CollectLeft, join_type=Left, on=[(id@1, id@0)], projection=[id@1, __datafusion_extracted_2@0, __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 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=get_field(s@1, level) > 5 AND DynamicFilter [ empty ] +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=get_field(s@1, level) > 5 AND DynamicFilter [ empty ], dynamic_rg_pruning=eligible # 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) @@ -1899,7 +1899,7 @@ physical_plan 01)ProjectionExec: expr=[__datafusion_extracted_3@0 as s.s[value], __datafusion_extracted_4@1 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_3@4, __datafusion_extracted_4@1] 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 ] +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 ], dynamic_rg_pruning=eligible # Verify correctness - only admin roles match (ids 1 and 4) query II @@ -1935,7 +1935,7 @@ logical_plan 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 ] +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 ], dynamic_rg_pruning=eligible # Verify correctness - all rows match since value >> level for all ids # simple_struct: (1,100), (2,200), (3,150), (4,300), (5,250) diff --git a/datafusion/sqllogictest/test_files/push_down_filter_parquet.slt b/datafusion/sqllogictest/test_files/push_down_filter_parquet.slt index 40bfe79dcc633..786c9a79f78aa 100644 --- a/datafusion/sqllogictest/test_files/push_down_filter_parquet.slt +++ b/datafusion/sqllogictest/test_files/push_down_filter_parquet.slt @@ -158,7 +158,7 @@ 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], output_ordering=[k@0 ASC NULLS LAST], file_type=parquet 03)--RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1, maintains_sort_order=true -04)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/large_table.parquet]]}, projection=[k, v], output_ordering=[k@0 ASC NULLS LAST], 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=[] +04)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/large_table.parquet]]}, projection=[k, v], output_ordering=[k@0 ASC NULLS LAST], file_type=parquet, predicate=v@1 >= 50 AND DynamicFilter [ empty ], dynamic_rg_pruning=eligible, pruning_predicate=v_null_count@1 != row_count@2 AND v_max@0 >= 50, required_guarantees=[] statement ok drop table small_table; @@ -206,7 +206,7 @@ EXPLAIN ANALYZE SELECT t FROM topk_pushdown ORDER BY t * t LIMIT 10; ---- Plan with Metrics 01)SortExec: TopK(fetch=10), expr=[t@0 * t@0 ASC NULLS LAST], preserve_partitioning=[false], filter=[t@0 * t@0 < 1884329474306198481], metrics=[output_rows=10, output_batches=1, row_replacements=10] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/topk_pushdown.parquet]]}, projection=[t], output_ordering=[t@0 ASC NULLS LAST], file_type=parquet, predicate=DynamicFilter [ t@0 * t@0 < 1884329474306198481 ], metrics=[output_rows=128, output_batches=1, files_ranges_pruned_statistics=1 total → 1 matched, row_groups_pruned_statistics=782 total → 782 matched, row_groups_pruned_bloom_filter=782 total → 782 matched, page_index_pages_pruned=0 total → 0 matched, page_index_rows_pruned=0 total → 0 matched, limit_pruned_row_groups=0 total → 0 matched, batches_split=0, file_open_errors=0, file_scan_errors=0, files_opened=1, files_processed=1, num_predicate_creation_errors=0, predicate_evaluation_errors=0, pushdown_rows_matched=128, pushdown_rows_pruned=99.87 K, predicate_cache_inner_records=128, predicate_cache_records=128, scan_efficiency_ratio=64.87% (258.7 K/398.8 K)] +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/topk_pushdown.parquet]]}, projection=[t], output_ordering=[t@0 ASC NULLS LAST], file_type=parquet, predicate=DynamicFilter [ t@0 * t@0 < 1884329474306198481 ], dynamic_rg_pruning=eligible, metrics=[output_rows=128, output_batches=1, files_ranges_pruned_statistics=1 total → 1 matched, row_groups_pruned_statistics=782 total → 782 matched, row_groups_pruned_bloom_filter=782 total → 782 matched, page_index_pages_pruned=0 total → 0 matched, page_index_rows_pruned=0 total → 0 matched, limit_pruned_row_groups=0 total → 0 matched, batches_split=0, file_open_errors=0, file_scan_errors=0, files_opened=1, files_processed=1, num_predicate_creation_errors=0, predicate_evaluation_errors=0, pushdown_rows_matched=128, pushdown_rows_pruned=99.87 K, predicate_cache_inner_records=128, predicate_cache_records=128, scan_efficiency_ratio=64.87% (258.7 K/398.8 K)] statement ok reset datafusion.explain.analyze_categories; @@ -257,7 +257,7 @@ EXPLAIN SELECT * FROM topk_single_col ORDER BY b DESC LIMIT 1; ---- physical_plan 01)SortExec: TopK(fetch=1), expr=[b@1 DESC], preserve_partitioning=[false] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/topk_single_col.parquet]]}, projection=[a, b, c], file_type=parquet, predicate=DynamicFilter [ empty ], sort_order_for_reorder=[b@1 DESC], reverse_row_groups=true +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/topk_single_col.parquet]]}, projection=[a, b, c], file_type=parquet, predicate=DynamicFilter [ empty ], sort_order_for_reorder=[b@1 DESC], reverse_row_groups=true, dynamic_rg_pruning=eligible statement ok set datafusion.explain.analyze_categories = 'rows'; @@ -268,7 +268,7 @@ EXPLAIN ANALYZE SELECT * FROM topk_single_col ORDER BY b DESC LIMIT 1; ---- Plan with Metrics 01)SortExec: TopK(fetch=1), expr=[b@1 DESC], preserve_partitioning=[false], filter=[b@1 IS NULL OR b@1 > bd], metrics=[output_rows=1, output_batches=1, row_replacements=1] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/topk_single_col.parquet]]}, projection=[a, b, c], file_type=parquet, predicate=DynamicFilter [ b@1 IS NULL OR b@1 > bd ], sort_order_for_reorder=[b@1 DESC], reverse_row_groups=true, pruning_predicate=b_null_count@0 > 0 OR b_null_count@0 != row_count@2 AND b_max@1 > bd, required_guarantees=[], metrics=[output_rows=4, output_batches=1, files_ranges_pruned_statistics=1 total → 1 matched, row_groups_pruned_statistics=1 total → 1 matched, row_groups_pruned_bloom_filter=1 total → 1 matched, page_index_pages_pruned=0 total → 0 matched, page_index_rows_pruned=0 total → 0 matched, limit_pruned_row_groups=0 total → 0 matched, batches_split=0, file_open_errors=0, file_scan_errors=0, files_opened=1, files_processed=1, num_predicate_creation_errors=0, predicate_evaluation_errors=0, pushdown_rows_matched=4, pushdown_rows_pruned=0, predicate_cache_inner_records=4, predicate_cache_records=4, scan_efficiency_ratio=21.62% (222/1.03 K)] +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/topk_single_col.parquet]]}, projection=[a, b, c], file_type=parquet, predicate=DynamicFilter [ b@1 IS NULL OR b@1 > bd ], sort_order_for_reorder=[b@1 DESC], reverse_row_groups=true, dynamic_rg_pruning=eligible, pruning_predicate=b_null_count@0 > 0 OR b_null_count@0 != row_count@2 AND b_max@1 > bd, required_guarantees=[], metrics=[output_rows=4, output_batches=1, files_ranges_pruned_statistics=1 total → 1 matched, row_groups_pruned_statistics=1 total → 1 matched, row_groups_pruned_bloom_filter=1 total → 1 matched, page_index_pages_pruned=0 total → 0 matched, page_index_rows_pruned=0 total → 0 matched, limit_pruned_row_groups=0 total → 0 matched, batches_split=0, file_open_errors=0, file_scan_errors=0, files_opened=1, files_processed=1, num_predicate_creation_errors=0, predicate_evaluation_errors=0, pushdown_rows_matched=4, pushdown_rows_pruned=0, predicate_cache_inner_records=4, predicate_cache_records=4, scan_efficiency_ratio=21.62% (222/1.03 K)] statement ok reset datafusion.explain.analyze_categories; @@ -319,7 +319,7 @@ EXPLAIN ANALYZE SELECT * FROM topk_multi_col ORDER BY b ASC NULLS LAST, a DESC L ---- Plan with Metrics 01)SortExec: TopK(fetch=2), expr=[b@1 ASC NULLS LAST, a@0 DESC], preserve_partitioning=[false], filter=[b@1 < bb OR b@1 = bb AND (a@0 IS NULL OR a@0 > ac)], metrics=[output_rows=2, output_batches=1, row_replacements=2] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/topk_multi_col.parquet]]}, projection=[a, b, c], file_type=parquet, predicate=DynamicFilter [ b@1 < bb OR b@1 = bb AND (a@0 IS NULL OR a@0 > ac) ], sort_order_for_reorder=[b@1 ASC NULLS LAST, a@0 DESC], pruning_predicate=b_null_count@1 != row_count@2 AND b_min@0 < bb OR b_null_count@1 != row_count@2 AND b_min@0 <= bb AND bb <= b_max@3 AND (a_null_count@4 > 0 OR a_null_count@4 != row_count@2 AND a_max@5 > ac), required_guarantees=[], metrics=[output_rows=4, output_batches=1, files_ranges_pruned_statistics=1 total → 1 matched, row_groups_pruned_statistics=1 total → 1 matched, row_groups_pruned_bloom_filter=1 total → 1 matched, page_index_pages_pruned=0 total → 0 matched, page_index_rows_pruned=0 total → 0 matched, limit_pruned_row_groups=0 total → 0 matched, batches_split=0, file_open_errors=0, file_scan_errors=0, files_opened=1, files_processed=1, num_predicate_creation_errors=0, predicate_evaluation_errors=0, pushdown_rows_matched=4, pushdown_rows_pruned=0, predicate_cache_inner_records=8, predicate_cache_records=8, scan_efficiency_ratio=21.62% (222/1.03 K)] +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/topk_multi_col.parquet]]}, projection=[a, b, c], file_type=parquet, predicate=DynamicFilter [ b@1 < bb OR b@1 = bb AND (a@0 IS NULL OR a@0 > ac) ], sort_order_for_reorder=[b@1 ASC NULLS LAST, a@0 DESC], dynamic_rg_pruning=eligible, pruning_predicate=b_null_count@1 != row_count@2 AND b_min@0 < bb OR b_null_count@1 != row_count@2 AND b_min@0 <= bb AND bb <= b_max@3 AND (a_null_count@4 > 0 OR a_null_count@4 != row_count@2 AND a_max@5 > ac), required_guarantees=[], metrics=[output_rows=4, output_batches=1, files_ranges_pruned_statistics=1 total → 1 matched, row_groups_pruned_statistics=1 total → 1 matched, row_groups_pruned_bloom_filter=1 total → 1 matched, page_index_pages_pruned=0 total → 0 matched, page_index_rows_pruned=0 total → 0 matched, limit_pruned_row_groups=0 total → 0 matched, batches_split=0, file_open_errors=0, file_scan_errors=0, files_opened=1, files_processed=1, num_predicate_creation_errors=0, predicate_evaluation_errors=0, pushdown_rows_matched=4, pushdown_rows_pruned=0, predicate_cache_inner_records=8, predicate_cache_records=8, scan_efficiency_ratio=21.62% (222/1.03 K)] statement ok reset datafusion.explain.analyze_categories; @@ -389,7 +389,7 @@ FROM join_probe p INNER JOIN join_build AS build Plan with Metrics 01)HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(a@0, a@0), (b@1, b@1)], projection=[a@3, b@4, c@2, e@5], metrics=[output_rows=2, output_batches=1, array_map_created_count=0, build_input_batches=1, build_input_rows=2, input_batches=1, input_rows=2, avg_fanout=100% (2/2), probe_hit_rate=100% (2/2)] 02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/join_build.parquet]]}, projection=[a, b, c], file_type=parquet, metrics=[output_rows=2, output_batches=1, files_ranges_pruned_statistics=1 total → 1 matched, row_groups_pruned_statistics=1 total → 1 matched, row_groups_pruned_bloom_filter=1 total → 1 matched, page_index_pages_pruned=0 total → 0 matched, page_index_rows_pruned=0 total → 0 matched, limit_pruned_row_groups=0 total → 0 matched, batches_split=0, file_open_errors=0, file_scan_errors=0, files_opened=1, files_processed=1, num_predicate_creation_errors=0, predicate_evaluation_errors=0, pushdown_rows_matched=0, pushdown_rows_pruned=0, predicate_cache_inner_records=0, predicate_cache_records=0, scan_efficiency_ratio=19.58% (196/1.00 K)] -03)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/join_probe.parquet]]}, projection=[a, b, e], file_type=parquet, predicate=DynamicFilter [ a@0 >= aa AND a@0 <= ab AND b@1 >= ba AND b@1 <= bb AND struct(a@0, b@1) IN (SET) ([{c0:aa,c1:ba}, {c0:ab,c1:bb}]) ], pruning_predicate=a_null_count@1 != row_count@2 AND a_max@0 >= aa AND a_null_count@1 != row_count@2 AND a_min@3 <= ab AND b_null_count@5 != row_count@2 AND b_max@4 >= ba AND b_null_count@5 != row_count@2 AND b_min@6 <= bb, required_guarantees=[], metrics=[output_rows=2, output_batches=1, files_ranges_pruned_statistics=1 total → 1 matched, row_groups_pruned_statistics=1 total → 1 matched, row_groups_pruned_bloom_filter=1 total → 1 matched, page_index_pages_pruned=0 total → 0 matched, page_index_rows_pruned=0 total → 0 matched, limit_pruned_row_groups=0 total → 0 matched, batches_split=0, file_open_errors=0, file_scan_errors=0, files_opened=1, files_processed=1, num_predicate_creation_errors=0, predicate_evaluation_errors=0, pushdown_rows_matched=2, pushdown_rows_pruned=2, predicate_cache_inner_records=8, predicate_cache_records=4, scan_efficiency_ratio=22.05% (228/1.03 K)] +03)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/join_probe.parquet]]}, projection=[a, b, e], file_type=parquet, predicate=DynamicFilter [ a@0 >= aa AND a@0 <= ab AND b@1 >= ba AND b@1 <= bb AND struct(a@0, b@1) IN (SET) ([{c0:aa,c1:ba}, {c0:ab,c1:bb}]) ], dynamic_rg_pruning=eligible, pruning_predicate=a_null_count@1 != row_count@2 AND a_max@0 >= aa AND a_null_count@1 != row_count@2 AND a_min@3 <= ab AND b_null_count@5 != row_count@2 AND b_max@4 >= ba AND b_null_count@5 != row_count@2 AND b_min@6 <= bb, required_guarantees=[], metrics=[output_rows=2, output_batches=1, files_ranges_pruned_statistics=1 total → 1 matched, row_groups_pruned_statistics=1 total → 1 matched, row_groups_pruned_bloom_filter=1 total → 1 matched, page_index_pages_pruned=0 total → 0 matched, page_index_rows_pruned=0 total → 0 matched, limit_pruned_row_groups=0 total → 0 matched, batches_split=0, file_open_errors=0, file_scan_errors=0, files_opened=1, files_processed=1, num_predicate_creation_errors=0, predicate_evaluation_errors=0, pushdown_rows_matched=2, pushdown_rows_pruned=2, predicate_cache_inner_records=8, predicate_cache_records=4, scan_efficiency_ratio=22.05% (228/1.03 K)] statement ok reset datafusion.explain.analyze_categories; @@ -475,8 +475,8 @@ Plan with Metrics 01)HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c@3, d@0)], metrics=[output_rows=2, output_batches=1, array_map_created_count=0, build_input_batches=1, build_input_rows=2, input_batches=1, input_rows=2, avg_fanout=100% (2/2), probe_hit_rate=100% (2/2)] 02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(a@0, b@0)], metrics=[output_rows=2, output_batches=1, array_map_created_count=0, build_input_batches=1, build_input_rows=2, input_batches=1, input_rows=2, avg_fanout=100% (2/2), probe_hit_rate=100% (2/2)] 03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/nested_t1.parquet]]}, projection=[a, x], file_type=parquet, metrics=[output_rows=2, output_batches=1, files_ranges_pruned_statistics=1 total → 1 matched, row_groups_pruned_statistics=1 total → 1 matched, row_groups_pruned_bloom_filter=1 total → 1 matched, page_index_pages_pruned=0 total → 0 matched, page_index_rows_pruned=0 total → 0 matched, limit_pruned_row_groups=0 total → 0 matched, batches_split=0, file_open_errors=0, file_scan_errors=0, files_opened=1, files_processed=1, num_predicate_creation_errors=0, predicate_evaluation_errors=0, pushdown_rows_matched=0, pushdown_rows_pruned=0, predicate_cache_inner_records=0, predicate_cache_records=0, scan_efficiency_ratio=17.37% (132/760)] -04)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/nested_t2.parquet]]}, projection=[b, c, y], file_type=parquet, predicate=DynamicFilter [ b@0 >= aa AND b@0 <= ab AND b@0 IN (SET) ([aa, ab]) ], pruning_predicate=b_null_count@1 != row_count@2 AND b_max@0 >= aa AND b_null_count@1 != row_count@2 AND b_min@3 <= ab AND (b_null_count@1 != row_count@2 AND b_min@3 <= aa AND aa <= b_max@0 OR b_null_count@1 != row_count@2 AND b_min@3 <= ab AND ab <= b_max@0), required_guarantees=[b in (aa, ab)], metrics=[output_rows=2, output_batches=1, files_ranges_pruned_statistics=1 total → 1 matched, row_groups_pruned_statistics=1 total → 1 matched, row_groups_pruned_bloom_filter=1 total → 1 matched, page_index_pages_pruned=1 total → 1 matched, page_index_rows_pruned=5 total → 5 matched, limit_pruned_row_groups=0 total → 0 matched, batches_split=0, file_open_errors=0, file_scan_errors=0, files_opened=1, files_processed=1, num_predicate_creation_errors=0, predicate_evaluation_errors=0, pushdown_rows_matched=2, pushdown_rows_pruned=3, predicate_cache_inner_records=5, predicate_cache_records=2, scan_efficiency_ratio=22.46% (234/1.04 K)] -05)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/nested_t3.parquet]]}, projection=[d, z], file_type=parquet, predicate=DynamicFilter [ d@0 >= ca AND d@0 <= cb AND hash_lookup ], pruning_predicate=d_null_count@1 != row_count@2 AND d_max@0 >= ca AND d_null_count@1 != row_count@2 AND d_min@3 <= cb, required_guarantees=[], metrics=[output_rows=2, output_batches=1, files_ranges_pruned_statistics=1 total → 1 matched, row_groups_pruned_statistics=1 total → 1 matched, row_groups_pruned_bloom_filter=1 total → 1 matched, page_index_pages_pruned=1 total → 1 matched, page_index_rows_pruned=8 total → 8 matched, limit_pruned_row_groups=0 total → 0 matched, batches_split=0, file_open_errors=0, file_scan_errors=0, files_opened=1, files_processed=1, num_predicate_creation_errors=0, predicate_evaluation_errors=0, pushdown_rows_matched=2, pushdown_rows_pruned=6, predicate_cache_inner_records=8, predicate_cache_records=2, scan_efficiency_ratio=21.45% (172/802)] +04)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/nested_t2.parquet]]}, projection=[b, c, y], file_type=parquet, predicate=DynamicFilter [ b@0 >= aa AND b@0 <= ab AND b@0 IN (SET) ([aa, ab]) ], dynamic_rg_pruning=eligible, pruning_predicate=b_null_count@1 != row_count@2 AND b_max@0 >= aa AND b_null_count@1 != row_count@2 AND b_min@3 <= ab AND (b_null_count@1 != row_count@2 AND b_min@3 <= aa AND aa <= b_max@0 OR b_null_count@1 != row_count@2 AND b_min@3 <= ab AND ab <= b_max@0), required_guarantees=[b in (aa, ab)], metrics=[output_rows=2, output_batches=1, files_ranges_pruned_statistics=1 total → 1 matched, row_groups_pruned_statistics=1 total → 1 matched, row_groups_pruned_bloom_filter=1 total → 1 matched, page_index_pages_pruned=1 total → 1 matched, page_index_rows_pruned=5 total → 5 matched, limit_pruned_row_groups=0 total → 0 matched, batches_split=0, file_open_errors=0, file_scan_errors=0, files_opened=1, files_processed=1, num_predicate_creation_errors=0, predicate_evaluation_errors=0, pushdown_rows_matched=2, pushdown_rows_pruned=3, predicate_cache_inner_records=5, predicate_cache_records=2, scan_efficiency_ratio=22.46% (234/1.04 K)] +05)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/nested_t3.parquet]]}, projection=[d, z], file_type=parquet, predicate=DynamicFilter [ d@0 >= ca AND d@0 <= cb AND hash_lookup ], dynamic_rg_pruning=eligible, pruning_predicate=d_null_count@1 != row_count@2 AND d_max@0 >= ca AND d_null_count@1 != row_count@2 AND d_min@3 <= cb, required_guarantees=[], metrics=[output_rows=2, output_batches=1, files_ranges_pruned_statistics=1 total → 1 matched, row_groups_pruned_statistics=1 total → 1 matched, row_groups_pruned_bloom_filter=1 total → 1 matched, page_index_pages_pruned=1 total → 1 matched, page_index_rows_pruned=8 total → 8 matched, limit_pruned_row_groups=0 total → 0 matched, batches_split=0, file_open_errors=0, file_scan_errors=0, files_opened=1, files_processed=1, num_predicate_creation_errors=0, predicate_evaluation_errors=0, pushdown_rows_matched=2, pushdown_rows_pruned=6, predicate_cache_inner_records=8, predicate_cache_records=2, scan_efficiency_ratio=21.45% (172/802)] statement ok reset datafusion.explain.analyze_categories; @@ -541,7 +541,7 @@ physical_plan 01)HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(a@0, d@0)] 02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/parent_build.parquet]]}, projection=[a, b, c], file_type=parquet, predicate=a@0 = aa, pruning_predicate=a_null_count@2 != row_count@3 AND a_min@0 <= aa AND aa <= a_max@1, required_guarantees=[a in (aa)] 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/parent_probe.parquet]]}, projection=[d, e, f], file_type=parquet, predicate=e@1 = ba AND d@0 = aa AND DynamicFilter [ empty ], pruning_predicate=e_null_count@2 != row_count@3 AND e_min@0 <= ba AND ba <= e_max@1 AND d_null_count@6 != row_count@3 AND d_min@4 <= aa AND aa <= d_max@5, required_guarantees=[d in (aa), e in (ba)] +04)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/parent_probe.parquet]]}, projection=[d, e, f], file_type=parquet, predicate=e@1 = ba AND d@0 = aa AND DynamicFilter [ empty ], dynamic_rg_pruning=eligible, pruning_predicate=e_null_count@2 != row_count@3 AND e_min@0 <= ba AND ba <= e_max@1 AND d_null_count@6 != row_count@3 AND d_min@4 <= aa AND aa <= d_max@5, required_guarantees=[d in (aa), e in (ba)] statement ok drop table parent_build; @@ -606,7 +606,7 @@ Plan with Metrics 01)SortExec: TopK(fetch=2), expr=[e@0 ASC NULLS LAST], preserve_partitioning=[false], filter=[e@0 < bb], metrics=[output_rows=2, output_batches=1, row_replacements=2] 02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(a@0, d@0)], projection=[e@2], metrics=[output_rows=2, output_batches=1, array_map_created_count=0, build_input_batches=1, build_input_rows=2, input_batches=1, input_rows=2, avg_fanout=100% (2/2), probe_hit_rate=100% (2/2)] 03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/topk_join_build.parquet]]}, projection=[a], file_type=parquet, metrics=[output_rows=2, output_batches=1, files_ranges_pruned_statistics=1 total → 1 matched, row_groups_pruned_statistics=1 total → 1 matched, row_groups_pruned_bloom_filter=1 total → 1 matched, page_index_pages_pruned=0 total → 0 matched, page_index_rows_pruned=0 total → 0 matched, limit_pruned_row_groups=0 total → 0 matched, batches_split=0, file_open_errors=0, file_scan_errors=0, files_opened=1, files_processed=1, num_predicate_creation_errors=0, predicate_evaluation_errors=0, pushdown_rows_matched=0, pushdown_rows_pruned=0, predicate_cache_inner_records=0, predicate_cache_records=0, scan_efficiency_ratio=6.39% (64/1.00 K)] -04)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/topk_join_probe.parquet]]}, projection=[d, e], file_type=parquet, predicate=DynamicFilter [ d@0 >= aa AND d@0 <= ab AND d@0 IN (SET) ([aa, ab]) ] AND DynamicFilter [ e@1 < bb ], pruning_predicate=d_null_count@1 != row_count@2 AND d_max@0 >= aa AND d_null_count@1 != row_count@2 AND d_min@3 <= ab AND (d_null_count@1 != row_count@2 AND d_min@3 <= aa AND aa <= d_max@0 OR d_null_count@1 != row_count@2 AND d_min@3 <= ab AND ab <= d_max@0) AND e_null_count@5 != row_count@2 AND e_min@4 < bb, required_guarantees=[d in (aa, ab)], metrics=[output_rows=2, output_batches=1, files_ranges_pruned_statistics=1 total → 1 matched, row_groups_pruned_statistics=1 total → 1 matched, row_groups_pruned_bloom_filter=1 total → 1 matched, page_index_pages_pruned=1 total → 1 matched, page_index_rows_pruned=4 total → 4 matched, limit_pruned_row_groups=0 total → 0 matched, batches_split=0, file_open_errors=0, file_scan_errors=0, files_opened=1, files_processed=1, num_predicate_creation_errors=0, predicate_evaluation_errors=0, pushdown_rows_matched=2, pushdown_rows_pruned=2, predicate_cache_inner_records=8, predicate_cache_records=4, scan_efficiency_ratio=14.89% (154/1.03 K)] +04)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/topk_join_probe.parquet]]}, projection=[d, e], file_type=parquet, predicate=DynamicFilter [ d@0 >= aa AND d@0 <= ab AND d@0 IN (SET) ([aa, ab]) ] AND DynamicFilter [ e@1 < bb ], dynamic_rg_pruning=eligible, pruning_predicate=d_null_count@1 != row_count@2 AND d_max@0 >= aa AND d_null_count@1 != row_count@2 AND d_min@3 <= ab AND (d_null_count@1 != row_count@2 AND d_min@3 <= aa AND aa <= d_max@0 OR d_null_count@1 != row_count@2 AND d_min@3 <= ab AND ab <= d_max@0) AND e_null_count@5 != row_count@2 AND e_min@4 < bb, required_guarantees=[d in (aa, ab)], metrics=[output_rows=2, output_batches=1, files_ranges_pruned_statistics=1 total → 1 matched, row_groups_pruned_statistics=1 total → 1 matched, row_groups_pruned_bloom_filter=1 total → 1 matched, page_index_pages_pruned=1 total → 1 matched, page_index_rows_pruned=4 total → 4 matched, limit_pruned_row_groups=0 total → 0 matched, batches_split=0, file_open_errors=0, file_scan_errors=0, files_opened=1, files_processed=1, num_predicate_creation_errors=0, predicate_evaluation_errors=0, pushdown_rows_matched=2, pushdown_rows_pruned=2, predicate_cache_inner_records=8, predicate_cache_records=4, scan_efficiency_ratio=14.89% (154/1.03 K)] statement ok reset datafusion.explain.analyze_categories; @@ -655,7 +655,7 @@ EXPLAIN ANALYZE SELECT b, a FROM topk_proj ORDER BY a LIMIT 2; ---- Plan with Metrics 01)SortExec: TopK(fetch=2), expr=[a@1 ASC NULLS LAST], preserve_partitioning=[false], filter=[a@1 < 2], metrics=[output_rows=2, output_batches=1, row_replacements=2] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/topk_proj.parquet]]}, projection=[b, a], file_type=parquet, predicate=DynamicFilter [ a@0 < 2 ], sort_order_for_reorder=[a@1 ASC NULLS LAST], pruning_predicate=a_null_count@1 != row_count@2 AND a_min@0 < 2, required_guarantees=[], metrics=[output_rows=3, output_batches=1, files_ranges_pruned_statistics=1 total → 1 matched, row_groups_pruned_statistics=1 total → 1 matched, row_groups_pruned_bloom_filter=1 total → 1 matched, page_index_pages_pruned=0 total → 0 matched, page_index_rows_pruned=0 total → 0 matched, limit_pruned_row_groups=0 total → 0 matched, batches_split=0, file_open_errors=0, file_scan_errors=0, files_opened=1, files_processed=1, num_predicate_creation_errors=0, predicate_evaluation_errors=0, pushdown_rows_matched=3, pushdown_rows_pruned=0, predicate_cache_inner_records=3, predicate_cache_records=3, scan_efficiency_ratio=13.21% (141/1.07 K)] +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/topk_proj.parquet]]}, projection=[b, a], file_type=parquet, predicate=DynamicFilter [ a@0 < 2 ], sort_order_for_reorder=[a@1 ASC NULLS LAST], dynamic_rg_pruning=eligible, pruning_predicate=a_null_count@1 != row_count@2 AND a_min@0 < 2, required_guarantees=[], metrics=[output_rows=3, output_batches=1, files_ranges_pruned_statistics=1 total → 1 matched, row_groups_pruned_statistics=1 total → 1 matched, row_groups_pruned_bloom_filter=1 total → 1 matched, page_index_pages_pruned=0 total → 0 matched, page_index_rows_pruned=0 total → 0 matched, limit_pruned_row_groups=0 total → 0 matched, batches_split=0, file_open_errors=0, file_scan_errors=0, files_opened=1, files_processed=1, num_predicate_creation_errors=0, predicate_evaluation_errors=0, pushdown_rows_matched=3, pushdown_rows_pruned=0, predicate_cache_inner_records=3, predicate_cache_records=3, scan_efficiency_ratio=13.21% (141/1.07 K)] # Case 2: prune — `SELECT a` — filter stays as `a < 2` on the scan. query TT @@ -663,7 +663,7 @@ EXPLAIN ANALYZE SELECT a FROM topk_proj ORDER BY a LIMIT 2; ---- Plan with Metrics 01)SortExec: TopK(fetch=2), expr=[a@0 ASC NULLS LAST], preserve_partitioning=[false], filter=[a@0 < 2], metrics=[output_rows=2, output_batches=1, row_replacements=2] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/topk_proj.parquet]]}, projection=[a], file_type=parquet, predicate=DynamicFilter [ a@0 < 2 ], sort_order_for_reorder=[a@0 ASC NULLS LAST], pruning_predicate=a_null_count@1 != row_count@2 AND a_min@0 < 2, required_guarantees=[], metrics=[output_rows=3, output_batches=1, files_ranges_pruned_statistics=1 total → 1 matched, row_groups_pruned_statistics=1 total → 1 matched, row_groups_pruned_bloom_filter=1 total → 1 matched, page_index_pages_pruned=0 total → 0 matched, page_index_rows_pruned=0 total → 0 matched, limit_pruned_row_groups=0 total → 0 matched, batches_split=0, file_open_errors=0, file_scan_errors=0, files_opened=1, files_processed=1, num_predicate_creation_errors=0, predicate_evaluation_errors=0, pushdown_rows_matched=3, pushdown_rows_pruned=0, predicate_cache_inner_records=3, predicate_cache_records=3, scan_efficiency_ratio=6.84% (73/1.07 K)] +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/topk_proj.parquet]]}, projection=[a], file_type=parquet, predicate=DynamicFilter [ a@0 < 2 ], sort_order_for_reorder=[a@0 ASC NULLS LAST], dynamic_rg_pruning=eligible, pruning_predicate=a_null_count@1 != row_count@2 AND a_min@0 < 2, required_guarantees=[], metrics=[output_rows=3, output_batches=1, files_ranges_pruned_statistics=1 total → 1 matched, row_groups_pruned_statistics=1 total → 1 matched, row_groups_pruned_bloom_filter=1 total → 1 matched, page_index_pages_pruned=0 total → 0 matched, page_index_rows_pruned=0 total → 0 matched, limit_pruned_row_groups=0 total → 0 matched, batches_split=0, file_open_errors=0, file_scan_errors=0, files_opened=1, files_processed=1, num_predicate_creation_errors=0, predicate_evaluation_errors=0, pushdown_rows_matched=3, pushdown_rows_pruned=0, predicate_cache_inner_records=3, predicate_cache_records=3, scan_efficiency_ratio=6.84% (73/1.07 K)] # Case 3: expression — `SELECT a+1 AS a_plus_1` — the TopK filter is on # `a_plus_1`, the scan predicate must read `a@0 + 1`. @@ -672,7 +672,7 @@ EXPLAIN ANALYZE SELECT a + 1 AS a_plus_1, b FROM topk_proj ORDER BY a_plus_1 LIM ---- Plan with Metrics 01)SortExec: TopK(fetch=2), expr=[a_plus_1@0 ASC NULLS LAST], preserve_partitioning=[false], filter=[a_plus_1@0 < 3], metrics=[output_rows=2, output_batches=1, row_replacements=2] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/topk_proj.parquet]]}, projection=[CAST(a@0 AS Int64) + 1 as a_plus_1, b], file_type=parquet, predicate=DynamicFilter [ CAST(a@0 AS Int64) + 1 < 3 ], metrics=[output_rows=3, output_batches=1, files_ranges_pruned_statistics=1 total → 1 matched, row_groups_pruned_statistics=1 total → 1 matched, row_groups_pruned_bloom_filter=1 total → 1 matched, page_index_pages_pruned=0 total → 0 matched, page_index_rows_pruned=0 total → 0 matched, limit_pruned_row_groups=0 total → 0 matched, batches_split=0, file_open_errors=0, file_scan_errors=0, files_opened=1, files_processed=1, num_predicate_creation_errors=0, predicate_evaluation_errors=0, pushdown_rows_matched=3, pushdown_rows_pruned=0, predicate_cache_inner_records=3, predicate_cache_records=3, scan_efficiency_ratio=13.21% (141/1.07 K)] +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/topk_proj.parquet]]}, projection=[CAST(a@0 AS Int64) + 1 as a_plus_1, b], file_type=parquet, predicate=DynamicFilter [ CAST(a@0 AS Int64) + 1 < 3 ], dynamic_rg_pruning=eligible, metrics=[output_rows=3, output_batches=1, files_ranges_pruned_statistics=1 total → 1 matched, row_groups_pruned_statistics=1 total → 1 matched, row_groups_pruned_bloom_filter=1 total → 1 matched, page_index_pages_pruned=0 total → 0 matched, page_index_rows_pruned=0 total → 0 matched, limit_pruned_row_groups=0 total → 0 matched, batches_split=0, file_open_errors=0, file_scan_errors=0, files_opened=1, files_processed=1, num_predicate_creation_errors=0, predicate_evaluation_errors=0, pushdown_rows_matched=3, pushdown_rows_pruned=0, predicate_cache_inner_records=3, predicate_cache_records=3, scan_efficiency_ratio=13.21% (141/1.07 K)] # Case 4: alias shadowing — `SELECT a+1 AS a` — the projection renames # `a+1` to `a`, so the TopK's `a < 3` must still be rewritten to @@ -682,7 +682,7 @@ EXPLAIN ANALYZE SELECT a + 1 AS a, b FROM topk_proj ORDER BY a LIMIT 2; ---- Plan with Metrics 01)SortExec: TopK(fetch=2), expr=[a@0 ASC NULLS LAST], preserve_partitioning=[false], filter=[a@0 < 3], metrics=[output_rows=2, output_batches=1, row_replacements=2] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/topk_proj.parquet]]}, projection=[CAST(a@0 AS Int64) + 1 as a, b], file_type=parquet, predicate=DynamicFilter [ CAST(a@0 AS Int64) + 1 < 3 ], sort_order_for_reorder=[a@0 ASC NULLS LAST], metrics=[output_rows=3, output_batches=1, files_ranges_pruned_statistics=1 total → 1 matched, row_groups_pruned_statistics=1 total → 1 matched, row_groups_pruned_bloom_filter=1 total → 1 matched, page_index_pages_pruned=0 total → 0 matched, page_index_rows_pruned=0 total → 0 matched, limit_pruned_row_groups=0 total → 0 matched, batches_split=0, file_open_errors=0, file_scan_errors=0, files_opened=1, files_processed=1, num_predicate_creation_errors=0, predicate_evaluation_errors=0, pushdown_rows_matched=3, pushdown_rows_pruned=0, predicate_cache_inner_records=3, predicate_cache_records=3, scan_efficiency_ratio=13.21% (141/1.07 K)] +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/topk_proj.parquet]]}, projection=[CAST(a@0 AS Int64) + 1 as a, b], file_type=parquet, predicate=DynamicFilter [ CAST(a@0 AS Int64) + 1 < 3 ], sort_order_for_reorder=[a@0 ASC NULLS LAST], dynamic_rg_pruning=eligible, metrics=[output_rows=3, output_batches=1, files_ranges_pruned_statistics=1 total → 1 matched, row_groups_pruned_statistics=1 total → 1 matched, row_groups_pruned_bloom_filter=1 total → 1 matched, page_index_pages_pruned=0 total → 0 matched, page_index_rows_pruned=0 total → 0 matched, limit_pruned_row_groups=0 total → 0 matched, batches_split=0, file_open_errors=0, file_scan_errors=0, files_opened=1, files_processed=1, num_predicate_creation_errors=0, predicate_evaluation_errors=0, pushdown_rows_matched=3, pushdown_rows_pruned=0, predicate_cache_inner_records=3, predicate_cache_records=3, scan_efficiency_ratio=13.21% (141/1.07 K)] statement ok reset datafusion.explain.analyze_categories; @@ -744,7 +744,7 @@ Plan with Metrics 04)----AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[min(join_agg_probe.value)], metrics=[output_rows=2, output_batches=2, spill_count=0, spilled_rows=0] 05)------RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=1, metrics=[output_rows=2, output_batches=2, spill_count=0, spilled_rows=0] 06)--------AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[min(join_agg_probe.value)], metrics=[output_rows=2, output_batches=1, spill_count=0, spilled_rows=0, skipped_aggregation_rows=0, reduction_factor=100% (2/2)] -07)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/join_agg_probe.parquet]]}, projection=[a, value], file_type=parquet, predicate=DynamicFilter [ a@0 >= h1 AND a@0 <= h2 AND a@0 IN (SET) ([h1, h2]) ], pruning_predicate=a_null_count@1 != row_count@2 AND a_max@0 >= h1 AND a_null_count@1 != row_count@2 AND a_min@3 <= h2 AND (a_null_count@1 != row_count@2 AND a_min@3 <= h1 AND h1 <= a_max@0 OR a_null_count@1 != row_count@2 AND a_min@3 <= h2 AND h2 <= a_max@0), required_guarantees=[a in (h1, h2)], metrics=[output_rows=2, output_batches=1, files_ranges_pruned_statistics=1 total → 1 matched, row_groups_pruned_statistics=1 total → 1 matched, row_groups_pruned_bloom_filter=1 total → 1 matched, page_index_pages_pruned=1 total → 1 matched, page_index_rows_pruned=4 total → 4 matched, limit_pruned_row_groups=0 total → 0 matched, batches_split=0, file_open_errors=0, file_scan_errors=0, files_opened=1, files_processed=1, num_predicate_creation_errors=0, predicate_evaluation_errors=0, pushdown_rows_matched=2, pushdown_rows_pruned=2, predicate_cache_inner_records=4, predicate_cache_records=2, scan_efficiency_ratio=19.07% (151/792)] +07)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/join_agg_probe.parquet]]}, projection=[a, value], file_type=parquet, predicate=DynamicFilter [ a@0 >= h1 AND a@0 <= h2 AND a@0 IN (SET) ([h1, h2]) ], dynamic_rg_pruning=eligible, pruning_predicate=a_null_count@1 != row_count@2 AND a_max@0 >= h1 AND a_null_count@1 != row_count@2 AND a_min@3 <= h2 AND (a_null_count@1 != row_count@2 AND a_min@3 <= h1 AND h1 <= a_max@0 OR a_null_count@1 != row_count@2 AND a_min@3 <= h2 AND h2 <= a_max@0), required_guarantees=[a in (h1, h2)], metrics=[output_rows=2, output_batches=1, files_ranges_pruned_statistics=1 total → 1 matched, row_groups_pruned_statistics=1 total → 1 matched, row_groups_pruned_bloom_filter=1 total → 1 matched, page_index_pages_pruned=1 total → 1 matched, page_index_rows_pruned=4 total → 4 matched, limit_pruned_row_groups=0 total → 0 matched, batches_split=0, file_open_errors=0, file_scan_errors=0, files_opened=1, files_processed=1, num_predicate_creation_errors=0, predicate_evaluation_errors=0, pushdown_rows_matched=2, pushdown_rows_pruned=2, predicate_cache_inner_records=4, predicate_cache_records=2, scan_efficiency_ratio=19.07% (151/792)] statement ok reset datafusion.explain.analyze_categories; @@ -807,7 +807,7 @@ ON nulls_build.a = nulls_probe.a AND nulls_build.b = nulls_probe.b; Plan with Metrics 01)HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(a@0, a@0), (b@1, b@1)], metrics=[output_rows=1, output_batches=1, array_map_created_count=0, build_input_batches=1, build_input_rows=3, input_batches=1, input_rows=1, avg_fanout=100% (1/1), probe_hit_rate=100% (1/1)] 02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/nulls_build.parquet]]}, projection=[a, b], file_type=parquet, metrics=[output_rows=3, output_batches=1, files_ranges_pruned_statistics=1 total → 1 matched, row_groups_pruned_statistics=1 total → 1 matched, row_groups_pruned_bloom_filter=1 total → 1 matched, page_index_pages_pruned=0 total → 0 matched, page_index_rows_pruned=0 total → 0 matched, limit_pruned_row_groups=0 total → 0 matched, batches_split=0, file_open_errors=0, file_scan_errors=0, files_opened=1, files_processed=1, num_predicate_creation_errors=0, predicate_evaluation_errors=0, pushdown_rows_matched=0, pushdown_rows_pruned=0, predicate_cache_inner_records=0, predicate_cache_records=0, scan_efficiency_ratio=18.6% (144/774)] -03)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/nulls_probe.parquet]]}, projection=[a, b, c], file_type=parquet, predicate=DynamicFilter [ a@0 >= aa AND a@0 <= ab AND b@1 >= 1 AND b@1 <= 2 AND struct(a@0, b@1) IN (SET) ([{c0:aa,c1:1}, {c0:,c1:2}, {c0:ab,c1:}]) ], pruning_predicate=a_null_count@1 != row_count@2 AND a_max@0 >= aa AND a_null_count@1 != row_count@2 AND a_min@3 <= ab AND b_null_count@5 != row_count@2 AND b_max@4 >= 1 AND b_null_count@5 != row_count@2 AND b_min@6 <= 2, required_guarantees=[], metrics=[output_rows=1, output_batches=1, files_ranges_pruned_statistics=1 total → 1 matched, row_groups_pruned_statistics=1 total → 1 matched, row_groups_pruned_bloom_filter=1 total → 1 matched, page_index_pages_pruned=0 total → 0 matched, page_index_rows_pruned=0 total → 0 matched, limit_pruned_row_groups=0 total → 0 matched, batches_split=0, file_open_errors=0, file_scan_errors=0, files_opened=1, files_processed=1, num_predicate_creation_errors=0, predicate_evaluation_errors=0, pushdown_rows_matched=1, pushdown_rows_pruned=3, predicate_cache_inner_records=8, predicate_cache_records=2, scan_efficiency_ratio=20.18% (225/1.11 K)] +03)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/nulls_probe.parquet]]}, projection=[a, b, c], file_type=parquet, predicate=DynamicFilter [ a@0 >= aa AND a@0 <= ab AND b@1 >= 1 AND b@1 <= 2 AND struct(a@0, b@1) IN (SET) ([{c0:aa,c1:1}, {c0:,c1:2}, {c0:ab,c1:}]) ], dynamic_rg_pruning=eligible, pruning_predicate=a_null_count@1 != row_count@2 AND a_max@0 >= aa AND a_null_count@1 != row_count@2 AND a_min@3 <= ab AND b_null_count@5 != row_count@2 AND b_max@4 >= 1 AND b_null_count@5 != row_count@2 AND b_min@6 <= 2, required_guarantees=[], metrics=[output_rows=1, output_batches=1, files_ranges_pruned_statistics=1 total → 1 matched, row_groups_pruned_statistics=1 total → 1 matched, row_groups_pruned_bloom_filter=1 total → 1 matched, page_index_pages_pruned=0 total → 0 matched, page_index_rows_pruned=0 total → 0 matched, limit_pruned_row_groups=0 total → 0 matched, batches_split=0, file_open_errors=0, file_scan_errors=0, files_opened=1, files_processed=1, num_predicate_creation_errors=0, predicate_evaluation_errors=0, pushdown_rows_matched=1, pushdown_rows_pruned=3, predicate_cache_inner_records=8, predicate_cache_records=2, scan_efficiency_ratio=20.18% (225/1.11 K)] statement ok reset datafusion.explain.analyze_categories; @@ -873,7 +873,7 @@ ON lj_build.a = lj_probe.a AND lj_build.b = lj_probe.b; Plan with Metrics 01)HashJoinExec: mode=CollectLeft, join_type=Left, on=[(a@0, a@0), (b@1, b@1)], metrics=[output_rows=2, output_batches=1, array_map_created_count=0, build_input_batches=1, build_input_rows=2, input_batches=2, input_rows=2, avg_fanout=100% (2/2), probe_hit_rate=100% (2/2)] 02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/lj_build.parquet]]}, projection=[a, b, c], file_type=parquet, metrics=[output_rows=2, output_batches=1, files_ranges_pruned_statistics=1 total → 1 matched, row_groups_pruned_statistics=1 total → 1 matched, row_groups_pruned_bloom_filter=1 total → 1 matched, page_index_pages_pruned=0 total → 0 matched, page_index_rows_pruned=0 total → 0 matched, limit_pruned_row_groups=0 total → 0 matched, batches_split=0, file_open_errors=0, file_scan_errors=0, files_opened=1, files_processed=1, num_predicate_creation_errors=0, predicate_evaluation_errors=0, pushdown_rows_matched=0, pushdown_rows_pruned=0, predicate_cache_inner_records=0, predicate_cache_records=0, scan_efficiency_ratio=19.58% (196/1.00 K)] -03)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/lj_probe.parquet]]}, projection=[a, b, e], file_type=parquet, predicate=DynamicFilter [ a@0 >= aa AND a@0 <= ab AND b@1 >= ba AND b@1 <= bb AND struct(a@0, b@1) IN (SET) ([{c0:aa,c1:ba}, {c0:ab,c1:bb}]) ], pruning_predicate=a_null_count@1 != row_count@2 AND a_max@0 >= aa AND a_null_count@1 != row_count@2 AND a_min@3 <= ab AND b_null_count@5 != row_count@2 AND b_max@4 >= ba AND b_null_count@5 != row_count@2 AND b_min@6 <= bb, required_guarantees=[], metrics=[output_rows=2, output_batches=1, files_ranges_pruned_statistics=1 total → 1 matched, row_groups_pruned_statistics=1 total → 1 matched, row_groups_pruned_bloom_filter=1 total → 1 matched, page_index_pages_pruned=0 total → 0 matched, page_index_rows_pruned=0 total → 0 matched, limit_pruned_row_groups=0 total → 0 matched, batches_split=0, file_open_errors=0, file_scan_errors=0, files_opened=1, files_processed=1, num_predicate_creation_errors=0, predicate_evaluation_errors=0, pushdown_rows_matched=2, pushdown_rows_pruned=2, predicate_cache_inner_records=8, predicate_cache_records=4, scan_efficiency_ratio=22.05% (228/1.03 K)] +03)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/lj_probe.parquet]]}, projection=[a, b, e], file_type=parquet, predicate=DynamicFilter [ a@0 >= aa AND a@0 <= ab AND b@1 >= ba AND b@1 <= bb AND struct(a@0, b@1) IN (SET) ([{c0:aa,c1:ba}, {c0:ab,c1:bb}]) ], dynamic_rg_pruning=eligible, pruning_predicate=a_null_count@1 != row_count@2 AND a_max@0 >= aa AND a_null_count@1 != row_count@2 AND a_min@3 <= ab AND b_null_count@5 != row_count@2 AND b_max@4 >= ba AND b_null_count@5 != row_count@2 AND b_min@6 <= bb, required_guarantees=[], metrics=[output_rows=2, output_batches=1, files_ranges_pruned_statistics=1 total → 1 matched, row_groups_pruned_statistics=1 total → 1 matched, row_groups_pruned_bloom_filter=1 total → 1 matched, page_index_pages_pruned=0 total → 0 matched, page_index_rows_pruned=0 total → 0 matched, limit_pruned_row_groups=0 total → 0 matched, batches_split=0, file_open_errors=0, file_scan_errors=0, files_opened=1, files_processed=1, num_predicate_creation_errors=0, predicate_evaluation_errors=0, pushdown_rows_matched=2, pushdown_rows_pruned=2, predicate_cache_inner_records=8, predicate_cache_records=4, scan_efficiency_ratio=22.05% (228/1.03 K)] # LEFT SEMI JOIN: only matching build rows are returned; probe scan still # receives the dynamic filter. @@ -889,7 +889,7 @@ WHERE EXISTS ( Plan with Metrics 01)HashJoinExec: mode=CollectLeft, join_type=LeftSemi, on=[(a@0, a@0), (b@1, b@1)], metrics=[output_rows=2, output_batches=1, array_map_created_count=0, build_input_batches=1, build_input_rows=2, input_batches=2, input_rows=4, avg_fanout=100% (2/2), probe_hit_rate=100% (2/2)] 02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/lj_build.parquet]]}, projection=[a, b, c], file_type=parquet, metrics=[output_rows=2, output_batches=1, files_ranges_pruned_statistics=1 total → 1 matched, row_groups_pruned_statistics=1 total → 1 matched, row_groups_pruned_bloom_filter=1 total → 1 matched, page_index_pages_pruned=0 total → 0 matched, page_index_rows_pruned=0 total → 0 matched, limit_pruned_row_groups=0 total → 0 matched, batches_split=0, file_open_errors=0, file_scan_errors=0, files_opened=1, files_processed=1, num_predicate_creation_errors=0, predicate_evaluation_errors=0, pushdown_rows_matched=0, pushdown_rows_pruned=0, predicate_cache_inner_records=0, predicate_cache_records=0, scan_efficiency_ratio=19.58% (196/1.00 K)] -03)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/lj_probe.parquet]]}, projection=[a, b], file_type=parquet, predicate=DynamicFilter [ a@0 >= aa AND a@0 <= ab AND b@1 >= ba AND b@1 <= bb AND struct(a@0, b@1) IN (SET) ([{c0:aa,c1:ba}, {c0:ab,c1:bb}]) ], pruning_predicate=a_null_count@1 != row_count@2 AND a_max@0 >= aa AND a_null_count@1 != row_count@2 AND a_min@3 <= ab AND b_null_count@5 != row_count@2 AND b_max@4 >= ba AND b_null_count@5 != row_count@2 AND b_min@6 <= bb, required_guarantees=[], metrics=[output_rows=2, output_batches=1, files_ranges_pruned_statistics=1 total → 1 matched, row_groups_pruned_statistics=1 total → 1 matched, row_groups_pruned_bloom_filter=1 total → 1 matched, page_index_pages_pruned=0 total → 0 matched, page_index_rows_pruned=0 total → 0 matched, limit_pruned_row_groups=0 total → 0 matched, batches_split=0, file_open_errors=0, file_scan_errors=0, files_opened=1, files_processed=1, num_predicate_creation_errors=0, predicate_evaluation_errors=0, pushdown_rows_matched=2, pushdown_rows_pruned=2, predicate_cache_inner_records=8, predicate_cache_records=4, scan_efficiency_ratio=14.89% (154/1.03 K)] +03)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/lj_probe.parquet]]}, projection=[a, b], file_type=parquet, predicate=DynamicFilter [ a@0 >= aa AND a@0 <= ab AND b@1 >= ba AND b@1 <= bb AND struct(a@0, b@1) IN (SET) ([{c0:aa,c1:ba}, {c0:ab,c1:bb}]) ], dynamic_rg_pruning=eligible, pruning_predicate=a_null_count@1 != row_count@2 AND a_max@0 >= aa AND a_null_count@1 != row_count@2 AND a_min@3 <= ab AND b_null_count@5 != row_count@2 AND b_max@4 >= ba AND b_null_count@5 != row_count@2 AND b_min@6 <= bb, required_guarantees=[], metrics=[output_rows=2, output_batches=1, files_ranges_pruned_statistics=1 total → 1 matched, row_groups_pruned_statistics=1 total → 1 matched, row_groups_pruned_bloom_filter=1 total → 1 matched, page_index_pages_pruned=0 total → 0 matched, page_index_rows_pruned=0 total → 0 matched, limit_pruned_row_groups=0 total → 0 matched, batches_split=0, file_open_errors=0, file_scan_errors=0, files_opened=1, files_processed=1, num_predicate_creation_errors=0, predicate_evaluation_errors=0, pushdown_rows_matched=2, pushdown_rows_pruned=2, predicate_cache_inner_records=8, predicate_cache_records=4, scan_efficiency_ratio=14.89% (154/1.03 K)] statement ok reset datafusion.explain.analyze_categories; @@ -959,7 +959,7 @@ FROM hl_probe p INNER JOIN hl_build AS build Plan with Metrics 01)HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(a@0, a@0), (b@1, b@1)], projection=[a@3, b@4, c@2, e@5], metrics=[output_rows=2, output_batches=1, array_map_created_count=0, build_input_batches=1, build_input_rows=2, input_batches=1, input_rows=2, avg_fanout=100% (2/2), probe_hit_rate=100% (2/2)] 02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/hl_build.parquet]]}, projection=[a, b, c], file_type=parquet, metrics=[output_rows=2, output_batches=1, files_ranges_pruned_statistics=1 total → 1 matched, row_groups_pruned_statistics=1 total → 1 matched, row_groups_pruned_bloom_filter=1 total → 1 matched, page_index_pages_pruned=0 total → 0 matched, page_index_rows_pruned=0 total → 0 matched, limit_pruned_row_groups=0 total → 0 matched, batches_split=0, file_open_errors=0, file_scan_errors=0, files_opened=1, files_processed=1, num_predicate_creation_errors=0, predicate_evaluation_errors=0, pushdown_rows_matched=0, pushdown_rows_pruned=0, predicate_cache_inner_records=0, predicate_cache_records=0, scan_efficiency_ratio=19.58% (196/1.00 K)] -03)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/hl_probe.parquet]]}, projection=[a, b, e], file_type=parquet, predicate=DynamicFilter [ a@0 >= aa AND a@0 <= ab AND b@1 >= ba AND b@1 <= bb AND hash_lookup ], pruning_predicate=a_null_count@1 != row_count@2 AND a_max@0 >= aa AND a_null_count@1 != row_count@2 AND a_min@3 <= ab AND b_null_count@5 != row_count@2 AND b_max@4 >= ba AND b_null_count@5 != row_count@2 AND b_min@6 <= bb, required_guarantees=[], metrics=[output_rows=2, output_batches=1, files_ranges_pruned_statistics=1 total → 1 matched, row_groups_pruned_statistics=1 total → 1 matched, row_groups_pruned_bloom_filter=1 total → 1 matched, page_index_pages_pruned=0 total → 0 matched, page_index_rows_pruned=0 total → 0 matched, limit_pruned_row_groups=0 total → 0 matched, batches_split=0, file_open_errors=0, file_scan_errors=0, files_opened=1, files_processed=1, num_predicate_creation_errors=0, predicate_evaluation_errors=0, pushdown_rows_matched=2, pushdown_rows_pruned=2, predicate_cache_inner_records=8, predicate_cache_records=4, scan_efficiency_ratio=22.05% (228/1.03 K)] +03)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/hl_probe.parquet]]}, projection=[a, b, e], file_type=parquet, predicate=DynamicFilter [ a@0 >= aa AND a@0 <= ab AND b@1 >= ba AND b@1 <= bb AND hash_lookup ], dynamic_rg_pruning=eligible, pruning_predicate=a_null_count@1 != row_count@2 AND a_max@0 >= aa AND a_null_count@1 != row_count@2 AND a_min@3 <= ab AND b_null_count@5 != row_count@2 AND b_max@4 >= ba AND b_null_count@5 != row_count@2 AND b_min@6 <= bb, required_guarantees=[], metrics=[output_rows=2, output_batches=1, files_ranges_pruned_statistics=1 total → 1 matched, row_groups_pruned_statistics=1 total → 1 matched, row_groups_pruned_bloom_filter=1 total → 1 matched, page_index_pages_pruned=0 total → 0 matched, page_index_rows_pruned=0 total → 0 matched, limit_pruned_row_groups=0 total → 0 matched, batches_split=0, file_open_errors=0, file_scan_errors=0, files_opened=1, files_processed=1, num_predicate_creation_errors=0, predicate_evaluation_errors=0, pushdown_rows_matched=2, pushdown_rows_pruned=2, predicate_cache_inner_records=8, predicate_cache_records=4, scan_efficiency_ratio=22.05% (228/1.03 K)] statement ok drop table hl_build; @@ -1008,7 +1008,7 @@ FROM int_build b INNER JOIN int_probe p Plan with Metrics 01)HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(id1@0, id1@0), (id2@1, id2@1)], projection=[id1@0, id2@1, value@2, data@5], metrics=[output_rows=2, output_batches=1, array_map_created_count=0, build_input_batches=1, build_input_rows=2, input_batches=1, input_rows=2, avg_fanout=100% (2/2), probe_hit_rate=100% (2/2)] 02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/int_build.parquet]]}, projection=[id1, id2, value], file_type=parquet, metrics=[output_rows=2, output_batches=1, files_ranges_pruned_statistics=1 total → 1 matched, row_groups_pruned_statistics=1 total → 1 matched, row_groups_pruned_bloom_filter=1 total → 1 matched, page_index_pages_pruned=0 total → 0 matched, page_index_rows_pruned=0 total → 0 matched, limit_pruned_row_groups=0 total → 0 matched, batches_split=0, file_open_errors=0, file_scan_errors=0, files_opened=1, files_processed=1, num_predicate_creation_errors=0, predicate_evaluation_errors=0, pushdown_rows_matched=0, pushdown_rows_pruned=0, predicate_cache_inner_records=0, predicate_cache_records=0, scan_efficiency_ratio=18.23% (204/1.12 K)] -03)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/int_probe.parquet]]}, projection=[id1, id2, data], file_type=parquet, predicate=DynamicFilter [ id1@0 >= 1 AND id1@0 <= 2 AND id2@1 >= 10 AND id2@1 <= 20 AND hash_lookup ], pruning_predicate=id1_null_count@1 != row_count@2 AND id1_max@0 >= 1 AND id1_null_count@1 != row_count@2 AND id1_min@3 <= 2 AND id2_null_count@5 != row_count@2 AND id2_max@4 >= 10 AND id2_null_count@5 != row_count@2 AND id2_min@6 <= 20, required_guarantees=[], metrics=[output_rows=2, output_batches=1, files_ranges_pruned_statistics=1 total → 1 matched, row_groups_pruned_statistics=1 total → 1 matched, row_groups_pruned_bloom_filter=1 total → 1 matched, page_index_pages_pruned=0 total → 0 matched, page_index_rows_pruned=0 total → 0 matched, limit_pruned_row_groups=0 total → 0 matched, batches_split=0, file_open_errors=0, file_scan_errors=0, files_opened=1, files_processed=1, num_predicate_creation_errors=0, predicate_evaluation_errors=0, pushdown_rows_matched=2, pushdown_rows_pruned=2, predicate_cache_inner_records=8, predicate_cache_records=4, scan_efficiency_ratio=20.67% (221/1.07 K)] +03)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/int_probe.parquet]]}, projection=[id1, id2, data], file_type=parquet, predicate=DynamicFilter [ id1@0 >= 1 AND id1@0 <= 2 AND id2@1 >= 10 AND id2@1 <= 20 AND hash_lookup ], dynamic_rg_pruning=eligible, pruning_predicate=id1_null_count@1 != row_count@2 AND id1_max@0 >= 1 AND id1_null_count@1 != row_count@2 AND id1_min@3 <= 2 AND id2_null_count@5 != row_count@2 AND id2_max@4 >= 10 AND id2_null_count@5 != row_count@2 AND id2_min@6 <= 20, required_guarantees=[], metrics=[output_rows=2, output_batches=1, files_ranges_pruned_statistics=1 total → 1 matched, row_groups_pruned_statistics=1 total → 1 matched, row_groups_pruned_bloom_filter=1 total → 1 matched, page_index_pages_pruned=0 total → 0 matched, page_index_rows_pruned=0 total → 0 matched, limit_pruned_row_groups=0 total → 0 matched, batches_split=0, file_open_errors=0, file_scan_errors=0, files_opened=1, files_processed=1, num_predicate_creation_errors=0, predicate_evaluation_errors=0, pushdown_rows_matched=2, pushdown_rows_pruned=2, predicate_cache_inner_records=8, predicate_cache_records=4, scan_efficiency_ratio=20.67% (221/1.07 K)] statement ok reset datafusion.explain.analyze_categories; diff --git a/datafusion/sqllogictest/test_files/push_down_filter_regression.slt b/datafusion/sqllogictest/test_files/push_down_filter_regression.slt index b86bd2c51d5b8..f28a314764138 100644 --- a/datafusion/sqllogictest/test_files/push_down_filter_regression.slt +++ b/datafusion/sqllogictest/test_files/push_down_filter_regression.slt @@ -146,7 +146,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: 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 ], dynamic_rg_pruning=eligible, 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; @@ -161,7 +161,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: 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 ], dynamic_rg_pruning=eligible # Expect dynamic filter available inside data source query TT @@ -171,7 +171,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: 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 ], dynamic_rg_pruning=eligible, 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 @@ -236,7 +236,7 @@ Plan with Metrics 01)AggregateExec: mode=Final, gby=[], aggr=[max(agg_dyn_e2e.column1)], metrics=[] 02)--CoalescePartitionsExec, metrics=[] 03)----AggregateExec: mode=Partial, gby=[], aggr=[max(agg_dyn_e2e.column1)], metrics=[] -04)------DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_regression/agg_dyn/file_0.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_regression/agg_dyn/file_1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_regression/agg_dyn/file_2.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_regression/agg_dyn/file_3.parquet]]}, projection=[column1], file_type=parquet, predicate=column1@0 > 1 AND DynamicFilter [ column1@0 > 4 ], pruning_predicate=column1_null_count@1 != row_count@2 AND column1_max@0 > 1 AND column1_null_count@1 != row_count@2 AND column1_max@0 > 4, required_guarantees=[], metrics=[] +04)------DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_regression/agg_dyn/file_0.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_regression/agg_dyn/file_1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_regression/agg_dyn/file_2.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_regression/agg_dyn/file_3.parquet]]}, projection=[column1], file_type=parquet, predicate=column1@0 > 1 AND DynamicFilter [ column1@0 > 4 ], dynamic_rg_pruning=eligible, pruning_predicate=column1_null_count@1 != row_count@2 AND column1_max@0 > 1 AND column1_null_count@1 != row_count@2 AND column1_max@0 > 4, required_guarantees=[], metrics=[] statement ok reset datafusion.explain.analyze_categories; @@ -323,7 +323,7 @@ Plan with Metrics 01)AggregateExec: mode=Final, gby=[], aggr=[min(agg_dyn_single.a)], metrics=[] 02)--CoalescePartitionsExec, metrics=[] 03)----AggregateExec: mode=Partial, gby=[], aggr=[min(agg_dyn_single.a)], metrics=[] -04)------DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_regression/agg_dyn_single/file_0.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_regression/agg_dyn_single/file_1.parquet]]}, projection=[a], file_type=parquet, predicate=DynamicFilter [ a@0 < 1 ], pruning_predicate=a_null_count@1 != row_count@2 AND a_min@0 < 1, required_guarantees=[], metrics=[] +04)------DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_regression/agg_dyn_single/file_0.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_regression/agg_dyn_single/file_1.parquet]]}, projection=[a], file_type=parquet, predicate=DynamicFilter [ a@0 < 1 ], dynamic_rg_pruning=eligible, pruning_predicate=a_null_count@1 != row_count@2 AND a_min@0 < 1, required_guarantees=[], metrics=[] # MAX(a) -> DynamicFilter [ a > 8 ] query TT @@ -333,7 +333,7 @@ Plan with Metrics 01)AggregateExec: mode=Final, gby=[], aggr=[max(agg_dyn_single.a)], metrics=[] 02)--CoalescePartitionsExec, metrics=[] 03)----AggregateExec: mode=Partial, gby=[], aggr=[max(agg_dyn_single.a)], metrics=[] -04)------DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_regression/agg_dyn_single/file_0.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_regression/agg_dyn_single/file_1.parquet]]}, projection=[a], file_type=parquet, predicate=DynamicFilter [ a@0 > 8 ], pruning_predicate=a_null_count@1 != row_count@2 AND a_max@0 > 8, required_guarantees=[], metrics=[] +04)------DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_regression/agg_dyn_single/file_0.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_regression/agg_dyn_single/file_1.parquet]]}, projection=[a], file_type=parquet, predicate=DynamicFilter [ a@0 > 8 ], dynamic_rg_pruning=eligible, pruning_predicate=a_null_count@1 != row_count@2 AND a_max@0 > 8, required_guarantees=[], metrics=[] # MIN(a), MAX(a) -> DynamicFilter [ a < 1 OR a > 8 ] query TT @@ -343,7 +343,7 @@ Plan with Metrics 01)AggregateExec: mode=Final, gby=[], aggr=[min(agg_dyn_single.a), max(agg_dyn_single.a)], metrics=[] 02)--CoalescePartitionsExec, metrics=[] 03)----AggregateExec: mode=Partial, gby=[], aggr=[min(agg_dyn_single.a), max(agg_dyn_single.a)], metrics=[] -04)------DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_regression/agg_dyn_single/file_0.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_regression/agg_dyn_single/file_1.parquet]]}, projection=[a], file_type=parquet, predicate=DynamicFilter [ a@0 < 1 OR a@0 > 8 ], pruning_predicate=a_null_count@1 != row_count@2 AND a_min@0 < 1 OR a_null_count@1 != row_count@2 AND a_max@3 > 8, required_guarantees=[], metrics=[] +04)------DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_regression/agg_dyn_single/file_0.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_regression/agg_dyn_single/file_1.parquet]]}, projection=[a], file_type=parquet, predicate=DynamicFilter [ a@0 < 1 OR a@0 > 8 ], dynamic_rg_pruning=eligible, pruning_predicate=a_null_count@1 != row_count@2 AND a_min@0 < 1 OR a_null_count@1 != row_count@2 AND a_max@3 > 8, required_guarantees=[], metrics=[] # MIN(a+1) -> no dynamic filter (expression input is not a plain column) query TT @@ -387,7 +387,7 @@ Plan with Metrics 01)AggregateExec: mode=Final, gby=[], aggr=[min(agg_dyn_two_col.a), max(agg_dyn_two_col.b)], metrics=[] 02)--CoalescePartitionsExec, metrics=[] 03)----AggregateExec: mode=Partial, gby=[], aggr=[min(agg_dyn_two_col.a), max(agg_dyn_two_col.b)], metrics=[] -04)------DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_regression/agg_dyn_two_col/file_0.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_regression/agg_dyn_two_col/file_1.parquet]]}, projection=[a, b], file_type=parquet, predicate=DynamicFilter [ a@0 < 1 OR b@1 > 9 ], pruning_predicate=a_null_count@1 != row_count@2 AND a_min@0 < 1 OR b_null_count@4 != row_count@2 AND b_max@3 > 9, required_guarantees=[], metrics=[] +04)------DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_regression/agg_dyn_two_col/file_0.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_regression/agg_dyn_two_col/file_1.parquet]]}, projection=[a, b], file_type=parquet, predicate=DynamicFilter [ a@0 < 1 OR b@1 > 9 ], dynamic_rg_pruning=eligible, pruning_predicate=a_null_count@1 != row_count@2 AND a_min@0 < 1 OR b_null_count@4 != row_count@2 AND b_max@3 > 9, required_guarantees=[], metrics=[] statement ok drop table agg_dyn_two_col; @@ -423,7 +423,7 @@ Plan with Metrics 01)AggregateExec: mode=Final, gby=[], aggr=[min(agg_dyn_mixed.a), max(agg_dyn_mixed.a), max(agg_dyn_mixed.b), min(agg_dyn_mixed.c + Int64(1))], metrics=[] 02)--CoalescePartitionsExec, metrics=[] 03)----AggregateExec: mode=Partial, gby=[], aggr=[min(agg_dyn_mixed.a), max(agg_dyn_mixed.a), max(agg_dyn_mixed.b), min(agg_dyn_mixed.c + Int64(1))], metrics=[] -04)------DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_regression/agg_dyn_mixed/file_0.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_regression/agg_dyn_mixed/file_1.parquet]]}, projection=[a, b, c], file_type=parquet, predicate=DynamicFilter [ a@0 < 1 OR a@0 > 8 OR b@1 > 12 ], pruning_predicate=a_null_count@1 != row_count@2 AND a_min@0 < 1 OR a_null_count@1 != row_count@2 AND a_max@3 > 8 OR b_null_count@5 != row_count@2 AND b_max@4 > 12, required_guarantees=[], metrics=[] +04)------DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_regression/agg_dyn_mixed/file_0.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_regression/agg_dyn_mixed/file_1.parquet]]}, projection=[a, b, c], file_type=parquet, predicate=DynamicFilter [ a@0 < 1 OR a@0 > 8 OR b@1 > 12 ], dynamic_rg_pruning=eligible, pruning_predicate=a_null_count@1 != row_count@2 AND a_min@0 < 1 OR a_null_count@1 != row_count@2 AND a_max@3 > 8 OR b_null_count@5 != row_count@2 AND b_max@4 > 12, required_guarantees=[], metrics=[] statement ok drop table agg_dyn_mixed; @@ -455,7 +455,7 @@ Plan with Metrics 01)AggregateExec: mode=Final, gby=[], aggr=[min(agg_dyn_nulls.a)], metrics=[] 02)--CoalescePartitionsExec, metrics=[] 03)----AggregateExec: mode=Partial, gby=[], aggr=[min(agg_dyn_nulls.a)], metrics=[] -04)------DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_regression/agg_dyn_nulls/file_0.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_regression/agg_dyn_nulls/file_1.parquet]]}, projection=[a], file_type=parquet, predicate=DynamicFilter [ true ], metrics=[] +04)------DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_regression/agg_dyn_nulls/file_0.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_regression/agg_dyn_nulls/file_1.parquet]]}, projection=[a], file_type=parquet, predicate=DynamicFilter [ true ], dynamic_rg_pruning=eligible, metrics=[] statement ok reset datafusion.explain.analyze_categories; diff --git a/datafusion/sqllogictest/test_files/repartition_subset_satisfaction.slt b/datafusion/sqllogictest/test_files/repartition_subset_satisfaction.slt index dbf31dec5e118..043a62314cb5c 100644 --- a/datafusion/sqllogictest/test_files/repartition_subset_satisfaction.slt +++ b/datafusion/sqllogictest/test_files/repartition_subset_satisfaction.slt @@ -380,7 +380,7 @@ physical_plan 12)----------------------CoalescePartitionsExec 13)------------------------FilterExec: service@1 = log, projection=[env@0, d_dkey@2] 14)--------------------------DataSourceExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/dimension/d_dkey=A/data.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/dimension/d_dkey=D/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/dimension/d_dkey=B/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/dimension/d_dkey=C/data.parquet]]}, projection=[env, service, d_dkey], file_type=parquet, predicate=service@1 = 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)] -15)----------------------DataSourceExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/fact/f_dkey=A/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/fact/f_dkey=B/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/fact/f_dkey=C/data.parquet]]}, projection=[timestamp, value, f_dkey], output_ordering=[f_dkey@2 ASC NULLS LAST, timestamp@0 ASC NULLS LAST], file_type=parquet, predicate=DynamicFilter [ empty ] +15)----------------------DataSourceExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/fact/f_dkey=A/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/fact/f_dkey=B/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/fact/f_dkey=C/data.parquet]]}, projection=[timestamp, value, f_dkey], output_ordering=[f_dkey@2 ASC NULLS LAST, timestamp@0 ASC NULLS LAST], file_type=parquet, predicate=DynamicFilter [ empty ], dynamic_rg_pruning=eligible # Verify results without subset satisfaction query TPR rowsort @@ -475,7 +475,7 @@ physical_plan 10)------------------CoalescePartitionsExec 11)--------------------FilterExec: service@1 = log, projection=[env@0, d_dkey@2] 12)----------------------DataSourceExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/dimension/d_dkey=A/data.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/dimension/d_dkey=D/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/dimension/d_dkey=B/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/dimension/d_dkey=C/data.parquet]]}, projection=[env, service, d_dkey], file_type=parquet, predicate=service@1 = 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/repartition_subset_satisfaction/fact/f_dkey=A/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/fact/f_dkey=B/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/fact/f_dkey=C/data.parquet]]}, projection=[timestamp, value, f_dkey], output_ordering=[f_dkey@2 ASC NULLS LAST, timestamp@0 ASC NULLS LAST], file_type=parquet, predicate=DynamicFilter [ empty ] +13)------------------DataSourceExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/fact/f_dkey=A/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/fact/f_dkey=B/data.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_subset_satisfaction/fact/f_dkey=C/data.parquet]]}, projection=[timestamp, value, f_dkey], output_ordering=[f_dkey@2 ASC NULLS LAST, timestamp@0 ASC NULLS LAST], file_type=parquet, predicate=DynamicFilter [ empty ], dynamic_rg_pruning=eligible # Verify results match with subset satisfaction query TPR rowsort @@ -517,7 +517,7 @@ prod 2023-01-01T09:12:30 197.7 # Config reset -# The SLT runner sets `target_partitions` to 4 instead of using the default, so +# The SLT runner sets `target_partitions` to 4 instead of using the default, so # reset it explicitly. statement ok set datafusion.execution.target_partitions = 4; diff --git a/datafusion/sqllogictest/test_files/sort_pushdown.slt b/datafusion/sqllogictest/test_files/sort_pushdown.slt index 36fb38f5b4026..f2442762f3fd2 100644 --- a/datafusion/sqllogictest/test_files/sort_pushdown.slt +++ b/datafusion/sqllogictest/test_files/sort_pushdown.slt @@ -43,7 +43,7 @@ logical_plan 02)--TableScan: sorted_parquet projection=[id, value, name] physical_plan 01)SortExec: TopK(fetch=3), expr=[id@0 DESC], preserve_partitioning=[false] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/sorted_data.parquet]]}, projection=[id, value, name], file_type=parquet, predicate=DynamicFilter [ empty ], sort_order_for_reorder=[id@0 DESC], reverse_row_groups=true +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/sorted_data.parquet]]}, projection=[id, value, name], file_type=parquet, predicate=DynamicFilter [ empty ], sort_order_for_reorder=[id@0 DESC], reverse_row_groups=true, dynamic_rg_pruning=eligible # Test 1.2: Verify results are correct query IIT @@ -74,7 +74,7 @@ logical_plan 02)--TableScan: sorted_parquet projection=[id, value, name] physical_plan 01)SortExec: TopK(fetch=3), expr=[id@0 DESC], preserve_partitioning=[false] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/sorted_data.parquet]]}, projection=[id, value, name], output_ordering=[id@0 ASC NULLS LAST], file_type=parquet, predicate=DynamicFilter [ empty ] +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/sorted_data.parquet]]}, projection=[id, value, name], output_ordering=[id@0 ASC NULLS LAST], file_type=parquet, predicate=DynamicFilter [ empty ], dynamic_rg_pruning=eligible # Re-enable statement ok @@ -91,7 +91,7 @@ logical_plan physical_plan 01)GlobalLimitExec: skip=2, fetch=3 02)--SortExec: TopK(fetch=5), expr=[id@0 DESC], preserve_partitioning=[false] -03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/sorted_data.parquet]]}, projection=[id, value, name], file_type=parquet, predicate=DynamicFilter [ empty ], sort_order_for_reorder=[id@0 DESC], reverse_row_groups=true +03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/sorted_data.parquet]]}, projection=[id, value, name], file_type=parquet, predicate=DynamicFilter [ empty ], sort_order_for_reorder=[id@0 DESC], reverse_row_groups=true, dynamic_rg_pruning=eligible query IIT SELECT * FROM sorted_parquet ORDER BY id DESC LIMIT 3 OFFSET 2; @@ -155,7 +155,7 @@ logical_plan 03)----TableScan: multi_rg_sorted projection=[id, category, value], partial_filters=[multi_rg_sorted.category = Utf8View("alpha") OR multi_rg_sorted.category = Utf8View("gamma")] physical_plan 01)SortExec: TopK(fetch=5), expr=[id@0 DESC], preserve_partitioning=[false] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/multi_rg_sorted.parquet]]}, projection=[id, category, value], file_type=parquet, predicate=(category@1 = alpha OR category@1 = gamma) AND DynamicFilter [ empty ], sort_order_for_reorder=[id@0 DESC], reverse_row_groups=true, pruning_predicate=category_null_count@2 != row_count@3 AND category_min@0 <= alpha AND alpha <= category_max@1 OR category_null_count@2 != row_count@3 AND category_min@0 <= gamma AND gamma <= category_max@1, required_guarantees=[category in (alpha, gamma)] +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/multi_rg_sorted.parquet]]}, projection=[id, category, value], file_type=parquet, predicate=(category@1 = alpha OR category@1 = gamma) AND DynamicFilter [ empty ], sort_order_for_reorder=[id@0 DESC], reverse_row_groups=true, dynamic_rg_pruning=eligible, pruning_predicate=category_null_count@2 != row_count@3 AND category_min@0 <= alpha AND alpha <= category_max@1 OR category_null_count@2 != row_count@3 AND category_min@0 <= gamma AND gamma <= category_max@1, required_guarantees=[category in (alpha, gamma)] # Verify the results are correct despite reverse scanning with row selection # Expected: gamma values (6, 5) then alpha values (2, 1), in DESC order by id @@ -272,7 +272,7 @@ logical_plan physical_plan 01)SortPreservingMergeExec: [id@0 DESC], fetch=3 02)--SortExec: TopK(fetch=3), expr=[id@0 DESC], preserve_partitioning=[true] -03)----DataSourceExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/sorted_multi/part1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/sorted_multi/part2.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/sorted_multi/part3.parquet]]}, projection=[id, value, name], file_type=parquet, predicate=DynamicFilter [ empty ], sort_order_for_reorder=[id@0 DESC], reverse_row_groups=true +03)----DataSourceExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/sorted_multi/part1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/sorted_multi/part2.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/sorted_multi/part3.parquet]]}, projection=[id, value, name], file_type=parquet, predicate=DynamicFilter [ empty ], sort_order_for_reorder=[id@0 DESC], reverse_row_groups=true, dynamic_rg_pruning=eligible # Verify correctness with repartitioning and multiple files query IIT @@ -381,7 +381,7 @@ logical_plan 03)----TableScan: timeseries_parquet projection=[timeframe, period_end, value], partial_filters=[timeseries_parquet.timeframe = Utf8View("quarterly")] physical_plan 01)SortExec: TopK(fetch=2), expr=[period_end@1 DESC], preserve_partitioning=[false] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/timeseries_sorted.parquet]]}, projection=[timeframe, period_end, value], file_type=parquet, predicate=timeframe@0 = quarterly AND DynamicFilter [ empty ], sort_order_for_reorder=[period_end@1 DESC], reverse_row_groups=true, pruning_predicate=timeframe_null_count@2 != row_count@3 AND timeframe_min@0 <= quarterly AND quarterly <= timeframe_max@1, required_guarantees=[timeframe in (quarterly)] +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/timeseries_sorted.parquet]]}, projection=[timeframe, period_end, value], file_type=parquet, predicate=timeframe@0 = quarterly AND DynamicFilter [ empty ], sort_order_for_reorder=[period_end@1 DESC], reverse_row_groups=true, dynamic_rg_pruning=eligible, pruning_predicate=timeframe_null_count@2 != row_count@3 AND timeframe_min@0 <= quarterly AND quarterly <= timeframe_max@1, required_guarantees=[timeframe in (quarterly)] # Test 2.2: Verify the results are correct query TIR @@ -440,7 +440,7 @@ logical_plan 02)--TableScan: timeseries_parquet projection=[timeframe, period_end, value] physical_plan 01)SortExec: TopK(fetch=3), expr=[timeframe@0 ASC NULLS LAST, period_end@1 DESC], preserve_partitioning=[false], sort_prefix=[timeframe@0 ASC NULLS LAST] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/timeseries_sorted.parquet]]}, projection=[timeframe, period_end, value], output_ordering=[timeframe@0 ASC NULLS LAST, period_end@1 ASC NULLS LAST], file_type=parquet, predicate=DynamicFilter [ empty ] +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/timeseries_sorted.parquet]]}, projection=[timeframe, period_end, value], output_ordering=[timeframe@0 ASC NULLS LAST, period_end@1 ASC NULLS LAST], file_type=parquet, predicate=DynamicFilter [ empty ], dynamic_rg_pruning=eligible # Test 2.7: Disable sort pushdown and verify filter still works statement ok @@ -458,7 +458,7 @@ logical_plan 03)----TableScan: timeseries_parquet projection=[timeframe, period_end, value], partial_filters=[timeseries_parquet.timeframe = Utf8View("quarterly")] physical_plan 01)SortExec: TopK(fetch=2), expr=[period_end@1 DESC], preserve_partitioning=[false] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/timeseries_sorted.parquet]]}, projection=[timeframe, period_end, value], output_ordering=[timeframe@0 ASC NULLS LAST, period_end@1 ASC NULLS LAST], file_type=parquet, predicate=timeframe@0 = quarterly AND DynamicFilter [ empty ], pruning_predicate=timeframe_null_count@2 != row_count@3 AND timeframe_min@0 <= quarterly AND quarterly <= timeframe_max@1, required_guarantees=[timeframe in (quarterly)] +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/timeseries_sorted.parquet]]}, projection=[timeframe, period_end, value], output_ordering=[timeframe@0 ASC NULLS LAST, period_end@1 ASC NULLS LAST], file_type=parquet, predicate=timeframe@0 = quarterly AND DynamicFilter [ empty ], dynamic_rg_pruning=eligible, pruning_predicate=timeframe_null_count@2 != row_count@3 AND timeframe_min@0 <= quarterly AND quarterly <= timeframe_max@1, required_guarantees=[timeframe in (quarterly)] # Results should still be correct query TIR @@ -491,7 +491,7 @@ logical_plan 03)----TableScan: timeseries_parquet projection=[timeframe, period_end, value], partial_filters=[timeseries_parquet.timeframe = Utf8View("daily") OR timeseries_parquet.timeframe = Utf8View("weekly")] physical_plan 01)SortExec: TopK(fetch=3), expr=[period_end@1 DESC], preserve_partitioning=[false] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/timeseries_sorted.parquet]]}, projection=[timeframe, period_end, value], file_type=parquet, predicate=(timeframe@0 = daily OR timeframe@0 = weekly) AND DynamicFilter [ empty ], sort_order_for_reorder=[period_end@1 DESC], reverse_row_groups=true, pruning_predicate=timeframe_null_count@2 != row_count@3 AND timeframe_min@0 <= daily AND daily <= timeframe_max@1 OR timeframe_null_count@2 != row_count@3 AND timeframe_min@0 <= weekly AND weekly <= timeframe_max@1, required_guarantees=[timeframe in (daily, weekly)] +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/timeseries_sorted.parquet]]}, projection=[timeframe, period_end, value], file_type=parquet, predicate=(timeframe@0 = daily OR timeframe@0 = weekly) AND DynamicFilter [ empty ], sort_order_for_reorder=[period_end@1 DESC], reverse_row_groups=true, dynamic_rg_pruning=eligible, pruning_predicate=timeframe_null_count@2 != row_count@3 AND timeframe_min@0 <= daily AND daily <= timeframe_max@1 OR timeframe_null_count@2 != row_count@3 AND timeframe_min@0 <= weekly AND weekly <= timeframe_max@1, required_guarantees=[timeframe in (daily, weekly)] # Test 2.9: Complex case - literal constant in sort expression itself # The literal 'constant' is ignored in sort analysis @@ -511,7 +511,7 @@ logical_plan 03)----TableScan: timeseries_parquet projection=[timeframe, period_end, value], partial_filters=[timeseries_parquet.timeframe = Utf8View("monthly")] physical_plan 01)SortExec: TopK(fetch=2), expr=[period_end@1 DESC], preserve_partitioning=[false] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/timeseries_sorted.parquet]]}, projection=[timeframe, period_end, value], file_type=parquet, predicate=timeframe@0 = monthly AND DynamicFilter [ empty ], sort_order_for_reorder=[period_end@1 DESC], reverse_row_groups=true, pruning_predicate=timeframe_null_count@2 != row_count@3 AND timeframe_min@0 <= monthly AND monthly <= timeframe_max@1, required_guarantees=[timeframe in (monthly)] +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/timeseries_sorted.parquet]]}, projection=[timeframe, period_end, value], file_type=parquet, predicate=timeframe@0 = monthly AND DynamicFilter [ empty ], sort_order_for_reorder=[period_end@1 DESC], reverse_row_groups=true, dynamic_rg_pruning=eligible, pruning_predicate=timeframe_null_count@2 != row_count@3 AND timeframe_min@0 <= monthly AND monthly <= timeframe_max@1, required_guarantees=[timeframe in (monthly)] # Verify results query TIR @@ -600,7 +600,7 @@ logical_plan 02)--TableScan: timestamp_parquet projection=[id, ts, volume, price] physical_plan 01)SortExec: TopK(fetch=3), expr=[ts@1 DESC], preserve_partitioning=[false] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/timestamp_sorted.parquet]]}, projection=[id, ts, volume, price], file_type=parquet, predicate=DynamicFilter [ empty ], sort_order_for_reorder=[ts@1 DESC], reverse_row_groups=true +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/timestamp_sorted.parquet]]}, projection=[id, ts, volume, price], file_type=parquet, predicate=DynamicFilter [ empty ], sort_order_for_reorder=[ts@1 DESC], reverse_row_groups=true, dynamic_rg_pruning=eligible # Verify results query IPIR @@ -626,7 +626,7 @@ logical_plan 02)--TableScan: timestamp_parquet projection=[id, ts, volume, price] physical_plan 01)SortExec: TopK(fetch=3), expr=[date_trunc(day, ts@1) DESC], preserve_partitioning=[false] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/timestamp_sorted.parquet]]}, projection=[id, ts, volume, price], file_type=parquet, predicate=DynamicFilter [ empty ], sort_order_for_reorder=[date_trunc(day, ts@1) DESC], reverse_row_groups=true +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/timestamp_sorted.parquet]]}, projection=[id, ts, volume, price], file_type=parquet, predicate=DynamicFilter [ empty ], sort_order_for_reorder=[date_trunc(day, ts@1) DESC], reverse_row_groups=true, dynamic_rg_pruning=eligible # Verify results (descending day) query IPIR @@ -686,7 +686,7 @@ logical_plan 02)--TableScan: multi_month_parquet projection=[id, ts, volume, price] physical_plan 01)SortExec: TopK(fetch=2), expr=[ts@1 DESC], preserve_partitioning=[false] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/multi_month_sorted.parquet]]}, projection=[id, ts, volume, price], file_type=parquet, predicate=DynamicFilter [ empty ], sort_order_for_reorder=[ts@1 DESC], reverse_row_groups=true +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/multi_month_sorted.parquet]]}, projection=[id, ts, volume, price], file_type=parquet, predicate=DynamicFilter [ empty ], sort_order_for_reorder=[ts@1 DESC], reverse_row_groups=true, dynamic_rg_pruning=eligible query IPIR SELECT * FROM multi_month_parquet @@ -712,7 +712,7 @@ logical_plan 02)--TableScan: multi_month_parquet projection=[id, ts, volume, price] physical_plan 01)SortExec: TopK(fetch=2), expr=[date_trunc(month, ts@1) DESC, ts@1 DESC], preserve_partitioning=[false] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/multi_month_sorted.parquet]]}, projection=[id, ts, volume, price], file_type=parquet, predicate=DynamicFilter [ empty ], sort_order_for_reorder=[date_trunc(month, ts@1) DESC, ts@1 DESC], reverse_row_groups=true +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/multi_month_sorted.parquet]]}, projection=[id, ts, volume, price], file_type=parquet, predicate=DynamicFilter [ empty ], sort_order_for_reorder=[date_trunc(month, ts@1) DESC, ts@1 DESC], reverse_row_groups=true, dynamic_rg_pruning=eligible query IPIR SELECT * FROM multi_month_parquet @@ -754,7 +754,7 @@ logical_plan 02)--TableScan: int_parquet projection=[id, small_val, big_val] physical_plan 01)SortExec: TopK(fetch=2), expr=[CAST(small_val@1 AS Int64) DESC], preserve_partitioning=[false] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/int_sorted.parquet]]}, projection=[id, small_val, big_val], file_type=parquet, predicate=DynamicFilter [ empty ], sort_order_for_reorder=[CAST(small_val@1 AS Int64) DESC], reverse_row_groups=true +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/int_sorted.parquet]]}, projection=[id, small_val, big_val], file_type=parquet, predicate=DynamicFilter [ empty ], sort_order_for_reorder=[CAST(small_val@1 AS Int64) DESC], reverse_row_groups=true, dynamic_rg_pruning=eligible query III SELECT * FROM int_parquet @@ -796,7 +796,7 @@ logical_plan 02)--TableScan: float_parquet projection=[id, value] physical_plan 01)SortExec: TopK(fetch=3), expr=[ceil(value@1) DESC], preserve_partitioning=[false] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/float_sorted.parquet]]}, projection=[id, value], file_type=parquet, predicate=DynamicFilter [ empty ], sort_order_for_reorder=[ceil(value@1) DESC], reverse_row_groups=true +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/float_sorted.parquet]]}, projection=[id, value], file_type=parquet, predicate=DynamicFilter [ empty ], sort_order_for_reorder=[ceil(value@1) DESC], reverse_row_groups=true, dynamic_rg_pruning=eligible query IR SELECT * FROM float_parquet @@ -839,7 +839,7 @@ logical_plan 02)--TableScan: signed_parquet projection=[id, value] physical_plan 01)SortExec: TopK(fetch=3), expr=[abs(value@1) DESC], preserve_partitioning=[false] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/signed_sorted.parquet]]}, projection=[id, value], output_ordering=[value@1 ASC NULLS LAST], file_type=parquet, predicate=DynamicFilter [ empty ] +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/signed_sorted.parquet]]}, projection=[id, value], output_ordering=[value@1 ASC NULLS LAST], file_type=parquet, predicate=DynamicFilter [ empty ], dynamic_rg_pruning=eligible # Results should still be correct (no optimization applied) query IR @@ -1988,7 +1988,7 @@ logical_plan 02)--TableScan: tb_overlap projection=[id, value] physical_plan 01)SortExec: TopK(fetch=5), expr=[id@0 DESC, value@1 DESC], preserve_partitioning=[false] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/tb_overlap/file_z.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/tb_overlap/file_y.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/tb_overlap/file_x.parquet]]}, projection=[id, value], file_type=parquet, predicate=DynamicFilter [ empty ], sort_order_for_reorder=[id@0 DESC, value@1 DESC], reverse_row_groups=true +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/tb_overlap/file_z.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/tb_overlap/file_y.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/tb_overlap/file_x.parquet]]}, projection=[id, value], file_type=parquet, predicate=DynamicFilter [ empty ], sort_order_for_reorder=[id@0 DESC, value@1 DESC], reverse_row_groups=true, dynamic_rg_pruning=eligible query II SELECT * FROM tb_overlap ORDER BY id DESC, value DESC LIMIT 5; @@ -2073,7 +2073,7 @@ logical_plan 02)--TableScan: tc_limit projection=[id, value] physical_plan 01)SortExec: TopK(fetch=3), expr=[id@0 DESC], preserve_partitioning=[false] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/tc_limit/file_c.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/tc_limit/file_b.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/tc_limit/file_a.parquet]]}, projection=[id, value], file_type=parquet, predicate=DynamicFilter [ empty ], sort_order_for_reorder=[id@0 DESC], reverse_row_groups=true +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/tc_limit/file_c.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/tc_limit/file_b.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/tc_limit/file_a.parquet]]}, projection=[id, value], file_type=parquet, predicate=DynamicFilter [ empty ], sort_order_for_reorder=[id@0 DESC], reverse_row_groups=true, dynamic_rg_pruning=eligible query II SELECT * FROM tc_limit ORDER BY id DESC LIMIT 3; @@ -2544,7 +2544,7 @@ logical_plan 02)--TableScan: th_reorder projection=[id, value] 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/sort_pushdown/th_reorder/data.parquet]]}, projection=[id, value], file_type=parquet, predicate=DynamicFilter [ empty ], sort_order_for_reorder=[id@0 ASC NULLS LAST] +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/th_reorder/data.parquet]]}, projection=[id, value], file_type=parquet, predicate=DynamicFilter [ empty ], sort_order_for_reorder=[id@0 ASC NULLS LAST], dynamic_rg_pruning=eligible # Results must be correct regardless of RG reorder. query II @@ -2563,7 +2563,7 @@ logical_plan 02)--TableScan: th_reorder projection=[id, value] physical_plan 01)SortExec: TopK(fetch=3), expr=[id@0 DESC], preserve_partitioning=[false] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/th_reorder/data.parquet]]}, projection=[id, value], file_type=parquet, predicate=DynamicFilter [ empty ], sort_order_for_reorder=[id@0 DESC], reverse_row_groups=true +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/th_reorder/data.parquet]]}, projection=[id, value], file_type=parquet, predicate=DynamicFilter [ empty ], sort_order_for_reorder=[id@0 DESC], reverse_row_groups=true, dynamic_rg_pruning=eligible query II SELECT * FROM th_reorder ORDER BY id DESC LIMIT 3; @@ -2649,7 +2649,7 @@ logical_plan 02)--TableScan: tj_scrambled projection=[id, value] physical_plan 01)SortExec: TopK(fetch=3), expr=[id@0 DESC], preserve_partitioning=[false] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/tj_scrambled/data.parquet]]}, projection=[id, value], file_type=parquet, predicate=DynamicFilter [ empty ], sort_order_for_reorder=[id@0 DESC], reverse_row_groups=true +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/tj_scrambled/data.parquet]]}, projection=[id, value], file_type=parquet, predicate=DynamicFilter [ empty ], sort_order_for_reorder=[id@0 DESC], reverse_row_groups=true, dynamic_rg_pruning=eligible # Test J.2: Results must be correct query II @@ -2839,7 +2839,7 @@ logical_plan 02)--TableScan: tl_sorted projection=[id, value] physical_plan 01)SortExec: TopK(fetch=3), expr=[id@0 DESC, value@1 ASC NULLS LAST], preserve_partitioning=[false] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/tl_multikey/data.parquet]]}, projection=[id, value], file_type=parquet, predicate=DynamicFilter [ empty ], sort_order_for_reorder=[id@0 DESC, value@1 ASC NULLS LAST], reverse_row_groups=true +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/sort_pushdown/tl_multikey/data.parquet]]}, projection=[id, value], file_type=parquet, predicate=DynamicFilter [ empty ], sort_order_for_reorder=[id@0 DESC, value@1 ASC NULLS LAST], reverse_row_groups=true, dynamic_rg_pruning=eligible query II SELECT id, value FROM tl_sorted ORDER BY id DESC, value ASC LIMIT 3; diff --git a/datafusion/sqllogictest/test_files/statistics_registry.slt b/datafusion/sqllogictest/test_files/statistics_registry.slt index c856e779a0877..89258bec299c1 100644 --- a/datafusion/sqllogictest/test_files/statistics_registry.slt +++ b/datafusion/sqllogictest/test_files/statistics_registry.slt @@ -104,9 +104,9 @@ physical_plan 02)--RepartitionExec: partitioning=Hash([small_id@2], 4), input_partitions=1, maintains_sort_order=true 03)----HashJoinExec: mode=Partitioned, join_type=Inner, on=[(customer_id@0, customer_id@1)], projection=[region_id@1, order_id@2, small_id@4] 04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/statistics_registry/customers.parquet]]}, projection=[customer_id, region_id], output_ordering=[region_id@1 ASC NULLS LAST], file_type=parquet -05)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/statistics_registry/orders.parquet]]}, projection=[order_id, customer_id, small_id], output_ordering=[order_id@0 ASC NULLS LAST], file_type=parquet, predicate=DynamicFilter [ empty ] +05)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/statistics_registry/orders.parquet]]}, projection=[order_id, customer_id, small_id], output_ordering=[order_id@0 ASC NULLS LAST], file_type=parquet, predicate=DynamicFilter [ empty ], dynamic_rg_pruning=eligible 06)--RepartitionExec: partitioning=Hash([small_id@0], 4), input_partitions=1, maintains_sort_order=true -07)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/statistics_registry/dim_small.parquet]]}, projection=[small_id, label], output_ordering=[small_id@0 ASC NULLS LAST], file_type=parquet, predicate=DynamicFilter [ empty ] +07)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/statistics_registry/dim_small.parquet]]}, projection=[small_id, label], output_ordering=[small_id@0 ASC NULLS LAST], file_type=parquet, predicate=DynamicFilter [ empty ], dynamic_rg_pruning=eligible # -- With registry ----------------------------------------------------------- # Conservative estimate 100 > 50: dim_small correctly swapped to build side @@ -127,7 +127,7 @@ physical_plan 04)--RepartitionExec: partitioning=Hash([small_id@2], 4), input_partitions=1, maintains_sort_order=true 05)----HashJoinExec: mode=Partitioned, join_type=Inner, on=[(customer_id@0, customer_id@1)], projection=[region_id@1, order_id@2, small_id@4] 06)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/statistics_registry/customers.parquet]]}, projection=[customer_id, region_id], output_ordering=[region_id@1 ASC NULLS LAST], file_type=parquet -07)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/statistics_registry/orders.parquet]]}, projection=[order_id, customer_id, small_id], output_ordering=[order_id@0 ASC NULLS LAST], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ] +07)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/statistics_registry/orders.parquet]]}, projection=[order_id, customer_id, small_id], output_ordering=[order_id@0 ASC NULLS LAST], file_type=parquet, predicate=DynamicFilter [ empty ] AND DynamicFilter [ empty ], dynamic_rg_pruning=eligible # -- Verify results are identical regardless of join order -------------------- diff --git a/datafusion/sqllogictest/test_files/topk.slt b/datafusion/sqllogictest/test_files/topk.slt index 8cab67dac0acb..d669e845ac7e6 100644 --- a/datafusion/sqllogictest/test_files/topk.slt +++ b/datafusion/sqllogictest/test_files/topk.slt @@ -316,7 +316,7 @@ explain select number, letter, age from partial_sorted order by number desc, let ---- physical_plan 01)SortExec: TopK(fetch=3), expr=[number@0 DESC, letter@1 ASC NULLS LAST, age@2 DESC], preserve_partitioning=[false], sort_prefix=[number@0 DESC, letter@1 ASC NULLS LAST] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/topk/partial_sorted/1.parquet]]}, projection=[number, letter, age], output_ordering=[number@0 DESC, letter@1 ASC NULLS LAST], file_type=parquet, predicate=DynamicFilter [ empty ] +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/topk/partial_sorted/1.parquet]]}, projection=[number, letter, age], output_ordering=[number@0 DESC, letter@1 ASC NULLS LAST], file_type=parquet, predicate=DynamicFilter [ empty ], dynamic_rg_pruning=eligible # Explain variations of the above query with different orderings, and different sort prefixes. @@ -326,28 +326,28 @@ explain select number, letter, age from partial_sorted order by age desc limit 3 ---- physical_plan 01)SortExec: TopK(fetch=3), expr=[age@2 DESC], preserve_partitioning=[false] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/topk/partial_sorted/1.parquet]]}, projection=[number, letter, age], file_type=parquet, predicate=DynamicFilter [ empty ], sort_order_for_reorder=[age@2 DESC], reverse_row_groups=true +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/topk/partial_sorted/1.parquet]]}, projection=[number, letter, age], file_type=parquet, predicate=DynamicFilter [ empty ], sort_order_for_reorder=[age@2 DESC], reverse_row_groups=true, dynamic_rg_pruning=eligible query TT explain select number, letter, age from partial_sorted order by number desc, letter desc limit 3; ---- physical_plan 01)SortExec: TopK(fetch=3), expr=[number@0 DESC, letter@1 DESC], preserve_partitioning=[false], sort_prefix=[number@0 DESC] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/topk/partial_sorted/1.parquet]]}, projection=[number, letter, age], output_ordering=[number@0 DESC, letter@1 ASC NULLS LAST], file_type=parquet, predicate=DynamicFilter [ empty ] +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/topk/partial_sorted/1.parquet]]}, projection=[number, letter, age], output_ordering=[number@0 DESC, letter@1 ASC NULLS LAST], file_type=parquet, predicate=DynamicFilter [ empty ], dynamic_rg_pruning=eligible query TT explain select number, letter, age from partial_sorted order by number asc limit 3; ---- physical_plan 01)SortExec: TopK(fetch=3), expr=[number@0 ASC NULLS LAST], preserve_partitioning=[false] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/topk/partial_sorted/1.parquet]]}, projection=[number, letter, age], file_type=parquet, predicate=DynamicFilter [ empty ], sort_order_for_reorder=[number@0 ASC NULLS LAST] +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/topk/partial_sorted/1.parquet]]}, projection=[number, letter, age], file_type=parquet, predicate=DynamicFilter [ empty ], sort_order_for_reorder=[number@0 ASC NULLS LAST], dynamic_rg_pruning=eligible query TT explain select number, letter, age from partial_sorted order by letter asc, number desc limit 3; ---- physical_plan 01)SortExec: TopK(fetch=3), expr=[letter@1 ASC NULLS LAST, number@0 DESC], preserve_partitioning=[false] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/topk/partial_sorted/1.parquet]]}, projection=[number, letter, age], file_type=parquet, predicate=DynamicFilter [ empty ], sort_order_for_reorder=[letter@1 ASC NULLS LAST, number@0 DESC] +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/topk/partial_sorted/1.parquet]]}, projection=[number, letter, age], file_type=parquet, predicate=DynamicFilter [ empty ], sort_order_for_reorder=[letter@1 ASC NULLS LAST, number@0 DESC], dynamic_rg_pruning=eligible # Explicit NULLS ordering cases (reversing the order of the NULLS on the number and letter orderings) query TT @@ -355,14 +355,14 @@ explain select number, letter, age from partial_sorted order by number desc, let ---- physical_plan 01)SortExec: TopK(fetch=3), expr=[number@0 DESC, letter@1 ASC], preserve_partitioning=[false], sort_prefix=[number@0 DESC] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/topk/partial_sorted/1.parquet]]}, projection=[number, letter, age], output_ordering=[number@0 DESC, letter@1 ASC NULLS LAST], file_type=parquet, predicate=DynamicFilter [ empty ] +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/topk/partial_sorted/1.parquet]]}, projection=[number, letter, age], output_ordering=[number@0 DESC, letter@1 ASC NULLS LAST], file_type=parquet, predicate=DynamicFilter [ empty ], dynamic_rg_pruning=eligible query TT explain select number, letter, age from partial_sorted order by number desc NULLS LAST, letter asc limit 3; ---- physical_plan 01)SortExec: TopK(fetch=3), expr=[number@0 DESC NULLS LAST, letter@1 ASC NULLS LAST], preserve_partitioning=[false] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/topk/partial_sorted/1.parquet]]}, projection=[number, letter, age], file_type=parquet, predicate=DynamicFilter [ empty ], sort_order_for_reorder=[number@0 DESC NULLS LAST, letter@1 ASC NULLS LAST], reverse_row_groups=true +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/topk/partial_sorted/1.parquet]]}, projection=[number, letter, age], file_type=parquet, predicate=DynamicFilter [ empty ], sort_order_for_reorder=[number@0 DESC NULLS LAST, letter@1 ASC NULLS LAST], reverse_row_groups=true, dynamic_rg_pruning=eligible # Verify that the sort prefix is correctly computed on the normalized ordering (removing redundant aliased columns) @@ -371,7 +371,7 @@ explain select number, letter, age, number as column4, letter as column5 from pa ---- physical_plan 01)SortExec: TopK(fetch=3), expr=[number@0 DESC, letter@1 ASC NULLS LAST, age@2 DESC], preserve_partitioning=[false], sort_prefix=[number@0 DESC, letter@1 ASC NULLS LAST] -02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/topk/partial_sorted/1.parquet]]}, projection=[number, letter, age, number@0 as column4, letter@1 as column5], output_ordering=[number@0 DESC, letter@1 ASC NULLS LAST], file_type=parquet, predicate=DynamicFilter [ empty ] +02)--DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/topk/partial_sorted/1.parquet]]}, projection=[number, letter, age, number@0 as column4, letter@1 as column5], output_ordering=[number@0 DESC, letter@1 ASC NULLS LAST], file_type=parquet, predicate=DynamicFilter [ empty ], dynamic_rg_pruning=eligible # Verify that the sort prefix is correctly computed over normalized, order-maintaining projections (number + 1, number, number + 1, age) query TT @@ -383,7 +383,7 @@ physical_plan 03)----ProjectionExec: expr=[__common_expr_1@0 as number_plus, number@1 as number, __common_expr_1@0 as other_number_plus, age@2 as age] 04)------ProjectionExec: expr=[CAST(number@0 AS Int64) + 1 as __common_expr_1, number@0 as number, age@1 as age] 05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1, maintains_sort_order=true -06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/topk/partial_sorted/1.parquet]]}, projection=[number, age], output_ordering=[number@0 DESC], file_type=parquet, predicate=DynamicFilter [ empty ] +06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/topk/partial_sorted/1.parquet]]}, projection=[number, age], output_ordering=[number@0 DESC], file_type=parquet, predicate=DynamicFilter [ empty ], dynamic_rg_pruning=eligible # Cleanup statement ok