diff --git a/Cargo.lock b/Cargo.lock index 512fbd3c678dd..37b18c3048f30 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -1968,6 +1968,7 @@ dependencies = [ "liblzma", "log", "object_store", + "parking_lot", "rand 0.9.2", "tempfile", "tokio", @@ -2660,6 +2661,7 @@ dependencies = [ "object_store", "postgres-types", "regex", + "serde_json", "sqllogictest", "sqlparser", "tempfile", diff --git a/benchmarks/bench.sh b/benchmarks/bench.sh index aa1ec477345c6..0c87afc03becf 100755 --- a/benchmarks/bench.sh +++ b/benchmarks/bench.sh @@ -820,7 +820,7 @@ run_clickbench_partitioned() { run_clickbench_pushdown() { RESULTS_FILE="${RESULTS_DIR}/clickbench_pushdown.json" echo "RESULTS_FILE: ${RESULTS_FILE}" - echo "Running clickbench (partitioned, 100 files) benchmark with pushdown_filters=true, reorder_filters=true..." + echo "Running clickbench (partitioned, 100 files) benchmark with pushdown_filters=true..." debug_run $CARGO_COMMAND --bin dfbench -- clickbench --pushdown --iterations 5 --path "${DATA_DIR}/hits_partitioned" --queries-path "${SCRIPT_DIR}/queries/clickbench/queries" -o "${RESULTS_FILE}" ${QUERY_ARG} ${LATENCY_ARG} } diff --git a/benchmarks/src/clickbench.rs b/benchmarks/src/clickbench.rs index 70aaeb7d2d192..40491b48ab330 100644 --- a/benchmarks/src/clickbench.rs +++ b/benchmarks/src/clickbench.rs @@ -58,7 +58,6 @@ pub struct RunOpt { /// /// Specifically, it enables: /// * `pushdown_filters = true` - /// * `reorder_filters = true` #[arg(long = "pushdown")] pushdown: bool, @@ -196,14 +195,12 @@ impl RunOpt { // Turn on Parquet filter pushdown if requested if self.pushdown { parquet_options.pushdown_filters = true; - parquet_options.reorder_filters = true; } if self.sorted_by.is_some() { // We should compare the dynamic topk optimization when data is sorted, so we make the // assumption that filter pushdown is also enabled in this case. parquet_options.pushdown_filters = true; - parquet_options.reorder_filters = true; } } diff --git a/datafusion-cli/src/print_format.rs b/datafusion-cli/src/print_format.rs index 6a6a0370b08ac..0443a7a289602 100644 --- a/datafusion-cli/src/print_format.rs +++ b/datafusion-cli/src/print_format.rs @@ -97,7 +97,7 @@ fn keep_only_maxrows(s: &str, maxrows: usize) -> String { let last_line = &lines[lines.len() - 1]; // bottom border line let spaces = last_line.len().saturating_sub(4); - let dotted_line = format!("| .{: SchemaRef { + let fields: Vec = (0..10) + .map(|i| Field::new(format!("c{i}"), DataType::Int32, false)) + .collect(); + Arc::new(Schema::new(fields)) + } + + /// return a batch with many columns and three rows + fn wide_column_batch() -> RecordBatch { + let arrays: Vec> = (0..10) + .map(|_| Arc::new(Int32Array::from(vec![0, 1, 2])) as _) + .collect(); + RecordBatch::try_new(wide_column_schema(), arrays).unwrap() + } + /// Slice the record batch into 2 batches fn split_batch(batch: &RecordBatch) -> Vec { assert!(batch.num_rows() > 1); diff --git a/datafusion-cli/tests/cli_integration.rs b/datafusion-cli/tests/cli_integration.rs index 7aa405338bffe..3cecba75e21b0 100644 --- a/datafusion-cli/tests/cli_integration.rs +++ b/datafusion-cli/tests/cli_integration.rs @@ -287,6 +287,23 @@ fn test_cli_with_unbounded_memory_pool() { assert_cmd_snapshot!(cmd); } +#[test] +fn test_cli_wide_result_set_no_crash() { + let mut settings = make_settings(); + + settings.set_snapshot_suffix("wide_result_set"); + + let _bound = settings.bind_to_scope(); + + let mut cmd = cli(); + let sql = "SELECT v1 as c0, v1+1 as c1, v1+2 as c2, v1+3 as c3, v1+4 as c4, \ + v1+5 as c5, v1+6 as c6, v1+7 as c7, v1+8 as c8, v1+9 as c9 \ + FROM generate_series(1, 100) as t1(v1);"; + cmd.args(["--maxrows", "5", "--command", sql]); + + assert_cmd_snapshot!(cmd); +} + #[tokio::test] async fn test_cli() { if env::var("TEST_STORAGE_INTEGRATION").is_err() { diff --git a/datafusion-cli/tests/snapshots/cli_wide_result_set_no_crash@wide_result_set.snap b/datafusion-cli/tests/snapshots/cli_wide_result_set_no_crash@wide_result_set.snap new file mode 100644 index 0000000000000..30b34f3c12baa --- /dev/null +++ b/datafusion-cli/tests/snapshots/cli_wide_result_set_no_crash@wide_result_set.snap @@ -0,0 +1,32 @@ +--- +source: datafusion-cli/tests/cli_integration.rs +assertion_line: 307 +info: + program: datafusion-cli + args: + - "--maxrows" + - "5" + - "--command" + - "SELECT v1 as c0, v1+1 as c1, v1+2 as c2, v1+3 as c3, v1+4 as c4, v1+5 as c5, v1+6 as c6, v1+7 as c7, v1+8 as c8, v1+9 as c9 FROM generate_series(1, 100) as t1(v1);" +--- +success: true +exit_code: 0 +----- stdout ----- +[CLI_VERSION] ++----+----+----+----+----+----+----+----+----+----+ +| c0 | c1 | c2 | c3 | c4 | c5 | c6 | c7 | c8 | c9 | ++----+----+----+----+----+----+----+----+----+----+ +| 1 | 2 | 3 | 4 | 5 | 6 | 7 | 8 | 9 | 10 | +| 2 | 3 | 4 | 5 | 6 | 7 | 8 | 9 | 10 | 11 | +| 3 | 4 | 5 | 6 | 7 | 8 | 9 | 10 | 11 | 12 | +| 4 | 5 | 6 | 7 | 8 | 9 | 10 | 11 | 12 | 13 | +| 5 | 6 | 7 | 8 | 9 | 10 | 11 | 12 | 13 | 14 | +| . | +| . | +| . | ++----+----+----+----+----+----+----+----+----+----+ +100 row(s) fetched. (First 5 displayed. Use --maxrows to adjust) +[ELAPSED] + + +----- stderr ----- diff --git a/datafusion-examples/examples/data_io/json_shredding.rs b/datafusion-examples/examples/data_io/json_shredding.rs index 72fbb56773123..174e05a914b29 100644 --- a/datafusion-examples/examples/data_io/json_shredding.rs +++ b/datafusion-examples/examples/data_io/json_shredding.rs @@ -90,7 +90,10 @@ pub async fn json_shredding() -> Result<()> { store.put(&path, payload).await?; // Set up query execution - let mut cfg = SessionConfig::new(); + let mut cfg = SessionConfig::default().set( + "datafusion.execution.parquet.filter_pushdown_min_bytes_per_sec", + &ScalarValue::Float64(Some(0.0)), + ); cfg.options_mut().execution.parquet.pushdown_filters = true; let ctx = SessionContext::new_with_config(cfg); ctx.runtime_env().register_object_store( diff --git a/datafusion/common/src/config.rs b/datafusion/common/src/config.rs index 85361ef5e17e1..803ca83603f3b 100644 --- a/datafusion/common/src/config.rs +++ b/datafusion/common/src/config.rs @@ -877,11 +877,6 @@ config_namespace! { /// reduce the number of rows decoded. This optimization is sometimes called "late materialization". pub pushdown_filters: bool, default = false - /// (reading) If true, filter expressions evaluated during the parquet decoding operation - /// will be reordered heuristically to minimize the cost of evaluation. If false, - /// the filters are applied in the same order as written in the query - pub reorder_filters: bool, default = false - /// (reading) Force the use of RowSelections for filter results, when /// pushdown_filters is enabled. If false, the reader will automatically /// choose between a RowSelection and a Bitmap based on the number and @@ -919,6 +914,49 @@ config_namespace! { /// parquet reader setting. 0 means no caching. pub max_predicate_cache_size: Option, default = None + /// (reading) Minimum bytes/sec throughput for adaptive filter pushdown. + /// Filters that achieve at least this throughput (bytes_saved / eval_time) + /// are promoted to row filters. + /// f64::INFINITY = no filters promoted (feature disabled). + /// 0.0 = all filters pushed as row filters (no adaptive logic). + /// Default: 104,857,600 bytes/sec (100 MiB/sec), empirically chosen based on + /// TPC-H, TPC-DS, and ClickBench benchmarks on an m4 MacBook Pro. + /// The optimal value for this setting likely depends on the relative + /// cost of CPU vs. IO in your environment, and to some extent the shape + /// of your query. + /// + /// **Interaction with `pushdown_filters`:** + /// This option only takes effect when `pushdown_filters = true`. + /// When pushdown is disabled, all filters run post-scan and this + /// threshold is ignored. + pub filter_pushdown_min_bytes_per_sec: f64, default = 104_857_600.0 + + /// (reading) Byte-ratio threshold for applying filters one at a time + /// (iterative pruning; aka row-level) vs. all at once (post-scan). + /// The ratio is computed as: (extra filter bytes not in projection) / (projected bytes). + /// Filters whose extra columns consume a smaller fraction than this threshold are placed as row filters. + /// Ratio of filter column bytes to projection bytes that controls + /// initial filter placement. Computed as + /// `filter_compressed_bytes / projection_compressed_bytes`. + /// Filters below this ratio start as row-level filters (enabling late + /// materialization); those above start as post-scan filters. + /// Default: 0.20 — filters whose columns are less than 20% of the + /// projection bytes start at row-level. + /// + /// **Interaction with `pushdown_filters`:** + /// Only takes effect when `pushdown_filters = true`. + pub filter_collecting_byte_ratio_threshold: f64, default = 0.20 + + /// (reading) Z-score for confidence intervals on filter effectiveness. + /// Controls how much statistical evidence is required before promoting + /// or demoting a filter. Lower values = faster decisions with less + /// confidence. Higher values = more conservative, requiring more data. + /// Default: 2.0 (~95% confidence). + /// + /// **Interaction with `pushdown_filters`:** + /// Only takes effect when `pushdown_filters = true`. + pub filter_confidence_z: f64, default = 2.0 + // The following options affect writing to parquet files // and map to parquet::file::properties::WriterProperties diff --git a/datafusion/common/src/dfschema.rs b/datafusion/common/src/dfschema.rs index de0aacf9e8bcd..e7d9e809daecc 100644 --- a/datafusion/common/src/dfschema.rs +++ b/datafusion/common/src/dfschema.rs @@ -1338,11 +1338,44 @@ impl SchemaExt for Schema { } } +/// Build a fully-qualified field name string. This is equivalent to +/// `format!("{q}.{name}")` when `qualifier` is `Some`, or just `name` when +/// `None`. We avoid going through the `fmt` machinery for performance reasons. pub fn qualified_name(qualifier: Option<&TableReference>, name: &str) -> String { - match qualifier { - Some(q) => format!("{q}.{name}"), - None => name.to_string(), - } + let qualifier = match qualifier { + None => return name.to_string(), + Some(q) => q, + }; + let (first, second, third) = match qualifier { + TableReference::Bare { table } => (table.as_ref(), None, None), + TableReference::Partial { schema, table } => { + (schema.as_ref(), Some(table.as_ref()), None) + } + TableReference::Full { + catalog, + schema, + table, + } => ( + catalog.as_ref(), + Some(schema.as_ref()), + Some(table.as_ref()), + ), + }; + + let extra = second.map_or(0, str::len) + third.map_or(0, str::len); + let mut s = String::with_capacity(first.len() + extra + 3 + name.len()); + s.push_str(first); + if let Some(second) = second { + s.push('.'); + s.push_str(second); + } + if let Some(third) = third { + s.push('.'); + s.push_str(third); + } + s.push('.'); + s.push_str(name); + s } #[cfg(test)] @@ -1351,6 +1384,36 @@ mod tests { use super::*; + /// `qualified_name` doesn't use `TableReference::Display` for performance + /// reasons, but check that the output is consistent. + #[test] + fn qualified_name_agrees_with_display() { + let cases: &[(Option, &str)] = &[ + (None, "col"), + (Some(TableReference::bare("t")), "c0"), + (Some(TableReference::partial("s", "t")), "c0"), + (Some(TableReference::full("c", "s", "t")), "c0"), + (Some(TableReference::bare("mytable")), "some_column_name"), + // Empty segments must be preserved so that distinct qualified + // fields don't collide in `DFSchema::field_names()`. + (Some(TableReference::bare("")), "col"), + (Some(TableReference::partial("s", "")), "col"), + (Some(TableReference::partial("", "t")), "col"), + (Some(TableReference::full("c", "", "t")), "col"), + (Some(TableReference::full("", "s", "t")), "col"), + (Some(TableReference::full("c", "s", "")), "col"), + (Some(TableReference::full("", "", "")), "col"), + ]; + for (qualifier, name) in cases { + let actual = qualified_name(qualifier.as_ref(), name); + let expected = match qualifier { + Some(q) => format!("{q}.{name}"), + None => name.to_string(), + }; + assert_eq!(actual, expected, "qualifier={qualifier:?} name={name}"); + } + } + #[test] fn qualifier_in_name() -> Result<()> { let col = Column::from_name("t1.c0"); diff --git a/datafusion/common/src/file_options/parquet_writer.rs b/datafusion/common/src/file_options/parquet_writer.rs index eaf5a1642e8e2..6cab1516d62b2 100644 --- a/datafusion/common/src/file_options/parquet_writer.rs +++ b/datafusion/common/src/file_options/parquet_writer.rs @@ -199,7 +199,7 @@ impl ParquetOptions { skip_metadata: _, metadata_size_hint: _, pushdown_filters: _, - reorder_filters: _, + force_filter_selections: _, // not used for writer props allow_single_file_parallelism: _, maximum_parallel_row_group_writers: _, @@ -210,6 +210,9 @@ impl ParquetOptions { coerce_int96: _, // not used for writer props skip_arrow_metadata: _, max_predicate_cache_size: _, + filter_pushdown_min_bytes_per_sec: _, // not used for writer props + filter_collecting_byte_ratio_threshold: _, // not used for writer props + filter_confidence_z: _, // not used for writer props } = self; let mut builder = WriterProperties::builder() @@ -470,7 +473,7 @@ mod tests { skip_metadata: defaults.skip_metadata, metadata_size_hint: defaults.metadata_size_hint, pushdown_filters: defaults.pushdown_filters, - reorder_filters: defaults.reorder_filters, + force_filter_selections: defaults.force_filter_selections, allow_single_file_parallelism: defaults.allow_single_file_parallelism, maximum_parallel_row_group_writers: defaults @@ -484,6 +487,10 @@ mod tests { coerce_int96: None, max_predicate_cache_size: defaults.max_predicate_cache_size, use_content_defined_chunking: defaults.use_content_defined_chunking.clone(), + filter_pushdown_min_bytes_per_sec: defaults.filter_pushdown_min_bytes_per_sec, + filter_collecting_byte_ratio_threshold: defaults + .filter_collecting_byte_ratio_threshold, + filter_confidence_z: defaults.filter_confidence_z, } } @@ -585,7 +592,7 @@ mod tests { skip_metadata: global_options_defaults.skip_metadata, metadata_size_hint: global_options_defaults.metadata_size_hint, pushdown_filters: global_options_defaults.pushdown_filters, - reorder_filters: global_options_defaults.reorder_filters, + force_filter_selections: global_options_defaults.force_filter_selections, allow_single_file_parallelism: global_options_defaults .allow_single_file_parallelism, @@ -607,6 +614,11 @@ mod tests { norm_level: c.norm_level, } }), + filter_pushdown_min_bytes_per_sec: global_options_defaults + .filter_pushdown_min_bytes_per_sec, + filter_collecting_byte_ratio_threshold: global_options_defaults + .filter_collecting_byte_ratio_threshold, + filter_confidence_z: global_options_defaults.filter_confidence_z, }, column_specific_options, key_value_metadata, diff --git a/datafusion/common/src/functional_dependencies.rs b/datafusion/common/src/functional_dependencies.rs index 63962998ad18b..24ca33c0c2c90 100644 --- a/datafusion/common/src/functional_dependencies.rs +++ b/datafusion/common/src/functional_dependencies.rs @@ -590,6 +590,53 @@ pub fn get_required_group_by_exprs_indices( .collect() } +/// Returns indices for the minimal subset of ORDER BY expressions that are +/// functionally equivalent to the original set of ORDER BY expressions. +pub fn get_required_sort_exprs_indices( + schema: &DFSchema, + sort_expr_names: &[String], +) -> Vec { + let dependencies = schema.functional_dependencies(); + let field_names = schema.field_names(); + + let mut known_field_indices = HashSet::new(); + let mut required_sort_expr_indices = Vec::new(); + + for (sort_expr_idx, sort_expr_name) in sort_expr_names.iter().enumerate() { + // If the sort expression doesn't correspond to a known schema field + // (e.g. a computed expression), we can't reason about it via functional + // dependencies, so conservatively keep it. + let Some(field_idx) = field_names + .iter() + .position(|field_name| field_name == sort_expr_name) + else { + required_sort_expr_indices.push(sort_expr_idx); + continue; + }; + + // A sort expression is removable if its value is functionally determined + // by fields that already appear earlier in the sort order: if the earlier + // fields are fixed, this one's value is fixed too, so it adds no ordering + // information. + let removable = dependencies.deps.iter().any(|dependency| { + dependency.target_indices.contains(&field_idx) + && dependency + .source_indices + .iter() + .all(|source_idx| known_field_indices.contains(source_idx)) + }); + + if removable { + continue; + } + + known_field_indices.insert(field_idx); + required_sort_expr_indices.push(sort_expr_idx); + } + + required_sort_expr_indices +} + /// Updates entries inside the `entries` vector with their corresponding /// indices inside the `proj_indices` vector. fn update_elements_with_matching_indices( diff --git a/datafusion/common/src/lib.rs b/datafusion/common/src/lib.rs index fdd04f752455e..996c563f0d8a2 100644 --- a/datafusion/common/src/lib.rs +++ b/datafusion/common/src/lib.rs @@ -82,7 +82,7 @@ pub use file_options::file_type::{ pub use functional_dependencies::{ Constraint, Constraints, Dependency, FunctionalDependence, FunctionalDependencies, aggregate_functional_dependencies, get_required_group_by_exprs_indices, - get_target_functional_dependencies, + get_required_sort_exprs_indices, get_target_functional_dependencies, }; use hashbrown::DefaultHashBuilder; pub use join_type::{JoinConstraint, JoinSide, JoinType}; diff --git a/datafusion/core/src/dataframe/parquet.rs b/datafusion/core/src/dataframe/parquet.rs index ccd5766f0a24d..a0b3c992fa076 100644 --- a/datafusion/core/src/dataframe/parquet.rs +++ b/datafusion/core/src/dataframe/parquet.rs @@ -164,7 +164,14 @@ mod tests { let plan = df.explain(false, false)?.collect().await?; // Filters all the way to Parquet let formatted = pretty::pretty_format_batches(&plan)?.to_string(); - assert!(formatted.contains("FilterExec: id@0 = 1"), "{formatted}"); + let data_source_exec_row = formatted + .lines() + .find(|line| line.contains("DataSourceExec:")) + .unwrap(); + assert!( + data_source_exec_row.contains("predicate=id@0 = 1"), + "{formatted}" + ); Ok(()) } diff --git a/datafusion/core/src/datasource/physical_plan/parquet.rs b/datafusion/core/src/datasource/physical_plan/parquet.rs index dd8c20628b43e..0401f4604137e 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet.rs @@ -167,9 +167,11 @@ mod tests { } if self.pushdown_predicate { + let mut opts = source.table_parquet_options().clone(); + opts.global.filter_pushdown_min_bytes_per_sec = 0.0; source = source - .with_pushdown_filters(true) - .with_reorder_filters(true); + .with_table_parquet_options(opts) + .with_pushdown_filters(true); } else { source = source.with_pushdown_filters(false); } diff --git a/datafusion/core/src/test_util/parquet.rs b/datafusion/core/src/test_util/parquet.rs index c53495421307b..6aa313ce720fc 100644 --- a/datafusion/core/src/test_util/parquet.rs +++ b/datafusion/core/src/test_util/parquet.rs @@ -57,8 +57,6 @@ pub struct TestParquetFile { pub struct ParquetScanOptions { /// Enable pushdown filters pub pushdown_filters: bool, - /// enable reordering filters - pub reorder_filters: bool, /// enable page index pub enable_page_index: bool, } @@ -68,8 +66,9 @@ impl ParquetScanOptions { pub fn config(&self) -> SessionConfig { let mut config = ConfigOptions::new(); config.execution.parquet.pushdown_filters = self.pushdown_filters; - config.execution.parquet.reorder_filters = self.reorder_filters; config.execution.parquet.enable_page_index = self.enable_page_index; + // Disable adaptive filter selection for tests that expect deterministic pushdown + config.execution.parquet.filter_pushdown_min_bytes_per_sec = 0.0; config.into() } } diff --git a/datafusion/core/tests/dataframe/dataframe_functions.rs b/datafusion/core/tests/dataframe/dataframe_functions.rs index 014f356cd64cd..2ada0411f4f8c 100644 --- a/datafusion/core/tests/dataframe/dataframe_functions.rs +++ b/datafusion/core/tests/dataframe/dataframe_functions.rs @@ -402,7 +402,7 @@ async fn test_fn_approx_median() -> Result<()> { +-----------------------+ | approx_median(test.b) | +-----------------------+ - | 10 | + | 10.0 | +-----------------------+ "); @@ -422,7 +422,7 @@ async fn test_fn_approx_percentile_cont() -> Result<()> { +---------------------------------------------------------------------------+ | approx_percentile_cont(Float64(0.5)) WITHIN GROUP [test.b ASC NULLS LAST] | +---------------------------------------------------------------------------+ - | 10 | + | 10.0 | +---------------------------------------------------------------------------+ "); @@ -437,7 +437,7 @@ async fn test_fn_approx_percentile_cont() -> Result<()> { +----------------------------------------------------------------------------+ | approx_percentile_cont(Float64(0.1)) WITHIN GROUP [test.b DESC NULLS LAST] | +----------------------------------------------------------------------------+ - | 100 | + | 100.0 | +----------------------------------------------------------------------------+ "); @@ -457,7 +457,7 @@ async fn test_fn_approx_percentile_cont() -> Result<()> { +--------------------------------------------------------------------+ | approx_percentile_cont(arg_2) WITHIN GROUP [test.b ASC NULLS LAST] | +--------------------------------------------------------------------+ - | 10 | + | 10.0 | +--------------------------------------------------------------------+ " ); @@ -477,7 +477,7 @@ async fn test_fn_approx_percentile_cont() -> Result<()> { +---------------------------------------------------------------------+ | approx_percentile_cont(arg_2) WITHIN GROUP [test.b DESC NULLS LAST] | +---------------------------------------------------------------------+ - | 100 | + | 100.0 | +---------------------------------------------------------------------+ " ); @@ -494,7 +494,7 @@ async fn test_fn_approx_percentile_cont() -> Result<()> { +------------------------------------------------------------------------------------+ | approx_percentile_cont(Float64(0.5),Int32(2)) WITHIN GROUP [test.b ASC NULLS LAST] | +------------------------------------------------------------------------------------+ - | 30 | + | 30.25 | +------------------------------------------------------------------------------------+ "); @@ -510,7 +510,7 @@ async fn test_fn_approx_percentile_cont() -> Result<()> { +-------------------------------------------------------------------------------------+ | approx_percentile_cont(Float64(0.1),Int32(2)) WITHIN GROUP [test.b DESC NULLS LAST] | +-------------------------------------------------------------------------------------+ - | 69 | + | 69.85 | +-------------------------------------------------------------------------------------+ "); diff --git a/datafusion/core/tests/dataframe/mod.rs b/datafusion/core/tests/dataframe/mod.rs index 26de1e23b38e1..e0830754399db 100644 --- a/datafusion/core/tests/dataframe/mod.rs +++ b/datafusion/core/tests/dataframe/mod.rs @@ -1204,26 +1204,26 @@ async fn window_using_aggregates() -> Result<()> { | first_value | last_val | approx_distinct | approx_median | median | max | min | c2 | c3 | +-------------+----------+-----------------+---------------+--------+-----+------+----+------+ | | | | | | | | 1 | -85 | - | -85 | -101 | 14 | -12 | -12 | 83 | -101 | 4 | -54 | - | -85 | -101 | 17 | -25 | -25 | 83 | -101 | 5 | -31 | - | -85 | -12 | 10 | -32 | -34 | 83 | -85 | 3 | 13 | - | -85 | -25 | 3 | -56 | -56 | -25 | -85 | 1 | -5 | - | -85 | -31 | 18 | -29 | -28 | 83 | -101 | 5 | 36 | - | -85 | -38 | 16 | -25 | -25 | 83 | -101 | 4 | 65 | - | -85 | -43 | 7 | -43 | -43 | 83 | -85 | 2 | 45 | - | -85 | -48 | 6 | -35 | -36 | 83 | -85 | 2 | -43 | - | -85 | -5 | 4 | -37 | -40 | -5 | -85 | 1 | 83 | - | -85 | -54 | 15 | -17 | -18 | 83 | -101 | 4 | -38 | - | -85 | -56 | 2 | -70 | -70 | -56 | -85 | 1 | -25 | - | -85 | -72 | 9 | -43 | -43 | 83 | -85 | 3 | -12 | - | -85 | -85 | 1 | -85 | -85 | -85 | -85 | 1 | -56 | - | -85 | 13 | 11 | -17 | -18 | 83 | -85 | 3 | 14 | - | -85 | 13 | 11 | -25 | -25 | 83 | -85 | 3 | 13 | - | -85 | 14 | 12 | -12 | -12 | 83 | -85 | 3 | 17 | - | -85 | 17 | 13 | -11 | -8 | 83 | -85 | 4 | -101 | - | -85 | 45 | 8 | -34 | -34 | 83 | -85 | 3 | -72 | - | -85 | 65 | 17 | -17 | -18 | 83 | -101 | 5 | -101 | - | -85 | 83 | 5 | -25 | -25 | 83 | -85 | 2 | -48 | + | -85 | -101 | 14 | -12.0 | -12 | 83 | -101 | 4 | -54 | + | -85 | -101 | 17 | -25.0 | -25 | 83 | -101 | 5 | -31 | + | -85 | -12 | 10 | -32.75 | -34 | 83 | -85 | 3 | 13 | + | -85 | -25 | 3 | -56.0 | -56 | -25 | -85 | 1 | -5 | + | -85 | -31 | 18 | -29.75 | -28 | 83 | -101 | 5 | 36 | + | -85 | -38 | 16 | -25.0 | -25 | 83 | -101 | 4 | 65 | + | -85 | -43 | 7 | -43.0 | -43 | 83 | -85 | 2 | 45 | + | -85 | -48 | 6 | -35.75 | -36 | 83 | -85 | 2 | -43 | + | -85 | -5 | 4 | -37.75 | -40 | -5 | -85 | 1 | 83 | + | -85 | -54 | 15 | -17.0 | -18 | 83 | -101 | 4 | -38 | + | -85 | -56 | 2 | -70.5 | -70 | -56 | -85 | 1 | -25 | + | -85 | -72 | 9 | -43.0 | -43 | 83 | -85 | 3 | -12 | + | -85 | -85 | 1 | -85.0 | -85 | -85 | -85 | 1 | -56 | + | -85 | 13 | 11 | -17.0 | -18 | 83 | -85 | 3 | 14 | + | -85 | 13 | 11 | -25.0 | -25 | 83 | -85 | 3 | 13 | + | -85 | 14 | 12 | -12.0 | -12 | 83 | -85 | 3 | 17 | + | -85 | 17 | 13 | -11.25 | -8 | 83 | -85 | 4 | -101 | + | -85 | 45 | 8 | -34.5 | -34 | 83 | -85 | 3 | -72 | + | -85 | 65 | 17 | -17.0 | -18 | 83 | -101 | 5 | -101 | + | -85 | 83 | 5 | -25.0 | -25 | 83 | -85 | 2 | -48 | +-------------+----------+-----------------+---------------+--------+-----+------+----+------+ " ); diff --git a/datafusion/core/tests/parquet/filter_pushdown.rs b/datafusion/core/tests/parquet/filter_pushdown.rs index e6266b2c088d7..6a5fc2c9beeed 100644 --- a/datafusion/core/tests/parquet/filter_pushdown.rs +++ b/datafusion/core/tests/parquet/filter_pushdown.rs @@ -444,7 +444,6 @@ impl<'a> TestCase<'a> { .read_with_options( ParquetScanOptions { pushdown_filters: false, - reorder_filters: false, enable_page_index: false, }, filter, @@ -455,7 +454,6 @@ impl<'a> TestCase<'a> { .read_with_options( ParquetScanOptions { pushdown_filters: true, - reorder_filters: false, enable_page_index: false, }, filter, @@ -464,24 +462,10 @@ impl<'a> TestCase<'a> { assert_eq!(no_pushdown, only_pushdown); - let pushdown_and_reordering = self - .read_with_options( - ParquetScanOptions { - pushdown_filters: true, - reorder_filters: true, - enable_page_index: false, - }, - filter, - ) - .await; - - assert_eq!(no_pushdown, pushdown_and_reordering); - let page_index_only = self .read_with_options( ParquetScanOptions { pushdown_filters: false, - reorder_filters: false, enable_page_index: true, }, filter, @@ -489,18 +473,17 @@ impl<'a> TestCase<'a> { .await; assert_eq!(no_pushdown, page_index_only); - let pushdown_reordering_and_page_index = self + let pushdown_and_page_index = self .read_with_options( ParquetScanOptions { pushdown_filters: true, - reorder_filters: true, enable_page_index: true, }, filter, ) .await; - assert_eq!(no_pushdown, pushdown_reordering_and_page_index); + assert_eq!(no_pushdown, pushdown_and_page_index); } /// Reads data from a test parquet file using the specified scan options @@ -633,6 +616,11 @@ async fn predicate_cache_default() -> datafusion_common::Result<()> { async fn predicate_cache_pushdown_default() -> datafusion_common::Result<()> { let mut config = SessionConfig::new(); config.options_mut().execution.parquet.pushdown_filters = true; + config + .options_mut() + .execution + .parquet + .filter_pushdown_min_bytes_per_sec = 0.0; let ctx = SessionContext::new_with_config(config); // The cache is on by default, and used when filter pushdown is enabled PredicateCacheTest { @@ -647,6 +635,11 @@ async fn predicate_cache_pushdown_default() -> datafusion_common::Result<()> { async fn predicate_cache_stats_issue_19561() -> datafusion_common::Result<()> { let mut config = SessionConfig::new(); config.options_mut().execution.parquet.pushdown_filters = true; + config + .options_mut() + .execution + .parquet + .filter_pushdown_min_bytes_per_sec = 0.0; // force to get multiple batches to trigger repeated metric compound bug config.options_mut().execution.batch_size = 1; let ctx = SessionContext::new_with_config(config); @@ -664,6 +657,11 @@ async fn predicate_cache_pushdown_default_selections_only() -> datafusion_common::Result<()> { let mut config = SessionConfig::new(); config.options_mut().execution.parquet.pushdown_filters = true; + config + .options_mut() + .execution + .parquet + .filter_pushdown_min_bytes_per_sec = 0.0; // forcing filter selections minimizes the number of rows read from the cache config .options_mut() diff --git a/datafusion/core/tests/physical_optimizer/filter_pushdown.rs b/datafusion/core/tests/physical_optimizer/filter_pushdown.rs index 4ff1fad8f52b9..1053db012938c 100644 --- a/datafusion/core/tests/physical_optimizer/filter_pushdown.rs +++ b/datafusion/core/tests/physical_optimizer/filter_pushdown.rs @@ -1031,7 +1031,7 @@ async fn test_hashjoin_dynamic_filter_pushdown_partitioned() { // expect the predicate to be pushed down into the probe side DataSource insta::assert_snapshot!( OptimizationTest::new(Arc::clone(&plan), FilterPushdown::new_post_optimization(), true), - @r" + @" OptimizationTest: input: - SortExec: expr=[a@0 DESC NULLS LAST], preserve_partitioning=[false] @@ -1049,7 +1049,7 @@ async fn test_hashjoin_dynamic_filter_pushdown_partitioned() { - RepartitionExec: partitioning=Hash([a@0, b@1], 12), input_partitions=1 - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true - RepartitionExec: partitioning=Hash([a@0, b@1], 12), input_partitions=1 - - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, e], file_type=test, pushdown_supported=true, predicate=DynamicFilter [ empty ] + - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, e], file_type=test, pushdown_supported=true, predicate=Optional(DynamicFilter [ empty ]) " ); @@ -1076,14 +1076,14 @@ async fn test_hashjoin_dynamic_filter_pushdown_partitioned() { #[cfg(not(feature = "force_hash_collisions"))] insta::assert_snapshot!( format!("{}", format_plan_for_test(&plan)), - @r" + @" - SortExec: expr=[a@0 DESC NULLS LAST], preserve_partitioning=[false] - CoalescePartitionsExec - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, a@0), (b@1, b@1)] - RepartitionExec: partitioning=Hash([a@0, b@1], 12), input_partitions=1 - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true - RepartitionExec: partitioning=Hash([a@0, b@1], 12), input_partitions=1 - - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, e], file_type=test, pushdown_supported=true, predicate=DynamicFilter [ CASE hash_repartition % 12 WHEN 5 THEN a@0 >= ab AND a@0 <= ab AND b@1 >= bb AND b@1 <= bb AND struct(a@0, b@1) IN (SET) ([{c0:ab,c1:bb}]) WHEN 8 THEN a@0 >= aa AND a@0 <= aa AND b@1 >= ba AND b@1 <= ba AND struct(a@0, b@1) IN (SET) ([{c0:aa,c1:ba}]) ELSE false END ] + - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, e], file_type=test, pushdown_supported=true, predicate=Optional(DynamicFilter [ CASE hash_repartition % 12 WHEN 5 THEN a@0 >= ab AND a@0 <= ab AND b@1 >= bb AND b@1 <= bb AND struct(a@0, b@1) IN (SET) ([{c0:ab,c1:bb}]) WHEN 8 THEN a@0 >= aa AND a@0 <= aa AND b@1 >= ba AND b@1 <= ba AND struct(a@0, b@1) IN (SET) ([{c0:aa,c1:ba}]) ELSE false END ]) " ); @@ -1231,7 +1231,7 @@ async fn test_hashjoin_dynamic_filter_pushdown_collect_left() { // expect the predicate to be pushed down into the probe side DataSource insta::assert_snapshot!( OptimizationTest::new(Arc::clone(&plan), FilterPushdown::new_post_optimization(), true), - @r" + @" OptimizationTest: input: - SortExec: expr=[a@0 DESC NULLS LAST], preserve_partitioning=[false] @@ -1247,7 +1247,7 @@ async fn test_hashjoin_dynamic_filter_pushdown_collect_left() { - HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(a@0, a@0), (b@1, b@1)] - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true - RepartitionExec: partitioning=Hash([a@0, b@1], 12), input_partitions=1 - - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, e], file_type=test, pushdown_supported=true, predicate=DynamicFilter [ empty ] + - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, e], file_type=test, pushdown_supported=true, predicate=Optional(DynamicFilter [ empty ]) " ); @@ -1273,13 +1273,13 @@ async fn test_hashjoin_dynamic_filter_pushdown_collect_left() { // Now check what our filter looks like insta::assert_snapshot!( format!("{}", format_plan_for_test(&plan)), - @r" + @" - SortExec: expr=[a@0 DESC NULLS LAST], preserve_partitioning=[false] - CoalescePartitionsExec - HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(a@0, a@0), (b@1, b@1)] - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true - RepartitionExec: partitioning=Hash([a@0, b@1], 12), input_partitions=1 - - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, e], file_type=test, pushdown_supported=true, 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}]) ] + - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b, e], file_type=test, pushdown_supported=true, predicate=Optional(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}]) ]) " ); @@ -2404,12 +2404,12 @@ async fn test_hashjoin_dynamic_filter_all_partitions_empty() { insta::assert_snapshot!( format_plan_for_test(&plan), - @r" + @" - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, a@0), (b@1, b@1)] - RepartitionExec: partitioning=Hash([a@0, b@1], 4), input_partitions=1 - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b], file_type=test, pushdown_supported=true - RepartitionExec: partitioning=Hash([a@0, b@1], 4), input_partitions=1 - - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b], file_type=test, pushdown_supported=true, predicate=DynamicFilter [ empty ] + - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b], file_type=test, pushdown_supported=true, predicate=Optional(DynamicFilter [ empty ]) " ); @@ -2429,12 +2429,12 @@ async fn test_hashjoin_dynamic_filter_all_partitions_empty() { // Test that filters are pushed down correctly to each side of the join insta::assert_snapshot!( format_plan_for_test(&plan), - @r" + @" - HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, a@0), (b@1, b@1)] - RepartitionExec: partitioning=Hash([a@0, b@1], 4), input_partitions=1 - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b], file_type=test, pushdown_supported=true - RepartitionExec: partitioning=Hash([a@0, b@1], 4), input_partitions=1 - - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b], file_type=test, pushdown_supported=true, predicate=DynamicFilter [ false ] + - DataSourceExec: file_groups={1 group: [[test.parquet]]}, projection=[a, b], file_type=test, pushdown_supported=true, predicate=Optional(DynamicFilter [ false ]) " ); } @@ -2954,18 +2954,28 @@ async fn test_discover_dynamic_filters_via_expressions_api() { use datafusion_physical_expr::expressions::DynamicFilterPhysicalExpr; use datafusion_physical_plan::joins::{HashJoinExec, PartitionMode}; + fn count_in_expr(expr: &dyn PhysicalExpr) -> usize { + let mut count = 0; + if expr.downcast_ref::().is_some() { + count += 1; + } + for child in expr.children() { + count += count_in_expr(child.as_ref()); + } + count + } + fn count_dynamic_filters(plan: &Arc) -> usize { let mut count = 0; - // Check expressions from this node using apply_expressions + // Check expressions from this node using apply_expressions. + // Walk each expression subtree to find DynamicFilterPhysicalExpr even + // when wrapped (e.g. by OptionalFilterPhysicalExpr). let _ = plan.apply_expressions(&mut |expr| { - if let Some(_df) = expr.downcast_ref::() { - count += 1; - } + count += count_in_expr(expr); Ok(TreeNodeRecursion::Continue) }); - // Recursively visit children for child in plan.children() { count += count_dynamic_filters(child); } diff --git a/datafusion/core/tests/sql/explain_analyze.rs b/datafusion/core/tests/sql/explain_analyze.rs index 8ab0d150a7272..dca95e111fd36 100644 --- a/datafusion/core/tests/sql/explain_analyze.rs +++ b/datafusion/core/tests/sql/explain_analyze.rs @@ -878,7 +878,7 @@ async fn parquet_explain_analyze() { .to_string(); // should contain aggregated stats - assert_contains!(&formatted, "output_rows=8"); + assert_contains!(&formatted, "output_rows=5"); assert_contains!( &formatted, "row_groups_pruned_bloom_filter=1 total \u{2192} 1 matched" diff --git a/datafusion/datasource-parquet/benches/parquet_nested_filter_pushdown.rs b/datafusion/datasource-parquet/benches/parquet_nested_filter_pushdown.rs index 02137b5a1d288..cd3d2da56be53 100644 --- a/datafusion/datasource-parquet/benches/parquet_nested_filter_pushdown.rs +++ b/datafusion/datasource-parquet/benches/parquet_nested_filter_pushdown.rs @@ -24,6 +24,7 @@ use arrow::array::{ use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; use criterion::{Criterion, Throughput, criterion_group, criterion_main}; use datafusion_common::ScalarValue; +use datafusion_datasource_parquet::selectivity::SelectivityTracker; use datafusion_datasource_parquet::{ParquetFileMetrics, build_row_filter}; use datafusion_expr::{Expr, col}; use datafusion_functions_nested::expr_fn::array_has; @@ -115,9 +116,17 @@ fn scan_with_predicate( let file_metrics = ParquetFileMetrics::new(0, &path.display().to_string(), &metrics); let builder = if pushdown { - if let Some(row_filter) = - build_row_filter(predicate, file_schema, &metadata, false, &file_metrics)? - { + let tracker = Arc::new(SelectivityTracker::new()); + let filters = vec![(0usize, Arc::clone(predicate))]; + let (maybe_row_filter, _unbuildable) = build_row_filter( + &filters, + file_schema, + &metadata, + 0, + &tracker, + &file_metrics, + )?; + if let Some(row_filter) = maybe_row_filter { builder.with_row_filter(row_filter) } else { builder diff --git a/datafusion/datasource-parquet/benches/parquet_struct_filter_pushdown.rs b/datafusion/datasource-parquet/benches/parquet_struct_filter_pushdown.rs index b52408d4222d8..cfc326d84fb6b 100644 --- a/datafusion/datasource-parquet/benches/parquet_struct_filter_pushdown.rs +++ b/datafusion/datasource-parquet/benches/parquet_struct_filter_pushdown.rs @@ -50,6 +50,7 @@ use arrow::array::{BooleanArray, Int32Array, RecordBatch, StringBuilder, StructA use arrow::datatypes::{DataType, Field, Fields, Schema, SchemaRef}; use criterion::{Criterion, Throughput, criterion_group, criterion_main}; use datafusion_common::ScalarValue; +use datafusion_datasource_parquet::selectivity::SelectivityTracker; use datafusion_datasource_parquet::{ParquetFileMetrics, build_row_filter}; use datafusion_expr::{Expr, col}; use datafusion_physical_expr::planner::logical2physical; @@ -210,9 +211,17 @@ fn scan( let mut filter_applied = false; let builder = if pushdown { - if let Some(row_filter) = - build_row_filter(predicate, file_schema, &metadata, false, &file_metrics)? - { + let tracker = Arc::new(SelectivityTracker::new()); + let filters = vec![(0usize, Arc::clone(predicate))]; + let (maybe_row_filter, _unbuildable) = build_row_filter( + &filters, + file_schema, + &metadata, + 0, + &tracker, + &file_metrics, + )?; + if let Some(row_filter) = maybe_row_filter { filter_applied = true; builder.with_row_filter(row_filter) } else { diff --git a/datafusion/datasource-parquet/src/file_format.rs b/datafusion/datasource-parquet/src/file_format.rs index 7dda7b1b12811..2feb7814c5733 100644 --- a/datafusion/datasource-parquet/src/file_format.rs +++ b/datafusion/datasource-parquet/src/file_format.rs @@ -501,6 +501,12 @@ impl FileFormat for ParquetFormat { ) -> Result> { let mut metadata_size_hint = None; + let filter_pushdown_min_bytes_per_sec = state + .config_options() + .execution + .parquet + .filter_pushdown_min_bytes_per_sec; + if let Some(metadata) = self.metadata_size_hint() { metadata_size_hint = Some(metadata); } @@ -510,7 +516,10 @@ impl FileFormat for ParquetFormat { .downcast_ref::() .cloned() .ok_or_else(|| internal_datafusion_err!("Expected ParquetSource"))?; - source = source.with_table_parquet_options(self.options.clone()); + let mut options = self.options.clone(); + options.global.filter_pushdown_min_bytes_per_sec = + filter_pushdown_min_bytes_per_sec; + source = source.with_table_parquet_options(options); // Use the CachedParquetFileReaderFactory let metadata_cache = state.runtime_env().cache_manager.get_file_metadata_cache(); diff --git a/datafusion/datasource-parquet/src/metrics.rs b/datafusion/datasource-parquet/src/metrics.rs index 8eb5912b919da..ece86af498f82 100644 --- a/datafusion/datasource-parquet/src/metrics.rs +++ b/datafusion/datasource-parquet/src/metrics.rs @@ -91,6 +91,8 @@ pub struct ParquetFileMetrics { /// number of rows that were stored in the cache after evaluating predicates /// reused for the output. pub predicate_cache_records: Gauge, + //// Time spent applying filters + pub filter_apply_time: Time, } impl ParquetFileMetrics { @@ -192,6 +194,10 @@ impl ParquetFileMetrics { .with_category(MetricCategory::Rows) .gauge("predicate_cache_records", partition); + let filter_apply_time = MetricBuilder::new(metrics) + .with_new_label("filename", filename.to_string()) + .subset_time("filter_apply_time", partition); + Self { files_ranges_pruned_statistics, predicate_evaluation_errors, @@ -211,6 +217,7 @@ impl ParquetFileMetrics { scan_efficiency_ratio, predicate_cache_inner_records, predicate_cache_records, + filter_apply_time, } } } diff --git a/datafusion/datasource-parquet/src/mod.rs b/datafusion/datasource-parquet/src/mod.rs index 9a907f4118a86..eb81383a93ca7 100644 --- a/datafusion/datasource-parquet/src/mod.rs +++ b/datafusion/datasource-parquet/src/mod.rs @@ -33,6 +33,7 @@ mod page_filter; mod reader; mod row_filter; mod row_group_filter; +pub mod selectivity; mod sort; pub mod source; mod supported_predicates; diff --git a/datafusion/datasource-parquet/src/opener.rs b/datafusion/datasource-parquet/src/opener.rs index bad1c684b47f5..854c708f06ddc 100644 --- a/datafusion/datasource-parquet/src/opener.rs +++ b/datafusion/datasource-parquet/src/opener.rs @@ -27,6 +27,7 @@ use crate::{ use arrow::array::{RecordBatch, RecordBatchOptions}; use arrow::datatypes::DataType; use datafusion_datasource::morsel::{Morsel, MorselPlan, MorselPlanner, Morselizer}; +use datafusion_physical_expr::conjunction; use datafusion_physical_expr::projection::{ProjectionExprs, Projector}; use datafusion_physical_expr::utils::reassign_expr_columns; use datafusion_physical_expr_adapter::replace_columns_with_literals; @@ -94,8 +95,13 @@ pub(super) struct ParquetMorselizer { pub(crate) limit: Option, /// If should keep the output rows in order pub preserve_order: bool, - /// Optional predicate to apply during the scan - pub predicate: Option>, + /// Optional predicate conjuncts to apply during the scan. Each conjunct + /// carries a stable `FilterId` used by the selectivity tracker. + pub predicate_conjuncts: + Option)>>, + /// Shared adaptive selectivity tracker used to decide per-filter placement + /// (row-level vs post-scan) at runtime. + pub selectivity_tracker: Arc, /// Table schema, including partition columns. pub table_schema: TableSchema, /// Optional hint for how large the initial request to read parquet metadata @@ -108,8 +114,6 @@ pub(super) struct ParquetMorselizer { /// Should the filters be evaluated during the parquet scan using /// [`DataFusionArrowPredicate`](row_filter::DatafusionArrowPredicate)? pub pushdown_filters: bool, - /// Should the filters be reordered to optimize the scan? - pub reorder_filters: bool, /// Should we force the reader to use RowSelections for filtering pub force_filter_selections: bool, /// Should the page index be read from parquet files, if present, to skip @@ -274,7 +278,9 @@ struct PreparedParquetOpen { output_schema: SchemaRef, projection: ProjectionExprs, predicate: Option>, - reorder_predicates: bool, + predicate_conjuncts: + Option)>>, + selectivity_tracker: Arc, pushdown_filters: bool, force_filter_selections: bool, enable_page_index: bool, @@ -597,15 +603,24 @@ impl ParquetMorselizer { )); let mut projection = self.projection.clone(); - let mut predicate = self.predicate.clone(); + let mut predicate_conjuncts = self.predicate_conjuncts.clone(); if !literal_columns.is_empty() { projection = projection.try_map_exprs(|expr| { replace_columns_with_literals(Arc::clone(&expr), &literal_columns) })?; - predicate = predicate - .map(|p| replace_columns_with_literals(p, &literal_columns)) - .transpose()?; + if let Some(ref mut conjuncts) = predicate_conjuncts { + for (_id, expr) in conjuncts.iter_mut() { + *expr = replace_columns_with_literals( + Arc::clone(expr), + &literal_columns, + )?; + } + } } + // Build a single combined predicate for file-level pruning. + let predicate: Option> = predicate_conjuncts + .as_ref() + .map(|c| conjunction(c.iter().map(|(_, e)| Arc::clone(e)))); let predicate_creation_errors = MetricBuilder::new(&self.metrics) .with_category(MetricCategory::Rows) @@ -643,7 +658,8 @@ impl ParquetMorselizer { output_schema, projection, predicate, - reorder_predicates: self.reorder_filters, + predicate_conjuncts, + selectivity_tracker: Arc::clone(&self.selectivity_tracker), pushdown_filters: self.pushdown_filters, force_filter_selections: self.force_filter_selections, enable_page_index: self.enable_page_index, @@ -819,6 +835,31 @@ impl MetadataLoadedParquetOpen { prepared.projection = prepared .projection .try_map_exprs(|p| simplifier.simplify(rewriter.rewrite(p)?))?; + // Adapt each per-filter conjunct individually, keeping + // FilterIds stable so the adaptive selectivity tracker can + // correlate runtime stats across files. Skip conjuncts that + // simplify to the literal `TRUE` — they add nothing to the + // predicate and would bloat the row-filter / post-scan buckets. + // Both the row-filter and post-scan paths downstream consume + // these in physical-file-schema space: the post-scan path + // widens the decoder's projection mask to include filter + // columns and evaluates against the pre-project wide batch, + // so logical-schema expressions are not required there. + if let Some(ref mut conjuncts) = prepared.predicate_conjuncts { + let mut adapted = Vec::with_capacity(conjuncts.len()); + for (id, expr) in conjuncts.drain(..) { + let rewritten = rewriter.rewrite(expr)?; + let simplified = simplifier.simplify(rewritten)?; + if let Some(lit) = simplified + .downcast_ref::( + ) && let ScalarValue::Boolean(Some(true)) = lit.value() + { + continue; + } + adapted.push((id, simplified)); + } + *conjuncts = adapted; + } } prepared.physical_file_schema = Arc::clone(&physical_file_schema); @@ -1075,25 +1116,87 @@ impl RowGroupsPrunedParquetOpen { let file_metadata = Arc::clone(reader_metadata.metadata()); let rg_metadata = file_metadata.row_groups(); - // Filter pushdown: evaluate predicates during scan - let row_filter = if let Some(predicate) = prepared - .pushdown_filters - .then_some(prepared.predicate.clone()) - .flatten() + // Adaptive filter placement. + // + // Ask the shared `SelectivityTracker` to split our predicate + // conjuncts (already adapted to `physical_file_schema` in the + // `PrepareFilters` state) into two buckets based on measured + // effectiveness across prior files: + // + // - `row_filters` — evaluated inside the Parquet decoder via + // `ArrowPredicate`s, enabling late-materialization savings. + // - `post_scan` — evaluated against the decoded wide batch just + // before the projector strips it down. Any filter-only columns + // the post-scan filter references are added to the decoder's + // projection mask below, so the filter can always be applied. + // + // For the first file we encounter, `partition_filters` uses a + // cheap byte-ratio heuristic (filter-column bytes / projection- + // column bytes) for initial placement. Subsequent files refine + // the placement using Welford statistics reported from the + // row-filter path (`DatafusionArrowPredicate::evaluate`) and + // the post-scan path (`apply_post_scan_filters_with_stats`). + let projection_column_indices: Vec = { + let mut idxs: Vec = prepared + .projection + .expr_iter() + .flat_map(|expr| { + datafusion_physical_expr::utils::collect_columns(&expr) + .into_iter() + .map(|c| c.index()) + .collect::>() + }) + .collect(); + idxs.sort_unstable(); + idxs.dedup(); + idxs + }; + let projection_compressed_bytes = row_filter::total_compressed_bytes( + &projection_column_indices, + file_metadata.as_ref(), + ); + + let (row_filter_conjuncts, mut post_scan_conjuncts) = if prepared.pushdown_filters + && let Some(conjuncts) = prepared.predicate_conjuncts.clone() + && !conjuncts.is_empty() { - let row_filter = row_filter::build_row_filter( - &predicate, - &prepared.physical_file_schema, + let partitioned = prepared.selectivity_tracker.partition_filters( + conjuncts, + projection_compressed_bytes, file_metadata.as_ref(), - prepared.reorder_predicates, - &prepared.file_metrics, ); + (partitioned.row_filters, partitioned.post_scan) + } else { + (Vec::new(), Vec::new()) + }; - match row_filter { - Ok(Some(filter)) => Some(filter), - Ok(None) => None, + // Build row-level `ArrowPredicate`s for the row_filters bucket. + // Any conjunct that `build_row_filter` reports as `unbuildable` + // falls through to the post-scan bucket so we never silently drop + // a filter — dropping would relax the user's predicate and return + // wrong results. Both buckets are already in `physical_file_schema` + // space (adapted in `PrepareFilters`), so `unbuildable` is mixed + // straight back into `post_scan_conjuncts` without further + // rewriting. + let row_filter = if !row_filter_conjuncts.is_empty() { + match row_filter::build_row_filter( + &row_filter_conjuncts, + &prepared.physical_file_schema, + file_metadata.as_ref(), + projection_compressed_bytes, + &prepared.selectivity_tracker, + &prepared.file_metrics, + ) { + Ok((row_filter, unbuildable)) => { + post_scan_conjuncts.extend(unbuildable); + row_filter + } Err(e) => { - debug!("Ignoring error building row filter for '{predicate:?}': {e}"); + debug!( + "Error building row filter for {row_filter_conjuncts:?}: {e}; \ + falling all row-filter candidates through to post-scan" + ); + post_scan_conjuncts.extend(row_filter_conjuncts); None } } @@ -1101,7 +1204,46 @@ impl RowGroupsPrunedParquetOpen { None }; - // Prune by limit if limit is set and limit order is not sensitive + // Pre-compute the per-filter "other-bytes-per-row" quantity — + // the bytes of projection columns *not* referenced by this filter, + // amortised across rows. This is what late materialization saves + // per pruned row and is the same cost metric the row-filter path + // reports to the tracker, so promote/demote rankings compare + // filters on a single common axis. + let total_rows: i64 = file_metadata + .row_groups() + .iter() + .map(|rg| rg.num_rows()) + .sum(); + let post_scan_other_bytes_per_row: Vec = post_scan_conjuncts + .iter() + .map(|(_, expr)| { + let filter_cols: Vec = + datafusion_physical_expr::utils::collect_columns(expr) + .iter() + .map(|c| c.index()) + .collect(); + let filter_compressed = row_filter::total_compressed_bytes( + &filter_cols, + file_metadata.as_ref(), + ); + if total_rows > 0 { + projection_compressed_bytes.saturating_sub(filter_compressed) as f64 + / total_rows as f64 + } else { + 0.0 + } + }) + .collect(); + + // Prune by limit if limit is set and limit order is not sensitive. + // + // If we have post-scan filters we still apply row-group limit pruning + // — the decoder may still stop early — but we purposely skip + // `decoder_builder.with_limit(limit)` below, and instead apply the + // limit downstream of the post-scan filter wrapper. Applying the + // limit at the decoder when a post-scan filter can still drop rows + // would return fewer rows than requested. if let (Some(limit), false) = (prepared.limit, prepared.preserve_order) { row_groups.prune_by_limit(limit, rg_metadata, &prepared.file_metrics); } @@ -1133,8 +1275,21 @@ impl RowGroupsPrunedParquetOpen { } let arrow_reader_metrics = ArrowReaderMetrics::enabled(); + // Build the decoder's projection/read plan over the UNION of the + // user projection and every post-scan filter's column references. + // This is what lets the post-scan path evaluate filters that touch + // columns which would otherwise be projected away: the decoder + // produces a "wide" batch containing both the user-visible + // projection *and* the extra columns the filters need, and the + // projector below strips the filter-only columns back out after + // the filter has had a chance to read them. + let read_plan_exprs: Vec> = prepared + .projection + .expr_iter() + .chain(post_scan_conjuncts.iter().map(|(_, expr)| Arc::clone(expr))) + .collect(); let read_plan = build_projection_read_plan( - prepared.projection.expr_iter(), + read_plan_exprs, &prepared.physical_file_schema, reader_metadata.parquet_schema(), ); @@ -1157,7 +1312,12 @@ impl RowGroupsPrunedParquetOpen { } decoder_builder = decoder_builder.with_row_groups(prepared_plan.row_group_indexes); - if let Some(limit) = prepared.limit { + // Only push the limit into the decoder when there are no post-scan + // filters. Otherwise the decoder would stop before the filters + // have had a chance to drop rows, returning too few matches. + if let Some(limit) = prepared.limit + && post_scan_conjuncts.is_empty() + { decoder_builder = decoder_builder.with_limit(limit); } if let Some(max_predicate_cache_size) = prepared.max_predicate_cache_size { @@ -1171,22 +1331,37 @@ impl RowGroupsPrunedParquetOpen { prepared.file_metrics.predicate_cache_inner_records.clone(); let predicate_cache_records = prepared.file_metrics.predicate_cache_records.clone(); + let filter_apply_time = prepared.file_metrics.filter_apply_time.clone(); // Check if we need to replace the schema to handle things like differing nullability or metadata. // See note below about file vs. output schema. let stream_schema = read_plan.projected_schema; let replace_schema = stream_schema != prepared.output_schema; - // Rebase column indices to match the narrowed stream schema. - // The projection expressions have indices based on physical_file_schema, - // but the stream only contains the columns selected by the ProjectionMask. + // Rebase column indices to match the (possibly widened) stream + // schema. Both the projection expressions (which only reference + // the user projection) and the post-scan filter expressions (which + // may reference additional columns) need this rebase — both + // originally used indices into `physical_file_schema`, but the + // decoder only decodes the subset of columns in the read plan's + // projection mask. let projection = prepared .projection .try_map_exprs(|expr| reassign_expr_columns(expr, &stream_schema))?; + let post_scan_filters: Vec<( + crate::selectivity::FilterId, + Arc, + )> = post_scan_conjuncts + .into_iter() + .map(|(id, expr)| { + reassign_expr_columns(expr, &stream_schema).map(|e| (id, e)) + }) + .collect::>()?; let projector = projection.make_projector(&stream_schema)?; let output_schema = Arc::clone(&prepared.output_schema); let files_ranges_pruned_statistics = prepared.file_metrics.files_ranges_pruned_statistics.clone(); + let stream = futures::stream::unfold( PushDecoderStreamState { decoder, @@ -1198,14 +1373,18 @@ impl RowGroupsPrunedParquetOpen { predicate_cache_inner_records, predicate_cache_records, baseline_metrics: prepared.baseline_metrics, + post_scan_filters, + selectivity_tracker: Arc::clone(&prepared.selectivity_tracker), + post_scan_other_bytes_per_row, + filter_apply_time, }, |state| async move { state.transition().await }, ) .fuse(); + let stream = stream.boxed(); // Wrap the stream so a dynamic filter can stop the file scan early. if let Some(file_pruner) = prepared.file_pruner { - let stream = stream.boxed(); Ok(EarlyStoppingStream::new( stream, file_pruner, @@ -1213,9 +1392,77 @@ impl RowGroupsPrunedParquetOpen { ) .boxed()) } else { - Ok(stream.boxed()) + Ok(stream) + } + } +} + +/// Apply post-scan filters to a single decoded `RecordBatch`, reporting +/// per-filter selectivity and evaluation cost to the shared +/// [`crate::selectivity::SelectivityTracker`] so the adaptive system can +/// promote filters to row-level (`RowFilter`) in subsequent files. +/// +/// Called by [`PushDecoderStreamState::transition`] against the +/// pre-projection "wide" batch — i.e. a batch containing the user +/// projection columns PLUS any columns the post-scan filters reference +/// that weren't already in the projection. This lets the filters evaluate +/// even against columns that would normally be projected away; the +/// projector (built from the original projection expressions) strips the +/// filter-only columns back out immediately after filtering. +/// +/// `other_bytes_per_row` is the per-filter bytes-per-row contribution of +/// the non-filter projection columns (i.e. the bytes late materialization +/// would save). The row-filter path reports the same quantity via +/// `DatafusionArrowPredicate::other_projected_bytes_per_row`, so the +/// tracker can rank filters on a single common axis. +fn apply_post_scan_filters_with_stats( + batch: RecordBatch, + filters: &[(crate::selectivity::FilterId, Arc)], + other_bytes_per_row: &[f64], + tracker: &crate::selectivity::SelectivityTracker, +) -> Result { + use arrow::array::BooleanArray; + use arrow::compute::{and, filter_record_batch}; + use datafusion_common::cast::as_boolean_array; + + if batch.num_rows() == 0 { + return Ok(batch); + } + + let input_rows = batch.num_rows() as u64; + let mut combined_mask: Option = None; + + for (i, (id, expr)) in filters.iter().enumerate() { + // Mid-stream drop, mirror of `DatafusionArrowPredicate::evaluate`. + // Set by the tracker on `OptionalFilterPhysicalExpr` whose CI + // upper bound has fallen below `min_bytes_per_sec`; correctness is + // preserved because the originating join independently enforces + // the predicate. We do not update the tracker for a skipped batch. + if tracker.is_filter_skipped(*id) { + continue; + } + + let start = datafusion_common::instant::Instant::now(); + let result = expr.evaluate(&batch)?.into_array(batch.num_rows())?; + let bool_arr = as_boolean_array(result.as_ref())?; + let nanos = start.elapsed().as_nanos() as u64; + let num_matched = bool_arr.true_count() as u64; + + let other_bytes = (other_bytes_per_row[i] * input_rows as f64) as u64; + tracker.update(*id, num_matched, input_rows, nanos, other_bytes); + + if num_matched < input_rows { + combined_mask = Some(match combined_mask { + Some(prev) => and(&prev, bool_arr)?, + None => bool_arr.clone(), + }); } } + + match combined_mask { + Some(mask) => Ok(filter_record_batch(&batch, &mask)?), + None => Ok(batch), + } } /// State for a stream that decodes a single Parquet file using a push-based decoder. @@ -1234,6 +1481,23 @@ struct PushDecoderStreamState { predicate_cache_inner_records: Gauge, predicate_cache_records: Gauge, baseline_metrics: BaselineMetrics, + /// Post-scan filters expressed against `stream_schema` (the wide + /// schema the decoder yields, which includes filter-only columns added + /// to the projection mask specifically so post-scan can evaluate + /// against them). Applied to each decoded batch before `project_batch` + /// narrows the batch down to the user-requested projection; the + /// projector naturally drops the filter-only columns after filtering. + post_scan_filters: Vec<(crate::selectivity::FilterId, Arc)>, + /// Shared adaptive tracker fed per-filter per-batch stats from + /// `apply_post_scan_filters_with_stats`, mirroring what the row-filter + /// `DatafusionArrowPredicate::evaluate` path reports. + selectivity_tracker: Arc, + /// Pre-computed "bytes per row of the non-filter projection" for each + /// post-scan filter (same cost metric the row-filter path reports, so + /// promote/demote decisions compare apples to apples). + post_scan_other_bytes_per_row: Vec, + /// Elapsed-time metric for post-scan filter application. + filter_apply_time: datafusion_physical_plan::metrics::Time, } impl PushDecoderStreamState { @@ -1271,11 +1535,45 @@ impl PushDecoderStreamState { Ok(DecodeResult::Data(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)); + // Apply post-scan filters against the pre-projection + // (wide) batch so any filter-only columns the decoder + // decoded for us are still present. After filtering, + // the projector strips them back out. + let filtered = if self.post_scan_filters.is_empty() { + Ok(batch) + } else { + let start = datafusion_common::instant::Instant::now(); + let out = apply_post_scan_filters_with_stats( + batch, + &self.post_scan_filters, + &self.post_scan_other_bytes_per_row, + &self.selectivity_tracker, + ); + self.filter_apply_time.add_elapsed(start); + out + }; + match filtered { + // Post-scan may filter every row in a batch. Skip + // fully-empty filtered batches and loop to the + // next decoded batch — emitting an empty batch + // would noisily turn into one visible output + // batch downstream. + Ok(b) if b.num_rows() == 0 => { + timer.stop(); + continue; + } + Ok(b) => { + let result = self.project_batch(&b); + timer.stop(); + drop(timer); + return Some((result, self)); + } + Err(e) => { + timer.stop(); + drop(timer); + return Some((Err(e), self)); + } + } } Ok(DecodeResult::Finished) => { return None; @@ -1667,7 +1965,6 @@ mod test { metadata_size_hint: Option, metrics: ExecutionPlanMetricsSet, pushdown_filters: bool, - reorder_filters: bool, force_filter_selections: bool, enable_page_index: bool, enable_bloom_filter: bool, @@ -1693,7 +1990,6 @@ mod test { metadata_size_hint: None, metrics: ExecutionPlanMetricsSet::new(), pushdown_filters: false, - reorder_filters: false, force_filter_selections: false, enable_page_index: false, enable_bloom_filter: false, @@ -1741,12 +2037,6 @@ mod test { self } - /// Enable filter reordering. - fn with_reorder_filters(mut self, enable: bool) -> Self { - self.reorder_filters = enable; - self - } - /// Enable row group stats pruning. fn with_row_group_stats_pruning(mut self, enable: bool) -> Self { self.enable_row_group_stats_pruning = enable; @@ -1789,13 +2079,26 @@ mod test { ProjectionExprs::from_indices(&all_indices, &file_schema) }; + use datafusion_physical_expr::split_conjunction; + let predicate_conjuncts: Option< + Vec<(crate::selectivity::FilterId, Arc)>, + > = self.predicate.as_ref().map(|p| { + split_conjunction(p) + .into_iter() + .enumerate() + .map(|(id, expr)| (id, Arc::clone(expr))) + .collect() + }); + let selectivity_tracker = + Arc::new(crate::selectivity::SelectivityTracker::new()); ParquetMorselizer { partition_index: self.partition_index, projection, batch_size: self.batch_size, limit: self.limit, preserve_order: self.preserve_order, - predicate: self.predicate, + predicate_conjuncts, + selectivity_tracker, table_schema, metadata_size_hint: self.metadata_size_hint, metrics: self.metrics, @@ -1803,7 +2106,6 @@ mod test { DefaultParquetFileReaderFactory::new(store), ), pushdown_filters: self.pushdown_filters, - reorder_filters: self.reorder_filters, force_filter_selections: self.force_filter_selections, enable_page_index: self.enable_page_index, enable_bloom_filter: self.enable_bloom_filter, @@ -2241,7 +2543,6 @@ mod test { .with_projection_indices(&[0]) .with_predicate(predicate) .with_pushdown_filters(true) // note that this is true! - .with_reorder_filters(true) .build() }; diff --git a/datafusion/datasource-parquet/src/row_filter.rs b/datafusion/datasource-parquet/src/row_filter.rs index c5c372055826b..cdb1d92bc7156 100644 --- a/datafusion/datasource-parquet/src/row_filter.rs +++ b/datafusion/datasource-parquet/src/row_filter.rs @@ -65,6 +65,7 @@ //! - `WHERE s['value'] > 5` — pushed down (accesses a primitive leaf) //! - `WHERE s IS NOT NULL` — not pushed down (references the whole struct) +use log::debug; use std::collections::BTreeSet; use std::sync::Arc; @@ -81,10 +82,10 @@ use parquet::schema::types::SchemaDescriptor; use datafusion_common::Result; use datafusion_common::cast::as_boolean_array; use datafusion_common::tree_node::{TreeNode, TreeNodeRecursion, TreeNodeVisitor}; +use datafusion_physical_expr::PhysicalExpr; use datafusion_physical_expr::ScalarFunctionExpr; use datafusion_physical_expr::expressions::{Column, Literal}; use datafusion_physical_expr::utils::{collect_columns, reassign_expr_columns}; -use datafusion_physical_expr::{PhysicalExpr, split_conjunction}; use datafusion_physical_plan::metrics; @@ -119,18 +120,49 @@ pub(crate) struct DatafusionArrowPredicate { rows_matched: metrics::Count, /// how long was spent evaluating this predicate time: metrics::Time, + /// Stable id used by the adaptive selectivity tracker to key per-filter + /// statistics across files. + filter_id: crate::selectivity::FilterId, + /// Shared handle to the adaptive selectivity tracker. Per-batch stats + /// are reported through `update()` after each `evaluate()` call. + tracker: Arc, + /// Estimated *late-materialization savings* per row for this filter: + /// the compressed bytes of projection columns that the filter does + /// NOT reference, amortised across the file's rows. When a pruned + /// row is dropped by the filter, these are the bytes the reader + /// avoids decoding further along the pipeline — the quantity the + /// adaptive tracker needs in order to rank filters by "cost avoided + /// per unit evaluation time". This MUST match the metric the + /// post-scan path reports in `apply_post_scan_filters_with_stats` + /// (see `opener.rs::post_scan_other_bytes_per_row`); if the two + /// paths disagreed, the tracker would rank row-filter and post-scan + /// candidates on incomparable axes and mis-promote or mis-demote. + other_projected_bytes_per_row: f64, + /// Mid-stream "drop" flag, shared with the + /// [`crate::selectivity::SelectivityTracker`]. The tracker flips this + /// when an `OptionalFilterPhysicalExpr` proves CPU-dominated and + /// ineffective; once set, [`Self::evaluate`] returns an all-true mask + /// without invoking `physical_expr`. Filter columns are still decoded + /// (the parquet decoder cannot be reconfigured mid-scan), so this only + /// reclaims CPU, not I/O. Flagged only for filters known to be + /// optional, so correctness is preserved by the join itself. + skip_flag: Arc, } impl DatafusionArrowPredicate { - /// Create a new `DatafusionArrowPredicate` from a `FilterCandidate` + /// Create a new `DatafusionArrowPredicate` from a `FilterCandidate`. pub fn try_new( candidate: FilterCandidate, rows_pruned: metrics::Count, rows_matched: metrics::Count, time: metrics::Time, + filter_id: crate::selectivity::FilterId, + tracker: Arc, + other_projected_bytes_per_row: f64, ) -> Result { let physical_expr = reassign_expr_columns(candidate.expr, &candidate.read_plan.projected_schema)?; + let skip_flag = tracker.skip_flag(filter_id); Ok(Self { physical_expr, @@ -138,6 +170,10 @@ impl DatafusionArrowPredicate { rows_pruned, rows_matched, time, + filter_id, + tracker, + other_projected_bytes_per_row, + skip_flag, }) } } @@ -148,10 +184,27 @@ impl ArrowPredicate for DatafusionArrowPredicate { } fn evaluate(&mut self, batch: RecordBatch) -> ArrowResult { + // Mid-stream drop: the tracker has decided this optional filter is + // pulling its weight no longer. Return an all-true mask to bypass + // expression evaluation entirely. We still bump `rows_matched` so + // the per-predicate count stays consistent with input rows; the + // tracker is intentionally NOT updated for skipped batches because + // (a) we have nothing meaningful to report and (b) flooding it + // with zero-cost samples would mask the underlying effectiveness + // signal if the flag is ever cleared. + if self.skip_flag.load(std::sync::atomic::Ordering::Acquire) { + let rows_in_batch = batch.num_rows(); + self.rows_matched.add(rows_in_batch); + return Ok(BooleanArray::from(vec![true; rows_in_batch])); + } + // scoped timer updates on drop let mut timer = self.time.timer(); + let start_nanos = datafusion_common::instant::Instant::now(); - self.physical_expr + let rows_in_batch = batch.num_rows(); + let result = self + .physical_expr .evaluate(&batch) .and_then(|v| v.into_array(batch.num_rows())) .and_then(|array| { @@ -161,13 +214,38 @@ impl ArrowPredicate for DatafusionArrowPredicate { self.rows_pruned.add(num_pruned); self.rows_matched.add(num_matched); timer.stop(); - Ok(bool_arr) + Ok((bool_arr, num_matched)) }) .map_err(|e| { ArrowError::ComputeError(format!( "Error evaluating filter predicate: {e:?}" )) - }) + }); + + match result { + Ok((bool_arr, num_matched)) => { + let eval_nanos = start_nanos.elapsed().as_nanos() as u64; + // Report *late-materialization savings* (bytes of non-filter + // projection columns the decoder would have had to read for + // each pruned row), matching the post-scan path. This is the + // quantity the SelectivityTracker converts into the + // "bytes saved per second of evaluation time" effectiveness + // metric. Reporting the filter's own byte cost instead + // would invert the promote/demote rankings. + let batch_bytes = (rows_in_batch as f64 + * self.other_projected_bytes_per_row) + .round() as u64; + self.tracker.update( + self.filter_id, + num_matched as u64, + rows_in_batch as u64, + eval_nanos, + batch_bytes, + ); + Ok(bool_arr) + } + Err(e) => Err(e), + } } } @@ -991,95 +1069,167 @@ fn size_of_columns(columns: &[usize], metadata: &ParquetMetaData) -> Result)>; + +/// Build row-level filters for the row-filter partition chosen by the +/// adaptive selectivity tracker. /// -/// # Arguments -/// * `expr` - The filter predicate, already adapted to reference columns in `file_schema` -/// * `file_schema` - The Arrow schema of the parquet file (the result of converting -/// the parquet schema to Arrow, potentially with type coercions applied) -/// * `metadata` - Parquet file metadata used for cost estimation -/// * `reorder_predicates` - If true, reorder predicates to minimize I/O -/// * `file_metrics` - Metrics for tracking filter performance +/// Each input filter keeps its stable filter id so the resulting +/// `ArrowPredicate`s can report per-batch statistics back to the tracker on +/// each `evaluate()` call, driving future promote/demote decisions. /// -/// # Returns -/// * `Ok(Some(row_filter))` if the expression can be used as a RowFilter -/// * `Ok(None)` if the expression cannot be used as a RowFilter -/// * `Err(e)` if an error occurs while building the filter +/// Filters that cannot be represented as an `ArrowPredicate` (e.g. whole +/// struct references or other unsupported patterns) are returned in the +/// second element of the returned tuple so the opener can apply them +/// post-scan instead of silently dropping them. /// -/// Note: The returned `RowFilter` may not contain all conjuncts from the original -/// expression. Conjuncts that cannot be evaluated as an `ArrowPredicate` are ignored. +/// # Arguments +/// * `filters` — The candidate filters paired with their stable ids. Assumed +/// to already be adapted to reference columns in `file_schema`. +/// * `file_schema` — The Arrow schema of the parquet file. +/// * `metadata` — Parquet file metadata used for cost estimation. +/// * `projection_compressed_bytes` — Total compressed bytes the user +/// projection reads across the file. Used to derive the per-filter +/// *late-materialization savings* reported to the tracker, so that +/// row-filter and post-scan candidates are ranked on a single common +/// axis. +/// * `tracker` — Shared adaptive selectivity tracker. +/// * `file_metrics` — Metrics for tracking filter performance. /// -/// For example, if the expression is `a = 1 AND b = 2 AND c = 3` and `b = 2` -/// cannot be evaluated for some reason, the returned `RowFilter` will contain -/// only `a = 1` and `c = 3`. +/// # Returns +/// * `Ok((Some(row_filter), unbuildable))` when at least one filter could be +/// represented as a row-level predicate. +/// * `Ok((None, unbuildable))` when no filters could be represented as row +/// filters; all are returned in `unbuildable`. +/// * `Err(e)` if an error occurs while building the filter. pub fn build_row_filter( - expr: &Arc, + filters: &[(crate::selectivity::FilterId, Arc)], file_schema: &SchemaRef, metadata: &ParquetMetaData, - reorder_predicates: bool, + projection_compressed_bytes: usize, + tracker: &Arc, file_metrics: &ParquetFileMetrics, -) -> Result> { +) -> Result<(Option, UnbuildableFilters)> { let rows_pruned = &file_metrics.pushdown_rows_pruned; let rows_matched = &file_metrics.pushdown_rows_matched; let time = &file_metrics.row_pushdown_eval_time; - // Split into conjuncts: - // `a = 1 AND b = 2 AND c = 3` -> [`a = 1`, `b = 2`, `c = 3`] - let predicates = split_conjunction(expr); - - // Determine which conjuncts can be evaluated as ArrowPredicates, if any - let mut candidates: Vec = predicates - .into_iter() - .map(|expr| { - FilterCandidateBuilder::new(Arc::clone(expr), Arc::clone(file_schema)) - .build(metadata) - }) - .collect::, _>>()? - .into_iter() - .flatten() - .collect(); - - // no candidates - if candidates.is_empty() { - return Ok(None); + // Total rows in the file, used to amortise compressed-byte totals + // over rows. We floor at 1 so empty files don't divide by zero; any + // effectiveness contribution in that degenerate case is irrelevant + // because there are no batches to track anyway. + let total_rows: i64 = metadata.row_groups().iter().map(|rg| rg.num_rows()).sum(); + let total_rows_f = total_rows.max(1) as f64; + + // Try to build a candidate for each filter independently. Any filter + // that can't be represented as an `ArrowPredicate`, for *any* reason + // (the candidate builder returned `None`, the builder returned an + // `Err`, or the `DatafusionArrowPredicate` constructor failed below), + // falls through into `unbuildable` so the caller can apply it + // post-scan. Silently dropping any conjunct here would relax the + // user's predicate and return wrong results — see the + // `post_scan_conjuncts` fallthrough in + // `ParquetOpener::build_stream`. + let mut buildable: Vec<(crate::selectivity::FilterId, FilterCandidate)> = + Vec::with_capacity(filters.len()); + let mut unbuildable: UnbuildableFilters = Vec::new(); + for (id, expr) in filters { + match FilterCandidateBuilder::new(Arc::clone(expr), Arc::clone(file_schema)) + .build(metadata) + { + Ok(Some(c)) => buildable.push((*id, c)), + Ok(None) => unbuildable.push((*id, Arc::clone(expr))), + Err(e) => { + debug!( + "failed to build row-filter candidate for {id}: {e}; falling through to post-scan" + ); + unbuildable.push((*id, Arc::clone(expr))); + } + } } - if reorder_predicates { - candidates.sort_unstable_by_key(|c| c.required_bytes); + if buildable.is_empty() { + return Ok((None, unbuildable)); } // To avoid double-counting metrics when multiple predicates are used: - // - All predicates should count rows_pruned (cumulative pruned rows) - // - Only the last predicate should count rows_matched (final result) - // This ensures: rows_matched + rows_pruned = total rows processed - let total_candidates = candidates.len(); - - candidates - .into_iter() - .enumerate() - .map(|(idx, candidate)| { - let is_last = idx == total_candidates - 1; - - // All predicates share the pruned counter (cumulative) - let predicate_rows_pruned = rows_pruned.clone(); - - // Only the last predicate tracks matched rows (final result) - let predicate_rows_matched = if is_last { - rows_matched.clone() - } else { - metrics::Count::new() - }; + // - All predicates share the cumulative rows_pruned counter + // - Only the last predicate writes to rows_matched (final pass count) + // This preserves the invariant: rows_matched + rows_pruned = total rows. + let total_candidates = buildable.len(); + + let mut predicates: Vec> = + Vec::with_capacity(total_candidates); + for (idx, (filter_id, candidate)) in buildable.into_iter().enumerate() { + let is_last = idx == total_candidates - 1; + let predicate_rows_pruned = rows_pruned.clone(); + let predicate_rows_matched = if is_last { + rows_matched.clone() + } else { + metrics::Count::new() + }; + // Late-materialization savings: bytes of the *non-filter* portion + // of the projection, per row. When the filter prunes a row, the + // decoder avoids decoding these bytes further downstream — that + // is the quantity the tracker needs as `batch_bytes` so its + // effectiveness metric (bytes-saved / eval-time) ranks filters + // by actual savings rather than by their own read cost. Match the + // post-scan path's formula in + // `opener.rs::post_scan_other_bytes_per_row`. + let other_projected_bytes_per_row = + projection_compressed_bytes.saturating_sub(candidate.required_bytes) as f64 + / total_rows_f; + // Remember the original expression before we move `candidate` into + // `try_new`, so that a failed predicate construction can fall back + // into `unbuildable` rather than being silently dropped. + let original_expr = Arc::clone(&candidate.expr); + match DatafusionArrowPredicate::try_new( + candidate, + predicate_rows_pruned, + predicate_rows_matched, + time.clone(), + filter_id, + Arc::clone(tracker), + other_projected_bytes_per_row, + ) { + Ok(pred) => predicates.push(Box::new(pred) as _), + Err(e) => { + debug!( + "failed to construct ArrowPredicate for filter {filter_id}: {e}; \ + falling through to post-scan" + ); + unbuildable.push((filter_id, original_expr)); + } + } + } - DatafusionArrowPredicate::try_new( - candidate, - predicate_rows_pruned, - predicate_rows_matched, - time.clone(), - ) - .map(|pred| Box::new(pred) as _) - }) - .collect::, _>>() - .map(|filters| Some(RowFilter::new(filters))) + if predicates.is_empty() { + Ok((None, unbuildable)) + } else { + Ok((Some(RowFilter::new(predicates)), unbuildable)) + } +} + +/// Estimate the total on-disk (compressed) byte cost of reading the given +/// leaf column indices across every row group in the file. Used by the +/// adaptive [`crate::selectivity::SelectivityTracker`] as a cheap proxy for +/// filter evaluation cost before runtime stats are available. +pub(crate) fn total_compressed_bytes( + column_indices: &[usize], + metadata: &ParquetMetaData, +) -> usize { + let mut total: i64 = 0; + for rg in metadata.row_groups() { + for &idx in column_indices { + if let Some(col) = rg.columns().get(idx) { + total += col.compressed_size(); + } + } + } + total.max(0) as usize } #[cfg(test)] @@ -1183,11 +1333,15 @@ mod test { .expect("building candidate") .expect("candidate expected"); + let test_tracker = Arc::new(crate::selectivity::SelectivityTracker::new()); let mut row_filter = DatafusionArrowPredicate::try_new( candidate, Count::new(), Count::new(), Time::new(), + 0, + Arc::clone(&test_tracker), + 0.0, ) .expect("creating filter predicate"); @@ -1222,11 +1376,15 @@ mod test { .expect("building candidate") .expect("candidate expected"); + let test_tracker = Arc::new(crate::selectivity::SelectivityTracker::new()); let mut row_filter = DatafusionArrowPredicate::try_new( candidate, Count::new(), Count::new(), Time::new(), + 0, + Arc::clone(&test_tracker), + 0.0, ) .expect("creating filter predicate"); @@ -1371,10 +1529,18 @@ mod test { let file_metrics = ParquetFileMetrics::new(0, &format!("{func_name}.parquet"), &metrics); - let row_filter = - build_row_filter(&expr, &file_schema, &metadata, false, &file_metrics) - .expect("building row filter") - .expect("row filter should exist"); + let tracker = Arc::new(crate::selectivity::SelectivityTracker::new()); + let filters = vec![(0usize, expr)]; + let (row_filter, _unbuildable) = build_row_filter( + &filters, + &file_schema, + &metadata, + 0, + &tracker, + &file_metrics, + ) + .expect("building row filter"); + let row_filter = row_filter.expect("row filter should exist"); let reader = parquet_reader_builder .with_row_filter(row_filter) @@ -1949,10 +2115,18 @@ mod test { let metrics = ExecutionPlanMetricsSet::new(); let file_metrics = ParquetFileMetrics::new(0, "struct_e2e.parquet", &metrics); - let row_filter = - build_row_filter(&expr, &file_schema, &metadata, false, &file_metrics) - .expect("building row filter") - .expect("row filter should exist"); + let tracker = Arc::new(crate::selectivity::SelectivityTracker::new()); + let filters = vec![(0usize, expr)]; + let (row_filter, _unbuildable) = build_row_filter( + &filters, + &file_schema, + &metadata, + 0, + &tracker, + &file_metrics, + ) + .expect("building row filter"); + let row_filter = row_filter.expect("row filter should exist"); let reader = parquet_reader_builder .with_row_filter(row_filter) diff --git a/datafusion/datasource-parquet/src/selectivity.rs b/datafusion/datasource-parquet/src/selectivity.rs new file mode 100644 index 0000000000000..cbcee64190978 --- /dev/null +++ b/datafusion/datasource-parquet/src/selectivity.rs @@ -0,0 +1,2015 @@ +// 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. + +//! Adaptive filter selectivity tracking for Parquet row filters. +//! +//! See [`SelectivityTracker`] for the main entry point, `FilterState` for the +//! per-filter lifecycle, `PartitionedFilters` for the output consumed by +//! `ParquetOpener::open`, and [`FilterId`] for stable filter identification. + +use log::debug; +use parking_lot::{Mutex, RwLock}; +use parquet::file::metadata::ParquetMetaData; +use std::collections::HashMap; +use std::sync::Arc; +use std::sync::atomic::{AtomicBool, Ordering}; + +use datafusion_physical_expr::utils::collect_columns; +use datafusion_physical_expr_common::physical_expr::{ + OptionalFilterPhysicalExpr, PhysicalExpr, snapshot_generation, +}; + +/// Stable identifier for a filter conjunct, assigned by `ParquetSource::with_predicate`. +pub type FilterId = usize; + +/// Re-evaluate the per-filter skip flag every Nth batch update. The CI +/// upper bound is a couple of arithmetic ops so this cap mostly serves to +/// keep cache lines for `is_optional` / `skip_flags` cold on the hot path. +const SKIP_FLAG_CHECK_INTERVAL: u64 = 4; + +/// Per-filter lifecycle state in the adaptive filter system. +/// +/// State transitions: +/// - **(unseen)** → [`RowFilter`](Self::RowFilter) or [`PostScan`](Self::PostScan) +/// on first encounter in [`SelectivityTracker::partition_filters`]. +/// - [`PostScan`](Self::PostScan) → [`RowFilter`](Self::RowFilter) when +/// effectiveness ≥ `min_bytes_per_sec` and enough rows have been observed. +/// - [`RowFilter`](Self::RowFilter) → [`PostScan`](Self::PostScan) when +/// effectiveness is below threshold (mandatory filter). +/// - [`RowFilter`](Self::RowFilter) → [`Dropped`](Self::Dropped) when +/// effectiveness is below threshold and the filter is optional +/// ([`OptionalFilterPhysicalExpr`]). +/// - [`RowFilter`](Self::RowFilter) → [`PostScan`](Self::PostScan)/[`Dropped`](Self::Dropped) +/// on periodic re-evaluation if effectiveness drops below threshold after +/// CI upper bound drops below threshold. +/// - **Any state** → re-evaluated when a dynamic filter's +/// `snapshot_generation` changes. +#[derive(Debug, Clone, Copy, PartialEq)] +pub(crate) enum FilterState { + /// Currently a row filter. + RowFilter, + /// Currently a post-scan filter. + PostScan, + /// Dropped entirely (insufficient throughput and optional). + Dropped, +} + +/// Result of partitioning filters into row filters vs post-scan. +/// +/// Produced by [`SelectivityTracker::partition_filters`], consumed by +/// `ParquetOpener::open` to build row-level predicates and post-scan filters. +/// +/// Filters are partitioned based on their effectiveness threshold. +#[derive(Debug, Clone, Default)] +pub(crate) struct PartitionedFilters { + /// Filters promoted past collection — individual chained ArrowPredicates + pub(crate) row_filters: Vec<(FilterId, Arc)>, + /// Filters demoted to post-scan (fast path only) + pub(crate) post_scan: Vec<(FilterId, Arc)>, +} + +/// Tracks selectivity statistics for a single filter expression. +#[derive(Debug, Clone, Default, Copy, PartialEq)] +struct SelectivityStats { + /// Number of rows that matched (passed) the filter + rows_matched: u64, + /// Total number of rows evaluated + rows_total: u64, + /// Cumulative evaluation time in nanoseconds + eval_nanos: u64, + /// Cumulative bytes across batches this filter has been evaluated on + bytes_seen: u64, + /// Welford's online algorithm: number of per-batch effectiveness samples + sample_count: u64, + /// Welford's online algorithm: running mean of per-batch effectiveness + eff_mean: f64, + /// Welford's online algorithm: running sum of squared deviations (M2) + eff_m2: f64, +} + +impl SelectivityStats { + /// Returns the effectiveness as an opaque ordering score (higher = run first). + /// + /// Currently computed as bytes/sec throughput using self-contained stats. + /// Callers should not assume the unit. + fn effectiveness(&self) -> Option { + if self.rows_total == 0 || self.eval_nanos == 0 || self.bytes_seen == 0 { + return None; + } + let rows_pruned = self.rows_total - self.rows_matched; + let bytes_per_row = self.bytes_seen as f64 / self.rows_total as f64; + let bytes_saved = rows_pruned as f64 * bytes_per_row; + Some(bytes_saved * 1_000_000_000.0 / self.eval_nanos as f64) + } + + /// Returns the lower bound of a confidence interval on mean effectiveness. + /// + /// Uses Welford's online variance to compute a one-sided CI: + /// `mean - z * stderr`. Returns `None` if fewer than 2 samples. + fn confidence_lower_bound(&self, confidence_z: f64) -> Option { + if self.sample_count < 2 { + return None; + } + let variance = self.eff_m2 / (self.sample_count - 1) as f64; + let stderr = (variance / self.sample_count as f64).sqrt(); + Some(self.eff_mean - confidence_z * stderr) + } + + /// Returns the upper bound of a confidence interval on mean effectiveness. + /// + /// Uses Welford's online variance: `mean + z * stderr`. + /// Returns `None` if fewer than 2 samples. + fn confidence_upper_bound(&self, confidence_z: f64) -> Option { + if self.sample_count < 2 { + return None; + } + let variance = self.eff_m2 / (self.sample_count - 1) as f64; + let stderr = (variance / self.sample_count as f64).sqrt(); + Some(self.eff_mean + confidence_z * stderr) + } + + /// Update stats with new observations. + fn update(&mut self, matched: u64, total: u64, eval_nanos: u64, batch_bytes: u64) { + self.rows_matched += matched; + self.rows_total += total; + self.eval_nanos += eval_nanos; + self.bytes_seen += batch_bytes; + + // Feed Welford's algorithm with per-batch effectiveness. We admit + // samples with `batch_bytes == 0` — that legitimately represents a + // filter whose projection is a subset of its referenced columns, so + // late materialization has nothing to save even when the filter + // does prune rows. Recording `batch_eff = 0` for such batches lets + // the mid-stream skip path detect "CPU spent, no late- + // materialization payoff" and drop the filter if it is optional. + if total > 0 && eval_nanos > 0 { + let rows_pruned = total - matched; + let bytes_per_row = if total > 0 { + batch_bytes as f64 / total as f64 + } else { + 0.0 + }; + let batch_eff = + (rows_pruned as f64 * bytes_per_row) * 1e9 / eval_nanos as f64; + + self.sample_count += 1; + let delta = batch_eff - self.eff_mean; + self.eff_mean += delta / self.sample_count as f64; + let delta2 = batch_eff - self.eff_mean; + self.eff_m2 += delta * delta2; + } + } +} + +/// Immutable configuration for a [`SelectivityTracker`]. +/// +/// Use the builder methods to customise, then call [`build()`](TrackerConfig::build) +/// to produce a ready-to-use tracker. +pub(crate) struct TrackerConfig { + /// Minimum bytes/sec throughput for promoting a filter (default: INFINITY = disabled). + pub min_bytes_per_sec: f64, + /// Byte-ratio threshold for initial filter placement (row-level vs post-scan). + /// Computed as `filter_compressed_bytes / projection_compressed_bytes`. + /// When low, the filter columns are small relative to the projection, + /// so row-level placement enables large late-materialization savings. + /// When high, the filter columns dominate the projection, so there's + /// little benefit from late materialization. + /// Default is 0.20. + pub byte_ratio_threshold: f64, + /// Z-score for confidence intervals on filter effectiveness. + /// Lower values (e.g. 1.0 or 0.0) will make the tracker more aggressive about promotion/demotion based on limited data. + /// Higher values (e.g. 3.0) will require more confidence before changing filter states. + /// Default is 2.0, corresponding to ~97.5% one-sided confidence. + /// Set to <= 0.0 to disable confidence intervals and promote/demote based on point estimates alone (not recommended). + /// Set to INFINITY to disable promotion entirely (overrides `min_bytes_per_sec`). + pub confidence_z: f64, +} + +impl TrackerConfig { + pub fn new() -> Self { + Self { + min_bytes_per_sec: f64::INFINITY, + byte_ratio_threshold: 0.20, + confidence_z: 2.0, + } + } + + pub fn with_min_bytes_per_sec(mut self, v: f64) -> Self { + self.min_bytes_per_sec = v; + self + } + + pub fn with_byte_ratio_threshold(mut self, v: f64) -> Self { + self.byte_ratio_threshold = v; + self + } + + pub fn with_confidence_z(mut self, v: f64) -> Self { + self.confidence_z = v; + self + } + + pub fn build(self) -> SelectivityTracker { + SelectivityTracker { + config: self, + filter_stats: RwLock::new(HashMap::new()), + skip_flags: RwLock::new(HashMap::new()), + is_optional: RwLock::new(HashMap::new()), + inner: Mutex::new(SelectivityTrackerInner::new()), + } + } +} + +impl Default for TrackerConfig { + fn default() -> Self { + Self::new() + } +} + +/// Cross-file adaptive system that measures filter effectiveness and decides +/// which filters are promoted to row-level predicates (pushed into the Parquet +/// reader) vs. applied post-scan (demoted) or dropped entirely. +/// +/// # Locking design +/// +/// All locks are **private** to this struct — external callers cannot hold a +/// guard across expensive work, and all lock-holding code paths are auditable +/// in this file alone. +/// +/// State is split across two independent locks to minimise contention between +/// the hot per-batch `update()` path and the cold per-file-open +/// `partition_filters()` path: +/// +/// - **`filter_stats`** (`RwLock>>`) +/// — `update()` acquires a *shared read* lock on the outer map, then a +/// per-filter `Mutex` to increment counters. Multiple threads updating +/// *different* filters never contend at all; threads updating the *same* +/// filter serialize only on the cheap per-filter `Mutex` (~100 ns). +/// `partition_filters()` also takes a read lock here when it needs to +/// inspect stats for promotion/demotion decisions, so it never blocks +/// `update()` callers. The write lock is taken only briefly in Phase 2 +/// of `partition_filters()` to insert entries for newly-seen filter IDs. +/// +/// - **`inner`** (`Mutex`) — holds the filter +/// state-machine (`filter_states`) and dynamic-filter generation tracking. +/// Only `partition_filters()` acquires this lock (once per file open), so +/// concurrent `update()` calls are completely unaffected. +/// +/// ## Lock ordering (deadlock-free) +/// +/// Locks are always acquired in the order `inner` → `filter_stats` → +/// per-filter `Mutex`. Because `update()` never acquires `inner`, no +/// cycle is possible. +/// +/// ## Correctness of concurrent access +/// +/// `update()` may write stats while `partition_filters()` reads them for +/// promotion/demotion. Both hold a shared `filter_stats` read lock; the +/// per-filter `Mutex` ensures they do not interleave on the same filter's +/// stats. One proceeds first; the other sees a consistent (slightly newer +/// or older) snapshot. This is benign — the single-lock design that +/// preceded this split already allowed stats to change between consecutive +/// reads within `partition_filters()`. +/// +/// On promote/demote, `partition_filters()` zeros a filter's stats via the +/// per-filter `Mutex`. An `update()` running concurrently may write one +/// stale batch's worth of data to the freshly-zeroed stats; this is quickly +/// diluted by hundreds of correct-context batches and is functionally +/// identical to the old design where `update()` queued behind the write +/// lock and ran immediately after. +/// +/// # Filter state machine +/// +/// ```text +/// ┌─────────┐ +/// │ New │ +/// └─────────┘ +/// │ +/// ▼ +/// ┌────────────────────────┐ +/// │ Estimated Cost │ +/// │Bytes needed for filter │ +/// └────────────────────────┘ +/// │ +/// ┌──────────────────┴──────────────────┐ +/// ┌────────▼────────┐ ┌────────▼────────┐ +/// │ Post-scan │ │ Row filter │ +/// │ │ │ │ +/// └─────────────────┘ └─────────────────┘ +/// │ │ +/// ▼ ▼ +/// ┌─────────────────┐ ┌─────────────────┐ +/// │ Effectiveness │ │ Effectiveness │ +/// │ Bytes pruned │ │ Bytes pruned │ +/// │ per │ │ per │ +/// │Second of compute│ │Second of compute│ +/// └─────────────────┘ └─────────────────┘ +/// │ │ +/// └──────────────────┬──────────────────┘ +/// ▼ +/// ┌───────────────────────────────────────────────┐ +/// │ New Scan │ +/// │ Move filters based on effectiveness. │ +/// │ Promote (move post-scan -> row filter). │ +/// │ Demote (move row-filter -> post-scan). │ +/// │ Disable (for optional filters; either row │ +/// │ filter or disabled). │ +/// └───────────────────────────────────────────────┘ +/// │ +/// ┌──────────────────┴──────────────────┐ +/// ┌────────▼────────┐ ┌────────▼────────┐ +/// │ Post-scan │ │ Row filter │ +/// │ │ │ │ +/// └─────────────────┘ └─────────────────┘ +/// ``` +/// +/// See `TrackerConfig` for configuration knobs. +pub struct SelectivityTracker { + config: TrackerConfig, + /// Per-filter selectivity statistics, each individually `Mutex`-protected. + /// + /// The outer `RwLock` is almost always read-locked: both `update()` (hot, + /// per-batch) and `partition_filters()` (cold, per-file-open) only need + /// shared access to look up existing entries. The write lock is taken + /// only when `partition_filters()` inserts entries for newly-seen filter + /// IDs — a brief, infrequent operation. + /// + /// Each inner `Mutex` protects a single filter's + /// counters, so concurrent `update()` calls on *different* filters + /// proceed in parallel with zero contention. + filter_stats: RwLock>>, + /// Per-filter "skip" flags — when set, the corresponding filter is + /// treated as a no-op by both the row-filter + /// (`DatafusionArrowPredicate::evaluate`) and the post-scan path + /// (`apply_post_scan_filters_with_stats`). This is the mid-stream + /// equivalent of dropping an optional filter: once the per-batch + /// `update()` path proves an `OptionalFilterPhysicalExpr` is + /// CPU-dominated and ineffective, it flips the flag and subsequent + /// batches stop paying the evaluation cost. The decoder still decodes + /// the filter columns (we cannot rebuild it mid-scan), so I/O is not + /// reclaimed; only the predicate evaluation is skipped. + /// + /// Only ever set for filters whose `is_optional` entry is `true` — + /// mandatory filters must always execute or queries return wrong rows. + skip_flags: RwLock>>, + /// Whether each filter is wrapped in an `OptionalFilterPhysicalExpr`, + /// captured at first-encounter in `partition_filters` so the per-batch + /// `update()` path can decide whether the filter is safe to no-op + /// without re-inspecting the expression tree on every batch. + is_optional: RwLock>, + /// Filter lifecycle state machine and dynamic-filter generation tracking. + /// + /// Only `partition_filters()` acquires this lock (once per file open). + /// `update()` never touches it, so the hot per-batch path is completely + /// decoupled from the cold state-machine path. + inner: Mutex, +} + +impl std::fmt::Debug for SelectivityTracker { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + f.debug_struct("SelectivityTracker") + .field("config.min_bytes_per_sec", &self.config.min_bytes_per_sec) + .finish() + } +} + +impl Default for SelectivityTracker { + fn default() -> Self { + Self::new() + } +} + +impl SelectivityTracker { + /// Create a new tracker with default settings (feature disabled). + pub fn new() -> Self { + TrackerConfig::new().build() + } + + /// Update stats for a filter after processing a batch. + /// + /// **Locking:** acquires `filter_stats.read()` (shared) then a per-filter + /// `Mutex`. Never touches `inner`, so this hot per-batch path cannot + /// contend with the cold per-file-open `partition_filters()` path. + /// + /// Silently skips unknown filter IDs (can occur if `update()` is called + /// before `partition_filters()` has registered the filter — in practice + /// this cannot happen because `partition_filters()` runs during file open + /// before any batches are processed). + /// + /// **Mid-stream drop:** after every `SKIP_FLAG_CHECK_INTERVAL`'th batch + /// we evaluate the CI upper bound; if it falls below + /// `min_bytes_per_sec` and the filter is wrapped in + /// `OptionalFilterPhysicalExpr`, we set the per-filter skip flag. + /// Subsequent calls to `DatafusionArrowPredicate::evaluate` (row-level) + /// and `apply_post_scan_filters_with_stats` (post-scan) observe the + /// flag and short-circuit their work for that filter. Mandatory + /// filters are never flagged because doing so would change the result + /// set. + pub(crate) fn update( + &self, + id: FilterId, + matched: u64, + total: u64, + eval_nanos: u64, + batch_bytes: u64, + ) { + let stats_map = self.filter_stats.read(); + let Some(entry) = stats_map.get(&id) else { + return; + }; + let mut stats = entry.lock(); + stats.update(matched, total, eval_nanos, batch_bytes); + + // Mid-stream drop check. Only consult the skip mechanism for + // filters we already know to be optional, and only after enough + // samples for `confidence_upper_bound` to be defined. The modulo + // gate keeps the per-batch overhead tiny on the hot path. + if !self.config.min_bytes_per_sec.is_finite() + || !stats.sample_count.is_multiple_of(SKIP_FLAG_CHECK_INTERVAL) + { + return; + } + let Some(ub) = stats.confidence_upper_bound(self.config.confidence_z) else { + return; + }; + if ub >= self.config.min_bytes_per_sec { + return; + } + drop(stats); + drop(stats_map); + + // Optionality is captured at first sight in `partition_filters` so + // we can answer this without re-walking the expression tree. + let is_optional = self.is_optional.read().get(&id).copied().unwrap_or(false); + if !is_optional { + return; + } + if let Some(flag) = self.skip_flags.read().get(&id) + && !flag.swap(true, Ordering::Release) + { + debug!( + "FilterId {id}: mid-stream skip — CI upper bound {ub} < {} bytes/sec", + self.config.min_bytes_per_sec + ); + } + } + + /// Returns the shared skip flag for `id`, creating one if absent. + /// + /// Cloned into [`crate::row_filter::DatafusionArrowPredicate`] so the + /// row-filter path can short-circuit when the per-batch update path + /// decides the filter has stopped pulling its weight. The post-scan + /// path uses [`Self::is_filter_skipped`] instead — it does not need a + /// long-lived handle. + pub(crate) fn skip_flag(&self, id: FilterId) -> Arc { + if let Some(existing) = self.skip_flags.read().get(&id) { + return Arc::clone(existing); + } + let mut write = self.skip_flags.write(); + Arc::clone( + write + .entry(id) + .or_insert_with(|| Arc::new(AtomicBool::new(false))), + ) + } + + /// Returns `true` when `id` has been mid-stream-dropped by the tracker. + /// + /// Cheap: a single `RwLock::read` plus an atomic load. Called from the + /// post-scan filter loop in `apply_post_scan_filters_with_stats`. + pub(crate) fn is_filter_skipped(&self, id: FilterId) -> bool { + self.skip_flags + .read() + .get(&id) + .is_some_and(|f| f.load(Ordering::Acquire)) + } + + /// Partition filters into row-level predicates vs post-scan filters. + /// + /// Called once per file open (cold path). + /// + /// **Locking — two phases:** + /// 1. Acquires `inner` (exclusive) and `filter_stats` (shared read) for + /// all decision logic — promotion, demotion, initial placement, and + /// sorting by effectiveness. Because `filter_stats` is only + /// read-locked, concurrent `update()` calls proceed unblocked. + /// 2. If new filter IDs were seen, briefly acquires `filter_stats` (write) + /// to insert per-filter `Mutex` entries so that future `update()` calls + /// can find them. + pub(crate) fn partition_filters( + &self, + filters: Vec<(FilterId, Arc)>, + projection_scan_size: usize, + metadata: &ParquetMetaData, + ) -> PartitionedFilters { + // Phase 1: inner.lock() + filter_stats.read() → all decision logic + let mut guard = self.inner.lock(); + let stats_map = self.filter_stats.read(); + let result = guard.partition_filters( + filters, + projection_scan_size, + metadata, + &self.config, + &stats_map, + ); + drop(stats_map); + drop(guard); + + // Phase 2: if new filters were seen, briefly acquire write lock to insert entries + if !result.new_filter_ids.is_empty() { + let mut stats_write = self.filter_stats.write(); + for id in &result.new_filter_ids { + stats_write + .entry(*id) + .or_insert_with(|| Mutex::new(SelectivityStats::default())); + } + } + if !result.new_optional_flags.is_empty() { + let mut optional_write = self.is_optional.write(); + let mut skip_write = self.skip_flags.write(); + for (id, is_optional) in result.new_optional_flags { + optional_write.entry(id).or_insert(is_optional); + skip_write + .entry(id) + .or_insert_with(|| Arc::new(AtomicBool::new(false))); + } + } + + result.partitioned + } + + /// Test helper: ensure a stats entry exists for the given filter ID. + /// In production, `partition_filters()` inserts entries for new filters. + /// Tests that call `update()` without prior `partition_filters()` need this. + #[cfg(test)] + fn ensure_stats_entry(&self, id: FilterId) { + let map = self.filter_stats.read(); + if map.get(&id).is_none() { + drop(map); + self.filter_stats + .write() + .entry(id) + .or_insert_with(|| Mutex::new(SelectivityStats::default())); + } + } +} + +/// Internal result from [`SelectivityTrackerInner::partition_filters`]. +/// +/// Carries both the partitioned filters and a list of newly-seen filter IDs +/// back to the outer [`SelectivityTracker::partition_filters`], which uses +/// `new_filter_ids` to insert per-filter `Mutex` entries into `filter_stats` +/// in a brief Phase 2 write lock. +struct PartitionResult { + partitioned: PartitionedFilters, + new_filter_ids: Vec, + /// `(FilterId, is_optional)` entries observed for the first time in this + /// `partition_filters` call. The outer `SelectivityTracker` records + /// optionality alongside `filter_stats` so that the hot `update()` path + /// can decide whether the per-filter skip flag is safe to flip without + /// inspecting the expression tree. + new_optional_flags: Vec<(FilterId, bool)>, +} + +/// Filter state-machine and generation tracking, guarded by the `Mutex` +/// inside [`SelectivityTracker`]. +/// +/// This struct intentionally does **not** contain per-filter stats — those +/// live in the separate `filter_stats` lock so that the hot `update()` path +/// can modify stats without acquiring this lock. Only the cold +/// `partition_filters()` path (once per file open) needs this lock. +#[derive(Debug)] +struct SelectivityTrackerInner { + /// Per-filter lifecycle state (RowFilter / PostScan / Dropped). + filter_states: HashMap, + /// Last-seen snapshot generation per filter, for detecting when a dynamic + /// filter's selectivity has changed (e.g. hash-join build side grew). + snapshot_generations: HashMap, +} + +impl SelectivityTrackerInner { + fn new() -> Self { + Self { + filter_states: HashMap::new(), + snapshot_generations: HashMap::new(), + } + } + + /// Check and update the snapshot generation for a filter. + fn note_generation( + &mut self, + id: FilterId, + generation: u64, + stats_map: &HashMap>, + ) { + if generation == 0 { + return; + } + match self.snapshot_generations.get(&id) { + Some(&prev_generation) if prev_generation == generation => {} + Some(_) => { + let current_state = self.filter_states.get(&id).copied(); + // Always reset stats since selectivity changed with new generation. + if let Some(entry) = stats_map.get(&id) { + *entry.lock() = SelectivityStats::default(); + } + self.snapshot_generations.insert(id, generation); + + // Optional/dynamic filters only get more selective over time + // (hash join build side accumulates more values). So if the + // filter was already working (RowFilter or PostScan), preserve + // its state. Only un-drop Dropped filters back to PostScan + // so they get another chance with the new selectivity. + if current_state == Some(FilterState::Dropped) { + debug!("FilterId {id} generation changed, un-dropping to PostScan"); + self.filter_states.insert(id, FilterState::PostScan); + } else { + debug!( + "FilterId {id} generation changed, resetting stats but preserving state {current_state:?}" + ); + } + } + None => { + self.snapshot_generations.insert(id, generation); + } + } + } + + /// Get the effectiveness for a filter by ID. + fn get_effectiveness_by_id( + &self, + id: FilterId, + stats_map: &HashMap>, + ) -> Option { + stats_map + .get(&id) + .and_then(|entry| entry.lock().effectiveness()) + } + + /// Demote a filter to post-scan or drop it entirely if optional. + fn demote_or_drop( + &mut self, + id: FilterId, + expr: &Arc, + post_scan: &mut Vec<(FilterId, Arc)>, + stats_map: &HashMap>, + ) { + if expr.downcast_ref::().is_none() { + self.filter_states.insert(id, FilterState::PostScan); + post_scan.push((id, Arc::clone(expr))); + // Reset stats for this filter so it can be re-evaluated as a post-scan filter. + if let Some(entry) = stats_map.get(&id) { + *entry.lock() = SelectivityStats::default(); + } + } else { + self.filter_states.insert(id, FilterState::Dropped); + } + } + + /// Promote a filter to row-level. + fn promote( + &mut self, + id: FilterId, + expr: Arc, + row_filters: &mut Vec<(FilterId, Arc)>, + stats_map: &HashMap>, + ) { + row_filters.push((id, expr)); + self.filter_states.insert(id, FilterState::RowFilter); + // Reset stats for this filter since it will be evaluated at row-level now. + if let Some(entry) = stats_map.get(&id) { + *entry.lock() = SelectivityStats::default(); + } + } + + /// Partition filters into collecting / promoted / post-scan buckets. + fn partition_filters( + &mut self, + filters: Vec<(FilterId, Arc)>, + projection_scan_size: usize, + metadata: &ParquetMetaData, + config: &TrackerConfig, + stats_map: &HashMap>, + ) -> PartitionResult { + let mut new_filter_ids = Vec::new(); + let mut new_optional_flags: Vec<(FilterId, bool)> = Vec::new(); + + // If min_bytes_per_sec is INFINITY -> all filters are post-scan. + if config.min_bytes_per_sec.is_infinite() { + debug!( + "Filter promotion disabled via min_bytes_per_sec=INFINITY; all {} filters post-scan", + filters.len() + ); + // Register all filter IDs so update() can find them + for (id, expr) in &filters { + if !stats_map.contains_key(id) { + new_filter_ids.push(*id); + new_optional_flags.push((*id, is_optional_filter(expr))); + } + } + return PartitionResult { + partitioned: PartitionedFilters { + row_filters: Vec::new(), + post_scan: filters, + }, + new_filter_ids, + new_optional_flags, + }; + } + // If min_bytes_per_sec is 0 -> all filters are promoted. + if config.min_bytes_per_sec == 0.0 { + debug!( + "All filters promoted via min_bytes_per_sec=0; all {} filters row-level", + filters.len() + ); + // Register all filter IDs so update() can find them + for (id, expr) in &filters { + if !stats_map.contains_key(id) { + new_filter_ids.push(*id); + new_optional_flags.push((*id, is_optional_filter(expr))); + } + } + return PartitionResult { + partitioned: PartitionedFilters { + row_filters: filters, + post_scan: Vec::new(), + }, + new_filter_ids, + new_optional_flags, + }; + } + + // Note snapshot generations for dynamic filter detection. + // This clears stats for any filter whose generation has changed since the last scan. + // This must be done before any other logic since it can change filter states and stats. + for &(id, ref expr) in &filters { + let generation = snapshot_generation(expr); + self.note_generation(id, generation, stats_map); + } + + // Separate into row filters and post-scan filters based on effectiveness and state. + let mut row_filters: Vec<(FilterId, Arc)> = Vec::new(); + let mut post_scan_filters: Vec<(FilterId, Arc)> = Vec::new(); + + let confidence_z = config.confidence_z; + for (id, expr) in filters { + let state = self.filter_states.get(&id).copied(); + + let Some(state) = state else { + // New filter: decide initial placement using the + // filter_bytes / projection_bytes ratio. This ratio captures + // the I/O tradeoff: + // + // - Low ratio (filter columns are small vs projection): row-filter + // enables late materialization — the large non-filter portion of + // the projection is only decoded for rows that pass the filter. + // + // - High ratio (filter columns are most of the projection): little + // benefit from late materialization since there's not much left + // to skip. Post-scan avoids row-filter overhead. + // + // Extra bytes (filter columns not in projection) are naturally + // included in filter_bytes, making the ratio higher and placement + // more conservative, which is correct since those bytes represent + // additional I/O cost for row-filter evaluation. + let filter_columns: Vec = collect_columns(&expr) + .iter() + .map(|col| col.index()) + .collect(); + let filter_bytes = + crate::row_filter::total_compressed_bytes(&filter_columns, metadata); + let byte_ratio = if projection_scan_size > 0 { + filter_bytes as f64 / projection_scan_size as f64 + } else { + 1.0 + }; + + if !stats_map.contains_key(&id) { + new_filter_ids.push(id); + new_optional_flags.push((id, is_optional_filter(&expr))); + } + + if byte_ratio <= config.byte_ratio_threshold { + debug!( + "FilterId {id}: New filter → Row filter (byte_ratio {byte_ratio:.4} <= {}) — {expr}", + config.byte_ratio_threshold + ); + self.filter_states.insert(id, FilterState::RowFilter); + row_filters.push((id, expr)); + } else { + debug!( + "FilterId {id}: New filter → Post-scan (byte_ratio {byte_ratio:.4} > {}) — {expr}", + config.byte_ratio_threshold + ); + self.filter_states.insert(id, FilterState::PostScan); + post_scan_filters.push((id, expr)); + } + continue; + }; + + match state { + FilterState::RowFilter => { + // Should we demote this filter based on CI upper bound? + if let Some(entry) = stats_map.get(&id) { + let stats = entry.lock(); + if let Some(ub) = stats.confidence_upper_bound(confidence_z) + && ub < config.min_bytes_per_sec + { + drop(stats); + debug!( + "FilterId {id}: Row filter → Post-scan via CI upper bound {ub} < {} bytes/sec — {expr}", + config.min_bytes_per_sec + ); + self.demote_or_drop( + id, + &expr, + &mut post_scan_filters, + stats_map, + ); + continue; + } + } + // If not demoted, keep as row filter. + row_filters.push((id, expr)); + } + FilterState::PostScan => { + // Should we promote this filter based on CI lower bound? + if let Some(entry) = stats_map.get(&id) { + let stats = entry.lock(); + if let Some(lb) = stats.confidence_lower_bound(confidence_z) + && lb >= config.min_bytes_per_sec + { + drop(stats); + debug!( + "FilterId {id}: Post-scan → Row filter via CI lower bound {lb} >= {} bytes/sec — {expr}", + config.min_bytes_per_sec + ); + self.promote(id, expr, &mut row_filters, stats_map); + continue; + } + } + // Should we drop this filter if it's optional and ineffective? + // Non-optional filters must stay as post-scan regardless. + if let Some(entry) = stats_map.get(&id) { + let stats = entry.lock(); + if let Some(ub) = stats.confidence_upper_bound(confidence_z) + && ub < config.min_bytes_per_sec + && expr.downcast_ref::().is_some() + { + drop(stats); + debug!( + "FilterId {id}: Post-scan → Dropped via CI upper bound {ub} < {} bytes/sec — {expr}", + config.min_bytes_per_sec + ); + self.filter_states.insert(id, FilterState::Dropped); + continue; + } + } + // Keep as post-scan filter (don't reset stats for mandatory filters). + post_scan_filters.push((id, expr)); + } + FilterState::Dropped => continue, + } + } + + // Sort row filters by: + // - Effectiveness (descending, higher = better) if available for both filters. + // - Scan size (ascending, cheapest first) as fallback — cheap filters prune + // rows before expensive ones, reducing downstream evaluation cost. + let cmp_row_filters = + |(id_a, expr_a): &(FilterId, Arc), + (id_b, expr_b): &(FilterId, Arc)| { + let eff_a = self.get_effectiveness_by_id(*id_a, stats_map); + let eff_b = self.get_effectiveness_by_id(*id_b, stats_map); + if let (Some(eff_a), Some(eff_b)) = (eff_a, eff_b) { + eff_b + .partial_cmp(&eff_a) + .unwrap_or(std::cmp::Ordering::Equal) + } else { + let size_a = filter_scan_size(expr_a, metadata); + let size_b = filter_scan_size(expr_b, metadata); + size_a.cmp(&size_b) + } + }; + row_filters.sort_by(cmp_row_filters); + // Post-scan filters: same logic (cheaper post-scan filters first to reduce + // the batch size for subsequent filters). + post_scan_filters.sort_by(cmp_row_filters); + + debug!( + "Partitioned filters: {} row-level, {} post-scan", + row_filters.len(), + post_scan_filters.len() + ); + PartitionResult { + partitioned: PartitionedFilters { + row_filters, + post_scan: post_scan_filters, + }, + new_filter_ids, + new_optional_flags, + } + } +} + +/// Returns `true` if `expr` is wrapped in [`OptionalFilterPhysicalExpr`]. +fn is_optional_filter(expr: &Arc) -> bool { + expr.downcast_ref::().is_some() +} + +/// Calculate the estimated number of bytes needed to evaluate a filter based on the columns +/// it references as if it were applied to the entire file. +/// This is used for initial placement of new filters before any stats are available, and as a fallback for filters without stats. +fn filter_scan_size(expr: &Arc, metadata: &ParquetMetaData) -> usize { + let columns: Vec = collect_columns(expr) + .iter() + .map(|col| col.index()) + .collect(); + + crate::row_filter::total_compressed_bytes(&columns, metadata) +} + +#[cfg(test)] +mod tests { + use super::*; + use datafusion_physical_expr::expressions::Column; + use parquet::basic::Type as PhysicalType; + use parquet::file::metadata::{ColumnChunkMetaData, FileMetaData, RowGroupMetaData}; + use parquet::schema::types::SchemaDescPtr; + use parquet::schema::types::Type as SchemaType; + use std::sync::Arc; + + mod helper_functions { + use super::*; + + /// Creates test ParquetMetaData with specified row groups and column sizes. + /// + /// # Arguments + /// * `specs` - Vec of (num_rows, vec![compressed_size]) tuples for each row group + pub fn create_test_metadata(specs: Vec<(i64, Vec)>) -> ParquetMetaData { + // Get the maximum number of columns from all specs + let num_columns = specs + .iter() + .map(|(_, sizes)| sizes.len()) + .max() + .unwrap_or(1); + let schema_descr = get_test_schema_descr_with_columns(num_columns); + + let row_group_metadata: Vec<_> = specs + .into_iter() + .map(|(num_rows, column_sizes)| { + let columns = column_sizes + .into_iter() + .enumerate() + .map(|(col_idx, size)| { + ColumnChunkMetaData::builder(schema_descr.column(col_idx)) + .set_num_values(num_rows) + .set_total_compressed_size(size as i64) + .build() + .unwrap() + }) + .collect(); + + RowGroupMetaData::builder(schema_descr.clone()) + .set_num_rows(num_rows) + .set_column_metadata(columns) + .build() + .unwrap() + }) + .collect(); + + let total_rows: i64 = row_group_metadata.iter().map(|rg| rg.num_rows()).sum(); + let file_metadata = + FileMetaData::new(1, total_rows, None, None, schema_descr.clone(), None); + + ParquetMetaData::new(file_metadata, row_group_metadata) + } + + /// Creates a simple column expression with given name and index. + pub fn col_expr(name: &str, index: usize) -> Arc { + Arc::new(Column::new(name, index)) + } + + /// Create schema with specified number of columns, each named "a", "b", etc. + pub fn get_test_schema_descr_with_columns(num_columns: usize) -> SchemaDescPtr { + use parquet::basic::LogicalType; + + let fields: Vec<_> = (0..num_columns) + .map(|i| { + let col_name = format!("{}", (b'a' + i as u8) as char); + SchemaType::primitive_type_builder( + &col_name, + PhysicalType::BYTE_ARRAY, + ) + .with_logical_type(Some(LogicalType::String)) + .build() + .unwrap() + }) + .map(Arc::new) + .collect(); + + let schema = SchemaType::group_type_builder("schema") + .with_fields(fields) + .build() + .unwrap(); + Arc::new(parquet::schema::types::SchemaDescriptor::new(Arc::new( + schema, + ))) + } + } + + mod selectivity_stats_tests { + use super::*; + + #[test] + fn test_effectiveness_basic_calculation() { + let mut stats = SelectivityStats::default(); + + // 100 rows total, 50 rows pruned (matched 50), 1 sec eval time, 10000 bytes seen + // bytes_per_row = 10000 / 100 = 100 + // bytes_saved = 50 * 100 = 5000 + // effectiveness = 5000 * 1e9 / 1e9 = 5000 + stats.update(50, 100, 1_000_000_000, 10_000); + + let eff = stats.effectiveness().unwrap(); + assert!((eff - 5000.0).abs() < 0.1); + } + + #[test] + fn test_effectiveness_zero_rows_total() { + let mut stats = SelectivityStats::default(); + stats.update(0, 0, 1_000_000_000, 10_000); + + assert_eq!(stats.effectiveness(), None); + } + + #[test] + fn test_effectiveness_zero_eval_nanos() { + let mut stats = SelectivityStats::default(); + stats.update(50, 100, 0, 10_000); + + assert_eq!(stats.effectiveness(), None); + } + + #[test] + fn test_effectiveness_zero_bytes_seen() { + let mut stats = SelectivityStats::default(); + stats.update(50, 100, 1_000_000_000, 0); + + assert_eq!(stats.effectiveness(), None); + } + + #[test] + fn test_effectiveness_all_rows_matched() { + let mut stats = SelectivityStats::default(); + // All rows matched (no pruning) + stats.update(100, 100, 1_000_000_000, 10_000); + + let eff = stats.effectiveness().unwrap(); + assert_eq!(eff, 0.0); + } + + #[test] + fn test_confidence_bounds_single_sample() { + let mut stats = SelectivityStats::default(); + stats.update(50, 100, 1_000_000_000, 10_000); + + // Single sample returns None for confidence bounds + assert_eq!(stats.confidence_lower_bound(2.0), None); + assert_eq!(stats.confidence_upper_bound(2.0), None); + } + + #[test] + fn test_welford_identical_samples() { + let mut stats = SelectivityStats::default(); + + // Add two identical samples + stats.update(50, 100, 1_000_000_000, 10_000); + stats.update(50, 100, 1_000_000_000, 10_000); + + // Variance should be 0 + assert_eq!(stats.sample_count, 2); + let lb = stats.confidence_lower_bound(2.0).unwrap(); + let ub = stats.confidence_upper_bound(2.0).unwrap(); + + // Both should be equal to the mean since variance is 0 + assert!((lb - ub).abs() < 0.01); + } + + #[test] + fn test_welford_variance_calculation() { + let mut stats = SelectivityStats::default(); + + // Add samples that will produce effectiveness values of ~100, ~200, ~300 + // These are constructed to give those exact effectiveness values + stats.update(50, 100, 1_000_000_000, 10_000); // eff ≈ 5000 + stats.update(40, 100, 1_000_000_000, 10_000); // eff ≈ 6000 + stats.update(30, 100, 1_000_000_000, 10_000); // eff ≈ 7000 + + // We should have 3 samples + assert_eq!(stats.sample_count, 3); + + // Mean should be 6000 + assert!((stats.eff_mean - 6000.0).abs() < 1.0); + + // Both bounds should be defined + let lb = stats.confidence_lower_bound(1.0).unwrap(); + let ub = stats.confidence_upper_bound(1.0).unwrap(); + + assert!(lb < stats.eff_mean); + assert!(ub > stats.eff_mean); + } + + #[test] + fn test_confidence_bounds_asymmetry() { + let mut stats = SelectivityStats::default(); + + stats.update(50, 100, 1_000_000_000, 10_000); + stats.update(40, 100, 1_000_000_000, 10_000); + + let lb = stats.confidence_lower_bound(2.0).unwrap(); + let ub = stats.confidence_upper_bound(2.0).unwrap(); + + // Bounds should be symmetric around the mean + let lower_dist = stats.eff_mean - lb; + let upper_dist = ub - stats.eff_mean; + + assert!((lower_dist - upper_dist).abs() < 0.01); + } + + #[test] + fn test_welford_incremental_vs_batch() { + // Create two identical stats objects + let mut stats_incremental = SelectivityStats::default(); + let mut stats_batch = SelectivityStats::default(); + + // Incremental: add one at a time + stats_incremental.update(50, 100, 1_000_000_000, 10_000); + stats_incremental.update(40, 100, 1_000_000_000, 10_000); + stats_incremental.update(30, 100, 1_000_000_000, 10_000); + + // Batch: simulate batch update (all at once) + stats_batch.update(120, 300, 3_000_000_000, 30_000); + + // Both should produce the same overall statistics + assert_eq!(stats_incremental.rows_total, stats_batch.rows_total); + assert_eq!(stats_incremental.rows_matched, stats_batch.rows_matched); + + // Means should be close + assert!((stats_incremental.eff_mean - stats_batch.eff_mean).abs() < 100.0); + } + + #[test] + fn test_effectiveness_numerical_stability() { + let mut stats = SelectivityStats::default(); + + // Test with large values to ensure numerical stability + stats.update( + 500_000_000, + 1_000_000_000, + 10_000_000_000_000, + 1_000_000_000_000, + ); + + let eff = stats.effectiveness(); + assert!(eff.is_some()); + assert!(eff.unwrap() > 0.0); + assert!(!eff.unwrap().is_nan()); + assert!(!eff.unwrap().is_infinite()); + } + } + + mod tracker_config_tests { + use super::*; + + #[test] + fn test_default_config() { + let config = TrackerConfig::default(); + + assert!(config.min_bytes_per_sec.is_infinite()); + assert_eq!(config.byte_ratio_threshold, 0.20); + assert_eq!(config.confidence_z, 2.0); + } + + #[test] + fn test_with_min_bytes_per_sec() { + let config = TrackerConfig::new().with_min_bytes_per_sec(1000.0); + + assert_eq!(config.min_bytes_per_sec, 1000.0); + } + + #[test] + fn test_with_byte_ratio_threshold() { + let config = TrackerConfig::new().with_byte_ratio_threshold(0.5); + + assert_eq!(config.byte_ratio_threshold, 0.5); + } + + #[test] + fn test_with_confidence_z() { + let config = TrackerConfig::new().with_confidence_z(3.0); + + assert_eq!(config.confidence_z, 3.0); + } + + #[test] + fn test_builder_chain() { + let config = TrackerConfig::new() + .with_min_bytes_per_sec(500.0) + .with_byte_ratio_threshold(0.3) + .with_confidence_z(1.5); + + assert_eq!(config.min_bytes_per_sec, 500.0); + assert_eq!(config.byte_ratio_threshold, 0.3); + assert_eq!(config.confidence_z, 1.5); + } + + #[test] + fn test_build_creates_tracker() { + let tracker = TrackerConfig::new().with_min_bytes_per_sec(1000.0).build(); + + // Tracker should be created and functional + assert_eq!(tracker.config.min_bytes_per_sec, 1000.0); + } + } + + mod state_machine_tests { + use super::helper_functions::*; + use super::*; + + #[test] + fn test_initial_placement_low_byte_ratio() { + let tracker = TrackerConfig::new() + .with_min_bytes_per_sec(1000.0) + .with_byte_ratio_threshold(0.2) + .build(); + + // Create metadata: 1 row group, 100 rows, 1000 bytes for column + let metadata = create_test_metadata(vec![(100, vec![1000])]); + + // Filter using column 0 (1000 bytes out of 1000 projection = 100% ratio > 0.2) + // So this should be placed in post-scan initially + let expr = col_expr("a", 0); + let filters = vec![(1, expr)]; + + let result = tracker.partition_filters(filters, 1000, &metadata); + + // With 100% byte ratio, should go to post-scan + assert_eq!(result.row_filters.len(), 0); + assert_eq!(result.post_scan.len(), 1); + } + + #[test] + fn test_initial_placement_filter_in_projection_low_ratio() { + let tracker = TrackerConfig::new() + .with_min_bytes_per_sec(1000.0) + .with_byte_ratio_threshold(0.5) + .build(); + + // Create metadata: 1 row group, 100 rows, 100 bytes for column + let metadata = create_test_metadata(vec![(100, vec![100])]); + + // Filter using column 0 which IS in the projection. + // filter_bytes=100, projection=1000, ratio=0.10 <= 0.5 → RowFilter + let expr = col_expr("a", 0); + let filters = vec![(1, expr)]; + + let result = tracker.partition_filters(filters, 1000, &metadata); + + assert_eq!(result.row_filters.len(), 1); + assert_eq!(result.post_scan.len(), 0); + } + + #[test] + fn test_initial_placement_high_byte_ratio() { + let tracker = TrackerConfig::new() + .with_min_bytes_per_sec(1000.0) + .with_byte_ratio_threshold(0.5) + .build(); + + // Create metadata: 1 row group, 100 rows, 100 bytes for column + let metadata = create_test_metadata(vec![(100, vec![100])]); + + // Filter using column 0 (100 bytes / 1000 projection = 10% ratio <= 0.5) + // So this should be placed in row-filter immediately + let expr = col_expr("a", 0); + let filters = vec![(1, expr)]; + + let result = tracker.partition_filters(filters, 1000, &metadata); + + // With 10% byte ratio, should go to row-filter + assert_eq!(result.row_filters.len(), 1); + assert_eq!(result.post_scan.len(), 0); + } + + #[test] + fn test_min_bytes_per_sec_infinity_disables_promotion() { + let tracker = TrackerConfig::new() + .with_min_bytes_per_sec(f64::INFINITY) + .build(); + + let metadata = create_test_metadata(vec![(100, vec![100])]); + let expr = col_expr("a", 0); + let filters = vec![(1, expr)]; + + let result = tracker.partition_filters(filters, 1000, &metadata); + + // All filters should go to post_scan when min_bytes_per_sec is INFINITY + assert_eq!(result.row_filters.len(), 0); + assert_eq!(result.post_scan.len(), 1); + } + + #[test] + fn test_min_bytes_per_sec_zero_promotes_all() { + let tracker = TrackerConfig::new().with_min_bytes_per_sec(0.0).build(); + + let metadata = create_test_metadata(vec![(100, vec![1000])]); + let expr = col_expr("a", 0); + let filters = vec![(1, expr)]; + + let result = tracker.partition_filters(filters, 1000, &metadata); + + // All filters should be promoted to row_filters when min_bytes_per_sec is 0 + assert_eq!(result.row_filters.len(), 1); + assert_eq!(result.post_scan.len(), 0); + } + + #[test] + fn test_promotion_via_confidence_lower_bound() { + let tracker = TrackerConfig::new() + .with_min_bytes_per_sec(1000.0) + .with_byte_ratio_threshold(0.5) // Force to PostScan initially + .with_confidence_z(0.5) // Lower z for easier promotion + .build(); + + let metadata = create_test_metadata(vec![(100, vec![1000])]); + let expr = col_expr("a", 0); + let filters = vec![(1, expr.clone())]; + + // First partition: goes to PostScan (high byte ratio) + let result = tracker.partition_filters(filters.clone(), 1000, &metadata); + assert_eq!(result.post_scan.len(), 1); + assert_eq!(result.row_filters.len(), 0); + + // Feed high effectiveness stats + for _ in 0..5 { + tracker.update(1, 10, 100, 100_000, 1000); // high effectiveness + } + + // Second partition: should be promoted to RowFilter + let result = tracker.partition_filters(filters, 1000, &metadata); + assert_eq!(result.row_filters.len(), 1); + assert_eq!(result.post_scan.len(), 0); + } + + #[test] + fn test_demotion_via_confidence_upper_bound() { + let tracker = TrackerConfig::new() + .with_min_bytes_per_sec(10000.0) + .with_byte_ratio_threshold(0.1) // Force to RowFilter initially + .with_confidence_z(0.5) // Lower z for easier demotion + .build(); + + let metadata = create_test_metadata(vec![(100, vec![100])]); + let expr = col_expr("a", 0); + let filters = vec![(1, expr.clone())]; + + // First partition: goes to RowFilter (low byte ratio) + let result = tracker.partition_filters(filters.clone(), 1000, &metadata); + assert_eq!(result.row_filters.len(), 1); + assert_eq!(result.post_scan.len(), 0); + + // Feed low effectiveness stats + for _ in 0..5 { + tracker.update(1, 100, 100, 100_000, 1000); // all rows matched, no pruning + } + + // Second partition: should be demoted to PostScan + let result = tracker.partition_filters(filters, 1000, &metadata); + assert_eq!(result.row_filters.len(), 0); + assert_eq!(result.post_scan.len(), 1); + } + + #[test] + fn test_demotion_resets_stats() { + let tracker = TrackerConfig::new() + .with_min_bytes_per_sec(10000.0) + .with_byte_ratio_threshold(0.1) + .with_confidence_z(0.5) + .build(); + + let metadata = create_test_metadata(vec![(100, vec![100])]); + let expr = col_expr("a", 0); + let filters = vec![(1, expr.clone())]; + + // Start as RowFilter + tracker.partition_filters(filters.clone(), 1000, &metadata); + + // Add stats + tracker.update(1, 100, 100, 100_000, 1000); + tracker.update(1, 100, 100, 100_000, 1000); + + // Demote + tracker.partition_filters(filters.clone(), 1000, &metadata); + + // Stats should be zeroed after demotion + let stats_map = tracker.filter_stats.read(); + assert_eq!( + *stats_map.get(&1).unwrap().lock(), + SelectivityStats::default() + ); + } + + #[test] + fn test_promotion_resets_stats() { + let tracker = TrackerConfig::new() + .with_min_bytes_per_sec(100.0) + .with_byte_ratio_threshold(0.5) + .with_confidence_z(0.5) + .build(); + + let metadata = create_test_metadata(vec![(100, vec![1000])]); + let expr = col_expr("a", 0); + let filters = vec![(1, expr.clone())]; + + // Start as PostScan + tracker.partition_filters(filters.clone(), 1000, &metadata); + + // Add stats + for _ in 0..3 { + tracker.update(1, 50, 100, 100_000, 1000); + } + + // Promote + tracker.partition_filters(filters.clone(), 1000, &metadata); + + // Stats should be zeroed after promotion + let stats_map = tracker.filter_stats.read(); + assert_eq!( + *stats_map.get(&1).unwrap().lock(), + SelectivityStats::default() + ); + } + + #[test] + fn test_optional_filter_dropping() { + let tracker = TrackerConfig::new() + .with_min_bytes_per_sec(10000.0) + .with_byte_ratio_threshold(0.5) + .with_confidence_z(0.5) + .build(); + + let metadata = create_test_metadata(vec![(100, vec![1000])]); + let expr = col_expr("a", 0); + let filters = vec![(1, expr.clone())]; + + // Start as PostScan + tracker.partition_filters(filters.clone(), 1000, &metadata); + + // Feed poor effectiveness stats + for _ in 0..5 { + tracker.update(1, 100, 100, 100_000, 1000); // no pruning + } + + // Next partition: should stay as PostScan (not dropped because not optional) + let result = tracker.partition_filters(filters, 1000, &metadata); + assert_eq!(result.post_scan.len(), 1); + assert_eq!(result.row_filters.len(), 0); + } + + #[test] + fn test_persistent_dropped_state() { + let tracker = TrackerConfig::new() + .with_min_bytes_per_sec(10000.0) + .with_byte_ratio_threshold(0.5) + .build(); + + let metadata = create_test_metadata(vec![(100, vec![1000])]); + let expr = col_expr("a", 0); + let filters = vec![(1, expr.clone())]; + + // Mark filter as dropped by manually setting state + tracker + .inner + .lock() + .filter_states + .insert(1, FilterState::Dropped); + + // On next partition, dropped filters should not reappear + let result = tracker.partition_filters(filters, 1000, &metadata); + assert_eq!(result.row_filters.len(), 0); + assert_eq!(result.post_scan.len(), 0); + } + } + + mod filter_ordering_tests { + use super::helper_functions::*; + use super::*; + + #[test] + fn test_filters_get_partitioned() { + let tracker = TrackerConfig::new() + .with_min_bytes_per_sec(1.0) // Very low threshold + .build(); + + let metadata = create_test_metadata(vec![(100, vec![100, 100, 100])]); + let filters = vec![ + (1, col_expr("a", 0)), + (2, col_expr("a", 1)), + (3, col_expr("a", 2)), + ]; + + // Partition should process all filters + let result = tracker.partition_filters(filters.clone(), 1000, &metadata); + + // With min_bytes_per_sec=1.0, filters should be partitioned + assert!(result.row_filters.len() + result.post_scan.len() > 0); + + // Add stats and partition again + tracker.update(1, 60, 100, 1_000_000, 100); + tracker.update(2, 10, 100, 1_000_000, 100); + tracker.update(3, 40, 100, 1_000_000, 100); + + let result2 = tracker.partition_filters(filters, 1000, &metadata); + + // Filters should still be partitioned + assert!(result2.row_filters.len() + result2.post_scan.len() > 0); + } + + #[test] + fn test_filters_processed_without_stats() { + let tracker = TrackerConfig::new() + .with_min_bytes_per_sec(1.0) // Very low threshold + .build(); + + // Different column sizes: 300, 200, 100 bytes + let metadata = create_test_metadata(vec![(100, vec![300, 200, 100])]); + let filters = vec![ + (1, col_expr("a", 0)), + (2, col_expr("a", 1)), + (3, col_expr("a", 2)), + ]; + + // First partition - no stats yet + let result = tracker.partition_filters(filters.clone(), 1000, &metadata); + + // All filters should be processed (partitioned into row/post-scan) + assert!(result.row_filters.len() + result.post_scan.len() > 0); + + // Filters should be consistent on repeated calls + let result2 = tracker.partition_filters(filters, 1000, &metadata); + assert_eq!( + result.row_filters.len() + result.post_scan.len(), + result2.row_filters.len() + result2.post_scan.len() + ); + } + + #[test] + fn test_filters_with_partial_stats() { + let tracker = TrackerConfig::new().with_min_bytes_per_sec(1.0).build(); + + // Give filter 2 larger bytes so it's prioritized when falling back to byte ratio + let metadata = create_test_metadata(vec![(100, vec![100, 300, 100])]); + let filters = vec![ + (1, col_expr("a", 0)), + (2, col_expr("a", 1)), + (3, col_expr("a", 2)), + ]; + + // First partition + let result1 = tracker.partition_filters(filters.clone(), 1000, &metadata); + assert!(result1.row_filters.len() + result1.post_scan.len() > 0); + + // Only add stats for filters 1 and 3, not 2 + tracker.update(1, 60, 100, 1_000_000, 100); + tracker.update(3, 60, 100, 1_000_000, 100); + + // Second partition with partial stats + let result2 = tracker.partition_filters(filters, 1000, &metadata); + assert!(result2.row_filters.len() + result2.post_scan.len() > 0); + } + + #[test] + fn test_ordering_stability_with_identical_values() { + let tracker = TrackerConfig::new().with_min_bytes_per_sec(0.0).build(); + + let metadata = create_test_metadata(vec![(100, vec![100, 100, 100])]); + let filters = vec![ + (1, col_expr("a", 0)), + (2, col_expr("a", 1)), + (3, col_expr("a", 2)), + ]; + + let result1 = tracker.partition_filters(filters.clone(), 1000, &metadata); + let result2 = tracker.partition_filters(filters, 1000, &metadata); + + // Without stats and with identical byte sizes, order should be stable + assert_eq!(result1.row_filters[0].0, result2.row_filters[0].0); + assert_eq!(result1.row_filters[1].0, result2.row_filters[1].0); + assert_eq!(result1.row_filters[2].0, result2.row_filters[2].0); + } + } + + mod dynamic_filter_tests { + use super::helper_functions::*; + use super::*; + + #[test] + fn test_generation_zero_ignored() { + let tracker = TrackerConfig::new() + .with_min_bytes_per_sec(1000.0) + .with_byte_ratio_threshold(0.5) + .build(); + + let metadata = create_test_metadata(vec![(100, vec![1000])]); + + // Create two filters with same ID but generation 0 and 1 + // Generation 0 should be ignored + let expr1 = col_expr("a", 0); + let filters1 = vec![(1, expr1)]; + + tracker.partition_filters(filters1, 1000, &metadata); + tracker.update(1, 50, 100, 100_000, 1000); + + // Generation 0 doesn't trigger state reset + let snapshot_gen = tracker.inner.lock().snapshot_generations.get(&1).copied(); + assert_eq!(snapshot_gen, None); + } + + #[test] + fn test_generation_change_clears_stats() { + let tracker = TrackerConfig::new() + .with_min_bytes_per_sec(1000.0) + .with_byte_ratio_threshold(0.5) + .build(); + + // Pre-populate stats entry so update() can find it + tracker.ensure_stats_entry(1); + + // Initialize generation to 100 + { + let mut inner = tracker.inner.lock(); + let stats = tracker.filter_stats.read(); + inner.note_generation(1, 100, &stats); + } + + // Add stats + tracker.update(1, 50, 100, 100_000, 1000); + tracker.update(1, 50, 100, 100_000, 1000); + + let stats_before = { + let stats_map = tracker.filter_stats.read(); + *stats_map.get(&1).unwrap().lock() != SelectivityStats::default() + }; + assert!(stats_before); + + // Simulate generation change to a different value + { + let mut inner = tracker.inner.lock(); + let stats = tracker.filter_stats.read(); + inner.note_generation(1, 101, &stats); + } + + // Stats should be zeroed on generation change + let stats_after = { + let stats_map = tracker.filter_stats.read(); + *stats_map.get(&1).unwrap().lock() == SelectivityStats::default() + }; + assert!(stats_after); + } + + #[test] + fn test_generation_unchanged_preserves_stats() { + let tracker = TrackerConfig::new().with_min_bytes_per_sec(1000.0).build(); + + // Pre-populate stats entry so update() can find it + tracker.ensure_stats_entry(1); + + // Manually set generation + { + let mut inner = tracker.inner.lock(); + let stats = tracker.filter_stats.read(); + inner.note_generation(1, 100, &stats); + } + + // Add stats + tracker.update(1, 50, 100, 100_000, 1000); + tracker.update(1, 50, 100, 100_000, 1000); + + let sample_count_before = { + let stats_map = tracker.filter_stats.read(); + stats_map.get(&1).map(|s| s.lock().sample_count) + }; + assert_eq!(sample_count_before, Some(2)); + + // Call note_generation with same generation + { + let mut inner = tracker.inner.lock(); + let stats = tracker.filter_stats.read(); + inner.note_generation(1, 100, &stats); + } + + // Stats should be preserved + let sample_count_after = { + let stats_map = tracker.filter_stats.read(); + stats_map.get(&1).map(|s| s.lock().sample_count) + }; + assert_eq!(sample_count_after, Some(2)); + } + + #[test] + fn test_generation_change_preserves_state() { + let tracker = TrackerConfig::new() + .with_min_bytes_per_sec(1000.0) + .with_byte_ratio_threshold(0.1) + .build(); + + let metadata = create_test_metadata(vec![(100, vec![100])]); + + // First partition: goes to RowFilter + let expr = col_expr("a", 0); + let filters = vec![(1, expr)]; + tracker.partition_filters(filters.clone(), 1000, &metadata); + + let state_before = tracker.inner.lock().filter_states.get(&1).copied(); + assert_eq!(state_before, Some(FilterState::RowFilter)); + + // Simulate generation change + { + let mut inner = tracker.inner.lock(); + let stats = tracker.filter_stats.read(); + inner.note_generation(1, 100, &stats); + } + + // State should be preserved despite stats being cleared + let state_after = tracker.inner.lock().filter_states.get(&1).copied(); + assert_eq!(state_after, Some(FilterState::RowFilter)); + } + + #[test] + fn test_generation_change_undrops_dropped_filter() { + let tracker = TrackerConfig::new() + .with_min_bytes_per_sec(1000.0) + .with_byte_ratio_threshold(0.1) + .build(); + + // Manually set filter state to Dropped + tracker + .inner + .lock() + .filter_states + .insert(1, FilterState::Dropped); + { + let mut inner = tracker.inner.lock(); + let stats = tracker.filter_stats.read(); + inner.note_generation(1, 100, &stats); + } + + // Simulate generation change + { + let mut inner = tracker.inner.lock(); + let stats = tracker.filter_stats.read(); + inner.note_generation(1, 101, &stats); + } + + // Dropped filter should be un-dropped to PostScan + let state_after = tracker.inner.lock().filter_states.get(&1).copied(); + assert_eq!(state_after, Some(FilterState::PostScan)); + } + + #[test] + fn test_multiple_filters_independent_generation_tracking() { + let tracker = TrackerConfig::new().with_min_bytes_per_sec(1000.0).build(); + + // Pre-populate stats entries so update() can find them + tracker.ensure_stats_entry(1); + tracker.ensure_stats_entry(2); + + // Set generations for multiple filters + { + let mut inner = tracker.inner.lock(); + let stats = tracker.filter_stats.read(); + inner.note_generation(1, 100, &stats); + inner.note_generation(2, 200, &stats); + } + + // Add stats to both + tracker.update(1, 50, 100, 100_000, 1000); + tracker.update(2, 50, 100, 100_000, 1000); + + // Change generation of filter 1 only + { + let mut inner = tracker.inner.lock(); + let stats = tracker.filter_stats.read(); + inner.note_generation(1, 101, &stats); + } + + // Filter 1 stats should be zeroed, filter 2 preserved + let stats_map = tracker.filter_stats.read(); + assert_eq!( + *stats_map.get(&1).unwrap().lock(), + SelectivityStats::default() + ); + assert_ne!( + *stats_map.get(&2).unwrap().lock(), + SelectivityStats::default() + ); + } + } + + mod integration_tests { + use super::helper_functions::*; + use super::*; + + #[test] + fn test_full_promotion_lifecycle() { + let tracker = TrackerConfig::new() + .with_min_bytes_per_sec(500.0) + .with_byte_ratio_threshold(0.5) // Force initial PostScan + .with_confidence_z(0.5) + .build(); + + let metadata = create_test_metadata(vec![(100, vec![1000])]); + let expr = col_expr("a", 0); + let filters = vec![(1, expr.clone())]; + + // Step 1: Initial placement (PostScan) + let result = tracker.partition_filters(filters.clone(), 1000, &metadata); + assert_eq!(result.post_scan.len(), 1); + assert_eq!(result.row_filters.len(), 0); + + // Step 2: Accumulate high effectiveness stats + for _ in 0..5 { + tracker.update(1, 10, 100, 100_000, 1000); // high effectiveness + } + + // Step 3: Promotion should occur + let result = tracker.partition_filters(filters.clone(), 1000, &metadata); + assert_eq!(result.row_filters.len(), 1); + assert_eq!(result.post_scan.len(), 0); + + // Step 4: Continue to partition without additional updates + let result = tracker.partition_filters(filters, 1000, &metadata); + assert_eq!(result.row_filters.len(), 1); + assert_eq!(result.post_scan.len(), 0); + } + + #[test] + fn test_full_demotion_lifecycle() { + let tracker = TrackerConfig::new() + .with_min_bytes_per_sec(10000.0) + .with_byte_ratio_threshold(0.1) // Force initial RowFilter + .with_confidence_z(0.5) + .build(); + + let metadata = create_test_metadata(vec![(100, vec![100])]); + let expr = col_expr("a", 0); + let filters = vec![(1, expr.clone())]; + + // Step 1: Initial placement (RowFilter) + let result = tracker.partition_filters(filters.clone(), 1000, &metadata); + assert_eq!(result.row_filters.len(), 1); + assert_eq!(result.post_scan.len(), 0); + + // Step 2: Accumulate low effectiveness stats + for _ in 0..5 { + tracker.update(1, 100, 100, 100_000, 1000); // no pruning + } + + // Step 3: Demotion should occur + let result = tracker.partition_filters(filters.clone(), 1000, &metadata); + assert_eq!(result.row_filters.len(), 0); + assert_eq!(result.post_scan.len(), 1); + + // Step 4: Continue to partition without additional updates + let result = tracker.partition_filters(filters, 1000, &metadata); + assert_eq!(result.row_filters.len(), 0); + assert_eq!(result.post_scan.len(), 1); + } + + #[test] + fn test_multiple_filters_mixed_states() { + let tracker = TrackerConfig::new() + .with_min_bytes_per_sec(1000.0) + .with_byte_ratio_threshold(0.4) // Force PostScan initially (500/1000=0.5 > 0.4) + .with_confidence_z(0.5) + .build(); + + let metadata = create_test_metadata(vec![(100, vec![500, 500])]); + let filters = vec![(1, col_expr("a", 0)), (2, col_expr("a", 1))]; + + // Initial partition: both go to PostScan (500/1000 = 0.5 > 0.4) + let result = tracker.partition_filters(filters.clone(), 1000, &metadata); + assert_eq!(result.post_scan.len(), 2); + + // Filter 1: high effectiveness (promote) + for _ in 0..3 { + tracker.update(1, 10, 100, 100_000, 500); + } + + // Filter 2: low effectiveness (stay PostScan) + for _ in 0..3 { + tracker.update(2, 100, 100, 100_000, 500); + } + + // Next partition: Filter 1 promoted, Filter 2 stays PostScan + let result = tracker.partition_filters(filters, 1000, &metadata); + assert_eq!(result.row_filters.len(), 1); + assert_eq!(result.post_scan.len(), 1); + assert_eq!(result.row_filters[0].0, 1); + assert_eq!(result.post_scan[0].0, 2); + } + + #[test] + fn test_empty_filter_list() { + let tracker = TrackerConfig::new().build(); + let metadata = create_test_metadata(vec![(100, vec![1000])]); + let filters = vec![]; + + let result = tracker.partition_filters(filters, 1000, &metadata); + + assert_eq!(result.row_filters.len(), 0); + assert_eq!(result.post_scan.len(), 0); + } + + #[test] + fn test_single_filter() { + let tracker = TrackerConfig::new().with_min_bytes_per_sec(0.0).build(); + + let metadata = create_test_metadata(vec![(100, vec![100])]); + let expr = col_expr("a", 0); + let filters = vec![(1, expr)]; + + let result = tracker.partition_filters(filters, 1000, &metadata); + + assert_eq!(result.row_filters.len(), 1); + assert_eq!(result.post_scan.len(), 0); + } + + #[test] + fn test_zero_effectiveness_stays_at_boundary() { + let tracker = TrackerConfig::new() + .with_min_bytes_per_sec(100.0) + .with_byte_ratio_threshold(0.1) + .with_confidence_z(0.5) + .build(); + + let metadata = create_test_metadata(vec![(100, vec![100])]); + let expr = col_expr("a", 0); + let filters = vec![(1, expr.clone())]; + + // Start as RowFilter + tracker.partition_filters(filters.clone(), 1000, &metadata); + + // All rows match (zero effectiveness) + for _ in 0..5 { + tracker.update(1, 100, 100, 100_000, 100); + } + + // Should demote due to CI upper bound being 0 + let result = tracker.partition_filters(filters, 1000, &metadata); + assert_eq!(result.row_filters.len(), 0); + assert_eq!(result.post_scan.len(), 1); + } + + #[test] + fn test_confidence_z_parameter_stored() { + // Test that different confidence_z values are properly stored in config + let tracker_conservative = TrackerConfig::new() + .with_min_bytes_per_sec(1000.0) + .with_byte_ratio_threshold(0.5) + .with_confidence_z(3.0) // Harder to promote + .build(); + + let tracker_aggressive = TrackerConfig::new() + .with_min_bytes_per_sec(1000.0) + .with_byte_ratio_threshold(0.5) + .with_confidence_z(0.5) // Easier to promote + .build(); + + // Verify configs are stored correctly + assert_eq!(tracker_conservative.config.confidence_z, 3.0); + assert_eq!(tracker_aggressive.config.confidence_z, 0.5); + + // The z-score affects confidence intervals during promotion/demotion decisions. + // With identical stats, higher z requires narrower confidence intervals, + // making promotion harder. With lower z, confidence intervals are wider, + // making promotion easier. This is tested in other integration tests + // that verify actual promotion/demotion behavior. + } + } +} diff --git a/datafusion/datasource-parquet/src/source.rs b/datafusion/datasource-parquet/src/source.rs index a014c8b2726e7..6da11aff5ff9e 100644 --- a/datafusion/datasource-parquet/src/source.rs +++ b/datafusion/datasource-parquet/src/source.rs @@ -39,8 +39,9 @@ use datafusion_common::tree_node::TreeNodeRecursion; use datafusion_datasource::TableSchema; use datafusion_datasource::file::FileSource; use datafusion_datasource::file_scan_config::FileScanConfig; +use datafusion_physical_expr::EquivalenceProperties; +use datafusion_physical_expr::conjunction; use datafusion_physical_expr::projection::ProjectionExprs; -use datafusion_physical_expr::{EquivalenceProperties, conjunction}; use datafusion_physical_expr_adapter::DefaultPhysicalExprAdapterFactory; use datafusion_physical_expr_common::physical_expr::PhysicalExpr; use datafusion_physical_expr_common::physical_expr::fmt_sql; @@ -277,8 +278,10 @@ pub struct ParquetSource { /// In particular, this is the schema of the table without partition columns, /// *not* the physical schema of the file. pub(crate) table_schema: TableSchema, - /// Optional predicate for row filtering during parquet scan - pub(crate) predicate: Option>, + /// Optional predicate conjuncts for row filtering during parquet scan. + /// Each conjunct is tagged with a stable FilterId for selectivity tracking. + pub(crate) predicate_conjuncts: + Option)>>, /// Optional user defined parquet file reader factory pub(crate) parquet_file_reader_factory: Option>, /// Batch size configuration @@ -294,6 +297,10 @@ pub struct ParquetSource { /// so we still need to sort them after reading, so the reverse scan is inexact. /// Used to optimize ORDER BY ... DESC on sorted data. reverse_row_groups: bool, + /// Tracks filter selectivity across files for adaptive filter reordering. + /// Shared across all openers - each opener reads stats and makes its own + /// decision about which filters to push down vs. apply post-scan. + pub(crate) selectivity_tracker: Arc, } impl ParquetSource { @@ -312,13 +319,16 @@ impl ParquetSource { table_schema, table_parquet_options: TableParquetOptions::default(), metrics: ExecutionPlanMetricsSet::new(), - predicate: None, + predicate_conjuncts: None, parquet_file_reader_factory: None, batch_size: None, metadata_size_hint: None, #[cfg(feature = "parquet_encryption")] encryption_factory: None, reverse_row_groups: false, + selectivity_tracker: Arc::new( + crate::selectivity::SelectivityTracker::default(), + ), } } @@ -327,6 +337,15 @@ impl ParquetSource { mut self, table_parquet_options: TableParquetOptions, ) -> Self { + // Update the selectivity tracker from the config + let opts = &table_parquet_options.global; + self.selectivity_tracker = Arc::new( + crate::selectivity::TrackerConfig::new() + .with_min_bytes_per_sec(opts.filter_pushdown_min_bytes_per_sec) + .with_byte_ratio_threshold(opts.filter_collecting_byte_ratio_threshold) + .with_confidence_z(opts.filter_confidence_z) + .build(), + ); self.table_parquet_options = table_parquet_options; self } @@ -342,11 +361,23 @@ impl ParquetSource { self } - /// Set predicate information + /// Set predicate information. + /// + /// The predicate is split into conjuncts and each is assigned a stable + /// `FilterId` (its index in the conjunct list). These IDs are used for + /// selectivity tracking across files, avoiding ExprKey mismatch issues + /// when expressions are rebased or simplified per-file. #[expect(clippy::needless_pass_by_value)] pub fn with_predicate(&self, predicate: Arc) -> Self { + use datafusion_physical_expr::split_conjunction; let mut conf = self.clone(); - conf.predicate = Some(Arc::clone(&predicate)); + let conjuncts: Vec<(crate::selectivity::FilterId, Arc)> = + split_conjunction(&predicate) + .into_iter() + .enumerate() + .map(|(id, expr)| (id, Arc::clone(expr))) + .collect(); + conf.predicate_conjuncts = Some(conjuncts); conf } @@ -367,8 +398,15 @@ impl ParquetSource { /// Optional predicate. #[deprecated(since = "50.2.0", note = "use `filter` instead")] - pub fn predicate(&self) -> Option<&Arc> { - self.predicate.as_ref() + pub fn predicate(&self) -> Option> { + self.combined_predicate() + } + + /// Build a combined predicate from the conjuncts, if any. + fn combined_predicate(&self) -> Option> { + self.predicate_conjuncts + .as_ref() + .map(|conjuncts| conjunction(conjuncts.iter().map(|(_, e)| Arc::clone(e)))) } /// return the optional file reader factory @@ -399,22 +437,6 @@ impl ParquetSource { self.table_parquet_options.global.pushdown_filters } - /// If true, the `RowFilter` made by `pushdown_filters` may try to - /// minimize the cost of filter evaluation by reordering the - /// predicate [`Expr`]s. If false, the predicates are applied in - /// the same order as specified in the query. Defaults to false. - /// - /// [`Expr`]: datafusion_expr::Expr - pub fn with_reorder_filters(mut self, reorder_filters: bool) -> Self { - self.table_parquet_options.global.reorder_filters = reorder_filters; - self - } - - /// Return the value described in [`Self::with_reorder_filters`] - fn reorder_filters(&self) -> bool { - self.table_parquet_options.global.reorder_filters - } - /// Return the value of [`datafusion_common::config::ParquetOptions::force_filter_selections`] fn force_filter_selections(&self) -> bool { self.table_parquet_options.global.force_filter_selections @@ -561,13 +583,13 @@ impl FileSource for ParquetSource { .expect("Batch size must set before creating ParquetMorselizer"), limit: base_config.limit, preserve_order: base_config.preserve_order, - predicate: self.predicate.clone(), + predicate_conjuncts: self.predicate_conjuncts.clone(), + selectivity_tracker: Arc::clone(&self.selectivity_tracker), table_schema: self.table_schema.clone(), metadata_size_hint: self.metadata_size_hint, metrics: self.metrics().clone(), parquet_file_reader_factory, pushdown_filters: self.pushdown_filters(), - reorder_filters: self.reorder_filters(), force_filter_selections: self.force_filter_selections(), enable_page_index: self.enable_page_index(), enable_bloom_filter: self.bloom_filter_on_read(), @@ -588,7 +610,7 @@ impl FileSource for ParquetSource { } fn filter(&self) -> Option> { - self.predicate.clone() + self.combined_predicate() } fn with_batch_size(&self, batch_size: usize) -> Arc { @@ -641,7 +663,7 @@ impl FileSource for ParquetSource { // the actual predicates are built in reference to the physical schema of // each file, which we do not have at this point and hence cannot use. // Instead, we use the logical schema of the file (the table schema without partition columns). - if let Some(predicate) = &self.predicate { + if let Some(predicate) = &self.combined_predicate() { let predicate_creation_errors = Count::new(); if let Some(pruning_predicate) = build_pruning_predicates( Some(predicate), @@ -718,13 +740,16 @@ impl FileSource for ParquetSource { PushedDown::No => None, }) .collect_vec(); - let predicate = match source.predicate { - Some(predicate) => { - conjunction(std::iter::once(predicate).chain(allowed_filters)) - } - None => conjunction(allowed_filters), - }; - source.predicate = Some(predicate); + // Merge existing conjuncts with new allowed filters + let mut all_conjuncts: Vec> = source + .predicate_conjuncts + .as_ref() + .map(|c| c.iter().map(|(_, e)| Arc::clone(e)).collect()) + .unwrap_or_default(); + all_conjuncts.extend(allowed_filters); + // Re-assign FilterIds by index + source.predicate_conjuncts = + Some(all_conjuncts.into_iter().enumerate().collect()); source = source.with_pushdown_filters(pushdown_filters); let source = Arc::new(source); // If pushdown_filters is false we tell our parents that they still have to handle the filters, @@ -835,8 +860,10 @@ impl FileSource for ParquetSource { ) -> datafusion_common::Result { // Visit predicate (filter) expression if present let mut tnr = TreeNodeRecursion::Continue; - if let Some(predicate) = &self.predicate { - tnr = tnr.visit_sibling(|| f(predicate.as_ref()))?; + if let Some(ref conjuncts) = self.predicate_conjuncts { + for (_, expr) in conjuncts { + tnr = tnr.visit_sibling(|| f(expr.as_ref()))?; + } } // Visit projection expressions @@ -861,8 +888,9 @@ mod tests { let parquet_source = ParquetSource::new(Arc::new(Schema::empty())).with_predicate(predicate); - // same value. but filter() call Arc::clone internally - assert_eq!(parquet_source.predicate(), parquet_source.filter().as_ref()); + // Both should return equivalent predicates + assert!(parquet_source.predicate().is_some()); + assert!(parquet_source.filter().is_some()); } #[test] diff --git a/datafusion/datasource/Cargo.toml b/datafusion/datasource/Cargo.toml index 4027521658977..40e2271f45205 100644 --- a/datafusion/datasource/Cargo.toml +++ b/datafusion/datasource/Cargo.toml @@ -64,6 +64,7 @@ itertools = { workspace = true } liblzma = { workspace = true, optional = true } log = { workspace = true } object_store = { workspace = true } +parking_lot = { workspace = true } rand = { workspace = true } tempfile = { workspace = true, optional = true } tokio = { workspace = true } diff --git a/datafusion/datasource/src/file_scan_config/mod.rs b/datafusion/datasource/src/file_scan_config/mod.rs index 3a3277be08003..04b74528d5ac1 100644 --- a/datafusion/datasource/src/file_scan_config/mod.rs +++ b/datafusion/datasource/src/file_scan_config/mod.rs @@ -24,7 +24,8 @@ use crate::file_groups::FileGroup; use crate::{ PartitionedFile, display::FileGroupsDisplay, file::FileSource, file_compression_type::FileCompressionType, file_stream::FileStreamBuilder, - source::DataSource, statistics::MinMaxStatistics, + file_stream::work_source::SharedWorkSource, source::DataSource, + statistics::MinMaxStatistics, }; use arrow::datatypes::FieldRef; use arrow::datatypes::{DataType, Schema, SchemaRef}; @@ -38,6 +39,7 @@ use datafusion_execution::{ }; use datafusion_expr::Operator; +use crate::source::OpenArgs; use datafusion_physical_expr::expressions::{BinaryExpr, Column}; use datafusion_physical_expr::projection::ProjectionExprs; use datafusion_physical_expr::utils::reassign_expr_columns; @@ -55,6 +57,7 @@ use datafusion_physical_plan::{ metrics::ExecutionPlanMetricsSet, }; use log::{debug, warn}; +use std::any::Any; use std::{fmt::Debug, fmt::Formatter, fmt::Result as FmtResult, sync::Arc}; /// [`FileScanConfig`] represents scanning data from a group of files @@ -578,6 +581,15 @@ impl DataSource for FileScanConfig { partition: usize, context: Arc, ) -> Result { + self.open_with_args(OpenArgs::new(partition, context)) + } + + fn open_with_args(&self, args: OpenArgs) -> Result { + let OpenArgs { + partition, + context, + sibling_state, + } = args; let object_store = context.runtime_env().object_store(&self.object_store_url)?; let batch_size = self .batch_size @@ -587,8 +599,17 @@ impl DataSource for FileScanConfig { let morselizer = source.create_morselizer(object_store, self, partition)?; + // Extract the shared work source from the sibling state if it exists. + // This allows multiple sibling streams to steal work from a single + // shared queue of unopened files. + let shared_work_source = sibling_state + .as_ref() + .and_then(|state| state.downcast_ref::()) + .cloned(); + let stream = FileStreamBuilder::new(self) .with_partition(partition) + .with_shared_work_source(shared_work_source) .with_morselizer(morselizer) .with_metrics(source.metrics()) .build()?; @@ -985,6 +1006,20 @@ impl DataSource for FileScanConfig { // Delegate to the file source self.file_source.apply_expressions(f) } + + /// Create any shared state that should be passed between sibling streams + /// during one execution. + /// + /// This returns `None` when sibling streams must not share work, such as + /// when file order must be preserved or the file groups define the output + /// partitioning needed for the rest of the plan + fn create_sibling_state(&self) -> Option> { + if self.preserve_order || self.partitioned_by_file_group { + return None; + } + + Some(Arc::new(SharedWorkSource::from_config(self)) as Arc) + } } impl FileScanConfig { @@ -1362,19 +1397,33 @@ mod tests { use super::*; use crate::TableSchema; + use crate::source::DataSourceExec; use crate::test_util::col; use crate::{ generate_test_files, test_util::MockSource, tests::aggr_test_schema, verify_sort_integrity, }; + use arrow::array::{Int32Array, RecordBatch}; use arrow::datatypes::Field; use datafusion_common::ColumnStatistics; use datafusion_common::stats::Precision; + use datafusion_common::tree_node::TreeNodeRecursion; + use datafusion_common::{Result, assert_batches_eq, internal_err}; + use datafusion_execution::TaskContext; use datafusion_expr::SortExpr; + use datafusion_physical_expr::PhysicalExpr; use datafusion_physical_expr::create_physical_sort_expr; use datafusion_physical_expr::expressions::Literal; use datafusion_physical_expr::projection::ProjectionExpr; + use datafusion_physical_expr::projection::ProjectionExprs; + use datafusion_physical_plan::ExecutionPlan; + use datafusion_physical_plan::execution_plan::collect; + use futures::FutureExt as _; + use futures::StreamExt as _; + use futures::stream; + use object_store::ObjectStore; + use std::fmt::Debug; #[derive(Clone)] struct InexactSortPushdownSource { @@ -1394,7 +1443,7 @@ mod tests { impl FileSource for InexactSortPushdownSource { fn create_file_opener( &self, - _object_store: Arc, + _object_store: Arc, _base_config: &FileScanConfig, _partition: usize, ) -> Result> { @@ -2278,6 +2327,88 @@ mod tests { assert_eq!(partition_stats.total_byte_size, Precision::Exact(800)); } + /// Regression test for reusing a `DataSourceExec` after its execution-local + /// shared work queue has been drained. + /// + /// This test uses a single file group with two files so the scan creates a + /// shared unopened-file queue. Executing after `reset_state` must recreate + /// the shared queue and return the same rows again. + #[tokio::test] + async fn reset_state_recreates_shared_work_source() -> Result<()> { + let schema = Arc::new(Schema::new(vec![Field::new( + "value", + DataType::Int32, + false, + )])); + let file_source = Arc::new( + MockSource::new(Arc::clone(&schema)) + .with_file_opener(Arc::new(ResetStateTestFileOpener { schema })), + ); + + let config = + FileScanConfigBuilder::new(ObjectStoreUrl::local_filesystem(), file_source) + .with_file_group(FileGroup::new(vec![ + PartitionedFile::new("file1.parquet", 100), + PartitionedFile::new("file2.parquet", 100), + ])) + .build(); + + let exec: Arc = DataSourceExec::from_data_source(config); + let task_ctx = Arc::new(TaskContext::default()); + + // Running the same scan after resetting the state, should + // produce the same answer. + let first_run = collect(Arc::clone(&exec), Arc::clone(&task_ctx)).await?; + let reset_exec = exec.reset_state()?; + let second_run = collect(reset_exec, task_ctx).await?; + + let expected = [ + "+-------+", + "| value |", + "+-------+", + "| 1 |", + "| 2 |", + "+-------+", + ]; + assert_batches_eq!(expected, &first_run); + assert_batches_eq!(expected, &second_run); + + Ok(()) + } + + /// Test-only `FileOpener` that turns file names like `file1.parquet` into a + /// single-batch stream containing that numeric value + #[derive(Debug)] + struct ResetStateTestFileOpener { + schema: SchemaRef, + } + + impl crate::file_stream::FileOpener for ResetStateTestFileOpener { + fn open( + &self, + file: PartitionedFile, + ) -> Result { + let value = file + .object_meta + .location + .as_ref() + .trim_start_matches("file") + .trim_end_matches(".parquet") + .parse::() + .expect("invalid test file name"); + let schema = Arc::clone(&self.schema); + Ok(async move { + let batch = RecordBatch::try_new( + schema, + vec![Arc::new(Int32Array::from(vec![value]))], + ) + .expect("test batch should be valid"); + Ok(stream::iter(vec![Ok(batch)]).boxed()) + } + .boxed()) + } + } + #[test] fn test_output_partitioning_not_partitioned_by_file_group() { let file_schema = aggr_test_schema(); @@ -2461,7 +2592,7 @@ mod tests { impl FileSource for ExactSortPushdownSource { fn create_file_opener( &self, - _object_store: Arc, + _object_store: Arc, _base_config: &FileScanConfig, _partition: usize, ) -> Result> { diff --git a/datafusion/datasource/src/file_stream/builder.rs b/datafusion/datasource/src/file_stream/builder.rs index efe9c39ce3b38..7034e902550a9 100644 --- a/datafusion/datasource/src/file_stream/builder.rs +++ b/datafusion/datasource/src/file_stream/builder.rs @@ -19,6 +19,7 @@ use std::sync::Arc; use crate::file_scan_config::FileScanConfig; use crate::file_stream::scan_state::ScanState; +use crate::file_stream::work_source::{SharedWorkSource, WorkSource}; use crate::morsel::{FileOpenerMorselizer, Morselizer}; use datafusion_common::{Result, internal_err}; use datafusion_physical_plan::metrics::{BaselineMetrics, ExecutionPlanMetricsSet}; @@ -33,10 +34,11 @@ pub struct FileStreamBuilder<'a> { morselizer: Option>, metrics: Option<&'a ExecutionPlanMetricsSet>, on_error: OnError, + shared_work_source: Option, } impl<'a> FileStreamBuilder<'a> { - /// Create a new builder. + /// Create a new builder for [`FileStream`]. pub fn new(config: &'a FileScanConfig) -> Self { Self { config, @@ -44,6 +46,7 @@ impl<'a> FileStreamBuilder<'a> { morselizer: None, metrics: None, on_error: OnError::Fail, + shared_work_source: None, } } @@ -81,6 +84,15 @@ impl<'a> FileStreamBuilder<'a> { self } + /// Configure the [`SharedWorkSource`] for sibling work stealing. + pub(crate) fn with_shared_work_source( + mut self, + shared_work_source: Option, + ) -> Self { + self.shared_work_source = shared_work_source; + self + } + /// Build the configured [`FileStream`]. pub fn build(self) -> Result { let Self { @@ -89,6 +101,7 @@ impl<'a> FileStreamBuilder<'a> { morselizer, metrics, on_error, + shared_work_source, } = self; let Some(partition) = partition else { @@ -106,10 +119,14 @@ impl<'a> FileStreamBuilder<'a> { "FileStreamBuilder invalid partition index: {partition}" ); }; + let work_source = match shared_work_source { + Some(shared) => WorkSource::Shared(shared), + None => WorkSource::Local(file_group.into_inner().into()), + }; let file_stream_metrics = FileStreamMetrics::new(metrics, partition); let scan_state = Box::new(ScanState::new( - file_group.into_inner(), + work_source, config.limit, morselizer, on_error, diff --git a/datafusion/datasource/src/file_stream/mod.rs b/datafusion/datasource/src/file_stream/mod.rs index ff71f16023080..e277690cff810 100644 --- a/datafusion/datasource/src/file_stream/mod.rs +++ b/datafusion/datasource/src/file_stream/mod.rs @@ -24,6 +24,7 @@ mod builder; mod metrics; mod scan_state; +pub(crate) mod work_source; use std::pin::Pin; use std::sync::Arc; @@ -175,6 +176,7 @@ mod tests { IoFutureId, MockMorselizer, MockPlanBuilder, MockPlanner, MorselId, PendingPlannerBuilder, PollsToResolve, }; + use crate::source::DataSource; use crate::tests::make_partition; use crate::{PartitionedFile, TableSchema}; use arrow::array::{AsArray, RecordBatch}; @@ -184,14 +186,22 @@ mod tests { use datafusion_execution::object_store::ObjectStoreUrl; use datafusion_physical_plan::metrics::ExecutionPlanMetricsSet; use futures::{FutureExt as _, StreamExt as _}; + use std::collections::{BTreeMap, VecDeque}; use std::sync::Arc; use std::sync::atomic::{AtomicUsize, Ordering}; - use crate::file_stream::{FileOpenFuture, FileOpener, FileStreamBuilder, OnError}; + use crate::file_stream::{ + FileOpenFuture, FileOpener, FileStream, FileStreamBuilder, OnError, + work_source::SharedWorkSource, + }; use crate::test_util::MockSource; use datafusion_common::{assert_batches_eq, exec_err, internal_err}; + /// Test identifier for one `FileStream` partition. + #[derive(Debug, Clone, Copy, PartialEq, Eq, PartialOrd, Ord, Hash)] + struct PartitionId(usize); + /// Test `FileOpener` which will simulate errors during file opening or scanning #[derive(Default)] struct TestOpener { @@ -758,8 +768,8 @@ mod tests { async fn morsel_two_ios_one_batch() -> Result<()> { let test = FileStreamMorselTest::new().with_file( MockPlanner::builder("file1.parquet") - .add_plan(PendingPlannerBuilder::new(IoFutureId(1)).build()) - .add_plan(PendingPlannerBuilder::new(IoFutureId(2)).build()) + .add_plan(PendingPlannerBuilder::new(IoFutureId(1))) + .add_plan(PendingPlannerBuilder::new(IoFutureId(2))) .add_plan(MockPlanBuilder::new().with_morsel(MorselId(10), 42)) .return_none(), ); @@ -871,8 +881,7 @@ mod tests { async fn morsel_ready_child_planner() -> Result<()> { let child_planner = MockPlanner::builder("child planner") .add_plan(MockPlanBuilder::new().with_morsel(MorselId(10), 42)) - .return_none() - .build(); + .return_none(); let test = FileStreamMorselTest::new().with_file( MockPlanner::builder("file1.parquet") @@ -1001,11 +1010,331 @@ mod tests { Ok(()) } - /// Tests how FileStream opens and processes files. + /// Return a morsel test with two partitions: + /// Partition 0: file1, file2, file3 + /// Partition 1: file4 + /// + /// Partition 1 has only 1 file but it polled first 4 times + fn two_partition_morsel_test() -> FileStreamMorselTest { + FileStreamMorselTest::new() + // Partition 0 has three files + .with_file_in_partition( + PartitionId(0), + MockPlanner::builder("file1.parquet") + .add_plan(MockPlanBuilder::new().with_morsel(MorselId(10), 101)) + .return_none(), + ) + .with_file_in_partition( + PartitionId(0), + MockPlanner::builder("file2.parquet") + .add_plan(MockPlanBuilder::new().with_morsel(MorselId(11), 102)) + .return_none(), + ) + .with_file_in_partition( + PartitionId(0), + MockPlanner::builder("file3.parquet") + .add_plan(MockPlanBuilder::new().with_morsel(MorselId(12), 103)) + .return_none(), + ) + // Partition 1 has only one file, but is polled first + .with_file_in_partition( + PartitionId(1), + MockPlanner::builder("file4.parquet") + .add_plan(MockPlanBuilder::new().with_morsel(MorselId(13), 201)) + .return_none(), + ) + .with_reads(vec![ + PartitionId(1), + PartitionId(1), + PartitionId(1), + PartitionId(1), + PartitionId(1), + ]) + } + + /// Verifies that an idle sibling stream can steal shared files from + /// another stream once it exhausts its own local work. + #[tokio::test] + async fn morsel_shared_files_can_be_stolen() -> Result<()> { + let test = two_partition_morsel_test().with_file_stream_events(false); + + // Partition 0 starts with 3 files, but Partition 1 is polled first. + // Since Partition 1 is polled first, it will run all the files even those + // that were assigned to Partition 0. + insta::assert_snapshot!(test.run().await.unwrap(), @r" + ----- Partition 0 ----- + Done + ----- Partition 1 ----- + Batch: 101 + Batch: 102 + Batch: 103 + Batch: 201 + Done + ----- File Stream Events ----- + (omitted due to with_file_stream_events(false)) + "); + + Ok(()) + } + + /// Verifies that a stream that must preserve order keeps its files local + /// and therefore cannot steal from a sibling shared queue. + #[tokio::test] + async fn morsel_preserve_order_keeps_files_local() -> Result<()> { + // same fixture as `morsel_shared_files_can_be_stolen` but marked as + // preserve-order + let test = two_partition_morsel_test() + .with_preserve_order(true) + .with_file_stream_events(false); + + // Even though that Partition 1 is polled first, it can not steal files + // from partition 0. The three files originally assigned to Partition 0 + // must be evaluated by Partition 0. + insta::assert_snapshot!(test.run().await.unwrap(), @r" + ----- Partition 0 ----- + Batch: 101 + Batch: 102 + Batch: 103 + Done + ----- Partition 1 ----- + Batch: 201 + Done + ----- File Stream Events ----- + (omitted due to with_file_stream_events(false)) + "); + + Ok(()) + } + + /// Verifies that `partitioned_by_file_group` disables shared work stealing. + #[tokio::test] + async fn morsel_partitioned_by_file_group_keeps_files_local() -> Result<()> { + // same fixture as `morsel_shared_files_can_be_stolen` but marked as + // preserve-partitioned + let test = two_partition_morsel_test() + .with_partitioned_by_file_group(true) + .with_file_stream_events(false); + + insta::assert_snapshot!(test.run().await.unwrap(), @r" + ----- Partition 0 ----- + Batch: 101 + Batch: 102 + Batch: 103 + Done + ----- Partition 1 ----- + Batch: 201 + Done + ----- File Stream Events ----- + (omitted due to with_file_stream_events(false)) + "); + + Ok(()) + } + + /// Verifies that an empty sibling can immediately steal shared files when + /// it is polled before the stream that originally owned them. + #[tokio::test] + async fn morsel_empty_sibling_can_steal() -> Result<()> { + let test = FileStreamMorselTest::new() + .with_file_in_partition( + PartitionId(0), + MockPlanner::builder("file1.parquet") + .add_plan(MockPlanBuilder::new().with_morsel(MorselId(10), 101)) + .return_none(), + ) + .with_file_in_partition( + PartitionId(0), + MockPlanner::builder("file2.parquet") + .add_plan(MockPlanBuilder::new().with_morsel(MorselId(11), 102)) + .return_none(), + ) + // Poll the empty sibling first so it steals both files. + .with_reads(vec![PartitionId(1), PartitionId(1), PartitionId(1)]) + .with_file_stream_events(false); + + insta::assert_snapshot!(test.run().await.unwrap(), @r" + ----- Partition 0 ----- + Done + ----- Partition 1 ----- + Batch: 101 + Batch: 102 + Done + ----- File Stream Events ----- + (omitted due to with_file_stream_events(false)) + "); + + Ok(()) + } + + /// Ensures that if a sibling is built and polled + /// before another sibling has been built and contributed its files to the + /// shared queue, the first sibling does not finish prematurely. + #[tokio::test] + async fn morsel_empty_sibling_can_finish_before_shared_work_exists() -> Result<()> { + let test = FileStreamMorselTest::new() + .with_file_in_partition( + PartitionId(0), + MockPlanner::builder("file1.parquet") + .add_plan(MockPlanBuilder::new().with_morsel(MorselId(10), 101)) + .return_none(), + ) + .with_file_in_partition( + PartitionId(0), + MockPlanner::builder("file2.parquet") + .add_plan(MockPlanBuilder::new().with_morsel(MorselId(11), 102)) + .return_none(), + ) + // Build streams lazily so partition 1 can poll the shared queue + // before partition 0 has contributed its files. Once partition 0 + // is built, a later poll of partition 1 can still steal one of + // them from the shared queue. + .with_build_streams_on_first_read(true) + .with_reads(vec![PartitionId(1), PartitionId(0), PartitionId(1)]) + .with_file_stream_events(false); + + // Partition 1 polls too early once, then later steals one file after + // partition 0 has populated the shared queue. + insta::assert_snapshot!(test.run().await.unwrap(), @r" + ----- Partition 0 ----- + Batch: 102 + Done + ----- Partition 1 ----- + Batch: 101 + Done + ----- File Stream Events ----- + (omitted due to with_file_stream_events(false)) + "); + + Ok(()) + } + + /// Verifies that a sibling hitting its limit does not count shared files + /// left in the queue as already processed by that stream. + #[tokio::test] + async fn morsel_shared_limit_does_not_double_count_files_processed() -> Result<()> { + let test = two_partition_morsel_test(); + let unlimited_config = test.test_config(); + let limited_config = test.clone().with_limit(1).test_config(); + let shared_work_source = limited_config + .create_sibling_state() + .and_then(|state| state.as_ref().downcast_ref::().cloned()) + .expect("shared work source"); + let limited_metrics = ExecutionPlanMetricsSet::new(); + let unlimited_metrics = ExecutionPlanMetricsSet::new(); + + let limited_stream = FileStreamBuilder::new(&limited_config) + .with_partition(1) + .with_shared_work_source(Some(shared_work_source.clone())) + .with_morselizer(Box::new(test.morselizer.clone())) + .with_metrics(&limited_metrics) + .build()?; + + let unlimited_stream = FileStreamBuilder::new(&unlimited_config) + .with_partition(0) + .with_shared_work_source(Some(shared_work_source)) + .with_morselizer(Box::new(test.morselizer)) + .with_metrics(&unlimited_metrics) + .build()?; + + let limited_output = drain_stream_output(limited_stream).await?; + let unlimited_output = drain_stream_output(unlimited_stream).await?; + + insta::assert_snapshot!(format!( + "----- Limited Stream -----\n{limited_output}\n----- Unlimited Stream -----\n{unlimited_output}" + ), @r" + ----- Limited Stream ----- + Batch: 101 + ----- Unlimited Stream ----- + Batch: 102 + Batch: 103 + Batch: 201 + "); + + assert_eq!( + metric_count(&limited_metrics, "files_opened"), + 1, + "the limited stream should only open the file that produced its output" + ); + assert_eq!( + metric_count(&limited_metrics, "files_processed"), + 1, + "the limited stream should only mark its own file as processed" + ); + assert_eq!( + metric_count(&unlimited_metrics, "files_opened"), + 3, + "the draining stream should open the remaining shared files" + ); + assert_eq!( + metric_count(&unlimited_metrics, "files_processed"), + 3, + "the draining stream should process exactly the files it opened" + ); + + Ok(()) + } + + /// Verifies that one fast sibling can drain shared files that originated + /// in more than one other partition. + #[tokio::test] + async fn morsel_one_sibling_can_drain_multiple_siblings() -> Result<()> { + let test = FileStreamMorselTest::new() + .with_file_in_partition( + PartitionId(0), + MockPlanner::builder("file1.parquet") + .add_plan(MockPlanBuilder::new().with_morsel(MorselId(10), 101)) + .return_none(), + ) + // Partition 1 has two files + .with_file_in_partition( + PartitionId(1), + MockPlanner::builder("file2.parquet") + .add_plan(MockPlanBuilder::new().with_morsel(MorselId(11), 102)) + .return_none(), + ) + .with_file_in_partition( + PartitionId(1), + MockPlanner::builder("file3.parquet") + .add_plan(MockPlanBuilder::new().with_morsel(MorselId(12), 103)) + .return_none(), + ) + // Partition 2 starts empty but is polled first, so it should drain + // the shared queue across both sibling partitions. + .with_reads(vec![ + PartitionId(2), + PartitionId(2), + PartitionId(1), + PartitionId(2), + ]) + .with_file_stream_events(false); + + insta::assert_snapshot!(test.run().await.unwrap(), @r" + ----- Partition 0 ----- + Done + ----- Partition 1 ----- + Batch: 103 + Done + ----- Partition 2 ----- + Batch: 101 + Batch: 102 + Done + ----- File Stream Events ----- + (omitted due to with_file_stream_events(false)) + "); + + Ok(()) + } + + /// Tests how one or more `FileStream`s consume morselized file work. #[derive(Clone)] struct FileStreamMorselTest { morselizer: MockMorselizer, - file_names: Vec, + partition_files: BTreeMap>, + preserve_order: bool, + partitioned_by_file_group: bool, + file_stream_events: bool, + build_streams_on_first_read: bool, + reads: Vec, limit: Option, } @@ -1014,75 +1343,238 @@ mod tests { fn new() -> Self { Self { morselizer: MockMorselizer::new(), - file_names: vec![], + partition_files: BTreeMap::new(), + preserve_order: false, + partitioned_by_file_group: false, + file_stream_events: true, + build_streams_on_first_read: false, + reads: vec![], limit: None, } } - /// Adds one file and its root planner to the test input. - fn with_file(mut self, planner: impl Into) -> Self { + /// Adds one file and its root planner to partition 0. + fn with_file(self, planner: impl Into) -> Self { + self.with_file_in_partition(PartitionId(0), planner) + } + + /// Adds one file and its root planner to the specified input partition. + fn with_file_in_partition( + mut self, + partition: PartitionId, + planner: impl Into, + ) -> Self { let planner = planner.into(); - self.file_names.push(planner.file_path().to_string()); - self.morselizer = self.morselizer.with_file(planner); + let file_path = planner.file_path().to_string(); + self.morselizer = self.morselizer.with_planner(planner); + self.partition_files + .entry(partition) + .or_default() + .push(file_path); self } - /// Sets a global output limit for the stream. + /// Marks the stream (and all partitions) to preserve the specified file + /// order. + fn with_preserve_order(mut self, preserve_order: bool) -> Self { + self.preserve_order = preserve_order; + self + } + + /// Marks the test scan as pre-partitioned by file group, which should + /// force each stream to keep its own files local. + fn with_partitioned_by_file_group( + mut self, + partitioned_by_file_group: bool, + ) -> Self { + self.partitioned_by_file_group = partitioned_by_file_group; + self + } + + /// Controls whether scheduler events are included in the snapshot. + /// + /// When disabled, `run()` still includes the event section header but + /// replaces the trace with a fixed placeholder so tests can focus only + /// on the output batches. + fn with_file_stream_events(mut self, file_stream_events: bool) -> Self { + self.file_stream_events = file_stream_events; + self + } + + /// Controls whether streams are all built up front or lazily on their + /// first read. + /// + /// The default builds all streams before polling begins, which matches + /// normal execution. Tests may enable lazy creation to model races + /// where one sibling polls before another has contributed its files to + /// the shared queue. + fn with_build_streams_on_first_read( + mut self, + build_streams_on_first_read: bool, + ) -> Self { + self.build_streams_on_first_read = build_streams_on_first_read; + self + } + + /// Sets the partition polling order. + /// + /// `run()` polls these partitions in the listed order first. After + /// those explicit reads are exhausted, it completes to round + /// robin across all configured partitions, skipping any streams that + /// have already finished. + /// + /// This allows testing early scheduling decisions explicit in a test + /// while avoiding a fully scripted poll trace for the remainder. + fn with_reads(mut self, reads: Vec) -> Self { + self.reads = reads; + self + } + + /// Sets a global output limit for all streams created by this test. fn with_limit(mut self, limit: usize) -> Self { self.limit = Some(limit); self } - /// Runs the test returns combined output and scheduler trace text as a String. + /// Runs the test and returns combined stream output and scheduler + /// trace text. async fn run(self) -> Result { let observer = self.morselizer.observer().clone(); observer.clear(); - let config = self.test_config(); let metrics_set = ExecutionPlanMetricsSet::new(); - let mut stream = FileStreamBuilder::new(&config) - .with_partition(0) - .with_morselizer(Box::new(self.morselizer)) - .with_metrics(&metrics_set) - .build()?; + let partition_count = self.num_partitions(); - let mut stream_contents = Vec::new(); - while let Some(result) = stream.next().await { - match result { - Ok(batch) => { - let col = batch.column(0).as_primitive::(); - let batch_id = col.value(0); - stream_contents.push(format!("Batch: {batch_id}")); - } - Err(e) => { - // Pull the actual message for external errors rather than - // relying on DataFusionError formatting, which changes - // if backtraces are enabled, etc - let message = if let DataFusionError::External(generic) = e { - generic.to_string() - } else { - e.to_string() - }; - stream_contents.push(format!("Error: {message}")); - } + let mut partitions = (0..partition_count) + .map(|_| PartitionState::new()) + .collect::>(); + + let mut build_order = Vec::new(); + for partition in self.reads.iter().map(|partition| partition.0) { + if !build_order.contains(&partition) { + build_order.push(partition); + } + } + for partition in 0..partition_count { + if !build_order.contains(&partition) { + build_order.push(partition); } } - stream_contents.push("Done".to_string()); - Ok(format!( - "----- Output Stream -----\n{}\n----- File Stream Events -----\n{}", - stream_contents.join("\n"), + let config = self.test_config(); + // `DataSourceExec::execute` creates one execution-local shared + // state object via `create_sibling_state()` and then passes it + // to `open_with_sibling_state(...)`. These tests build + // `FileStream`s directly, bypassing `DataSourceExec`, so they must + // perform the same setup explicitly when exercising sibling-stream + // work stealing. + let shared_work_source = config.create_sibling_state().and_then(|state| { + state.as_ref().downcast_ref::().cloned() + }); + if !self.build_streams_on_first_read { + for partition in build_order { + let stream = FileStreamBuilder::new(&config) + .with_partition(partition) + .with_shared_work_source(shared_work_source.clone()) + .with_morselizer(Box::new(self.morselizer.clone())) + .with_metrics(&metrics_set) + .build()?; + partitions[partition].set_stream(stream); + } + } + + let mut initial_reads: VecDeque<_> = self.reads.into(); + let mut next_round_robin = 0; + + while !initial_reads.is_empty() + || partitions.iter().any(PartitionState::is_active) + { + let partition = if let Some(partition) = initial_reads.pop_front() { + partition.0 + } else { + let partition = next_round_robin; + next_round_robin = (next_round_robin + 1) % partition_count.max(1); + partition + }; + + let partition_state = &mut partitions[partition]; + + if self.build_streams_on_first_read && !partition_state.built { + let stream = FileStreamBuilder::new(&config) + .with_partition(partition) + .with_shared_work_source(shared_work_source.clone()) + .with_morselizer(Box::new(self.morselizer.clone())) + .with_metrics(&metrics_set) + .build()?; + partition_state.set_stream(stream); + } + + let Some(stream) = partition_state.stream.as_mut() else { + continue; + }; + + match stream.next().await { + Some(result) => partition_state.push_output(format_result(result)), + None => partition_state.finish(), + } + } + + let output_text = if partition_count == 1 { + format!( + "----- Output Stream -----\n{}", + partitions[0].output.join("\n") + ) + } else { + partitions + .into_iter() + .enumerate() + .map(|(partition, state)| { + format!( + "----- Partition {} -----\n{}", + partition, + state.output.join("\n") + ) + }) + .collect::>() + .join("\n") + }; + + let file_stream_events = if self.file_stream_events { observer.format_events() + } else { + "(omitted due to with_file_stream_events(false))".to_string() + }; + + Ok(format!( + "{output_text}\n----- File Stream Events -----\n{file_stream_events}", )) } - /// Builds the `FileScanConfig` for the configured mock file set. + /// Returns the number of configured partitions, including empty ones + /// that appear only in the explicit read schedule. + fn num_partitions(&self) -> usize { + self.partition_files + .keys() + .map(|partition| partition.0 + 1) + .chain(self.reads.iter().map(|partition| partition.0 + 1)) + .max() + .unwrap_or(1) + } + + /// Builds a `FileScanConfig` covering every configured partition. fn test_config(&self) -> FileScanConfig { - let file_group = self - .file_names - .iter() - .map(|name| PartitionedFile::new(name, 10)) - .collect(); + let file_groups = (0..self.num_partitions()) + .map(|partition| { + self.partition_files + .get(&PartitionId(partition)) + .into_iter() + .flat_map(|files| files.iter()) + .map(|name| PartitionedFile::new(name, 10)) + .collect::>() + .into() + }) + .collect::>(); + let table_schema = TableSchema::new( Arc::new(Schema::new(vec![Field::new("i", DataType::Int32, false)])), vec![], @@ -1091,9 +1583,94 @@ mod tests { ObjectStoreUrl::parse("test:///").unwrap(), Arc::new(MockSource::new(table_schema)), ) - .with_file_group(file_group) + .with_file_groups(file_groups) .with_limit(self.limit) + .with_preserve_order(self.preserve_order) + .with_partitioned_by_file_group(self.partitioned_by_file_group) .build() } } + + /// Formats one stream poll result into a stable snapshot line. + fn format_result(result: Result) -> String { + match result { + Ok(batch) => { + let col = batch.column(0).as_primitive::(); + let batch_id = col.value(0); + format!("Batch: {batch_id}") + } + Err(e) => { + // Pull the actual message for external errors rather than + // relying on DataFusionError formatting, which changes if + // backtraces are enabled, etc. + let message = if let DataFusionError::External(generic) = e { + generic.to_string() + } else { + e.to_string() + }; + format!("Error: {message}") + } + } + } + + async fn drain_stream_output(stream: FileStream) -> Result { + let output = stream + .collect::>() + .await + .into_iter() + .map(|result| result.map(|batch| format_result(Ok(batch)))) + .collect::>>()?; + Ok(output.join("\n")) + } + + fn metric_count(metrics: &ExecutionPlanMetricsSet, name: &str) -> usize { + metrics + .clone_inner() + .sum_by_name(name) + .unwrap_or_else(|| panic!("missing metric: {name}")) + .as_usize() + } + + /// Test-only state for one stream partition in [`FileStreamMorselTest`]. + struct PartitionState { + /// Whether the `FileStream` for this partition has been built yet. + built: bool, + /// The live stream, if this partition has not finished yet. + stream: Option, + /// Snapshot lines produced by this partition. + output: Vec, + } + + impl PartitionState { + /// Create an unbuilt partition with no output yet. + fn new() -> Self { + Self { + built: false, + stream: None, + output: vec![], + } + } + + /// Returns true if this partition might still produce output. + fn is_active(&self) -> bool { + !self.built || self.stream.is_some() + } + + /// Records that this partition's stream has been built. + fn set_stream(&mut self, stream: FileStream) { + self.stream = Some(stream); + self.built = true; + } + + /// Records one formatted output line for this partition. + fn push_output(&mut self, line: String) { + self.output.push(line); + } + + /// Marks this partition as finished. + fn finish(&mut self) { + self.push_output("Done".to_string()); + self.stream = None; + } + } } diff --git a/datafusion/datasource/src/file_stream/scan_state.rs b/datafusion/datasource/src/file_stream/scan_state.rs index 025164c29c8f2..21125cd08896c 100644 --- a/datafusion/datasource/src/file_stream/scan_state.rs +++ b/datafusion/datasource/src/file_stream/scan_state.rs @@ -19,7 +19,6 @@ use datafusion_common::internal_datafusion_err; use std::collections::VecDeque; use std::task::{Context, Poll}; -use crate::PartitionedFile; use crate::morsel::{Morsel, MorselPlanner, Morselizer, PendingMorselPlanner}; use arrow::record_batch::RecordBatch; use datafusion_common::{DataFusionError, Result}; @@ -27,6 +26,7 @@ use datafusion_physical_plan::metrics::ScopedTimerGuard; use futures::stream::BoxStream; use futures::{FutureExt as _, StreamExt as _}; +use super::work_source::WorkSource; use super::{FileStreamMetrics, OnError}; /// State [`FileStreamState::Scan`]. @@ -45,7 +45,7 @@ use super::{FileStreamMetrics, OnError}; /// # State Transitions /// /// ```text -/// file_iter +/// work_source /// | /// v /// morselizer.plan_file(file) @@ -62,8 +62,8 @@ use super::{FileStreamMetrics, OnError}; /// /// [`FileStreamState::Scan`]: super::FileStreamState::Scan pub(super) struct ScanState { - /// Files that still need to be planned. - file_iter: VecDeque, + /// Unopened files that still need to be planned for this stream. + work_source: WorkSource, /// Remaining row limit, if any. remain: Option, /// The morselizer used to plan files. @@ -76,7 +76,10 @@ pub(super) struct ScanState { ready_morsels: VecDeque>, /// The active reader, if any. reader: Option>>, - /// The currently outstanding I/O, if any. + /// The single planner currently blocked on I/O, if any. + /// + /// Once the I/O completes, yields the next planner and is pushed back + /// onto `ready_planners`. pending_planner: Option, /// Metrics for the active scan queues. metrics: FileStreamMetrics, @@ -84,15 +87,14 @@ pub(super) struct ScanState { impl ScanState { pub(super) fn new( - file_iter: impl Into>, + work_source: WorkSource, remain: Option, morselizer: Box, on_error: OnError, metrics: FileStreamMetrics, ) -> Self { - let file_iter = file_iter.into(); Self { - file_iter, + work_source, remain, morselizer, on_error, @@ -170,7 +172,7 @@ impl ScanState { (batch, false) } else { let batch = batch.slice(0, *remain); - let done = 1 + self.file_iter.len(); + let done = 1 + self.work_source.skipped_on_limit(); self.metrics.files_processed.add(done); *remain = 0; (batch, true) @@ -263,8 +265,8 @@ impl ScanState { }; } - // No outstanding work remains, so morselize the next unopened file. - let part_file = match self.file_iter.pop_front() { + // No outstanding work remains, so begin planning the next unopened file. + let part_file = match self.work_source.pop_front() { Some(part_file) => part_file, None => return ScanAndReturn::Done(None), }; diff --git a/datafusion/datasource/src/file_stream/work_source.rs b/datafusion/datasource/src/file_stream/work_source.rs new file mode 100644 index 0000000000000..7f31dacca9592 --- /dev/null +++ b/datafusion/datasource/src/file_stream/work_source.rs @@ -0,0 +1,98 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +use std::collections::VecDeque; +use std::sync::Arc; + +use crate::PartitionedFile; +use crate::file_groups::FileGroup; +use crate::file_scan_config::FileScanConfig; +use parking_lot::Mutex; + +/// Source of work for `ScanState`. +/// +/// Streams that may share work across siblings use [`WorkSource::Shared`], +/// while streams that can not share work (e.g. because they must preserve file +/// order) use [`WorkSource::Local`]. +#[derive(Debug, Clone)] +pub(super) enum WorkSource { + /// Files this stream will plan locally without sharing them. + Local(VecDeque), + /// Files shared with sibling streams. + Shared(SharedWorkSource), +} + +impl WorkSource { + /// Pop the next file to plan from this work source. + pub(super) fn pop_front(&mut self) -> Option { + match self { + Self::Local(files) => files.pop_front(), + Self::Shared(shared) => shared.pop_front(), + } + } + + /// Return how many queued files should be counted as already processed + /// when this stream stops early after hitting a global limit. + pub(super) fn skipped_on_limit(&self) -> usize { + match self { + Self::Local(files) => files.len(), + Self::Shared(_) => 0, + } + } +} + +/// Shared source of work for sibling `FileStream`s +/// +/// The queue is created once per execution and shared by all reorderable +/// sibling streams for that execution. Whichever stream becomes idle first may +/// take the next unopened file from the front of the queue. +/// +/// It uses a [`Mutex`] internally to provide thread-safe access +/// to the shared file queue. +#[derive(Debug, Clone)] +pub(crate) struct SharedWorkSource { + inner: Arc, +} + +#[derive(Debug, Default)] +pub(super) struct SharedWorkSourceInner { + files: Mutex>, +} + +impl SharedWorkSource { + /// Create a shared work source containing the provided unopened files. + pub(crate) fn new(files: impl IntoIterator) -> Self { + let files = files.into_iter().collect(); + Self { + inner: Arc::new(SharedWorkSourceInner { + files: Mutex::new(files), + }), + } + } + + /// Create a shared work source for the unopened files in `config`. + pub(crate) fn from_config(config: &FileScanConfig) -> Self { + Self::new(config.file_groups.iter().flat_map(FileGroup::iter).cloned()) + } + + /// Pop the next file from the shared work queue. + /// + /// Returns `None` if the queue is empty + fn pop_front(&self) -> Option { + self.inner.files.lock().pop_front() + } +} diff --git a/datafusion/datasource/src/morsel/mocks.rs b/datafusion/datasource/src/morsel/mocks.rs index cd1fa3732ffea..ceb0e720691a7 100644 --- a/datafusion/datasource/src/morsel/mocks.rs +++ b/datafusion/datasource/src/morsel/mocks.rs @@ -295,8 +295,11 @@ impl MockPlanBuilder { } /// Add a ready child planner - pub(crate) fn with_ready_planner(self, ready_planners: MockPlanner) -> Self { - self.with_ready_planners(vec![ready_planners]) + pub(crate) fn with_ready_planner( + self, + ready_planner: impl Into, + ) -> Self { + self.with_ready_planners(vec![ready_planner.into()]) } /// Add ready child planners produced by this planning step. @@ -430,8 +433,9 @@ impl MockMorselizer { &self.observer } - /// Associates a file path with the planner spec used to open it. - pub(crate) fn with_file(mut self, planner: MockPlanner) -> Self { + /// Specify the return planner for the specified file_path + pub(crate) fn with_planner(mut self, planner: impl Into) -> Self { + let planner = planner.into(); self.files.insert(planner.file_path.clone(), planner); self } diff --git a/datafusion/datasource/src/source.rs b/datafusion/datasource/src/source.rs index 02a15502dacec..420c6b508ce4f 100644 --- a/datafusion/datasource/src/source.rs +++ b/datafusion/datasource/src/source.rs @@ -20,7 +20,7 @@ use std::any::Any; use std::fmt; use std::fmt::{Debug, Formatter}; -use std::sync::Arc; +use std::sync::{Arc, OnceLock}; use datafusion_physical_expr::projection::ProjectionExprs; use datafusion_physical_plan::execution_plan::{ @@ -124,11 +124,21 @@ use datafusion_physical_plan::filter_pushdown::{ /// └─────────────────────┘ /// ``` pub trait DataSource: Any + Send + Sync + Debug { + /// Open the specified output partition and return its stream of + /// [`RecordBatch`]es. + /// + /// This should be used by data sources that do not need any sibling + /// coordination. Data sources that want to use per-execution shared state + /// (for example, to reorder work across partitions at runtime) should + /// implement [`Self::open_with_args`] instead. + /// + /// [`RecordBatch`]: arrow::record_batch::RecordBatch fn open( &self, partition: usize, context: Arc, ) -> Result; + /// Format this source for display in explain plans fn fmt_as(&self, t: DisplayFormatType, f: &mut Formatter) -> fmt::Result; @@ -246,6 +256,55 @@ pub trait DataSource: Any + Send + Sync + Debug { ) -> Option> { None } + + /// Create per execution state to share across sibling instances of this + /// data source during one execution. + /// + /// Returns `None` (the default) if this data source has + /// no sibling-shared execution state. + fn create_sibling_state(&self) -> Option> { + None + } + + /// Open a partition using optional sibling-shared execution state. + /// + /// The default implementation ignores the additional state and delegates to + /// [`Self::open`]. + fn open_with_args(&self, args: OpenArgs) -> Result { + self.open(args.partition, args.context) + } +} + +/// Arguments for [`DataSource::open_with_args`] +#[derive(Debug, Clone)] +pub struct OpenArgs { + /// Which partition to open + pub partition: usize, + /// The task context for execution + pub context: Arc, + /// Optional sibling-shared execution state, see + /// [`DataSource::create_sibling_state`] for details. + pub sibling_state: Option>, +} + +impl OpenArgs { + /// Create a new OpenArgs with required arguments + pub fn new(partition: usize, context: Arc) -> Self { + Self { + partition, + context, + sibling_state: None, + } + } + + /// Set sibling shared state + pub fn with_shared_state( + mut self, + sibling_state: Option>, + ) -> Self { + self.sibling_state = sibling_state; + self + } } impl dyn DataSource { @@ -276,6 +335,12 @@ pub struct DataSourceExec { data_source: Arc, /// Cached plan properties such as sort order cache: Arc, + /// Per execution state shared across partitions of this plan. + /// + /// Created by [`DataSource::create_sibling_state`] + /// and then passed to + /// [`DataSource::open_with_args`]. + execution_state: Arc>>>, } impl DisplayAs for DataSourceExec { @@ -349,8 +414,15 @@ impl ExecutionPlan for DataSourceExec { partition: usize, context: Arc, ) -> Result { - let stream = self.data_source.open(partition, Arc::clone(&context))?; + let shared_state = self + .execution_state + .get_or_init(|| self.data_source.create_sibling_state()) + .clone(); + let args = OpenArgs::new(partition, Arc::clone(&context)) + .with_shared_state(shared_state); + let stream = self.data_source.open_with_args(args)?; let batch_size = context.session_config().batch_size(); + log::debug!( "Batch splitting enabled for partition {partition}: batch_size={batch_size}" ); @@ -386,8 +458,13 @@ impl ExecutionPlan for DataSourceExec { fn with_fetch(&self, limit: Option) -> Option> { let data_source = self.data_source.with_fetch(limit)?; let cache = Arc::clone(&self.cache); + let execution_state = Arc::new(OnceLock::new()); - Some(Arc::new(Self { data_source, cache })) + Some(Arc::new(Self { + data_source, + cache, + execution_state, + })) } fn fetch(&self) -> Option { @@ -480,6 +557,12 @@ impl ExecutionPlan for DataSourceExec { as Arc }) } + + fn reset_state(self: Arc) -> Result> { + let mut new_exec = Arc::unwrap_or_clone(self); + new_exec.execution_state = Arc::new(OnceLock::new()); + Ok(Arc::new(new_exec)) + } } impl DataSourceExec { @@ -493,6 +576,7 @@ impl DataSourceExec { Self { data_source, cache: Arc::new(cache), + execution_state: Arc::new(OnceLock::new()), } } @@ -504,6 +588,7 @@ impl DataSourceExec { pub fn with_data_source(mut self, data_source: Arc) -> Self { self.cache = Arc::new(Self::compute_properties(&data_source)); self.data_source = data_source; + self.execution_state = Arc::new(OnceLock::new()); self } diff --git a/datafusion/datasource/src/test_util.rs b/datafusion/datasource/src/test_util.rs index ffd4b6123eec5..b59ce58a420a8 100644 --- a/datafusion/datasource/src/test_util.rs +++ b/datafusion/datasource/src/test_util.rs @@ -34,6 +34,7 @@ pub(crate) struct MockSource { filter: Option>, table_schema: crate::table_schema::TableSchema, projection: crate::projection::SplitProjection, + file_opener: Option>, } impl Default for MockSource { @@ -45,6 +46,7 @@ impl Default for MockSource { filter: None, projection: crate::projection::SplitProjection::unprojected(&table_schema), table_schema, + file_opener: None, } } } @@ -57,6 +59,7 @@ impl MockSource { filter: None, projection: crate::projection::SplitProjection::unprojected(&table_schema), table_schema, + file_opener: None, } } @@ -64,6 +67,11 @@ impl MockSource { self.filter = Some(filter); self } + + pub fn with_file_opener(mut self, file_opener: Arc) -> Self { + self.file_opener = Some(file_opener); + self + } } impl FileSource for MockSource { @@ -73,7 +81,9 @@ impl FileSource for MockSource { _base_config: &FileScanConfig, _partition: usize, ) -> Result> { - unimplemented!() + self.file_opener.clone().ok_or_else(|| { + datafusion_common::internal_datafusion_err!("MockSource missing FileOpener") + }) } fn filter(&self) -> Option> { diff --git a/datafusion/expr-common/src/signature.rs b/datafusion/expr-common/src/signature.rs index 42d4de939a8e8..3e941f00c2ee3 100644 --- a/datafusion/expr-common/src/signature.rs +++ b/datafusion/expr-common/src/signature.rs @@ -21,7 +21,6 @@ use std::fmt::Display; use std::hash::Hash; use std::sync::Arc; -use crate::type_coercion::aggregates::NUMERICS; use arrow::datatypes::{ DECIMAL32_MAX_PRECISION, DECIMAL64_MAX_PRECISION, DECIMAL128_MAX_PRECISION, DataType, Decimal128Type, DecimalType, Field, IntervalUnit, TimeUnit, @@ -596,6 +595,20 @@ impl Display for ArrayFunctionArgument { } } +static NUMERICS: &[DataType] = &[ + DataType::Int8, + DataType::Int16, + DataType::Int32, + DataType::Int64, + DataType::UInt8, + DataType::UInt16, + DataType::UInt32, + DataType::UInt64, + DataType::Float16, + DataType::Float32, + DataType::Float64, +]; + impl TypeSignature { pub fn to_string_repr(&self) -> Vec { match self { diff --git a/datafusion/expr-common/src/type_coercion/aggregates.rs b/datafusion/expr-common/src/type_coercion/aggregates.rs index df86ff582d658..ada0bd26b8d06 100644 --- a/datafusion/expr-common/src/type_coercion/aggregates.rs +++ b/datafusion/expr-common/src/type_coercion/aggregates.rs @@ -20,8 +20,7 @@ use arrow::datatypes::{DataType, FieldRef}; use datafusion_common::{Result, internal_err, plan_err}; -// TODO: remove usage of these (INTEGERS and NUMERICS) in favour of signatures -// see https://github.com/apache/datafusion/issues/18092 +#[deprecated(since = "54.0.0", note = "Use functions signatures")] pub static INTEGERS: &[DataType] = &[ DataType::Int8, DataType::Int16, @@ -33,6 +32,7 @@ pub static INTEGERS: &[DataType] = &[ DataType::UInt64, ]; +#[deprecated(since = "54.0.0", note = "Use functions signatures")] pub static NUMERICS: &[DataType] = &[ DataType::Int8, DataType::Int16, diff --git a/datafusion/expr/Cargo.toml b/datafusion/expr/Cargo.toml index 265651cfe0eb0..8cec01feb30b5 100644 --- a/datafusion/expr/Cargo.toml +++ b/datafusion/expr/Cargo.toml @@ -66,3 +66,6 @@ sqlparser = { workspace = true, optional = true } ctor = { workspace = true } env_logger = { workspace = true } insta = { workspace = true } +# Makes sure `test_display_pg_json` behaves in a consistent way regardless of +# feature unification with dependencies +serde_json = { workspace = true, features = ["preserve_order"] } diff --git a/datafusion/expr/src/test/function_stub.rs b/datafusion/expr/src/test/function_stub.rs index 3be03260a9761..a1f29b649b2f8 100644 --- a/datafusion/expr/src/test/function_stub.rs +++ b/datafusion/expr/src/test/function_stub.rs @@ -29,13 +29,14 @@ use datafusion_common::plan_err; use datafusion_common::{Result, exec_err, not_impl_err, utils::take_function_args}; use crate::Volatility::Immutable; -use crate::type_coercion::aggregates::NUMERICS; use crate::{ - Accumulator, AggregateUDFImpl, Expr, GroupsAccumulator, ReversedUDAF, Signature, + Accumulator, AggregateUDFImpl, Coercion, Expr, GroupsAccumulator, ReversedUDAF, + Signature, TypeSignature, TypeSignatureClass, expr::AggregateFunction, function::{AccumulatorArgs, StateFieldsArgs}, utils::AggregateOrderSensitivity, }; +use datafusion_common::types::{NativeType, logical_float64}; macro_rules! create_func { ($UDAF:ty, $AGGREGATE_UDF_FN:ident) => { @@ -444,9 +445,22 @@ pub struct Avg { impl Avg { pub fn new() -> Self { + let signature = Signature::one_of( + vec![ + TypeSignature::Coercible(vec![Coercion::new_exact( + TypeSignatureClass::Decimal, + )]), + TypeSignature::Coercible(vec![Coercion::new_implicit( + TypeSignatureClass::Native(logical_float64()), + vec![TypeSignatureClass::Integer, TypeSignatureClass::Float], + NativeType::Float64, + )]), + ], + Immutable, + ); Self { aliases: vec![String::from("mean")], - signature: Signature::uniform(1, NUMERICS.to_vec(), Immutable), + signature, } } } diff --git a/datafusion/functions-aggregate/benches/array_agg.rs b/datafusion/functions-aggregate/benches/array_agg.rs index 793c2aac96293..b0d8148c3ea65 100644 --- a/datafusion/functions-aggregate/benches/array_agg.rs +++ b/datafusion/functions-aggregate/benches/array_agg.rs @@ -20,7 +20,6 @@ use std::sync::Arc; use arrow::array::{ Array, ArrayRef, ArrowPrimitiveType, AsArray, ListArray, NullBufferBuilder, - PrimitiveArray, }; use arrow::datatypes::{Field, Int64Type}; use criterion::{Criterion, criterion_group, criterion_main}; @@ -28,6 +27,7 @@ use datafusion_expr::Accumulator; use datafusion_functions_aggregate::array_agg::ArrayAggAccumulator; use arrow::buffer::OffsetBuffer; +use arrow::util::bench_util::create_primitive_array; use rand::Rng; use rand::SeedableRng; use rand::distr::{Distribution, StandardUniform}; @@ -54,24 +54,6 @@ fn merge_batch_bench(c: &mut Criterion, name: &str, values: ArrayRef) { }); } -pub fn create_primitive_array(size: usize, null_density: f32) -> PrimitiveArray -where - T: ArrowPrimitiveType, - StandardUniform: Distribution, -{ - let mut rng = seedable_rng(); - - (0..size) - .map(|_| { - if rng.random::() < null_density { - None - } else { - Some(rng.random()) - } - }) - .collect() -} - /// Create List array with the given item data type, null density, null locations and zero length lists density /// Creates a random (but fixed-seeded) array of a given size and null density pub fn create_list_array( diff --git a/datafusion/functions-aggregate/benches/count_distinct.rs b/datafusion/functions-aggregate/benches/count_distinct.rs index e742a3e5c1267..4d9e8c5b67b31 100644 --- a/datafusion/functions-aggregate/benches/count_distinct.rs +++ b/datafusion/functions-aggregate/benches/count_distinct.rs @@ -18,12 +18,13 @@ use std::sync::Arc; use arrow::array::{ - ArrayRef, Int8Array, Int16Array, Int64Array, UInt8Array, UInt16Array, + Array, ArrayRef, Int8Array, Int16Array, Int32Array, Int64Array, UInt8Array, + UInt16Array, UInt32Array, }; use arrow::datatypes::{DataType, Field, Schema}; use criterion::{Criterion, criterion_group, criterion_main}; use datafusion_expr::function::AccumulatorArgs; -use datafusion_expr::{Accumulator, AggregateUDFImpl}; +use datafusion_expr::{Accumulator, AggregateUDFImpl, EmitTo}; use datafusion_functions_aggregate::count::Count; use datafusion_physical_expr::expressions::col; use rand::rngs::StdRng; @@ -87,6 +88,44 @@ fn create_i16_array(n_distinct: usize) -> Int16Array { .collect() } +fn create_u32_array(n_distinct: usize) -> UInt32Array { + let mut rng = StdRng::seed_from_u64(42); + (0..BATCH_SIZE) + .map(|_| Some(rng.random_range(0..n_distinct as u32))) + .collect() +} + +fn create_i32_array(n_distinct: usize) -> Int32Array { + let mut rng = StdRng::seed_from_u64(42); + (0..BATCH_SIZE) + .map(|_| Some(rng.random_range(0..n_distinct as i32))) + .collect() +} + +fn prepare_args(data_type: DataType) -> (Arc, AccumulatorArgs<'static>) { + let schema = Arc::new(Schema::new(vec![Field::new("f", data_type, true)])); + let schema_leaked: &'static Schema = Box::leak(Box::new((*schema).clone())); + let expr = col("f", schema_leaked).unwrap(); + let expr_leaked: &'static _ = Box::leak(Box::new(expr)); + let return_field: Arc = Field::new("f", DataType::Int64, true).into(); + let return_field_leaked: &'static _ = Box::leak(Box::new(return_field.clone())); + let expr_field = expr_leaked.return_field(schema_leaked).unwrap(); + let expr_field_leaked: &'static _ = Box::leak(Box::new(expr_field)); + + let accumulator_args = AccumulatorArgs { + return_field: return_field_leaked.clone(), + schema: schema_leaked, + expr_fields: std::slice::from_ref(expr_field_leaked), + ignore_nulls: false, + order_bys: &[], + is_reversed: false, + name: "count(distinct f)", + is_distinct: true, + exprs: std::slice::from_ref(expr_leaked), + }; + (schema, accumulator_args) +} + fn count_distinct_benchmark(c: &mut Criterion) { for pct in [80, 99] { let n_distinct = BATCH_SIZE * pct / 100; @@ -148,7 +187,273 @@ fn count_distinct_benchmark(c: &mut Criterion) { .unwrap() }) }); + + // 32-bit integer types + for pct in [80, 99] { + let n_distinct = BATCH_SIZE * pct / 100; + + // UInt32 + let values = Arc::new(create_u32_array(n_distinct)) as ArrayRef; + c.bench_function(&format!("count_distinct u32 {pct}% distinct"), |b| { + b.iter(|| { + let mut accumulator = prepare_accumulator(DataType::UInt32); + accumulator + .update_batch(std::slice::from_ref(&values)) + .unwrap() + }) + }); + + // Int32 + let values = Arc::new(create_i32_array(n_distinct)) as ArrayRef; + c.bench_function(&format!("count_distinct i32 {pct}% distinct"), |b| { + b.iter(|| { + let mut accumulator = prepare_accumulator(DataType::Int32); + accumulator + .update_batch(std::slice::from_ref(&values)) + .unwrap() + }) + }); + } +} + +/// Create group indices with uniform distribution +fn create_uniform_groups(num_groups: usize) -> Vec { + let mut rng = StdRng::seed_from_u64(42); + (0..BATCH_SIZE) + .map(|_| rng.random_range(0..num_groups)) + .collect() +} + +/// Create group indices with skewed distribution (80% in 20% of groups) +fn create_skewed_groups(num_groups: usize) -> Vec { + let mut rng = StdRng::seed_from_u64(42); + let hot_groups = (num_groups / 5).max(1); + (0..BATCH_SIZE) + .map(|_| { + if rng.random_range(0..100) < 80 { + rng.random_range(0..hot_groups) + } else { + rng.random_range(0..num_groups) + } + }) + .collect() +} + +fn count_distinct_groups_benchmark(c: &mut Criterion) { + let count_fn = Count::new(); + + let group_counts = [100, 1000, 10000]; + let cardinalities = [("low", 20), ("mid", 80), ("high", 99)]; + let distributions = ["uniform", "skewed"]; + + // i64 benchmarks + for num_groups in group_counts { + for (card_name, distinct_pct) in cardinalities { + for dist in distributions { + let name = format!("i64_g{num_groups}_{card_name}_{dist}"); + let n_distinct = BATCH_SIZE * distinct_pct / 100; + let values = Arc::new(create_i64_array(n_distinct)) as ArrayRef; + let group_indices = if dist == "uniform" { + create_uniform_groups(num_groups) + } else { + create_skewed_groups(num_groups) + }; + + let (_schema, args) = prepare_args(DataType::Int64); + + if count_fn.groups_accumulator_supported(args.clone()) { + c.bench_function(&format!("count_distinct_groups {name}"), |b| { + b.iter(|| { + let mut acc = + count_fn.create_groups_accumulator(args.clone()).unwrap(); + acc.update_batch( + std::slice::from_ref(&values), + &group_indices, + None, + num_groups, + ) + .unwrap(); + acc.evaluate(EmitTo::All).unwrap() + }) + }); + } else { + let arr = values.as_any().downcast_ref::().unwrap(); + let mut group_rows: Vec> = vec![Vec::new(); num_groups]; + for (idx, &group_idx) in group_indices.iter().enumerate() { + if arr.is_valid(idx) { + group_rows[group_idx].push(arr.value(idx)); + } + } + let group_arrays: Vec = group_rows + .iter() + .map(|rows| Arc::new(Int64Array::from(rows.clone())) as ArrayRef) + .collect(); + + c.bench_function(&format!("count_distinct_groups {name}"), |b| { + b.iter(|| { + let mut accumulators: Vec<_> = (0..num_groups) + .map(|_| prepare_accumulator(DataType::Int64)) + .collect(); + + for (group_idx, batch) in group_arrays.iter().enumerate() { + if !batch.is_empty() { + accumulators[group_idx] + .update_batch(std::slice::from_ref(batch)) + .unwrap(); + } + } + + let _results: Vec<_> = accumulators + .iter_mut() + .map(|acc| acc.evaluate().unwrap()) + .collect(); + }) + }); + } + } + } + } + + // i32 benchmarks + for num_groups in group_counts { + for (card_name, distinct_pct) in cardinalities { + for dist in distributions { + let name = format!("i32_g{num_groups}_{card_name}_{dist}"); + let n_distinct = BATCH_SIZE * distinct_pct / 100; + let values = Arc::new(create_i32_array(n_distinct)) as ArrayRef; + let group_indices = if dist == "uniform" { + create_uniform_groups(num_groups) + } else { + create_skewed_groups(num_groups) + }; + + let (_schema, args) = prepare_args(DataType::Int32); + + if count_fn.groups_accumulator_supported(args.clone()) { + c.bench_function(&format!("count_distinct_groups {name}"), |b| { + b.iter(|| { + let mut acc = + count_fn.create_groups_accumulator(args.clone()).unwrap(); + acc.update_batch( + std::slice::from_ref(&values), + &group_indices, + None, + num_groups, + ) + .unwrap(); + acc.evaluate(EmitTo::All).unwrap() + }) + }); + } else { + let arr = values.as_any().downcast_ref::().unwrap(); + let mut group_rows: Vec> = vec![Vec::new(); num_groups]; + for (idx, &group_idx) in group_indices.iter().enumerate() { + if arr.is_valid(idx) { + group_rows[group_idx].push(arr.value(idx)); + } + } + let group_arrays: Vec = group_rows + .iter() + .map(|rows| Arc::new(Int32Array::from(rows.clone())) as ArrayRef) + .collect(); + + c.bench_function(&format!("count_distinct_groups {name}"), |b| { + b.iter(|| { + let mut accumulators: Vec<_> = (0..num_groups) + .map(|_| prepare_accumulator(DataType::Int32)) + .collect(); + + for (group_idx, batch) in group_arrays.iter().enumerate() { + if !batch.is_empty() { + accumulators[group_idx] + .update_batch(std::slice::from_ref(batch)) + .unwrap(); + } + } + + let _results: Vec<_> = accumulators + .iter_mut() + .map(|acc| acc.evaluate().unwrap()) + .collect(); + }) + }); + } + } + } + } + + // u32 benchmarks + for num_groups in group_counts { + for (card_name, distinct_pct) in cardinalities { + for dist in distributions { + let name = format!("u32_g{num_groups}_{card_name}_{dist}"); + let n_distinct = BATCH_SIZE * distinct_pct / 100; + let values = Arc::new(create_u32_array(n_distinct)) as ArrayRef; + let group_indices = if dist == "uniform" { + create_uniform_groups(num_groups) + } else { + create_skewed_groups(num_groups) + }; + + let (_schema, args) = prepare_args(DataType::UInt32); + + if count_fn.groups_accumulator_supported(args.clone()) { + c.bench_function(&format!("count_distinct_groups {name}"), |b| { + b.iter(|| { + let mut acc = + count_fn.create_groups_accumulator(args.clone()).unwrap(); + acc.update_batch( + std::slice::from_ref(&values), + &group_indices, + None, + num_groups, + ) + .unwrap(); + acc.evaluate(EmitTo::All).unwrap() + }) + }); + } else { + let arr = values.as_any().downcast_ref::().unwrap(); + let mut group_rows: Vec> = vec![Vec::new(); num_groups]; + for (idx, &group_idx) in group_indices.iter().enumerate() { + if arr.is_valid(idx) { + group_rows[group_idx].push(arr.value(idx)); + } + } + let group_arrays: Vec = group_rows + .iter() + .map(|rows| Arc::new(UInt32Array::from(rows.clone())) as ArrayRef) + .collect(); + + c.bench_function(&format!("count_distinct_groups {name}"), |b| { + b.iter(|| { + let mut accumulators: Vec<_> = (0..num_groups) + .map(|_| prepare_accumulator(DataType::UInt32)) + .collect(); + + for (group_idx, batch) in group_arrays.iter().enumerate() { + if !batch.is_empty() { + accumulators[group_idx] + .update_batch(std::slice::from_ref(batch)) + .unwrap(); + } + } + + let _results: Vec<_> = accumulators + .iter_mut() + .map(|acc| acc.evaluate().unwrap()) + .collect(); + }) + }); + } + } + } + } } -criterion_group!(benches, count_distinct_benchmark); +criterion_group!( + benches, + count_distinct_benchmark, + count_distinct_groups_benchmark +); criterion_main!(benches); diff --git a/datafusion/functions-aggregate/src/approx_median.rs b/datafusion/functions-aggregate/src/approx_median.rs index c016ada695716..162dc224f2ccb 100644 --- a/datafusion/functions-aggregate/src/approx_median.rs +++ b/datafusion/functions-aggregate/src/approx_median.rs @@ -74,16 +74,11 @@ impl ApproxMedian { pub fn new() -> Self { Self { signature: Signature::one_of( - vec![ - TypeSignature::Coercible(vec![Coercion::new_exact( - TypeSignatureClass::Integer, - )]), - TypeSignature::Coercible(vec![Coercion::new_implicit( - TypeSignatureClass::Float, - vec![TypeSignatureClass::Decimal], - NativeType::Float64, - )]), - ], + vec![TypeSignature::Coercible(vec![Coercion::new_implicit( + TypeSignatureClass::Float, + vec![TypeSignatureClass::Numeric], + NativeType::Float64, + )])], Volatility::Immutable, ), } diff --git a/datafusion/functions-aggregate/src/approx_percentile_cont.rs b/datafusion/functions-aggregate/src/approx_percentile_cont.rs index 5e9e5beb0cb4b..3f1adcca12362 100644 --- a/datafusion/functions-aggregate/src/approx_percentile_cont.rs +++ b/datafusion/functions-aggregate/src/approx_percentile_cont.rs @@ -23,23 +23,20 @@ use arrow::array::{Array, Float16Array}; use arrow::compute::{filter, is_not_null}; use arrow::datatypes::FieldRef; use arrow::{ - array::{ - ArrayRef, Float32Array, Float64Array, Int8Array, Int16Array, Int32Array, - Int64Array, UInt8Array, UInt16Array, UInt32Array, UInt64Array, - }, + array::{ArrayRef, Float32Array, Float64Array}, datatypes::{DataType, Field}, }; +use datafusion_common::types::{NativeType, logical_float64}; use datafusion_common::{ DataFusionError, Result, ScalarValue, downcast_value, internal_err, not_impl_err, plan_err, }; use datafusion_expr::expr::{AggregateFunction, Sort}; use datafusion_expr::function::{AccumulatorArgs, StateFieldsArgs}; -use datafusion_expr::type_coercion::aggregates::{INTEGERS, NUMERICS}; use datafusion_expr::utils::format_state_name; use datafusion_expr::{ - Accumulator, AggregateUDFImpl, Documentation, Expr, Signature, TypeSignature, - Volatility, + Accumulator, AggregateUDFImpl, Coercion, Documentation, Expr, Signature, + TypeSignature, TypeSignatureClass, Volatility, }; use datafusion_functions_aggregate_common::tdigest::{DEFAULT_MAX_SIZE, TDigest}; use datafusion_macros::user_doc; @@ -132,22 +129,44 @@ impl Default for ApproxPercentileCont { impl ApproxPercentileCont { /// Create a new [`ApproxPercentileCont`] aggregate function. pub fn new() -> Self { - let mut variants = Vec::with_capacity(NUMERICS.len() * (INTEGERS.len() + 1)); // Accept any numeric value paired with a float64 percentile - for num in NUMERICS { - variants.push(TypeSignature::Exact(vec![num.clone(), DataType::Float64])); - // Additionally accept an integer number of centroids for T-Digest - for int in INTEGERS { - variants.push(TypeSignature::Exact(vec![ - num.clone(), - DataType::Float64, - int.clone(), - ])) - } - } - Self { - signature: Signature::one_of(variants, Volatility::Immutable), - } + let signature = Signature::one_of( + vec![ + // 2 args - numeric, percentile (float) + TypeSignature::Coercible(vec![ + Coercion::new_implicit( + TypeSignatureClass::Float, + vec![TypeSignatureClass::Numeric], + NativeType::Float64, + ), + Coercion::new_implicit( + TypeSignatureClass::Native(logical_float64()), + vec![TypeSignatureClass::Numeric], + NativeType::Float64, + ), + ]), + // 3 args - numeric, percentile (float), number of centroid for T-Digest (integer) + TypeSignature::Coercible(vec![ + Coercion::new_implicit( + TypeSignatureClass::Float, + vec![TypeSignatureClass::Numeric], + NativeType::Float64, + ), + Coercion::new_implicit( + TypeSignatureClass::Native(logical_float64()), + vec![TypeSignatureClass::Numeric], + NativeType::Float64, + ), + Coercion::new_implicit( + TypeSignatureClass::Integer, + vec![TypeSignatureClass::Numeric], + NativeType::Int64, + ), + ]), + ], + Volatility::Immutable, + ); + Self { signature } } pub(crate) fn create_accumulator( @@ -177,17 +196,7 @@ impl ApproxPercentileCont { let data_type = args.expr_fields[0].data_type(); let accumulator: ApproxPercentileAccumulator = match data_type { - DataType::UInt8 - | DataType::UInt16 - | DataType::UInt32 - | DataType::UInt64 - | DataType::Int8 - | DataType::Int16 - | DataType::Int32 - | DataType::Int64 - | DataType::Float16 - | DataType::Float32 - | DataType::Float64 => { + DataType::Float16 | DataType::Float32 | DataType::Float64 => { if let Some(max_size) = tdigest_max_size { ApproxPercentileAccumulator::new_with_max_size( percentile, @@ -374,38 +383,6 @@ impl ApproxPercentileAccumulator { .map(|v| v.to_f64()) .collect::>()) } - DataType::Int64 => { - let array = downcast_value!(values, Int64Array); - Ok(array.values().iter().map(|v| *v as f64).collect::>()) - } - DataType::Int32 => { - let array = downcast_value!(values, Int32Array); - Ok(array.values().iter().map(|v| *v as f64).collect::>()) - } - DataType::Int16 => { - let array = downcast_value!(values, Int16Array); - Ok(array.values().iter().map(|v| *v as f64).collect::>()) - } - DataType::Int8 => { - let array = downcast_value!(values, Int8Array); - Ok(array.values().iter().map(|v| *v as f64).collect::>()) - } - DataType::UInt64 => { - let array = downcast_value!(values, UInt64Array); - Ok(array.values().iter().map(|v| *v as f64).collect::>()) - } - DataType::UInt32 => { - let array = downcast_value!(values, UInt32Array); - Ok(array.values().iter().map(|v| *v as f64).collect::>()) - } - DataType::UInt16 => { - let array = downcast_value!(values, UInt16Array); - Ok(array.values().iter().map(|v| *v as f64).collect::>()) - } - DataType::UInt8 => { - let array = downcast_value!(values, UInt8Array); - Ok(array.values().iter().map(|v| *v as f64).collect::>()) - } e => internal_err!( "APPROX_PERCENTILE_CONT is not expected to receive the type {e:?}" ), @@ -439,14 +416,6 @@ impl Accumulator for ApproxPercentileAccumulator { // These acceptable return types MUST match the validation in // ApproxPercentile::create_accumulator. Ok(match &self.return_type { - DataType::Int8 => ScalarValue::Int8(Some(q as i8)), - DataType::Int16 => ScalarValue::Int16(Some(q as i16)), - DataType::Int32 => ScalarValue::Int32(Some(q as i32)), - DataType::Int64 => ScalarValue::Int64(Some(q as i64)), - DataType::UInt8 => ScalarValue::UInt8(Some(q as u8)), - DataType::UInt16 => ScalarValue::UInt16(Some(q as u16)), - DataType::UInt32 => ScalarValue::UInt32(Some(q as u32)), - DataType::UInt64 => ScalarValue::UInt64(Some(q as u64)), DataType::Float16 => ScalarValue::Float16(Some(half::f16::from_f64(q))), DataType::Float32 => ScalarValue::Float32(Some(q as f32)), DataType::Float64 => ScalarValue::Float64(Some(q)), diff --git a/datafusion/functions-aggregate/src/approx_percentile_cont_with_weight.rs b/datafusion/functions-aggregate/src/approx_percentile_cont_with_weight.rs index 81a4787dd522f..6ada47fb38040 100644 --- a/datafusion/functions-aggregate/src/approx_percentile_cont_with_weight.rs +++ b/datafusion/functions-aggregate/src/approx_percentile_cont_with_weight.rs @@ -24,13 +24,13 @@ use arrow::compute::{and, filter, is_not_null}; use arrow::datatypes::FieldRef; use arrow::{array::ArrayRef, datatypes::DataType}; use datafusion_common::ScalarValue; +use datafusion_common::types::{NativeType, logical_float64}; use datafusion_common::{Result, not_impl_err, plan_err}; -use datafusion_expr::Volatility::Immutable; use datafusion_expr::expr::{AggregateFunction, Sort}; use datafusion_expr::function::{AccumulatorArgs, StateFieldsArgs}; -use datafusion_expr::type_coercion::aggregates::{INTEGERS, NUMERICS}; use datafusion_expr::{ - Accumulator, AggregateUDFImpl, Documentation, Expr, Signature, TypeSignature, + Accumulator, AggregateUDFImpl, Coercion, Documentation, Expr, Signature, + TypeSignature, TypeSignatureClass, Volatility, }; use datafusion_functions_aggregate_common::tdigest::{Centroid, TDigest}; use datafusion_macros::user_doc; @@ -125,26 +125,54 @@ impl Default for ApproxPercentileContWithWeight { impl ApproxPercentileContWithWeight { /// Create a new [`ApproxPercentileContWithWeight`] aggregate function. pub fn new() -> Self { - let mut variants = Vec::with_capacity(NUMERICS.len() * (INTEGERS.len() + 1)); - // Accept any numeric value paired with weight and float64 percentile - for num in NUMERICS { - variants.push(TypeSignature::Exact(vec![ - num.clone(), - num.clone(), - DataType::Float64, - ])); - // Additionally accept an integer number of centroids for T-Digest - for int in INTEGERS { - variants.push(TypeSignature::Exact(vec![ - num.clone(), - num.clone(), - DataType::Float64, - int.clone(), - ])); - } - } + let signature = Signature::one_of( + vec![ + // 3 args - numeric, weight (float), percentile (float) + TypeSignature::Coercible(vec![ + Coercion::new_implicit( + TypeSignatureClass::Float, + vec![TypeSignatureClass::Numeric], + NativeType::Float64, + ), + Coercion::new_implicit( + TypeSignatureClass::Float, + vec![TypeSignatureClass::Numeric], + NativeType::Float64, + ), + Coercion::new_implicit( + TypeSignatureClass::Native(logical_float64()), + vec![TypeSignatureClass::Numeric], + NativeType::Float64, + ), + ]), + // 4 args - numeric, weight (float), percentile (float), centroid (integer) + TypeSignature::Coercible(vec![ + Coercion::new_implicit( + TypeSignatureClass::Float, + vec![TypeSignatureClass::Numeric], + NativeType::Float64, + ), + Coercion::new_implicit( + TypeSignatureClass::Float, + vec![TypeSignatureClass::Numeric], + NativeType::Float64, + ), + Coercion::new_implicit( + TypeSignatureClass::Native(logical_float64()), + vec![TypeSignatureClass::Numeric], + NativeType::Float64, + ), + Coercion::new_implicit( + TypeSignatureClass::Integer, + vec![TypeSignatureClass::Numeric], + NativeType::Int64, + ), + ]), + ], + Volatility::Immutable, + ); Self { - signature: Signature::one_of(variants, Immutable), + signature, approx_percentile_cont: ApproxPercentileCont::new(), } } diff --git a/datafusion/functions-aggregate/src/utils.rs b/datafusion/functions-aggregate/src/utils.rs index 5e1925fcdbb5d..6d816e54bdaf2 100644 --- a/datafusion/functions-aggregate/src/utils.rs +++ b/datafusion/functions-aggregate/src/utils.rs @@ -54,6 +54,11 @@ pub(crate) fn validate_percentile_expr( let percentile = match scalar_value { ScalarValue::Float32(Some(value)) => value as f64, ScalarValue::Float64(Some(value)) => value, + ScalarValue::Float32(None) | ScalarValue::Float64(None) => { + return plan_err!( + "Percentile value for '{fn_name}' must be Float32 or Float64 literal (got null)" + ); + } sv => { return plan_err!( "Percentile value for '{fn_name}' must be Float32 or Float64 literal (got data type {})", diff --git a/datafusion/functions-nested/Cargo.toml b/datafusion/functions-nested/Cargo.toml index 13b00296151fb..d2204a7be7209 100644 --- a/datafusion/functions-nested/Cargo.toml +++ b/datafusion/functions-nested/Cargo.toml @@ -64,6 +64,7 @@ log = { workspace = true } memchr = { workspace = true } [dev-dependencies] +arrow = { workspace = true, features = ["test_utils"] } criterion = { workspace = true, features = ["async_tokio"] } rand = { workspace = true } diff --git a/datafusion/functions-nested/benches/array_min_max.rs b/datafusion/functions-nested/benches/array_min_max.rs index 391a84d148816..45838da79f95b 100644 --- a/datafusion/functions-nested/benches/array_min_max.rs +++ b/datafusion/functions-nested/benches/array_min_max.rs @@ -17,73 +17,19 @@ use std::sync::Arc; -use arrow::array::{ArrayRef, Int64Array, ListArray}; -use arrow::buffer::{NullBuffer, OffsetBuffer}; -use arrow::datatypes::{DataType, Field}; +use arrow::array::{Array, ArrayRef}; +use arrow::datatypes::{DataType, Field, Int64Type}; +use arrow::util::bench_util::create_primitive_list_array_with_seed; use criterion::{BenchmarkId, Criterion, criterion_group, criterion_main}; use datafusion_common::config::ConfigOptions; use datafusion_expr::{ColumnarValue, ScalarFunctionArgs, ScalarUDFImpl}; use datafusion_functions_nested::min_max::ArrayMax; -use rand::rngs::StdRng; -use rand::{Rng, SeedableRng}; const NUM_ROWS: usize = 8192; const SEED: u64 = 42; const LIST_NULL_DENSITY: f64 = 0.1; const ELEMENT_NULL_DENSITY: f64 = 0.1; -fn create_int64_list_array( - num_rows: usize, - list_size: usize, - element_null_density: f64, -) -> ArrayRef { - let mut rng = StdRng::seed_from_u64(SEED); - let total_values = num_rows * list_size; - - if element_null_density > 0.0 { - let values: Vec> = (0..total_values) - .map(|_| { - if rng.random::() < element_null_density { - None - } else { - Some(rng.random::() % 10_000) - } - }) - .collect(); - let values_array = Arc::new(Int64Array::from(values)); - - let offsets: Vec = (0..=num_rows).map(|i| (i * list_size) as i32).collect(); - let nulls: Vec = (0..num_rows) - .map(|_| rng.random::() >= LIST_NULL_DENSITY) - .collect(); - - Arc::new(ListArray::new( - Arc::new(Field::new("item", DataType::Int64, true)), - OffsetBuffer::new(offsets.into()), - values_array, - Some(NullBuffer::from(nulls)), - )) - } else { - // No element nulls — values array has no null buffer - let values: Vec = (0..total_values) - .map(|_| rng.random::() % 10_000) - .collect(); - let values_array = Arc::new(Int64Array::from(values)); - - let offsets: Vec = (0..=num_rows).map(|i| (i * list_size) as i32).collect(); - let nulls: Vec = (0..num_rows) - .map(|_| rng.random::() >= LIST_NULL_DENSITY) - .collect(); - - Arc::new(ListArray::new( - Arc::new(Field::new("item", DataType::Int64, false)), - OffsetBuffer::new(offsets.into()), - values_array, - Some(NullBuffer::from(nulls)), - )) - } -} - fn criterion_benchmark(c: &mut Criterion) { let udf = ArrayMax::new(); let config_options = Arc::new(ConfigOptions::default()); @@ -91,7 +37,14 @@ fn criterion_benchmark(c: &mut Criterion) { for list_size in [10, 100, 1000] { for (label, null_density) in [("nulls", ELEMENT_NULL_DENSITY), ("no_nulls", 0.0)] { - let list_array = create_int64_list_array(NUM_ROWS, list_size, null_density); + let list_array: ArrayRef = + Arc::new(create_primitive_list_array_with_seed::( + NUM_ROWS, + LIST_NULL_DENSITY as f32, + null_density as f32, + list_size, + SEED, + )); let args = vec![ColumnarValue::Array(Arc::clone(&list_array))]; let arg_fields = vec![Field::new("arg_0", list_array.data_type().clone(), true).into()]; diff --git a/datafusion/functions-nested/benches/array_remove.rs b/datafusion/functions-nested/benches/array_remove.rs index 6fd153548a63b..3384bd0e46db6 100644 --- a/datafusion/functions-nested/benches/array_remove.rs +++ b/datafusion/functions-nested/benches/array_remove.rs @@ -16,11 +16,12 @@ // under the License. use arrow::array::{ - Array, ArrayRef, BinaryArray, BooleanArray, Decimal128Array, FixedSizeBinaryArray, - Float64Array, Int64Array, ListArray, StringArray, + Array, ArrayRef, BinaryArray, BooleanArray, FixedSizeBinaryArray, ListArray, + StringArray, }; use arrow::buffer::OffsetBuffer; -use arrow::datatypes::{DataType, Field}; +use arrow::datatypes::{DataType, Decimal128Type, Field, Float64Type, Int64Type}; +use arrow::util::bench_util::create_primitive_list_array_with_seed; use criterion::{ criterion_group, criterion_main, {BenchmarkId, Criterion}, }; @@ -55,7 +56,15 @@ fn bench_array_remove_int64(c: &mut Criterion) { let mut group = c.benchmark_group("array_remove_int64"); for &array_size in ARRAY_SIZES { - let list_array = create_int64_list_array(NUM_ROWS, array_size, NULL_DENSITY); + let list_array: ArrayRef = + Arc::new(create_primitive_list_array_with_seed::( + NUM_ROWS, + 0.0, + NULL_DENSITY as f32, + array_size, + SEED, + )); + let element_to_remove = ScalarValue::Int64(Some(1)); let args = create_args(list_array.clone(), element_to_remove.clone()); @@ -96,7 +105,14 @@ fn bench_array_remove_f64(c: &mut Criterion) { let mut group = c.benchmark_group("array_remove_f64"); for &array_size in ARRAY_SIZES { - let list_array = create_f64_list_array(NUM_ROWS, array_size, NULL_DENSITY); + let list_array: ArrayRef = + Arc::new(create_primitive_list_array_with_seed::( + NUM_ROWS, + 0.0, + NULL_DENSITY as f32, + array_size, + SEED, + )); let element_to_remove = ScalarValue::Float64(Some(1.0)); let args = create_args(list_array.clone(), element_to_remove.clone()); @@ -260,8 +276,17 @@ fn bench_array_remove_decimal64(c: &mut Criterion) { let mut group = c.benchmark_group("array_remove_decimal64"); for &array_size in ARRAY_SIZES { - let list_array = create_decimal64_list_array(NUM_ROWS, array_size, NULL_DENSITY); - let element_to_remove = ScalarValue::Decimal128(Some(100_i128), 10, 2); + let list_array: ArrayRef = Arc::new(create_primitive_list_array_with_seed::< + i32, + Decimal128Type, + >( + NUM_ROWS, + 0.0, + NULL_DENSITY as f32, + array_size, + SEED, + )); + let element_to_remove = ScalarValue::Decimal128(Some(100_i128), 38, 10); let args = create_args(list_array.clone(), element_to_remove.clone()); group.bench_with_input( @@ -276,7 +301,7 @@ fn bench_array_remove_decimal64(c: &mut Criterion) { arg_fields: vec![ Field::new("arr", list_array.data_type().clone(), false) .into(), - Field::new("el", DataType::Decimal128(10, 2), false) + Field::new("el", DataType::Decimal128(38, 10), false) .into(), ], number_rows: NUM_ROWS, @@ -348,66 +373,6 @@ fn create_args(list_array: ArrayRef, element: ScalarValue) -> Vec ] } -fn create_int64_list_array( - num_rows: usize, - array_size: usize, - null_density: f64, -) -> ArrayRef { - let mut rng = StdRng::seed_from_u64(SEED); - let values = (0..num_rows * array_size) - .map(|_| { - if rng.random::() < null_density { - None - } else { - Some(rng.random_range(0..array_size as i64)) - } - }) - .collect::(); - let offsets = (0..=num_rows) - .map(|i| (i * array_size) as i32) - .collect::>(); - - Arc::new( - ListArray::try_new( - Arc::new(Field::new("item", DataType::Int64, true)), - OffsetBuffer::new(offsets.into()), - Arc::new(values), - None, - ) - .unwrap(), - ) -} - -fn create_f64_list_array( - num_rows: usize, - array_size: usize, - null_density: f64, -) -> ArrayRef { - let mut rng = StdRng::seed_from_u64(SEED); - let values = (0..num_rows * array_size) - .map(|_| { - if rng.random::() < null_density { - None - } else { - Some(rng.random_range(0..array_size as i64) as f64) - } - }) - .collect::(); - let offsets = (0..=num_rows) - .map(|i| (i * array_size) as i32) - .collect::>(); - - Arc::new( - ListArray::try_new( - Arc::new(Field::new("item", DataType::Float64, true)), - OffsetBuffer::new(offsets.into()), - Arc::new(values), - None, - ) - .unwrap(), - ) -} - fn create_string_list_array( num_rows: usize, array_size: usize, @@ -500,38 +465,6 @@ fn create_boolean_list_array( ) } -fn create_decimal64_list_array( - num_rows: usize, - array_size: usize, - null_density: f64, -) -> ArrayRef { - let mut rng = StdRng::seed_from_u64(SEED); - let values = (0..num_rows * array_size) - .map(|_| { - if rng.random::() < null_density { - None - } else { - Some(rng.random_range(0..array_size) as i128 * 100) - } - }) - .collect::() - .with_precision_and_scale(10, 2) - .unwrap(); - let offsets = (0..=num_rows) - .map(|i| (i * array_size) as i32) - .collect::>(); - - Arc::new( - ListArray::try_new( - Arc::new(Field::new("item", DataType::Decimal128(10, 2), true)), - OffsetBuffer::new(offsets.into()), - Arc::new(values), - None, - ) - .unwrap(), - ) -} - fn create_fixed_size_binary_list_array( num_rows: usize, array_size: usize, diff --git a/datafusion/functions-nested/benches/array_repeat.rs b/datafusion/functions-nested/benches/array_repeat.rs index 0ce8db00ceb8f..42372322e2812 100644 --- a/datafusion/functions-nested/benches/array_repeat.rs +++ b/datafusion/functions-nested/benches/array_repeat.rs @@ -15,9 +15,13 @@ // specific language governing permissions and limitations // under the License. -use arrow::array::{ArrayRef, BooleanArray, Float64Array, Int64Array, ListArray}; +use arrow::array::{ArrayRef, ListArray}; use arrow::buffer::OffsetBuffer; -use arrow::datatypes::{DataType, Field}; +use arrow::datatypes::{DataType, Field, Int64Type}; +use arrow::util::bench_util::{ + create_boolean_array, create_f64_array, create_primitive_array, + create_primitive_list_array_with_seed, create_string_array_with_max_len, +}; use criterion::{ criterion_group, criterion_main, {BenchmarkId, Criterion}, }; @@ -32,7 +36,8 @@ use std::hint::black_box; use std::sync::Arc; const NUM_ROWS: &[usize] = &[100, 1000, 10000]; -const REPEAT_COUNTS: &[u64] = &[5, 50]; +// Must be of type i64 because ArrayRepeat's second argument is Int64 +const REPEAT_COUNTS: &[i64] = &[5, 50]; const SEED: u64 = 42; const NULL_DENSITY: f64 = 0.1; @@ -52,7 +57,10 @@ fn bench_array_repeat_int64(c: &mut Criterion) { let mut group = c.benchmark_group("array_repeat_int64"); for &num_rows in NUM_ROWS { - let element_array = create_int64_array(num_rows, NULL_DENSITY); + let element_array: ArrayRef = Arc::new(create_primitive_array::( + num_rows, + NULL_DENSITY as f32, + )); for &repeat_count in REPEAT_COUNTS { let args = vec![ @@ -71,7 +79,7 @@ fn bench_array_repeat_int64(c: &mut Criterion) { args: args.clone(), arg_fields: vec![ Field::new("element", DataType::Int64, false).into(), - Field::new("count", DataType::UInt64, false).into(), + Field::new("count", DataType::Int64, false).into(), ], number_rows: num_rows, return_field: Field::new( @@ -100,7 +108,11 @@ fn bench_array_repeat_string(c: &mut Criterion) { let mut group = c.benchmark_group("array_repeat_string"); for &num_rows in NUM_ROWS { - let element_array = create_string_array(num_rows, NULL_DENSITY); + let element_array = Arc::new(create_string_array_with_max_len::( + num_rows, + NULL_DENSITY as f32, + 100, + )); for &repeat_count in REPEAT_COUNTS { let args = vec![ @@ -119,7 +131,7 @@ fn bench_array_repeat_string(c: &mut Criterion) { args: args.clone(), arg_fields: vec![ Field::new("element", DataType::Utf8, false).into(), - Field::new("count", DataType::UInt64, false).into(), + Field::new("count", DataType::Int64, false).into(), ], number_rows: num_rows, return_field: Field::new( @@ -148,7 +160,14 @@ fn bench_array_repeat_nested_int64_list(c: &mut Criterion) { let mut group = c.benchmark_group("array_repeat_nested_int64"); for &num_rows in NUM_ROWS { - let list_array = create_int64_list_array(num_rows, 5, NULL_DENSITY); + let list_array: ArrayRef = + Arc::new(create_primitive_list_array_with_seed::( + num_rows, + NULL_DENSITY as f32, + NULL_DENSITY as f32, + 5, + SEED, + )); for &repeat_count in REPEAT_COUNTS { let args = vec![ @@ -172,7 +191,7 @@ fn bench_array_repeat_nested_int64_list(c: &mut Criterion) { false, ) .into(), - Field::new("count", DataType::UInt64, false).into(), + Field::new("count", DataType::Int64, false).into(), ], number_rows: num_rows, return_field: Field::new( @@ -201,7 +220,7 @@ fn bench_array_repeat_float64(c: &mut Criterion) { let mut group = c.benchmark_group("array_repeat_float64"); for &num_rows in NUM_ROWS { - let element_array = create_float64_array(num_rows, NULL_DENSITY); + let element_array = Arc::new(create_f64_array(num_rows, NULL_DENSITY as f32)); for &repeat_count in REPEAT_COUNTS { let args = vec![ @@ -221,7 +240,7 @@ fn bench_array_repeat_float64(c: &mut Criterion) { arg_fields: vec![ Field::new("element", DataType::Float64, false) .into(), - Field::new("count", DataType::UInt64, false).into(), + Field::new("count", DataType::Int64, false).into(), ], number_rows: num_rows, return_field: Field::new( @@ -250,7 +269,11 @@ fn bench_array_repeat_boolean(c: &mut Criterion) { let mut group = c.benchmark_group("array_repeat_boolean"); for &num_rows in NUM_ROWS { - let element_array = create_boolean_array(num_rows, NULL_DENSITY); + let element_array = Arc::new(create_boolean_array( + num_rows, + NULL_DENSITY as f32, + f32::MAX, + )); for &repeat_count in REPEAT_COUNTS { let args = vec![ @@ -270,7 +293,7 @@ fn bench_array_repeat_boolean(c: &mut Criterion) { arg_fields: vec![ Field::new("element", DataType::Boolean, false) .into(), - Field::new("count", DataType::UInt64, false).into(), + Field::new("count", DataType::Int64, false).into(), ], number_rows: num_rows, return_field: Field::new( @@ -323,7 +346,7 @@ fn bench_array_repeat_nested_string_list(c: &mut Criterion) { false, ) .into(), - Field::new("count", DataType::UInt64, false).into(), + Field::new("count", DataType::Int64, false).into(), ], number_rows: num_rows, return_field: Field::new( @@ -348,98 +371,6 @@ fn bench_array_repeat_nested_string_list(c: &mut Criterion) { group.finish(); } -fn create_int64_array(num_rows: usize, null_density: f64) -> ArrayRef { - let mut rng = StdRng::seed_from_u64(SEED); - let values = (0..num_rows) - .map(|_| { - if rng.random::() < null_density { - None - } else { - Some(rng.random_range(0..1000)) - } - }) - .collect::(); - - Arc::new(values) -} - -fn create_string_array(num_rows: usize, null_density: f64) -> ArrayRef { - let mut rng = StdRng::seed_from_u64(SEED); - use arrow::array::StringArray; - - let values = (0..num_rows) - .map(|_| { - if rng.random::() < null_density { - None - } else { - Some(format!("value_{}", rng.random_range(0..100))) - } - }) - .collect::(); - - Arc::new(values) -} - -fn create_int64_list_array( - num_rows: usize, - array_size: usize, - null_density: f64, -) -> ArrayRef { - let mut rng = StdRng::seed_from_u64(SEED); - let values = (0..num_rows * array_size) - .map(|_| { - if rng.random::() < null_density { - None - } else { - Some(rng.random_range(0..1000)) - } - }) - .collect::(); - let offsets = (0..=num_rows) - .map(|i| (i * array_size) as i32) - .collect::>(); - - Arc::new( - ListArray::try_new( - Arc::new(Field::new("item", DataType::Int64, true)), - OffsetBuffer::new(offsets.into()), - Arc::new(values), - None, - ) - .unwrap(), - ) -} - -fn create_float64_array(num_rows: usize, null_density: f64) -> ArrayRef { - let mut rng = StdRng::seed_from_u64(SEED); - let values = (0..num_rows) - .map(|_| { - if rng.random::() < null_density { - None - } else { - Some(rng.random_range(0.0..1000.0)) - } - }) - .collect::(); - - Arc::new(values) -} - -fn create_boolean_array(num_rows: usize, null_density: f64) -> ArrayRef { - let mut rng = StdRng::seed_from_u64(SEED); - let values = (0..num_rows) - .map(|_| { - if rng.random::() < null_density { - None - } else { - Some(rng.random()) - } - }) - .collect::(); - - Arc::new(values) -} - fn create_string_list_array( num_rows: usize, array_size: usize, diff --git a/datafusion/functions-nested/benches/array_to_string.rs b/datafusion/functions-nested/benches/array_to_string.rs index 286ed4eeb0003..4b63d705480bf 100644 --- a/datafusion/functions-nested/benches/array_to_string.rs +++ b/datafusion/functions-nested/benches/array_to_string.rs @@ -15,9 +15,10 @@ // specific language governing permissions and limitations // under the License. -use arrow::array::{ArrayRef, Float64Array, Int64Array, ListArray, StringArray}; +use arrow::array::{Array, ArrayRef, ListArray, StringArray}; use arrow::buffer::OffsetBuffer; -use arrow::datatypes::{DataType, Field}; +use arrow::datatypes::{DataType, Field, Float64Type, Int64Type}; +use arrow::util::bench_util::create_primitive_list_array_with_seed; use criterion::{BenchmarkId, Criterion, criterion_group, criterion_main}; use datafusion_common::ScalarValue; use datafusion_common::config::ConfigOptions; @@ -89,29 +90,13 @@ fn bench_array_to_string( } fn create_int64_list_array(array_size: usize) -> ArrayRef { - let mut rng = StdRng::seed_from_u64(SEED); - let values = (0..NUM_ROWS * array_size) - .map(|_| { - if rng.random::() < NULL_DENSITY { - None - } else { - Some(rng.random_range(0..1000)) - } - }) - .collect::(); - let offsets = (0..=NUM_ROWS) - .map(|i| (i * array_size) as i32) - .collect::>(); - - Arc::new( - ListArray::try_new( - Arc::new(Field::new("item", DataType::Int64, true)), - OffsetBuffer::new(offsets.into()), - Arc::new(values), - None, - ) - .unwrap(), - ) + Arc::new(create_primitive_list_array_with_seed::( + NUM_ROWS, + 0.0, + NULL_DENSITY as f32, + array_size, + SEED, + )) } fn create_nested_int64_list_array(array_size: usize) -> ArrayRef { @@ -133,29 +118,13 @@ fn create_nested_int64_list_array(array_size: usize) -> ArrayRef { } fn create_float64_list_array(array_size: usize) -> ArrayRef { - let mut rng = StdRng::seed_from_u64(SEED); - let values = (0..NUM_ROWS * array_size) - .map(|_| { - if rng.random::() < NULL_DENSITY { - None - } else { - Some(rng.random_range(-1000.0..1000.0)) - } - }) - .collect::(); - let offsets = (0..=NUM_ROWS) - .map(|i| (i * array_size) as i32) - .collect::>(); - - Arc::new( - ListArray::try_new( - Arc::new(Field::new("item", DataType::Float64, true)), - OffsetBuffer::new(offsets.into()), - Arc::new(values), - None, - ) - .unwrap(), - ) + Arc::new(create_primitive_list_array_with_seed::( + NUM_ROWS, + 0.0, + NULL_DENSITY as f32, + array_size, + SEED, + )) } fn create_string_list_array(array_size: usize) -> ArrayRef { diff --git a/datafusion/optimizer/Cargo.toml b/datafusion/optimizer/Cargo.toml index e3a6733532324..a767526feb930 100644 --- a/datafusion/optimizer/Cargo.toml +++ b/datafusion/optimizer/Cargo.toml @@ -79,3 +79,7 @@ insta = { workspace = true } [[bench]] name = "projection_unnecessary" harness = false + +[[bench]] +name = "optimize_projections" +harness = false diff --git a/datafusion/optimizer/benches/optimize_projections.rs b/datafusion/optimizer/benches/optimize_projections.rs new file mode 100644 index 0000000000000..d190c5ceabb2f --- /dev/null +++ b/datafusion/optimizer/benches/optimize_projections.rs @@ -0,0 +1,235 @@ +// 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. + +//! Micro-benchmarks for the `OptimizeProjections` logical optimizer rule. +//! +//! Each case models a plan shape typical of TPC-H, TPC-DS, or ClickBench. +//! Schemas use realistic widths and the rule operates on a fresh +//! `LogicalPlan` per iteration (construction is in the criterion setup +//! closure and excluded from measurement). + +use std::hint::black_box; + +use arrow::datatypes::{DataType, Field, Schema}; +use criterion::{BatchSize, Criterion, criterion_group, criterion_main}; +use datafusion_expr::{ + JoinType, LogicalPlan, LogicalPlanBuilder, col, lit, logical_plan::table_scan, +}; +use datafusion_functions_aggregate::expr_fn::sum; +use datafusion_optimizer::optimize_projections::OptimizeProjections; +use datafusion_optimizer::{OptimizerContext, OptimizerRule}; + +fn table(name: &str, cols: usize) -> LogicalPlan { + let fields: Vec = (0..cols) + .map(|i| Field::new(format!("c{i}"), DataType::Int32, true)) + .collect(); + table_scan(Some(name), &Schema::new(fields), None) + .unwrap() + .build() + .unwrap() +} + +fn scan_with_filter(name: &str, cols: usize, filter_col: usize) -> LogicalPlan { + LogicalPlanBuilder::from(table(name, cols)) + .filter(col(format!("{name}.c{filter_col}")).gt(lit(0i32))) + .unwrap() + .build() + .unwrap() +} + +/// TPC-H Q3-like: customer ⨝ orders ⨝ lineitem with filters above each scan, +/// GROUP BY 3 keys, 1 SUM aggregate. Models the canonical filter→join→aggregate +/// analytical shape after PushDownFilter. +fn plan_tpch_q3() -> LogicalPlan { + let customer = scan_with_filter("customer", 8, 6); + let orders = scan_with_filter("orders", 9, 4); + let lineitem = scan_with_filter("lineitem", 16, 10); + + LogicalPlanBuilder::from(customer) + .join_on( + orders, + JoinType::Inner, + vec![col("customer.c0").eq(col("orders.c1"))], + ) + .unwrap() + .join_on( + lineitem, + JoinType::Inner, + vec![col("lineitem.c0").eq(col("orders.c0"))], + ) + .unwrap() + .aggregate( + vec![col("lineitem.c0"), col("orders.c4"), col("orders.c7")], + vec![sum(col("lineitem.c5") - col("lineitem.c6"))], + ) + .unwrap() + .build() + .unwrap() +} + +/// TPC-H Q5-like: 6-way join through region→nation→customer→orders→lineitem +/// →supplier, GROUP BY 1 key, 1 SUM. Exercises nested-join pruning depth. +fn plan_tpch_q5() -> LogicalPlan { + let region = scan_with_filter("region", 3, 1); + let nation = table("nation", 4); + let customer = table("customer", 8); + let orders = table("orders", 9); + let lineitem = table("lineitem", 16); + let supplier = table("supplier", 7); + + LogicalPlanBuilder::from(region) + .join_on( + nation, + JoinType::Inner, + vec![col("region.c0").eq(col("nation.c2"))], + ) + .unwrap() + .join_on( + customer, + JoinType::Inner, + vec![col("nation.c0").eq(col("customer.c3"))], + ) + .unwrap() + .join_on( + orders, + JoinType::Inner, + vec![col("customer.c0").eq(col("orders.c1"))], + ) + .unwrap() + .join_on( + lineitem, + JoinType::Inner, + vec![col("lineitem.c0").eq(col("orders.c0"))], + ) + .unwrap() + .join_on( + supplier, + JoinType::Inner, + vec![col("lineitem.c2").eq(col("supplier.c0"))], + ) + .unwrap() + .aggregate( + vec![col("nation.c1")], + vec![sum(col("lineitem.c5") - col("lineitem.c6"))], + ) + .unwrap() + .build() + .unwrap() +} + +/// ClickBench-style: single wide `hits` table (100 cols), conjunctive filter, +/// GROUP BY 2 keys, 2 SUM aggregates. Stresses wide-schema column lookup. +fn plan_clickbench_groupby() -> LogicalPlan { + let hits = table("hits", 100); + let predicate = col("hits.c5") + .gt(lit(100i32)) + .and(col("hits.c12").lt(lit(1000i32))); + LogicalPlanBuilder::from(hits) + .filter(predicate) + .unwrap() + .aggregate( + vec![col("hits.c3"), col("hits.c7")], + vec![sum(col("hits.c42")), sum(col("hits.c60"))], + ) + .unwrap() + .build() + .unwrap() +} + +/// TPC-DS-style CTE shape: a SubqueryAlias wrapping a filter+projection over +/// a wide fact table, joined back on two dimension tables and aggregated. +fn plan_tpcds_subquery() -> LogicalPlan { + let store_sales = table("store_sales", 23); + let customer = table("customer", 18); + let item = table("item", 22); + + let sub = LogicalPlanBuilder::from(store_sales) + .filter(col("store_sales.c5").gt(lit(0i32))) + .unwrap() + .project(vec![ + col("store_sales.c0"), + col("store_sales.c3"), + col("store_sales.c13"), + ]) + .unwrap() + .alias("sub") + .unwrap() + .build() + .unwrap(); + + LogicalPlanBuilder::from(customer) + .join_on( + sub, + JoinType::Inner, + vec![col("customer.c0").eq(col("sub.c3"))], + ) + .unwrap() + .join_on( + item, + JoinType::Inner, + vec![col("item.c0").eq(col("sub.c0"))], + ) + .unwrap() + .aggregate(vec![col("customer.c2")], vec![sum(col("sub.c13"))]) + .unwrap() + .build() + .unwrap() +} + +/// Narrow 10-column table, single filter, project 3 cols. Guards against +/// regressions on the common small-schema case where a lookup-map fix for +/// wide schemas might hurt by adding hashing overhead. +fn plan_small_schema() -> LogicalPlan { + LogicalPlanBuilder::from(table("t", 10)) + .filter(col("t.c3").gt(lit(0i32))) + .unwrap() + .project(vec![col("t.c0"), col("t.c1"), col("t.c5")]) + .unwrap() + .build() + .unwrap() +} + +type BenchCase = (&'static str, fn() -> LogicalPlan); + +fn bench_optimize_projections(c: &mut Criterion) { + let rule = OptimizeProjections::new(); + let config = OptimizerContext::new(); + let mut group = c.benchmark_group("optimize_projections"); + + let cases: &[BenchCase] = &[ + ("tpch_q3", plan_tpch_q3), + ("tpch_q5", plan_tpch_q5), + ("clickbench_groupby", plan_clickbench_groupby), + ("tpcds_subquery", plan_tpcds_subquery), + ("small_schema", plan_small_schema), + ]; + + for (name, build) in cases { + group.bench_function(*name, |b| { + b.iter_batched( + build, + |plan| black_box(rule.rewrite(plan, &config).unwrap()), + BatchSize::SmallInput, + ); + }); + } + + group.finish(); +} + +criterion_group!(benches, bench_optimize_projections); +criterion_main!(benches); diff --git a/datafusion/optimizer/src/analyzer/type_coercion.rs b/datafusion/optimizer/src/analyzer/type_coercion.rs index 253428288ff49..bb17d13f5c769 100644 --- a/datafusion/optimizer/src/analyzer/type_coercion.rs +++ b/datafusion/optimizer/src/analyzer/type_coercion.rs @@ -1912,7 +1912,10 @@ mod test { .err() .unwrap() .strip_backtrace(); - assert!(err.starts_with("Error during planning: Failed to coerce arguments to satisfy a call to 'avg' function: coercion from Utf8 to the signature Uniform(1, [Int8, Int16, Int32, Int64, UInt8, UInt16, UInt32, UInt64, Float16, Float32, Float64]) failed")); + assert!( + err.contains("Function 'avg' failed to match any signature"), + "Err: {err:?}" + ); Ok(()) } diff --git a/datafusion/optimizer/src/decorrelate_lateral_join.rs b/datafusion/optimizer/src/decorrelate_lateral_join.rs index b9c9fabb8efb2..ea25ab479f070 100644 --- a/datafusion/optimizer/src/decorrelate_lateral_join.rs +++ b/datafusion/optimizer/src/decorrelate_lateral_join.rs @@ -28,7 +28,7 @@ use datafusion_expr::{Expr, Join, expr}; use datafusion_common::tree_node::{ Transformed, TransformedResult, TreeNode, TreeNodeRecursion, }; -use datafusion_common::{Column, DFSchema, Result, TableReference}; +use datafusion_common::{Column, DFSchema, Result, ScalarValue, TableReference}; use datafusion_expr::logical_plan::{JoinType, Subquery}; use datafusion_expr::utils::conjunction; use datafusion_expr::{LogicalPlan, LogicalPlanBuilder, SubqueryAlias}; @@ -71,14 +71,13 @@ impl OptimizerRule for DecorrelateLateralJoin { } } -// Build the decorrelated join based on the original lateral join query. For -// now, we only support cross/inner lateral joins. +// Build the decorrelated join based on the original lateral join query. +// Supports INNER and LEFT lateral joins. fn rewrite_internal(join: Join) -> Result> { - // TODO: Support outer joins - // - if join.join_type != JoinType::Inner { + if !matches!(join.join_type, JoinType::Inner | JoinType::Left) { return Ok(Transformed::no(LogicalPlan::Join(join))); } + let original_join_type = join.join_type; // The right side is wrapped in a Subquery node when it contains outer // references. Quickly skip joins that don't have this structure. @@ -106,7 +105,7 @@ fn rewrite_internal(join: Join) -> Result> { let original_join_filter = join.filter.clone(); // Walk the subquery plan bottom-up, extracting correlated filter - // predicates into join conditions and converting scalar aggregates + // predicates into join conditions and converting ungrouped aggregates // into group-by aggregates keyed on the correlation columns. let mut pull_up = PullUpCorrelatedExpr::new().with_need_handle_count_bug(true); let rewritten_subquery = subquery_plan.clone().rewrite(&mut pull_up).data()?; @@ -120,12 +119,9 @@ fn rewrite_internal(join: Join) -> Result> { return Ok(Transformed::no(LogicalPlan::Join(join))); } - // We apply the correlation predicates (extracted from the subquery's WHERE) - // as the ON clause of the rewritten join. The original ON clause is applied - // as a post-join predicate. Semantically, this is important when the join - // is rewritten as a left join; we only want outer join semantics for the - // correlation predicates (which is required for "count bug" handling), not - // the original join predicates. + // The correlation predicates (extracted from the subquery's WHERE) become + // the rewritten join's ON clause. See below for discussion of how the + // user's original ON clause is handled. let correlation_filter = conjunction(pull_up.join_filters); // Look up each aggregate's default value on empty input (e.g., COUNT → 0, @@ -157,23 +153,85 @@ fn rewrite_internal(join: Join) -> Result> { (rewritten_subquery, correlation_filter, original_join_filter) }; - // Use a left join when a scalar aggregation was pulled up (preserves - // outer rows with no matches), otherwise keep inner join. - // SELECT * FROM t0, LATERAL (SELECT sum(v1) FROM t1 WHERE t0.v0 = t1.v0); → left join - // SELECT * FROM t0, LATERAL (SELECT * FROM t1 WHERE t0.v0 = t1.v0); → inner join - let join_type = if pull_up.pulled_up_scalar_agg { - JoinType::Left - } else { - JoinType::Inner - }; + // For LEFT lateral joins, verify that all column references in the + // correlation filter are resolvable within the join's left and right + // schemas. If the lateral subquery references columns from an outer scope, + // the extracted filter will contain unresolvable columns and we must skip + // decorrelation. + // + // INNER lateral joins do not need this check: later optimizer passes + // (filter pushdown, join reordering) can restructure the plan to resolve + // cross-scope references. LEFT joins cannot be freely reordered. + if original_join_type == JoinType::Left + && let Some(ref filter) = correlation_filter + { + let left_schema = join.left.schema(); + let right_schema = right_plan.schema(); + let has_outer_scope_refs = filter + .column_refs() + .iter() + .any(|col| !left_schema.has_column(col) && !right_schema.has_column(col)); + if has_outer_scope_refs { + return Ok(Transformed::no(LogicalPlan::Join(join))); + } + } + + // Use a left join when the user wrote LEFT LATERAL or when a scalar + // aggregation was pulled up (preserves outer rows with no matches). + let join_type = + if original_join_type == JoinType::Left || pull_up.pulled_up_scalar_agg { + JoinType::Left + } else { + JoinType::Inner + }; + + // The correlation predicates (extracted from the subquery's WHERE) are + // turned into the rewritten join's ON clause. There are three cases that + // determine how the user's original ON clause is handled: + // + // - INNER lateral: user ON clause becomes a post-join filter. This restores + // inner-join semantics if the join is upgraded to LEFT for count-bug + // handling. + // + // - LEFT lateral with grouped (or no) agg: user ON clause is merged into + // the rewritten ON clause, alongside the correlation predicates. LEFT + // join semantics correctly preserve unmatched rows with NULLs. + // + // - LEFT lateral with an ungrouped aggregate (which decorrelation converts + // to a group-by keyed on the correlation columns): user ON clause cannot + // be placed in the join condition (it would conflict with count-bug + // compensation) or as a post-join filter (that would remove + // left-preserved rows). Instead, a projection is added after count-bug + // compensation that replaces each right-side column with NULL when the ON + // condition is not satisfied: + // + // CASE WHEN (on_cond) IS NOT TRUE THEN NULL ELSE END + // + // This simulates LEFT JOIN semantics for the user's ON clause without + // interfering with count-bug compensation. + let (join_filter, post_join_filter, on_condition_for_projection) = + if original_join_type == JoinType::Left { + if pull_up.pulled_up_scalar_agg { + (correlation_filter, None, original_join_filter) + } else { + let combined = conjunction( + correlation_filter.into_iter().chain(original_join_filter), + ); + (combined, None, None) + } + } else { + (correlation_filter, original_join_filter, None) + }; + let left_field_count = join.left.schema().fields().len(); let new_plan = LogicalPlanBuilder::from(join.left) - .join_on(right_plan, join_type, correlation_filter)? + .join_on(right_plan, join_type, join_filter)? .build()?; - // Handle the count bug: after a left join, unmatched outer rows get NULLs - // for all right-side columns. But COUNT(*) over an empty group should - // return 0, not NULL. Add a projection that wraps affected expressions: + // Handle the count bug: in the rewritten left join, unmatched outer + // rows get NULLs for all right-side columns. But some aggregates + // have non-NULL defaults on empty input (e.g., COUNT returns 0, not + // NULL). Add a projection that wraps those columns: // CASE WHEN __always_true IS NULL THEN ELSE END let new_plan = if let Some(expr_map) = collected_count_expr_map { let join_schema = new_plan.schema(); @@ -202,12 +260,7 @@ fn rewrite_internal(join: Join) -> Result> { )], else_expr: Some(Box::new(col)), }); - proj_exprs.push(Expr::Alias(expr::Alias { - expr: Box::new(case_expr), - relation: qualifier.cloned(), - name: name.to_string(), - metadata: None, - })); + proj_exprs.push(case_expr.alias_qualified(qualifier.cloned(), name)); continue; } proj_exprs.push(col); @@ -220,8 +273,47 @@ fn rewrite_internal(join: Join) -> Result> { new_plan }; - // Apply the original ON clause as a post-join filter. - let new_plan = if let Some(on_filter) = original_join_filter { + // For LEFT lateral joins with an ungrouped aggregate, simulate LEFT JOIN + // semantics for the user's ON clause by adding a projection that replaces + // right-side columns with NULL when the ON condition is false (see + // commentary above). + // + // Note: the ON condition expression is duplicated per column, so this + // assumes it is deterministic. + let new_plan = if let Some(on_cond) = on_condition_for_projection { + let schema = Arc::clone(new_plan.schema()); + let mut proj_exprs: Vec = vec![]; + + for (i, (qualifier, field)) in schema.iter().enumerate() { + let col = Expr::Column(Column::new(qualifier.cloned(), field.name())); + + if i < left_field_count { + proj_exprs.push(col); + continue; + } + + let typed_null = + Expr::Literal(ScalarValue::try_from(field.data_type())?, None); + let case_expr = Expr::Case(expr::Case { + expr: None, + when_then_expr: vec![( + Box::new(Expr::IsNotTrue(Box::new(on_cond.clone()))), + Box::new(typed_null), + )], + else_expr: Some(Box::new(col)), + }); + proj_exprs.push(case_expr.alias_qualified(qualifier.cloned(), field.name())); + } + + LogicalPlanBuilder::from(new_plan) + .project(proj_exprs)? + .build()? + } else { + new_plan + }; + + // Apply the original ON clause as a post-join filter (INNER lateral only). + let new_plan = if let Some(on_filter) = post_join_filter { LogicalPlanBuilder::from(new_plan) .filter(on_filter)? .build()? diff --git a/datafusion/optimizer/src/eliminate_duplicated_expr.rs b/datafusion/optimizer/src/eliminate_duplicated_expr.rs index 113c92c2c8e99..97aa6e1d8480d 100644 --- a/datafusion/optimizer/src/eliminate_duplicated_expr.rs +++ b/datafusion/optimizer/src/eliminate_duplicated_expr.rs @@ -19,8 +19,8 @@ use crate::optimizer::ApplyOrder; use crate::{OptimizerConfig, OptimizerRule}; -use datafusion_common::Result; use datafusion_common::tree_node::Transformed; +use datafusion_common::{Result, get_required_sort_exprs_indices, internal_err}; use datafusion_expr::logical_plan::LogicalPlan; use datafusion_expr::{Aggregate, Expr, Sort, SortExpr}; use std::hash::{Hash, Hasher}; @@ -76,12 +76,36 @@ impl OptimizerRule for EliminateDuplicatedExpr { .map(|wrapper| wrapper.0) .collect(); + let sort_expr_names = unique_exprs + .iter() + .map(|sort_expr| sort_expr.expr.schema_name().to_string()) + .collect::>(); + let required_indices = get_required_sort_exprs_indices( + sort.input.schema().as_ref(), + &sort_expr_names, + ); + + let unique_exprs = if required_indices.len() < unique_exprs.len() { + required_indices + .into_iter() + .map(|idx| unique_exprs[idx].clone()) + .collect() + } else { + unique_exprs + }; + let transformed = if len != unique_exprs.len() { Transformed::yes } else { Transformed::no }; + if unique_exprs.is_empty() { + return internal_err!( + "FD pruning unexpectedly removed all ORDER BY expressions" + ); + } + Ok(transformed(LogicalPlan::Sort(Sort { expr: unique_exprs, input: sort.input, @@ -130,7 +154,8 @@ mod tests { @ $expected:literal $(,)? ) => {{ let optimizer_ctx = OptimizerContext::new().with_max_passes(1); - let rules: Vec> = vec![Arc::new(EliminateDuplicatedExpr::new())]; + let rules: Vec> = + vec![Arc::new(EliminateDuplicatedExpr::new())]; assert_optimized_plan_eq_snapshot!( optimizer_ctx, rules, diff --git a/datafusion/optimizer/src/optimize_projections/mod.rs b/datafusion/optimizer/src/optimize_projections/mod.rs index 46eb6a725a4f0..14badcf1435d5 100644 --- a/datafusion/optimizer/src/optimize_projections/mod.rs +++ b/datafusion/optimizer/src/optimize_projections/mod.rs @@ -21,7 +21,6 @@ mod required_indices; use crate::optimizer::ApplyOrder; use crate::{OptimizerConfig, OptimizerRule}; -use std::collections::HashSet; use std::sync::Arc; use datafusion_common::{ @@ -147,26 +146,39 @@ fn optimize_projections( // `aggregate.aggr_expr`: let (group_by_reqs, aggregate_reqs) = indices.split_off(n_group_exprs); - // Get absolutely necessary GROUP BY fields: - let group_by_expr_existing = aggregate - .group_expr - .iter() - .map(|group_by_expr| group_by_expr.schema_name().to_string()) - .collect::>(); - - let new_group_bys = if let Some(simplest_groupby_indices) = - get_required_group_by_exprs_indices( - aggregate.input.schema(), - &group_by_expr_existing, - ) { - // Some of the fields in the GROUP BY may be required by the - // parent even if these fields are unnecessary in terms of - // functional dependency. - group_by_reqs - .append(&simplest_groupby_indices) - .get_at_indices(&aggregate.group_expr) - } else { + // Get absolutely necessary GROUP BY fields. + // + // When the input has no functional dependencies, we can + // short-circuit this analysis. + let new_group_bys = if aggregate + .input + .schema() + .functional_dependencies() + .is_empty() + { aggregate.group_expr + } else { + let group_by_expr_existing = aggregate + .group_expr + .iter() + .map(|group_by_expr| group_by_expr.schema_name().to_string()) + .collect::>(); + + if let Some(simplest_groupby_indices) = + get_required_group_by_exprs_indices( + aggregate.input.schema(), + &group_by_expr_existing, + ) + { + // Some of the fields in the GROUP BY may be required by + // the parent even if these fields are unnecessary in + // terms of functional dependency. + group_by_reqs + .append(&simplest_groupby_indices) + .get_at_indices(&aggregate.group_expr) + } else { + aggregate.group_expr + } }; // Only use the absolutely necessary aggregate expressions required @@ -682,56 +694,6 @@ fn rewrite_expr(expr: Expr, input: &Projection) -> Result> { }) } -/// Accumulates outer-referenced columns by the -/// given expression, `expr`. -/// -/// # Parameters -/// -/// * `expr` - The expression to analyze for outer-referenced columns. -/// * `columns` - A mutable reference to a `HashSet` where detected -/// columns are collected. -fn outer_columns<'a>(expr: &'a Expr, columns: &mut HashSet<&'a Column>) { - // inspect_expr_pre doesn't handle subquery references, so find them explicitly - expr.apply(|expr| { - match expr { - Expr::OuterReferenceColumn(_, col) => { - columns.insert(col); - } - Expr::ScalarSubquery(subquery) => { - outer_columns_helper_multi(&subquery.outer_ref_columns, columns); - } - Expr::Exists(exists) => { - outer_columns_helper_multi(&exists.subquery.outer_ref_columns, columns); - } - Expr::InSubquery(insubquery) => { - outer_columns_helper_multi( - &insubquery.subquery.outer_ref_columns, - columns, - ); - } - _ => {} - }; - Ok(TreeNodeRecursion::Continue) - }) - // unwrap: closure above never returns Err, so can not be Err here - .unwrap(); -} - -/// A recursive subroutine that accumulates outer-referenced columns by the -/// given expressions (`exprs`). -/// -/// # Parameters -/// -/// * `exprs` - The expressions to analyze for outer-referenced columns. -/// * `columns` - A mutable reference to a `HashSet` where detected -/// columns are collected. -fn outer_columns_helper_multi<'a, 'b>( - exprs: impl IntoIterator, - columns: &'b mut HashSet<&'a Column>, -) { - exprs.into_iter().for_each(|e| outer_columns(e, columns)); -} - /// Splits requirement indices for a join into left and right children based on /// the join type. /// diff --git a/datafusion/optimizer/src/optimize_projections/required_indices.rs b/datafusion/optimizer/src/optimize_projections/required_indices.rs index c1e0885c9b5f2..5e73a9fbeceda 100644 --- a/datafusion/optimizer/src/optimize_projections/required_indices.rs +++ b/datafusion/optimizer/src/optimize_projections/required_indices.rs @@ -17,8 +17,7 @@ //! [`RequiredIndices`] helper for OptimizeProjection -use crate::optimize_projections::outer_columns; -use datafusion_common::tree_node::TreeNodeRecursion; +use datafusion_common::tree_node::{TreeNode, TreeNodeRecursion}; use datafusion_common::{Column, DFSchemaRef, Result}; use datafusion_expr::{Expr, LogicalPlan}; @@ -105,44 +104,59 @@ impl RequiredIndices { /// Adds the indices of the fields referred to by the given expression /// `expr` within the given schema (`input_schema`). /// - /// Self is NOT compacted (and thus this method is not pub) + /// Self is NOT compacted (duplicate indices are removed by a subsequent + /// [`Self::compact`] call), and thus this method is not pub. /// /// # Parameters /// /// * `input_schema`: The input schema to analyze for index requirements. /// * `expr`: An expression for which we want to find necessary field indices. fn add_expr(&mut self, input_schema: &DFSchemaRef, expr: &Expr) { - // TODO could remove these clones (and visit the expression directly) - let mut cols = expr.column_refs(); - // Get outer-referenced (subquery) columns: - outer_columns(expr, &mut cols); - self.indices.reserve(cols.len()); - for col in cols { - if let Some(idx) = input_schema.maybe_index_of_column(col) { - self.indices.push(idx); + // `apply` does not descend into subqueries, so recurse manually to + // handle those cases. + expr.apply(|e| { + match e { + Expr::Column(c) | Expr::OuterReferenceColumn(_, c) => { + if let Some(idx) = input_schema.maybe_index_of_column(c) { + self.indices.push(idx); + } + } + Expr::ScalarSubquery(sub) => { + self.add_exprs(input_schema, &sub.outer_ref_columns); + } + Expr::Exists(ex) => { + self.add_exprs(input_schema, &ex.subquery.outer_ref_columns); + } + Expr::InSubquery(isq) => { + self.add_exprs(input_schema, &isq.subquery.outer_ref_columns); + } + _ => {} } + Ok(TreeNodeRecursion::Continue) + }) + .expect("traversal is infallible"); + } + + /// Like [`Self::add_expr`], but for multiple expressions. + fn add_exprs<'a>( + &mut self, + input_schema: &DFSchemaRef, + exprs: impl IntoIterator, + ) { + for expr in exprs { + self.add_expr(input_schema, expr); } } /// Adds the indices of the fields referred to by the given expressions - /// `within the given schema. - /// - /// # Parameters - /// - /// * `input_schema`: The input schema to analyze for index requirements. - /// * `exprs`: the expressions for which we want to find field indices. + /// within the given schema. pub fn with_exprs<'a>( - self, + mut self, schema: &DFSchemaRef, exprs: impl IntoIterator, ) -> Self { - exprs - .into_iter() - .fold(self, |mut acc, expr| { - acc.add_expr(schema, expr); - acc - }) - .compact() + self.add_exprs(schema, exprs); + self.compact() } /// Adds all `indices` into this instance. diff --git a/datafusion/physical-expr-common/src/physical_expr.rs b/datafusion/physical-expr-common/src/physical_expr.rs index a42a1560cb769..9faff1b12320a 100644 --- a/datafusion/physical-expr-common/src/physical_expr.rs +++ b/datafusion/physical-expr-common/src/physical_expr.rs @@ -687,6 +687,156 @@ pub fn is_volatile(expr: &Arc) -> bool { is_volatile } +/// A transparent wrapper that marks a [`PhysicalExpr`] as *optional* — i.e., +/// droppable without affecting query correctness. +/// +/// This is used for filters that are performance hints (e.g., dynamic join +/// filters) as opposed to mandatory predicates. The selectivity tracker can +/// detect this wrapper via `expr.as_any().downcast_ref::()` +/// and choose to drop the filter entirely when it is not cost-effective. +/// +/// All [`PhysicalExpr`] methods are delegated to the wrapped inner expression. +/// +/// Currently used by `HashJoinExec` for dynamic join filters. When the +/// selectivity tracker drops such a filter, the join still enforces +/// correctness independently — "dropped" simply means the filter is never +/// applied as a scan-time optimization. +#[derive(Debug)] +pub struct OptionalFilterPhysicalExpr { + inner: Arc, +} + +impl OptionalFilterPhysicalExpr { + /// Create a new optional filter wrapping the given expression. + pub fn new(inner: Arc) -> Self { + Self { inner } + } + + /// Returns a clone of the inner (unwrapped) expression. + pub fn inner(&self) -> Arc { + Arc::clone(&self.inner) + } +} + +impl Display for OptionalFilterPhysicalExpr { + fn fmt(&self, f: &mut Formatter<'_>) -> fmt::Result { + write!(f, "Optional({})", self.inner) + } +} + +impl PartialEq for OptionalFilterPhysicalExpr { + fn eq(&self, other: &Self) -> bool { + self.inner.as_ref() == other.inner.as_ref() + } +} + +impl Eq for OptionalFilterPhysicalExpr {} + +impl Hash for OptionalFilterPhysicalExpr { + fn hash(&self, state: &mut H) { + self.inner.as_ref().hash(state); + } +} + +impl PhysicalExpr for OptionalFilterPhysicalExpr { + fn data_type(&self, input_schema: &Schema) -> Result { + self.inner.data_type(input_schema) + } + + fn nullable(&self, input_schema: &Schema) -> Result { + self.inner.nullable(input_schema) + } + + fn evaluate(&self, batch: &RecordBatch) -> Result { + self.inner.evaluate(batch) + } + + fn return_field(&self, input_schema: &Schema) -> Result { + self.inner.return_field(input_schema) + } + + fn evaluate_selection( + &self, + batch: &RecordBatch, + selection: &BooleanArray, + ) -> Result { + self.inner.evaluate_selection(batch, selection) + } + + fn children(&self) -> Vec<&Arc> { + vec![&self.inner] + } + + fn with_new_children( + self: Arc, + children: Vec>, + ) -> Result> { + assert_eq_or_internal_err!( + children.len(), + 1, + "OptionalFilterPhysicalExpr: expected 1 child" + ); + Ok(Arc::new(OptionalFilterPhysicalExpr::new(Arc::clone( + &children[0], + )))) + } + + fn evaluate_bounds(&self, children: &[&Interval]) -> Result { + self.inner.evaluate_bounds(children) + } + + fn propagate_constraints( + &self, + interval: &Interval, + children: &[&Interval], + ) -> Result>> { + self.inner.propagate_constraints(interval, children) + } + + fn evaluate_statistics(&self, children: &[&Distribution]) -> Result { + self.inner.evaluate_statistics(children) + } + + fn propagate_statistics( + &self, + parent: &Distribution, + children: &[&Distribution], + ) -> Result>> { + self.inner.propagate_statistics(parent, children) + } + + fn get_properties(&self, children: &[ExprProperties]) -> Result { + self.inner.get_properties(children) + } + + fn fmt_sql(&self, f: &mut Formatter<'_>) -> fmt::Result { + self.inner.fmt_sql(f) + } + + fn snapshot(&self) -> Result>> { + // Always unwrap the Optional wrapper for snapshot consumers (e.g. PruningPredicate). + // If inner has a snapshot, use it; otherwise return the inner directly. + Ok(Some(match self.inner.snapshot()? { + Some(snap) => snap, + None => Arc::clone(&self.inner), + })) + } + + fn snapshot_generation(&self) -> u64 { + // The wrapper itself is not dynamic; tree-walking picks up + // inner's generation via children(). + 0 + } + + fn is_volatile_node(&self) -> bool { + self.inner.is_volatile_node() + } + + fn placement(&self) -> ExpressionPlacement { + self.inner.placement() + } +} + #[cfg(test)] mod test { use crate::physical_expr::PhysicalExpr; @@ -694,6 +844,7 @@ mod test { use arrow::datatypes::{DataType, Schema}; use datafusion_expr_common::columnar_value::ColumnarValue; use std::fmt::{Display, Formatter}; + use std::hash::{Hash, Hasher}; use std::sync::Arc; #[derive(Debug, PartialEq, Eq, Hash)] @@ -868,4 +1019,104 @@ mod test { &BooleanArray::from(vec![true; 5]), ); } + + #[test] + fn test_optional_filter_downcast() { + use super::OptionalFilterPhysicalExpr; + + let inner: Arc = Arc::new(TestExpr {}); + let optional = Arc::new(OptionalFilterPhysicalExpr::new(Arc::clone(&inner))); + + // Can downcast to detect the wrapper + let as_physical: Arc = optional; + assert!( + as_physical + .downcast_ref::() + .is_some() + ); + + // Inner expr is NOT detectable as optional + assert!(inner.downcast_ref::().is_none()); + } + + #[test] + fn test_optional_filter_delegates_evaluate() { + use super::OptionalFilterPhysicalExpr; + + let inner: Arc = Arc::new(TestExpr {}); + let optional = OptionalFilterPhysicalExpr::new(Arc::clone(&inner)); + + let batch = + unsafe { RecordBatch::new_unchecked(Arc::new(Schema::empty()), vec![], 5) }; + let result = optional.evaluate(&batch).unwrap(); + let array = result.to_array(5).unwrap(); + assert_eq!(array.len(), 5); + } + + #[test] + fn test_optional_filter_children_and_with_new_children() { + use super::OptionalFilterPhysicalExpr; + + let inner: Arc = Arc::new(TestExpr {}); + let optional = Arc::new(OptionalFilterPhysicalExpr::new(Arc::clone(&inner))); + + // children() returns the inner + let children = optional.children(); + assert_eq!(children.len(), 1); + + // with_new_children preserves the wrapper + let new_inner: Arc = Arc::new(TestExpr {}); + let rewrapped = Arc::clone(&optional) + .with_new_children(vec![new_inner]) + .unwrap(); + assert!( + rewrapped + .downcast_ref::() + .is_some() + ); + } + + #[test] + fn test_optional_filter_inner() { + use super::OptionalFilterPhysicalExpr; + + let inner: Arc = Arc::new(TestExpr {}); + let optional = OptionalFilterPhysicalExpr::new(Arc::clone(&inner)); + + // inner() returns a clone of the wrapped expression + let unwrapped = optional.inner(); + assert!(unwrapped.downcast_ref::().is_some()); + } + + #[test] + fn test_optional_filter_snapshot_generation_zero() { + use super::OptionalFilterPhysicalExpr; + + let inner: Arc = Arc::new(TestExpr {}); + let optional = OptionalFilterPhysicalExpr::new(inner); + + assert_eq!(optional.snapshot_generation(), 0); + } + + #[test] + fn test_optional_filter_eq_hash() { + use super::OptionalFilterPhysicalExpr; + use std::collections::hash_map::DefaultHasher; + + let inner1: Arc = Arc::new(TestExpr {}); + let inner2: Arc = Arc::new(TestExpr {}); + + let opt1 = OptionalFilterPhysicalExpr::new(inner1); + let opt2 = OptionalFilterPhysicalExpr::new(inner2); + + // Same inner type → equal + assert_eq!(opt1, opt2); + + // Same hash + let mut h1 = DefaultHasher::new(); + let mut h2 = DefaultHasher::new(); + opt1.hash(&mut h1); + opt2.hash(&mut h2); + assert_eq!(h1.finish(), h2.finish()); + } } diff --git a/datafusion/physical-expr/src/projection.rs b/datafusion/physical-expr/src/projection.rs index ed4f2ff0bde46..62c4d425706d7 100644 --- a/datafusion/physical-expr/src/projection.rs +++ b/datafusion/physical-expr/src/projection.rs @@ -656,7 +656,7 @@ impl ProjectionExprs { mut stats: Statistics, output_schema: &Schema, ) -> Result { - let mut column_statistics = vec![]; + let mut column_statistics = Vec::with_capacity(self.exprs.len()); for proj_expr in self.exprs.iter() { let expr = &proj_expr.expr; diff --git a/datafusion/physical-plan/src/joins/hash_join/exec.rs b/datafusion/physical-plan/src/joins/hash_join/exec.rs index 735375441f549..2a45ca26ee9dc 100644 --- a/datafusion/physical-plan/src/joins/hash_join/exec.rs +++ b/datafusion/physical-plan/src/joins/hash_join/exec.rs @@ -90,7 +90,9 @@ use datafusion_physical_expr::projection::{ProjectionRef, combine_projections}; use datafusion_physical_expr::{PhysicalExpr, PhysicalExprRef}; use datafusion_common::hash_utils::RandomState; -use datafusion_physical_expr_common::physical_expr::fmt_sql; +use datafusion_physical_expr_common::physical_expr::{ + OptionalFilterPhysicalExpr, fmt_sql, +}; use datafusion_physical_expr_common::utils::evaluate_expressions_to_arrays; use futures::TryStreamExt; use parking_lot::Mutex; @@ -1645,9 +1647,12 @@ impl ExecutionPlan for HashJoinExec { if phase == FilterPushdownPhase::Post && self.allow_join_dynamic_filter_pushdown(config) { - // Add actual dynamic filter to right side (probe side) + // Add actual dynamic filter to right side (probe side), + // wrapped as optional so it can be dropped if ineffective. let dynamic_filter = Self::create_dynamic_filter(&self.on); - right_child = right_child.with_self_filter(dynamic_filter); + let wrapped: Arc = + Arc::new(OptionalFilterPhysicalExpr::new(dynamic_filter)); + right_child = right_child.with_self_filter(wrapped); } Ok(FilterDescription::new() @@ -1669,8 +1674,12 @@ impl ExecutionPlan for HashJoinExec { // Note that we don't check PushdDownPredicate::discrimnant because even if nothing said // "yes, I can fully evaluate this filter" things might still use it for statistics -> it's worth updating let predicate = Arc::clone(&filter.predicate); - if let Ok(dynamic_filter) = - Arc::downcast::(predicate) + // Unwrap OptionalFilterPhysicalExpr if present to get the inner DynamicFilterPhysicalExpr + let inner = predicate + .downcast_ref::() + .map(|opt| opt.inner()) + .unwrap_or(predicate); + if let Ok(dynamic_filter) = Arc::downcast::(inner) { // We successfully pushed down our self filter - we need to make a new node with the dynamic filter let new_node = self diff --git a/datafusion/physical-plan/src/joins/nested_loop_join.rs b/datafusion/physical-plan/src/joins/nested_loop_join.rs index 93d7e7258772f..030fa5bbf94ff 100644 --- a/datafusion/physical-plan/src/joins/nested_loop_join.rs +++ b/datafusion/physical-plan/src/joins/nested_loop_join.rs @@ -207,6 +207,13 @@ pub struct NestedLoopJoinExec { /// Each output stream waits on the `OnceAsync` to signal the completion of /// the build(left) side data, and buffer them all for later joining. build_side_data: OnceAsync, + /// Shared left-side spill data for OOM fallback. + /// + /// When `build_side_data` fails with OOM, the first partition to + /// initiate fallback spills the entire left side to disk. Other + /// partitions share the same spill file via this `OnceAsync`, + /// avoiding redundant re-execution of the left child. + left_spill_data: Arc>, /// Information of index and left / right placement of columns column_indices: Vec, /// Projection to apply to the output of the join @@ -290,6 +297,7 @@ impl NestedLoopJoinExecBuilder { join_type, join_schema, build_side_data: Default::default(), + left_spill_data: Arc::new(OnceAsync::default()), column_indices, projection, metrics: Default::default(), @@ -492,6 +500,7 @@ impl NestedLoopJoinExec { right, metrics: ExecutionPlanMetricsSet::new(), build_side_data: Default::default(), + left_spill_data: Arc::new(OnceAsync::default()), cache: Arc::clone(&self.cache), filter: self.filter.clone(), join_type: self.join_type, @@ -655,6 +664,7 @@ impl ExecutionPlan for NestedLoopJoinExec { SpillState::Pending { left_plan: Arc::clone(&self.left), task_context: Arc::clone(&context), + left_spill_data: Arc::clone(&self.left_spill_data), } } else { SpillState::Disabled @@ -863,6 +873,20 @@ enum NLJState { EmitLeftUnmatched, Done, } +/// Shared data for the left-side spill fallback. +/// +/// When the in-memory `OnceFut` path fails with OOM, the first partition +/// spills the entire left side to disk. This struct holds the spill file +/// reference so other partitions can read from the same file. +pub(crate) struct LeftSpillData { + /// SpillManager used to read the spill file (has the left schema) + spill_manager: SpillManager, + /// The spill file containing all left-side batches + spill_file: RefCountedTempFile, + /// Left-side schema + schema: SchemaRef, +} + /// Tracks the state of the memory-limited spill fallback for NLJ. /// /// The NLJ always starts with the standard OnceFut path. If the in-memory @@ -882,6 +906,9 @@ pub(crate) enum SpillState { left_plan: Arc, /// TaskContext for re-execution and SpillManager creation task_context: Arc, + /// Shared OnceAsync for left-side spill data. The first partition + /// to initiate fallback spills the left side; others share the file. + left_spill_data: Arc>, }, /// Fallback has been triggered. Left data is being loaded in chunks @@ -892,16 +919,20 @@ pub(crate) enum SpillState { /// State for active memory-limited spill execution. /// Boxed inside [`SpillState::Active`] to reduce enum size. pub(crate) struct SpillStateActive { - /// Left input stream for incremental buffering - left_stream: SendableRecordBatchStream, + /// Shared future for left-side spill data. All partitions wait on + /// the same future — the first to poll triggers the actual spill. + left_spill_fut: OnceFut, + /// Left input stream for incremental chunk reading (from spill file). + /// None until `left_spill_fut` resolves. + left_stream: Option, + /// Left-side schema (set once `left_spill_fut` resolves) + left_schema: Option, /// Memory reservation for left-side buffering reservation: MemoryReservation, /// Accumulated left batches for the current chunk pending_batches: Vec, - /// Left-side schema (for concat_batches) - left_schema: SchemaRef, /// SpillManager for right-side spilling - spill_manager: SpillManager, + right_spill_manager: SpillManager, /// In-progress spill file for writing right batches during first pass right_spill_in_progress: Option, /// Completed right-side spill file (available after first pass) @@ -1263,19 +1294,18 @@ impl NestedLoopJoinStream { /// Switch from the standard OnceFut path to memory-limited mode. /// - /// Re-executes the left child to get a fresh stream, creates a - /// SpillManager for right-side spilling, and transitions the spill - /// state from `Pending` to `Active`. The next call to - /// `handle_buffering_left` will dispatch to - /// `handle_buffering_left_memory_limited`. + /// Uses the shared `left_spill_data` OnceAsync so that only the first + /// partition to reach this point re-executes the left child and spills + /// it to disk. Other partitions share the same spill file. fn initiate_fallback(&mut self) -> Result<()> { // Take ownership of Pending state - let (left_plan, context) = + let (left_plan, context, left_spill_data) = match std::mem::replace(&mut self.spill_state, SpillState::Disabled) { SpillState::Pending { left_plan, task_context, - } => (left_plan, task_context), + left_spill_data, + } => (left_plan, task_context, left_spill_data), _ => { return internal_err!( "initiate_fallback called in non-Pending spill state" @@ -1283,9 +1313,42 @@ impl NestedLoopJoinStream { } }; - // Re-execute left child to get a fresh stream - let left_stream = left_plan.execute(0, Arc::clone(&context))?; - let left_schema = left_stream.schema(); + // Use OnceAsync to ensure only the first partition spills the left + // side. Other partitions will get the same OnceFut that resolves + // to the shared spill file. + let left_spill_fut = left_spill_data.try_once(|| { + let plan = Arc::clone(&left_plan); + let ctx = Arc::clone(&context); + let spill_metrics = self.metrics.spill_metrics.clone(); + Ok(async move { + let mut stream = plan.execute(0, Arc::clone(&ctx))?; + let schema = stream.schema(); + let left_spill_manager = SpillManager::new( + ctx.runtime_env(), + spill_metrics, + Arc::clone(&schema), + ) + .with_compression_type(ctx.session_config().spill_compression()); + + let result = left_spill_manager + .spill_record_batch_stream_and_return_max_batch_memory( + &mut stream, + "NestedLoopJoin left spill", + ) + .await?; + + match result { + Some((file, _max_batch_memory)) => Ok(LeftSpillData { + spill_manager: left_spill_manager, + spill_file: file, + schema, + }), + None => { + internal_err!("Left side produced no data to spill") + } + } + }) + })?; // Create reservation with can_spill for fair memory allocation let reservation = MemoryConsumer::new("NestedLoopJoinLoad[fallback]".to_string()) @@ -1294,7 +1357,7 @@ impl NestedLoopJoinStream { // Create SpillManager for right-side spilling let right_schema = self.right_data.schema(); - let spill_manager = SpillManager::new( + let right_spill_manager = SpillManager::new( context.runtime_env(), self.metrics.spill_metrics.clone(), right_schema, @@ -1302,11 +1365,12 @@ impl NestedLoopJoinStream { .with_compression_type(context.session_config().spill_compression()); self.spill_state = SpillState::Active(Box::new(SpillStateActive { - left_stream, + left_spill_fut, + left_stream: None, + left_schema: None, reservation, pending_batches: Vec::new(), - left_schema, - spill_manager, + right_spill_manager, right_spill_in_progress: None, right_spill_file: None, right_max_batch_memory: 0, @@ -1378,11 +1442,44 @@ impl NestedLoopJoinStream { ); }; + // On first entry (or after re-entry for a new chunk pass when + // left_stream was consumed), wait for the shared left spill + // future to resolve and then open a stream from the spill file. + if active.left_stream.is_none() { + match active.left_spill_fut.get_shared(cx) { + Poll::Ready(Ok(spill_data)) => { + match spill_data + .spill_manager + .read_spill_as_stream(spill_data.spill_file.clone(), None) + { + Ok(stream) => { + active.left_schema = Some(Arc::clone(&spill_data.schema)); + active.left_stream = Some(stream); + } + Err(e) => { + return ControlFlow::Break(Poll::Ready(Some(Err(e)))); + } + } + } + Poll::Ready(Err(e)) => { + return ControlFlow::Break(Poll::Ready(Some(Err(e)))); + } + Poll::Pending => { + return ControlFlow::Break(Poll::Pending); + } + } + } + + let left_stream = active + .left_stream + .as_mut() + .expect("left_stream must be set after spill future resolves"); + // Poll left stream for more batches. // Note: pending_batches may already contain a batch from the // previous chunk iteration (the batch that triggered the memory limit). loop { - match active.left_stream.poll_next_unpin(cx) { + match left_stream.poll_next_unpin(cx) { Poll::Ready(Some(Ok(batch))) => { if batch.num_rows() == 0 { continue; @@ -1431,13 +1528,18 @@ impl NestedLoopJoinStream { return ControlFlow::Continue(()); } - let merged_batch = - match concat_batches(&active.left_schema, &active.pending_batches) { - Ok(batch) => batch, - Err(e) => { - return ControlFlow::Break(Poll::Ready(Some(Err(e.into())))); - } - }; + let merged_batch = match concat_batches( + active + .left_schema + .as_ref() + .expect("left_schema must be set"), + &active.pending_batches, + ) { + Ok(batch) => batch, + Err(e) => { + return ControlFlow::Break(Poll::Ready(Some(Err(e.into())))); + } + }; active.pending_batches.clear(); // Build visited bitmap if needed for this join type @@ -1472,7 +1574,7 @@ impl NestedLoopJoinStream { // Set up right-side stream for this pass if !active.is_first_right_pass { if let Some(file) = active.right_spill_file.as_ref() { - match active.spill_manager.read_spill_as_stream( + match active.right_spill_manager.read_spill_as_stream( file.clone(), Some(active.right_max_batch_memory), ) { @@ -1487,7 +1589,7 @@ impl NestedLoopJoinStream { } else { // First pass: create InProgressSpillFile for right side match active - .spill_manager + .right_spill_manager .create_in_progress_file("NestedLoopJoin right spill") { Ok(file) => { diff --git a/datafusion/physical-plan/src/spill/spill_manager.rs b/datafusion/physical-plan/src/spill/spill_manager.rs index c81c01bbe31a3..1664256e65888 100644 --- a/datafusion/physical-plan/src/spill/spill_manager.rs +++ b/datafusion/physical-plan/src/spill/spill_manager.rs @@ -160,9 +160,22 @@ impl SpillManager { Ok(file.map(|f| (f, max_record_batch_size))) } - /// Reads a spill file as a stream. The file must be created by the current `SpillManager`. - /// This method will generate output in FIFO order: the batch appended first - /// will be read first. + /// Reads a spill file as a stream. The file must be created by the current + /// `SpillManager`; otherwise behavior is undefined. + /// + /// Output is produced in FIFO order: the batch appended first is read first. + /// + /// # Arg `max_record_batch_memory` + /// + /// Most callers should pass `None`. This is mainly useful for the + /// memory-limited sort-preserving merge path. + /// + /// When provided, this value is used only as a validation hint. If a + /// decoded batch exceeds this threshold, a debug-level log message is + /// emitted. + /// + /// That path uses the maximum spilled batch size to conservatively estimate + /// the merge degree when merging multiple sorted runs. pub fn read_spill_as_stream( &self, spill_file_path: RefCountedTempFile, diff --git a/datafusion/proto-common/proto/datafusion_common.proto b/datafusion/proto-common/proto/datafusion_common.proto index 31ece63577b4f..b159742f3697d 100644 --- a/datafusion/proto-common/proto/datafusion_common.proto +++ b/datafusion/proto-common/proto/datafusion_common.proto @@ -535,7 +535,7 @@ message ParquetOptions { bool pruning = 2; // default = true bool skip_metadata = 3; // default = true bool pushdown_filters = 5; // default = false - bool reorder_filters = 6; // default = false + reserved 6; // was reorder_filters bool force_filter_selections = 34; // default = false uint64 data_pagesize_limit = 7; // default = 1024 * 1024 uint64 write_batch_size = 8; // default = 1024 @@ -605,6 +605,18 @@ message ParquetOptions { } CdcOptions content_defined_chunking = 35; + + oneof filter_collecting_byte_ratio_threshold_opt { + double filter_collecting_byte_ratio_threshold = 40; + } + + oneof filter_confidence_z_opt { + double filter_confidence_z = 41; + } + + oneof filter_pushdown_min_bytes_per_sec_opt { + double filter_pushdown_min_bytes_per_sec = 42; + } } message CdcOptions { diff --git a/datafusion/proto-common/src/from_proto/mod.rs b/datafusion/proto-common/src/from_proto/mod.rs index 4b7a91f38c201..4804a9605bd44 100644 --- a/datafusion/proto-common/src/from_proto/mod.rs +++ b/datafusion/proto-common/src/from_proto/mod.rs @@ -1023,7 +1023,7 @@ impl TryFrom<&protobuf::ParquetOptions> for ParquetOptions { }) .unwrap_or(None), pushdown_filters: value.pushdown_filters, - reorder_filters: value.reorder_filters, + force_filter_selections: value.force_filter_selections, data_pagesize_limit: value.data_pagesize_limit as usize, write_batch_size: value.write_batch_size as usize, @@ -1092,14 +1092,20 @@ impl TryFrom<&protobuf::ParquetOptions> for ParquetOptions { use_content_defined_chunking: value.content_defined_chunking.map(|cdc| { let defaults = CdcOptions::default(); CdcOptions { - // proto3 uses 0 as the wire default for uint64; a zero chunk size is - // invalid, so treat it as "field not set" and fall back to the default. min_chunk_size: if cdc.min_chunk_size != 0 { cdc.min_chunk_size as usize } else { defaults.min_chunk_size }, max_chunk_size: if cdc.max_chunk_size != 0 { cdc.max_chunk_size as usize } else { defaults.max_chunk_size }, - // norm_level = 0 is a valid value (and the default), so pass it through directly. norm_level: cdc.norm_level, } }), + filter_pushdown_min_bytes_per_sec: value.filter_pushdown_min_bytes_per_sec_opt.map(|opt| match opt { + protobuf::parquet_options::FilterPushdownMinBytesPerSecOpt::FilterPushdownMinBytesPerSec(v) => v, + }).unwrap_or(f64::INFINITY), + filter_collecting_byte_ratio_threshold: value.filter_collecting_byte_ratio_threshold_opt.map(|opt| match opt { + protobuf::parquet_options::FilterCollectingByteRatioThresholdOpt::FilterCollectingByteRatioThreshold(v) => v, + }).unwrap_or(0.2), + filter_confidence_z: value.filter_confidence_z_opt.map(|opt| match opt { + protobuf::parquet_options::FilterConfidenceZOpt::FilterConfidenceZ(v) => v, + }).unwrap_or(2.0), }) } } diff --git a/datafusion/proto-common/src/generated/pbjson.rs b/datafusion/proto-common/src/generated/pbjson.rs index 77a3b71488ece..5ded52808966d 100644 --- a/datafusion/proto-common/src/generated/pbjson.rs +++ b/datafusion/proto-common/src/generated/pbjson.rs @@ -5782,9 +5782,6 @@ impl serde::Serialize for ParquetOptions { if self.pushdown_filters { len += 1; } - if self.reorder_filters { - len += 1; - } if self.force_filter_selections { len += 1; } @@ -5869,6 +5866,15 @@ impl serde::Serialize for ParquetOptions { if self.max_predicate_cache_size_opt.is_some() { len += 1; } + if self.filter_collecting_byte_ratio_threshold_opt.is_some() { + len += 1; + } + if self.filter_confidence_z_opt.is_some() { + len += 1; + } + if self.filter_pushdown_min_bytes_per_sec_opt.is_some() { + len += 1; + } let mut struct_ser = serializer.serialize_struct("datafusion_common.ParquetOptions", len)?; if self.enable_page_index { struct_ser.serialize_field("enablePageIndex", &self.enable_page_index)?; @@ -5882,9 +5888,6 @@ impl serde::Serialize for ParquetOptions { if self.pushdown_filters { struct_ser.serialize_field("pushdownFilters", &self.pushdown_filters)?; } - if self.reorder_filters { - struct_ser.serialize_field("reorderFilters", &self.reorder_filters)?; - } if self.force_filter_selections { struct_ser.serialize_field("forceFilterSelections", &self.force_filter_selections)?; } @@ -6037,6 +6040,27 @@ impl serde::Serialize for ParquetOptions { } } } + if let Some(v) = self.filter_collecting_byte_ratio_threshold_opt.as_ref() { + match v { + parquet_options::FilterCollectingByteRatioThresholdOpt::FilterCollectingByteRatioThreshold(v) => { + struct_ser.serialize_field("filterCollectingByteRatioThreshold", v)?; + } + } + } + if let Some(v) = self.filter_confidence_z_opt.as_ref() { + match v { + parquet_options::FilterConfidenceZOpt::FilterConfidenceZ(v) => { + struct_ser.serialize_field("filterConfidenceZ", v)?; + } + } + } + if let Some(v) = self.filter_pushdown_min_bytes_per_sec_opt.as_ref() { + match v { + parquet_options::FilterPushdownMinBytesPerSecOpt::FilterPushdownMinBytesPerSec(v) => { + struct_ser.serialize_field("filterPushdownMinBytesPerSec", v)?; + } + } + } struct_ser.end() } } @@ -6054,8 +6078,6 @@ impl<'de> serde::Deserialize<'de> for ParquetOptions { "skipMetadata", "pushdown_filters", "pushdownFilters", - "reorder_filters", - "reorderFilters", "force_filter_selections", "forceFilterSelections", "data_pagesize_limit", @@ -6110,6 +6132,12 @@ impl<'de> serde::Deserialize<'de> for ParquetOptions { "coerceInt96", "max_predicate_cache_size", "maxPredicateCacheSize", + "filter_collecting_byte_ratio_threshold", + "filterCollectingByteRatioThreshold", + "filter_confidence_z", + "filterConfidenceZ", + "filter_pushdown_min_bytes_per_sec", + "filterPushdownMinBytesPerSec", ]; #[allow(clippy::enum_variant_names)] @@ -6118,7 +6146,6 @@ impl<'de> serde::Deserialize<'de> for ParquetOptions { Pruning, SkipMetadata, PushdownFilters, - ReorderFilters, ForceFilterSelections, DataPagesizeLimit, WriteBatchSize, @@ -6147,6 +6174,9 @@ impl<'de> serde::Deserialize<'de> for ParquetOptions { BloomFilterNdv, CoerceInt96, MaxPredicateCacheSize, + FilterCollectingByteRatioThreshold, + FilterConfidenceZ, + FilterPushdownMinBytesPerSec, } impl<'de> serde::Deserialize<'de> for GeneratedField { fn deserialize(deserializer: D) -> std::result::Result @@ -6172,7 +6202,6 @@ impl<'de> serde::Deserialize<'de> for ParquetOptions { "pruning" => Ok(GeneratedField::Pruning), "skipMetadata" | "skip_metadata" => Ok(GeneratedField::SkipMetadata), "pushdownFilters" | "pushdown_filters" => Ok(GeneratedField::PushdownFilters), - "reorderFilters" | "reorder_filters" => Ok(GeneratedField::ReorderFilters), "forceFilterSelections" | "force_filter_selections" => Ok(GeneratedField::ForceFilterSelections), "dataPagesizeLimit" | "data_pagesize_limit" => Ok(GeneratedField::DataPagesizeLimit), "writeBatchSize" | "write_batch_size" => Ok(GeneratedField::WriteBatchSize), @@ -6201,6 +6230,9 @@ impl<'de> serde::Deserialize<'de> for ParquetOptions { "bloomFilterNdv" | "bloom_filter_ndv" => Ok(GeneratedField::BloomFilterNdv), "coerceInt96" | "coerce_int96" => Ok(GeneratedField::CoerceInt96), "maxPredicateCacheSize" | "max_predicate_cache_size" => Ok(GeneratedField::MaxPredicateCacheSize), + "filterCollectingByteRatioThreshold" | "filter_collecting_byte_ratio_threshold" => Ok(GeneratedField::FilterCollectingByteRatioThreshold), + "filterConfidenceZ" | "filter_confidence_z" => Ok(GeneratedField::FilterConfidenceZ), + "filterPushdownMinBytesPerSec" | "filter_pushdown_min_bytes_per_sec" => Ok(GeneratedField::FilterPushdownMinBytesPerSec), _ => Err(serde::de::Error::unknown_field(value, FIELDS)), } } @@ -6224,7 +6256,6 @@ impl<'de> serde::Deserialize<'de> for ParquetOptions { let mut pruning__ = None; let mut skip_metadata__ = None; let mut pushdown_filters__ = None; - let mut reorder_filters__ = None; let mut force_filter_selections__ = None; let mut data_pagesize_limit__ = None; let mut write_batch_size__ = None; @@ -6253,6 +6284,9 @@ impl<'de> serde::Deserialize<'de> for ParquetOptions { let mut bloom_filter_ndv_opt__ = None; let mut coerce_int96_opt__ = None; let mut max_predicate_cache_size_opt__ = None; + let mut filter_collecting_byte_ratio_threshold_opt__ = None; + let mut filter_confidence_z_opt__ = None; + let mut filter_pushdown_min_bytes_per_sec_opt__ = None; while let Some(k) = map_.next_key()? { match k { GeneratedField::EnablePageIndex => { @@ -6279,12 +6313,6 @@ impl<'de> serde::Deserialize<'de> for ParquetOptions { } pushdown_filters__ = Some(map_.next_value()?); } - GeneratedField::ReorderFilters => { - if reorder_filters__.is_some() { - return Err(serde::de::Error::duplicate_field("reorderFilters")); - } - reorder_filters__ = Some(map_.next_value()?); - } GeneratedField::ForceFilterSelections => { if force_filter_selections__.is_some() { return Err(serde::de::Error::duplicate_field("forceFilterSelections")); @@ -6467,6 +6495,24 @@ impl<'de> serde::Deserialize<'de> for ParquetOptions { } max_predicate_cache_size_opt__ = map_.next_value::<::std::option::Option<::pbjson::private::NumberDeserialize<_>>>()?.map(|x| parquet_options::MaxPredicateCacheSizeOpt::MaxPredicateCacheSize(x.0)); } + GeneratedField::FilterCollectingByteRatioThreshold => { + if filter_collecting_byte_ratio_threshold_opt__.is_some() { + return Err(serde::de::Error::duplicate_field("filterCollectingByteRatioThreshold")); + } + filter_collecting_byte_ratio_threshold_opt__ = map_.next_value::<::std::option::Option<::pbjson::private::NumberDeserialize<_>>>()?.map(|x| parquet_options::FilterCollectingByteRatioThresholdOpt::FilterCollectingByteRatioThreshold(x.0)); + } + GeneratedField::FilterConfidenceZ => { + if filter_confidence_z_opt__.is_some() { + return Err(serde::de::Error::duplicate_field("filterConfidenceZ")); + } + filter_confidence_z_opt__ = map_.next_value::<::std::option::Option<::pbjson::private::NumberDeserialize<_>>>()?.map(|x| parquet_options::FilterConfidenceZOpt::FilterConfidenceZ(x.0)); + } + GeneratedField::FilterPushdownMinBytesPerSec => { + if filter_pushdown_min_bytes_per_sec_opt__.is_some() { + return Err(serde::de::Error::duplicate_field("filterPushdownMinBytesPerSec")); + } + filter_pushdown_min_bytes_per_sec_opt__ = map_.next_value::<::std::option::Option<::pbjson::private::NumberDeserialize<_>>>()?.map(|x| parquet_options::FilterPushdownMinBytesPerSecOpt::FilterPushdownMinBytesPerSec(x.0)); + } } } Ok(ParquetOptions { @@ -6474,7 +6520,6 @@ impl<'de> serde::Deserialize<'de> for ParquetOptions { pruning: pruning__.unwrap_or_default(), skip_metadata: skip_metadata__.unwrap_or_default(), pushdown_filters: pushdown_filters__.unwrap_or_default(), - reorder_filters: reorder_filters__.unwrap_or_default(), force_filter_selections: force_filter_selections__.unwrap_or_default(), data_pagesize_limit: data_pagesize_limit__.unwrap_or_default(), write_batch_size: write_batch_size__.unwrap_or_default(), @@ -6503,6 +6548,9 @@ impl<'de> serde::Deserialize<'de> for ParquetOptions { bloom_filter_ndv_opt: bloom_filter_ndv_opt__, coerce_int96_opt: coerce_int96_opt__, max_predicate_cache_size_opt: max_predicate_cache_size_opt__, + filter_collecting_byte_ratio_threshold_opt: filter_collecting_byte_ratio_threshold_opt__, + filter_confidence_z_opt: filter_confidence_z_opt__, + filter_pushdown_min_bytes_per_sec_opt: filter_pushdown_min_bytes_per_sec_opt__, }) } } diff --git a/datafusion/proto-common/src/generated/prost.rs b/datafusion/proto-common/src/generated/prost.rs index 1251a51ab0983..a712486fb8438 100644 --- a/datafusion/proto-common/src/generated/prost.rs +++ b/datafusion/proto-common/src/generated/prost.rs @@ -790,9 +790,6 @@ pub struct ParquetOptions { #[prost(bool, tag = "5")] pub pushdown_filters: bool, /// default = false - #[prost(bool, tag = "6")] - pub reorder_filters: bool, - /// default = false #[prost(bool, tag = "34")] pub force_filter_selections: bool, /// default = 1024 * 1024 @@ -874,6 +871,21 @@ pub struct ParquetOptions { pub max_predicate_cache_size_opt: ::core::option::Option< parquet_options::MaxPredicateCacheSizeOpt, >, + #[prost( + oneof = "parquet_options::FilterCollectingByteRatioThresholdOpt", + tags = "40" + )] + pub filter_collecting_byte_ratio_threshold_opt: ::core::option::Option< + parquet_options::FilterCollectingByteRatioThresholdOpt, + >, + #[prost(oneof = "parquet_options::FilterConfidenceZOpt", tags = "41")] + pub filter_confidence_z_opt: ::core::option::Option< + parquet_options::FilterConfidenceZOpt, + >, + #[prost(oneof = "parquet_options::FilterPushdownMinBytesPerSecOpt", tags = "42")] + pub filter_pushdown_min_bytes_per_sec_opt: ::core::option::Option< + parquet_options::FilterPushdownMinBytesPerSecOpt, + >, } /// Nested message and enum types in `ParquetOptions`. pub mod parquet_options { @@ -932,6 +944,21 @@ pub mod parquet_options { #[prost(uint64, tag = "33")] MaxPredicateCacheSize(u64), } + #[derive(Clone, Copy, PartialEq, ::prost::Oneof)] + pub enum FilterCollectingByteRatioThresholdOpt { + #[prost(double, tag = "40")] + FilterCollectingByteRatioThreshold(f64), + } + #[derive(Clone, Copy, PartialEq, ::prost::Oneof)] + pub enum FilterConfidenceZOpt { + #[prost(double, tag = "41")] + FilterConfidenceZ(f64), + } + #[derive(Clone, Copy, PartialEq, ::prost::Oneof)] + pub enum FilterPushdownMinBytesPerSecOpt { + #[prost(double, tag = "42")] + FilterPushdownMinBytesPerSec(f64), + } } #[derive(Clone, Copy, PartialEq, Eq, Hash, ::prost::Message)] pub struct CdcOptions { diff --git a/datafusion/proto-common/src/to_proto/mod.rs b/datafusion/proto-common/src/to_proto/mod.rs index 65089f029b866..6f2e5d2b4faa6 100644 --- a/datafusion/proto-common/src/to_proto/mod.rs +++ b/datafusion/proto-common/src/to_proto/mod.rs @@ -877,7 +877,7 @@ impl TryFrom<&ParquetOptions> for protobuf::ParquetOptions { skip_metadata: value.skip_metadata, metadata_size_hint_opt: value.metadata_size_hint.map(|v| protobuf::parquet_options::MetadataSizeHintOpt::MetadataSizeHint(v as u64)), pushdown_filters: value.pushdown_filters, - reorder_filters: value.reorder_filters, + force_filter_selections: value.force_filter_selections, data_pagesize_limit: value.data_pagesize_limit as u64, write_batch_size: value.write_batch_size as u64, @@ -911,6 +911,9 @@ impl TryFrom<&ParquetOptions> for protobuf::ParquetOptions { norm_level: cdc.norm_level, } ), + filter_pushdown_min_bytes_per_sec_opt: Some(protobuf::parquet_options::FilterPushdownMinBytesPerSecOpt::FilterPushdownMinBytesPerSec(value.filter_pushdown_min_bytes_per_sec)), + filter_collecting_byte_ratio_threshold_opt: Some(protobuf::parquet_options::FilterCollectingByteRatioThresholdOpt::FilterCollectingByteRatioThreshold(value.filter_collecting_byte_ratio_threshold)), + filter_confidence_z_opt: Some(protobuf::parquet_options::FilterConfidenceZOpt::FilterConfidenceZ(value.filter_confidence_z)), }) } } diff --git a/datafusion/proto/src/generated/datafusion_proto_common.rs b/datafusion/proto/src/generated/datafusion_proto_common.rs index 1251a51ab0983..a712486fb8438 100644 --- a/datafusion/proto/src/generated/datafusion_proto_common.rs +++ b/datafusion/proto/src/generated/datafusion_proto_common.rs @@ -790,9 +790,6 @@ pub struct ParquetOptions { #[prost(bool, tag = "5")] pub pushdown_filters: bool, /// default = false - #[prost(bool, tag = "6")] - pub reorder_filters: bool, - /// default = false #[prost(bool, tag = "34")] pub force_filter_selections: bool, /// default = 1024 * 1024 @@ -874,6 +871,21 @@ pub struct ParquetOptions { pub max_predicate_cache_size_opt: ::core::option::Option< parquet_options::MaxPredicateCacheSizeOpt, >, + #[prost( + oneof = "parquet_options::FilterCollectingByteRatioThresholdOpt", + tags = "40" + )] + pub filter_collecting_byte_ratio_threshold_opt: ::core::option::Option< + parquet_options::FilterCollectingByteRatioThresholdOpt, + >, + #[prost(oneof = "parquet_options::FilterConfidenceZOpt", tags = "41")] + pub filter_confidence_z_opt: ::core::option::Option< + parquet_options::FilterConfidenceZOpt, + >, + #[prost(oneof = "parquet_options::FilterPushdownMinBytesPerSecOpt", tags = "42")] + pub filter_pushdown_min_bytes_per_sec_opt: ::core::option::Option< + parquet_options::FilterPushdownMinBytesPerSecOpt, + >, } /// Nested message and enum types in `ParquetOptions`. pub mod parquet_options { @@ -932,6 +944,21 @@ pub mod parquet_options { #[prost(uint64, tag = "33")] MaxPredicateCacheSize(u64), } + #[derive(Clone, Copy, PartialEq, ::prost::Oneof)] + pub enum FilterCollectingByteRatioThresholdOpt { + #[prost(double, tag = "40")] + FilterCollectingByteRatioThreshold(f64), + } + #[derive(Clone, Copy, PartialEq, ::prost::Oneof)] + pub enum FilterConfidenceZOpt { + #[prost(double, tag = "41")] + FilterConfidenceZ(f64), + } + #[derive(Clone, Copy, PartialEq, ::prost::Oneof)] + pub enum FilterPushdownMinBytesPerSecOpt { + #[prost(double, tag = "42")] + FilterPushdownMinBytesPerSec(f64), + } } #[derive(Clone, Copy, PartialEq, Eq, Hash, ::prost::Message)] pub struct CdcOptions { diff --git a/datafusion/proto/src/logical_plan/file_formats.rs b/datafusion/proto/src/logical_plan/file_formats.rs index 4b111a49e42a0..8a31af28bfbdb 100644 --- a/datafusion/proto/src/logical_plan/file_formats.rs +++ b/datafusion/proto/src/logical_plan/file_formats.rs @@ -377,7 +377,7 @@ mod parquet { parquet_options::MetadataSizeHintOpt::MetadataSizeHint(size as u64) }), pushdown_filters: global_options.global.pushdown_filters, - reorder_filters: global_options.global.reorder_filters, + force_filter_selections: global_options.global.force_filter_selections, data_pagesize_limit: global_options.global.data_pagesize_limit as u64, write_batch_size: global_options.global.write_batch_size as u64, @@ -431,6 +431,9 @@ mod parquet { norm_level: cdc.norm_level, } }), + filter_pushdown_min_bytes_per_sec_opt: Some(parquet_options::FilterPushdownMinBytesPerSecOpt::FilterPushdownMinBytesPerSec(global_options.global.filter_pushdown_min_bytes_per_sec)), + filter_collecting_byte_ratio_threshold_opt: Some(parquet_options::FilterCollectingByteRatioThresholdOpt::FilterCollectingByteRatioThreshold(global_options.global.filter_collecting_byte_ratio_threshold)), + filter_confidence_z_opt: Some(parquet_options::FilterConfidenceZOpt::FilterConfidenceZ(global_options.global.filter_confidence_z)), }), column_specific_options: column_specific_options.into_iter().map(|(column_name, options)| { ParquetColumnSpecificOptions { @@ -480,7 +483,7 @@ mod parquet { parquet_options::MetadataSizeHintOpt::MetadataSizeHint(size) => *size as usize, }), pushdown_filters: proto.pushdown_filters, - reorder_filters: proto.reorder_filters, + force_filter_selections: proto.force_filter_selections, data_pagesize_limit: proto.data_pagesize_limit as usize, write_batch_size: proto.write_batch_size as usize, @@ -533,14 +536,20 @@ mod parquet { use_content_defined_chunking: proto.content_defined_chunking.map(|cdc| { let defaults = CdcOptions::default(); CdcOptions { - // proto3 uses 0 as the wire default for uint64; a zero chunk size is - // invalid, so treat it as "field not set" and fall back to the default. min_chunk_size: if cdc.min_chunk_size != 0 { cdc.min_chunk_size as usize } else { defaults.min_chunk_size }, max_chunk_size: if cdc.max_chunk_size != 0 { cdc.max_chunk_size as usize } else { defaults.max_chunk_size }, - // norm_level = 0 is a valid value (and the default), so pass it through directly. norm_level: cdc.norm_level, } }), + filter_pushdown_min_bytes_per_sec: proto.filter_pushdown_min_bytes_per_sec_opt.as_ref().map(|opt| match opt { + parquet_options::FilterPushdownMinBytesPerSecOpt::FilterPushdownMinBytesPerSec(v) => *v, + }).unwrap_or(f64::INFINITY), + filter_collecting_byte_ratio_threshold: proto.filter_collecting_byte_ratio_threshold_opt.as_ref().map(|opt| match opt { + parquet_options::FilterCollectingByteRatioThresholdOpt::FilterCollectingByteRatioThreshold(v) => *v, + }).unwrap_or(0.2), + filter_confidence_z: proto.filter_confidence_z_opt.as_ref().map(|opt| match opt { + parquet_options::FilterConfidenceZOpt::FilterConfidenceZ(v) => *v, + }).unwrap_or(2.0), } } } diff --git a/datafusion/proto/src/physical_plan/mod.rs b/datafusion/proto/src/physical_plan/mod.rs index 206f4378d3d3b..6d8552ded1016 100644 --- a/datafusion/proto/src/physical_plan/mod.rs +++ b/datafusion/proto/src/physical_plan/mod.rs @@ -863,6 +863,12 @@ impl protobuf::PhysicalPlanNode { let reader_factory = Arc::new(CachedParquetFileReaderFactory::new(store, metadata_cache)); + options.global.filter_pushdown_min_bytes_per_sec = ctx + .session_config() + .options() + .execution + .parquet + .filter_pushdown_min_bytes_per_sec; let mut source = ParquetSource::new(table_schema) .with_parquet_file_reader_factory(reader_factory) .with_table_parquet_options(options); diff --git a/datafusion/spark/src/function/string/is_valid_utf8.rs b/datafusion/spark/src/function/string/is_valid_utf8.rs new file mode 100644 index 0000000000000..04958a25317d2 --- /dev/null +++ b/datafusion/spark/src/function/string/is_valid_utf8.rs @@ -0,0 +1,118 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +use arrow::datatypes::{DataType, Field, FieldRef}; +use datafusion::logical_expr::{ColumnarValue, Signature, Volatility}; +use datafusion_common::{Result, internal_err}; +use datafusion_expr::{ReturnFieldArgs, ScalarFunctionArgs, ScalarUDFImpl}; + +use arrow::array::{Array, ArrayRef, BooleanArray}; +use arrow::buffer::BooleanBuffer; +use datafusion_common::cast::{ + as_binary_array, as_binary_view_array, as_large_binary_array, +}; +use datafusion_common::utils::take_function_args; +use datafusion_functions::utils::make_scalar_function; + +use std::sync::Arc; + +/// Spark-compatible `is_valid_utf8` expression +/// +#[derive(Debug, PartialEq, Eq, Hash)] +pub struct SparkIsValidUtf8 { + signature: Signature, +} + +impl Default for SparkIsValidUtf8 { + fn default() -> Self { + Self::new() + } +} + +impl SparkIsValidUtf8 { + pub fn new() -> Self { + Self { + signature: Signature::uniform( + 1, + vec![ + DataType::Utf8, + DataType::LargeUtf8, + DataType::Utf8View, + DataType::Binary, + DataType::BinaryView, + DataType::LargeBinary, + ], + Volatility::Immutable, + ), + } + } +} + +impl ScalarUDFImpl for SparkIsValidUtf8 { + fn name(&self) -> &str { + "is_valid_utf8" + } + + fn signature(&self) -> &Signature { + &self.signature + } + + fn return_type(&self, _arg_types: &[DataType]) -> Result { + internal_err!("return_field_from_args should be used instead") + } + + fn return_field_from_args(&self, _args: ReturnFieldArgs) -> Result { + Ok(Arc::new(Field::new(self.name(), DataType::Boolean, true))) + } + + fn invoke_with_args(&self, args: ScalarFunctionArgs) -> Result { + make_scalar_function(spark_is_valid_utf8_inner, vec![])(&args.args) + } +} + +fn spark_is_valid_utf8_inner(args: &[ArrayRef]) -> Result { + let [array] = take_function_args("is_valid_utf8", args)?; + match array.data_type() { + DataType::Utf8 | DataType::Utf8View | DataType::LargeUtf8 => { + Ok(Arc::new(BooleanArray::new( + BooleanBuffer::new_set(array.len()), + array.nulls().cloned(), + ))) + } + DataType::Binary => Ok(Arc::new( + as_binary_array(array)? + .iter() + .map(|x| x.map(|y| str::from_utf8(y).is_ok())) + .collect::(), + )), + DataType::LargeBinary => Ok(Arc::new( + as_large_binary_array(array)? + .iter() + .map(|x| x.map(|y| str::from_utf8(y).is_ok())) + .collect::(), + )), + DataType::BinaryView => Ok(Arc::new( + as_binary_view_array(array)? + .iter() + .map(|x| x.map(|y| str::from_utf8(y).is_ok())) + .collect::(), + )), + data_type => { + internal_err!("is_valid_utf8 does not support: {data_type}") + } + } +} diff --git a/datafusion/spark/src/function/string/mod.rs b/datafusion/spark/src/function/string/mod.rs index e0f6878fdea7b..64d603cb8bb67 100644 --- a/datafusion/spark/src/function/string/mod.rs +++ b/datafusion/spark/src/function/string/mod.rs @@ -22,6 +22,7 @@ pub mod concat; pub mod elt; pub mod format_string; pub mod ilike; +pub mod is_valid_utf8; pub mod length; pub mod like; pub mod luhn_check; @@ -49,6 +50,7 @@ make_udf_function!(substring::SparkSubstring, substring); make_udf_function!(base64::SparkUnBase64, unbase64); make_udf_function!(soundex::SparkSoundex, soundex); make_udf_function!(make_valid_utf8::SparkMakeValidUtf8, make_valid_utf8); +make_udf_function!(is_valid_utf8::SparkIsValidUtf8, is_valid_utf8); pub mod expr_fn { use datafusion_functions::export_functions; @@ -115,6 +117,11 @@ pub mod expr_fn { str )); export_functions!((soundex, "Returns Soundex code of the string.", str)); + export_functions!(( + is_valid_utf8, + "Returns true if str is a valid UTF-8 string, otherwise returns false", + str + )); export_functions!(( make_valid_utf8, "Returns the original string if str is a valid UTF-8 string, otherwise returns a new string whose invalid UTF8 byte sequences are replaced using the UNICODE replacement character U+FFFD.", @@ -139,5 +146,6 @@ pub fn functions() -> Vec> { unbase64(), soundex(), make_valid_utf8(), + is_valid_utf8(), ] } diff --git a/datafusion/sql/src/expr/mod.rs b/datafusion/sql/src/expr/mod.rs index d599373f0a313..a753c91162bea 100644 --- a/datafusion/sql/src/expr/mod.rs +++ b/datafusion/sql/src/expr/mod.rs @@ -40,7 +40,9 @@ use datafusion_expr::{ }; use crate::planner::{ContextProvider, PlannerContext, SqlToRel}; -use datafusion_functions_nested::expr_fn::{array_has, array_max, array_min}; +use datafusion_functions_nested::expr_fn::{ + array_has, array_max, array_min, array_position, cardinality, +}; mod binary_op; mod function; @@ -635,7 +637,11 @@ impl SqlToRel<'_, S> { schema, planner_context, ), - _ => not_impl_err!("ALL only supports subquery comparison currently"), + _ => { + let left_expr = self.sql_to_expr(*left, schema, planner_context)?; + let right_expr = self.sql_to_expr(*right, schema, planner_context)?; + plan_all_op(&left_expr, &right_expr, &compare_op) + } }, #[expect(deprecated)] SQLExpr::Wildcard(_token) => Ok(Expr::Wildcard { @@ -1297,6 +1303,64 @@ fn plan_any_op( } } +/// Plans `needle ALL(haystack)` with proper SQL NULL semantics. +/// +/// CASE/WHEN structure: +/// WHEN arr IS NULL → NULL +/// WHEN empty → TRUE +/// WHEN lhs IS NULL → NULL +/// WHEN decisive_condition → FALSE +/// WHEN has_nulls → NULL +/// ELSE → TRUE +fn plan_all_op( + needle: &Expr, + haystack: &Expr, + compare_op: &BinaryOperator, +) -> Result { + let null_arr_check = haystack.clone().is_null(); + let empty_check = cardinality(haystack.clone()).eq(lit(0u64)); + let null_lhs_check = needle.clone().is_null(); + // DataFusion's array_position uses is_null() checks internally (not equality), + // so it can locate NULL elements even though NULL = NULL is NULL in standard SQL. + let has_nulls = + array_position(haystack.clone(), lit(ScalarValue::Null), lit(1i64)).is_not_null(); + + let decisive_condition = match compare_op { + BinaryOperator::NotEq => array_has(haystack.clone(), needle.clone()), + BinaryOperator::Eq => { + let all_equal = array_min(haystack.clone()) + .eq(needle.clone()) + .and(array_max(haystack.clone()).eq(needle.clone())); + Expr::Not(Box::new(all_equal)) + } + BinaryOperator::Gt => { + Expr::Not(Box::new(needle.clone().gt(array_max(haystack.clone())))) + } + BinaryOperator::Lt => { + Expr::Not(Box::new(needle.clone().lt(array_min(haystack.clone())))) + } + BinaryOperator::GtEq => { + Expr::Not(Box::new(needle.clone().gt_eq(array_max(haystack.clone())))) + } + BinaryOperator::LtEq => { + Expr::Not(Box::new(needle.clone().lt_eq(array_min(haystack.clone())))) + } + _ => { + return plan_err!( + "Unsupported AllOp: '{compare_op}', only '=', '<>', '>', '<', '>=', '<=' are supported" + ); + } + }; + + let null_bool = lit(ScalarValue::Boolean(None)); + when(null_arr_check, null_bool.clone()) + .when(empty_check, lit(true)) + .when(null_lhs_check, null_bool.clone()) + .when(decisive_condition, lit(false)) + .when(has_nulls, null_bool) + .otherwise(lit(true)) +} + #[cfg(test)] mod tests { use std::collections::HashMap; diff --git a/datafusion/sql/src/expr/value.rs b/datafusion/sql/src/expr/value.rs index bd75ac36306fb..13a47f545cf7e 100644 --- a/datafusion/sql/src/expr/value.rs +++ b/datafusion/sql/src/expr/value.rs @@ -45,7 +45,7 @@ impl SqlToRel<'_, S> { pub(crate) fn parse_value( &self, value: Value, - param_data_types: &[FieldRef], + param_data_types: &[Option], ) -> Result { match value { Value::Number(n, _) => self.parse_sql_number(&n, false), @@ -105,7 +105,7 @@ impl SqlToRel<'_, S> { /// Both named (`$foo`) and positional (`$1`, `$2`, ...) placeholder styles are supported. fn create_placeholder_expr( param: String, - param_data_types: &[FieldRef], + param_data_types: &[Option], ) -> Result { // Try to parse the placeholder as a number. If the placeholder does not have a valid // positional value, assume we have a named placeholder. @@ -124,13 +124,13 @@ impl SqlToRel<'_, S> { // FIXME: This branch is shared by params from PREPARE and CREATE FUNCTION, but // only CREATE FUNCTION currently supports named params. For now, we rewrite // these to positional params. - let named_param_pos = param_data_types - .iter() - .position(|v| v.name() == ¶m[1..]); + let named_param_pos = param_data_types.iter().position(|v| { + v.as_ref().is_some_and(|field| field.name() == ¶m[1..]) + }); match named_param_pos { Some(pos) => Ok(Expr::Placeholder(Placeholder::new_with_field( format!("${}", pos + 1), - param_data_types.get(pos).cloned(), + param_data_types.get(pos).and_then(|v| v.clone()), ))), None => plan_err!("Unknown placeholder: {param}"), } @@ -139,13 +139,12 @@ impl SqlToRel<'_, S> { }; // Check if the placeholder is in the parameter list // FIXME: In the CREATE FUNCTION branch, param_type = None should raise an error - let param_type = param_data_types.get(idx); + let param_type = param_data_types.get(idx).and_then(|v| v.clone()); // Data type of the parameter debug!("type of param {param} param_data_types[idx]: {param_type:?}"); Ok(Expr::Placeholder(Placeholder::new_with_field( - param, - param_type.cloned(), + param, param_type, ))) } diff --git a/datafusion/sql/src/planner.rs b/datafusion/sql/src/planner.rs index 32daf65a71fa4..1bd257adab32d 100644 --- a/datafusion/sql/src/planner.rs +++ b/datafusion/sql/src/planner.rs @@ -257,7 +257,7 @@ impl IdentNormalizer { pub struct PlannerContext { /// Data types for numbered parameters ($1, $2, etc), if supplied /// in `PREPARE` statement - prepare_param_data_types: Arc>, + prepare_param_data_types: Arc>>, /// Map of CTE name to logical plan of the WITH clause. /// Use `Arc` to allow cheap cloning ctes: HashMap>, @@ -298,7 +298,7 @@ impl PlannerContext { /// Update the PlannerContext with provided prepare_param_data_types pub fn with_prepare_param_data_types( mut self, - prepare_param_data_types: Vec, + prepare_param_data_types: Vec>, ) -> Self { self.prepare_param_data_types = prepare_param_data_types.into(); self @@ -378,7 +378,7 @@ impl PlannerContext { } /// Return the types of parameters (`$1`, `$2`, etc) if known - pub fn prepare_param_data_types(&self) -> &[FieldRef] { + pub fn prepare_param_data_types(&self) -> &[Option] { &self.prepare_param_data_types } diff --git a/datafusion/sql/src/statement.rs b/datafusion/sql/src/statement.rs index b91e38e53776a..587ed02d13188 100644 --- a/datafusion/sql/src/statement.rs +++ b/datafusion/sql/src/statement.rs @@ -790,8 +790,10 @@ impl SqlToRel<'_, S> { .collect::>()?; // Create planner context with parameters - let mut planner_context = - PlannerContext::new().with_prepare_param_data_types(fields.clone()); + let mut planner_context = PlannerContext::new() + .with_prepare_param_data_types( + fields.iter().cloned().map(Some).collect(), + ); // Build logical plan for inner statement of the prepare statement let plan = self.sql_statement_to_plan_with_context_impl( @@ -808,7 +810,9 @@ impl SqlToRel<'_, S> { }) .collect(); fields.extend(param_types.iter().cloned()); - planner_context.with_prepare_param_data_types(param_types); + planner_context.with_prepare_param_data_types( + param_types.into_iter().map(Some).collect(), + ); } Ok(LogicalPlan::Statement(PlanStatement::Prepare(Prepare { @@ -1341,7 +1345,13 @@ impl SqlToRel<'_, S> { } } let mut planner_context = PlannerContext::new() - .with_prepare_param_data_types(arg_types.unwrap_or_default()); + .with_prepare_param_data_types( + arg_types + .unwrap_or_default() + .into_iter() + .map(Some) + .collect(), + ); let function_body = match function_body { Some(r) => Some(self.sql_to_expr( @@ -2340,7 +2350,12 @@ impl SqlToRel<'_, S> { } } } - let prepare_param_data_types = prepare_param_data_types.into_values().collect(); + let prepare_param_data_types = { + let len = prepare_param_data_types.keys().last().map_or(0, |&k| k + 1); + (0..len) + .map(|i| prepare_param_data_types.remove(&i)) + .collect() + }; // Projection let mut planner_context = diff --git a/datafusion/sql/src/unparser/dialect.rs b/datafusion/sql/src/unparser/dialect.rs index 194f443749e50..05438a478524c 100644 --- a/datafusion/sql/src/unparser/dialect.rs +++ b/datafusion/sql/src/unparser/dialect.rs @@ -664,6 +664,51 @@ impl Dialect for BigQueryDialect { fn unnest_as_table_factor(&self) -> bool { true } + + fn supports_column_alias_in_table_alias(&self) -> bool { + false + } + + fn float64_ast_dtype(&self) -> ast::DataType { + ast::DataType::Float64 + } + + fn utf8_cast_dtype(&self) -> ast::DataType { + ast::DataType::String(None) + } + + fn large_utf8_cast_dtype(&self) -> ast::DataType { + ast::DataType::String(None) + } + + fn timestamp_cast_dtype( + &self, + _time_unit: &TimeUnit, + _tz: &Option>, + ) -> ast::DataType { + ast::DataType::Timestamp(None, TimezoneInfo::None) + } + + fn date_field_extract_style(&self) -> DateFieldExtractStyle { + DateFieldExtractStyle::Extract + } + + fn interval_style(&self) -> IntervalStyle { + IntervalStyle::SQLStandard + } + + fn scalar_function_to_sql_overrides( + &self, + unparser: &Unparser, + func_name: &str, + args: &[Expr], + ) -> Result> { + if func_name == "date_part" { + return date_part_to_sql(unparser, self.date_field_extract_style(), args); + } + + Ok(None) + } } impl BigQueryDialect { diff --git a/datafusion/sql/src/unparser/expr.rs b/datafusion/sql/src/unparser/expr.rs index 5954e2971850c..4885dde077d13 100644 --- a/datafusion/sql/src/unparser/expr.rs +++ b/datafusion/sql/src/unparser/expr.rs @@ -3526,4 +3526,53 @@ mod tests { } Ok(()) } + + #[test] + fn test_bigquery_dialect_overrides() -> Result<()> { + let bigquery_dialect: Arc = Arc::new(BigQueryDialect::new()); + let unparser = Unparser::new(bigquery_dialect.as_ref()); + + // date_field_extract_style: EXTRACT instead of date_part + let expr = Expr::ScalarFunction(ScalarFunction { + func: Arc::new(ScalarUDF::new_from_impl( + datafusion_functions::datetime::date_part::DatePartFunc::new(), + )), + args: vec![lit("YEAR"), col("date_col")], + }); + let actual = format!("{}", unparser.expr_to_sql(&expr)?); + assert_eq!(actual, "EXTRACT(YEAR FROM `date_col`)"); + + // interval_style: SQL standard instead of PostgresVerbose + let expr = interval_year_month_lit("3 months"); + let actual = format!("{}", unparser.expr_to_sql(&expr)?); + assert_eq!(actual, "INTERVAL '3' MONTH"); + + // float64_ast_dtype: FLOAT64 instead of DOUBLE + let expr = cast(col("a"), DataType::Float64); + let actual = format!("{}", unparser.expr_to_sql(&expr)?); + assert_eq!(actual, "CAST(`a` AS FLOAT64)"); + + // supports_column_alias_in_table_alias: false + assert!(!bigquery_dialect.supports_column_alias_in_table_alias()); + + // utf8_cast_dtype: STRING instead of VARCHAR + let expr = cast(col("a"), DataType::Utf8); + let actual = format!("{}", unparser.expr_to_sql(&expr)?); + assert_eq!(actual, "CAST(`a` AS STRING)"); + + // large_utf8_cast_dtype: STRING instead of TEXT + let expr = cast(col("a"), DataType::LargeUtf8); + let actual = format!("{}", unparser.expr_to_sql(&expr)?); + assert_eq!(actual, "CAST(`a` AS STRING)"); + + // timestamp_cast_dtype: TIMESTAMP (no WITH TIME ZONE) + let expr = cast( + col("a"), + DataType::Timestamp(TimeUnit::Microsecond, Some("+00:00".into())), + ); + let actual = format!("{}", unparser.expr_to_sql(&expr)?); + assert_eq!(actual, "CAST(`a` AS TIMESTAMP)"); + + Ok(()) + } } diff --git a/datafusion/sql/tests/cases/params.rs b/datafusion/sql/tests/cases/params.rs index 396f619400c74..68c560ead68cd 100644 --- a/datafusion/sql/tests/cases/params.rs +++ b/datafusion/sql/tests/cases/params.rs @@ -1041,6 +1041,22 @@ fn test_prepare_statement_unknown_hash_param() { ); } +#[test] +fn test_insert_infer_with_function_wrapped_placeholder() { + let plan = logical_plan( + "INSERT INTO person (id, first_name, age) VALUES ($1, character_length($2), $3)", + ) + .unwrap(); + + let actual_types = plan.get_parameter_types().unwrap(); + let expected_types: HashMap> = HashMap::from([ + ("$1".to_string(), Some(DataType::UInt32)), + ("$2".to_string(), None), + ("$3".to_string(), Some(DataType::Int32)), + ]); + assert_eq!(actual_types, expected_types); +} + #[test] fn test_prepare_statement_bad_list_idx() { let sql = "SELECT id from person where id = $foo"; diff --git a/datafusion/sqllogictest/Cargo.toml b/datafusion/sqllogictest/Cargo.toml index d7bb2583c9d8c..1159b7f3b703a 100644 --- a/datafusion/sqllogictest/Cargo.toml +++ b/datafusion/sqllogictest/Cargo.toml @@ -84,9 +84,17 @@ substrait = ["datafusion-substrait"] [dev-dependencies] env_logger = { workspace = true } regex = { workspace = true } +# Required to make sure tests for pg display behaves consistently +# regardless of feature unification with dependencies +serde_json = { workspace = true, features = ["preserve_order"] } tokio = { workspace = true, features = ["rt-multi-thread"] } [[test]] harness = false name = "sqllogictests" path = "bin/sqllogictests.rs" + +# Required because we pull serde_json with a feature to get consistent pg display, +# but its not directly used. +[package.metadata.cargo-machete] +ignored = "serde_json" diff --git a/datafusion/sqllogictest/test_files/aggregate.slt b/datafusion/sqllogictest/test_files/aggregate.slt index 962d1d510395e..00ca0482a31e1 100644 --- a/datafusion/sqllogictest/test_files/aggregate.slt +++ b/datafusion/sqllogictest/test_files/aggregate.slt @@ -174,28 +174,22 @@ statement error DataFusion error: Schema error: Schema contains duplicate unqual SELECT approx_distinct(c9) count_c9, approx_distinct(cast(c9 as varchar)) count_c9_str FROM aggregate_test_100 # csv_query_approx_percentile_cont_with_weight -statement error Failed to coerce arguments to satisfy a call to 'approx_percentile_cont_with_weight' function +statement error Function 'approx_percentile_cont_with_weight' failed to match any signature SELECT approx_percentile_cont_with_weight(c2, 0.95) WITHIN GROUP (ORDER BY c1) FROM aggregate_test_100 -statement error Failed to coerce arguments to satisfy a call to 'approx_percentile_cont_with_weight' function +statement error Function 'approx_percentile_cont_with_weight' failed to match any signature SELECT approx_percentile_cont_with_weight(c1, 0.95) WITHIN GROUP (ORDER BY c3) FROM aggregate_test_100 -statement error Failed to coerce arguments to satisfy a call to 'approx_percentile_cont_with_weight' function +statement error Function 'approx_percentile_cont_with_weight' failed to match any signature SELECT approx_percentile_cont_with_weight(c2, c1) WITHIN GROUP (ORDER BY c3) FROM aggregate_test_100 # csv_query_approx_percentile_cont_with_histogram_bins statement error DataFusion error: Error during planning: Tdigest max_size value for 'APPROX_PERCENTILE_CONT' must be UInt > 0 literal \(got data type Int64\)\. SELECT c1, approx_percentile_cont(0.95, -1000) WITHIN GROUP (ORDER BY c3) AS c3_p95 FROM aggregate_test_100 GROUP BY 1 ORDER BY 1 -statement error Failed to coerce arguments to satisfy a call to 'approx_percentile_cont' function +statement error Function 'approx_percentile_cont' failed to match any signature SELECT approx_percentile_cont(0.95, c1) WITHIN GROUP (ORDER BY c3) FROM aggregate_test_100 -statement error DataFusion error: Error during planning: Failed to coerce arguments to satisfy a call to 'approx_percentile_cont' function: coercion from Int16, Float64, Float64 to the signature OneOf(.*) failed(.|\n)* -SELECT approx_percentile_cont(0.95, 111.1) WITHIN GROUP (ORDER BY c3) FROM aggregate_test_100 - -statement error DataFusion error: Error during planning: Failed to coerce arguments to satisfy a call to 'approx_percentile_cont' function: coercion from Float64, Float64, Float64 to the signature OneOf(.*) failed(.|\n)* -SELECT approx_percentile_cont(0.95, 111.1) WITHIN GROUP (ORDER BY c12) FROM aggregate_test_100 - statement error DataFusion error: Error during planning: Percentile value for 'APPROX_PERCENTILE_CONT' must be a literal SELECT approx_percentile_cont(c12) WITHIN GROUP (ORDER BY c12) FROM aggregate_test_100 @@ -948,16 +942,16 @@ SELECT c2, var_samp(CASE WHEN c12 > 0.90 THEN c12 ELSE null END) FROM aggregate_ # csv_query_approx_median_1 -query I +query R SELECT approx_median(c2) FROM aggregate_test_100 ---- 3 # csv_query_approx_median_2 -query I +query R SELECT approx_median(c6) FROM aggregate_test_100 ---- -1146409980542786560 +1146409980542786600 # csv_query_approx_median_3 query R @@ -1006,7 +1000,7 @@ SELECT median(col_i8), median(distinct col_i8) FROM median_table -14 100 # approx_distinct_median_i8 -query I +query R SELECT approx_median(distinct col_i8) FROM median_table ---- 100 @@ -2032,26 +2026,29 @@ SELECT (ABS(1 - CAST(approx_percentile_cont(0.9) WITHIN GROUP (ORDER BY c11) AS true # percentile_cont_with_nulls -query I +query R SELECT APPROX_PERCENTILE_CONT(0.5) WITHIN GROUP (ORDER BY v) FROM (VALUES (1), (2), (3), (NULL), (NULL), (NULL)) as t (v); ---- 2 # percentile_cont_with_nulls_only -query I +query R SELECT APPROX_PERCENTILE_CONT(0.5) WITHIN GROUP (ORDER BY v) FROM (VALUES (CAST(NULL as INT))) as t (v); ---- NULL +query error DataFusion error: Error during planning: Percentile value for 'APPROX_PERCENTILE_CONT' must be Float32 or Float64 literal \(got null\) +SELECT APPROX_PERCENTILE_CONT(NULL) WITHIN GROUP (ORDER BY v) FROM (VALUES (CAST(NULL as INT))) as t (v); + # percentile_cont_with_weight_with_nulls -query I +query R SELECT APPROX_PERCENTILE_CONT_WITH_WEIGHT(w, 0.5) WITHIN GROUP (ORDER BY v) FROM (VALUES (1, 1), (2, 1), (3, 1), (4, NULL), (NULL, 1), (NULL, NULL)) as t (v, w); ---- 2 # percentile_cont_with_weight_nulls_only -query I +query R SELECT APPROX_PERCENTILE_CONT_WITH_WEIGHT(1, 0.5) WITHIN GROUP (ORDER BY v) FROM (VALUES (CAST(NULL as INT))) as t (v); ---- NULL @@ -2332,36 +2329,36 @@ b 5 NULL 20135.4 b NULL NULL 7732.315789473684 # csv_query_approx_percentile_cont_with_weight -query TI +query TR SELECT c1, approx_percentile_cont(0.95) WITHIN GROUP (ORDER BY c3) AS c3_p95 FROM aggregate_test_100 GROUP BY 1 ORDER BY 1 ---- -a 73 +a 73.55 b 68 -c 122 -d 124 -e 115 +c 122.5 +d 124.2 +e 115.6 # csv_query_approx_percentile_cont_with_weight (should be the same as above) -query TI +query TR SELECT c1, approx_percentile_cont(c3, 0.95) AS c3_p95 FROM aggregate_test_100 GROUP BY 1 ORDER BY 1 ---- -a 73 +a 73.55 b 68 -c 122 -d 124 -e 115 +c 122.5 +d 124.2 +e 115.6 # using approx_percentile_cont on 2 columns with same signature -query TII +query TRR SELECT c1, approx_percentile_cont(c2, 0.95) AS c2, approx_percentile_cont(c3, 0.95) AS c3 FROM aggregate_test_100 GROUP BY 1 ORDER BY 1 ---- -a 5 73 +a 5 73.55 b 5 68 -c 5 122 -d 5 124 -e 5 115 +c 5 122.5 +d 5 124.2 +e 5 115.6 # error is unique to this UDAF query TRR @@ -2375,73 +2372,82 @@ e 3 40.333333333333 -query TI +query TR SELECT c1, approx_percentile_cont(0.95) WITHIN GROUP (ORDER BY c3 DESC) AS c3_p95 FROM aggregate_test_100 GROUP BY 1 ORDER BY 1 ---- a -101 -b -114 -c -109 -d -98 -e -93 +b -114.3 +c -109.475 +d -98.6 +e -93.65 # csv_query_approx_percentile_cont_with_weight (2) -query TI +query TR SELECT c1, approx_percentile_cont_with_weight(1, 0.95) WITHIN GROUP (ORDER BY c3) AS c3_p95 FROM aggregate_test_100 GROUP BY 1 ORDER BY 1 ---- -a 73 +a 73.55 b 68 -c 122 -d 124 -e 115 +c 122.5 +d 124.2 +e 115.6 # csv_query_approx_percentile_cont_with_weight alternate syntax -query TI +query TR SELECT c1, approx_percentile_cont_with_weight(c3, 1, 0.95) AS c3_p95 FROM aggregate_test_100 GROUP BY 1 ORDER BY 1 ---- -a 73 +a 73.55 b 68 -c 122 -d 124 -e 115 +c 122.5 +d 124.2 +e 115.6 - -query TI +query TR SELECT c1, approx_percentile_cont_with_weight(1, 0.95) WITHIN GROUP (ORDER BY c3 DESC) AS c3_p95 FROM aggregate_test_100 GROUP BY 1 ORDER BY 1 ---- a -101 -b -114 -c -109 -d -98 -e -93 +b -114.3 +c -109.475 +d -98.6 +e -93.65 # csv_query_approx_percentile_cont_with_histogram_bins -query TI +query TR SELECT c1, approx_percentile_cont(0.95, 200) WITHIN GROUP (ORDER BY c3) AS c3_p95 FROM aggregate_test_100 GROUP BY 1 ORDER BY 1 ---- -a 73 +a 73.55 b 68 -c 122 -d 124 -e 115 +c 122.5 +d 124.2 +e 115.6 -query TI +query TR +SELECT c1, approx_percentile_cont(0.95, 200.1) WITHIN GROUP (ORDER BY c3) AS c3_p95 FROM aggregate_test_100 GROUP BY 1 ORDER BY 1 +---- +a 73.55 +b 68 +c 122.5 +d 124.2 +e 115.6 + + +query TR SELECT c1, approx_percentile_cont_with_weight(c2, 0.95) WITHIN GROUP (ORDER BY c3) AS c3_p95 FROM aggregate_test_100 GROUP BY 1 ORDER BY 1 ---- a 65 b 68 c 122 -d 123 -e 110 +d 123.15 +e 110.266666666667 # approx_percentile_cont_with_weight with centroids -query TI +query TR SELECT c1, approx_percentile_cont_with_weight(c2, 0.95, 200) WITHIN GROUP (ORDER BY c3) AS c3_p95 FROM aggregate_test_100 GROUP BY 1 ORDER BY 1 ---- a 65 b 68 c 122 -d 123 -e 110 +d 123.15 +e 110.266666666667 # csv_query_sum_crossjoin query TTI @@ -3834,10 +3840,10 @@ SELECT COUNT(DISTINCT c1) FROM test # TODO: aggregate_with_alias # test_approx_percentile_cont_decimal_support -query TI +query TR SELECT c1, approx_percentile_cont(cast(0.85 as decimal(10,2))) WITHIN GROUP (ORDER BY c2) apc FROM aggregate_test_100 GROUP BY 1 ORDER BY 1 ---- -a 4 +a 4.175 b 5 c 4 d 4 @@ -5978,7 +5984,7 @@ select median(a) from (select 1 as a where 1=0); ---- NULL -query I +query R select approx_median(a) from (select 1 as a where 1=0); ---- NULL diff --git a/datafusion/sqllogictest/test_files/aggregate_skip_partial.slt b/datafusion/sqllogictest/test_files/aggregate_skip_partial.slt index e12ac5782e3a4..cd95426a9b0bb 100644 --- a/datafusion/sqllogictest/test_files/aggregate_skip_partial.slt +++ b/datafusion/sqllogictest/test_files/aggregate_skip_partial.slt @@ -145,7 +145,7 @@ GROUP BY 1, 2 ORDER BY 1 LIMIT 5; -2117946883 d -2117946883 NULL NULL NULL -2098805236 c -2098805236 NULL NULL NULL -query ITIIII +query ITRRRR SELECT c5, c1, APPROX_MEDIAN(c5), APPROX_MEDIAN(CASE WHEN c1 = 'a' THEN c5 ELSE NULL END), @@ -318,14 +318,14 @@ SELECT c2, median(c5), median(c11) FROM aggregate_test_100 GROUP BY c2 ORDER BY 5 604973998 0.49842384 # Test approx_median for int / float -query IIR +query IRR SELECT c2, approx_median(c5), approx_median(c11) FROM aggregate_test_100 GROUP BY c2 ORDER BY c2; ---- -1 191655437 0.59926736 +1 191655437.25 0.59926736 2 -587831330 0.43230486 3 240273900 0.40199697 4 762932956 0.48515016 -5 593204320 0.5156586 +5 593204320.5 0.5156586 # Test approx_distinct for varchar / int query III @@ -399,14 +399,14 @@ SELECT c2, median(c3), median(c11) FROM aggregate_test_100_null GROUP BY c2 ORDE 5 -35 0.5536642 # Test approx_median with nullable fields -query IIR +query IRR SELECT c2, approx_median(c3), approx_median(c11) FROM aggregate_test_100_null GROUP BY c2 ORDER BY c2; ---- 1 12 0.6067944 2 1 0.46076488 3 14 0.40154034 -4 -7 0.48515016 -5 -39 0.5536642 +4 -7.75 0.48515016 +5 -39.75 0.5536642 # Test approx_distinct with nullable fields query II @@ -516,7 +516,7 @@ FROM aggregate_test_100 GROUP BY c2 ORDER BY c2; 5 64 -59 # Test approx_median with filter -query III +query IRR SELECT c2, approx_median(c3) FILTER (WHERE c3 > 0), @@ -526,7 +526,7 @@ FROM aggregate_test_100 GROUP BY c2 ORDER BY c2; 1 57 -56 2 52 -60 3 71 -76 -4 65 -64 +4 65 -64.75 5 64 -59 # Test count with nullable fields and filter @@ -662,20 +662,20 @@ FROM aggregate_test_100_null GROUP BY c2 ORDER BY c2; 5 -22 # Test approx_median with nullable fields and filter -query IIR +query IRR SELECT c2, approx_median(c3) FILTER (WHERE c5 > 0), approx_median(c11) FILTER (WHERE c5 < 0) FROM aggregate_test_100_null GROUP BY c2 ORDER BY c2; ---- 1 -5 0.6623719 -2 12 0.52930677 +2 12.25 0.52930677 3 13 0.32792538 4 -38 0.49774808 -5 -21 0.47652745 +5 -21.75 0.47652745 # Test approx_median with nullable fields and nullable filter -query II +query IR SELECT c2, approx_median(c3) FILTER (WHERE c11 > 0.5) FROM aggregate_test_100_null GROUP BY c2 ORDER BY c2; diff --git a/datafusion/sqllogictest/test_files/array/array_all.slt b/datafusion/sqllogictest/test_files/array/array_all.slt new file mode 100644 index 0000000000000..70ba15edbf47b --- /dev/null +++ b/datafusion/sqllogictest/test_files/array/array_all.slt @@ -0,0 +1,221 @@ +# 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. + +## all operator + +# = ALL: true when all elements equal val +query B +select 5 = ALL(make_array(5, 5, 5)); +---- +true + +query B +select 5 = ALL(make_array(5, 5, 3)); +---- +false + +# <> ALL: true when val differs from every element +query B +select 5 <> ALL(make_array(1, 2, 3)); +---- +true + +query B +select 5 <> ALL(make_array(1, 2, 5)); +---- +false + +# > ALL: true when val greater than all elements +query B +select 10 > ALL(make_array(1, 2, 3)); +---- +true + +query B +select 3 > ALL(make_array(1, 2, 3)); +---- +false + +# < ALL: true when val less than all elements +query B +select 0 < ALL(make_array(1, 2, 3)); +---- +true + +query B +select 2 < ALL(make_array(1, 2, 3)); +---- +false + +# >= ALL: true when val >= all elements +query B +select 5 >= ALL(make_array(1, 2, 5)); +---- +true + +query B +select 4 >= ALL(make_array(1, 2, 5)); +---- +false + +# <= ALL: true when val <= all elements +query B +select 1 <= ALL(make_array(1, 2, 5)); +---- +true + +query B +select 2 <= ALL(make_array(1, 2, 5)); +---- +false + +# Empty arrays: all operators return TRUE (vacuous truth) +query B +select 5 = ALL(arrow_cast(make_array(), 'List(Int64)')); +---- +true + +query B +select 5 <> ALL(arrow_cast(make_array(), 'List(Int64)')); +---- +true + +query B +select 5 > ALL(arrow_cast(make_array(), 'List(Int64)')); +---- +true + +query B +select 5 < ALL(arrow_cast(make_array(), 'List(Int64)')); +---- +true + +query B +select 5 >= ALL(arrow_cast(make_array(), 'List(Int64)')); +---- +true + +query B +select 5 <= ALL(arrow_cast(make_array(), 'List(Int64)')); +---- +true + +# NULL LHS with empty array returns TRUE (vacuous truth) +query B +select NULL = ALL(arrow_cast(make_array(), 'List(Int64)')); +---- +true + +# NULL LHS with non-empty array returns NULL +query B +select NULL = ALL(make_array(1, 2, 3)); +---- +NULL + +query B +select NULL > ALL(make_array(1, 2, 3)); +---- +NULL + +query B +select NULL <> ALL(make_array(1, 2, 3)); +---- +NULL + +# All-NULL arrays: returns NULL +query B +select 5 = ALL(make_array(NULL::INT, NULL::INT)); +---- +NULL + +query B +select 5 <> ALL(make_array(NULL::INT, NULL::INT)); +---- +NULL + +query B +select 5 > ALL(make_array(NULL::INT, NULL::INT)); +---- +NULL + +query B +select 5 < ALL(make_array(NULL::INT, NULL::INT)); +---- +NULL + +query B +select 5 >= ALL(make_array(NULL::INT, NULL::INT)); +---- +NULL + +query B +select 5 <= ALL(make_array(NULL::INT, NULL::INT)); +---- +NULL + +# Mixed NULL + non-NULL (non-NULL elements satisfy, but NULLs present → NULL) +query B +select 5 > ALL(make_array(3, NULL)); +---- +NULL + +query B +select 5 >= ALL(make_array(5, NULL)); +---- +NULL + +query B +select 1 < ALL(make_array(3, NULL)); +---- +NULL + +query B +select 1 <= ALL(make_array(1, NULL)); +---- +NULL + +# Mixed NULL + non-NULL (not satisfying condition → FALSE wins over NULL) +query B +select 5 > ALL(make_array(6, NULL)); +---- +false + +query B +select 5 < ALL(make_array(3, NULL)); +---- +false + +query B +select 5 = ALL(make_array(5, 3, NULL)); +---- +false + +# NULL array input returns NULL +query B +select 5 = ALL(NULL::INT[]); +---- +NULL + +query B +select 5 > ALL(NULL::INT[]); +---- +NULL + +query B +select 5 < ALL(NULL::INT[]); +---- +NULL diff --git a/datafusion/sqllogictest/test_files/clickbench_extended.slt b/datafusion/sqllogictest/test_files/clickbench_extended.slt index ee3e33551ee3e..6b0d78cdba8f3 100644 --- a/datafusion/sqllogictest/test_files/clickbench_extended.slt +++ b/datafusion/sqllogictest/test_files/clickbench_extended.slt @@ -52,7 +52,7 @@ query IIIIII SELECT "ClientIP", "WatchID", COUNT(*) c, MIN("ResponseStartTiming") tmin, MEDIAN("ResponseStartTiming") tmed, MAX("ResponseStartTiming") tmax FROM hits WHERE "JavaEnable" = 0 GROUP BY "ClientIP", "WatchID" HAVING c > 1 ORDER BY tmed DESC LIMIT 10; ---- -query IIIIII +query IIIIRI SELECT "ClientIP", "WatchID", COUNT(*) c, MIN("ResponseStartTiming") tmin, APPROX_PERCENTILE_CONT(0.95) WITHIN GROUP (ORDER BY "ResponseStartTiming") tp95, MAX("ResponseStartTiming") tmax FROM 'hits' WHERE "JavaEnable" = 0 GROUP BY "ClientIP", "WatchID" HAVING c > 1 ORDER BY tp95 DESC 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 179a611d37e1f..9fa53bb5a3a4d 100644 --- a/datafusion/sqllogictest/test_files/dynamic_filter_pushdown_config.slt +++ b/datafusion/sqllogictest/test_files/dynamic_filter_pushdown_config.slt @@ -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=Optional(DynamicFilter [ empty ]) # 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=Optional(DynamicFilter [ empty ]) # 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=Optional(DynamicFilter [ empty ]) # LEFT SEMI JOIN (physical LeftSemi): reverse table roles so optimizer keeps LeftSemi # (right_parquet has 3 rows < left_parquet has 5 rows, so no swap occurs). @@ -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=Optional(DynamicFilter [ empty ]) # LEFT SEMI (physical LeftSemi) correctness: only right rows with matching left ids query IT rowsort @@ -337,8 +337,8 @@ 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)----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 ] +04)----FilterExec: Optional(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=Optional(DynamicFilter [ empty ]) 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 ] +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=Optional(DynamicFilter [ empty ]) 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=Optional(DynamicFilter [ empty ]) # LEFT MARK JOIN: the OR prevents decorrelation to LeftSemi, so the optimizer # uses LeftMark. Self-generated dynamic filter pushes to the probe side. @@ -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=Optional(DynamicFilter [ empty ]) # LEFT MARK correctness: all right rows match EXISTS, so all 3 appear query IT rowsort @@ -445,7 +445,7 @@ 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 ] +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=Optional(DynamicFilter [ empty ]) AND DynamicFilter [ empty ] # Correctness check query IT @@ -479,7 +479,7 @@ 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 ] +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=Optional(DynamicFilter [ empty ]) AND DynamicFilter [ empty ] # 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=Optional(DynamicFilter [ empty ]) # Enable TopK, disable Join statement ok @@ -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=Optional(DynamicFilter [ empty ]) # Test 6: Regression test for issue #20213 - dynamic filter applied to wrong table # when subquery join has same column names on both sides. diff --git a/datafusion/sqllogictest/test_files/information_schema.slt b/datafusion/sqllogictest/test_files/information_schema.slt index b04c78bd2774c..889602b7d60a4 100644 --- a/datafusion/sqllogictest/test_files/information_schema.slt +++ b/datafusion/sqllogictest/test_files/information_schema.slt @@ -245,6 +245,9 @@ datafusion.execution.parquet.dictionary_enabled true datafusion.execution.parquet.dictionary_page_size_limit 1048576 datafusion.execution.parquet.enable_page_index true datafusion.execution.parquet.encoding NULL +datafusion.execution.parquet.filter_collecting_byte_ratio_threshold 0.2 +datafusion.execution.parquet.filter_confidence_z 2 +datafusion.execution.parquet.filter_pushdown_min_bytes_per_sec 104857600 datafusion.execution.parquet.force_filter_selections false datafusion.execution.parquet.max_predicate_cache_size NULL datafusion.execution.parquet.max_row_group_size 1048576 @@ -253,7 +256,6 @@ datafusion.execution.parquet.maximum_parallel_row_group_writers 1 datafusion.execution.parquet.metadata_size_hint 524288 datafusion.execution.parquet.pruning true datafusion.execution.parquet.pushdown_filters false -datafusion.execution.parquet.reorder_filters false datafusion.execution.parquet.schema_force_view_types true datafusion.execution.parquet.skip_arrow_metadata false datafusion.execution.parquet.skip_metadata true @@ -392,6 +394,9 @@ datafusion.execution.parquet.dictionary_enabled true (writing) Sets if dictionar datafusion.execution.parquet.dictionary_page_size_limit 1048576 (writing) Sets best effort maximum dictionary page size, in bytes datafusion.execution.parquet.enable_page_index true (reading) If true, reads the Parquet data page level metadata (the Page Index), if present, to reduce the I/O and number of rows decoded. datafusion.execution.parquet.encoding NULL (writing) Sets default encoding for any column. Valid values are: plain, plain_dictionary, rle, bit_packed, delta_binary_packed, delta_length_byte_array, delta_byte_array, rle_dictionary, and byte_stream_split. These values are not case sensitive. If NULL, uses default parquet writer setting +datafusion.execution.parquet.filter_collecting_byte_ratio_threshold 0.2 (reading) Byte-ratio threshold for applying filters one at a time (iterative pruning; aka row-level) vs. all at once (post-scan). The ratio is computed as: (extra filter bytes not in projection) / (projected bytes). Filters whose extra columns consume a smaller fraction than this threshold are placed as row filters. Ratio of filter column bytes to projection bytes that controls initial filter placement. Computed as `filter_compressed_bytes / projection_compressed_bytes`. Filters below this ratio start as row-level filters (enabling late materialization); those above start as post-scan filters. Default: 0.20 — filters whose columns are less than 20% of the projection bytes start at row-level. **Interaction with `pushdown_filters`:** Only takes effect when `pushdown_filters = true`. +datafusion.execution.parquet.filter_confidence_z 2 (reading) Z-score for confidence intervals on filter effectiveness. Controls how much statistical evidence is required before promoting or demoting a filter. Lower values = faster decisions with less confidence. Higher values = more conservative, requiring more data. Default: 2.0 (~95% confidence). **Interaction with `pushdown_filters`:** Only takes effect when `pushdown_filters = true`. +datafusion.execution.parquet.filter_pushdown_min_bytes_per_sec 104857600 (reading) Minimum bytes/sec throughput for adaptive filter pushdown. Filters that achieve at least this throughput (bytes_saved / eval_time) are promoted to row filters. f64::INFINITY = no filters promoted (feature disabled). 0.0 = all filters pushed as row filters (no adaptive logic). Default: 104,857,600 bytes/sec (100 MiB/sec), empirically chosen based on TPC-H, TPC-DS, and ClickBench benchmarks on an m4 MacBook Pro. The optimal value for this setting likely depends on the relative cost of CPU vs. IO in your environment, and to some extent the shape of your query. **Interaction with `pushdown_filters`:** This option only takes effect when `pushdown_filters = true`. When pushdown is disabled, all filters run post-scan and this threshold is ignored. datafusion.execution.parquet.force_filter_selections false (reading) Force the use of RowSelections for filter results, when pushdown_filters is enabled. If false, the reader will automatically choose between a RowSelection and a Bitmap based on the number and pattern of selected rows. datafusion.execution.parquet.max_predicate_cache_size NULL (reading) The maximum predicate cache size, in bytes. When `pushdown_filters` is enabled, sets the maximum memory used to cache the results of predicate evaluation between filter evaluation and output generation. Decreasing this value will reduce memory usage, but may increase IO and CPU usage. None means use the default parquet reader setting. 0 means no caching. datafusion.execution.parquet.max_row_group_size 1048576 (writing) Target maximum number of rows in each row group (defaults to 1M rows). Writing larger row groups requires more memory to write, but can get better compression and be faster to read. @@ -400,7 +405,6 @@ datafusion.execution.parquet.maximum_parallel_row_group_writers 1 (writing) By d datafusion.execution.parquet.metadata_size_hint 524288 (reading) If specified, the parquet reader will try and fetch the last `size_hint` bytes of the parquet file optimistically. If not specified, two reads are required: One read to fetch the 8-byte parquet footer and another to fetch the metadata length encoded in the footer Default setting to 512 KiB, which should be sufficient for most parquet files, it can reduce one I/O operation per parquet file. If the metadata is larger than the hint, two reads will still be performed. datafusion.execution.parquet.pruning true (reading) If true, the parquet reader attempts to skip entire row groups based on the predicate in the query and the metadata (min/max values) stored in the parquet file datafusion.execution.parquet.pushdown_filters false (reading) If true, filter expressions are be applied during the parquet decoding operation to reduce the number of rows decoded. This optimization is sometimes called "late materialization". -datafusion.execution.parquet.reorder_filters false (reading) If true, filter expressions evaluated during the parquet decoding operation will be reordered heuristically to minimize the cost of evaluation. If false, the filters are applied in the same order as written in the query datafusion.execution.parquet.schema_force_view_types true (reading) If true, parquet reader will read columns of `Utf8/Utf8Large` with `Utf8View`, and `Binary/BinaryLarge` with `BinaryView`. datafusion.execution.parquet.skip_arrow_metadata false (writing) Skip encoding the embedded arrow metadata in the KV_meta This is analogous to the `ArrowWriterOptions::with_skip_arrow_metadata`. Refer to datafusion.execution.parquet.skip_metadata true (reading) If true, the parquet reader skip the optional embedded metadata that may be in the file Schema. This setting can help avoid schema conflicts when querying multiple parquet files with schemas containing compatible types but different metadata diff --git a/datafusion/sqllogictest/test_files/joins.slt b/datafusion/sqllogictest/test_files/joins.slt index c6c8f9b6bcf4d..f25c22ee88df3 100644 --- a/datafusion/sqllogictest/test_files/joins.slt +++ b/datafusion/sqllogictest/test_files/joins.slt @@ -2865,7 +2865,7 @@ physical_plan 02)--HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(t2_id@0, t1_id@0)] 03)----DataSourceExec: partitions=1, partition_sizes=[1] 04)----SortExec: expr=[t1_id@0 ASC NULLS LAST], preserve_partitioning=[true] -05)------FilterExec: DynamicFilter [ empty ] +05)------FilterExec: Optional(DynamicFilter [ empty ]) 06)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 07)----------DataSourceExec: partitions=1, partition_sizes=[1] @@ -2901,7 +2901,7 @@ physical_plan 02)--HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(t2_id@0, t1_id@0)] 03)----DataSourceExec: partitions=1, partition_sizes=[1] 04)----SortExec: expr=[t1_id@0 ASC NULLS LAST], preserve_partitioning=[true] -05)------FilterExec: DynamicFilter [ empty ] +05)------FilterExec: Optional(DynamicFilter [ empty ]) 06)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 07)----------DataSourceExec: partitions=1, partition_sizes=[1] @@ -2958,7 +2958,7 @@ physical_plan 02)--HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(t2_id@0, t1_id@0)] 03)----DataSourceExec: partitions=1, partition_sizes=[1] 04)----SortExec: expr=[t1_id@0 ASC NULLS LAST], preserve_partitioning=[true] -05)------FilterExec: DynamicFilter [ empty ] +05)------FilterExec: Optional(DynamicFilter [ empty ]) 06)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 07)----------DataSourceExec: partitions=1, partition_sizes=[1] @@ -2994,7 +2994,7 @@ physical_plan 02)--HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(t2_id@0, t1_id@0)] 03)----DataSourceExec: partitions=1, partition_sizes=[1] 04)----SortExec: expr=[t1_id@0 ASC NULLS LAST], preserve_partitioning=[true] -05)------FilterExec: DynamicFilter [ empty ] +05)------FilterExec: Optional(DynamicFilter [ empty ]) 06)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 07)----------DataSourceExec: partitions=1, partition_sizes=[1] @@ -3052,7 +3052,7 @@ physical_plan 02)--HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(t2_id@0, t1_id@0)], filter=t2_name@1 != t1_name@0 03)----DataSourceExec: partitions=1, partition_sizes=[1] 04)----SortExec: expr=[t1_id@0 ASC NULLS LAST], preserve_partitioning=[true] -05)------FilterExec: DynamicFilter [ empty ] +05)------FilterExec: Optional(DynamicFilter [ empty ]) 06)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 07)----------DataSourceExec: partitions=1, partition_sizes=[1] @@ -3069,7 +3069,7 @@ physical_plan 02)--HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(t2_id@0, t1_id@0)], filter=t2_name@0 != t1_name@1 03)----DataSourceExec: partitions=1, partition_sizes=[1] 04)----SortExec: expr=[t1_id@0 ASC NULLS LAST], preserve_partitioning=[true] -05)------FilterExec: DynamicFilter [ empty ] +05)------FilterExec: Optional(DynamicFilter [ empty ]) 06)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 07)----------DataSourceExec: partitions=1, partition_sizes=[1] @@ -3124,7 +3124,7 @@ physical_plan 02)--HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(t2_id@0, t1_id@0)], filter=t2_name@1 != t1_name@0 03)----DataSourceExec: partitions=1, partition_sizes=[1] 04)----SortExec: expr=[t1_id@0 ASC NULLS LAST], preserve_partitioning=[true] -05)------FilterExec: DynamicFilter [ empty ] +05)------FilterExec: Optional(DynamicFilter [ empty ]) 06)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 07)----------DataSourceExec: partitions=1, partition_sizes=[1] @@ -3141,7 +3141,7 @@ physical_plan 02)--HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(t2_id@0, t1_id@0)], filter=t2_name@0 != t1_name@1 03)----DataSourceExec: partitions=1, partition_sizes=[1] 04)----SortExec: expr=[t1_id@0 ASC NULLS LAST], preserve_partitioning=[true] -05)------FilterExec: DynamicFilter [ empty ] +05)------FilterExec: Optional(DynamicFilter [ empty ]) 06)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 07)----------DataSourceExec: partitions=1, partition_sizes=[1] @@ -4149,7 +4149,7 @@ physical_plan 01)HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(b@1, y@1)], filter=a@0 < x@1 02)--DataSourceExec: partitions=1, partition_sizes=[0] 03)--SortExec: expr=[x@0 ASC NULLS LAST], preserve_partitioning=[false] -04)----FilterExec: DynamicFilter [ empty ] +04)----FilterExec: Optional(DynamicFilter [ empty ]) 05)------DataSourceExec: partitions=1, partition_sizes=[0] # Test full join with limit @@ -4452,7 +4452,7 @@ physical_plan 04)------FilterExec: b@1 > 3, projection=[a@0] 05)--------DataSourceExec: partitions=2, partition_sizes=[1, 1] 06)----SortExec: expr=[c@2 DESC], preserve_partitioning=[true] -07)------FilterExec: DynamicFilter [ empty ] +07)------FilterExec: Optional(DynamicFilter [ empty ]) 08)--------DataSourceExec: partitions=2, partition_sizes=[1, 1] query TT @@ -4473,7 +4473,7 @@ physical_plan 04)------FilterExec: b@1 > 3, projection=[a@0] 05)--------DataSourceExec: partitions=2, partition_sizes=[1, 1] 06)----SortExec: expr=[c@2 DESC NULLS LAST], preserve_partitioning=[true] -07)------FilterExec: DynamicFilter [ empty ] +07)------FilterExec: Optional(DynamicFilter [ empty ]) 08)--------DataSourceExec: partitions=2, partition_sizes=[1, 1] query III @@ -4687,14 +4687,22 @@ query TT explain SELECT j1_string, j2_string FROM j1 LEFT JOIN LATERAL (SELECT * FROM j2 WHERE j1_id < j2_id) AS j2 ON(true); ---- logical_plan -01)Left Join: -02)--TableScan: j1 projection=[j1_string] -03)--SubqueryAlias: j2 -04)----Projection: j2.j2_string -05)------Subquery: -06)--------Filter: outer_ref(j1.j1_id) < j2.j2_id -07)----------TableScan: j2 projection=[j2_string, j2_id] -physical_plan_error This feature is not implemented: Physical plan does not support logical expression OuterReferenceColumn(Field { name: "j1_id", data_type: Int32, nullable: true }, Column { relation: Some(Bare { table: "j1" }), name: "j1_id" }) +01)Projection: j1.j1_string, j2.j2_string +02)--Left Join: Filter: j1.j1_id < j2.j2_id +03)----TableScan: j1 projection=[j1_string, j1_id] +04)----SubqueryAlias: j2 +05)------TableScan: j2 projection=[j2_string, j2_id] +physical_plan +01)NestedLoopJoinExec: join_type=Left, filter=j1_id@0 < j2_id@1, projection=[j1_string@0, j2_string@2] +02)--DataSourceExec: partitions=1, partition_sizes=[0] +03)--DataSourceExec: partitions=1, partition_sizes=[0] + +# INNER lateral with multi-scope correlation: the subquery references +# j1 (grandparent scope) and j2 (parent scope). The optimizer +# restructures this into a valid plan via join reordering. +query TITITI +SELECT * FROM j1, (j2 CROSS JOIN LATERAL (SELECT * FROM j3 WHERE j1_id + j2_id = j3_id) AS j3_lat); +---- query TT explain SELECT * FROM j1, (j2 LEFT JOIN LATERAL (SELECT * FROM j3 WHERE j1_id + j2_id = j3_id) AS j3 ON(true)); diff --git a/datafusion/sqllogictest/test_files/lateral_join.slt b/datafusion/sqllogictest/test_files/lateral_join.slt index ec34d62a439ae..cae3e67153246 100644 --- a/datafusion/sqllogictest/test_files/lateral_join.slt +++ b/datafusion/sqllogictest/test_files/lateral_join.slt @@ -368,7 +368,230 @@ ORDER BY t1.id, sub.data; 2 b z 1 ########################################################### -# Section 3: Nested LATERAL joins +# Section 3: LEFT JOIN LATERAL +# +# LEFT lateral joins preserve all outer rows, producing +# NULLs on the right side when the lateral subquery returns +# no matching rows. +########################################################### + +# Basic LEFT JOIN LATERAL with ON true: unmatched outer rows get NULLs +query ITT +SELECT t1.id, t1.val, sub.data +FROM t1 LEFT JOIN LATERAL ( + SELECT t2.data FROM t2 WHERE t2.t1_id = t1.id +) sub ON true +ORDER BY t1.id, sub.data; +---- +1 a x +1 a y +2 b z +3 c NULL + +# LEFT JOIN LATERAL with non-trivial ON condition (no aggregate) +query ITT +SELECT t1.id, t1.val, sub.data +FROM t1 LEFT JOIN LATERAL ( + SELECT t2.data FROM t2 WHERE t2.t1_id = t1.id +) sub ON sub.data = 'x' +ORDER BY t1.id; +---- +1 a x +2 b NULL +3 c NULL + +# LEFT JOIN LATERAL with COUNT + ON true: count bug gives 0 for empty groups +query ITI +SELECT t1.id, t1.val, sub.cnt +FROM t1 LEFT JOIN LATERAL ( + SELECT count(*) AS cnt FROM t2 WHERE t2.t1_id = t1.id +) sub ON true +ORDER BY t1.id; +---- +1 a 2 +2 b 1 +3 c 0 + +# LEFT JOIN LATERAL with SUM + ON true: SUM returns NULL for empty groups +query ITI +SELECT t1.id, t1.val, sub.total +FROM t1 LEFT JOIN LATERAL ( + SELECT sum(t2.id) AS total FROM t2 WHERE t2.t1_id = t1.id +) sub ON true +ORDER BY t1.id; +---- +1 a 30 +2 b 30 +3 c NULL + +# LEFT JOIN LATERAL with COUNT + non-trivial ON: ON-nullification +# For t1.id=3, the lateral produces cnt=0 (count bug default), but +# ON 0>0 is false, so the LEFT join preserves the row with NULLs. +query ITI +SELECT t1.id, t1.val, sub.cnt +FROM t1 LEFT JOIN LATERAL ( + SELECT count(*) AS cnt FROM t2 WHERE t2.t1_id = t1.id +) sub ON sub.cnt > 0 +ORDER BY t1.id; +---- +1 a 2 +2 b 1 +3 c NULL + +# LEFT JOIN LATERAL with COUNT + non-trivial ON referencing both sides +query ITI +SELECT t1.id, t1.val, sub.cnt +FROM t1 LEFT JOIN LATERAL ( + SELECT count(*) AS cnt FROM t2 WHERE t2.t1_id = t1.id +) sub ON sub.cnt * 2 > t1.id +ORDER BY t1.id; +---- +1 a 2 +2 b NULL +3 c NULL + +# Multiple aggregates (COUNT + SUM) + non-trivial ON +# COUNT should be NULL (not 0) when ON fails; SUM should also be NULL +query ITII +SELECT t1.id, t1.val, sub.cnt, sub.total +FROM t1 LEFT JOIN LATERAL ( + SELECT count(*) AS cnt, sum(t2.id) AS total + FROM t2 WHERE t2.t1_id = t1.id +) sub ON sub.cnt > 0 +ORDER BY t1.id; +---- +1 a 2 30 +2 b 1 30 +3 c NULL NULL + +# LEFT JOIN LATERAL with GROUP BY (not scalar agg, no count bug) +query ITTI +SELECT t1.id, t1.val, sub.data, sub.cnt +FROM t1 LEFT JOIN LATERAL ( + SELECT t2.data, count(*) AS cnt FROM t2 + WHERE t2.t1_id = t1.id GROUP BY t2.data +) sub ON true +ORDER BY t1.id, sub.data; +---- +1 a x 1 +1 a y 1 +2 b z 1 +3 c NULL NULL + +# LEFT JOIN LATERAL with inequality correlation +query II +SELECT i1.i, sub.j +FROM integers i1 LEFT JOIN LATERAL ( + SELECT i2.i AS j FROM integers i2 WHERE i2.i > i1.i +) sub ON true +ORDER BY i1.i, sub.j; +---- +1 2 +1 3 +2 3 +3 NULL +NULL NULL + +# LEFT JOIN LATERAL where no inner rows match any outer row +query IT +SELECT t1.id, sub.data +FROM t1 LEFT JOIN LATERAL ( + SELECT t2.data FROM t2 WHERE t2.t1_id = t1.id AND t2.t1_id > 999 +) sub ON true +ORDER BY t1.id; +---- +1 NULL +2 NULL +3 NULL + +# Validate: LEFT LATERAL with COUNT + ON true gives same result +# as comma LATERAL (both use count bug compensation) +query ITI +SELECT t1.id, t1.val, sub.cnt +FROM t1, LATERAL (SELECT count(*) AS cnt FROM t2 WHERE t2.t1_id = t1.id) AS sub +ORDER BY t1.id; +---- +1 a 2 +2 b 1 +3 c 0 + +# Post-join WHERE filter must eliminate NULL-producing LEFT lateral rows. +# Regression test for the class of bug in duckdb/duckdb#21609. +query ITT +SELECT t1.id, t1.val, sub.data +FROM t1 LEFT JOIN LATERAL ( + SELECT t2.data FROM t2 WHERE t2.t1_id = t1.id +) sub ON true +WHERE sub.data IS NOT NULL +ORDER BY t1.id, sub.data; +---- +1 a x +1 a y +2 b z + +# Post-join WHERE after count-bug compensation: the WHERE must see the +# compensated value (0 for COUNT, not NULL). +query ITI +SELECT t1.id, t1.val, sub.cnt +FROM t1 LEFT JOIN LATERAL ( + SELECT count(*) AS cnt FROM t2 WHERE t2.t1_id = t1.id +) sub ON true +WHERE sub.cnt > 0 +ORDER BY t1.id; +---- +1 a 2 +2 b 1 + +# Full pipeline: LEFT + COUNT + non-trivial ON + post-join WHERE. +# Decorrelation → count-bug compensation → ON-nullification → WHERE. +query ITI +SELECT t1.id, t1.val, sub.cnt +FROM t1 LEFT JOIN LATERAL ( + SELECT count(*) AS cnt FROM t2 WHERE t2.t1_id = t1.id +) sub ON sub.cnt > 0 +WHERE t1.id <= 2 +ORDER BY t1.id; +---- +1 a 2 +2 b 1 + +# Empty left table + LEFT LATERAL produces 0 rows +query IT +SELECT t1.id, sub.data +FROM (SELECT 1 AS id WHERE false) t1 +LEFT JOIN LATERAL (SELECT t2.data FROM t2 WHERE t2.t1_id = t1.id) sub ON true; +---- + +# LEFT LATERAL + NULL correlation keys + COUNT: count-bug compensation +# must produce 0 when the correlation key is NULL (NULL=NULL is unknown). +query II +SELECT i1.i, sub.cnt +FROM integers i1 LEFT JOIN LATERAL ( + SELECT count(*) AS cnt FROM integers i2 WHERE i2.i = i1.i +) sub ON true +ORDER BY i1.i; +---- +1 1 +2 1 +3 1 +NULL 0 + +# LEFT JOIN LATERAL with GROUP BY + non-trivial ON: the ON condition +# is merged into the join condition (no ON-nullification needed). +query ITTI +SELECT t1.id, t1.val, sub.data, sub.cnt +FROM t1 LEFT JOIN LATERAL ( + SELECT t2.data, count(*) AS cnt FROM t2 + WHERE t2.t1_id = t1.id GROUP BY t2.data +) sub ON sub.data >= 'y' +ORDER BY t1.id, sub.data; +---- +1 a y 1 +2 b z 1 +3 c NULL NULL + +########################################################### +# Section 4: Nested LATERAL joins ########################################################### # Both levels correlate via WHERE with their sibling table @@ -403,8 +626,25 @@ ORDER BY t1.id, sub.t2_id; 1 20 1 2 30 1 +# Nested LEFT JOIN LATERAL: outer LEFT preserves unmatched outer rows +query III +SELECT t1.id, sub.t2_id, sub.t3_id +FROM t1 LEFT JOIN LATERAL ( + SELECT t2_sub.id AS t2_id, t3_sub.id AS t3_id + FROM t2 AS t2_sub LEFT JOIN LATERAL ( + SELECT t3.id FROM t3 WHERE t3.t2_id = t2_sub.id + ) AS t3_sub ON true + WHERE t2_sub.t1_id = t1.id +) AS sub ON true +ORDER BY t1.id, sub.t2_id, sub.t3_id; +---- +1 10 100 +1 20 200 +2 30 300 +3 NULL NULL + ########################################################### -# Section 4: Semantically invalid queries (permanent errors) +# Section 5: Semantically invalid queries (permanent errors) ########################################################### # FULL/RIGHT JOIN LATERAL are invalid per the SQL standard: the right @@ -426,13 +666,12 @@ statement error SELECT * FROM integers, LATERAL (SELECT SUM(i) AS s) t; ########################################################### -# Section 5: Currently unsupported patterns +# Section 6: Currently unsupported patterns # -# These are valid SQL that DataFusion cannot decorrelate yet. +# These are valid queries that DataFusion cannot decorrelate yet. # The primary limitation is that outer references must appear -# in WHERE clauses. Outer references in SELECT expressions, -# inside aggregate arguments, or in LEFT JOIN LATERAL are not -# yet supported. +# in WHERE clauses: outer references in SELECT expressions or +# inside aggregate arguments are not yet supported. ########################################################### # --- Outer reference in SELECT expression (not WHERE) --- @@ -460,23 +699,26 @@ FROM integers i1, LATERAL (SELECT sub1.j + 1 AS k) sub2 ORDER BY i1.i; -# --- LEFT JOIN LATERAL --- +# --- LEFT JOIN LATERAL with outer ref in SELECT --- +# These fail because outer references in the SELECT list (not WHERE) +# cannot be decorrelated yet, not because LEFT lateral is unsupported. +# LEFT lateral with outer refs only in WHERE works — see Section 3. -# LEFT JOIN preserving unmatched outer rows with filter +# LEFT JOIN with outer ref in SELECT expression statement error OuterReferenceColumn SELECT i1.i, sub.b FROM integers i1 LEFT JOIN LATERAL (SELECT i1.i AS b WHERE i1.i IN (1, 3)) sub ON (i1.i = sub.b) ORDER BY i1.i; -# LEFT JOIN where all rows match +# LEFT JOIN with outer ref in SELECT, all rows match statement error OuterReferenceColumn SELECT i1.i, sub.b FROM integers i1 LEFT JOIN LATERAL (SELECT i1.i AS b) sub ON (i1.i = sub.b) ORDER BY i1.i; -# LEFT JOIN where no rows match (all NULLs on right) +# LEFT JOIN with outer ref in SELECT, no rows match statement error OuterReferenceColumn SELECT i1.i, sub.b FROM integers i1 @@ -506,7 +748,7 @@ SELECT name, total FROM students, (SELECT SUM(grade) AS total FROM exams WHERE exams.sid = students.id) grades; ########################################################### -# Section 6: EXPLAIN plan verification +# Section 7: EXPLAIN plan verification ########################################################### # Verify the COUNT bug fix: Left Join with CASE WHEN compensation @@ -536,6 +778,24 @@ physical_plan 09)--------------AggregateExec: mode=Partial, gby=[t1_id@0 as t1_id], aggr=[count(Int64(1))] 10)----------------DataSourceExec: partitions=1, partition_sizes=[1] +# Verify LEFT lateral without aggregate decorrelates to left join +# (no post-join filter — ON merged into join condition) +query TT +EXPLAIN SELECT t1.id, sub.data +FROM t1 LEFT JOIN LATERAL (SELECT t2.data FROM t2 WHERE t2.t1_id = t1.id) AS sub ON true; +---- +logical_plan +01)Projection: t1.id, sub.data +02)--Left Join: t1.id = sub.t1_id +03)----TableScan: t1 projection=[id] +04)----SubqueryAlias: sub +05)------Projection: t2.data, t2.t1_id +06)--------TableScan: t2 projection=[t1_id, data] +physical_plan +01)HashJoinExec: mode=CollectLeft, join_type=Left, on=[(id@0, t1_id@1)], projection=[id@0, data@1] +02)--DataSourceExec: partitions=1, partition_sizes=[1] +03)--DataSourceExec: partitions=1, partition_sizes=[1] + # Verify non-aggregate lateral decorrelates to inner join query TT EXPLAIN SELECT t1.id, sub.data diff --git a/datafusion/sqllogictest/test_files/metadata.slt b/datafusion/sqllogictest/test_files/metadata.slt index 42589481f909e..f584e3c342271 100644 --- a/datafusion/sqllogictest/test_files/metadata.slt +++ b/datafusion/sqllogictest/test_files/metadata.slt @@ -83,7 +83,7 @@ select count(distinct name) from table_with_metadata; 2 # Regression test: prevent field metadata loss per https://github.com/apache/datafusion/issues/12687 -query I +query R select approx_median(distinct id) from table_with_metadata; ---- 2 diff --git a/datafusion/sqllogictest/test_files/nested_loop_join_spill.slt b/datafusion/sqllogictest/test_files/nested_loop_join_spill.slt index 5b383f3edf6cc..7b5da1d4b8e03 100644 --- a/datafusion/sqllogictest/test_files/nested_loop_join_spill.slt +++ b/datafusion/sqllogictest/test_files/nested_loop_join_spill.slt @@ -39,8 +39,9 @@ INNER JOIN generate_series(1, 1) AS t2(v2) 100000 1 100000 # --- Verify spill metrics via EXPLAIN ANALYZE --- -# The NestedLoopJoinExec line should show spill_count=1, confirming -# the memory-limited fallback path was taken and right side was spilled. +# The NestedLoopJoinExec line should show spill_count=2, confirming +# the memory-limited fallback path was taken (left side spilled once, +# right side spilled once). query TT EXPLAIN ANALYZE SELECT count(*) FROM generate_series(1, 100000) AS t1(v1) @@ -50,7 +51,7 @@ INNER JOIN generate_series(1, 1) AS t2(v2) Plan with Metrics 01)ProjectionExec: expr=[count(Int64(1))@0 as count(*)], metrics=[] 02)--AggregateExec: mode=Single, gby=[], aggr=[count(Int64(1))], metrics=[] -03)----NestedLoopJoinExec: join_type=Inner, filter=v1@0 + v2@1 > 0, projection=[], metrics=[output_rows=100.0 K, spill_count=1, ] +03)----NestedLoopJoinExec: join_type=Inner, filter=v1@0 + v2@1 > 0, projection=[], metrics=[output_rows=100.0 K, spill_count=2, ] 04)------ProjectionExec: expr=[value@0 as v1], metrics=[] 05)--------LazyMemoryExec: partitions=1, batch_generators=[generate_series: start=1, end=100000, batch_size=8192], metrics=[] 06)------ProjectionExec: expr=[value@0 as v2], metrics=[] diff --git a/datafusion/sqllogictest/test_files/order.slt b/datafusion/sqllogictest/test_files/order.slt index 488a32c7acde5..ffd48d5996576 100644 --- a/datafusion/sqllogictest/test_files/order.slt +++ b/datafusion/sqllogictest/test_files/order.slt @@ -260,6 +260,64 @@ physical_plan 02)--SortExec: expr=[c2@1 ASC NULLS LAST, c3@2 ASC NULLS LAST], preserve_partitioning=[false] 03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c3], file_type=csv, has_header=true + +# eliminate redundant fd sort expr +query TT +explain SELECT c2, SUM(c3) AS total_sal FROM aggregate_test_100 GROUP BY c2 ORDER BY c2, total_sal +---- +logical_plan +01)Sort: aggregate_test_100.c2 ASC NULLS LAST +02)--Projection: aggregate_test_100.c2, sum(aggregate_test_100.c3) AS total_sal +03)----Aggregate: groupBy=[[aggregate_test_100.c2]], aggr=[[sum(CAST(aggregate_test_100.c3 AS Int64))]] +04)------TableScan: aggregate_test_100 projection=[c2, c3] +physical_plan +01)SortPreservingMergeExec: [c2@0 ASC NULLS LAST] +02)--SortExec: expr=[c2@0 ASC NULLS LAST], preserve_partitioning=[true] +03)----ProjectionExec: expr=[c2@0 as c2, sum(aggregate_test_100.c3)@1 as total_sal] +04)------AggregateExec: mode=FinalPartitioned, gby=[c2@0 as c2], aggr=[sum(aggregate_test_100.c3)] +05)--------RepartitionExec: partitioning=Hash([c2@0], 4), input_partitions=4 +06)----------AggregateExec: mode=Partial, gby=[c2@0 as c2], aggr=[sum(aggregate_test_100.c3)] +07)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +08)--------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c3], file_type=csv, has_header=true + +# keep order by when dependency comes later +query TT +explain SELECT c2, SUM(c3) AS total_sal FROM aggregate_test_100 GROUP BY c2 ORDER BY total_sal, c2 +---- +logical_plan +01)Sort: total_sal ASC NULLS LAST, aggregate_test_100.c2 ASC NULLS LAST +02)--Projection: aggregate_test_100.c2, sum(aggregate_test_100.c3) AS total_sal +03)----Aggregate: groupBy=[[aggregate_test_100.c2]], aggr=[[sum(CAST(aggregate_test_100.c3 AS Int64))]] +04)------TableScan: aggregate_test_100 projection=[c2, c3] +physical_plan +01)SortPreservingMergeExec: [total_sal@1 ASC NULLS LAST, c2@0 ASC NULLS LAST] +02)--SortExec: expr=[total_sal@1 ASC NULLS LAST, c2@0 ASC NULLS LAST], preserve_partitioning=[true] +03)----ProjectionExec: expr=[c2@0 as c2, sum(aggregate_test_100.c3)@1 as total_sal] +04)------AggregateExec: mode=FinalPartitioned, gby=[c2@0 as c2], aggr=[sum(aggregate_test_100.c3)] +05)--------RepartitionExec: partitioning=Hash([c2@0], 4), input_partitions=4 +06)----------AggregateExec: mode=Partial, gby=[c2@0 as c2], aggr=[sum(aggregate_test_100.c3)] +07)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +08)--------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c3], file_type=csv, has_header=true + +# eliminate redundant sort expr even with non schema expr +query TT +explain SELECT c2, SUM(c3) AS total_sal FROM aggregate_test_100 GROUP BY c2 ORDER BY c2, total_sal, abs(c2) +---- +logical_plan +01)Sort: aggregate_test_100.c2 ASC NULLS LAST, abs(aggregate_test_100.c2) ASC NULLS LAST +02)--Projection: aggregate_test_100.c2, sum(aggregate_test_100.c3) AS total_sal +03)----Aggregate: groupBy=[[aggregate_test_100.c2]], aggr=[[sum(CAST(aggregate_test_100.c3 AS Int64))]] +04)------TableScan: aggregate_test_100 projection=[c2, c3] +physical_plan +01)SortPreservingMergeExec: [c2@0 ASC NULLS LAST, abs(c2@0) ASC NULLS LAST] +02)--SortExec: expr=[c2@0 ASC NULLS LAST, abs(c2@0) ASC NULLS LAST], preserve_partitioning=[true] +03)----ProjectionExec: expr=[c2@0 as c2, sum(aggregate_test_100.c3)@1 as total_sal] +04)------AggregateExec: mode=FinalPartitioned, gby=[c2@0 as c2], aggr=[sum(aggregate_test_100.c3)] +05)--------RepartitionExec: partitioning=Hash([c2@0], 4), input_partitions=4 +06)----------AggregateExec: mode=Partial, gby=[c2@0 as c2], aggr=[sum(aggregate_test_100.c3)] +07)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +08)--------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c3], file_type=csv, has_header=true + query II SELECT c2, c3 FROM aggregate_test_100 ORDER BY c2, c3, c2 ---- @@ -1521,7 +1579,7 @@ query TT EXPLAIN SELECT c1, c2 FROM table_with_ordered_pk ORDER BY c1, c2; ---- logical_plan -01)Sort: table_with_ordered_pk.c1 ASC NULLS LAST, table_with_ordered_pk.c2 ASC NULLS LAST +01)Sort: table_with_ordered_pk.c1 ASC NULLS LAST 02)--TableScan: table_with_ordered_pk projection=[c1, c2] physical_plan DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/aggregate_agg_multi_order.csv]]}, projection=[c1, c2], output_ordering=[c1@0 ASC NULLS LAST], constraints=[PrimaryKey([0])], file_type=csv, has_header=true diff --git a/datafusion/sqllogictest/test_files/parquet_filter_pushdown.slt b/datafusion/sqllogictest/test_files/parquet_filter_pushdown.slt index 85f9549357138..ec08e3917211a 100644 --- a/datafusion/sqllogictest/test_files/parquet_filter_pushdown.slt +++ b/datafusion/sqllogictest/test_files/parquet_filter_pushdown.slt @@ -902,9 +902,6 @@ DROP TABLE t_struct_filter; statement ok set datafusion.execution.parquet.pushdown_filters = true; -statement ok -set datafusion.execution.parquet.reorder_filters = true; - statement ok COPY ( SELECT @@ -935,8 +932,5 @@ A 78 statement ok set datafusion.execution.parquet.pushdown_filters = false; -statement ok -set datafusion.execution.parquet.reorder_filters = false; - statement ok DROP TABLE dict_filter_bug; diff --git a/datafusion/sqllogictest/test_files/preserve_file_partitioning.slt b/datafusion/sqllogictest/test_files/preserve_file_partitioning.slt index 175d7d90cd8ed..a841143cb10ae 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=Optional(DynamicFilter [ empty ]) # 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=Optional(DynamicFilter [ empty ]) 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=Optional(DynamicFilter [ empty ]) 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 a40c1b8c7e246..7b43ec651a53b 100644 --- a/datafusion/sqllogictest/test_files/projection_pushdown.slt +++ b/datafusion/sqllogictest/test_files/projection_pushdown.slt @@ -637,7 +637,7 @@ SELECT id, s['value'] + 1 FROM simple_struct WHERE id > 1 ORDER BY id LIMIT 2; # 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; @@ -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=Optional(DynamicFilter [ empty ]) # 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=Optional(DynamicFilter [ empty ]) # 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 Optional(DynamicFilter [ empty ]) # 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=Optional(DynamicFilter [ empty ]) # 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=Optional(DynamicFilter [ empty ]) # 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 Optional(DynamicFilter [ empty ]) # Verify correctness - left join with level > 5 condition # Only join_right rows with level > 5 are matched: id=1 (level=10), id=4 (level=8) @@ -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=Optional(DynamicFilter [ empty ]) # 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=Optional(DynamicFilter [ empty ]) # Verify correctness - all rows match since value >> level for all ids # simple_struct: (1,100), (2,200), (3,150), (4,300), (5,250) diff --git a/datafusion/sqllogictest/test_files/push_down_filter_parquet.slt b/datafusion/sqllogictest/test_files/push_down_filter_parquet.slt index 8469c32a17033..3800172f6bad8 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], file_type=parquet 03)--RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -04)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/large_table.parquet]]}, projection=[k, v], file_type=parquet, predicate=v@1 >= 50 AND DynamicFilter [ empty ], pruning_predicate=v_null_count@1 != row_count@2 AND v_max@0 >= 50, required_guarantees=[] +04)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/push_down_filter_parquet/large_table.parquet]]}, projection=[k, v], file_type=parquet, predicate=v@1 >= 50 AND Optional(DynamicFilter [ empty ]), 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 ], 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=0, pushdown_rows_pruned=0, predicate_cache_inner_records=0, predicate_cache_records=0, scan_efficiency_ratio=64.87% (258.7 K/398.8 K)] statement ok reset datafusion.explain.analyze_categories; @@ -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 ], 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=22.37% (240/1.07 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 ], 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=0, pushdown_rows_pruned=0, predicate_cache_inner_records=0, predicate_cache_records=0, scan_efficiency_ratio=22.37% (240/1.07 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) ], 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=22.37% (240/1.07 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) ], 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=0, pushdown_rows_pruned=0, predicate_cache_inner_records=0, predicate_cache_records=0, scan_efficiency_ratio=22.37% (240/1.07 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=20.48% (214/1.04 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.78% (246/1.08 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=Optional(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=0, pushdown_rows_pruned=0, predicate_cache_inner_records=0, predicate_cache_records=0, scan_efficiency_ratio=22.78% (246/1.08 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=18.23% (144/790)] -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=23.2% (252/1.09 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=22.12% (184/832)] +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=Optional(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=0, pushdown_rows_pruned=0, predicate_cache_inner_records=0, predicate_cache_records=0, scan_efficiency_ratio=23.2% (252/1.09 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=Optional(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=0, pushdown_rows_pruned=0, predicate_cache_inner_records=0, predicate_cache_records=0, scan_efficiency_ratio=22.12% (184/832)] 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 Optional(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)] 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.7% (70/1.04 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=15.37% (166/1.08 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=Optional(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=0, pushdown_rows_pruned=0, predicate_cache_inner_records=0, predicate_cache_records=0, scan_efficiency_ratio=15.37% (166/1.08 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 ], 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.72% (153/1.11 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 ], 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=0, pushdown_rows_pruned=0, predicate_cache_inner_records=0, predicate_cache_records=0, scan_efficiency_ratio=13.72% (153/1.11 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 ], 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=7.09% (79/1.11 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 ], 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=0, pushdown_rows_pruned=0, predicate_cache_inner_records=0, predicate_cache_records=0, scan_efficiency_ratio=7.09% (79/1.11 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.72% (153/1.11 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 ], 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=13.72% (153/1.11 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 ], 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.72% (153/1.11 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 ], 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=13.72% (153/1.11 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.81% (163/823)] +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=Optional(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=0, pushdown_rows_pruned=0, predicate_cache_inner_records=0, predicate_cache_records=0, scan_efficiency_ratio=19.81% (163/823)] 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=21.1% (237/1.12 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=Optional(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=0, pushdown_rows_pruned=0, predicate_cache_inner_records=0, predicate_cache_records=0, scan_efficiency_ratio=21.1% (237/1.12 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=20.48% (214/1.04 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.78% (246/1.08 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=Optional(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=0, pushdown_rows_pruned=0, predicate_cache_inner_records=0, predicate_cache_records=0, scan_efficiency_ratio=22.78% (246/1.08 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=20.48% (214/1.04 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=15.37% (166/1.08 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=Optional(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=0, pushdown_rows_pruned=0, predicate_cache_inner_records=0, predicate_cache_records=0, scan_efficiency_ratio=15.37% (166/1.08 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=20.48% (214/1.04 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.78% (246/1.08 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=Optional(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=0, pushdown_rows_pruned=0, predicate_cache_inner_records=0, predicate_cache_records=0, scan_efficiency_ratio=22.78% (246/1.08 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=19.02% (222/1.17 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=21.43% (239/1.11 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=Optional(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=0, pushdown_rows_pruned=0, predicate_cache_inner_records=0, predicate_cache_records=0, scan_efficiency_ratio=21.43% (239/1.11 K)] 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..a83cb1113715d 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=Optional(DynamicFilter [ empty ]) # 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=Optional(DynamicFilter [ empty ]) # 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/spark/string/is_valid_utf8.slt b/datafusion/sqllogictest/test_files/spark/string/is_valid_utf8.slt new file mode 100644 index 0000000000000..9b04595334ae1 --- /dev/null +++ b/datafusion/sqllogictest/test_files/spark/string/is_valid_utf8.slt @@ -0,0 +1,203 @@ +# 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. + +statement ok +CREATE TABLE test_is_valid_utf8(value STRING) AS VALUES + (arrow_cast('Hello, world!', 'Utf8')), + (arrow_cast('Spark', 'Utf8')), + (arrow_cast('DataFusion', 'Utf8')), + (arrow_cast('ASCII only 123 !@#', 'Utf8')), + (arrow_cast(NULL, 'Utf8')); + +query B +SELECT is_valid_utf8(value) FROM test_is_valid_utf8; +---- +true +true +true +true +NULL + +query B +SELECT is_valid_utf8(NULL::string); +---- +NULL + +query B +SELECT is_valid_utf8('Hello, world!'::string); +---- +true + +query B +SELECT is_valid_utf8('😀🎉✨'::string); +---- +true + +query B +SELECT is_valid_utf8(''::string); +---- +true + +query B +SELECT is_valid_utf8('ASCII only 123 !@#'::string); +---- +true + +query B +SELECT is_valid_utf8(arrow_cast(x'C2A9', 'Binary')); +---- +true + +query B +SELECT is_valid_utf8(arrow_cast(x'C2AE', 'Binary')); +---- +true + +query B +SELECT is_valid_utf8(arrow_cast(x'E282AC', 'Binary')); +---- +true + +query B +SELECT is_valid_utf8(arrow_cast(x'E284A2', 'Binary')); +---- +true + +query B +SELECT is_valid_utf8(arrow_cast(x'F09F9880', 'Binary')); +---- +true + +query B +SELECT is_valid_utf8(arrow_cast(x'F09F8E89', 'Binary')); +---- +true + +query B +SELECT is_valid_utf8(arrow_cast(x'80', 'Binary')); +---- +false + +query B +SELECT is_valid_utf8(arrow_cast(x'BF', 'Binary')); +---- +false + +query B +SELECT is_valid_utf8(arrow_cast(x'808080', 'Binary')); +---- +false + +query B +SELECT is_valid_utf8(arrow_cast(x'C2', 'Binary')); +---- +false + +query B +SELECT is_valid_utf8(arrow_cast(x'E2', 'Binary')); +---- +false + +query B +SELECT is_valid_utf8(arrow_cast(x'F0', 'Binary')); +---- +false + +query B +SELECT is_valid_utf8(arrow_cast(x'E282', 'Binary')); +---- +false + +query B +SELECT is_valid_utf8(arrow_cast(x'C081', 'Binary')); +---- +false + +query B +SELECT is_valid_utf8(arrow_cast(x'E08080', 'Binary')); +---- +false + +query B +SELECT is_valid_utf8(arrow_cast(x'F0808080', 'Binary')); +---- +false + +query B +SELECT is_valid_utf8(arrow_cast(x'FE', 'Binary')); +---- +false + +query B +SELECT is_valid_utf8(arrow_cast(x'FF', 'Binary')); +---- +false + +query B +SELECT is_valid_utf8(arrow_cast(x'61C262', 'Binary')); +---- +false + +query B +SELECT is_valid_utf8(arrow_cast(x'41BF42', 'Binary')); +---- +false + +query B +SELECT is_valid_utf8(arrow_cast(x'ED9FBF', 'Binary')); +---- +true + +query B +SELECT is_valid_utf8(arrow_cast(x'EDA080', 'Binary')); +---- +false + +query B +SELECT is_valid_utf8(arrow_cast(x'EDBFBF', 'Binary')); +---- +false + +query B +SELECT is_valid_utf8(arrow_cast(x'F48FBFBF', 'Binary')); +---- +true + +query B +SELECT is_valid_utf8(arrow_cast(x'F4908080', 'Binary')); +---- +false + +query B +SELECT is_valid_utf8(arrow_cast(x'6162C2A963', 'Binary')); +---- +true + +query B +SELECT is_valid_utf8(arrow_cast(x'6162806364', 'Binary')); +---- +false + +query B +SELECT is_valid_utf8(arrow_cast(x'610062', 'Binary')); +---- +true + +query B +SELECT is_valid_utf8(arrow_cast(x'', 'Binary')); +---- +true diff --git a/datafusion/sqllogictest/test_files/window.slt b/datafusion/sqllogictest/test_files/window.slt index ae83045961488..caaf22f0adbd8 100644 --- a/datafusion/sqllogictest/test_files/window.slt +++ b/datafusion/sqllogictest/test_files/window.slt @@ -2425,11 +2425,9 @@ SELECT c9, rn1 FROM (SELECT c9, 145294611 96 # test_c9_rn_ordering_alias_opposite_direction3 -# These test check for whether datafusion is aware of the ordering of the column generated by ROW_NUMBER() window function. -# Physical plan should have a SortExec after BoundedWindowAggExec. -# The reason is that ordering of the table after BoundedWindowAggExec can be described as rn1 ASC, and also c9 DESC. -# However, the requirement is rn1 ASC, c9 ASC (lexicographical order). Hence existing ordering cannot satisfy requirement -# (Requirement is finer than existing ordering) +# These tests check whether DataFusion tracks the ordering of the column generated by ROW_NUMBER() window function. +# The outer ORDER BY can be simplified by ordering equivalence, so the plan should not need an additional SortExec +# beyond the one required to satisfy the window input order. query TT EXPLAIN SELECT c9, rn1 FROM (SELECT c9, ROW_NUMBER() OVER(ORDER BY c9 DESC) as rn1 @@ -2439,13 +2437,13 @@ EXPLAIN SELECT c9, rn1 FROM (SELECT c9, LIMIT 5 ---- logical_plan -01)Sort: rn1 ASC NULLS LAST, aggregate_test_100.c9 ASC NULLS LAST, fetch=5 +01)Sort: rn1 ASC NULLS LAST, fetch=5 02)--Projection: aggregate_test_100.c9, row_number() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS rn1 03)----WindowAggr: windowExpr=[[row_number() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] 04)------TableScan: aggregate_test_100 projection=[c9] physical_plan -01)SortExec: TopK(fetch=5), expr=[rn1@1 ASC NULLS LAST, c9@0 ASC NULLS LAST], preserve_partitioning=[false], sort_prefix=[rn1@1 ASC NULLS LAST] -02)--ProjectionExec: expr=[c9@0 as c9, row_number() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@1 as rn1] +01)ProjectionExec: expr=[c9@0 as c9, row_number() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@1 as rn1] +02)--GlobalLimitExec: skip=0, fetch=5 03)----BoundedWindowAggExec: wdw=[row_number() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Field { "row_number() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW": UInt64 }, frame: RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], mode=[Sorted] 04)------SortExec: expr=[c9@0 DESC], preserve_partitioning=[false] 05)--------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c9], file_type=csv, has_header=true @@ -2514,7 +2512,7 @@ EXPLAIN SELECT c9, rn1 FROM (SELECT c9, LIMIT 5 ---- logical_plan -01)Sort: rn1 ASC NULLS LAST, aggregate_test_100.c9 DESC NULLS FIRST, fetch=5 +01)Sort: rn1 ASC NULLS LAST, fetch=5 02)--Projection: aggregate_test_100.c9, row_number() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS rn1 03)----WindowAggr: windowExpr=[[row_number() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] 04)------TableScan: aggregate_test_100 projection=[c9] diff --git a/dev/release/README.md b/dev/release/README.md index 2f99d52b200d2..2ca495cbb135f 100644 --- a/dev/release/README.md +++ b/dev/release/README.md @@ -77,7 +77,7 @@ We then publish the code in the approved artifacts to crates.io. ### 1. Create Release Branch -First create a new release branch from `main` in the apache repository. +First create a new release branch from `main` in the `apache` repository. For example, to create the `branch-50` branch for the `50.x.y` release series: @@ -88,11 +88,28 @@ git checkout -b branch-50 # create local branch git push -u apache branch-50 # push branch to apache remote ``` -### 2. Add a protection to release candidate branch +### 2. Prepare PR to Update the Release Version -To protect a release candidate branch from accidental merges, run: +Manually update the DataFusion version in the root `Cargo.toml` to +reflect the new release version. Ensure `Cargo.lock` is updated accordingly by +running: ```shell +cargo check -p datafusion +``` + +Then commit the changes and create a PR targeting the release branch `branch-N`. + +```shell +git commit -a -m 'Update version' +``` + +### 3. Protect Release Branch + +To protect a release candidate branch from accidental merges, create PR against `main`: + +```shell +git fetch apache && git checkout -b protect_branch_50 ./dev/release/add-branch-protection.sh 50 ``` @@ -104,28 +121,22 @@ branch-50: required_approving_review_count: 1 ``` -- Create a PR. +- Commit changes +- Push to `origin/protect_branch_50` +- Create a PR against `main`. - Merge to `main`. +- Notify community in Discord/Slack that release branch is created -### 3. Prepare PR to Update Changelog and the Release Version - -First, prepare a PR to update the changelog and versions to reflect the planned -release. See [#18173](https://github.com/apache/datafusion/pull/18173) for an example. +### 4. Backporting urgent changes -- This PR should be the last one merged before creating a release - candidate, so the changelog includes all changes. -- If there are code changes between RCs, create and merge a new PR to update - the changelog before creating the next RC. +After release branch `branch-N` created, protected and got its version updated, please check if there are any backports expected from the community. +Please refer to [Backport Flow](../../docs/source/contributor-guide/release_management.md#backport-workflow) for more details. -**Step 1**: Manually update the DataFusion version in the root `Cargo.toml` to -reflect the new release version. Ensure `Cargo.lock` is updated accordingly by -running: +Backports are important and sometimes unexpected, so please proceed to next release steps once all expected backports are applied. -```shell -cargo check -p datafusion -``` +### 5. Prepare PR to Update Changelog -**Step 2**: Update the [changelog] in `dev/changelog/`. Each release has its +Update the changelog in `dev/changelog/`. Each release has its own file, such as `dev/changelog/50.0.0.md`, which should include all changes since the previous release. @@ -168,14 +179,12 @@ prettier -w dev/changelog/51.0.0.md Then commit the changes and create a PR targeting the release branch. ```shell -git commit -a -m 'Update version' +git commit -a -m 'Update changelog' ``` -Remember to merge any fixes back to the `main` branch as well. - -### 4. Prepare Release Candidate Artifacts +### 6. Prepare Release Candidate Artifacts -After the PR gets merged, you are ready to create release artifacts based off the +After the changelog updates merged to `branch-N`, you are ready to create release artifacts based off the merged commit. - You must be a committer to run these scripts because they upload to the @@ -191,20 +200,12 @@ Pick numbers in sequential order, with `1` for `rc1`, `2` for `rc2`, etc. While the official release artifacts are signed tarballs and zip files, we also tag the commit it was created from for convenience and code archaeology. Release tags -look like `38.0.0`, and release candidate tags look like `38.0.0-rc0`. See [the list of existing +look like `50.3.0`, and release candidate tags look like `50.3.0-rc1`. See [the list of existing tags]. [the list of existing tags]: https://github.com/apache/datafusion/tags -Create and push the RC tag: - -```shell -git fetch apache -git tag -rc apache/branch-X -git push apache -rc -``` - -For example, to create the `50.3.0-rc1` tag from `branch-50`, use: +Create and push the RC tag, for example, to create the `50.3.0-rc1` tag from `branch-50`, use: ```shell git fetch apache @@ -212,6 +213,8 @@ git tag 50.3.0-rc1 apache/branch-50 git push apache 50.3.0-rc1 ``` +Please make sure the format is correct, tools like Homebrew listens for tags and in case of malformed tags users would be notified for non-existent version + #### Create, Sign, and Upload Artifacts Run the `create-tarball.sh` script with the `` tag and `` number you determined in previous steps: @@ -219,19 +222,19 @@ Run the `create-tarball.sh` script with the `` tag and `` number yo For example, to create the `50.3.0-rc1` artifacts: ```shell -GH_TOKEN= ./dev/release/create-tarball.sh 50.3.0 1 +GITHUB_TOKEN= ./dev/release/create-tarball.sh 50.3.0 1 ``` The `create-tarball.sh` script 1. Creates and uploads all release candidate artifacts to the [datafusion dev](https://dist.apache.org/repos/dist/dev/datafusion) location on the - apache distribution SVN server + Apache distribution SVN server 2. Provides you an email template to send to `dev@datafusion.apache.org` for release voting. -### 5. Vote on Release Candidate Artifacts +### 7. Vote on Release Candidate Artifacts Send the email output from the script to `dev@datafusion.apache.org`. @@ -248,19 +251,13 @@ review the release candidate. ./dev/release/verify-release-candidate.sh 50.3.0 1 ``` -#### If the Release is not Approved - -If the release is not approved, fix whatever the problem is, make a new release -candidate and try again. - -Reminders: +#### If Changes Are Requested -1. Update the changelog and create a new PR if there are any code changes, and merge it before creating the next RC. -2. Merge any fixes back to the `main` branch as well. +If the release is not approved or urgent backports requested, please start over from [here](#4-backporting-urgent-changes) #### If the Vote Passes: Announce the Result -Call the vote on the Arrow dev list by replying to the RC voting thread. The +Call the vote on the Datafusion dev list by replying to the RC voting thread. The reply should have a new subject constructed by adding the `[RESULT]` prefix to the old subject line. @@ -271,11 +268,9 @@ The vote has passed with +1 votes. Thank you to all who helped with the release verification. ``` -### 6. Finalize the Release +### 8. Finalize the Release -NOTE: steps in this section can only be done by PMC members. - -#### After the release is approved +NOTE: steps in this section can only be done by PMC members after release is approved. Move artifacts to the release location in SVN, e.g. https://dist.apache.org/repos/dist/release/datafusion/datafusion-50.3.0/, using @@ -287,17 +282,17 @@ the `release-tarball.sh` script: Congratulations! The release is now official! -### 7. Create Release git tags +### 9. Create Release git tags Tag the same release candidate commit with the final release tag ```shell -git co apache/50.3.0-rc1 +git checkout 50.3.0-rc1 git tag 50.3.0 git push apache 50.3.0 ``` -### 8. Publish on Crates.io +### 10. Publish on Crates.io Only approved releases of the tarball should be published to crates.io, in order to conform to Apache Software Foundation @@ -314,48 +309,61 @@ to all DataFusion crates. Download and unpack the official release tarball Verify that the Cargo.toml in the tarball contains the correct version -(e.g. `version = "38.0.0"`) and then publish the crates by running the following commands +(e.g. `version = "50.3.0"`) and then publish the crates by running the following commands ```shell -(cd datafusion/doc && cargo publish) -(cd datafusion/common-runtime && cargo publish) (cd datafusion/common && cargo publish) -(cd datafusion/proto-common && cargo publish) -(cd datafusion/macros && cargo publish) +(cd datafusion/common-runtime && cargo publish) +(cd datafusion/doc && cargo publish) (cd datafusion/expr-common && cargo publish) +(cd datafusion/macros && cargo publish) +(cd datafusion/proto-common && cargo publish) (cd datafusion/physical-expr-common && cargo publish) -(cd datafusion/functions-window-common && cargo publish) (cd datafusion/functions-aggregate-common && cargo publish) +(cd datafusion/functions-window-common && cargo publish) (cd datafusion/expr && cargo publish) (cd datafusion/execution && cargo publish) -(cd datafusion/physical-expr && cargo publish) (cd datafusion/functions && cargo publish) -(cd datafusion/functions-window && cargo publish) +(cd datafusion/physical-expr && cargo publish) (cd datafusion/functions-aggregate && cargo publish) +(cd datafusion/functions-window && cargo publish) (cd datafusion/physical-expr-adapter && cargo publish) -(cd datafusion/physical-plan && cargo publish) (cd datafusion/functions-nested && cargo publish) +(cd datafusion/physical-plan && cargo publish) (cd datafusion/session && cargo publish) -(cd datafusion/datasource && cargo publish) -(cd datafusion/catalog && cargo publish) -(cd datafusion/spark && cargo publish) (cd datafusion/sql && cargo publish) +(cd datafusion/datasource && cargo publish) (cd datafusion/optimizer && cargo publish) -(cd datafusion/functions-table && cargo publish) -(cd datafusion/datasource-json && cargo publish) -(cd datafusion/datasource-csv && cargo publish) -(cd datafusion/datasource-avro && cargo publish) +(cd datafusion/catalog && cargo publish) (cd datafusion/datasource-arrow && cargo publish) +(cd datafusion/datasource-avro && cargo publish) +(cd datafusion/datasource-csv && cargo publish) +(cd datafusion/datasource-json && cargo publish) (cd datafusion/pruning && cargo publish) (cd datafusion/datasource-parquet && cargo publish) -(cd datafusion/catalog-listing && cargo publish) +(cd datafusion/functions-table && cargo publish) (cd datafusion/physical-optimizer && cargo publish) +(cd datafusion/catalog-listing && cargo publish) (cd datafusion/core && cargo publish) (cd datafusion-cli && cargo publish) -(cd datafusion/substrait && cargo publish) -(cd datafusion/sqllogictest && cargo publish) (cd datafusion/proto && cargo publish) +(cd datafusion/spark && cargo publish) +(cd datafusion/substrait && cargo publish) (cd datafusion/ffi && cargo publish) +(cd datafusion/sqllogictest && cargo publish) +``` + +Crates.io publishing depends on crates dependency tree, this list might contain wrong order. +If it happens crates.io fails with wrong dependency message like below, just rerun all publishing commands. + +```shell +error: failed to prepare local package for uploading + +Caused by: + failed to select a version for the requirement `datafusion-proto = "^53.1.0"` + candidate versions found which didn't match: 53.0.0, 52.5.0, 52.4.0, ... + location searched: crates.io index + required by package `datafusion-ffi v53.1.0 (/private/tmp/apache-datafusion-53.1.0/datafusion/ffi)` ``` ### Publish datafusion-cli on Homebrew @@ -363,7 +371,22 @@ Verify that the Cargo.toml in the tarball contains the correct version Note: [`datafusion` formula](https://formulae.brew.sh/formula/datafusion) is [updated automatically](https://github.com/Homebrew/homebrew-core/pulls?q=is%3Apr+datafusion+is%3Aclosed), so no action is needed. -### 9: Add the release to Apache Reporter +### 11. Sync Changelog and Version to main + +To sync changelog version and create PR against `main`: + +```shell +git fetch apache && git checkout -b sync_change_log_version +``` + +- Cherry-pick or patch the version updates from step 2 +- Cherry-pick or patch the changelog updates from step 5 +- Commit changes, `git commit -a -m 'Sync changelog and version to main'` +- Push to `origin/sync_change_log_version` +- Create a PR against `main`. +- Merge to `main`. + +### 12. Add the release to Apache Reporter When you have published the release, please help the project by adding the release to [Apache Reporter](https://reporter.apache.org/). The reporter system should @@ -374,7 +397,7 @@ the examples from previous releases. The release information is used to generate a template for a board report (see example from Apache Arrow project [here](https://github.com/apache/arrow/pull/14357)). -### 10: Delete Old RCs and Releases +### 13. Delete Old RCs and Releases See the ASF documentation on [when to archive](https://www.apache.org/legal/release-policy.html#when-to-archive) for more information. @@ -409,11 +432,3 @@ To delete a release: ```shell svn delete -m "delete old DataFusion release" https://dist.apache.org/repos/dist/release/datafusion/datafusion-50.0.0 ``` - -### 11. Forward-port changelog to `main` - -After the release is published, forward port the version update and changelog to -`main` so that it is up to date for the next release. See [#21053] for an -example PR that forward-ports the changelog to `main`. - -[#21053]: https://github.com/apache/datafusion/pull/21053 diff --git a/dev/release/download-python-wheels.py b/dev/release/download-python-wheels.py index 3f97d0a05c3d0..b0998d4dc7733 100644 --- a/dev/release/download-python-wheels.py +++ b/dev/release/download-python-wheels.py @@ -39,11 +39,11 @@ def main(): args = parser.parse_args() tag = args.tag - ghp_token = os.environ.get("GH_TOKEN") + ghp_token = os.environ.get("GITHUB_TOKEN") if not ghp_token: print( "ERROR: Personal Github token is required to download workflow artifacts. " - "Please specify a token through GH_TOKEN environment variable.") + "Please specify a token through GITHUB_TOKEN environment variable.") sys.exit(1) print(f"Downloading latest python wheels for RC tag {tag}...") diff --git a/docs/source/library-user-guide/upgrading/54.0.0.md b/docs/source/library-user-guide/upgrading/54.0.0.md index d5fe197854c4d..c277f69d0bee2 100644 --- a/docs/source/library-user-guide/upgrading/54.0.0.md +++ b/docs/source/library-user-guide/upgrading/54.0.0.md @@ -321,3 +321,29 @@ clusters (e.g., ZWJ emoji sequences). For ASCII and most common Unicode text, behavior is unchanged. [#17861]: https://github.com/apache/datafusion/pull/17861 + +### `approx_percentile_cont`, `approx_percentile_cont_with_weight`, `approx_median` now coerce to floats + +The type signatures of `approx_percentile_cont`, `approx_percentile_cont_with_weight`, and +`approx_median` now coerce integer input values to `Float64` before computing the approximation. +As a result, these functions always return a float, even when the input column is an integer type. + +**Who is affected:** + +- Queries or downstream code that relied on `approx_percentile_cont` / `approx_percentile_cont_with_weight` / + `approx_median` returning an integer type when given an integer column. + +**Migration guide:** + +If downstream code checks or relies on the return type being an integer, add an explicit +`CAST` back to the desired integer type, or update the type assertion: + +```sql +-- Before (returned Int64): +SELECT approx_percentile_cont(quantity, 0.5) FROM orders; + +-- After (returns Float64); cast if an integer result is required: +SELECT CAST(approx_percentile_cont(quantity, 0.5) AS BIGINT) FROM orders; +``` + +[#21074]: https://github.com/apache/datafusion/pull/21074 diff --git a/docs/source/user-guide/configs.md b/docs/source/user-guide/configs.md index 49c9eea29ef73..4fbb403cadd47 100644 --- a/docs/source/user-guide/configs.md +++ b/docs/source/user-guide/configs.md @@ -85,13 +85,15 @@ The following configuration settings are available: | datafusion.execution.parquet.skip_metadata | true | (reading) If true, the parquet reader skip the optional embedded metadata that may be in the file Schema. This setting can help avoid schema conflicts when querying multiple parquet files with schemas containing compatible types but different metadata | | datafusion.execution.parquet.metadata_size_hint | 524288 | (reading) If specified, the parquet reader will try and fetch the last `size_hint` bytes of the parquet file optimistically. If not specified, two reads are required: One read to fetch the 8-byte parquet footer and another to fetch the metadata length encoded in the footer Default setting to 512 KiB, which should be sufficient for most parquet files, it can reduce one I/O operation per parquet file. If the metadata is larger than the hint, two reads will still be performed. | | datafusion.execution.parquet.pushdown_filters | false | (reading) If true, filter expressions are be applied during the parquet decoding operation to reduce the number of rows decoded. This optimization is sometimes called "late materialization". | -| datafusion.execution.parquet.reorder_filters | false | (reading) If true, filter expressions evaluated during the parquet decoding operation will be reordered heuristically to minimize the cost of evaluation. If false, the filters are applied in the same order as written in the query | | datafusion.execution.parquet.force_filter_selections | false | (reading) Force the use of RowSelections for filter results, when pushdown_filters is enabled. If false, the reader will automatically choose between a RowSelection and a Bitmap based on the number and pattern of selected rows. | | datafusion.execution.parquet.schema_force_view_types | true | (reading) If true, parquet reader will read columns of `Utf8/Utf8Large` with `Utf8View`, and `Binary/BinaryLarge` with `BinaryView`. | | datafusion.execution.parquet.binary_as_string | false | (reading) If true, parquet reader will read columns of `Binary/LargeBinary` with `Utf8`, and `BinaryView` with `Utf8View`. Parquet files generated by some legacy writers do not correctly set the UTF8 flag for strings, causing string columns to be loaded as BLOB instead. | | datafusion.execution.parquet.coerce_int96 | NULL | (reading) If true, parquet reader will read columns of physical type int96 as originating from a different resolution than nanosecond. This is useful for reading data from systems like Spark which stores microsecond resolution timestamps in an int96 allowing it to write values with a larger date range than 64-bit timestamps with nanosecond resolution. | | datafusion.execution.parquet.bloom_filter_on_read | true | (reading) Use any available bloom filters when reading parquet files | | datafusion.execution.parquet.max_predicate_cache_size | NULL | (reading) The maximum predicate cache size, in bytes. When `pushdown_filters` is enabled, sets the maximum memory used to cache the results of predicate evaluation between filter evaluation and output generation. Decreasing this value will reduce memory usage, but may increase IO and CPU usage. None means use the default parquet reader setting. 0 means no caching. | +| datafusion.execution.parquet.filter_pushdown_min_bytes_per_sec | 104857600 | (reading) Minimum bytes/sec throughput for adaptive filter pushdown. Filters that achieve at least this throughput (bytes_saved / eval_time) are promoted to row filters. f64::INFINITY = no filters promoted (feature disabled). 0.0 = all filters pushed as row filters (no adaptive logic). Default: 104,857,600 bytes/sec (100 MiB/sec), empirically chosen based on TPC-H, TPC-DS, and ClickBench benchmarks on an m4 MacBook Pro. The optimal value for this setting likely depends on the relative cost of CPU vs. IO in your environment, and to some extent the shape of your query. **Interaction with `pushdown_filters`:** This option only takes effect when `pushdown_filters = true`. When pushdown is disabled, all filters run post-scan and this threshold is ignored. | +| datafusion.execution.parquet.filter_collecting_byte_ratio_threshold | 0.2 | (reading) Byte-ratio threshold for applying filters one at a time (iterative pruning; aka row-level) vs. all at once (post-scan). The ratio is computed as: (extra filter bytes not in projection) / (projected bytes). Filters whose extra columns consume a smaller fraction than this threshold are placed as row filters. Ratio of filter column bytes to projection bytes that controls initial filter placement. Computed as `filter_compressed_bytes / projection_compressed_bytes`. Filters below this ratio start as row-level filters (enabling late materialization); those above start as post-scan filters. Default: 0.20 — filters whose columns are less than 20% of the projection bytes start at row-level. **Interaction with `pushdown_filters`:** Only takes effect when `pushdown_filters = true`. | +| datafusion.execution.parquet.filter_confidence_z | 2 | (reading) Z-score for confidence intervals on filter effectiveness. Controls how much statistical evidence is required before promoting or demoting a filter. Lower values = faster decisions with less confidence. Higher values = more conservative, requiring more data. Default: 2.0 (~95% confidence). **Interaction with `pushdown_filters`:** Only takes effect when `pushdown_filters = true`. | | datafusion.execution.parquet.data_pagesize_limit | 1048576 | (writing) Sets best effort maximum size of data page in bytes | | datafusion.execution.parquet.write_batch_size | 1024 | (writing) Sets write_batch_size in rows | | datafusion.execution.parquet.writer_version | 1.0 | (writing) Sets parquet writer version valid values are "1.0" and "2.0" | diff --git a/docs/source/user-guide/sql/format_options.md b/docs/source/user-guide/sql/format_options.md index 338508031413c..3f4b8bc3f184b 100644 --- a/docs/source/user-guide/sql/format_options.md +++ b/docs/source/user-guide/sql/format_options.md @@ -147,7 +147,6 @@ The following options are available when reading or writing Parquet files. If an | SKIP_METADATA | No | If true, skips optional embedded metadata in the file schema. | `'skip_metadata'` | true | | METADATA_SIZE_HINT | No | Sets the size hint (in bytes) for fetching Parquet file metadata. | `'metadata_size_hint'` | None | | PUSHDOWN_FILTERS | No | If true, enables filter pushdown during Parquet decoding. | `'pushdown_filters'` | false | -| REORDER_FILTERS | No | If true, enables heuristic reordering of filters during Parquet decoding. | `'reorder_filters'` | false | | SCHEMA_FORCE_VIEW_TYPES | No | If true, reads Utf8/Binary columns as view types. | `'schema_force_view_types'` | true | | BINARY_AS_STRING | No | If true, reads Binary columns as strings. | `'binary_as_string'` | false | | DATA_PAGESIZE_LIMIT | No | Sets best effort maximum size of data page in bytes. | `'data_pagesize_limit'` | 1048576 | diff --git a/docs/source/user-guide/sql/select.md b/docs/source/user-guide/sql/select.md index 7ab6ed1aea05d..3564884b041ad 100644 --- a/docs/source/user-guide/sql/select.md +++ b/docs/source/user-guide/sql/select.md @@ -86,7 +86,7 @@ SELECT a FROM table WHERE a > 10 ## JOIN clause -DataFusion supports `INNER JOIN`, `LEFT OUTER JOIN`, `RIGHT OUTER JOIN`, `FULL OUTER JOIN`, `NATURAL JOIN`, `CROSS JOIN`, `LEFT SEMI JOIN`, `RIGHT SEMI JOIN`, `LEFT ANTI JOIN`, `RIGHT ANTI JOIN`, and `LATERAL JOIN`. +DataFusion supports `INNER JOIN`, `LEFT OUTER JOIN`, `RIGHT OUTER JOIN`, `FULL OUTER JOIN`, `NATURAL JOIN`, `CROSS JOIN`, `LEFT SEMI JOIN`, `RIGHT SEMI JOIN`, `LEFT ANTI JOIN`, `RIGHT ANTI JOIN`, `LATERAL JOIN`, and `LEFT JOIN LATERAL`. The following examples are based on this table: @@ -318,11 +318,53 @@ ORDER BY dept; +------+-------+-----+ ``` +#### LEFT JOIN LATERAL + +`LEFT JOIN LATERAL` preserves all rows from the left table. When the lateral +subquery produces no matching rows, the right-side columns are filled with +NULLs. + +```sql +SELECT d.name AS dept, e.name AS emp +FROM departments d +LEFT JOIN LATERAL ( + SELECT employees.name FROM employees WHERE employees.dept_id = d.id +) AS e ON true +ORDER BY dept, emp; ++-------+-------+ +| dept | emp | ++-------+-------+ +| Eng | Carol | +| HR | Alice | +| HR | Bob | +| Sales | NULL | ++-------+-------+ +``` + +The `ON` clause can also filter results. Rows that do not satisfy the `ON` +condition are preserved with NULLs, just like a regular `LEFT JOIN`: + +```sql +SELECT d.name AS dept, sub.cnt +FROM departments d +LEFT JOIN LATERAL ( + SELECT count(*) AS cnt + FROM employees WHERE employees.dept_id = d.id +) AS sub ON sub.cnt > 0 +ORDER BY dept; ++-------+------+ +| dept | cnt | ++-------+------+ +| Eng | 1 | +| HR | 2 | +| Sales | NULL | ++-------+------+ +``` + #### Limitations The following patterns are not yet supported: -- `LEFT JOIN LATERAL` (lateral join with outer join semantics). - Outer references in the `SELECT` list of the lateral subquery (e.g., `LATERAL (SELECT outer.col + 1)`). - `HAVING` in lateral subqueries.